From a8bf82c052ed290ca9866f2ad666b4e42bce20ce Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 7 Feb 2023 16:41:08 -0800 Subject: [PATCH 01/15] array columns! changes: * add support for storing nested arrays of string, long, and double values as specialized nested columns instead of breaking them into separate element columns * nested column typic mimic behavior means that columns ingested with only root arrays of primitive values will be ARRAY typed columns * neat test stuff --- .../data/input/impl/DelimitedInputFormat.java | 13 + .../data/input/impl/JsonInputFormat.java | 8 + .../data/input/impl/MapInputRowParser.java | 23 +- .../org/apache/druid/math/expr/ExprEval.java | 2 +- .../input/impl/InputRowParserSerdeTest.java | 1 - .../data/input/impl/JsonLineReaderTest.java | 6 +- .../org/apache/druid/math/expr/EvalTest.java | 2 +- .../apache/druid/math/expr/ExprEvalTest.java | 2 +- .../expression/NestedDataExpressions.java | 16 +- ...alueStringGroupByVectorColumnSelector.java | 3 +- .../druid/query/metadata/SegmentAnalyzer.java | 5 + .../segment/NestedDataColumnIndexer.java | 128 ++-- .../druid/segment/NestedDataColumnMerger.java | 318 ++++++-- .../druid/segment/UnnestStorageAdapter.java | 2 +- .../druid/segment/data/FrontCodedIndexed.java | 7 + .../data/FrontCodedIntArrayIndexed.java | 524 +++++++++++++ .../data/FrontCodedIntArrayIndexedWriter.java | 340 +++++++++ .../incremental/IncrementalIndexSchema.java | 5 + .../ArrayOfLiteralsFieldColumnWriter.java | 80 ++ .../CompressedNestedDataComplexColumn.java | 368 ++++++++- .../nested/GlobalDictionaryIdLookup.java | 25 + .../GlobalDictionarySortedCollector.java | 32 +- .../nested/GlobalDimensionDictionary.java | 143 +++- .../nested/LocalDimensionDictionary.java | 4 +- .../nested/NestedDataColumnSerializer.java | 116 ++- .../nested/NestedDataColumnSupplier.java | 60 +- .../segment/nested/NestedDataColumnV3.java | 1 + .../segment/nested/NestedDataColumnV4.java | 1 + .../segment/nested/NestedDataColumnV5.java | 88 +++ ...edFieldLiteralDictionaryEncodedColumn.java | 93 ++- .../segment/nested/NestedLiteralTypeInfo.java | 83 ++ .../nested/StructuredDataProcessor.java | 107 ++- .../VariantLiteralFieldColumnWriter.java | 29 + .../virtual/NestedFieldVirtualColumn.java | 9 +- .../druid/data/input/ResourceInputSource.java | 110 +++ .../druid/query/NestedDataTestUtils.java | 619 +++++++-------- .../aggregation/AggregationTestHelper.java | 46 -- .../expression/NestedDataExpressionsTest.java | 2 +- .../groupby/NestedDataGroupByQueryTest.java | 64 +- .../query/scan/NestedDataScanQueryTest.java | 131 +++- .../query/topn/NestedDataTopNQueryTest.java | 32 +- .../apache/druid/segment/IndexBuilder.java | 200 ++++- .../segment/NestedDataColumnIndexerTest.java | 29 +- .../segment/data/FrontCodedIndexedTest.java | 26 + .../data/FrontCodedIntArrayIndexedTest.java | 455 +++++++++++ .../nested/NestedDataColumnSupplierTest.java | 365 ++++++++- ...NestedFieldLiteralColumnSelectorsTest.java | 28 +- .../nested/NestedLiteralTypeInfoTest.java | 174 +++++ .../resources/nested-array-test-data.json | 14 + .../resources/nested-numeric-test-data.json | 10 + .../resources/nested-simple-test-data.json | 8 + .../resources/nested-simple-test-data.tsv | 8 + .../resources/nested-types-test-data.json | 8 + .../numeric-nested-test-data-parser.json | 20 - .../resources/numeric-nested-test-data.json | 10 - .../simple-nested-test-data-aggs.json | 6 - .../simple-nested-test-data-parser.json | 33 - .../simple-nested-test-data-tsv-parser.json | 41 - ...simple-nested-test-data-tsv-transform.json | 24 - .../resources/simple-nested-test-data.json | 8 - .../resources/simple-nested-test-data.tsv | 8 - .../resources/types-test-data-parser.json | 16 - .../src/test/resources/types-test-data.json | 8 - .../sql/calcite/expression/Expressions.java | 6 +- .../ArrayContainsOperatorConversion.java | 2 +- .../NestedDataOperatorConversions.java | 159 ++++ .../calcite/planner/DruidOperatorTable.java | 3 + .../sql/calcite/BaseCalciteQueryTest.java | 17 + .../calcite/CalciteNestedDataQueryTest.java | 710 +++++++++++++++++- .../sql/calcite/util/TestDataBuilder.java | 66 +- ... => wikiticker-2015-09-12-sampled.json.gz} | Bin 71 files changed, 5185 insertions(+), 925 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java create mode 100644 processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java create mode 100644 processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java create mode 100644 processing/src/test/java/org/apache/druid/data/input/ResourceInputSource.java create mode 100644 processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/nested/NestedLiteralTypeInfoTest.java create mode 100644 processing/src/test/resources/nested-array-test-data.json create mode 100644 processing/src/test/resources/nested-numeric-test-data.json create mode 100644 processing/src/test/resources/nested-simple-test-data.json create mode 100644 processing/src/test/resources/nested-simple-test-data.tsv create mode 100644 processing/src/test/resources/nested-types-test-data.json delete mode 100644 processing/src/test/resources/numeric-nested-test-data-parser.json delete mode 100644 processing/src/test/resources/numeric-nested-test-data.json delete mode 100644 processing/src/test/resources/simple-nested-test-data-aggs.json delete mode 100644 processing/src/test/resources/simple-nested-test-data-parser.json delete mode 100644 processing/src/test/resources/simple-nested-test-data-tsv-parser.json delete mode 100644 processing/src/test/resources/simple-nested-test-data-tsv-transform.json delete mode 100644 processing/src/test/resources/simple-nested-test-data.json delete mode 100644 processing/src/test/resources/simple-nested-test-data.tsv delete mode 100644 processing/src/test/resources/types-test-data-parser.json delete mode 100644 processing/src/test/resources/types-test-data.json rename sql/src/test/resources/calcite/tests/{wikiticker-2015-09-12-sampled.jgz => wikiticker-2015-09-12-sampled.json.gz} (100%) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java index 599bd2e634c0..e69d5c307a7d 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java @@ -31,6 +31,7 @@ import javax.annotation.Nullable; import java.io.File; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -41,6 +42,18 @@ public class DelimitedInputFormat extends FlatTextInputFormat { public static final String TYPE_KEY = "tsv"; + + public static DelimitedInputFormat ofColumns(String... columns) + { + return new DelimitedInputFormat( + Arrays.asList(columns), + null, + null, + false, + false, + 0 + ); + } private static final String DEFAULT_DELIMITER = "\t"; @JsonCreator diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java index 821c5bf4355e..e886aeb2bfbb 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java @@ -41,6 +41,14 @@ public class JsonInputFormat extends NestedInputFormat { public static final String TYPE_KEY = "json"; + public static final JsonInputFormat DEFAULT = new JsonInputFormat( + JSONPathSpec.DEFAULT, + null, + null, + null, + null + ); + private final Map featureSpec; private final ObjectMapper objectMapper; private final boolean keepNullColumns; diff --git a/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java b/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java index 58930804d096..f028131f8720 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Sets; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.MapBasedInputRow; @@ -77,22 +76,36 @@ public static InputRow parse(InputRowSchema inputRowSchema, Map * 3) If isIncludeAllDimensions is not set and {@link DimensionsSpec#getDimensionNames()} is empty, * the dimensions in the given map is returned. * - * In any case, the returned list does not include any dimensions in {@link DimensionsSpec#getDimensionExclusions()}. + * In any case, the returned list does not include any dimensions in {@link DimensionsSpec#getDimensionExclusions()} + * or {@link TimestampSpec#getTimestampColumn()}. */ private static List findDimensions( + TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec, Map rawInputRow ) { if (dimensionsSpec.isIncludeAllDimensions()) { LinkedHashSet dimensions = new LinkedHashSet<>(dimensionsSpec.getDimensionNames()); - dimensions.addAll(Sets.difference(rawInputRow.keySet(), dimensionsSpec.getDimensionExclusions())); + for (String field : rawInputRow.keySet()) { + if (timestampSpec.getTimestampColumn().equals(field) || dimensionsSpec.getDimensionExclusions().contains(field)) { + continue; + } + dimensions.add(field); + } return new ArrayList<>(dimensions); } else { if (!dimensionsSpec.getDimensionNames().isEmpty()) { return dimensionsSpec.getDimensionNames(); } else { - return new ArrayList<>(Sets.difference(rawInputRow.keySet(), dimensionsSpec.getDimensionExclusions())); + List dimensions = new ArrayList<>(); + for (String field : rawInputRow.keySet()) { + if (timestampSpec.getTimestampColumn().equals(field) || dimensionsSpec.getDimensionExclusions().contains(field)) { + continue; + } + dimensions.add(field); + } + return dimensions; } } } @@ -104,7 +117,7 @@ static InputRow parse( Map theMap ) throws ParseException { - final List dimensionsToUse = findDimensions(dimensionsSpec, theMap); + final List dimensionsToUse = findDimensions(timestampSpec, dimensionsSpec, theMap); final DateTime timestamp; try { 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 dbfc11d3c4ee..daf9ce9749c7 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 @@ -163,7 +163,7 @@ public static NonnullPair coerceListToArray(@Nullable Object[] array = new Object[val.size()]; int i = 0; for (Object o : val) { - array[i++] = o == null ? null : ExprEval.ofType(ExpressionType.LONG, o).value(); + array[i++] = ExprEval.ofType(ExpressionType.LONG, o).value(); } return new NonnullPair<>(ExpressionType.LONG_ARRAY, array); } else if (coercedType == Float.class || coercedType == Double.class) { diff --git a/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java b/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java index 7e1b487cae2c..34a3722fc207 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java @@ -233,7 +233,6 @@ public void testFlattenParse() throws Exception "hey0barx", "metA", "missing", - "timestamp", "foo.bar1", "blah", "newmet", diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java index 5b0a3a391ab3..dea2efb0194c 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java @@ -188,7 +188,7 @@ public void testParseRowKeepNullColumns() throws IOException int numActualIterations = 0; while (iterator.hasNext()) { final InputRow row = iterator.next(); - Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "timestamp", "bar", "foo"), row.getDimensions()); + Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "bar", "foo"), row.getDimensions()); Assert.assertTrue(row.getDimension("bar").isEmpty()); Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); Assert.assertTrue(row.getDimension("path_omg").isEmpty()); @@ -234,7 +234,7 @@ public void testKeepNullColumnsWithNoNullValues() throws IOException int numActualIterations = 0; while (iterator.hasNext()) { final InputRow row = iterator.next(); - Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "timestamp", "bar", "foo"), row.getDimensions()); + Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "bar", "foo"), row.getDimensions()); Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("bar"))); Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); Assert.assertEquals("a", Iterables.getOnlyElement(row.getDimension("path_omg"))); @@ -280,7 +280,7 @@ public void testFalseKeepNullColumns() throws IOException int numActualIterations = 0; while (iterator.hasNext()) { final InputRow row = iterator.next(); - Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "timestamp", "foo"), row.getDimensions()); + Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "foo"), row.getDimensions()); Assert.assertTrue(row.getDimension("bar").isEmpty()); Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); Assert.assertEquals("a", Iterables.getOnlyElement(row.getDimension("path_omg"))); 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 c3322ba19f7a..4cd3d14e8974 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 @@ -1081,7 +1081,7 @@ public void testBestEffortOf() eval = ExprEval.bestEffortOf(new Object[] {1L, 2L, null, 3L}); Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type()); - Assert.assertArrayEquals(new Object[] {1L, 2L, null, 3L}, (Object[]) eval.value()); + Assert.assertArrayEquals(new Object[] {1L, 2L, NullHandling.defaultLongValue(), 3L}, (Object[]) eval.value()); eval = ExprEval.bestEffortOf(ImmutableList.of(1L, 2L, 3L)); Assert.assertEquals(ExpressionType.LONG_ARRAY, 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 15dc241a025e..8da1ca730b0f 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 @@ -279,7 +279,7 @@ public void test_coerceListToArray() coerced = ExprEval.coerceListToArray(withNumberNulls, false); Assert.assertEquals(ExpressionType.LONG_ARRAY, coerced.lhs); - Assert.assertArrayEquals(new Object[]{1L, null, 3L}, coerced.rhs); + Assert.assertArrayEquals(new Object[]{1L, NullHandling.defaultLongValue(), 3L}, coerced.rhs); List withStringMix = ImmutableList.of(1L, "b", 3L); coerced = ExprEval.coerceListToArray(withStringMix, false); diff --git a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java index 833f46428b7d..57bcf85b7c39 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java +++ b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java @@ -480,11 +480,25 @@ public Expr apply(List args) final StructuredDataProcessor processor = new StructuredDataProcessor() { @Override - public StructuredDataProcessor.ProcessedLiteral processLiteralField(ArrayList fieldPath, Object fieldValue) + public StructuredDataProcessor.ProcessedLiteral processLiteralField(ArrayList fieldPath, @Nullable Object fieldValue) { // do nothing, we only want the list of fields returned by this processor return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; } + + @Nullable + @Override + public ProcessedLiteral processArrayOfLiteralsField( + ArrayList fieldPath, + @Nullable Object maybeArrayOfLiterals + ) + { + ExprEval eval = ExprEval.bestEffortOf(maybeArrayOfLiterals); + if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) { + return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; + } + return null; + } }; final class JsonPathsExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DictionaryBuildingSingleValueStringGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DictionaryBuildingSingleValueStringGroupByVectorColumnSelector.java index 937ddc339c9d..83f49e1c834e 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DictionaryBuildingSingleValueStringGroupByVectorColumnSelector.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DictionaryBuildingSingleValueStringGroupByVectorColumnSelector.java @@ -25,6 +25,7 @@ import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.DictionaryBuilding; import org.apache.druid.query.groupby.epinephelinae.collection.MemoryPointer; +import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.vector.VectorObjectSelector; import java.util.ArrayList; @@ -71,7 +72,7 @@ public int writeKeys( int stateFootprintIncrease = 0; for (int i = startRow, j = keyOffset; i < endRow; i++, j += keySize) { - final String value = (String) vector[i]; + final String value = DimensionHandlerUtils.convertObjectToString(vector[i]); final int dictId = reverseDictionary.getInt(value); if (dictId < 0) { final int nextId = dictionary.size(); 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 859396170b19..6834d039d03a 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 @@ -134,6 +134,11 @@ public Map analyze(Segment segment) analysis = analyzeStringColumn(capabilities, storageAdapter, columnName); } break; + case ARRAY: + // todo (clint): this is wack, but works for now because arrays are always nested complex columns... + final ColumnHolder arrayHolder = index != null ? index.getColumnHolder(columnName) : null; + analysis = analyzeComplexColumn(capabilities, numRows, arrayHolder); + break; case COMPLEX: final ColumnHolder columnHolder = index != null ? index.getColumnHolder(columnName) : null; analysis = analyzeComplexColumn(capabilities, numRows, columnHolder); diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java index df64897eb7a6..0e40982ebdb9 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java @@ -19,8 +19,10 @@ package org.apache.druid.segment; +import com.google.common.base.Preconditions; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.MutableBitmap; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.dimension.DimensionSpec; @@ -42,6 +44,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.Map; import java.util.Objects; import java.util.SortedMap; @@ -59,16 +62,43 @@ public class NestedDataColumnIndexer implements DimensionIndexer processLiteralField(ArrayList fieldPath, Object fieldValue) + public ProcessedLiteral processLiteralField(ArrayList fieldPath, @Nullable Object fieldValue) { - final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath); - LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName); - if (fieldIndexer == null) { - estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName); - fieldIndexer = new LiteralFieldIndexer(globalDictionary); - fieldIndexers.put(fieldName, fieldIndexer); + // null value is always added to the global dictionary as id 0, so we can ignore them here + if (fieldValue != null) { + // why not + final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath); + ExprEval eval = ExprEval.bestEffortOf(fieldValue); + LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName); + if (fieldIndexer == null) { + estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName); + fieldIndexer = new LiteralFieldIndexer(globalDictionary); + fieldIndexers.put(fieldName, fieldIndexer); + } + return fieldIndexer.processValue(eval); } - return fieldIndexer.processValue(fieldValue); + return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; + } + + @Nullable + @Override + public ProcessedLiteral processArrayOfLiteralsField( + ArrayList fieldPath, + Object maybeArrayOfLiterals + ) + { + final ExprEval maybeLiteralArray = ExprEval.bestEffortOf(maybeArrayOfLiterals); + if (maybeLiteralArray.type().isArray() && maybeLiteralArray.type().getElementType().isPrimitive()) { + final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath); + LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName); + if (fieldIndexer == null) { + estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName); + fieldIndexer = new LiteralFieldIndexer(globalDictionary); + fieldIndexers.put(fieldName, fieldIndexer); + } + return fieldIndexer.processValue(maybeLiteralArray); + } + return null; } }; @@ -145,6 +175,10 @@ public DimensionSelector makeDimensionSelector( final int dimIndex = desc.getIndex(); final ColumnValueSelector rootLiteralSelector = getRootLiteralValueSelector(currEntry, dimIndex); if (rootLiteralSelector != null) { + final LiteralFieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); + if (root.getTypes().getSingleType().isArray()) { + throw new UnsupportedOperationException("Not supported"); + } return new BaseSingleValueDimensionSelector() { @Nullable @@ -411,7 +445,7 @@ public Object getObject() if (0 <= dimIndex && dimIndex < dims.length) { final StructuredData data = (StructuredData) dims[dimIndex]; if (data != null) { - return data.getValue(); + return ExprEval.bestEffortOf(data.getValue()).value(); } } @@ -437,41 +471,51 @@ static class LiteralFieldIndexer this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet(); } - private StructuredDataProcessor.ProcessedLiteral processValue(@Nullable Object value) + private StructuredDataProcessor.ProcessedLiteral processValue(ExprEval eval) { - // null value is always added to the global dictionary as id 0, so we can ignore them here - if (value != null) { - // why not - ExprEval eval = ExprEval.bestEffortOf(value); - final ColumnType columnType = ExpressionType.toColumnType(eval.type()); - - switch (columnType.getType()) { - case LONG: - globalDimensionDictionary.addLongValue(eval.asLong()); - typeSet.add(ColumnType.LONG); - return new StructuredDataProcessor.ProcessedLiteral<>( - eval.asLong(), - StructuredDataProcessor.getLongObjectEstimateSize() - ); - case DOUBLE: - globalDimensionDictionary.addDoubleValue(eval.asDouble()); - typeSet.add(ColumnType.DOUBLE); - return new StructuredDataProcessor.ProcessedLiteral<>( - eval.asDouble(), - StructuredDataProcessor.getDoubleObjectEstimateSize() - ); - case STRING: - default: - final String asString = eval.asString(); - globalDimensionDictionary.addStringValue(asString); - typeSet.add(ColumnType.STRING); - return new StructuredDataProcessor.ProcessedLiteral<>( - eval.asString(), - StructuredDataProcessor.estimateStringSize(asString) - ); - } + final ColumnType columnType = ExpressionType.toColumnType(eval.type()); + int sizeEstimate; + switch (columnType.getType()) { + case LONG: + typeSet.add(ColumnType.LONG); + sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong()); + return new StructuredDataProcessor.ProcessedLiteral<>(eval.asLong(), sizeEstimate); + case DOUBLE: + typeSet.add(ColumnType.DOUBLE); + sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble()); + return new StructuredDataProcessor.ProcessedLiteral<>(eval.asDouble(), sizeEstimate); + case ARRAY: + // skip empty arrays for now, they will always be called 'string' arrays, which isn't very helpful here since + // it will pollute the type set + Preconditions.checkNotNull(columnType.getElementType(), "Array element type must not be null"); + switch (columnType.getElementType().getType()) { + case LONG: + typeSet.add(ColumnType.LONG_ARRAY); + final Object[] longArray = eval.asArray(); + sizeEstimate = globalDimensionDictionary.addLongArray(longArray); + return new StructuredDataProcessor.ProcessedLiteral<>(longArray, sizeEstimate); + case DOUBLE: + typeSet.add(ColumnType.DOUBLE_ARRAY); + final Object[] doubleArray = eval.asArray(); + sizeEstimate = globalDimensionDictionary.addDoubleArray(doubleArray); + return new StructuredDataProcessor.ProcessedLiteral<>(doubleArray, sizeEstimate); + case STRING: + final Object[] stringArray = eval.asArray(); + if (!Arrays.stream(stringArray).allMatch(Objects::isNull)) { + typeSet.add(ColumnType.STRING_ARRAY); + } + sizeEstimate = globalDimensionDictionary.addStringArray(stringArray); + return new StructuredDataProcessor.ProcessedLiteral<>(stringArray, sizeEstimate); + default: + throw new IAE("Unhandled type: %s", columnType); + } + case STRING: + default: + typeSet.add(ColumnType.STRING); + final String asString = eval.asString(); + sizeEstimate = globalDimensionDictionary.addStringValue(asString); + return new StructuredDataProcessor.ProcessedLiteral<>(eval.asString(), sizeEstimate); } - return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; } public NestedLiteralTypeInfo.MutableTypeSet getTypes() diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java index bf672c35221a..7199d538d660 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java @@ -19,6 +19,8 @@ package org.apache.druid.segment; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterators; import com.google.common.collect.PeekingIterator; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.io.Closer; @@ -28,10 +30,12 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.StringEncodingStrategies; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAdapter; import org.apache.druid.segment.nested.CompressedNestedDataComplexColumn; +import org.apache.druid.segment.nested.GlobalDictionaryIdLookup; import org.apache.druid.segment.nested.GlobalDictionarySortedCollector; import org.apache.druid.segment.nested.NestedDataColumnSerializer; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; @@ -42,8 +46,12 @@ import javax.annotation.Nullable; import java.io.IOException; import java.nio.IntBuffer; +import java.util.Arrays; import java.util.Comparator; +import java.util.Iterator; import java.util.List; +import java.util.NoSuchElementException; +import java.util.PriorityQueue; import java.util.SortedMap; import java.util.TreeMap; @@ -86,93 +94,123 @@ public NestedDataColumnMerger( @Override public void writeMergedValueDictionary(List adapters) throws IOException { + try { + long dimStartTime = System.currentTimeMillis(); + + int numMergeIndex = 0; + GlobalDictionarySortedCollector sortedLookup = null; + final Indexed[] sortedLookups = new Indexed[adapters.size()]; + final Indexed[] sortedLongLookups = new Indexed[adapters.size()]; + final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()]; + final Iterable[] sortedArrayLookups = new Iterable[adapters.size()]; + + final SortedMap mergedFields = new TreeMap<>(); + + for (int i = 0; i < adapters.size(); i++) { + final IndexableAdapter adapter = adapters.get(i); + final GlobalDictionarySortedCollector dimValues; + if (adapter instanceof IncrementalIndexAdapter) { + dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields); + } else if (adapter instanceof QueryableIndexIndexableAdapter) { + dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields); + } else { + throw new ISE("Unable to merge columns of unsupported adapter %s", adapter.getClass()); + } - long dimStartTime = System.currentTimeMillis(); - - int numMergeIndex = 0; - GlobalDictionarySortedCollector sortedLookup = null; - final Indexed[] sortedLookups = new Indexed[adapters.size()]; - final Indexed[] sortedLongLookups = new Indexed[adapters.size()]; - final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()]; + boolean allNulls = dimValues == null || allNull(dimValues.getSortedStrings()) && + allNull(dimValues.getSortedLongs()) && + allNull(dimValues.getSortedDoubles()) && + dimValues.getArrayCardinality() == 0; + sortedLookup = dimValues; + if (!allNulls) { + sortedLookups[i] = dimValues.getSortedStrings(); + sortedLongLookups[i] = dimValues.getSortedLongs(); + sortedDoubleLookups[i] = dimValues.getSortedDoubles(); + sortedArrayLookups[i] = dimValues.getSortedArrays(); + numMergeIndex++; + } + } - final SortedMap mergedFields = new TreeMap<>(); + descriptorBuilder = new ColumnDescriptor.Builder(); - for (int i = 0; i < adapters.size(); i++) { - final IndexableAdapter adapter = adapters.get(i); - final GlobalDictionarySortedCollector dimValues; - if (adapter instanceof IncrementalIndexAdapter) { - dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields); - } else if (adapter instanceof QueryableIndexIndexableAdapter) { - dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields); + final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer( + name, + indexSpec, + segmentWriteOutMedium, + progressIndicator, + closer + ); + serializer = defaultSerializer; + + final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder() + .withTypeName(NestedDataComplexTypeSerde.TYPE_NAME) + .withDelegate(serializer) + .build(); + descriptorBuilder.setValueType(ValueType.COMPLEX) + .setHasMultipleValues(false) + .addSerde(partSerde); + + defaultSerializer.open(); + defaultSerializer.serializeFields(mergedFields); + + int stringCardinality; + int longCardinality; + int doubleCardinality; + int arrayCardinality; + if (numMergeIndex == 1) { + defaultSerializer.serializeStringDictionary(sortedLookup.getSortedStrings()); + defaultSerializer.serializeLongDictionary(sortedLookup.getSortedLongs()); + defaultSerializer.serializeDoubleDictionary(sortedLookup.getSortedDoubles()); + defaultSerializer.serializeArrayDictionary(() -> new ArrayDictionaryMergingIterator( + sortedArrayLookups, + defaultSerializer.getGlobalLookup() + )); + stringCardinality = sortedLookup.getStringCardinality(); + longCardinality = sortedLookup.getLongCardinality(); + doubleCardinality = sortedLookup.getDoubleCardinality(); + arrayCardinality = sortedLookup.getArrayCardinality(); } else { - throw new ISE("Unable to merge columns of unsupported adapter %s", adapter.getClass()); + SimpleDictionaryMergingIterator dictionaryMergeIterator = new SimpleDictionaryMergingIterator<>( + sortedLookups, + STRING_MERGING_COMPARATOR + ); + SimpleDictionaryMergingIterator longDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>( + sortedLongLookups, + LONG_MERGING_COMPARATOR + ); + SimpleDictionaryMergingIterator doubleDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>( + sortedDoubleLookups, + DOUBLE_MERGING_COMPARATOR + ); + defaultSerializer.serializeStringDictionary(() -> dictionaryMergeIterator); + defaultSerializer.serializeLongDictionary(() -> longDictionaryMergeIterator); + defaultSerializer.serializeDoubleDictionary(() -> doubleDictionaryMergeIterator); + + final ArrayDictionaryMergingIterator arrayDictionaryMergingIterator = new ArrayDictionaryMergingIterator( + sortedArrayLookups, + defaultSerializer.getGlobalLookup() + ); + defaultSerializer.serializeArrayDictionary(() -> arrayDictionaryMergingIterator); + stringCardinality = dictionaryMergeIterator.getCardinality(); + longCardinality = longDictionaryMergeIterator.getCardinality(); + doubleCardinality = doubleDictionaryMergeIterator.getCardinality(); + arrayCardinality = arrayDictionaryMergingIterator.getCardinality(); } - boolean allNulls = allNull(dimValues.getSortedStrings()) && - allNull(dimValues.getSortedLongs()) && - allNull(dimValues.getSortedDoubles()); - sortedLookup = dimValues; - if (!allNulls) { - sortedLookups[i] = dimValues.getSortedStrings(); - sortedLongLookups[i] = dimValues.getSortedLongs(); - sortedDoubleLookups[i] = dimValues.getSortedDoubles(); - numMergeIndex++; - } - } - - int cardinality = 0; - descriptorBuilder = new ColumnDescriptor.Builder(); - - final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer( - name, - indexSpec, - segmentWriteOutMedium, - progressIndicator, - closer - ); - serializer = defaultSerializer; - - final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder() - .withTypeName(NestedDataComplexTypeSerde.TYPE_NAME) - .withDelegate(serializer) - .build(); - descriptorBuilder.setValueType(ValueType.COMPLEX) - .setHasMultipleValues(false) - .addSerde(partSerde); - - defaultSerializer.open(); - defaultSerializer.serializeFields(mergedFields); - - if (numMergeIndex > 1) { - SimpleDictionaryMergingIterator dictionaryMergeIterator = new SimpleDictionaryMergingIterator<>( - sortedLookups, - STRING_MERGING_COMPARATOR + log.debug( + "Completed dim[%s] conversions with string cardinality[%,d], long cardinality[%,d], double cardinality[%,d], array cardinality[%,d] in %,d millis.", + name, + stringCardinality, + longCardinality, + doubleCardinality, + arrayCardinality, + System.currentTimeMillis() - dimStartTime ); - SimpleDictionaryMergingIterator longDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>( - sortedLongLookups, - LONG_MERGING_COMPARATOR - ); - SimpleDictionaryMergingIterator doubleDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>( - sortedDoubleLookups, - DOUBLE_MERGING_COMPARATOR - ); - defaultSerializer.serializeStringDictionary(() -> dictionaryMergeIterator); - defaultSerializer.serializeLongDictionary(() -> longDictionaryMergeIterator); - defaultSerializer.serializeDoubleDictionary(() -> doubleDictionaryMergeIterator); - cardinality = dictionaryMergeIterator.getCardinality(); - } else if (numMergeIndex == 1) { - defaultSerializer.serializeStringDictionary(sortedLookup.getSortedStrings()); - defaultSerializer.serializeLongDictionary(sortedLookup.getSortedLongs()); - defaultSerializer.serializeDoubleDictionary(sortedLookup.getSortedDoubles()); - cardinality = sortedLookup.size(); } - - log.debug( - "Completed dim[%s] conversions with cardinality[%,d] in %,d millis.", - name, - cardinality, - System.currentTimeMillis() - dimStartTime - ); + catch (Throwable ioe) { + log.error(ioe, "Failed to merge dictionary for column [%s]", name); + throw ioe; + } } @Nullable @@ -234,7 +272,9 @@ private GlobalDictionarySortedCollector getSortedIndexFromV1QueryableAdapterNest return new GlobalDictionarySortedCollector( new StringEncodingStrategies.Utf8ToStringIndexed(column.getStringDictionary()), column.getLongDictionary(), - column.getDoubleDictionary() + column.getDoubleDictionary(), + column.getArraysIterable(), + column.getArrayDictionary().size() ); } @@ -280,4 +320,122 @@ private boolean allNull(Indexed dimValues) } return true; } + + public static class ArrayDictionaryMergingIterator implements Iterator + { + private static final Comparator> PEEKING_ITERATOR_COMPARATOR = + (lhs, rhs) -> FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR.compare(lhs.peek(), rhs.peek()); + + protected final PriorityQueue> pQueue; + protected int counter; + + public ArrayDictionaryMergingIterator(Iterable[] dimValueLookups, GlobalDictionaryIdLookup idLookup) + { + pQueue = new PriorityQueue<>(PEEKING_ITERATOR_COMPARATOR); + + for (Iterable dimValueLookup : dimValueLookups) { + if (dimValueLookup == null) { + continue; + } + final PeekingIterator iter = Iterators.peekingIterator( + new IdLookupArrayIterator(idLookup, dimValueLookup.iterator()) + ); + if (iter.hasNext()) { + pQueue.add(iter); + } + } + } + + @Override + public boolean hasNext() + { + return !pQueue.isEmpty(); + } + + @Override + public int[] next() + { + PeekingIterator smallest = pQueue.remove(); + if (smallest == null) { + throw new NoSuchElementException(); + } + final int[] value = smallest.next(); + if (smallest.hasNext()) { + pQueue.add(smallest); + } + + while (!pQueue.isEmpty() && Arrays.equals(value, pQueue.peek().peek())) { + PeekingIterator same = pQueue.remove(); + same.next(); + if (same.hasNext()) { + pQueue.add(same); + } + } + counter++; + + return value; + } + + public int getCardinality() + { + return counter; + } + + @Override + public void remove() + { + throw new UnsupportedOperationException("remove"); + } + } + + public static class IdLookupArrayIterator implements Iterator + { + private final GlobalDictionaryIdLookup idLookup; + private final Iterator delegate; + + public IdLookupArrayIterator( + GlobalDictionaryIdLookup idLookup, + Iterator delegate + ) + { + this.idLookup = idLookup; + this.delegate = delegate; + } + + @Override + public boolean hasNext() + { + return delegate.hasNext(); + } + + @Override + public int[] next() + { + final Object[] next = delegate.next(); + if (next == null) { + return null; + } + final int[] newIdsWhoDis = new int[next.length]; + for (int i = 0; i < next.length; i++) { + if (next[i] == null) { + newIdsWhoDis[i] = 0; + } else if (next[i] instanceof String) { + newIdsWhoDis[i] = idLookup.lookupString((String) next[i]); + } else if (next[i] instanceof Long) { + newIdsWhoDis[i] = idLookup.lookupLong((Long) next[i]); + } else if (next[i] instanceof Double) { + newIdsWhoDis[i] = idLookup.lookupDouble((Double) next[i]); + } else { + newIdsWhoDis[i] = -1; + } + Preconditions.checkArgument( + newIdsWhoDis[i] >= 0, + "unknown global id [%s] for value [%s]", + newIdsWhoDis[i], + next[i] + ); + } + return newIdsWhoDis; + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index f76ab89270af..1bec2f4ce691 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -101,7 +101,7 @@ public Sequence makeCursors( Cursor retVal = cursor; ColumnCapabilities capabilities = cursor.getColumnSelectorFactory().getColumnCapabilities(dimensionToUnnest); if (capabilities != null) { - if (capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) { + if (!capabilities.isArray() && capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) { retVal = new UnnestDimensionCursor( retVal, retVal.getColumnSelectorFactory(), diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java index 2596f7ec2bf9..06f086a89fc8 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java @@ -170,6 +170,10 @@ public int indexOf(@Nullable ByteBuffer value) return hasNull ? 0 : -1; } + if (numBuckets == 0) { + return hasNull ? -2 : -1; + } + int minBucketIndex = 0; int maxBucketIndex = numBuckets - 1; while (minBucketIndex < maxBucketIndex) { @@ -269,6 +273,9 @@ public boolean isSorted() @Override public Iterator iterator() { + if (adjustedNumValues == 0) { + return Collections.emptyIterator(); + } if (hasNull && adjustedNumValues == 1) { return Collections.singletonList(null).iterator(); } diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java new file mode 100644 index 000000000000..d950d7a14dcf --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java @@ -0,0 +1,524 @@ +/* + * 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.data; + +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Collections; +import java.util.Iterator; +import java.util.NoSuchElementException; + +/** + * {@link Indexed} specialized for storing int arrays, which must be sorted and unique, using 'front coding'. + * + * Front coding is a type of delta encoding, where sorted values are grouped into buckets. The first value of the bucket + * is written entirely, and remaining values are stored as a pair of an integer which indicates how much of the first + * int array of the bucket to use as a prefix, followed by the remaining ints after the prefix to complete the value. + * + * front coded indexed layout: + * | version | bucket size | has null? | number of values | size of "offsets" + "buckets" | "offsets" | "buckets" | + * | ------- | ----------- | --------- | ---------------- | ----------------------------- | --------- | --------- | + * | byte | byte | byte | vbyte int | vbyte int | int[] | bucket[] | + * + * "offsets" are the ending offsets of each bucket stored in order, stored as plain integers for easy random access. + * + * bucket layout: + * | first value | prefix length | fragment | ... | prefix length | fragment | + * | ----------- | ------------- | -------- | --- | ------------- | -------- | + * | int[] | vbyte int | int[] | ... | vbyte int | int[] | + * + * int array layout: + * | length | ints | + * | ----------- | ----- | + * | vbyte int | int[] | + * + * + * Getting a value first picks the appropriate bucket, finds its offset in the underlying buffer, then scans the bucket + * values to seek to the correct position of the value within the bucket in order to reconstruct it using the prefix + * length. + * + * Finding the index of a value involves binary searching the first values of each bucket to find the correct bucket, + * then a linear scan within the bucket to find the matching value (or negative insertion point -1 for values that + * are not present). + * + * The value iterator reads an entire bucket at a time, reconstructing the values into an array to iterate within the + * bucket before moving onto the next bucket as the iterator is consumed. + * + * This class is not thread-safe since during operation modifies positions of a shared buffer. + */ +public final class FrontCodedIntArrayIndexed implements Indexed +{ + public static Supplier read(ByteBuffer buffer, ByteOrder ordering) + { + final ByteBuffer orderedBuffer = buffer.asReadOnlyBuffer().order(ordering); + final byte version = orderedBuffer.get(); + Preconditions.checkArgument(version == 0, "only V0 exists, encountered " + version); + final int bucketSize = Byte.toUnsignedInt(orderedBuffer.get()); + final boolean hasNull = NullHandling.IS_NULL_BYTE == orderedBuffer.get(); + final int numValues = VByte.readInt(orderedBuffer); + // size of offsets + values + final int size = VByte.readInt(orderedBuffer); + final int offsetsPosition = orderedBuffer.position(); + // move position to end of buffer + buffer.position(offsetsPosition + size); + + return () -> new FrontCodedIntArrayIndexed( + buffer, + ordering, + bucketSize, + numValues, + hasNull, + offsetsPosition + ); + } + + private final ByteBuffer buffer; + private final int adjustedNumValues; + private final int adjustIndex; + private final int bucketSize; + private final int numBuckets; + private final int div; + private final int rem; + private final int offsetsPosition; + private final int bucketsPosition; + private final boolean hasNull; + private final int lastBucketNumValues; + + private FrontCodedIntArrayIndexed( + ByteBuffer buffer, + ByteOrder order, + int bucketSize, + int numValues, + boolean hasNull, + int offsetsPosition + ) + { + if (Integer.bitCount(bucketSize) != 1) { + throw new ISE("bucketSize must be a power of two but was[%,d]", bucketSize); + } + this.buffer = buffer.asReadOnlyBuffer().order(order); + this.bucketSize = bucketSize; + this.hasNull = hasNull; + + this.numBuckets = (int) Math.ceil((double) numValues / (double) bucketSize); + this.adjustIndex = hasNull ? 1 : 0; + this.adjustedNumValues = numValues + adjustIndex; + this.div = Integer.numberOfTrailingZeros(bucketSize); + this.rem = bucketSize - 1; + this.lastBucketNumValues = (numValues & rem) == 0 ? bucketSize : numValues & rem; + this.offsetsPosition = offsetsPosition; + this.bucketsPosition = offsetsPosition + ((numBuckets - 1) * Integer.BYTES); + } + + @Override + public int size() + { + return adjustedNumValues; + } + + @Nullable + @Override + public int[] get(int index) + { + if (hasNull && index == 0) { + return null; + } + Indexed.checkIndex(index, adjustedNumValues); + + // due to vbyte encoding, the null value is not actually stored in the bucket (no negative values), so we adjust + // the index + final int adjustedIndex = index - adjustIndex; + // find the bucket which contains the value with maths + final int bucket = adjustedIndex >> div; + final int bucketIndex = adjustedIndex & rem; + final int offset = getBucketOffset(bucket); + buffer.position(offset); + return getFromBucket(buffer, bucketIndex); + } + + @Override + public int indexOf(@Nullable int[] value) + { + // performs binary search using the first values of each bucket to locate the appropriate bucket, and then does + // a linear scan to find the value within the bucket + if (value == null) { + return hasNull ? 0 : -1; + } + + if (numBuckets == 0) { + return hasNull ? -2 : -1; + } + + int minBucketIndex = 0; + int maxBucketIndex = numBuckets - 1; + while (minBucketIndex < maxBucketIndex) { + int currentBucket = (minBucketIndex + maxBucketIndex) >>> 1; + int currBucketFirstValueIndex = currentBucket * bucketSize; + + // compare against first value in "current" bucket + final int offset = getBucketOffset(currentBucket); + buffer.position(offset); + final int firstLength = VByte.readInt(buffer); + final int firstOffset = buffer.position(); + int comparison = compareBucketFirstValue(buffer, firstLength, value); + // save the length of the shared prefix with the first value of the bucket and the value to match so we + // can use it later to skip over all values in the bucket that share a longer prefix with the first value + // (the bucket is sorted, so the prefix length gets smaller as values increase) + final int sharedPrefix = (buffer.position() - firstOffset) / Integer.BYTES; + if (comparison == 0) { + if (firstLength == value.length) { + // it turns out that the first value in current bucket is what we are looking for, short circuit + return currBucketFirstValueIndex + adjustIndex; + } else { + comparison = Integer.compare(firstLength, value.length); + } + } + + // we also compare against the adjacent bucket to determine if the value is actually in this bucket or + // if we need to keep searching buckets + final int nextOffset = getBucketOffset(currentBucket + 1); + buffer.position(nextOffset); + final int nextLength = VByte.readInt(buffer); + int comparisonNext = compareBucketFirstValue(buffer, nextLength, value); + if (comparisonNext == 0) { + if (nextLength == value.length) { + // it turns out that the first value in next bucket is what we are looking for, go ahead and short circuit + // for that as well, even though we weren't going to scan that bucket on this iteration... + return (currBucketFirstValueIndex + adjustIndex) + bucketSize; + } else { + comparisonNext = Integer.compare(nextLength, value.length); + } + } + + if (comparison < 0 && comparisonNext > 0) { + // this is exactly the right bucket + // find the value in the bucket (or where it would be if it were present) + buffer.position(firstOffset + (firstLength * Integer.BYTES)); + + return findValueInBucket(value, currBucketFirstValueIndex, bucketSize, sharedPrefix); + } else if (comparison < 0) { + minBucketIndex = currentBucket + 1; + } else { + maxBucketIndex = currentBucket - 1; + } + } + + // this is where we ended up, try to find the value in the bucket + final int bucketIndexBase = minBucketIndex * bucketSize; + final int numValuesInBucket; + if (minBucketIndex == numBuckets - 1) { + numValuesInBucket = lastBucketNumValues; + } else { + numValuesInBucket = bucketSize; + } + final int offset = getBucketOffset(minBucketIndex); + + // like we did in the loop, except comparison being smaller the first value here is a short circuit + buffer.position(offset); + final int firstLength = VByte.readInt(buffer); + final int firstOffset = buffer.position(); + int comparison = compareBucketFirstValue(buffer, firstLength, value); + final int sharedPrefix = (buffer.position() - firstOffset) / Integer.BYTES; + if (comparison == 0) { + if (firstLength == value.length) { + // it turns out that the first value in current bucket is what we are looking for, short circuit + return bucketIndexBase + adjustIndex; + } else { + comparison = Integer.compare(firstLength, value.length); + } + } + + if (comparison > 0) { + // value preceedes bucket, so bail out + return -(bucketIndexBase + adjustIndex) - 1; + } + + buffer.position(firstOffset + (firstLength * Integer.BYTES)); + + return findValueInBucket(value, bucketIndexBase, numValuesInBucket, sharedPrefix); + } + + @Override + public boolean isSorted() + { + // FrontCodedIndexed only supports sorted values + return true; + } + + @Override + public Iterator iterator() + { + if (adjustedNumValues == 0) { + return Collections.emptyIterator(); + } + if (hasNull && adjustedNumValues == 1) { + return Collections.singletonList(null).iterator(); + } + + ByteBuffer copy = buffer.asReadOnlyBuffer().order(buffer.order()); + copy.position(bucketsPosition); + final int[][] firstBucket = readBucket(copy, numBuckets > 1 ? bucketSize : lastBucketNumValues); + // iterator decodes and buffers a bucket at a time, paging through buckets as the iterator is consumed + return new Iterator() + { + private int currIndex = 0; + private int currentBucketIndex = 0; + private int[][] currentBucket = firstBucket; + + @Override + public boolean hasNext() + { + return currIndex < adjustedNumValues; + } + + @Override + public int[] next() + { + // null is handled special + if (hasNull && currIndex == 0) { + currIndex++; + return null; + } + if (!hasNext()) { + throw new NoSuchElementException(); + } + final int adjustedCurrIndex = hasNull ? currIndex - 1 : currIndex; + final int bucketNum = adjustedCurrIndex >> div; + // load next bucket if needed + if (bucketNum != currentBucketIndex) { + final int offset = copy.getInt(offsetsPosition + ((bucketNum - 1) * Integer.BYTES)); + copy.position(bucketsPosition + offset); + currentBucket = readBucket( + copy, + bucketNum < (numBuckets - 1) ? bucketSize : lastBucketNumValues + ); + currentBucketIndex = bucketNum; + } + int offset = adjustedCurrIndex & rem; + currIndex++; + return currentBucket[offset]; + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("buffer", buffer); + inspector.visit("hasNulls", hasNull); + inspector.visit("bucketSize", bucketSize); + } + + private int getBucketOffset(int bucket) + { + // get offset of that bucket in the value buffer, subtract 1 to get the starting position because we only store the + // ending offset, so look at the ending offset of the previous bucket, or 0 if this is the first bucket + return bucketsPosition + (bucket > 0 ? buffer.getInt(offsetsPosition + ((bucket - 1) * Integer.BYTES)) : 0); + } + + + /** + * Performs byte-by-byte comparison of the first value in a bucket with the specified value. Note that this method + * MUST be prepared before calling, as it expects the length of the first value to have already been read externally, + * and the buffer position to be at the start of the first bucket value. The final buffer position will be the + * 'shared prefix length' of the first value in the bucket and the value to compare. + * + * Bytes are compared using {@link StringUtils#compareUtf8UsingJavaStringOrdering(byte, byte)}. Therefore, when the + * values are UTF-8 encoded strings, the ordering is compatible with {@link String#compareTo(String)}. + */ + private static int compareBucketFirstValue(ByteBuffer bucketBuffer, int length, int[] value) + { + final int startOffset = bucketBuffer.position(); + final int commonLength = Math.min(length, value.length); + // save the length of the shared prefix with the first value of the bucket and the value to match so we + // can use it later to skip over all values in the bucket that share a longer prefix with the first value + // (the bucket is sorted, so the prefix length gets smaller as values increase) + int sharedPrefix; + int comparison = 0; + for (sharedPrefix = 0; sharedPrefix < commonLength; sharedPrefix++) { + comparison = Integer.compare(bucketBuffer.getInt(), value[sharedPrefix]); + if (comparison != 0) { + bucketBuffer.position(startOffset + (sharedPrefix * Integer.BYTES)); + break; + } + } + return comparison; + } + + /** + * Finds a value in a bucket among the fragments. The first value is assumed to have been already compared against + * and be smaller than the value we are looking for. This comparison is the source of the 'shared prefix', which is + * the length which the value has in common with the first value of the bucket. + * + * This method uses this shared prefix length to skip more expensive byte by byte full value comparisons when + * possible by comparing the shared prefix length with the prefix length of the fragment. Since the bucket is always + * sorted, prefix lengths shrink as you progress to higher indexes, and we can use this to reason that a fragment + * with a longer prefix length than the shared prefix will always sort before the value we are looking for, and values + * which have a shorter prefix will always be greater than the value we are looking for, so we only need to do a + * full comparison if the prefix length is the same + * + * this method modifies the position of {@link #buffer} + */ + private int findValueInBucket( + int[] value, + int currBucketFirstValueIndex, + int bucketSize, + int sharedPrefix + ) + { + int relativePosition = 0; + int prefixLength; + // scan through bucket values until we find match or compare numValues + int insertionPoint = 1; + while (++relativePosition < bucketSize) { + prefixLength = VByte.readInt(buffer); + if (prefixLength > sharedPrefix) { + final int skip = VByte.readInt(buffer); + buffer.position(buffer.position() + (skip * Integer.BYTES)); + insertionPoint++; + } else if (prefixLength < sharedPrefix) { + // prefix is smaller, that means this value sorts ahead of it + break; + } else { + final int fragmentLength = VByte.readInt(buffer); + final int common = Math.min(fragmentLength, value.length - prefixLength); + int fragmentComparison = 0; + for (int i = 0; i < common; i++) { + fragmentComparison = Integer.compare( + buffer.getInt(buffer.position() + (i * Integer.BYTES)), + value[prefixLength + i] + ); + if (fragmentComparison != 0) { + break; + } + } + if (fragmentComparison == 0) { + fragmentComparison = Integer.compare(prefixLength + fragmentLength, value.length); + } + + if (fragmentComparison == 0) { + return (currBucketFirstValueIndex + adjustIndex) + relativePosition; + } else if (fragmentComparison < 0) { + buffer.position(buffer.position() + (fragmentLength * Integer.BYTES)); + insertionPoint++; + } else { + break; + } + } + } + // (-(insertion point) - 1) + return -(currBucketFirstValueIndex + adjustIndex) + (-(insertionPoint) - 1); + } + + /** + * Get a value from a bucket at a relative position. + * + * This method modifies the position of the buffer. + */ + static int[] getFromBucket(ByteBuffer buffer, int offset) + { + int prefixPosition; + if (offset == 0) { + final int length = VByte.readInt(buffer); + final int[] firstValue = new int[length]; + for (int i = 0; i < length; i++) { + firstValue[i] = buffer.getInt(); + } + return firstValue; + } else { + final int firstLength = VByte.readInt(buffer); + prefixPosition = buffer.position(); + buffer.position(buffer.position() + (firstLength * Integer.BYTES)); + } + int pos = 0; + int prefixLength; + int fragmentLength; + int fragmentPosition; + // scan through bucket values until we reach offset + do { + prefixLength = VByte.readInt(buffer); + if (++pos < offset) { + // not there yet, no need to read anything other than the length to skip ahead + final int skipLength = VByte.readInt(buffer); + buffer.position(buffer.position() + (skipLength * Integer.BYTES)); + } else { + // we've reached our destination + fragmentLength = VByte.readInt(buffer); + fragmentPosition = buffer.position(); + break; + } + } while (true); + final int valueLength = prefixLength + fragmentLength; + final int[] value = new int[valueLength]; + for (int i = 0; i < valueLength; i++) { + if (i < prefixLength) { + value[i] = buffer.getInt(prefixPosition + (i * Integer.BYTES)); + } else { + value[i] = buffer.getInt(fragmentPosition + ((i - prefixLength) * Integer.BYTES)); + } + } + return value; + } + + + /** + * Read an entire bucket from a {@link ByteBuffer}, returning an array of reconstructed value bytes. + * + * This method modifies the position of the buffer. + */ + private static int[][] readBucket(ByteBuffer bucket, int numValues) + { + final int length = VByte.readInt(bucket); + final int[] prefix = new int[length]; + for (int i = 0; i < length; i++) { + prefix[i] = bucket.getInt(); + } + final int[][] bucketValues = new int[numValues][]; + bucketValues[0] = prefix; + int pos = 1; + while (pos < numValues) { + final int prefixLength = VByte.readInt(bucket); + final int fragmentLength = VByte.readInt(bucket); + final int[] value = new int[prefixLength + fragmentLength]; + for (int i = 0; i < prefixLength; i++) { + value[i] = prefix[i]; + } + for (int i = prefixLength; i < value.length; i++) { + value[i] = bucket.getInt(); + } + bucketValues[pos++] = value; + } + return bucketValues; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java new file mode 100644 index 000000000000..39c6f2713f6d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java @@ -0,0 +1,340 @@ +/* + * 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.data; + +import com.google.common.primitives.Ints; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.io.Channels; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; +import java.util.Arrays; +import java.util.Comparator; + +public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter +{ + private static final int MAX_LOG_BUFFER_SIZE = 26; + + public static final Comparator ARRAY_COMPARATOR = (o1, o2) -> { + //noinspection ArrayEquality + if (o1 == o2) { + return 0; + } + if (o1 == null) { + return -1; + } + if (o2 == null) { + return 1; + } + final int iter = Math.min(o1.length, o2.length); + for (int i = 0; i < iter; i++) { + final int cmp = Integer.compare(o1[i], o2[i]); + if (cmp == 0) { + continue; + } + return cmp; + } + return Integer.compare(o1.length, o2.length); + }; + + private final SegmentWriteOutMedium segmentWriteOutMedium; + private final int bucketSize; + private final ByteOrder byteOrder; + private final int[][] bucketBuffer; + private final ByteBuffer getOffsetBuffer; + private final int div; + + @Nullable + private int[] prevObject = null; + @Nullable + private WriteOutBytes headerOut = null; + @Nullable + private WriteOutBytes valuesOut = null; + private int numWritten = 0; + private ByteBuffer scratch; + private int logScratchSize = 10; + private boolean isClosed = false; + private boolean hasNulls = false; + + public FrontCodedIntArrayIndexedWriter( + SegmentWriteOutMedium segmentWriteOutMedium, + ByteOrder byteOrder, + int bucketSize + ) + { + if (Integer.bitCount(bucketSize) != 1 || bucketSize < 1 || bucketSize > 128) { + throw new IAE("bucketSize must be a power of two (from 1 up to 128) but was[%,d]", bucketSize); + } + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.scratch = ByteBuffer.allocate(1 << logScratchSize).order(byteOrder); + this.bucketSize = bucketSize; + this.byteOrder = byteOrder; + this.bucketBuffer = new int[bucketSize][]; + this.getOffsetBuffer = ByteBuffer.allocate(Integer.BYTES).order(byteOrder); + this.div = Integer.numberOfTrailingZeros(bucketSize); + } + + @Override + public void open() throws IOException + { + headerOut = segmentWriteOutMedium.makeWriteOutBytes(); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); + } + + @Override + public void write(@Nullable int[] value) throws IOException + { + + if (prevObject != null && ARRAY_COMPARATOR.compare(prevObject, value) >= 0) { + throw new ISE( + "Values must be sorted and unique. Element [%s] with value [%s] is before or equivalent to [%s]", + numWritten, + value == null ? null : Arrays.toString(value), + Arrays.toString(prevObject) + ); + } + + if (value == null) { + hasNulls = true; + return; + } + + // if the bucket buffer is full, write the bucket + if (numWritten > 0 && (numWritten % bucketSize) == 0) { + resetScratch(); + int written; + // write the bucket, growing scratch buffer as necessary + do { + written = writeBucket(scratch, bucketBuffer, bucketSize); + if (written < 0) { + growScratch(); + } + } while (written < 0); + scratch.flip(); + Channels.writeFully(valuesOut, scratch); + + resetScratch(); + // write end offset for current value + scratch.putInt((int) valuesOut.size()); + scratch.flip(); + Channels.writeFully(headerOut, scratch); + } + + bucketBuffer[numWritten % bucketSize] = value; + + ++numWritten; + prevObject = value; + } + + + @Override + public long getSerializedSize() throws IOException + { + if (!isClosed) { + flush(); + } + int headerAndValues = Ints.checkedCast(headerOut.size() + valuesOut.size()); + return Byte.BYTES + + Byte.BYTES + + Byte.BYTES + + VByte.computeIntSize(numWritten) + + VByte.computeIntSize(headerAndValues) + + headerAndValues; + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + if (!isClosed) { + flush(); + } + resetScratch(); + // version 0 + scratch.put((byte) 0); + scratch.put((byte) bucketSize); + scratch.put(hasNulls ? NullHandling.IS_NULL_BYTE : NullHandling.IS_NOT_NULL_BYTE); + VByte.writeInt(scratch, numWritten); + VByte.writeInt(scratch, Ints.checkedCast(headerOut.size() + valuesOut.size())); + scratch.flip(); + Channels.writeFully(channel, scratch); + headerOut.writeTo(channel); + valuesOut.writeTo(channel); + } + + @Override + public boolean isSorted() + { + return true; + } + + @Nullable + @Override + public int[] get(int index) throws IOException + { + if (index == 0 && hasNulls) { + return null; + } + final int adjustedIndex = hasNulls ? index - 1 : index; + final int relativeIndex = adjustedIndex % bucketSize; + // check for current page + if (adjustedIndex >= numWritten - bucketSize) { + return bucketBuffer[relativeIndex]; + } else { + final int bucket = adjustedIndex >> div; + long startOffset; + if (bucket == 0) { + startOffset = 0; + } else { + startOffset = getBucketOffset(bucket - 1); + } + long endOffset = getBucketOffset(bucket); + int bucketSize = Ints.checkedCast(endOffset - startOffset); + if (bucketSize == 0) { + return null; + } + final ByteBuffer bucketBuffer = ByteBuffer.allocate(bucketSize).order(byteOrder); + valuesOut.readFully(startOffset, bucketBuffer); + bucketBuffer.clear(); + return FrontCodedIntArrayIndexed.getFromBucket(bucketBuffer, relativeIndex); + } + } + + private long getBucketOffset(int index) throws IOException + { + getOffsetBuffer.clear(); + headerOut.readFully(index * (long) Integer.BYTES, getOffsetBuffer); + return getOffsetBuffer.getInt(0); + } + + private void flush() throws IOException + { + if (numWritten == 0) { + return; + } + int remainder = numWritten % bucketSize; + resetScratch(); + int written; + do { + written = writeBucket(scratch, bucketBuffer, remainder == 0 ? bucketSize : remainder); + if (written < 0) { + growScratch(); + } + } while (written < 0); + scratch.flip(); + Channels.writeFully(valuesOut, scratch); + resetScratch(); + isClosed = true; + } + + private void resetScratch() + { + scratch.position(0); + scratch.limit(scratch.capacity()); + } + + private void growScratch() + { + if (logScratchSize < MAX_LOG_BUFFER_SIZE) { + this.scratch = ByteBuffer.allocate(1 << ++logScratchSize).order(byteOrder); + } else { + throw new IllegalStateException("scratch buffer to big to write buckets"); + } + } + + /** + * Write bucket of values to a {@link ByteBuffer}. The first value is written completely, subsequent values are + * written with an integer to indicate how much of the first value in the bucket is a prefix of the value, followed + * by the remaining bytes of the value. + * + * Uses {@link VByte} encoded integers to indicate prefix length and value length. + */ + public static int writeBucket(ByteBuffer buffer, int[][] values, int numValues) + { + int written = 0; + int[] first = null; + while (written < numValues) { + int[] next = values[written]; + if (written == 0) { + first = next; + // the first value in the bucket is written completely as it is + int rem = writeValue(buffer, first); + // wasn't enough room, bail out + if (rem < 0) { + return rem; + } + } else { + // all other values must be partitioned into a prefix length and suffix bytes + int prefixLength = 0; + for (; prefixLength < first.length; prefixLength++) { + final int cmp = Integer.compare(first[prefixLength], next[prefixLength]); + if (cmp != 0) { + break; + } + } + final int[] suffix = new int[next.length - prefixLength]; + System.arraycopy(next, prefixLength, suffix, 0, suffix.length); + int rem = buffer.remaining() - VByte.computeIntSize(prefixLength); + // wasn't enough room, bail out + if (rem < 0) { + return rem; + } + VByte.writeInt(buffer, prefixLength); + rem = writeValue(buffer, suffix); + // wasn't enough room, bail out + if (rem < 0) { + return rem; + } + } + written++; + } + return written; + } + + /** + * Write a variable length int[] value to a {@link ByteBuffer}, storing the length as a {@link VByte} encoded + * integer followed by the value itself. Returns the number of bytes written to the buffer. This method returns a + * negative value if there is no room available in the buffer, so that it can be grown if needed. + */ + public static int writeValue(ByteBuffer buffer, int[] ints) + { + int remaining = buffer.remaining() - VByte.computeIntSize(ints.length) - ints.length; + if (remaining < 0) { + return remaining; + } + final int pos = buffer.position(); + VByte.writeInt(buffer, ints.length); + for (int anInt : ints) { + remaining = buffer.remaining() - Integer.BYTES; + if (remaining < 0) { + return remaining; + } + buffer.putInt(anInt); + } + return buffer.position() - pos; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java index 2931eaeb2b08..d127f3d89bb7 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java @@ -33,6 +33,11 @@ */ public class IncrementalIndexSchema { + public static IncrementalIndexSchema.Builder builder() + { + return new Builder(); + } + private final long minTimestamp; private final TimestampSpec timestampSpec; private final Granularity gran; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java new file mode 100644 index 000000000000..58d13c84dd47 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +public class ArrayOfLiteralsFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter +{ + protected ArrayOfLiteralsFieldColumnWriter( + String columnName, + String fieldName, + SegmentWriteOutMedium segmentWriteOutMedium, + IndexSpec indexSpec, + GlobalDictionaryIdLookup globalDictionaryIdLookup + ) + { + super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); + } + + @Override + int[] processValue(Object value) + { + if (value instanceof Object[]) { + Object[] array = (Object[]) value; + final int[] newIdsWhoDis = new int[array.length]; + for (int i = 0; i < array.length; i++) { + if (array[i] == null) { + newIdsWhoDis[i] = 0; + } else if (array[i] instanceof String) { + newIdsWhoDis[i] = globalDictionaryIdLookup.lookupString((String) array[i]); + } else if (array[i] instanceof Long) { + newIdsWhoDis[i] = globalDictionaryIdLookup.lookupLong((Long) array[i]); + } else if (array[i] instanceof Double) { + newIdsWhoDis[i] = globalDictionaryIdLookup.lookupDouble((Double) array[i]); + } else { + newIdsWhoDis[i] = -1; + } + Preconditions.checkArgument(newIdsWhoDis[i] >= 0, "unknown global id [%s] for value [%s]", newIdsWhoDis[i], array[i]); + } + return newIdsWhoDis; + } + return null; + } + + @Override + int lookupGlobalId(int[] value) + { + return globalDictionaryIdLookup.lookupArray(value); + } + + @Override + void writeColumnTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + writeLongAndDoubleColumnLength(channel, 0, 0); + encodedValueSerializer.writeTo(channel, smoosher); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java index 77919c4bffd8..8608ff2c3d5f 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java @@ -22,7 +22,9 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; +import com.google.common.primitives.Doubles; import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; @@ -30,6 +32,7 @@ import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.BaseSingleValueDimensionSelector; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.NilColumnValueSelector; @@ -41,7 +44,6 @@ import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.ColumnarDoubles; import org.apache.druid.segment.data.ColumnarInts; import org.apache.druid.segment.data.ColumnarLongs; @@ -51,6 +53,7 @@ import org.apache.druid.segment.data.CompressedVariableSizedBlobColumn; import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier; import org.apache.druid.segment.data.FixedIndexed; +import org.apache.druid.segment.data.FrontCodedIntArrayIndexed; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ObjectStrategy; @@ -58,6 +61,7 @@ import org.apache.druid.segment.data.VSizeColumnarInts; import org.apache.druid.segment.data.WritableSupplier; import org.apache.druid.segment.serde.DictionaryEncodedColumnPartSerde; +import org.apache.druid.segment.serde.NoIndexesColumnIndexSupplier; import org.apache.druid.segment.vector.NilVectorSelector; import org.apache.druid.segment.vector.ReadableVectorInspector; import org.apache.druid.segment.vector.ReadableVectorOffset; @@ -71,6 +75,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -94,6 +99,8 @@ public abstract class CompressedNestedDataComplexColumn stringDictionarySupplier; private final Supplier> longDictionarySupplier; private final Supplier> doubleDictionarySupplier; + private final Supplier arrayDictionarySupplier; + private final SmooshedFileMapper fileMapper; private final String rootFieldPath; @@ -112,6 +119,7 @@ public CompressedNestedDataComplexColumn( Supplier stringDictionary, Supplier> longDictionarySupplier, Supplier> doubleDictionarySupplier, + Supplier arrayDictionarySupplier, SmooshedFileMapper fileMapper, String rootFieldPath ) @@ -123,6 +131,7 @@ public CompressedNestedDataComplexColumn( this.stringDictionarySupplier = stringDictionary; this.longDictionarySupplier = longDictionarySupplier; this.doubleDictionarySupplier = doubleDictionarySupplier; + this.arrayDictionarySupplier = arrayDictionarySupplier; this.fileMapper = fileMapper; this.closer = Closer.create(); this.compressedRawColumnSupplier = compressedRawColumnSupplier; @@ -170,6 +179,60 @@ public FixedIndexed getDoubleDictionary() return doubleDictionarySupplier.get(); } + public FrontCodedIntArrayIndexed getArrayDictionary() + { + return arrayDictionarySupplier.get(); + } + + public Iterable getArraysIterable() + { + Iterable arrays = () -> { + final TStringDictionary stringDictionary = stringDictionarySupplier.get(); + final FixedIndexed longDictionary = longDictionarySupplier.get(); + final FixedIndexed doubleDictionary = doubleDictionarySupplier.get(); + + return new Iterator() + { + final Iterator delegate = arrayDictionarySupplier.get().iterator(); + + @Override + public boolean hasNext() + { + return delegate.hasNext(); + } + + @Override + public Object[] next() + { + final int[] next = delegate.next(); + final Object[] nextArray = new Object[next.length]; + for (int i = 0; i < nextArray.length; i++) { + nextArray[i] = lookupId(next[i]); + } + return nextArray; + } + + private Object lookupId(int globalId) + { + if (globalId == 0) { + return null; + } + final int adjustLongId = stringDictionary.size(); + final int adjustDoubleId = stringDictionary.size() + longDictionary.size(); + if (globalId < adjustLongId) { + return StringUtils.fromUtf8Nullable(stringDictionary.get(globalId)); + } else if (globalId < adjustDoubleId) { + return longDictionary.get(globalId - adjustLongId); + } else if (globalId < adjustDoubleId + doubleDictionary.size()) { + return doubleDictionary.get(globalId - adjustDoubleId); + } + throw new IAE("Unknown globalId [%s]", globalId); + } + }; + }; + return arrays; + } + public ImmutableBitmap getNullValues() { return nullValues; @@ -327,7 +390,11 @@ public void close() } @Override - public DimensionSelector makeDimensionSelector(List path, ReadableOffset readableOffset, ExtractionFn fn) + public DimensionSelector makeDimensionSelector( + List path, + ReadableOffset readableOffset, + ExtractionFn fn + ) { final String field = getField(path); Preconditions.checkNotNull(field, "Null field"); @@ -335,9 +402,43 @@ public DimensionSelector makeDimensionSelector(List path, Readab if (fields.indexOf(field) >= 0) { DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(field).getColumn(); return col.makeDimensionSelector(readableOffset, fn); - } else { - return DimensionSelector.constant(null); } + if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { + final NestedPathPart lastPath = path.get(path.size() - 1); + final String arrayField = getField(path.subList(0, path.size() - 1)); + if (fields.indexOf(arrayField) >= 0) { + final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); + ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset); + return new BaseSingleValueDimensionSelector() + { + @Nullable + @Override + protected String getValue() + { + Object o = arraySelector.getObject(); + if (o instanceof Object[]) { + Object[] array = (Object[]) o; + if (elementNumber < array.length) { + Object element = array[elementNumber]; + if (element == null) { + return null; + } + return String.valueOf(element); + } + } + return null; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + arraySelector.inspectRuntimeShape(inspector); + } + }; + } + } + return DimensionSelector.constant(null); } @Override @@ -349,9 +450,73 @@ public ColumnValueSelector makeColumnValueSelector(List path, if (fields.indexOf(field) >= 0) { BaseColumn col = getColumnHolder(field).getColumn(); return col.makeColumnValueSelector(readableOffset); - } else { - return NilColumnValueSelector.instance(); } + if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { + final NestedPathPart lastPath = path.get(path.size() - 1); + final String arrayField = getField(path.subList(0, path.size() - 1)); + if (fields.indexOf(arrayField) >= 0) { + final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); + ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset); + return new ColumnValueSelector() + { + @Override + public boolean isNull() + { + Object o = getObject(); + return !(o instanceof Number); + } + + @Override + public long getLong() + { + Object o = getObject(); + return o instanceof Number ? ((Number) o).longValue() : 0L; + } + + @Override + public float getFloat() + { + Object o = getObject(); + return o instanceof Number ? ((Number) o).floatValue() : 0f; + } + + @Override + public double getDouble() + { + Object o = getObject(); + return o instanceof Number ? ((Number) o).doubleValue() : 0.0; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + arraySelector.inspectRuntimeShape(inspector); + } + + @Nullable + @Override + public Object getObject() + { + Object o = arraySelector.getObject(); + if (o instanceof Object[]) { + Object[] array = (Object[]) o; + if (elementNumber < array.length) { + return array[elementNumber]; + } + } + return null; + } + + @Override + public Class classOfObject() + { + return Object.class; + } + }; + } + } + return NilColumnValueSelector.instance(); } @Override @@ -380,9 +545,58 @@ public VectorObjectSelector makeVectorObjectSelector(List path, if (fields.indexOf(field) >= 0) { BaseColumn col = getColumnHolder(field).getColumn(); return col.makeVectorObjectSelector(readableOffset); - } else { - return NilVectorSelector.create(readableOffset); } + if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { + final NestedPathPart lastPath = path.get(path.size() - 1); + final String arrayField = getField(path.subList(0, path.size() - 1)); + if (fields.indexOf(arrayField) >= 0) { + final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); + VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset); + + return new VectorObjectSelector() + { + private final Object[] elements = new Object[arraySelector.getMaxVectorSize()]; + private int id = ReadableVectorInspector.NULL_ID; + + @Override + public Object[] getObjectVector() + { + if (readableOffset.getId() != id) { + final Object[] delegate = arraySelector.getObjectVector(); + for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) { + Object maybeArray = delegate[i]; + if (maybeArray instanceof Object[]) { + Object[] anArray = (Object[]) maybeArray; + if (elementNumber < anArray.length) { + elements[i] = anArray[elementNumber]; + } else { + elements[i] = null; + } + } else { + elements[i] = null; + } + } + id = readableOffset.getId(); + } + return elements; + } + + @Override + public int getMaxVectorSize() + { + return arraySelector.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return arraySelector.getCurrentVectorSize(); + } + }; + } + } + return NilVectorSelector.create(readableOffset); } @Override @@ -394,11 +608,124 @@ public VectorValueSelector makeVectorValueSelector(List path, Re if (fields.indexOf(field) >= 0) { BaseColumn col = getColumnHolder(field).getColumn(); return col.makeVectorValueSelector(readableOffset); - } else { - return NilVectorSelector.create(readableOffset); } - } + if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { + final NestedPathPart lastPath = path.get(path.size() - 1); + final String arrayField = getField(path.subList(0, path.size() - 1)); + if (fields.indexOf(arrayField) >= 0) { + final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); + VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset); + + return new VectorValueSelector() + { + private final long[] longs = new long[readableOffset.getMaxVectorSize()]; + private final double[] doubles = new double[readableOffset.getMaxVectorSize()]; + private final float[] floats = new float[readableOffset.getMaxVectorSize()]; + private final boolean[] nulls = new boolean[readableOffset.getMaxVectorSize()]; + private int id = ReadableVectorInspector.NULL_ID; + + private void computeNumbers() + { + if (readableOffset.getId() != id) { + final Object[] maybeArrays = arraySelector.getObjectVector(); + for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) { + Object maybeArray = maybeArrays[i]; + if (maybeArray instanceof Object[]) { + Object[] anArray = (Object[]) maybeArray; + if (elementNumber < anArray.length) { + if (anArray[elementNumber] instanceof Number) { + Number n = (Number) anArray[elementNumber]; + longs[i] = n.longValue(); + doubles[i] = n.doubleValue(); + floats[i] = n.floatValue(); + nulls[i] = false; + } else { + Double d = anArray[elementNumber] instanceof String + ? Doubles.tryParse((String) anArray[elementNumber]) + : null; + if (d != null) { + longs[i] = d.longValue(); + doubles[i] = d; + floats[i] = d.floatValue(); + nulls[i] = false; + } else { + nullElement(i); + } + } + } else { + nullElement(i); + } + } else { + // not an array? + nullElement(i); + } + } + id = readableOffset.getId(); + } + } + + private void nullElement(int i) + { + longs[i] = 0L; + doubles[i] = 0L; + floats[i] = 0L; + nulls[i] = true; + } + + @Override + public long[] getLongVector() + { + if (readableOffset.getId() != id) { + computeNumbers(); + } + return longs; + } + + @Override + public float[] getFloatVector() + { + if (readableOffset.getId() != id) { + computeNumbers(); + } + return floats; + } + @Override + public double[] getDoubleVector() + { + if (readableOffset.getId() != id) { + computeNumbers(); + } + return doubles; + } + + @Nullable + @Override + public boolean[] getNullVector() + { + if (readableOffset.getId() != id) { + computeNumbers(); + } + return nulls; + } + + @Override + public int getMaxVectorSize() + { + return arraySelector.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return arraySelector.getCurrentVectorSize(); + } + }; + } + } + return NilVectorSelector.create(readableOffset); + } @Nullable @@ -426,6 +753,12 @@ public ColumnIndexSupplier getColumnIndexSupplier(List path) { final String field = getField(path); if (fields.indexOf(field) < 0) { + if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { + final String arrayField = getField(path.subList(0, path.size() - 1)); + if (fields.indexOf(arrayField) >= 0) { + return NoIndexesColumnIndexSupplier.getInstance(); + } + } return null; } return getColumnHolder(field).getIndexSupplier(); @@ -491,10 +824,16 @@ private ColumnHolder readNestedFieldColumn(String field) int doublesLength = bb.getInt(); dataBuffer.position(dataBuffer.position() + Integer.BYTES + Integer.BYTES); int pos = dataBuffer.position(); - final Supplier longs = longsLength > 0 ? CompressedColumnarLongsSupplier.fromByteBuffer(dataBuffer, metadata.getByteOrder()) : () -> null; + final Supplier longs = longsLength > 0 ? CompressedColumnarLongsSupplier.fromByteBuffer( + dataBuffer, + metadata.getByteOrder() + ) : () -> null; dataBuffer.position(pos + longsLength); pos = dataBuffer.position(); - final Supplier doubles = doublesLength > 0 ? CompressedColumnarDoublesSuppliers.fromByteBuffer(dataBuffer, metadata.getByteOrder()) : () -> null; + final Supplier doubles = doublesLength > 0 ? CompressedColumnarDoublesSuppliers.fromByteBuffer( + dataBuffer, + metadata.getByteOrder() + ) : () -> null; dataBuffer.position(pos + doublesLength); final WritableSupplier ints; if (version == DictionaryEncodedColumnPartSerde.VERSION.COMPRESSED) { @@ -503,7 +842,7 @@ private ColumnHolder readNestedFieldColumn(String field) ints = VSizeColumnarInts.readFromByteBuffer(dataBuffer); } ColumnType theType = types.getSingleType(); - columnBuilder.setType(theType == null ? ValueType.STRING : theType.getType()); + columnBuilder.setType(theType == null ? ColumnType.STRING : theType); GenericIndexed rBitmaps = GenericIndexed.read( dataBuffer, @@ -521,6 +860,7 @@ private ColumnHolder readNestedFieldColumn(String field) stringDictionarySupplier.get(), longDictionarySupplier.get(), doubleDictionarySupplier.get(), + arrayDictionarySupplier != null ? arrayDictionarySupplier.get() : null, localDict, hasNull ? rBitmaps.get(0) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java index 821f3ecc26f0..a4f28e20919e 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java @@ -24,8 +24,10 @@ import it.unimi.dsi.fastutil.doubles.Double2IntMap; import it.unimi.dsi.fastutil.longs.Long2IntLinkedOpenHashMap; import it.unimi.dsi.fastutil.longs.Long2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntAVLTreeMap; import it.unimi.dsi.fastutil.objects.Object2IntLinkedOpenHashMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter; import javax.annotation.Nullable; @@ -41,6 +43,8 @@ public class GlobalDictionaryIdLookup private final Double2IntMap doubleLookup; + private final Object2IntMap arrayLookup; + private int dictionarySize; public GlobalDictionaryIdLookup() @@ -51,6 +55,13 @@ public GlobalDictionaryIdLookup() longLookup.defaultReturnValue(-1); this.doubleLookup = new Double2IntLinkedOpenHashMap(); doubleLookup.defaultReturnValue(-1); + this.arrayLookup = new Object2IntAVLTreeMap<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR); + this.arrayLookup.defaultReturnValue(-1); + } + + public int getSize() + { + return dictionarySize; } public void addString(@Nullable String value) @@ -99,4 +110,18 @@ public int lookupDouble(@Nullable Double value) } return doubleLookup.get(value.doubleValue()); } + + public void addArray(int[] value) + { + int id = dictionarySize++; + arrayLookup.put(value, id); + } + + public int lookupArray(@Nullable int[] value) + { + if (value == null) { + return 0; + } + return arrayLookup.getInt(value); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java index 7cb775cc5482..861900ca45dc 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java @@ -32,16 +32,22 @@ public class GlobalDictionarySortedCollector private final Indexed sortedStrings; private final Indexed sortedLongs; private final Indexed sortedDoubles; + private final Iterable sortedArrays; + private final int arrayCount; public GlobalDictionarySortedCollector( Indexed sortedStrings, Indexed sortedLongs, - Indexed sortedDoubles + Indexed sortedDoubles, + Iterable sortedArrays, + int arrayCount ) { this.sortedStrings = sortedStrings; this.sortedLongs = sortedLongs; this.sortedDoubles = sortedDoubles; + this.sortedArrays = sortedArrays; + this.arrayCount = arrayCount; } public Indexed getSortedStrings() @@ -59,8 +65,28 @@ public Indexed getSortedDoubles() return sortedDoubles; } - public int size() + public Iterable getSortedArrays() { - return sortedStrings.size() + sortedLongs.size() + sortedDoubles.size(); + return sortedArrays; + } + + public int getStringCardinality() + { + return sortedStrings.size(); + } + + public int getLongCardinality() + { + return sortedLongs.size(); + } + + public int getDoubleCardinality() + { + return sortedDoubles.size(); + } + + public int getArrayCardinality() + { + return arrayCount; } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java index 4f22577b41b3..31690b32f2db 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java @@ -19,18 +19,23 @@ package org.apache.druid.segment.nested; +import com.google.common.base.Preconditions; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ComparatorDimensionDictionary; import org.apache.druid.segment.ComparatorSortedDimensionDictionary; import org.apache.druid.segment.DimensionDictionary; import org.apache.druid.segment.NestedDataColumnIndexer; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.IndexedIterable; import javax.annotation.Nullable; +import java.util.Comparator; import java.util.Iterator; +import java.util.Set; +import java.util.TreeSet; /** * Used by {@link NestedDataColumnIndexer} to build the global value dictionary, which can be converted into a @@ -42,45 +47,107 @@ public class GlobalDimensionDictionary private final ComparatorDimensionDictionary stringDictionary; private final ComparatorDimensionDictionary longDictionary; private final ComparatorDimensionDictionary doubleDictionary; + private final Set stringArrays; + private final Set longArrays; + private final Set doubleArrays; + + private int arrayBytesSizeEstimate; public GlobalDimensionDictionary() { - this.stringDictionary = new ComparatorDimensionDictionary(GenericIndexed.STRING_STRATEGY) { + this.stringDictionary = new ComparatorDimensionDictionary(GenericIndexed.STRING_STRATEGY) + { @Override public long estimateSizeOfValue(String value) { return StructuredDataProcessor.estimateStringSize(value); } }; - this.longDictionary = new ComparatorDimensionDictionary(ColumnType.LONG.getNullableStrategy()) { + this.longDictionary = new ComparatorDimensionDictionary(ColumnType.LONG.getNullableStrategy()) + { @Override public long estimateSizeOfValue(Long value) { return StructuredDataProcessor.getLongObjectEstimateSize(); } }; - this.doubleDictionary = new ComparatorDimensionDictionary(ColumnType.DOUBLE.getNullableStrategy()) { + this.doubleDictionary = new ComparatorDimensionDictionary(ColumnType.DOUBLE.getNullableStrategy()) + { @Override public long estimateSizeOfValue(Double value) { return StructuredDataProcessor.getDoubleObjectEstimateSize(); } }; + this.stringArrays = new TreeSet<>(ColumnType.STRING_ARRAY.getNullableStrategy()); + this.longArrays = new TreeSet<>(ColumnType.LONG_ARRAY.getNullableStrategy()); + this.doubleArrays = new TreeSet<>(ColumnType.DOUBLE_ARRAY.getNullableStrategy()); } - public void addLongValue(@Nullable Long value) + public int addLongValue(@Nullable Long value) { longDictionary.add(value); + return StructuredDataProcessor.getLongObjectEstimateSize(); } - public void addDoubleValue(@Nullable Double value) + public int addDoubleValue(@Nullable Double value) { doubleDictionary.add(value); + return StructuredDataProcessor.getDoubleObjectEstimateSize(); } - public void addStringValue(@Nullable String value) + public int addStringValue(@Nullable String value) { stringDictionary.add(value); + return StructuredDataProcessor.estimateStringSize(value); + } + + public int addStringArray(@Nullable Object[] value) + { + if (value == null) { + return 0; + } + stringArrays.add(value); + int sizeEstimate = 0; + for (Object o : value) { + if (o != null) { + sizeEstimate += addStringValue((String) o); + } + } + arrayBytesSizeEstimate += sizeEstimate; + return sizeEstimate; + } + + public int addLongArray(@Nullable Object[] value) + { + if (value == null) { + return 0; + } + longArrays.add(value); + int sizeEstimate = 0; + for (Object o : value) { + if (o != null) { + sizeEstimate += addLongValue((Long) o); + } + } + arrayBytesSizeEstimate += sizeEstimate; + return sizeEstimate; + } + + public int addDoubleArray(@Nullable Object[] value) + { + if (value == null) { + return 0; + } + doubleArrays.add(value); + int sizeEstimate = 0; + for (Object o : value) { + if (o != null) { + sizeEstimate += addDoubleValue((Double) o); + } + } + arrayBytesSizeEstimate += sizeEstimate; + return sizeEstimate; } public GlobalDictionarySortedCollector getSortedCollector() @@ -184,7 +251,9 @@ public Double get(int index) public int indexOf(Double value) { int id = doubleDictionary.getId(value); - return id < 0 ? DimensionDictionary.ABSENT_VALUE_ID : sortedDoubleDimensionDictionary.getSortedIdFromUnsortedId(id); + return id < 0 + ? DimensionDictionary.ABSENT_VALUE_ID + : sortedDoubleDimensionDictionary.getSortedIdFromUnsortedId(id); } @Override @@ -199,16 +268,70 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) // nothing to inspect } }; - return new GlobalDictionarySortedCollector(strings, longs, doubles); + + // offset by 1 because nulls are ignored by the indexer, but always global id 0 + final int adjustLongs = 1 + strings.size(); + final int adjustDoubles = adjustLongs + longs.size(); + TreeSet sortedArrays = new TreeSet<>(new Comparator() + { + @Override + public int compare(Object[] o1, Object[] o2) + { + return FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR.compare(convertArray(o1), convertArray(o2)); + } + + @Nullable + private int[] convertArray(Object[] array) + { + if (array == null) { + return null; + } + final int[] newIdsWhoDis = new int[array.length]; + for (int i = 0; i < array.length; i++) { + if (array[i] == null) { + newIdsWhoDis[i] = 0; + } else if (array[i] instanceof String) { + // offset by 1 because nulls are ignored by the indexer, but always global id 0 + newIdsWhoDis[i] = 1 + strings.indexOf((String) array[i]); + } else if (array[i] instanceof Long) { + newIdsWhoDis[i] = longs.indexOf((Long) array[i]) + adjustLongs; + } else if (array[i] instanceof Double) { + newIdsWhoDis[i] = doubles.indexOf((Double) array[i]) + adjustDoubles; + } else { + newIdsWhoDis[i] = -1; + } + Preconditions.checkArgument( + newIdsWhoDis[i] >= 0, + "unknown global id [%s] for value [%s]", + newIdsWhoDis[i], + array[i] + ); + } + return newIdsWhoDis; + } + }); + sortedArrays.addAll(stringArrays); + sortedArrays.addAll(longArrays); + sortedArrays.addAll(doubleArrays); + + return new GlobalDictionarySortedCollector(strings, longs, doubles, sortedArrays, sortedArrays.size()); } public long sizeInBytes() { - return stringDictionary.sizeInBytes() + longDictionary.sizeInBytes() + doubleDictionary.sizeInBytes(); + return stringDictionary.sizeInBytes() + + longDictionary.sizeInBytes() + + doubleDictionary.sizeInBytes() + + arrayBytesSizeEstimate; } public int getCardinality() { - return stringDictionary.size() + longDictionary.size() + doubleDictionary.size(); + return stringDictionary.size() + + longDictionary.size() + + doubleDictionary.size() + + stringArrays.size() + + longArrays.size() + + doubleArrays.size(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/LocalDimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/nested/LocalDimensionDictionary.java index 97b447daddf9..21a546781864 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/LocalDimensionDictionary.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/LocalDimensionDictionary.java @@ -25,6 +25,8 @@ public class LocalDimensionDictionary { private final Int2IntOpenHashMap globalIdToLocalId = new Int2IntOpenHashMap(); + private int nextLocalId = 0; + public LocalDimensionDictionary() { this.globalIdToLocalId.defaultReturnValue(-1); @@ -35,8 +37,6 @@ public Int2IntOpenHashMap getGlobalIdToLocalId() return globalIdToLocalId; } - private int nextLocalId = 0; - public int add(int originalValue) { int prev = globalIdToLocalId.get(originalValue); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java index f73b72209c85..0f8c03924c07 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java @@ -24,6 +24,7 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; @@ -46,11 +47,13 @@ import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.data.DictionaryWriter; import org.apache.druid.segment.data.FixedIndexedWriter; +import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.GenericIndexedWriter; import org.apache.druid.segment.serde.Serializer; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; +import javax.annotation.Nullable; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; @@ -67,6 +70,7 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer processLiteralField(ArrayList fieldPath, Object fieldValue) + public StructuredDataProcessor.ProcessedLiteral processLiteralField(ArrayList fieldPath, @Nullable Object fieldValue) { final GlobalDictionaryEncodedFieldColumnWriter writer = fieldWriters.get( NestedPathFinder.toNormalizedJsonPath(fieldPath) @@ -104,6 +108,32 @@ public StructuredDataProcessor.ProcessedLiteral processLiteralField(ArrayList } return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; } + + @Nullable + @Override + public ProcessedLiteral processArrayOfLiteralsField( + ArrayList fieldPath, + @Nullable Object maybeArrayOfLiterals + ) + { + ExprEval eval = ExprEval.bestEffortOf(maybeArrayOfLiterals); + if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) { + final GlobalDictionaryEncodedFieldColumnWriter writer = fieldWriters.get( + NestedPathFinder.toNormalizedJsonPath(fieldPath) + ); + if (writer != null) { + try { + writer.addValue(rowCount, eval.value()); + // serializer doesn't use size estimate + return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; + } + catch (IOException e) { + throw new RuntimeException(":("); + } + } + } + return null; + } }; private byte[] metadataBytes; @@ -114,6 +144,7 @@ public StructuredDataProcessor.ProcessedLiteral processLiteralField(ArrayList private DictionaryWriter dictionaryWriter; private FixedIndexedWriter longDictionaryWriter; private FixedIndexedWriter doubleDictionaryWriter; + private FrontCodedIntArrayIndexedWriter arrayDictionaryWriter; private CompressedVariableSizedBlobColumnSerializer rawWriter; private ByteBufferWriter nullBitmapWriter; private MutableBitmap nullRowsBitmap; @@ -121,6 +152,11 @@ public StructuredDataProcessor.ProcessedLiteral processLiteralField(ArrayList private int rowCount = 0; private boolean closedForWrite = false; + private boolean stringDictionarySerialized = false; + private boolean longDictionarySerialized = false; + private boolean doubleDictionarySerialized = false; + private boolean arrayDictionarySerialized = false; + public NestedDataColumnSerializer( String name, IndexSpec indexSpec, @@ -136,6 +172,11 @@ public NestedDataColumnSerializer( this.globalDictionaryIdLookup = new GlobalDictionaryIdLookup(); } + public GlobalDictionaryIdLookup getGlobalLookup() + { + return globalDictionaryIdLookup; + } + @Override public void open() throws IOException { @@ -170,6 +211,13 @@ public void open() throws IOException ); doubleDictionaryWriter.open(); + arrayDictionaryWriter = new FrontCodedIntArrayIndexedWriter( + segmentWriteOutMedium, + ByteOrder.nativeOrder(), + 4 + ); + arrayDictionaryWriter.open(); + rawWriter = new CompressedVariableSizedBlobColumnSerializer( getInternalFileName(name, RAW_FILE_NAME), segmentWriteOutMedium, @@ -215,7 +263,7 @@ public void serializeFields(SortedMap dictionaryValues) throws IOException { + if (stringDictionarySerialized) { + throw new ISE("String dictionary already serialized for column [%s], cannot serialize again", name); + } + // null is always 0 dictionaryWriter.write(null); globalDictionaryIdLookup.addString(null); for (String value : dictionaryValues) { @@ -251,10 +313,17 @@ public void serializeStringDictionary(Iterable dictionaryValues) throws dictionaryWriter.write(value); globalDictionaryIdLookup.addString(value); } + stringDictionarySerialized = true; } public void serializeLongDictionary(Iterable dictionaryValues) throws IOException { + if (!stringDictionarySerialized) { + throw new ISE("Must serialize string value dictionary before serializing long dictionary for column [%s]", name); + } + if (longDictionarySerialized) { + throw new ISE("Long dictionary already serialized for column [%s], cannot serialize again", name); + } for (Long value : dictionaryValues) { if (value == null) { continue; @@ -262,10 +331,20 @@ public void serializeLongDictionary(Iterable dictionaryValues) throws IOEx longDictionaryWriter.write(value); globalDictionaryIdLookup.addLong(value); } + longDictionarySerialized = true; } public void serializeDoubleDictionary(Iterable dictionaryValues) throws IOException { + if (!stringDictionarySerialized) { + throw new ISE("Must serialize string value dictionary before serializing double dictionary for column [%s]", name); + } + if (!longDictionarySerialized) { + throw new ISE("Must serialize long value dictionary before serializing double dictionary for column [%s]", name); + } + if (doubleDictionarySerialized) { + throw new ISE("Double dictionary already serialized for column [%s], cannot serialize again", name); + } for (Double value : dictionaryValues) { if (value == null) { continue; @@ -273,11 +352,39 @@ public void serializeDoubleDictionary(Iterable dictionaryValues) throws doubleDictionaryWriter.write(value); globalDictionaryIdLookup.addDouble(value); } + doubleDictionarySerialized = true; + } + + public void serializeArrayDictionary(Iterable dictionaryValues) throws IOException + { + if (!stringDictionarySerialized) { + throw new ISE("Must serialize string value dictionary before serializing array dictionary for column [%s]", name); + } + if (!longDictionarySerialized) { + throw new ISE("Must serialize long value dictionary before serializing array dictionary for column [%s]", name); + } + if (!doubleDictionarySerialized) { + throw new ISE("Must serialize double value dictionary before serializing array dictionary for column [%s]", name); + } + if (arrayDictionarySerialized) { + throw new ISE("Array dictionary already serialized for column [%s], cannot serialize again", name); + } + for (int[] value : dictionaryValues) { + if (value == null) { + continue; + } + arrayDictionaryWriter.write(value); + globalDictionaryIdLookup.addArray(value); + } + arrayDictionarySerialized = true; } @Override public void serialize(ColumnValueSelector selector) throws IOException { + if (!arrayDictionarySerialized) { + throw new ISE("Must serialize value dictionaries before serializing values for column [%s]", name); + } StructuredData data = StructuredData.wrap(selector.getObject()); if (data == null) { nullRowsBitmap.add(rowCount); @@ -330,8 +437,8 @@ public void writeTo( { Preconditions.checkState(closedForWrite, "Not closed yet!"); Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); - // version 4 - channel.write(ByteBuffer.wrap(new byte[]{0x04})); + // version 5 + channel.write(ByteBuffer.wrap(new byte[]{0x05})); channel.write(ByteBuffer.wrap(metadataBytes)); fieldsWriter.writeTo(channel, smoosher); fieldsInfoWriter.writeTo(channel, smoosher); @@ -340,6 +447,7 @@ public void writeTo( writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME); writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME); writeInternal(smoosher, doubleDictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME); + writeInternal(smoosher, arrayDictionaryWriter, ARRAY_DICTIONARY_FILE_NAME); writeInternal(smoosher, rawWriter, RAW_FILE_NAME); if (!nullRowsBitmap.isEmpty()) { writeInternal(smoosher, nullBitmapWriter, NULL_BITMAP_FILE_NAME); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java index 507a531f4bf2..92a29c6d9db5 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.data.EncodedStringDictionaryWriter; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.FrontCodedIndexed; +import org.apache.druid.segment.data.FrontCodedIntArrayIndexed; import org.apache.druid.segment.data.GenericIndexed; import javax.annotation.Nullable; @@ -72,7 +73,7 @@ public static NestedDataColumnSupplier read( { final byte version = bb.get(); - if (version == 0x03 || version == 0x04) { + if (version == 0x03 || version == 0x04 || version == 0x05) { try { final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); final NestedDataColumnMetadata metadata; @@ -84,6 +85,8 @@ public static NestedDataColumnSupplier read( final Supplier frontCodedStringDictionarySupplier; final Supplier> longDictionarySupplier; final Supplier> doubleDictionarySupplier; + final Supplier arrayDictionarySupplier; + ColumnType simpleType; metadata = jsonMapper.readValue( @@ -99,7 +102,7 @@ public static NestedDataColumnSupplier read( simpleType = ColumnType.STRING; } else if (fields.size() == 1 && ((version == 0x03 && NestedPathFinder.JQ_PATH_ROOT.equals(fields.get(0))) || - (version == 0x04 && NestedPathFinder.JSON_PATH_ROOT.equals(fields.get(0)))) + ((version == 0x04 || version == 0x05) && NestedPathFinder.JSON_PATH_ROOT.equals(fields.get(0)))) ) { simpleType = fieldInfo.getTypes(0).getSingleType(); } else { @@ -162,6 +165,19 @@ public static NestedDataColumnSupplier read( metadata.getByteOrder(), Double.BYTES ); + if (version == 0x05) { + final ByteBuffer arrayDictionarybuffer = loadInternalFile( + mapper, + metadata, + NestedDataColumnSerializer.ARRAY_DICTIONARY_FILE_NAME + ); + arrayDictionarySupplier = FrontCodedIntArrayIndexed.read( + arrayDictionarybuffer, + metadata.getByteOrder() + ); + } else { + arrayDictionarySupplier = null; + } final ByteBuffer rawBuffer = loadInternalFile(mapper, metadata, NestedDataColumnSerializer.RAW_FILE_NAME); compressedRawColumnSupplier = CompressedVariableSizedBlobColumnSupplier.fromByteBuffer( NestedDataColumnSerializer.getInternalFileName( @@ -194,6 +210,7 @@ public static NestedDataColumnSupplier read( frontCodedStringDictionarySupplier, longDictionarySupplier, doubleDictionarySupplier, + arrayDictionarySupplier, columnConfig, mapper, simpleType @@ -217,6 +234,7 @@ public static NestedDataColumnSupplier read( private final Supplier frontCodedStringDictionarySupplier; private final Supplier> longDictionarySupplier; private final Supplier> doubleDictionarySupplier; + private final Supplier arrayDictionarySupplier; private final ColumnConfig columnConfig; private final SmooshedFileMapper fileMapper; @@ -234,6 +252,7 @@ private NestedDataColumnSupplier( Supplier frontCodedStringDictionarySupplier, Supplier> longDictionarySupplier, Supplier> doubleDictionarySupplier, + Supplier arrayDictionarySupplier, ColumnConfig columnConfig, SmooshedFileMapper fileMapper, @Nullable ColumnType simpleType @@ -249,6 +268,7 @@ private NestedDataColumnSupplier( this.frontCodedStringDictionarySupplier = frontCodedStringDictionarySupplier; this.longDictionarySupplier = longDictionarySupplier; this.doubleDictionarySupplier = doubleDictionarySupplier; + this.arrayDictionarySupplier = arrayDictionarySupplier; this.columnConfig = columnConfig; this.fileMapper = fileMapper; this.simpleType = simpleType; @@ -259,8 +279,10 @@ public ComplexColumn get() { if (version == 0x03) { return makeV3(); + } else if (version == 0x04) { + return makeV4(); } - return makeV4(); + return makeV5(); } @Nullable @@ -329,6 +351,38 @@ private NestedDataColumnV4 makeV4() ); } + private NestedDataColumnV5 makeV5() + { + if (frontCodedStringDictionarySupplier != null) { + return new NestedDataColumnV5<>( + metadata, + columnConfig, + compressedRawColumnSupplier, + nullValues, + fields, + fieldInfo, + frontCodedStringDictionarySupplier, + longDictionarySupplier, + doubleDictionarySupplier, + arrayDictionarySupplier, + fileMapper + ); + } + return new NestedDataColumnV5<>( + metadata, + columnConfig, + compressedRawColumnSupplier, + nullValues, + fields, + fieldInfo, + stringDictionary::singleThreaded, + longDictionarySupplier, + doubleDictionarySupplier, + arrayDictionarySupplier, + fileMapper + ); + } + private static ByteBuffer loadInternalFile( SmooshedFileMapper fileMapper, NestedDataColumnMetadata metadata, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java index 8c2cc91963c5..18527a6ba305 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java @@ -58,6 +58,7 @@ public NestedDataColumnV3( stringDictionary, longDictionarySupplier, doubleDictionarySupplier, + null, fileMapper, NestedPathFinder.JQ_PATH_ROOT ); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java index 96c5f56e2bd4..b5f00af308c6 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java @@ -57,6 +57,7 @@ public NestedDataColumnV4( stringDictionary, longDictionarySupplier, doubleDictionarySupplier, + null, fileMapper, NestedPathFinder.JSON_PATH_ROOT ); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java new file mode 100644 index 000000000000..cb7463c487ca --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Supplier; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier; +import org.apache.druid.segment.data.FixedIndexed; +import org.apache.druid.segment.data.FrontCodedIntArrayIndexed; +import org.apache.druid.segment.data.GenericIndexed; +import org.apache.druid.segment.data.Indexed; + +import java.nio.ByteBuffer; +import java.util.List; + +public class NestedDataColumnV5> + extends CompressedNestedDataComplexColumn +{ + public NestedDataColumnV5( + NestedDataColumnMetadata metadata, + ColumnConfig columnConfig, + CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, + ImmutableBitmap nullValues, + GenericIndexed fields, + NestedLiteralTypeInfo fieldInfo, + Supplier stringDictionary, + Supplier> longDictionarySupplier, + Supplier> doubleDictionarySupplier, + Supplier arrayDictionarySupplier, + SmooshedFileMapper fileMapper + ) + { + super( + metadata, + columnConfig, + compressedRawColumnSupplier, + nullValues, + fields, + fieldInfo, + stringDictionary, + longDictionarySupplier, + doubleDictionarySupplier, + arrayDictionarySupplier, + fileMapper, + NestedPathFinder.JSON_PATH_ROOT + ); + } + + @Override + public List parsePath(String path) + { + return NestedPathFinder.parseJsonPath(path); + } + + @Override + public String getFieldFileName(String fileNameBase, String field, int fieldIndex) + { + return NestedDataColumnSerializer.getInternalFileName( + fileNameBase, + NestedDataColumnSerializer.NESTED_FIELD_PREFIX + fieldIndex + ); + } + + @Override + public String getField(List path) + { + return NestedPathFinder.toNormalizedJsonPath(path); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java index be942a9dfd08..7fcf9b3884fc 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java @@ -46,6 +46,7 @@ import org.apache.druid.segment.data.ColumnarInts; import org.apache.druid.segment.data.ColumnarLongs; import org.apache.druid.segment.data.FixedIndexed; +import org.apache.druid.segment.data.FrontCodedIntArrayIndexed; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.ReadableOffset; @@ -82,11 +83,15 @@ public class NestedFieldLiteralDictionaryEncodedColumn globalLongDictionary; private final FixedIndexed globalDoubleDictionary; + private final FrontCodedIntArrayIndexed globalArrayDictionary; + private final FixedIndexed dictionary; private final ImmutableBitmap nullBitmap; private final int adjustLongId; private final int adjustDoubleId; + private final int adjustArrayId; + public NestedFieldLiteralDictionaryEncodedColumn( NestedLiteralTypeInfo.TypeSet types, @@ -96,6 +101,7 @@ public NestedFieldLiteralDictionaryEncodedColumn( TStringDictionary globalDictionary, FixedIndexed globalLongDictionary, FixedIndexed globalDoubleDictionary, + @Nullable FrontCodedIntArrayIndexed globalArrayDictionary, FixedIndexed dictionary, ImmutableBitmap nullBitmap ) @@ -108,10 +114,12 @@ public NestedFieldLiteralDictionaryEncodedColumn( this.globalDictionary = globalDictionary; this.globalLongDictionary = globalLongDictionary; this.globalDoubleDictionary = globalDoubleDictionary; + this.globalArrayDictionary = globalArrayDictionary; this.dictionary = dictionary; this.nullBitmap = nullBitmap; this.adjustLongId = globalDictionary.size(); this.adjustDoubleId = adjustLongId + globalLongDictionary.size(); + this.adjustArrayId = adjustDoubleId + globalDoubleDictionary.size(); } @Override @@ -215,6 +223,18 @@ private int getIdFromGlobalDictionary(@Nullable String val) } } + private Object lookupGlobalScalarObject(int globalId) + { + if (globalId < globalDictionary.size()) { + return StringUtils.fromUtf8Nullable(globalDictionary.get(globalId)); + } else if (globalId < globalDictionary.size() + globalLongDictionary.size()) { + return globalLongDictionary.get(globalId - adjustLongId); + } else if (globalId < globalDictionary.size() + globalLongDictionary.size() + globalDoubleDictionary.size()) { + return globalDoubleDictionary.get(globalId - adjustDoubleId); + } + throw new IllegalArgumentException("not a scalar in the dictionary"); + } + @Override public DimensionSelector makeDimensionSelector( ReadableOffset offset, @@ -524,7 +544,7 @@ public boolean isNull() }; } } - if (singleType == null) { + if (singleType == null || singleType.isArray()) { return new ColumnValueSelector() { @@ -538,12 +558,18 @@ public Object getObject() { final int localId = column.get(offset.getOffset()); final int globalId = dictionary.get(localId); - if (globalId < adjustLongId) { - return StringUtils.fromUtf8Nullable(globalDictionary.get(globalId)); - } else if (globalId < adjustDoubleId) { - return globalLongDictionary.get(globalId - adjustLongId); + if (globalId < adjustArrayId) { + return lookupGlobalScalarObject(globalId); } else { - return globalDoubleDictionary.get(globalId - adjustDoubleId); + int[] arr = globalArrayDictionary.get(globalId - adjustArrayId); + if (arr == null) { + return null; + } + final Object[] array = new Object[arr.length]; + for (int i = 0; i < arr.length; i++) { + array[i] = lookupGlobalScalarObject(arr[i]); + } + return array; } } @@ -698,6 +724,61 @@ public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(R @Override public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset) { + if (singleType != null && singleType.isArray()) { + return new VectorObjectSelector() + { + private final int[] vector = new int[offset.getMaxVectorSize()]; + private final Object[] objects = new Object[offset.getMaxVectorSize()]; + private int id = ReadableVectorInspector.NULL_ID; + + @Override + + public Object[] getObjectVector() + { + if (id == offset.getId()) { + return objects; + } + + if (offset.isContiguous()) { + column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize()); + } else { + column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize()); + } + for (int i = 0; i < offset.getCurrentVectorSize(); i++) { + final int globalId = dictionary.get(vector[i]); + if (globalId < adjustArrayId) { + objects[i] = lookupGlobalScalarObject(globalId); + } else { + int[] arr = globalArrayDictionary.get(globalId - adjustArrayId); + if (arr == null) { + objects[i] = null; + } else { + final Object[] array = new Object[arr.length]; + for (int j = 0; j < arr.length; j++) { + array[j] = lookupGlobalScalarObject(arr[j]); + } + objects[i] = array; + } + } + } + id = offset.getId(); + + return objects; + } + + @Override + public int getMaxVectorSize() + { + return offset.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return offset.getCurrentVectorSize(); + } + }; + } final class StringVectorSelector extends StringDictionaryEncodedColumn.StringVectorObjectSelector { public StringVectorSelector() diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedLiteralTypeInfo.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedLiteralTypeInfo.java index 4b4b89df8330..f2ba58865a97 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedLiteralTypeInfo.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedLiteralTypeInfo.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.nested; +import com.google.common.base.Preconditions; import com.google.common.collect.Sets; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; @@ -31,6 +32,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; +import java.util.Objects; import java.util.Set; /** @@ -43,6 +45,12 @@ public class NestedLiteralTypeInfo private static final byte LONG_MASK = 1 << 2; private static final byte DOUBLE_MASK = 1 << 3; + private static final byte STRING_ARRAY_MASK = 1 << 4; + + private static final byte LONG_ARRAY_MASK = 1 << 5; + + private static final byte DOUBLE_ARRAY_MASK = 1 << 6; + public static NestedLiteralTypeInfo read(ByteBuffer buffer, int length) { NestedLiteralTypeInfo typeInfo = new NestedLiteralTypeInfo(buffer); @@ -92,6 +100,25 @@ public String toString() { return convertToSet(types).toString(); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TypeSet typeSet = (TypeSet) o; + return types == typeSet.types; + } + + @Override + public int hashCode() + { + return Objects.hash(types); + } } public static class MutableTypeSet @@ -120,6 +147,22 @@ public MutableTypeSet add(ColumnType type) case DOUBLE: types |= DOUBLE_MASK; break; + case ARRAY: + Preconditions.checkNotNull(type.getElementType(), "ElementType must not be null"); + switch (type.getElementType().getType()) { + case STRING: + types |= STRING_ARRAY_MASK; + break; + case LONG: + types |= LONG_ARRAY_MASK; + break; + case DOUBLE: + types |= DOUBLE_ARRAY_MASK; + break; + default: + throw new ISE("Unsupported nested array type: [%s]", type.asTypeString()); + } + break; default: throw new ISE("Unsupported nested type: [%s]", type.asTypeString()); } @@ -154,6 +197,25 @@ public String toString() { return convertToSet(types).toString(); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MutableTypeSet typeSet = (MutableTypeSet) o; + return types == typeSet.types; + } + + @Override + public int hashCode() + { + return Objects.hash(types); + } } public static class Writer implements Serializer @@ -209,6 +271,18 @@ private static ColumnType getSingleType(byte types) singleType = ColumnType.DOUBLE; count++; } + if ((types & STRING_ARRAY_MASK) > 0) { + singleType = ColumnType.STRING_ARRAY; + count++; + } + if ((types & LONG_ARRAY_MASK) > 0) { + singleType = ColumnType.LONG_ARRAY; + count++; + } + if ((types & DOUBLE_ARRAY_MASK) > 0) { + singleType = ColumnType.DOUBLE_ARRAY; + count++; + } return count == 1 ? singleType : null; } @@ -224,6 +298,15 @@ public static Set convertToSet(byte types) if ((types & DOUBLE_MASK) > 0) { theTypes.add(ColumnType.DOUBLE); } + if ((types & STRING_ARRAY_MASK) > 0) { + theTypes.add(ColumnType.STRING_ARRAY); + } + if ((types & DOUBLE_ARRAY_MASK) > 0) { + theTypes.add(ColumnType.DOUBLE_ARRAY); + } + if ((types & LONG_ARRAY_MASK) > 0) { + theTypes.add(ColumnType.LONG_ARRAY); + } return theTypes; } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/StructuredDataProcessor.java b/processing/src/main/java/org/apache/druid/segment/nested/StructuredDataProcessor.java index 8549deda44fd..b944865ee613 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/StructuredDataProcessor.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/StructuredDataProcessor.java @@ -31,7 +31,28 @@ public abstract class StructuredDataProcessor { - public abstract ProcessedLiteral processLiteralField(ArrayList fieldPath, Object fieldValue); + protected StructuredDataProcessor() + { + } + + /** + * process a literal value that is definitely not a {@link Map}, {@link List}, or {@link Object[]} + */ + public abstract ProcessedLiteral processLiteralField( + ArrayList fieldPath, + @Nullable Object fieldValue + ); + + /** + * process a {@link List} or {@link Object[]} that might be an array of literals. If the object was an array of + * literals, returns a {@link ProcessedLiteral}, else returns null. + */ + @Nullable + public abstract ProcessedLiteral processArrayOfLiteralsField( + ArrayList fieldPath, + @Nullable Object maybeArrayOfLiterals + ); + /** * Process fields, returning a list of all paths to literal fields, represented as an ordered sequence of @@ -46,6 +67,8 @@ public ProcessResults processFields(Object raw) toProcess.add(new MapField(newPath, (Map) raw)); } else if (raw instanceof List) { toProcess.add(new ListField(newPath, (List) raw)); + } else if (raw instanceof Object[]) { + toProcess.add(new ArrayField(newPath, (Object[]) raw)); } else { return new ProcessResults().addLiteralField(newPath, processLiteralField(newPath, raw).getSize()); } @@ -58,6 +81,8 @@ public ProcessResults processFields(Object raw) accumulator.merge(processMapField(toProcess, (MapField) next)); } else if (next instanceof ListField) { accumulator.merge(processListField(toProcess, (ListField) next)); + } else if (next instanceof ArrayField) { + accumulator.merge(processArrayField(toProcess, (ArrayField) next)); } } return accumulator; @@ -92,24 +117,62 @@ private ProcessResults processListField(Queue toProcess, ListField list) // start with object reference, is probably a bit bigger than this... final ProcessResults results = new ProcessResults().withSize(8); final List theList = list.getList(); - for (int i = 0; i < theList.size(); i++) { - final ArrayList newPath = new ArrayList<>(list.getPath()); - newPath.add(new NestedPathArrayElement(i)); - final Object element = StructuredData.unwrap(theList.get(i)); - // maps and lists go back into the queue - if (element instanceof Map) { - toProcess.add(new MapField(newPath, (Map) element)); - } else if (element instanceof List) { - toProcess.add(new ListField(newPath, (List) element)); - } else { - // literals get processed - results.addLiteralField(newPath, processLiteralField(newPath, element).getSize()); + // check to see if the processor treats arrays of literals as literals, if so we can stop processing here + ProcessedLiteral maybeArrayOfLiterals = processArrayOfLiteralsField(list.getPath(), theList); + if (maybeArrayOfLiterals != null) { + results.addLiteralField(list.getPath(), maybeArrayOfLiterals.getSize()); + } else { + // else we have to dig into the list and process each element + for (int i = 0; i < theList.size(); i++) { + final ArrayList newPath = new ArrayList<>(list.getPath()); + newPath.add(new NestedPathArrayElement(i)); + final Object element = StructuredData.unwrap(theList.get(i)); + // maps and lists go back into the queue + if (element instanceof Map) { + toProcess.add(new MapField(newPath, (Map) element)); + } else if (element instanceof List) { + toProcess.add(new ListField(newPath, (List) element)); + } else if (element instanceof Object[]) { + toProcess.add(new ArrayField(newPath, (Object[]) element)); + } else { + results.addLiteralField(newPath, processLiteralField(newPath, element).getSize()); + } } } return results; } - abstract static class Field + private ProcessResults processArrayField(Queue toProcess, ArrayField array) + { + // start with object reference, is probably a bit bigger than this... + final ProcessResults results = new ProcessResults().withSize(8); + final Object[] theArray = array.getArray(); + // check to see if the processor treats arrays of literals as literals, if so we can stop processing here + ProcessedLiteral maybeArrayOfLiterals = processArrayOfLiteralsField(array.getPath(), theArray); + if (maybeArrayOfLiterals != null) { + results.addLiteralField(array.getPath(), maybeArrayOfLiterals.getSize()); + } else { + // else we have to dig into the list and process each element + for (int i = 0; i < theArray.length; i++) { + final ArrayList newPath = new ArrayList<>(array.getPath()); + newPath.add(new NestedPathArrayElement(i)); + final Object element = StructuredData.unwrap(theArray[i]); + // maps and lists go back into the queue + if (element instanceof Map) { + toProcess.add(new MapField(newPath, (Map) element)); + } else if (element instanceof List) { + toProcess.add(new ListField(newPath, (List) element)); + } else if (element instanceof Object[]) { + toProcess.add(new ArrayField(newPath, (Object[]) element)); + } else { + results.addLiteralField(newPath, processLiteralField(newPath, element).getSize()); + } + } + } + return results; + } + + private abstract static class Field { private final ArrayList path; @@ -140,6 +203,22 @@ public List getList() } } + static class ArrayField extends Field + { + private final Object[] array; + + ArrayField(ArrayList path, Object[] array) + { + super(path); + this.array = array; + } + + public Object[] getArray() + { + return array; + } + } + static class MapField extends Field { private final Map map; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java index 7440c1200471..88156202630a 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.nested; +import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; @@ -42,6 +43,32 @@ public VariantLiteralFieldColumnWriter( super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); } + + @Override + Object processValue(Object value) + { + if (value instanceof Object[]) { + Object[] array = (Object[]) value; + final int[] newIdsWhoDis = new int[array.length]; + for (int i = 0; i < array.length; i++) { + if (array[i] == null) { + newIdsWhoDis[i] = 0; + } else if (array[i] instanceof String) { + newIdsWhoDis[i] = globalDictionaryIdLookup.lookupString((String) array[i]); + } else if (array[i] instanceof Long) { + newIdsWhoDis[i] = globalDictionaryIdLookup.lookupLong((Long) array[i]); + } else if (array[i] instanceof Double) { + newIdsWhoDis[i] = globalDictionaryIdLookup.lookupDouble((Double) array[i]); + } else { + newIdsWhoDis[i] = -1; + } + Preconditions.checkArgument(newIdsWhoDis[i] >= 0, "unknown global id [%s] for value [%s]", newIdsWhoDis[i], array[i]); + } + return newIdsWhoDis; + } + return super.processValue(value); + } + @Override int lookupGlobalId(Object value) { @@ -52,6 +79,8 @@ int lookupGlobalId(Object value) return globalDictionaryIdLookup.lookupLong((Long) value); } else if (value instanceof Double) { return globalDictionaryIdLookup.lookupDouble((Double) value); + } else if (value instanceof int[]) { + return globalDictionaryIdLookup.lookupArray((int[]) value); } else { return globalDictionaryIdLookup.lookupString(String.valueOf(value)); } diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java index 62cd303bc933..896d47b7e64a 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java @@ -655,12 +655,13 @@ public ColumnCapabilities capabilities(ColumnInspector inspector, String columnN if (capabilities.is(ValueType.COMPLEX) && capabilities.getComplexTypeName().equals(NestedDataComplexTypeSerde.TYPE_NAME) && capabilities.isDictionaryEncoded().isTrue()) { + final boolean useDictionary = parts.isEmpty() || !(parts.get(parts.size() - 1) instanceof NestedPathArrayElement); return ColumnCapabilitiesImpl.createDefault() .setType(expectedType != null ? expectedType : ColumnType.STRING) - .setDictionaryEncoded(true) - .setDictionaryValuesSorted(true) - .setDictionaryValuesUnique(true) - .setHasBitmapIndexes(true) + .setDictionaryEncoded(useDictionary) + .setDictionaryValuesSorted(useDictionary) + .setDictionaryValuesUnique(useDictionary) + .setHasBitmapIndexes(useDictionary) .setHasNulls(expectedType == null || (expectedType.isNumeric() && NullHandling.sqlCompatible())); } diff --git a/processing/src/test/java/org/apache/druid/data/input/ResourceInputSource.java b/processing/src/test/java/org/apache/druid/data/input/ResourceInputSource.java new file mode 100644 index 000000000000..44aa52615d41 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/data/input/ResourceInputSource.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import org.apache.druid.data.input.impl.InputEntityIteratingReader; +import org.apache.druid.utils.CompressionUtils; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Collections; + +/** + * {@link InputSource} backed by {@link ClassLoader#getResourceAsStream(String)}, for all your embedded test data + * ingestion needs + */ +public class ResourceInputSource extends AbstractInputSource +{ + public static ResourceInputSource of(ClassLoader loader, String resourceFile) + { + return new ResourceInputSource(loader, resourceFile); + } + + private final ClassLoader classLoader; + private final String resourceFile; + + private ResourceInputSource(ClassLoader classLoader, String resourceFile) + { + this.classLoader = classLoader; + this.resourceFile = resourceFile; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public boolean needsFormat() + { + return true; + } + + @Override + protected InputSourceReader formattableReader( + InputRowSchema inputRowSchema, + InputFormat inputFormat, + File temporaryDirectory + ) + { + return new InputEntityIteratingReader( + inputRowSchema, + inputFormat, + Collections.singletonList(new ResourceStreamEntity(classLoader, resourceFile)).iterator(), + temporaryDirectory + ); + } + + public static class ResourceStreamEntity implements InputEntity + { + private final ClassLoader classLoader; + private final String resourceFile; + + public ResourceStreamEntity(ClassLoader classLoader, String resourceFile) + { + this.classLoader = classLoader; + this.resourceFile = resourceFile; + } + + @Nullable + @Override + public URI getUri() + { + try { + return classLoader.getResource(resourceFile).toURI(); + } + catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + + @Override + public InputStream open() throws IOException + { + final InputStream resourceStream = classLoader.getResourceAsStream(resourceFile); + return CompressionUtils.decompress(resourceStream, resourceFile); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java index d75f92e7f1a6..0dcbd97d05b5 100644 --- a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java +++ b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java @@ -19,432 +19,444 @@ package org.apache.druid.query; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import com.google.common.io.Files; -import org.apache.commons.io.IOUtils; -import org.apache.commons.io.LineIterator; -import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.ResourceInputSource; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.nary.TrinaryFn; import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.query.aggregation.AggregationTestHelper; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.segment.IncrementalIndexSegment; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.NestedDataDimensionSchema; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.transform.ExpressionTransform; +import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.SegmentId; import org.junit.rules.TemporaryFolder; import java.io.ByteArrayInputStream; import java.io.File; -import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.io.SequenceInputStream; -import java.nio.charset.StandardCharsets; +import java.nio.file.Files; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.function.BiFunction; public class NestedDataTestUtils { - public static final String SIMPLE_DATA_FILE = "simple-nested-test-data.json"; - public static final String SIMPLE_PARSER_FILE = "simple-nested-test-data-parser.json"; - public static final String SIMPLE_DATA_TSV_FILE = "simple-nested-test-data.tsv"; - public static final String SIMPLE_PARSER_TSV_FILE = "simple-nested-test-data-tsv-parser.json"; - public static final String SIMPLE_PARSER_TSV_TRANSFORM_FILE = "simple-nested-test-data-tsv-transform.json"; - public static final String SIMPLE_AGG_FILE = "simple-nested-test-data-aggs.json"; - - public static final String TYPES_DATA_FILE = "types-test-data.json"; - public static final String TYPES_PARSER_FILE = "types-test-data-parser.json"; - - public static final String NUMERIC_DATA_FILE = "numeric-nested-test-data.json"; - public static final String NUMERIC_PARSER_FILE = "numeric-nested-test-data-parser.json"; + public static final String SIMPLE_DATA_FILE = "nested-simple-test-data.json"; + public static final String SIMPLE_DATA_TSV_FILE = "nested-simple-test-data.tsv"; + public static final String NUMERIC_DATA_FILE = "nested-numeric-test-data.json"; + public static final String TYPES_DATA_FILE = "nested-types-test-data.json"; + public static final String ARRAY_TYPES_DATA_FILE = "nested-array-test-data.json"; public static final ObjectMapper JSON_MAPPER; + public static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("timestamp", null, null); + + public static final DimensionsSpec AUTO_DISCOVERY = + DimensionsSpec.builder() + .setUseNestedColumnIndexerForSchemaDiscovery(true) + .build(); + + public static final DimensionsSpec TSV_SCHEMA = + DimensionsSpec.builder() + .setDimensions( + Arrays.asList( + new StringDimensionSchema("dim"), + new NestedDataDimensionSchema("nest_json"), + new NestedDataDimensionSchema("nester_json"), + new NestedDataDimensionSchema("variant_json"), + new NestedDataDimensionSchema("list_json") + ) + ) + .build(); + public static final InputRowSchema AUTO_SCHEMA = new InputRowSchema( + TIMESTAMP_SPEC, + AUTO_DISCOVERY, + null + ); + + public static final InputRowSchema SIMPLE_DATA_TSV_SCHEMA = new InputRowSchema( + TIMESTAMP_SPEC, + TSV_SCHEMA, + null + ); + + public static DelimitedInputFormat SIMPLE_DATA_TSV_INPUT_FORMAT = DelimitedInputFormat.ofColumns( + "timestamp", + "dim", + "nest", + "nester", + "variant", + "list" + ); + public static final TransformSpec SIMPLE_DATA_TSV_TRANSFORM = new TransformSpec( + null, + Arrays.asList( + new ExpressionTransform("nest_json", "parse_json(nest)", TestExprMacroTable.INSTANCE), + new ExpressionTransform("nester_json", "parse_json(nester)", TestExprMacroTable.INSTANCE), + new ExpressionTransform("variant_json", "parse_json(variant)", TestExprMacroTable.INSTANCE), + new ExpressionTransform("list_json", "parse_json(list)", TestExprMacroTable.INSTANCE) + ) + ); + + public static final AggregatorFactory[] COUNT = new AggregatorFactory[]{ + new CountAggregatorFactory("count") + }; + static { JSON_MAPPER = TestHelper.makeJsonMapper(); JSON_MAPPER.registerModules(NestedDataModule.getJacksonModulesList()); } - public static List createSegments( - AggregationTestHelper helper, + public static List createSimpleSegmentsTsv( TemporaryFolder tempFolder, - Closer closer, - Granularity granularity, - boolean rollup, - int maxRowCount - ) throws Exception + Closer closer + ) + throws Exception { - return createSegments( - helper, + return createSimpleNestedTestDataTsvSegments( tempFolder, closer, - SIMPLE_DATA_FILE, - SIMPLE_PARSER_FILE, - SIMPLE_AGG_FILE, - granularity, - rollup, - maxRowCount + Granularities.NONE, + true ); } - public static List createTsvSegments( - AggregationTestHelper helper, + public static List createSimpleNestedTestDataTsvSegments( TemporaryFolder tempFolder, Closer closer, Granularity granularity, - boolean rollup, - int maxRowCount + boolean rollup ) throws Exception { return createSegments( - helper, tempFolder, closer, SIMPLE_DATA_TSV_FILE, - SIMPLE_PARSER_TSV_FILE, - SIMPLE_PARSER_TSV_TRANSFORM_FILE, - SIMPLE_AGG_FILE, + SIMPLE_DATA_TSV_INPUT_FORMAT, + TIMESTAMP_SPEC, + SIMPLE_DATA_TSV_SCHEMA.getDimensionsSpec(), + SIMPLE_DATA_TSV_TRANSFORM, + COUNT, granularity, - rollup, - maxRowCount + rollup ); } - public static Segment createIncrementalIndex( - Granularity granularity, - boolean rollup, - boolean deserializeComplexMetrics, - int maxRowCount - ) - throws Exception + public static Segment createSimpleNestedTestDataIncrementalIndex(TemporaryFolder tempFolder) throws Exception { - return createIncrementalIndex( + return createIncrementalIndexForJsonInput( + tempFolder, SIMPLE_DATA_FILE, - SIMPLE_PARSER_FILE, - SIMPLE_AGG_FILE, - granularity, - rollup, - deserializeComplexMetrics, - maxRowCount + Granularities.NONE, + true, + 1000 ); } - public static List createSegments( - AggregationTestHelper helper, + public static List createSimpleNestedTestDataSegments( TemporaryFolder tempFolder, - Closer closer, - String inputFileName, - String parserJsonFileName, - String aggJsonFileName, - Granularity granularity, - boolean rollup, - int maxRowCount - ) throws Exception + Closer closer + ) + throws Exception { - File segmentDir = tempFolder.newFolder(); - File inputFile = readFileFromClasspath(inputFileName); - FileInputStream inputDataStream = new FileInputStream(inputFile); - String parserJson = readFileFromClasspathAsString(parserJsonFileName); - String aggJson = readFileFromClasspathAsString(aggJsonFileName); - - helper.createIndex( - inputDataStream, - parserJson, - aggJson, - segmentDir, - 0, - granularity, - maxRowCount, - rollup - ); - inputDataStream.close(); - - final List segments = Lists.transform( - ImmutableList.of(segmentDir), - dir -> { - try { - return closer.register(new QueryableIndexSegment(helper.getIndexIO().loadIndex(dir), SegmentId.dummy(""))); - } - catch (IOException ex) { - throw new RuntimeException(ex); - } - } + return createSegmentsForJsonInput( + tempFolder, + closer, + SIMPLE_DATA_FILE, + Granularities.NONE, + true ); + } - return segments; + public static Segment createIncrementalIndexForJsonInput(TemporaryFolder tempFolder, String fileName) + throws Exception + { + return createIncrementalIndexForJsonInput( + tempFolder, + fileName, + Granularities.NONE, + true, + 1000 + ); } - public static List createSegments( - AggregationTestHelper helper, + public static Segment createIncrementalIndexForJsonInput( TemporaryFolder tempFolder, - Closer closer, - String inputFileName, - String parserJsonFileName, - String transformSpecJsonFileName, - String aggJsonFileName, + String file, Granularity granularity, boolean rollup, int maxRowCount - ) throws Exception + ) + throws Exception { - File segmentDir = tempFolder.newFolder(); - File inputFile = readFileFromClasspath(inputFileName); - FileInputStream inputDataStream = new FileInputStream(inputFile); - String parserJson = readFileFromClasspathAsString(parserJsonFileName); - String transformSpecJson = readFileFromClasspathAsString(transformSpecJsonFileName); - String aggJson = readFileFromClasspathAsString(aggJsonFileName); - - helper.createIndex( - inputDataStream, - parserJson, - transformSpecJson, - aggJson, - segmentDir, - 0, + return createIncrementalIndex( + tempFolder, + file, + JsonInputFormat.DEFAULT, + TIMESTAMP_SPEC, + AUTO_DISCOVERY, + TransformSpec.NONE, + COUNT, granularity, - maxRowCount, - rollup - ); - inputDataStream.close(); - - final List segments = Lists.transform( - ImmutableList.of(segmentDir), - dir -> { - try { - return closer.register(new QueryableIndexSegment(helper.getIndexIO().loadIndex(dir), SegmentId.dummy(""))); - } - catch (IOException ex) { - throw new RuntimeException(ex); - } - } + rollup, + maxRowCount ); - - return segments; } - public static List createSegmentsWithConcatenatedInput( - AggregationTestHelper helper, + public static List createSegmentsForJsonInput( TemporaryFolder tempFolder, Closer closer, + String inputFile, Granularity granularity, - boolean rollup, - int maxRowCount, - int numCopies, - int numSegments + boolean rollup ) throws Exception { - return createSegmentsWithConcatenatedInput( - helper, + return createSegments( tempFolder, closer, - SIMPLE_DATA_FILE, - SIMPLE_PARSER_FILE, - null, - SIMPLE_AGG_FILE, + inputFile, + JsonInputFormat.DEFAULT, + TIMESTAMP_SPEC, + AUTO_DISCOVERY, + TransformSpec.NONE, + COUNT, granularity, - rollup, - maxRowCount, - numCopies, - numSegments + rollup ); } - /** - * turn small test data into bigger test data by duplicating itself into a bigger stream - */ - public static List createSegmentsWithConcatenatedInput( - AggregationTestHelper helper, + public static List createSegmentsWithConcatenatedJsonInput( TemporaryFolder tempFolder, Closer closer, - String inputFileName, - String parserJsonFileName, - String transformSpecJsonFileName, - String aggJsonFileName, + String inputFile, Granularity granularity, boolean rollup, - int maxRowCount, int numCopies, int numSegments ) throws Exception { - String parserJson = readFileFromClasspathAsString(parserJsonFileName); - String transformSpecJson = transformSpecJsonFileName != null ? readFileFromClasspathAsString(transformSpecJsonFileName) : null; - String aggJson = readFileFromClasspathAsString(aggJsonFileName); - - List segmentDirs = Lists.newArrayListWithCapacity(numSegments); + List inputFiles = Lists.newArrayListWithCapacity(numSegments); for (int i = 0; i < numSegments; i++) { - List inputStreams = Lists.newArrayListWithCapacity(numCopies); - for (int j = 0; j < numCopies; j++) { - inputStreams.add(new FileInputStream(readFileFromClasspath(inputFileName))); - if (j + 1 < numCopies) { - inputStreams.add(new ByteArrayInputStream(StringUtils.toUtf8("\n"))); - } - } - SequenceInputStream inputDataStream = new SequenceInputStream(Collections.enumeration(inputStreams)); - File segmentDir = tempFolder.newFolder(); - helper.createIndex( - inputDataStream, - parserJson, - transformSpecJson, - aggJson, - segmentDir, - 0, - granularity, - maxRowCount, - rollup - ); - inputDataStream.close(); - segmentDirs.add(segmentDir); + File file = selfConcatenateResourceFile(tempFolder, inputFile, numCopies); + inputFiles.add(new LocalInputSource(file.getParentFile(), file.getName())); } - - final List segments = Lists.transform( - segmentDirs, - dir -> { - try { - return closer.register(new QueryableIndexSegment(helper.getIndexIO().loadIndex(dir), SegmentId.dummy(""))); - } - catch (IOException ex) { - throw new RuntimeException(ex); - } - } - ); - - return segments; - } - - public static Segment createIncrementalIndex( - String inputFileName, - String parserJsonFileName, - String aggJsonFileName, - Granularity granularity, - boolean rollup, - boolean deserializeComplexMetrics, - int maxRowCount - ) - throws Exception - { - File inputFile = readFileFromClasspath(inputFileName); - FileInputStream inputDataStream = new FileInputStream(inputFile); - String parserJson = readFileFromClasspathAsString(parserJsonFileName); - String aggJson = readFileFromClasspathAsString(aggJsonFileName); - StringInputRowParser parser = JSON_MAPPER.readValue(parserJson, StringInputRowParser.class); - - LineIterator iter = IOUtils.lineIterator(inputDataStream, "UTF-8"); - List aggregatorSpecs = JSON_MAPPER.readValue( - aggJson, - new TypeReference>() - { - } - ); - IncrementalIndex index = AggregationTestHelper.createIncrementalIndex( - iter, - parser, - parser.getParseSpec().getDimensionsSpec().getDimensions(), - aggregatorSpecs.toArray(new AggregatorFactory[0]), - 0, + return createSegments( + tempFolder, + closer, + inputFiles, + JsonInputFormat.DEFAULT, + TIMESTAMP_SPEC, + AUTO_DISCOVERY, + TransformSpec.NONE, + COUNT, granularity, - deserializeComplexMetrics, - maxRowCount, rollup ); - inputDataStream.close(); - return new IncrementalIndexSegment(index, SegmentId.dummy("test_datasource")); } - public static Segment createDefaultHourlyIncrementalIndex() throws Exception - { - return createIncrementalIndex(Granularities.HOUR, true, true, 1000); - } - - public static Segment createDefaultDailyIncrementalIndex() throws Exception - { - return createIncrementalIndex(Granularities.DAY, true, true, 1000); - } - - public static List createDefaultHourlySegments( - AggregationTestHelper helper, + public static List createSegmentsForJsonInput( TemporaryFolder tempFolder, - Closer closer + Closer closer, + String inputFile ) throws Exception { - return createSegments( - helper, + return createSegmentsForJsonInput( tempFolder, closer, - Granularities.HOUR, - true, - 1000 + inputFile, + Granularities.NONE, + true ); } - public static List createDefaultHourlySegmentsTsv( - AggregationTestHelper helper, + public static Segment createIncrementalIndex( TemporaryFolder tempFolder, - Closer closer + String inputFileName, + InputFormat inputFormat, + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + TransformSpec transformSpec, + AggregatorFactory[] aggregators, + Granularity queryGranularity, + boolean rollup, + int maxRowCount ) throws Exception { - return createTsvSegments( - helper, - tempFolder, - closer, - Granularities.HOUR, - true, - 1000 - ); + IndexBuilder bob = IndexBuilder.create() + .schema( + IncrementalIndexSchema.builder() + .withTimestampSpec(timestampSpec) + .withDimensionsSpec(dimensionsSpec) + .withMetrics(aggregators) + .withQueryGranularity(queryGranularity) + .withRollup(rollup) + .withMinTimestamp(0) + .build() + ) + .inputSource( + ResourceInputSource.of( + NestedDataTestUtils.class.getClassLoader(), + inputFileName + ) + ) + .inputFormat(inputFormat) + .transform(transformSpec) + .inputTmpDir(tempFolder.newFolder()); + + return new IncrementalIndexSegment(bob.buildIncrementalIndex(), SegmentId.dummy("test_datasource")); } - public static List createDefaultDaySegments( - AggregationTestHelper helper, + public static List createSegments( TemporaryFolder tempFolder, - Closer closer - ) - throws Exception + Closer closer, + String input, + InputFormat inputFormat, + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + TransformSpec transformSpec, + AggregatorFactory[] aggregators, + Granularity queryGranularity, + boolean rollup + ) throws Exception { return createSegments( - helper, tempFolder, closer, - Granularities.DAY, - true, - 1000 + Collections.singletonList(ResourceInputSource.of(NestedDataTestUtils.class.getClassLoader(), input)), + inputFormat, + timestampSpec, + dimensionsSpec, + transformSpec, + aggregators, + queryGranularity, + rollup ); } - public static File readFileFromClasspath(String fileName) + public static List createSegments( + TemporaryFolder tempFolder, + Closer closer, + List inputs, + InputFormat inputFormat, + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + TransformSpec transformSpec, + AggregatorFactory[] aggregators, + Granularity queryGranularity, + boolean rollup + ) throws Exception { - return new File(NestedDataTestUtils.class.getClassLoader().getResource(fileName).getFile()); + final List segments = Lists.newArrayListWithCapacity(inputs.size()); + for (InputSource inputSource : inputs) { + final File segmentDir = tempFolder.newFolder(); + IndexBuilder bob = IndexBuilder.create() + .tmpDir(segmentDir) + .schema( + IncrementalIndexSchema.builder() + .withTimestampSpec(timestampSpec) + .withDimensionsSpec(dimensionsSpec) + .withMetrics(aggregators) + .withQueryGranularity(queryGranularity) + .withRollup(rollup) + .withMinTimestamp(0) + .build() + ) + .inputSource(inputSource) + .inputFormat(inputFormat) + .transform(transformSpec) + .inputTmpDir(tempFolder.newFolder()); + segments.add( + new QueryableIndexSegment( + closer.register(bob.buildMMappedIndex()), + SegmentId.dummy("test_datasource") + ) + ); + } + + return segments; } - public static String readFileFromClasspathAsString(String fileName) throws IOException + /** + * turn a small file into bigger file with a bunch of copies of itself + */ + public static File selfConcatenateResourceFile( + TemporaryFolder tempFolder, + String inputFileName, + int numCopies + ) throws IOException { - return Files.asCharSource(readFileFromClasspath(fileName), StandardCharsets.UTF_8).read(); + List inputStreams = Lists.newArrayListWithCapacity(numCopies); + for (int i = 0; i < numCopies; i++) { + InputStream stream = NestedDataTestUtils.class.getClassLoader().getResourceAsStream(inputFileName); + inputStreams.add(stream); + if (i + 1 < numCopies) { + inputStreams.add(new ByteArrayInputStream(StringUtils.toUtf8("\n"))); + } + } + File tmpFile = tempFolder.newFile(); + try ( + SequenceInputStream inputDataStream = new SequenceInputStream(Collections.enumeration(inputStreams)); + OutputStream outStream = Files.newOutputStream(tmpFile.toPath()) + ) { + final byte[] buffer = new byte[8096]; + int bytesRead; + while ((bytesRead = inputDataStream.read(buffer)) != -1) { + outStream.write(buffer, 0, bytesRead); + } + } + + return tmpFile; } - public static List>> getSegmentGenerators() + public static List>> getSegmentGenerators( + String jsonInputFile + ) { - final List>> segmentsGenerators = new ArrayList<>(); - segmentsGenerators.add(new TrinaryFn>() + final List>> segmentsGenerators = + new ArrayList<>(); + segmentsGenerators.add(new BiFunction>() { @Override - public List apply(AggregationTestHelper helper, TemporaryFolder tempFolder, Closer closer) + public List apply(TemporaryFolder tempFolder, Closer closer) { try { return ImmutableList.builder() - .addAll(NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer)) - .add(NestedDataTestUtils.createDefaultHourlyIncrementalIndex()) + .addAll( + NestedDataTestUtils.createSegmentsForJsonInput( + tempFolder, + closer, + jsonInputFile + ) + ) + .add(NestedDataTestUtils.createIncrementalIndexForJsonInput(tempFolder, jsonInputFile)) .build(); } catch (Exception e) { @@ -458,15 +470,15 @@ public String toString() return "mixed"; } }); - segmentsGenerators.add(new TrinaryFn>() + segmentsGenerators.add(new BiFunction>() { @Override - public List apply(AggregationTestHelper helper, TemporaryFolder tempFolder, Closer closer) + public List apply(TemporaryFolder tempFolder, Closer closer) { try { return ImmutableList.of( - NestedDataTestUtils.createDefaultHourlyIncrementalIndex(), - NestedDataTestUtils.createDefaultHourlyIncrementalIndex() + NestedDataTestUtils.createIncrementalIndexForJsonInput(tempFolder, jsonInputFile), + NestedDataTestUtils.createIncrementalIndexForJsonInput(tempFolder, jsonInputFile) ); } catch (Exception e) { @@ -480,15 +492,26 @@ public String toString() return "incremental"; } }); - segmentsGenerators.add(new TrinaryFn>() + segmentsGenerators.add(new BiFunction>() { @Override - public List apply(AggregationTestHelper helper, TemporaryFolder tempFolder, Closer closer) + public List apply(TemporaryFolder tempFolder, Closer closer) { try { return ImmutableList.builder() - .addAll(NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer)) - .addAll(NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer)) + .addAll( + NestedDataTestUtils.createSegmentsForJsonInput( + tempFolder, + closer, + jsonInputFile + ) + ) + .addAll(NestedDataTestUtils.createSegmentsForJsonInput( + tempFolder, + closer, + jsonInputFile + ) + ) .build(); } catch (Exception e) { diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java index 5cad59dffb61..65a59fc528e3 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java @@ -82,8 +82,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; -import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.segment.transform.TransformingStringInputRowParser; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; import org.apache.druid.utils.CloseableUtils; @@ -509,50 +507,6 @@ public void createIndex( } } - public void createIndex( - InputStream inputDataStream, - String parserJson, - String transformSpecJson, - String aggregators, - File outDir, - long minTimestamp, - Granularity gran, - int maxRowCount, - boolean rollup - ) throws Exception - { - try { - StringInputRowParser parser = mapper.readValue(parserJson, StringInputRowParser.class); - TransformSpec transformSpec; - if (transformSpecJson != null) { - transformSpec = mapper.readValue(transformSpecJson, TransformSpec.class); - parser = new TransformingStringInputRowParser(parser.getParseSpec(), parser.getEncoding(), transformSpec); - } - - LineIterator iter = IOUtils.lineIterator(inputDataStream, "UTF-8"); - List aggregatorSpecs = mapper.readValue( - aggregators, - new TypeReference>() - { - } - ); - - createIndex( - iter, - parser, - aggregatorSpecs.toArray(new AggregatorFactory[0]), - outDir, - minTimestamp, - gran, - true, - maxRowCount, - rollup - ); - } - finally { - Closeables.close(inputDataStream, true); - } - } public void createIndex( Iterator rows, diff --git a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java index 8f6183251022..ae621e7efda4 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java +++ b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java @@ -134,7 +134,7 @@ public void testJsonPathsExpression() expr = Parser.parse("json_paths(nester)", MACRO_TABLE); eval = expr.eval(inputBindings); Assert.assertEquals(ExpressionType.STRING_ARRAY, eval.type()); - Assert.assertArrayEquals(new Object[]{"$.x[0]", "$.y.a", "$.x[1]", "$.y.b", "$.x[2]"}, (Object[]) eval.value()); + Assert.assertArrayEquals(new Object[]{"$.x", "$.y.a", "$.y.b"}, (Object[]) eval.value()); } @Test diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java index 3038ccaedf58..030a40b29c3c 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java @@ -25,8 +25,6 @@ import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.nary.TrinaryFn; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.NestedDataTestUtils; @@ -60,6 +58,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Supplier; @RunWith(Parameterized.class) public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest @@ -73,12 +73,12 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest private final GroupByQueryConfig config; private final QueryContexts.Vectorize vectorize; private final AggregationTestHelper helper; - private final TrinaryFn> segmentsGenerator; + private final BiFunction> segmentsGenerator; private final String segmentsName; public NestedDataGroupByQueryTest( GroupByQueryConfig config, - TrinaryFn> segmentGenerator, + BiFunction> segmentGenerator, String vectorize ) { @@ -107,11 +107,11 @@ public Map getContext() public static Collection constructorFeeder() { final List constructors = new ArrayList<>(); - final List>> segmentsGenerators = - NestedDataTestUtils.getSegmentGenerators(); + final List>> segmentsGenerators = + NestedDataTestUtils.getSegmentGenerators(NestedDataTestUtils.SIMPLE_DATA_FILE); for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { - for (TrinaryFn> generatorFn : segmentsGenerators) { + for (BiFunction> generatorFn : segmentsGenerators) { for (String vectorize : new String[]{"false", "true", "force"}) { constructors.add(new Object[]{config, generatorFn, vectorize}); } @@ -444,8 +444,8 @@ public void testGroupBySomeFieldOnLongColumn() runResults( groupQuery, ImmutableList.of( - new Object[]{1609459200000L, NullHandling.defaultLongValue(), 8L}, - new Object[]{1609545600000L, NullHandling.defaultLongValue(), 8L} + new Object[]{1672531200000L, NullHandling.defaultLongValue(), 8L}, + new Object[]{1672617600000L, NullHandling.defaultLongValue(), 8L} ), false, true @@ -466,7 +466,7 @@ public void testGroupBySomeFieldOnLongColumnFilter() new NestedFieldVirtualColumn("__time", "$", "v0") ) .setAggregatorSpecs(new CountAggregatorFactory("count")) - .setDimFilter(new SelectorDimFilter("v0", "1609459200000", null)) + .setDimFilter(new SelectorDimFilter("v0", "1672531200000", null)) .setContext(getContext()) .build(); @@ -474,7 +474,7 @@ public void testGroupBySomeFieldOnLongColumnFilter() runResults( groupQuery, ImmutableList.of( - new Object[]{1609459200000L, 8L} + new Object[]{1672531200000L, 8L} ), false, true @@ -495,7 +495,7 @@ public void testGroupBySomeFieldOnLongColumnFilterExpectedType() new NestedFieldVirtualColumn("__time", "$", "v0", ColumnType.STRING) ) .setAggregatorSpecs(new CountAggregatorFactory("count")) - .setDimFilter(new SelectorDimFilter("v0", "1609459200000", null)) + .setDimFilter(new SelectorDimFilter("v0", "1672531200000", null)) .setContext(getContext()) .build(); @@ -503,7 +503,7 @@ public void testGroupBySomeFieldOnLongColumnFilterExpectedType() runResults( groupQuery, ImmutableList.of( - new Object[]{"1609459200000", 8L} + new Object[]{"1672531200000", 8L} ), true, false @@ -542,14 +542,18 @@ private void runResults(GroupByQuery groupQuery, List expectedResults) runResults(groupQuery, expectedResults, false, false); } - private void runResults(GroupByQuery groupQuery, List expectedResults, boolean hasUnknownCardinality, boolean hasNonStringOutput) + private void runResults( + GroupByQuery groupQuery, + List expectedResults, + boolean hasUnknownCardinality, + boolean hasNonStringOutput + ) { + Supplier> runner = + () -> helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(tempFolder, closer), groupQuery).toList(); if (GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) { if (hasUnknownCardinality) { - Throwable t = Assert.assertThrows( - RuntimeException.class, - () -> helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), groupQuery).toList() - ); + Throwable t = Assert.assertThrows(RuntimeException.class, runner::get); Assert.assertEquals( "java.lang.UnsupportedOperationException: GroupBy v1 does not support dimension selectors with unknown cardinality.", t.getMessage() @@ -557,10 +561,7 @@ private void runResults(GroupByQuery groupQuery, List expectedResults, return; } if (hasNonStringOutput) { - Throwable t = Assert.assertThrows( - RuntimeException.class, - () -> helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), groupQuery).toList() - ); + Throwable t = Assert.assertThrows(RuntimeException.class, runner::get); Assert.assertEquals( "java.lang.UnsupportedOperationException: GroupBy v1 only supports dimensions with an outputType of STRING.", t.getMessage() @@ -570,22 +571,14 @@ private void runResults(GroupByQuery groupQuery, List expectedResults, } if (!"segments".equals(segmentsName)) { if (GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) { - Throwable t = Assert.assertThrows( - RuntimeException.class, - () -> helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), groupQuery) - .toList() - ); + Throwable t = Assert.assertThrows(RuntimeException.class, runner::get); Assert.assertEquals( "java.lang.UnsupportedOperationException: GroupBy v1 does not support dimension selectors with unknown cardinality.", t.getMessage() ); return; } else if (vectorize == QueryContexts.Vectorize.FORCE) { - Throwable t = Assert.assertThrows( - RuntimeException.class, - () -> helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), groupQuery) - .toList() - ); + Throwable t = Assert.assertThrows(RuntimeException.class, runner::get); Assert.assertEquals( "java.util.concurrent.ExecutionException: java.lang.RuntimeException: org.apache.druid.java.util.common.ISE: Cannot vectorize!", t.getMessage() @@ -594,12 +587,7 @@ private void runResults(GroupByQuery groupQuery, List expectedResults, } } - Sequence seq = helper.runQueryOnSegmentsObjs( - segmentsGenerator.apply(helper, tempFolder, closer), - groupQuery - ); - - List results = seq.toList(); + List results = runner.get(); verifyResults( groupQuery.getResultRowSignature(), results, diff --git a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java index 268c19bf9177..22f2c53cb526 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.databind.Module; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -32,6 +34,8 @@ import org.apache.druid.query.NestedDataTestUtils; import org.apache.druid.query.Query; import org.apache.druid.query.aggregation.AggregationTestHelper; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.ordering.StringComparators; @@ -39,6 +43,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; +import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.virtual.NestedFieldVirtualColumn; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; @@ -97,7 +102,7 @@ public void testIngestAndScanSegments() throws Exception .limit(100) .context(ImmutableMap.of()) .build(); - List segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer); + List segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer); final Sequence seq = helper.runQueryOnSegmentsObjs(segs, scanQuery); @@ -126,15 +131,16 @@ public void testIngestAndScanSegmentsRollup() throws Exception .build(); List segs = ImmutableList.builder().addAll( NestedDataTestUtils.createSegments( - helper, tempFolder, closer, NestedDataTestUtils.NUMERIC_DATA_FILE, - NestedDataTestUtils.NUMERIC_PARSER_FILE, - NestedDataTestUtils.SIMPLE_AGG_FILE, + JsonInputFormat.DEFAULT, + NestedDataTestUtils.TIMESTAMP_SPEC, + NestedDataTestUtils.AUTO_DISCOVERY, + TransformSpec.NONE, + NestedDataTestUtils.COUNT, Granularities.YEAR, - true, - 1000 + true ) ).build(); @@ -171,9 +177,9 @@ public void testIngestAndScanSegmentsRealtime() throws Exception .context(ImmutableMap.of()) .build(); List realtimeSegs = ImmutableList.of( - NestedDataTestUtils.createDefaultHourlyIncrementalIndex() + NestedDataTestUtils.createSimpleNestedTestDataIncrementalIndex(tempFolder) ); - List segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer); + List segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer); final Sequence seq = helper.runQueryOnSegmentsObjs(realtimeSegs, scanQuery); @@ -242,9 +248,9 @@ public void testIngestAndScanSegmentsRealtimeWithFallback() throws Exception .context(ImmutableMap.of()) .build(); List realtimeSegs = ImmutableList.of( - NestedDataTestUtils.createDefaultHourlyIncrementalIndex() + NestedDataTestUtils.createSimpleNestedTestDataIncrementalIndex(tempFolder) ); - List segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer); + List segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer); final Sequence seq = helper.runQueryOnSegmentsObjs(realtimeSegs, scanQuery); @@ -278,7 +284,7 @@ public void testIngestAndScanSegmentsTsv() throws Exception .limit(100) .context(ImmutableMap.of()) .build(); - List segs = NestedDataTestUtils.createDefaultHourlySegmentsTsv(helper, tempFolder, closer); + List segs = NestedDataTestUtils.createSimpleSegmentsTsv(tempFolder, closer); final Sequence seq = helper.runQueryOnSegmentsObjs(segs, scanQuery); @@ -302,13 +308,12 @@ public void testIngestWithMergesAndScanSegments() throws Exception .limit(100) .context(ImmutableMap.of()) .build(); - List segs = NestedDataTestUtils.createSegments( - helper, + List segs = NestedDataTestUtils.createSegmentsForJsonInput( tempFolder, closer, + NestedDataTestUtils.SIMPLE_DATA_FILE, Granularities.HOUR, - true, - 3 + true ); final Sequence seq = helper.runQueryOnSegmentsObjs(segs, scanQuery); @@ -334,13 +339,12 @@ public void testIngestWithMoreMergesAndScanSegments() throws Exception .build(); - List segs = NestedDataTestUtils.createSegmentsWithConcatenatedInput( - helper, + List segs = NestedDataTestUtils.createSegmentsWithConcatenatedJsonInput( tempFolder, closer, + NestedDataTestUtils.SIMPLE_DATA_FILE, Granularities.HOUR, false, - 5, 10, 1 ); @@ -369,13 +373,12 @@ public void testIngestWithMoreMergesAndScanSegmentsRollup() throws Exception // same rows over and over so expect same 8 rows after rollup - List segs = NestedDataTestUtils.createSegmentsWithConcatenatedInput( - helper, + List segs = NestedDataTestUtils.createSegmentsWithConcatenatedJsonInput( tempFolder, closer, - Granularities.HOUR, + NestedDataTestUtils.SIMPLE_DATA_FILE, + Granularities.YEAR, true, - 5, 100, 1 ); @@ -407,7 +410,7 @@ public void testIngestAndScanSegmentsAndFilter() throws Exception .limit(100) .context(ImmutableMap.of()) .build(); - List segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer); + List segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer); final Sequence seq = helper.runQueryOnSegmentsObjs(segs, scanQuery); @@ -446,7 +449,7 @@ public void testIngestAndScanSegmentsAndRangeFilter() throws Exception .limit(100) .context(ImmutableMap.of()) .build(); - List segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer); + List segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer); final Sequence seq = helper.runQueryOnSegmentsObjs(segs, scanQuery); @@ -472,25 +475,89 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscovery() throws Exception .build(); List realtimeSegs = ImmutableList.of( NestedDataTestUtils.createIncrementalIndex( + tempFolder, NestedDataTestUtils.TYPES_DATA_FILE, - NestedDataTestUtils.TYPES_PARSER_FILE, - NestedDataTestUtils.SIMPLE_AGG_FILE, + JsonInputFormat.DEFAULT, + NestedDataTestUtils.TIMESTAMP_SPEC, + NestedDataTestUtils.AUTO_DISCOVERY, + TransformSpec.NONE, + NestedDataTestUtils.COUNT, Granularities.DAY, true, - false, 1000 ) ); List segs = NestedDataTestUtils.createSegments( - helper, tempFolder, closer, NestedDataTestUtils.TYPES_DATA_FILE, - NestedDataTestUtils.TYPES_PARSER_FILE, - NestedDataTestUtils.SIMPLE_AGG_FILE, + JsonInputFormat.DEFAULT, + NestedDataTestUtils.TIMESTAMP_SPEC, + NestedDataTestUtils.AUTO_DISCOVERY, + TransformSpec.NONE, + NestedDataTestUtils.COUNT, Granularities.DAY, - true, - 100 + true + ); + + + final Sequence seq = helper.runQueryOnSegmentsObjs(realtimeSegs, scanQuery); + final Sequence seq2 = helper.runQueryOnSegmentsObjs(segs, scanQuery); + + List resultsRealtime = seq.toList(); + List resultsSegments = seq2.toList(); + logResults(resultsSegments); + logResults(resultsRealtime); + Assert.assertEquals(1, resultsRealtime.size()); + Assert.assertEquals(resultsRealtime.size(), resultsSegments.size()); + Assert.assertEquals(resultsSegments.get(0).getEvents().toString(), resultsRealtime.get(0).getEvents().toString()); + } + + @Test + public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryArrayTypes() throws Exception + { + Druids.ScanQueryBuilder builder = Druids.newScanQueryBuilder() + .dataSource("test_datasource") + .intervals( + new MultipleIntervalSegmentSpec( + Collections.singletonList(Intervals.ETERNITY) + ) + ) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .limit(100) + .context(ImmutableMap.of()); + if (NullHandling.replaceWithDefault()) { + // null elements are replaced with default values if druid.generic.useDefaultValueForNull=true + // ... but not until after they are persisted, so realtime query results don't match of course... + builder.columns("arrayString", "arrayLong", "arrayDouble"); + } + Query scanQuery = builder.build(); + final AggregatorFactory[] aggs = new AggregatorFactory[]{new CountAggregatorFactory("count")}; + List realtimeSegs = ImmutableList.of( + NestedDataTestUtils.createIncrementalIndex( + tempFolder, + NestedDataTestUtils.ARRAY_TYPES_DATA_FILE, + JsonInputFormat.DEFAULT, + NestedDataTestUtils.TIMESTAMP_SPEC, + NestedDataTestUtils.AUTO_DISCOVERY, + TransformSpec.NONE, + aggs, + Granularities.NONE, + true, + 1000 + ) + ); + List segs = NestedDataTestUtils.createSegments( + tempFolder, + closer, + NestedDataTestUtils.ARRAY_TYPES_DATA_FILE, + JsonInputFormat.DEFAULT, + NestedDataTestUtils.TIMESTAMP_SPEC, + NestedDataTestUtils.AUTO_DISCOVERY, + TransformSpec.NONE, + aggs, + Granularities.NONE, + true ); diff --git a/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java index 58cbf46c5c0e..392df70e1d7c 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java @@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.nary.TrinaryFn; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.NestedDataTestUtils; @@ -55,6 +54,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.function.BiFunction; @RunWith(Parameterized.class) public class NestedDataTopNQueryTest extends InitializedNullHandlingTest @@ -65,11 +65,11 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest public final TemporaryFolder tempFolder = new TemporaryFolder(); private final AggregationTestHelper helper; - private final TrinaryFn> segmentsGenerator; + private final BiFunction> segmentsGenerator; private final Closer closer; public NestedDataTopNQueryTest( - TrinaryFn> segmentGenerator + BiFunction> segmentGenerator ) { NestedDataModule.registerHandlersAndSerde(); @@ -85,10 +85,10 @@ public NestedDataTopNQueryTest( public static Collection constructorFeeder() { final List constructors = new ArrayList<>(); - final List>> segmentsGenerators = - NestedDataTestUtils.getSegmentGenerators(); + final List>> segmentsGenerators = + NestedDataTestUtils.getSegmentGenerators(NestedDataTestUtils.SIMPLE_DATA_FILE); - for (TrinaryFn> generatorFn : segmentsGenerators) { + for (BiFunction> generatorFn : segmentsGenerators) { constructors.add(new Object[]{generatorFn}); } return constructors; @@ -114,7 +114,7 @@ public void testGroupBySomeField() .build(); - Sequence> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), topN); + Sequence> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(tempFolder, closer), topN); Sequence resultsSeq = new TopNQueryQueryToolChest(new TopNQueryConfig()).resultsAsArrays(topN, seq); @@ -123,10 +123,10 @@ public void testGroupBySomeField() verifyResults( results, ImmutableList.of( - new Object[]{1609459200000L, null, 8L}, - new Object[]{1609459200000L, "100", 2L}, - new Object[]{1609459200000L, "200", 2L}, - new Object[]{1609459200000L, "300", 4L} + new Object[]{1672531200000L, null, 8L}, + new Object[]{1672531200000L, "100", 2L}, + new Object[]{1672531200000L, "200", 2L}, + new Object[]{1672531200000L, "300", 4L} ) ); } @@ -148,7 +148,7 @@ public void testGroupBySomeFieldAggregateSomeField() .build(); - Sequence> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), topN); + Sequence> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(tempFolder, closer), topN); Sequence resultsSeq = new TopNQueryQueryToolChest(new TopNQueryConfig()).resultsAsArrays(topN, seq); @@ -157,10 +157,10 @@ public void testGroupBySomeFieldAggregateSomeField() verifyResults( results, ImmutableList.of( - new Object[]{1609459200000L, null, NullHandling.defaultDoubleValue()}, - new Object[]{1609459200000L, "100", 200.0}, - new Object[]{1609459200000L, "200", 400.0}, - new Object[]{1609459200000L, "300", 1200.0} + new Object[]{1672531200000L, null, NullHandling.defaultDoubleValue()}, + new Object[]{1672531200000L, "100", 200.0}, + new Object[]{1672531200000L, "200", 400.0}, + new Object[]{1672531200000L, "300", 1200.0} ) ); } 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 f0d9b1fc328a..c2871172454e 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -22,12 +22,18 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.segment.FrameSegment; import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.column.ColumnConfig; @@ -36,10 +42,13 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; +import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -56,18 +65,27 @@ public class IndexBuilder private static final int ROWS_PER_INDEX_FOR_MERGING = 1; private static final int DEFAULT_MAX_ROWS = Integer.MAX_VALUE; - private IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() - .withMetrics(new CountAggregatorFactory("count")) - .build(); + private final ObjectMapper jsonMapper; + private final IndexIO indexIO; + private final List rows = new ArrayList<>(); + private SegmentWriteOutMediumFactory segmentWriteOutMediumFactory = OffHeapMemorySegmentWriteOutMediumFactory.instance(); private IndexMerger indexMerger; private File tmpDir; private IndexSpec indexSpec = new IndexSpec(); private int maxRows = DEFAULT_MAX_ROWS; - - private final ObjectMapper jsonMapper; - private final IndexIO indexIO; - private final List rows = new ArrayList<>(); + private int intermediatePersistSize = ROWS_PER_INDEX_FOR_MERGING; + private IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() + .withMetrics(new CountAggregatorFactory("count")) + .build(); + @Nullable + private InputSource inputSource = null; + @Nullable + private InputFormat inputFormat = null; + @Nullable + private TransformSpec transformSpec = null; + @Nullable + private File inputSourceTmpDir = null; private IndexBuilder(ObjectMapper jsonMapper, ColumnConfig columnConfig) { @@ -121,6 +139,50 @@ public IndexBuilder tmpDir(File tmpDir) return this; } + public IndexBuilder inputSource(InputSource inputSource) + { + this.inputSource = inputSource; + return this; + } + + public IndexBuilder inputFormat(InputFormat inputFormat) + { + this.inputFormat = inputFormat; + return this; + } + + public IndexBuilder transform(TransformSpec transformSpec) + { + this.transformSpec = transformSpec; + return this; + } + + public IndexBuilder inputTmpDir(File inputSourceTmpDir) + { + this.inputSourceTmpDir = inputSourceTmpDir; + return this; + } + + public IndexBuilder rows( + InputSource inputSource, + InputFormat inputFormat, + InputRowSchema rowSchema, + TransformSpec transformSpec, + File tmp + ) + throws IOException + { + rows.clear(); + InputSourceReader reader = inputSource.reader(rowSchema, inputFormat, tmp); + InputSourceReader transformingReader = transformSpec.decorate(reader); + try (CloseableIterator rowIterator = transformingReader.read()) { + while (rowIterator.hasNext()) { + rows.add(rowIterator.next()); + } + } + return this; + } + public IndexBuilder rows(Iterable rows) { this.rows.clear(); @@ -128,8 +190,30 @@ public IndexBuilder rows(Iterable rows) return this; } + public IndexBuilder maxRows(int maxRows) + { + this.maxRows = maxRows; + return this; + } + + public IndexBuilder intermediaryPersistSize(int rows) + { + this.intermediatePersistSize = rows; + return this; + } + public IncrementalIndex buildIncrementalIndex() { + if (inputSource != null) { + return buildIncrementalIndexWithInputSource( + schema, + inputSource, + inputFormat, + transformSpec, + inputSourceTmpDir, + maxRows + ); + } return buildIncrementalIndexWithRows(schema, maxRows, rows); } @@ -157,20 +241,63 @@ public QueryableIndex buildMMappedIndex() public QueryableIndex buildMMappedMergedIndex() { - IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); Preconditions.checkNotNull(tmpDir, "tmpDir"); - final List persisted = new ArrayList<>(); - try { - for (int i = 0; i < rows.size(); i += ROWS_PER_INDEX_FOR_MERGING) { + if (inputSource != null) { + Preconditions.checkNotNull(inputSource, "inputSource"); + Preconditions.checkNotNull(inputFormat, "inputFormat"); + Preconditions.checkNotNull(inputSourceTmpDir, "inputSourceTmpDir"); + + TransformSpec tranformer = transformSpec != null ? transformSpec : TransformSpec.NONE; + InputRowSchema rowSchema = new InputRowSchema(schema.getTimestampSpec(), schema.getDimensionsSpec(), null); + InputSourceReader reader = inputSource.reader(rowSchema, inputFormat, inputSourceTmpDir); + InputSourceReader transformingReader = tranformer.decorate(reader); + return mergeIndexes(indexMerger, persisted, transformingReader::read); + } + + return mergeIndexes(indexMerger, persisted, () -> CloseableIterators.withEmptyBaggage(rows.iterator())); + } + + @Nonnull + private QueryableIndex mergeIndexes( + IndexMerger indexMerger, + List persisted, + IteratorSupplier iteratorSupplier + ) + { + try (CloseableIterator rowIterator = iteratorSupplier.get()) { + int i = 0; + IncrementalIndex incrementalIndex = new OnheapIncrementalIndex.Builder() + .setIndexSchema(schema) + .setMaxRowCount(maxRows) + .build(); + while (rowIterator.hasNext()) { + if (i < intermediatePersistSize) { + incrementalIndex.add(rowIterator.next()); + i++; + } else { + persisted.add( + TestHelper.getTestIndexIO().loadIndex( + indexMerger.persist( + incrementalIndex, + new File(tmpDir, StringUtils.format("testIndex-%s", UUID.randomUUID().toString())), + indexSpec, + null + ) + ) + ); + incrementalIndex = new OnheapIncrementalIndex.Builder() + .setIndexSchema(schema) + .setMaxRowCount(maxRows) + .build(); + i = 0; + } + } + if (i != 0) { persisted.add( TestHelper.getTestIndexIO().loadIndex( indexMerger.persist( - buildIncrementalIndexWithRows( - schema, - maxRows, - rows.subList(i, Math.min(rows.size(), i + ROWS_PER_INDEX_FOR_MERGING)) - ), + incrementalIndex, new File(tmpDir, StringUtils.format("testIndex-%s", UUID.randomUUID().toString())), indexSpec, null @@ -178,6 +305,7 @@ public QueryableIndex buildMMappedMergedIndex() ) ); } + final QueryableIndex merged = TestHelper.getTestIndexIO().loadIndex( indexMerger.mergeQueryableIndex( persisted, @@ -267,4 +395,44 @@ private static IncrementalIndex buildIncrementalIndexWithRows( } return incrementalIndex; } + + private static IncrementalIndex buildIncrementalIndexWithInputSource( + IncrementalIndexSchema schema, + InputSource inputSource, + InputFormat inputFormat, + @Nullable TransformSpec transformSpec, + File inputSourceTmpDir, + int maxRows + ) + { + Preconditions.checkNotNull(schema, "schema"); + Preconditions.checkNotNull(inputSource, "inputSource"); + Preconditions.checkNotNull(inputFormat, "inputFormat"); + Preconditions.checkNotNull(inputSourceTmpDir, "inputSourceTmpDir"); + + final IncrementalIndex incrementalIndex = new OnheapIncrementalIndex.Builder() + .setIndexSchema(schema) + .setMaxRowCount(maxRows) + .build(); + TransformSpec tranformer = transformSpec != null ? transformSpec : TransformSpec.NONE; + InputRowSchema rowSchema = new InputRowSchema(schema.getTimestampSpec(), schema.getDimensionsSpec(), null); + InputSourceReader reader = inputSource.reader(rowSchema, inputFormat, inputSourceTmpDir); + InputSourceReader transformingReader = tranformer.decorate(reader); + try (CloseableIterator rowIterator = transformingReader.read()) { + while (rowIterator.hasNext()) { + incrementalIndex.add(rowIterator.next()); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + return incrementalIndex; + } + + + @FunctionalInterface + interface IteratorSupplier + { + CloseableIterator get() throws IOException; + } } diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java index e77cffbc30e3..a7f33f70fecf 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java @@ -94,34 +94,34 @@ public void testKeySizeEstimation() // new raw value, new fields key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false); - Assert.assertEquals(276, key.getEffectiveSizeBytes()); - Assert.assertEquals(5, indexer.getCardinality()); + Assert.assertEquals(168, key.getEffectiveSizeBytes()); + Assert.assertEquals(6, indexer.getCardinality()); // new raw value, re-use fields and dictionary key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false); - Assert.assertEquals(56, key.getEffectiveSizeBytes()); - Assert.assertEquals(5, indexer.getCardinality()); + Assert.assertEquals(104, key.getEffectiveSizeBytes()); + Assert.assertEquals(6, indexer.getCardinality()); // new raw value, new fields key = indexer.processRowValsToUnsortedEncodedKeyComponent( ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)), false ); - Assert.assertEquals(286, key.getEffectiveSizeBytes()); - Assert.assertEquals(5, indexer.getCardinality()); + Assert.assertEquals(166, key.getEffectiveSizeBytes()); + Assert.assertEquals(6, indexer.getCardinality()); // new raw value key = indexer.processRowValsToUnsortedEncodedKeyComponent( ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)), false ); - Assert.assertEquals(118, key.getEffectiveSizeBytes()); - Assert.assertEquals(5, indexer.getCardinality()); + Assert.assertEquals(166, key.getEffectiveSizeBytes()); + Assert.assertEquals(6, indexer.getCardinality()); key = indexer.processRowValsToUnsortedEncodedKeyComponent("", false); if (NullHandling.replaceWithDefault()) { Assert.assertEquals(0, key.getEffectiveSizeBytes()); - Assert.assertEquals(6, indexer.getCardinality()); + Assert.assertEquals(7, indexer.getCardinality()); } else { Assert.assertEquals(104, key.getEffectiveSizeBytes()); - Assert.assertEquals(6, indexer.getCardinality()); + Assert.assertEquals(7, indexer.getCardinality()); } } @@ -364,7 +364,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index UnsupportedOperationException.class, () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) ); - Assert.assertEquals(StructuredData.wrap(new Object[]{"a"}), valueSelector.getObject()); + Assert.assertArrayEquals(new Object[]{"a"}, (Object[]) valueSelector.getObject()); columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); @@ -372,8 +372,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index UnsupportedOperationException.class, () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) ); - Assert.assertEquals(StructuredData.wrap(new Object[]{"b", "c"}), valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); + Assert.assertArrayEquals(new Object[]{"b", "c"}, (Object[]) valueSelector.getObject()); columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); @@ -381,9 +380,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index UnsupportedOperationException.class, () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) ); - // raw data is left as is, so is currently still a list while in incremental index... - Assert.assertEquals(StructuredData.wrap(ImmutableList.of("d", "e")), valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); + Assert.assertArrayEquals(new Object[]{"d", "e"}, (Object[]) valueSelector.getObject()); columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); diff --git a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java index c9e561353657..e1b756f0a44b 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java @@ -293,6 +293,32 @@ public void testFrontCodedOnlyNull() throws IOException Assert.assertFalse(utf8Iterator.hasNext()); } + @Test + public void testFrontCodedEmpty() throws IOException + { + ByteBuffer buffer = ByteBuffer.allocate(1 << 6).order(order); + List theList = Collections.emptyList(); + fillBuffer(buffer, theList, 4); + + buffer.position(0); + FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( + buffer, + buffer.order() + ).get(); + + Assert.assertEquals(0, codedUtf8Indexed.size()); + Throwable t = Assert.assertThrows(IAE.class, () -> codedUtf8Indexed.get(0)); + Assert.assertEquals("Index[0] >= size[0]", t.getMessage()); + Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(-1)); + Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(theList.size())); + + Assert.assertEquals(-1, codedUtf8Indexed.indexOf(null)); + Assert.assertEquals(-1, codedUtf8Indexed.indexOf(StringUtils.toUtf8ByteBuffer("hello"))); + + Iterator utf8Iterator = codedUtf8Indexed.iterator(); + Assert.assertFalse(utf8Iterator.hasNext()); + } + @Test public void testBucketSizes() throws IOException { diff --git a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java new file mode 100644 index 000000000000..abc9a2ee60d0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java @@ -0,0 +1,455 @@ +/* + * 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.data; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.TreeSet; +import java.util.concurrent.ThreadLocalRandom; + +@RunWith(Parameterized.class) +public class FrontCodedIntArrayIndexedTest +{ + @Parameterized.Parameters(name = "{0}") + public static Collection constructorFeeder() + { + return ImmutableList.of(new Object[]{ByteOrder.LITTLE_ENDIAN}, new Object[]{ByteOrder.BIG_ENDIAN}); + } + + private final ByteOrder order; + + public FrontCodedIntArrayIndexedTest(ByteOrder byteOrder) + { + this.order = byteOrder; + } + + @Test + public void testFrontCodedIntArrayIndexed() throws IOException + { + ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order); + TreeSet values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR); + values.add(new int[]{1, 2, 3}); + values.add(new int[]{1, 2}); + values.add(new int[]{1, 3}); + values.add(new int[]{1, 2, 4}); + values.add(new int[]{1, 3, 4}); + values.add(new int[]{1, 2, 1}); + + fillBuffer(buffer, values, 4); + + buffer.position(0); + FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( + buffer, + buffer.order() + ).get(); + + Iterator indexedIterator = codedIndexed.iterator(); + Iterator expectedIterator = values.iterator(); + int ctr = 0; + while (expectedIterator.hasNext() && indexedIterator.hasNext()) { + final int[] expectedNext = expectedIterator.next(); + final int[] next = indexedIterator.next(); + Assert.assertArrayEquals(expectedNext, next); + Assert.assertEquals(ctr, codedIndexed.indexOf(next)); + ctr++; + } + Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext()); + } + + + @Test + public void testFrontCodedIntArrayIndexedSingleBucket() throws IOException + { + ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order); + TreeSet values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR); + values.add(new int[]{1, 2, 3}); + values.add(new int[]{1, 2}); + values.add(new int[]{1, 3}); + values.add(new int[]{1, 2, 4}); + values.add(new int[]{1, 3, 4}); + values.add(new int[]{1, 2, 1}); + fillBuffer(buffer, values, 16); + + FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( + buffer, + buffer.order() + ).get(); + + Iterator expectedIterator = values.iterator(); + Iterator indexedIterator = codedIndexed.iterator(); + int ctr = 0; + while (indexedIterator.hasNext() && expectedIterator.hasNext()) { + final int[] expectedNext = expectedIterator.next(); + final int[] next = indexedIterator.next(); + Assert.assertArrayEquals(expectedNext, next); + Assert.assertEquals(ctr, codedIndexed.indexOf(next)); + ctr++; + } + Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext()); + } + + @Test + public void testFrontCodedIntArrayIndexedBigger() throws IOException + { + final int sizeBase = 10000; + final int bucketSize = 16; + final ByteBuffer buffer = ByteBuffer.allocate(1 << 24).order(order); + for (int sizeAdjust = 0; sizeAdjust < bucketSize; sizeAdjust++) { + final TreeSet values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR); + while (values.size() < sizeBase + sizeAdjust) { + int length = ThreadLocalRandom.current().nextInt(10); + final int[] val = new int[length]; + for (int j = 0; j < length; j++) { + val[j] = ThreadLocalRandom.current().nextInt(0, 10_000); + } + values.add(val); + } + fillBuffer(buffer, values, bucketSize); + + FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( + buffer, + buffer.order() + ).get(); + + Iterator expectedIterator = values.iterator(); + Iterator indexedIterator = codedIndexed.iterator(); + int ctr = 0; + while (indexedIterator.hasNext() && expectedIterator.hasNext()) { + final int[] expectedNext = expectedIterator.next(); + final int[] next = indexedIterator.next(); + Assert.assertArrayEquals(expectedNext, next); + Assert.assertEquals(ctr, codedIndexed.indexOf(next)); + ctr++; + } + Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext()); + Assert.assertEquals(ctr, sizeBase + sizeAdjust); + } + } + + @Test + public void testFrontCodedIntArrayIndexedBiggerWithNulls() throws IOException + { + final int sizeBase = 10000; + final int bucketSize = 16; + final ByteBuffer buffer = ByteBuffer.allocate(1 << 25).order(order); + for (int sizeAdjust = 0; sizeAdjust < bucketSize; sizeAdjust++) { + TreeSet values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR); + values.add(null); + while (values.size() < sizeBase + sizeAdjust + 1) { + int length = ThreadLocalRandom.current().nextInt(10); + final int[] val = new int[length]; + for (int j = 0; j < length; j++) { + val[j] = ThreadLocalRandom.current().nextInt(0, 10_000); + } + values.add(val); + } + fillBuffer(buffer, values, bucketSize); + + FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( + buffer, + buffer.order() + ).get(); + + Iterator expectedIterator = values.iterator(); + Iterator indexedIterator = codedIndexed.iterator(); + int ctr = 0; + while (indexedIterator.hasNext() && expectedIterator.hasNext()) { + final int[] expectedNext = expectedIterator.next(); + final int[] next = indexedIterator.next(); + Assert.assertArrayEquals(expectedNext, next); + Assert.assertEquals(ctr, codedIndexed.indexOf(next)); + ctr++; + } + Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext()); + Assert.assertEquals(ctr, sizeBase + sizeAdjust + 1); + } + } + + @Test + public void testFrontCodedIntArrayIndexedIndexOf() throws IOException + { + ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order); + TreeSet values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR); + values.add(new int[]{1, 2}); + values.add(new int[]{1, 2, 1}); + values.add(new int[]{1, 2, 3}); + values.add(new int[]{1, 2, 4}); + values.add(new int[]{1, 3}); + values.add(new int[]{1, 3, 4}); + + fillBuffer(buffer, values, 4); + + FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( + buffer, + buffer.order() + ).get(); + Assert.assertEquals(-1, codedIndexed.indexOf(new int[]{1})); + Assert.assertEquals(0, codedIndexed.indexOf(new int[]{1, 2})); + Assert.assertEquals(1, codedIndexed.indexOf(new int[]{1, 2, 1})); + Assert.assertEquals(-3, codedIndexed.indexOf(new int[]{1, 2, 2})); + Assert.assertEquals(4, codedIndexed.indexOf(new int[]{1, 3})); + Assert.assertEquals(-7, codedIndexed.indexOf(new int[]{1, 4, 4})); + Assert.assertEquals(-7, codedIndexed.indexOf(new int[]{9, 1, 1})); + } + + + @Test + public void testFrontCodedIntArrayIndexedIndexOfWithNull() throws IOException + { + ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order); + TreeSet values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR); + values.add(null); + values.add(new int[]{1, 2}); + values.add(new int[]{1, 2, 1}); + values.add(new int[]{1, 2, 3}); + values.add(new int[]{1, 2, 4}); + values.add(new int[]{1, 3}); + values.add(new int[]{1, 3, 4}); + fillBuffer(buffer, values, 4); + + FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( + buffer, + buffer.order() + ).get(); + Assert.assertEquals(0, codedIndexed.indexOf(null)); + Assert.assertEquals(-2, codedIndexed.indexOf(new int[]{1})); + Assert.assertEquals(1, codedIndexed.indexOf(new int[]{1, 2})); + Assert.assertEquals(2, codedIndexed.indexOf(new int[]{1, 2, 1})); + Assert.assertEquals(-4, codedIndexed.indexOf(new int[]{1, 2, 2})); + Assert.assertEquals(5, codedIndexed.indexOf(new int[]{1, 3})); + Assert.assertEquals(-8, codedIndexed.indexOf(new int[]{1, 4, 4})); + Assert.assertEquals(-8, codedIndexed.indexOf(new int[]{9, 1, 1})); + } + + + @Test + public void testFrontCodedOnlyNull() throws IOException + { + ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order); + List theList = Collections.singletonList(null); + fillBuffer(buffer, theList, 4); + + buffer.position(0); + FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( + buffer, + buffer.order() + ).get(); + + Assert.assertNull(codedIndexed.get(0)); + Assert.assertThrows(IllegalArgumentException.class, () -> codedIndexed.get(-1)); + Assert.assertThrows(IllegalArgumentException.class, () -> codedIndexed.get(theList.size())); + + Assert.assertEquals(0, codedIndexed.indexOf(null)); + Assert.assertEquals(-2, codedIndexed.indexOf(new int[]{1, 2, 3, 4})); + + Iterator iterator = codedIndexed.iterator(); + Assert.assertTrue(iterator.hasNext()); + Assert.assertNull(iterator.next()); + Assert.assertFalse(iterator.hasNext()); + } + + @Test + public void testFrontCodedEmpty() throws IOException + { + ByteBuffer buffer = ByteBuffer.allocate(1 << 6).order(order); + List theList = Collections.emptyList(); + fillBuffer(buffer, theList, 4); + + buffer.position(0); + FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( + buffer, + buffer.order() + ).get(); + + Assert.assertEquals(0, codedUtf8Indexed.size()); + Throwable t = Assert.assertThrows(IAE.class, () -> codedUtf8Indexed.get(0)); + Assert.assertEquals("Index[0] >= size[0]", t.getMessage()); + Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(-1)); + Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(theList.size())); + + Assert.assertEquals(-1, codedUtf8Indexed.indexOf(null)); + Assert.assertEquals(-1, codedUtf8Indexed.indexOf(StringUtils.toUtf8ByteBuffer("hello"))); + + Iterator utf8Iterator = codedUtf8Indexed.iterator(); + Assert.assertFalse(utf8Iterator.hasNext()); + } + + @Test + public void testBucketSizes() throws IOException + { + final int numValues = 10000; + final ByteBuffer buffer = ByteBuffer.allocate(1 << 25).order(order); + final int[] bucketSizes = new int[]{ + 1, + 1 << 1, + 1 << 2, + 1 << 3, + 1 << 4, + 1 << 5, + 1 << 6, + 1 << 7 + }; + + TreeSet values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR); + values.add(null); + while (values.size() < numValues + 1) { + int length = ThreadLocalRandom.current().nextInt(10); + final int[] val = new int[length]; + for (int j = 0; j < length; j++) { + val[j] = ThreadLocalRandom.current().nextInt(0, 10_000); + } + values.add(val); + } + for (int bucketSize : bucketSizes) { + fillBuffer(buffer, values, bucketSize); + FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( + buffer, + buffer.order() + ).get(); + + Iterator expectedIterator = values.iterator(); + Iterator iterator = codedIndexed.iterator(); + int ctr = 0; + while (iterator.hasNext() && expectedIterator.hasNext()) { + final int[] expectedNext = expectedIterator.next(); + final int[] next = iterator.next(); + Assert.assertArrayEquals(expectedNext, next); + + Assert.assertEquals(ctr, codedIndexed.indexOf(next)); + ctr++; + } + Assert.assertEquals(expectedIterator.hasNext(), iterator.hasNext()); + Assert.assertEquals(ctr, numValues + 1); + } + } + + @Test + public void testBadBucketSize() + { + OnHeapMemorySegmentWriteOutMedium medium = new OnHeapMemorySegmentWriteOutMedium(); + + Assert.assertThrows( + IAE.class, + () -> new FrontCodedIntArrayIndexedWriter( + medium, + ByteOrder.nativeOrder(), + 0 + ) + ); + + Assert.assertThrows( + IAE.class, + () -> new FrontCodedIntArrayIndexedWriter( + medium, + ByteOrder.nativeOrder(), + 15 + ) + ); + + Assert.assertThrows( + IAE.class, + () -> new FrontCodedIntArrayIndexedWriter( + medium, + ByteOrder.nativeOrder(), + 256 + ) + ); + } + + private static long fillBuffer(ByteBuffer buffer, Iterable sortedIterable, int bucketSize) throws IOException + { + Iterator sortedInts = sortedIterable.iterator(); + buffer.position(0); + OnHeapMemorySegmentWriteOutMedium medium = new OnHeapMemorySegmentWriteOutMedium(); + FrontCodedIntArrayIndexedWriter writer = new FrontCodedIntArrayIndexedWriter( + medium, + buffer.order(), + bucketSize + ); + writer.open(); + int index = 0; + while (sortedInts.hasNext()) { + final int[] next = sortedInts.next(); + writer.write(next); + if (next == null) { + Assert.assertNull(writer.get(index)); + } else { + Assert.assertArrayEquals(next, writer.get(index)); + } + index++; + } + + // check 'get' again so that we aren't always reading from current page + index = 0; + sortedInts = sortedIterable.iterator(); + while (sortedInts.hasNext()) { + final int[] next = sortedInts.next(); + if (next == null) { + Assert.assertNull("row " + index, writer.get(index)); + } else { + Assert.assertArrayEquals("row " + index, next, writer.get(index)); + } + index++; + } + + WritableByteChannel channel = new WritableByteChannel() + { + @Override + public int write(ByteBuffer src) + { + int size = src.remaining(); + buffer.put(src); + return size; + } + + @Override + public boolean isOpen() + { + return true; + } + + @Override + public void close() + { + } + }; + long size = writer.getSerializedSize(); + buffer.position(0); + writer.writeTo(channel, null); + Assert.assertEquals(size, buffer.position()); + buffer.position(0); + return size; + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index 38764b093af4..3653dfaf2d98 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -26,7 +26,9 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.collections.bitmap.RoaringBitmapFactory; +import org.apache.druid.collections.bitmap.WrappedRoaringBitmap; import org.apache.druid.common.config.NullHandling; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.concurrent.Execs; @@ -42,6 +44,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.NestedDataColumnIndexer; +import org.apache.druid.segment.NestedDataColumnMerger; import org.apache.druid.segment.ObjectColumnSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.SimpleAscendingOffset; @@ -55,6 +58,11 @@ import org.apache.druid.segment.column.NullValueIndex; import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.column.TypeStrategy; +import org.apache.druid.segment.vector.BitmapVectorOffset; +import org.apache.druid.segment.vector.NoFilterVectorOffset; +import org.apache.druid.segment.vector.VectorObjectSelector; +import org.apache.druid.segment.vector.VectorValueSelector; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.utils.CompressionUtils; @@ -102,12 +110,39 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest TestHelper.makeMap("x", 4L, "y", 2.0, "z", "e", "v", 11111L, "nullish", null) ); + List> arrayTestData = ImmutableList.of( + TestHelper.makeMap("s", new Object[]{"a", "b", "c"}, "l", new Object[]{1L, 2L, 3L}, "d", new Object[]{1.1, 2.2}), + TestHelper.makeMap( + "s", + new Object[]{null, "b", "c"}, + "l", + new Object[]{1L, NullHandling.defaultLongValue(), 3L}, + "d", + new Object[]{2.2, 2.2} + ), + TestHelper.makeMap( + "s", + new Object[]{"b", "c"}, + "l", + new Object[]{NullHandling.defaultLongValue(), NullHandling.defaultLongValue()}, + "d", + new Object[]{1.1, NullHandling.defaultDoubleValue(), 2.2} + ), + TestHelper.makeMap("s", new Object[]{"a", "b", "c", "d"}, "l", new Object[]{4L, 2L, 3L}), + TestHelper.makeMap("s", new Object[]{"d", "b", "c", "a"}, "d", new Object[]{1.1, 2.2}), + TestHelper.makeMap("l", new Object[]{1L, 2L, 3L}, "d", new Object[]{3.1, 2.2, 1.9}) + ); + Closer closer = Closer.create(); SmooshedFileMapper fileMapper; ByteBuffer baseBuffer; + SmooshedFileMapper arrayFileMapper; + + ByteBuffer arrayBaseBuffer; + @BeforeClass public static void staticSetup() { @@ -118,11 +153,22 @@ public static void staticSetup() public void setup() throws IOException { final String fileNameBase = "test"; - TmpFileSegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance(); - final File tmpFile = tempFolder.newFolder(); - try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { - + final String arrayFileNameBase = "array"; + fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), data); + baseBuffer = fileMapper.mapFile(fileNameBase); + arrayFileMapper = smooshify(arrayFileNameBase, tempFolder.newFolder(), arrayTestData); + arrayBaseBuffer = arrayFileMapper.mapFile(arrayFileNameBase); + } + private SmooshedFileMapper smooshify( + String fileNameBase, + File tmpFile, + List> data + ) + throws IOException + { + SegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance(); + try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { NestedDataColumnSerializer serializer = new NestedDataColumnSerializer( fileNameBase, new IndexSpec(), @@ -145,6 +191,12 @@ public void setup() throws IOException serializer.serializeStringDictionary(globalDictionarySortedCollector.getSortedStrings()); serializer.serializeLongDictionary(globalDictionarySortedCollector.getSortedLongs()); serializer.serializeDoubleDictionary(globalDictionarySortedCollector.getSortedDoubles()); + serializer.serializeArrayDictionary( + () -> new NestedDataColumnMerger.ArrayDictionaryMergingIterator( + new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, + serializer.getGlobalLookup() + ) + ); SettableSelector valueSelector = new SettableSelector(); for (Object o : data) { @@ -156,8 +208,7 @@ public void setup() throws IOException serializer.writeTo(writer, smoosher); } smoosher.close(); - fileMapper = closer.register(SmooshedFileMapper.load(tmpFile)); - baseBuffer = fileMapper.mapFile(fileNameBase); + return closer.register(SmooshedFileMapper.load(tmpFile)); } } @@ -185,6 +236,24 @@ public void testBasicFunctionality() throws IOException } } + @Test + public void testArrayFunctionality() throws IOException + { + ColumnBuilder bob = new ColumnBuilder(); + bob.setFileMapper(arrayFileMapper); + NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read( + arrayBaseBuffer, + bob, + () -> 0, + NestedDataComplexTypeSerde.OBJECT_MAPPER, + new OnlyPositionalReadsTypeStrategy<>(ColumnType.LONG.getStrategy()), + new OnlyPositionalReadsTypeStrategy<>(ColumnType.DOUBLE.getStrategy()) + ); + try (NestedDataComplexColumn column = (NestedDataComplexColumn) supplier.get()) { + smokeTestArrays(column); + } + } + @Test public void testConcurrency() throws ExecutionException, InterruptedException { @@ -310,7 +379,10 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException NullValueIndex zNulls = zIndexSupplier.as(NullValueIndex.class); final List vPath = NestedPathFinder.parseJsonPath("$.v"); - Assert.assertEquals(ImmutableSet.of(ColumnType.STRING, ColumnType.LONG, ColumnType.DOUBLE), column.getColumnTypes(vPath)); + Assert.assertEquals( + ImmutableSet.of(ColumnType.STRING, ColumnType.LONG, ColumnType.DOUBLE), + column.getColumnTypes(vPath) + ); Assert.assertEquals(ColumnType.STRING, column.getColumnHolder(vPath).getCapabilities().toColumnType()); ColumnValueSelector vSelector = column.makeColumnValueSelector(vPath, offset); DimensionSelector vDimSelector = column.makeDimensionSelector(vPath, offset, null); @@ -344,9 +416,262 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException testPath(row, i, "x", xSelector, xDimSelector, xValueIndex, xPredicateIndex, xNulls, ColumnType.LONG); testPath(row, i, "y", ySelector, yDimSelector, yValueIndex, yPredicateIndex, yNulls, ColumnType.DOUBLE); testPath(row, i, "z", zSelector, zDimSelector, zValueIndex, zPredicateIndex, zNulls, ColumnType.STRING); - testPath(row, i, "nullish", nullishSelector, nullishDimSelector, nullishValueIndex, nullishPredicateIndex, nullishNulls, ColumnType.STRING); + testPath( + row, + i, + "nullish", + nullishSelector, + nullishDimSelector, + nullishValueIndex, + nullishPredicateIndex, + nullishNulls, + ColumnType.STRING + ); + + offset.increment(); + } + } + + private void smokeTestArrays(NestedDataComplexColumn column) throws IOException + { + SimpleAscendingOffset offset = new SimpleAscendingOffset(arrayTestData.size()); + NoFilterVectorOffset vectorOffset = new NoFilterVectorOffset(4, 0, arrayTestData.size()); + WrappedRoaringBitmap bitmap = new WrappedRoaringBitmap(); + for (int i = 0; i < arrayTestData.size(); i++) { + if (i % 2 == 0) { + bitmap.add(i); + } + } + BitmapVectorOffset bitmapVectorOffset = new BitmapVectorOffset( + 4, + bitmap.toImmutableBitmap(), + 0, + arrayTestData.size() + ); + + ColumnValueSelector rawSelector = column.makeColumnValueSelector(offset); + VectorObjectSelector rawVectorSelector = column.makeVectorObjectSelector(vectorOffset); + VectorObjectSelector rawVectorSelectorFiltered = column.makeVectorObjectSelector(bitmapVectorOffset); + + final List sPath = NestedPathFinder.parseJsonPath("$.s"); + Assert.assertEquals(ImmutableSet.of(ColumnType.STRING_ARRAY), column.getColumnTypes(sPath)); + Assert.assertEquals(ColumnType.STRING_ARRAY, column.getColumnHolder(sPath).getCapabilities().toColumnType()); + ColumnValueSelector sSelector = column.makeColumnValueSelector(sPath, offset); + VectorObjectSelector sVectorSelector = column.makeVectorObjectSelector(sPath, vectorOffset); + VectorObjectSelector sVectorSelectorFiltered = column.makeVectorObjectSelector(sPath, bitmapVectorOffset); + ColumnIndexSupplier sIndexSupplier = column.getColumnIndexSupplier(sPath); + Assert.assertNotNull(sIndexSupplier); + Assert.assertNull(sIndexSupplier.as(StringValueSetIndex.class)); + Assert.assertNull(sIndexSupplier.as(DruidPredicateIndex.class)); + NullValueIndex sNulls = sIndexSupplier.as(NullValueIndex.class); + + final List sElementPath = NestedPathFinder.parseJsonPath("$.s[1]"); + ColumnValueSelector sElementSelector = column.makeColumnValueSelector(sElementPath, offset); + VectorObjectSelector sElementVectorSelector = column.makeVectorObjectSelector(sElementPath, vectorOffset); + VectorObjectSelector sElementFilteredVectorSelector = column.makeVectorObjectSelector( + sElementPath, + bitmapVectorOffset + ); + ColumnIndexSupplier sElementIndexSupplier = column.getColumnIndexSupplier(sElementPath); + Assert.assertNotNull(sElementIndexSupplier); + Assert.assertNull(sElementIndexSupplier.as(StringValueSetIndex.class)); + Assert.assertNull(sElementIndexSupplier.as(DruidPredicateIndex.class)); + Assert.assertNull(sElementIndexSupplier.as(NullValueIndex.class)); + + final List lPath = NestedPathFinder.parseJsonPath("$.l"); + Assert.assertEquals(ImmutableSet.of(ColumnType.LONG_ARRAY), column.getColumnTypes(lPath)); + Assert.assertEquals(ColumnType.LONG_ARRAY, column.getColumnHolder(lPath).getCapabilities().toColumnType()); + ColumnValueSelector lSelector = column.makeColumnValueSelector(lPath, offset); + VectorObjectSelector lVectorSelector = column.makeVectorObjectSelector(lPath, vectorOffset); + VectorObjectSelector lVectorSelectorFiltered = column.makeVectorObjectSelector(lPath, bitmapVectorOffset); + ColumnIndexSupplier lIndexSupplier = column.getColumnIndexSupplier(lPath); + Assert.assertNotNull(lIndexSupplier); + Assert.assertNull(lIndexSupplier.as(StringValueSetIndex.class)); + Assert.assertNull(lIndexSupplier.as(DruidPredicateIndex.class)); + NullValueIndex lNulls = lIndexSupplier.as(NullValueIndex.class); + + final List lElementPath = NestedPathFinder.parseJsonPath("$.l[1]"); + ColumnValueSelector lElementSelector = column.makeColumnValueSelector(lElementPath, offset); + VectorValueSelector lElementVectorSelector = column.makeVectorValueSelector(lElementPath, vectorOffset); + VectorObjectSelector lElementVectorObjectSelector = column.makeVectorObjectSelector(lElementPath, vectorOffset); + VectorValueSelector lElementFilteredVectorSelector = column.makeVectorValueSelector( + lElementPath, + bitmapVectorOffset + ); + ColumnIndexSupplier lElementIndexSupplier = column.getColumnIndexSupplier(lElementPath); + Assert.assertNotNull(lElementIndexSupplier); + Assert.assertNull(lElementIndexSupplier.as(StringValueSetIndex.class)); + Assert.assertNull(lElementIndexSupplier.as(DruidPredicateIndex.class)); + Assert.assertNull(lElementIndexSupplier.as(NullValueIndex.class)); + + final List dPath = NestedPathFinder.parseJsonPath("$.d"); + Assert.assertEquals(ImmutableSet.of(ColumnType.DOUBLE_ARRAY), column.getColumnTypes(dPath)); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, column.getColumnHolder(dPath).getCapabilities().toColumnType()); + ColumnValueSelector dSelector = column.makeColumnValueSelector(dPath, offset); + VectorObjectSelector dVectorSelector = column.makeVectorObjectSelector(dPath, vectorOffset); + VectorObjectSelector dVectorSelectorFiltered = column.makeVectorObjectSelector(dPath, bitmapVectorOffset); + ColumnIndexSupplier dIndexSupplier = column.getColumnIndexSupplier(dPath); + Assert.assertNotNull(dIndexSupplier); + Assert.assertNull(dIndexSupplier.as(StringValueSetIndex.class)); + Assert.assertNull(dIndexSupplier.as(DruidPredicateIndex.class)); + NullValueIndex dNulls = dIndexSupplier.as(NullValueIndex.class); + + final List dElementPath = NestedPathFinder.parseJsonPath("$.d[1]"); + ColumnValueSelector dElementSelector = column.makeColumnValueSelector(dElementPath, offset); + VectorValueSelector dElementVectorSelector = column.makeVectorValueSelector(dElementPath, vectorOffset); + VectorObjectSelector dElementVectorObjectSelector = column.makeVectorObjectSelector(dElementPath, vectorOffset); + VectorValueSelector dElementFilteredVectorSelector = column.makeVectorValueSelector( + dElementPath, + bitmapVectorOffset + ); + ColumnIndexSupplier dElementIndexSupplier = column.getColumnIndexSupplier(dElementPath); + Assert.assertNotNull(dElementIndexSupplier); + Assert.assertNull(dElementIndexSupplier.as(StringValueSetIndex.class)); + Assert.assertNull(dElementIndexSupplier.as(DruidPredicateIndex.class)); + Assert.assertNull(dElementIndexSupplier.as(NullValueIndex.class)); + + + ImmutableBitmap sNullIndex = sNulls.forNull().computeBitmapResult(resultFactory); + ImmutableBitmap lNullIndex = lNulls.forNull().computeBitmapResult(resultFactory); + ImmutableBitmap dNullIndex = dNulls.forNull().computeBitmapResult(resultFactory); + + int rowCounter = 0; + while (offset.withinBounds()) { + Map row = arrayTestData.get(rowCounter); + Assert.assertEquals( + JSON_MAPPER.writeValueAsString(row), + JSON_MAPPER.writeValueAsString(StructuredData.unwrap(rawSelector.getObject())) + ); + + Object[] s = (Object[]) row.get("s"); + Object[] l = (Object[]) row.get("l"); + Object[] d = (Object[]) row.get("d"); + Assert.assertArrayEquals(s, (Object[]) sSelector.getObject()); + Assert.assertArrayEquals(l, (Object[]) lSelector.getObject()); + Assert.assertArrayEquals(d, (Object[]) dSelector.getObject()); + Assert.assertEquals(s == null, sNullIndex.get(rowCounter)); + Assert.assertEquals(l == null, lNullIndex.get(rowCounter)); + Assert.assertEquals(d == null, dNullIndex.get(rowCounter)); + + if (s == null || s.length < 1) { + Assert.assertNull(sElementSelector.getObject()); + } else { + Assert.assertEquals(s[1], sElementSelector.getObject()); + } + if (l == null || l.length < 1) { + Assert.assertTrue(lElementSelector.isNull()); + Assert.assertNull(lElementSelector.getObject()); + } else { + Assert.assertEquals(l[1], lElementSelector.getLong()); + Assert.assertEquals(l[1], lElementSelector.getObject()); + } + if (d == null || d.length < 1) { + Assert.assertTrue(dElementSelector.isNull()); + Assert.assertNull(dElementSelector.getObject()); + } else { + Assert.assertEquals((Double) d[1], dElementSelector.getDouble(), 0.0); + Assert.assertEquals(d[1], dElementSelector.getObject()); + } offset.increment(); + rowCounter++; + } + + rowCounter = 0; + while (!vectorOffset.isDone()) { + final Object[] rawVector = rawVectorSelector.getObjectVector(); + final Object[] sVector = sVectorSelector.getObjectVector(); + final Object[] lVector = lVectorSelector.getObjectVector(); + final Object[] dVector = dVectorSelector.getObjectVector(); + final Object[] sElementVector = sElementVectorSelector.getObjectVector(); + final long[] lElementVector = lElementVectorSelector.getLongVector(); + final boolean[] lElementNulls = lElementVectorSelector.getNullVector(); + final Object[] lElementObjectVector = lElementVectorObjectSelector.getObjectVector(); + final double[] dElementVector = dElementVectorSelector.getDoubleVector(); + final boolean[] dElementNulls = dElementVectorSelector.getNullVector(); + final Object[] dElementObjectVector = dElementVectorObjectSelector.getObjectVector(); + + for (int i = 0; i < vectorOffset.getCurrentVectorSize(); i++, rowCounter++) { + + Map row = arrayTestData.get(rowCounter); + Assert.assertEquals( + JSON_MAPPER.writeValueAsString(row), + JSON_MAPPER.writeValueAsString(StructuredData.unwrap(rawVector[i])) + ); + Object[] s = (Object[]) row.get("s"); + Object[] l = (Object[]) row.get("l"); + Object[] d = (Object[]) row.get("d"); + + Assert.assertArrayEquals(s, (Object[]) sVector[i]); + Assert.assertArrayEquals(l, (Object[]) lVector[i]); + Assert.assertArrayEquals(d, (Object[]) dVector[i]); + + if (s == null || s.length < 1) { + Assert.assertNull(sElementVector[i]); + } else { + Assert.assertEquals(s[1], sElementVector[i]); + } + if (l == null || l.length < 1) { + Assert.assertTrue(lElementNulls[i]); + Assert.assertNull(lElementObjectVector[i]); + } else { + Assert.assertEquals(l[1], lElementVector[i]); + Assert.assertEquals(l[1], lElementObjectVector[i]); + } + if (d == null || d.length < 1) { + Assert.assertTrue(dElementNulls[i]); + Assert.assertNull(dElementObjectVector[i]); + } else { + Assert.assertEquals((Double) d[1], dElementVector[i], 0.0); + Assert.assertEquals(d[1], dElementObjectVector[i]); + } + } + vectorOffset.advance(); + } + + rowCounter = 0; + while (!bitmapVectorOffset.isDone()) { + final Object[] rawVector = rawVectorSelectorFiltered.getObjectVector(); + final Object[] sVector = sVectorSelectorFiltered.getObjectVector(); + final Object[] lVector = lVectorSelectorFiltered.getObjectVector(); + final Object[] dVector = dVectorSelectorFiltered.getObjectVector(); + final Object[] sElementVector = sElementFilteredVectorSelector.getObjectVector(); + final long[] lElementVector = lElementFilteredVectorSelector.getLongVector(); + final boolean[] lElementNulls = lElementFilteredVectorSelector.getNullVector(); + final double[] dElementVector = dElementFilteredVectorSelector.getDoubleVector(); + final boolean[] dElementNulls = dElementFilteredVectorSelector.getNullVector(); + + for (int i = 0; i < bitmapVectorOffset.getCurrentVectorSize(); i++, rowCounter += 2) { + Map row = arrayTestData.get(rowCounter); + Assert.assertEquals( + JSON_MAPPER.writeValueAsString(row), + JSON_MAPPER.writeValueAsString(StructuredData.unwrap(rawVector[i])) + ); + Object[] s = (Object[]) row.get("s"); + Object[] l = (Object[]) row.get("l"); + Object[] d = (Object[]) row.get("d"); + + Assert.assertArrayEquals(s, (Object[]) sVector[i]); + Assert.assertArrayEquals(l, (Object[]) lVector[i]); + Assert.assertArrayEquals(d, (Object[]) dVector[i]); + + if (s == null || s.length < 1) { + Assert.assertNull(sElementVector[i]); + } else { + Assert.assertEquals(s[1], sElementVector[i]); + } + if (l == null || l.length < 1) { + Assert.assertTrue(lElementNulls[i]); + } else { + Assert.assertEquals(l[1], lElementVector[i]); + } + if (d == null || d.length < 1) { + Assert.assertTrue(dElementNulls[i]); + } else { + Assert.assertEquals((Double) d[1], dElementVector[i], 0.0); + } + } + bitmapVectorOffset.advance(); } } @@ -384,11 +709,19 @@ private void testPath( Assert.assertEquals(dimSelector.idLookup().lookupId(dimSelectorLookupVal), dimSelector.getRow().get(0)); Assert.assertTrue(valueSetIndex.forValue(theString).computeBitmapResult(resultFactory).get(rowNumber)); - Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(theString))).computeBitmapResult(resultFactory).get(rowNumber)); - Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(theString)).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(theString))) + .computeBitmapResult(resultFactory) + .get(rowNumber)); + Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(theString)) + .computeBitmapResult(resultFactory) + .get(rowNumber)); Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber)); - Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH))).computeBitmapResult(resultFactory).get(rowNumber)); - Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH))) + .computeBitmapResult(resultFactory) + .get(rowNumber)); + Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) + .computeBitmapResult(resultFactory) + .get(rowNumber)); Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber)); Assert.assertTrue(dimSelector.makeValueMatcher(theString).matches()); @@ -406,9 +739,13 @@ private void testPath( Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(rowNumber)); Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber)); Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber)); - Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) + .computeBitmapResult(resultFactory) + .get(rowNumber)); Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber)); - Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) + .computeBitmapResult(resultFactory) + .get(rowNumber)); Assert.assertTrue(dimSelector.makeValueMatcher((String) null).matches()); Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnSelectorsTest.java index cf65e483ba63..0c7e68cd2c96 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnSelectorsTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.Module; import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.UOE; @@ -41,6 +42,7 @@ import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector; import org.apache.druid.segment.vector.BaseLongVectorValueSelector; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; @@ -326,15 +328,16 @@ private VirtualColumns makeNestedNumericVirtualColumns() private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns virtualColumns) throws Exception { List segments = NestedDataTestUtils.createSegments( - helper, tempFolder, closer, NestedDataTestUtils.NUMERIC_DATA_FILE, - NestedDataTestUtils.NUMERIC_PARSER_FILE, - NestedDataTestUtils.SIMPLE_AGG_FILE, - Granularities.DAY, - true, - 1000 + JsonInputFormat.DEFAULT, + NestedDataTestUtils.TIMESTAMP_SPEC, + NestedDataTestUtils.AUTO_DISCOVERY, + TransformSpec.NONE, + NestedDataTestUtils.COUNT, + Granularities.NONE, + true ); Assert.assertEquals(1, segments.size()); StorageAdapter storageAdapter = segments.get(0).asStorageAdapter(); @@ -355,15 +358,16 @@ private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns vir private VectorColumnSelectorFactory getVectorColumnSelectorFactory(VirtualColumns virtualColumns) throws Exception { List segments = NestedDataTestUtils.createSegments( - helper, tempFolder, closer, NestedDataTestUtils.NUMERIC_DATA_FILE, - NestedDataTestUtils.NUMERIC_PARSER_FILE, - NestedDataTestUtils.SIMPLE_AGG_FILE, - Granularities.DAY, - true, - 1000 + JsonInputFormat.DEFAULT, + NestedDataTestUtils.TIMESTAMP_SPEC, + NestedDataTestUtils.AUTO_DISCOVERY, + TransformSpec.NONE, + NestedDataTestUtils.COUNT, + Granularities.NONE, + true ); Assert.assertEquals(1, segments.size()); StorageAdapter storageAdapter = segments.get(0).asStorageAdapter(); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedLiteralTypeInfoTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedLiteralTypeInfoTest.java new file mode 100644 index 000000000000..bcf5e7c2c356 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedLiteralTypeInfoTest.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import nl.jqno.equalsverifier.EqualsVerifier; +import nl.jqno.equalsverifier.Warning; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; +import java.util.List; +import java.util.Set; + +public class NestedLiteralTypeInfoTest +{ + private static final ByteBuffer BUFFER = ByteBuffer.allocate(1024).order(ByteOrder.nativeOrder()); + + @Test + public void testSingleType() throws IOException + { + List supportedTypes = ImmutableList.of( + ColumnType.STRING, + ColumnType.LONG, + ColumnType.DOUBLE, + ColumnType.STRING_ARRAY, + ColumnType.LONG_ARRAY, + ColumnType.DOUBLE_ARRAY + ); + + for (ColumnType type : supportedTypes) { + testSingleType(type); + } + } + + @Test + public void testMultiType() throws IOException + { + List> tests = ImmutableList.of( + ImmutableSet.of(ColumnType.STRING, ColumnType.LONG), + ImmutableSet.of(ColumnType.LONG, ColumnType.DOUBLE), + ImmutableSet.of(ColumnType.STRING, ColumnType.LONG, ColumnType.DOUBLE), + ImmutableSet.of(ColumnType.DOUBLE, ColumnType.DOUBLE_ARRAY), + ImmutableSet.of(ColumnType.LONG_ARRAY, ColumnType.DOUBLE_ARRAY) + ); + + for (Set typeSet : tests) { + testMultiType(typeSet); + } + } + + @Test + public void testEqualsAndHashCode() + { + EqualsVerifier.forClass(NestedLiteralTypeInfo.TypeSet.class) + .usingGetClass() + .verify(); + + EqualsVerifier.forClass(NestedLiteralTypeInfo.MutableTypeSet.class) + .suppress(Warning.NONFINAL_FIELDS) + .usingGetClass() + .verify(); + } + + private void testSingleType(ColumnType columnType) throws IOException + { + NestedLiteralTypeInfo.MutableTypeSet typeSet = new NestedLiteralTypeInfo.MutableTypeSet(); + Assert.assertNull(typeSet.getSingleType()); + Assert.assertTrue(typeSet.isEmpty()); + + typeSet.add(columnType); + + Assert.assertEquals(columnType, typeSet.getSingleType()); + Assert.assertEquals(ImmutableSet.of(columnType), NestedLiteralTypeInfo.convertToSet(typeSet.getByteValue())); + + writeTypeSet(typeSet); + NestedLiteralTypeInfo info = new NestedLiteralTypeInfo(BUFFER); + Assert.assertEquals(0, BUFFER.position()); + + NestedLiteralTypeInfo.TypeSet roundTrip = info.getTypes(0); + Assert.assertEquals(columnType, roundTrip.getSingleType()); + + NestedLiteralTypeInfo info2 = NestedLiteralTypeInfo.read(BUFFER, 1); + Assert.assertEquals(info.getTypes(0), info2.getTypes(0)); + Assert.assertEquals(1, BUFFER.position()); + } + + private void testMultiType(Set columnTypes) throws IOException + { + NestedLiteralTypeInfo.MutableTypeSet typeSet = new NestedLiteralTypeInfo.MutableTypeSet(); + Assert.assertNull(typeSet.getSingleType()); + Assert.assertTrue(typeSet.isEmpty()); + + NestedLiteralTypeInfo.MutableTypeSet merge = new NestedLiteralTypeInfo.MutableTypeSet(); + for (ColumnType columnType : columnTypes) { + typeSet.add(columnType); + merge.merge(new NestedLiteralTypeInfo.MutableTypeSet().add(columnType).getByteValue()); + } + + Assert.assertEquals(merge.getByteValue(), typeSet.getByteValue()); + Assert.assertNull(typeSet.getSingleType()); + Assert.assertEquals(columnTypes, NestedLiteralTypeInfo.convertToSet(typeSet.getByteValue())); + + writeTypeSet(typeSet); + NestedLiteralTypeInfo info = new NestedLiteralTypeInfo(BUFFER); + Assert.assertEquals(0, BUFFER.position()); + + NestedLiteralTypeInfo.TypeSet roundTrip = info.getTypes(0); + Assert.assertNull(roundTrip.getSingleType()); + Assert.assertEquals(columnTypes, NestedLiteralTypeInfo.convertToSet(roundTrip.getByteValue())); + + NestedLiteralTypeInfo info2 = NestedLiteralTypeInfo.read(BUFFER, 1); + Assert.assertEquals(info.getTypes(0), info2.getTypes(0)); + Assert.assertEquals(1, BUFFER.position()); + } + + private static void writeTypeSet(NestedLiteralTypeInfo.MutableTypeSet typeSet) throws IOException + { + BUFFER.position(0); + NestedLiteralTypeInfo.Writer writer = new NestedLiteralTypeInfo.Writer(new OnHeapMemorySegmentWriteOutMedium()); + writer.open(); + writer.write(typeSet); + Assert.assertEquals(1, writer.getSerializedSize()); + + WritableByteChannel channel = new WritableByteChannel() + { + @Override + public int write(ByteBuffer src) + { + int size = src.remaining(); + BUFFER.put(src); + return size; + } + + @Override + public boolean isOpen() + { + return true; + } + + @Override + public void close() + { + } + }; + writer.writeTo(channel, null); + Assert.assertEquals(1, BUFFER.position()); + + BUFFER.position(0); + } +} diff --git a/processing/src/test/resources/nested-array-test-data.json b/processing/src/test/resources/nested-array-test-data.json new file mode 100644 index 000000000000..921077103e3a --- /dev/null +++ b/processing/src/test/resources/nested-array-test-data.json @@ -0,0 +1,14 @@ +{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"x": 1},{"x":2}]} +{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayObject":[{"x": 3},{"x":4}]} +{"timestamp": "2023-01-01T00:00:00", "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayObject":[null,{"x":2}]} +{"timestamp": "2023-01-01T00:00:00", "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayObject":[{"x": null},{"x":2}]} +{"timestamp": "2023-01-01T00:00:00", "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":null, "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayObject":[{"x": 1000},{"y":2000}]} +{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"a": 1},{"b":2}]} +{"timestamp": "2023-01-01T00:00:00", "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null, 1.1], "arrayVariant":null, "arrayObject":[{"x": 1},{"x":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"x": 1},{"x":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayObject":[{"x": 3},{"x":4}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayObject":[null,{"x":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayObject":[{"x": null},{"x":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":null, "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayObject":[{"x": 1000},{"y":2000}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"a": 1},{"b":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null, 1.1], "arrayVariant":null, "arrayObject":[{"x": 1},{"x":2}]} \ No newline at end of file diff --git a/processing/src/test/resources/nested-numeric-test-data.json b/processing/src/test/resources/nested-numeric-test-data.json new file mode 100644 index 000000000000..ba79dca0e937 --- /dev/null +++ b/processing/src/test/resources/nested-numeric-test-data.json @@ -0,0 +1,10 @@ +{"timestamp": "2023-01-01", "nest": {"long": 100, "double": 100.1, "mixed_numeric": 100, "mixed":"foo", "sparse_long": 100, "sparse_mixed_numeric": 100, "sparse_mixed": "foo"}} +{"timestamp": "2023-01-01", "nest": {"long": 200, "double": 200.2, "mixed_numeric": 200.4, "mixed":"200", "sparse_double": 200.2, "sparse_mixed_numeric": 200.2}} +{"timestamp": "2023-01-01", "nest": {"long": 300, "double": 300.3, "mixed_numeric": 300, "mixed":300, "sparse_long": 300, "sparse_double": 300.3, "sparse_mixed": 300}} +{"timestamp": "2023-01-01", "nest": {"long": 400, "double": 400.4, "mixed_numeric": 400.4, "mixed":"bar", "sparse_double": 400.4, "sparse_mixed_numeric": 400}} +{"timestamp": "2023-01-01", "nest": {"long": 500, "double": 500.5, "mixed_numeric": 500, "mixed":500.5, "sparse_long": 500, "sparse_mixed_numeric": 500.5, "sparse_mixed": 500.5}} +{"timestamp": "2023-01-02", "nest": {"long": 100, "double": 100.1, "mixed_numeric": 100, "mixed":"foo", "sparse_long": 100, "sparse_mixed_numeric": 100, "sparse_mixed": "foo"}} +{"timestamp": "2023-01-02", "nest": {"long": 200, "double": 200.2, "mixed_numeric": 200.4, "mixed":"200", "sparse_double": 200.2, "sparse_mixed_numeric": 200.2}} +{"timestamp": "2023-01-02", "nest": {"long": 300, "double": 300.3, "mixed_numeric": 300, "mixed":300, "sparse_long": 300, "sparse_double": 300.3, "sparse_mixed": 300}} +{"timestamp": "2023-01-02", "nest": null} +{"timestamp": "2023-01-02", "nest": {"long": 500, "double": 500.5, "mixed_numeric": 500, "mixed":500.5, "sparse_long": 500, "sparse_mixed_numeric": 500.5, "sparse_mixed": 500.5}} \ No newline at end of file diff --git a/processing/src/test/resources/nested-simple-test-data.json b/processing/src/test/resources/nested-simple-test-data.json new file mode 100644 index 000000000000..2bc3017bf68b --- /dev/null +++ b/processing/src/test/resources/nested-simple-test-data.json @@ -0,0 +1,8 @@ +{"timestamp": "2023-01-01", "dim": "hello", "nest": {"x": 100, "y": 200, "z": 300}, "nester":{ "x": ["a", "b", "c"], "y": {"a": "a", "b": "b", "c": [1, 2, 3]}}, "variant": {"a": ["hello", "world"], "b": {"x": "hello", "y": "world"}}, "list":[{"x": 5, "y": 10}, {"x": 15, "y": 22}]} +{"timestamp": "2023-01-01", "dim": "hello", "nester":{ "x": ["x", "y", "z"]}, "list":[{"x": 35, "y": 310}, {"x": 315, "y": 322}]} +{"timestamp": "2023-01-01", "dim": "hello", "nest":{ "x": 300, "y": 800}, "nester": "hello"} +{"timestamp": "2023-01-01", "dim": "100", "nest":{ "y": 500}, "list":[{"x": 115, "y": 410}, {"x": 415, "y": 422}]} +{"timestamp": "2023-01-02", "dim": "world", "nest": {"x": 200, "y": 100, "z": 101}, "nester":{ "x": ["x", "y", "z"], "y": {"a": "b", "b": "c", "c": [4, 5, 6]}}, "variant": {"b": ["hello", "world"], "c": {"x": ["hello"], "y": "world"}}} +{"timestamp": "2023-01-02", "dim": "hello", "nester":{ "x": ["x", "y", "z"]}} +{"timestamp": "2023-01-02", "dim": "hello", "nest":{ "x": 300, "y": 800}, "nester": 1} +{"timestamp": "2023-01-02", "dim": "hello", "nest":{ "y": 500}, "list":[{"x": 615, "y": 610}, {"x": 715, "y": 722}]} \ No newline at end of file diff --git a/processing/src/test/resources/nested-simple-test-data.tsv b/processing/src/test/resources/nested-simple-test-data.tsv new file mode 100644 index 000000000000..ca958eb3d294 --- /dev/null +++ b/processing/src/test/resources/nested-simple-test-data.tsv @@ -0,0 +1,8 @@ +2023-01-01 hello {"x":100,"y":200,"z":300} {"x":["a","b","c"],"y":{"a":"a","b":"b","c":[1,2,3]}} {"a":["hello","world"],"b":{"x":"hello","y":"world"}} [{"x":5,"y":10},{"x":15,"y":22}] +2023-01-01 hello {"x":["x","y","z"]} [{"x":35,"y":310},{"x":315,"y":322}] +2023-01-01 hello {"x":300,"y":800} "hello" +2023-01-01 hello {"y":500} [{"x":115,"y":410},{"x":415,"y":422}] +2023-01-02 hello {"x":200,"y":100,"z":101} {"x":["x","y","z"],"y":{"a":"b","b":"c","c":[4,5,6]}} {"b":["hello","world"],"c":{"x":["hello"],"y":"world"}} +2023-01-02 hello {"x":["x","y","z"]} +2023-01-02 hello {"x":300,"y":800} 1 +2023-01-02 hello {"y":500} [{"x":615,"y":610},{"x":715,"y":722}] \ No newline at end of file diff --git a/processing/src/test/resources/nested-types-test-data.json b/processing/src/test/resources/nested-types-test-data.json new file mode 100644 index 000000000000..2d7505136b52 --- /dev/null +++ b/processing/src/test/resources/nested-types-test-data.json @@ -0,0 +1,8 @@ +{"timestamp": "2023-01-01", "str":"a", "long":1, "double":1.0, "variant": 1} +{"timestamp": "2023-01-01", "str":"", "long":2, "variant": "b"} +{"timestamp": "2023-01-01", "str":"null", "long":3, "double":2.0, "variant": 3.0} +{"timestamp": "2023-01-01", "str":"b", "long":4, "double":3.3, "variant": "4"} +{"timestamp": "2023-01-01", "str":"c", "long": null, "double":4.4, "variant": "hello"} +{"timestamp": "2023-01-01", "str":"d", "long":5, "double":5.9} +{"timestamp": "2023-01-01", "str":null, "double":null, "variant": 51} +{"timestamp": "2023-01-01", "long":6, "double":1.0, "variant": null} \ No newline at end of file diff --git a/processing/src/test/resources/numeric-nested-test-data-parser.json b/processing/src/test/resources/numeric-nested-test-data-parser.json deleted file mode 100644 index 9ba5cd0c57bd..000000000000 --- a/processing/src/test/resources/numeric-nested-test-data-parser.json +++ /dev/null @@ -1,20 +0,0 @@ -{ - "type": "string", - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [ - { - "type": "json", - "name": "nest" - } - ], - "dimensionExclusions": [], - "spatialDimensions": [] - } - } -} diff --git a/processing/src/test/resources/numeric-nested-test-data.json b/processing/src/test/resources/numeric-nested-test-data.json deleted file mode 100644 index 1a6b2205c026..000000000000 --- a/processing/src/test/resources/numeric-nested-test-data.json +++ /dev/null @@ -1,10 +0,0 @@ -{"timestamp": "2021-01-01", "nest": {"long": 100, "double": 100.1, "mixed_numeric": 100, "mixed":"foo", "sparse_long": 100, "sparse_mixed_numeric": 100, "sparse_mixed": "foo"}} -{"timestamp": "2021-01-01", "nest": {"long": 200, "double": 200.2, "mixed_numeric": 200.4, "mixed":"200", "sparse_double": 200.2, "sparse_mixed_numeric": 200.2}} -{"timestamp": "2021-01-01", "nest": {"long": 300, "double": 300.3, "mixed_numeric": 300, "mixed":300, "sparse_long": 300, "sparse_double": 300.3, "sparse_mixed": 300}} -{"timestamp": "2021-01-01", "nest": {"long": 400, "double": 400.4, "mixed_numeric": 400.4, "mixed":"bar", "sparse_double": 400.4, "sparse_mixed_numeric": 400}} -{"timestamp": "2021-01-01", "nest": {"long": 500, "double": 500.5, "mixed_numeric": 500, "mixed":500.5, "sparse_long": 500, "sparse_mixed_numeric": 500.5, "sparse_mixed": 500.5}} -{"timestamp": "2021-01-02", "nest": {"long": 100, "double": 100.1, "mixed_numeric": 100, "mixed":"foo", "sparse_long": 100, "sparse_mixed_numeric": 100, "sparse_mixed": "foo"}} -{"timestamp": "2021-01-02", "nest": {"long": 200, "double": 200.2, "mixed_numeric": 200.4, "mixed":"200", "sparse_double": 200.2, "sparse_mixed_numeric": 200.2}} -{"timestamp": "2021-01-02", "nest": {"long": 300, "double": 300.3, "mixed_numeric": 300, "mixed":300, "sparse_long": 300, "sparse_double": 300.3, "sparse_mixed": 300}} -{"timestamp": "2021-01-02", "nest": null} -{"timestamp": "2021-01-02", "nest": {"long": 500, "double": 500.5, "mixed_numeric": 500, "mixed":500.5, "sparse_long": 500, "sparse_mixed_numeric": 500.5, "sparse_mixed": 500.5}} \ No newline at end of file diff --git a/processing/src/test/resources/simple-nested-test-data-aggs.json b/processing/src/test/resources/simple-nested-test-data-aggs.json deleted file mode 100644 index 0df3d6d72765..000000000000 --- a/processing/src/test/resources/simple-nested-test-data-aggs.json +++ /dev/null @@ -1,6 +0,0 @@ -[ - { - "type": "count", - "name": "count" - } -] \ No newline at end of file diff --git a/processing/src/test/resources/simple-nested-test-data-parser.json b/processing/src/test/resources/simple-nested-test-data-parser.json deleted file mode 100644 index 5b48d02c8391..000000000000 --- a/processing/src/test/resources/simple-nested-test-data-parser.json +++ /dev/null @@ -1,33 +0,0 @@ -{ - "type": "string", - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [ - "dim", - { - "type": "json", - "name": "nest" - }, - { - "type": "json", - "name": "nester" - }, - { - "type": "json", - "name": "variant" - }, - { - "type": "json", - "name": "list" - } - ], - "dimensionExclusions": [], - "spatialDimensions": [] - } - } -} diff --git a/processing/src/test/resources/simple-nested-test-data-tsv-parser.json b/processing/src/test/resources/simple-nested-test-data-tsv-parser.json deleted file mode 100644 index 8d69dd0cda52..000000000000 --- a/processing/src/test/resources/simple-nested-test-data-tsv-parser.json +++ /dev/null @@ -1,41 +0,0 @@ -{ - "type": "string", - "parseSpec": { - "format": "tsv", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [ - "dim", - { - "type": "json", - "name": "nest_json" - }, - { - "type": "json", - "name": "nester_json" - }, - { - "type": "json", - "name": "variant_json" - }, - { - "type": "json", - "name": "list_json" - } - ], - "dimensionExclusions": [], - "spatialDimensions": [] - }, - "columns": [ - "timestamp", - "dim", - "nest", - "nester", - "variant", - "list" - ] - } -} diff --git a/processing/src/test/resources/simple-nested-test-data-tsv-transform.json b/processing/src/test/resources/simple-nested-test-data-tsv-transform.json deleted file mode 100644 index 5be210434b45..000000000000 --- a/processing/src/test/resources/simple-nested-test-data-tsv-transform.json +++ /dev/null @@ -1,24 +0,0 @@ -{ - "transforms": [ - { - "type": "expression", - "name": "nest_json", - "expression": "parse_json(nest)" - }, - { - "type": "expression", - "name": "nester_json", - "expression": "parse_json(nester)" - }, - { - "type": "expression", - "name": "variant_json", - "expression": "parse_json(variant)" - }, - { - "type": "expression", - "name": "list_json", - "expression": "parse_json(list)" - } - ] -} \ No newline at end of file diff --git a/processing/src/test/resources/simple-nested-test-data.json b/processing/src/test/resources/simple-nested-test-data.json deleted file mode 100644 index 3def4206ac9c..000000000000 --- a/processing/src/test/resources/simple-nested-test-data.json +++ /dev/null @@ -1,8 +0,0 @@ -{"timestamp": "2021-01-01", "dim": "hello", "nest": {"x": 100, "y": 200, "z": 300}, "nester":{ "x": ["a", "b", "c"], "y": {"a": "a", "b": "b", "c": [1, 2, 3]}}, "variant": {"a": ["hello", "world"], "b": {"x": "hello", "y": "world"}}, "list":[{"x": 5, "y": 10}, {"x": 15, "y": 22}]} -{"timestamp": "2021-01-01", "dim": "hello", "nester":{ "x": ["x", "y", "z"]}, "list":[{"x": 35, "y": 310}, {"x": 315, "y": 322}]} -{"timestamp": "2021-01-01", "dim": "hello", "nest":{ "x": 300, "y": 800}, "nester": "hello"} -{"timestamp": "2021-01-01", "dim": "100", "nest":{ "y": 500}, "list":[{"x": 115, "y": 410}, {"x": 415, "y": 422}]} -{"timestamp": "2021-01-02", "dim": "world", "nest": {"x": 200, "y": 100, "z": 101}, "nester":{ "x": ["x", "y", "z"], "y": {"a": "b", "b": "c", "c": [4, 5, 6]}}, "variant": {"b": ["hello", "world"], "c": {"x": ["hello"], "y": "world"}}} -{"timestamp": "2021-01-02", "dim": "hello", "nester":{ "x": ["x", "y", "z"]}} -{"timestamp": "2021-01-02", "dim": "hello", "nest":{ "x": 300, "y": 800}, "nester": 1} -{"timestamp": "2021-01-02", "dim": "hello", "nest":{ "y": 500}, "list":[{"x": 615, "y": 610}, {"x": 715, "y": 722}]} \ No newline at end of file diff --git a/processing/src/test/resources/simple-nested-test-data.tsv b/processing/src/test/resources/simple-nested-test-data.tsv deleted file mode 100644 index ad78b0ce2010..000000000000 --- a/processing/src/test/resources/simple-nested-test-data.tsv +++ /dev/null @@ -1,8 +0,0 @@ -2021-01-01 hello {"x":100,"y":200,"z":300} {"x":["a","b","c"],"y":{"a":"a","b":"b","c":[1,2,3]}} {"a":["hello","world"],"b":{"x":"hello","y":"world"}} [{"x":5,"y":10},{"x":15,"y":22}] -2021-01-01 hello {"x":["x","y","z"]} [{"x":35,"y":310},{"x":315,"y":322}] -2021-01-01 hello {"x":300,"y":800} "hello" -2021-01-01 hello {"y":500} [{"x":115,"y":410},{"x":415,"y":422}] -2021-01-02 hello {"x":200,"y":100,"z":101} {"x":["x","y","z"],"y":{"a":"b","b":"c","c":[4,5,6]}} {"b":["hello","world"],"c":{"x":["hello"],"y":"world"}} -2021-01-02 hello {"x":["x","y","z"]} -2021-01-02 hello {"x":300,"y":800} 1 -2021-01-02 hello {"y":500} [{"x":615,"y":610},{"x":715,"y":722}] \ No newline at end of file diff --git a/processing/src/test/resources/types-test-data-parser.json b/processing/src/test/resources/types-test-data-parser.json deleted file mode 100644 index c148d6ef2d92..000000000000 --- a/processing/src/test/resources/types-test-data-parser.json +++ /dev/null @@ -1,16 +0,0 @@ -{ - "type": "string", - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [], - "dimensionExclusions": [], - "spatialDimensions": [], - "useNestedColumnIndexerForSchemaDiscovery": true - } - } -} diff --git a/processing/src/test/resources/types-test-data.json b/processing/src/test/resources/types-test-data.json deleted file mode 100644 index b4f5aa07fc6e..000000000000 --- a/processing/src/test/resources/types-test-data.json +++ /dev/null @@ -1,8 +0,0 @@ -{"timestamp": "2021-01-01", "str":"a", "long":1, "double":1.0, "variant": 1} -{"timestamp": "2021-01-01", "str":"", "long":2, "variant": "b"} -{"timestamp": "2021-01-01", "str":"null", "long":3, "double":2.0, "variant": 3.0} -{"timestamp": "2021-01-01", "str":"b", "long":4, "double":3.3, "variant": "4"} -{"timestamp": "2021-01-01", "str":"c", "long": null, "double":4.4, "variant": "hello"} -{"timestamp": "2021-01-01", "str":"d", "long":5, "double":5.9} -{"timestamp": "2021-01-01", "str":null, "double":null, "variant": 51} -{"timestamp": "2021-01-01", "long":6, "double":1.0, "variant": null} \ No newline at end of file diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java index ff92f83c3772..abb38d5330b3 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java @@ -251,10 +251,8 @@ private static DruidExpression fieldAccessToDruidExpression( // and throw an exception while returning false from isValidDruidQuery() method if (index < 0) { throw new CannotBuildQueryException(StringUtils.format( - "Expression referred to nonexistent index[%d] in row[%s]", - index, - rowSignature - )); + "Cannot build query as column name [%s] does not exist in row [%s]", ref.getField().getName(), rowSignature) + ); } final Optional columnType = rowSignature.getColumnType(index); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java index fec2baca6f0e..6f2cd8f00154 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java @@ -95,7 +95,7 @@ public DimFilter toDruidFilter( final DruidExpression leftExpr = druidExpressions.get(0); final DruidExpression rightExpr = druidExpressions.get(1); - if (leftExpr.isSimpleExtraction()) { + if (leftExpr.isSimpleExtraction() && !(leftExpr.getDruidType() != null && leftExpr.getDruidType().isArray())) { Expr expr = Parser.parse(rightExpr.getExpression(), plannerContext.getExprMacroTable()); // To convert this expression filter into an And of Selector filters, we need to extract all array elements. // For now, we can optimize only when rightExpr is a literal because there is no way to extract the array elements diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java index 66a950cb2775..b6e0df60e869 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java @@ -40,6 +40,7 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeTransforms; import org.apache.calcite.sql2rel.SqlRexConvertlet; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.InputBindings; @@ -54,6 +55,7 @@ import org.apache.druid.sql.calcite.expression.Expressions; import org.apache.druid.sql.calcite.expression.OperatorConversions; import org.apache.druid.sql.calcite.expression.SqlOperatorConversion; +import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; import org.apache.druid.sql.calcite.planner.convertlet.DruidConvertletFactory; @@ -289,6 +291,33 @@ public SqlRexConvertlet createConvertlet(PlannerContext plannerContext) call.operand(0), call.operand(1) ); + } else if (SqlTypeName.ARRAY.equals(sqlType.getSqlTypeName())) { + ColumnType elementType = Calcites.getColumnTypeForRelDataType(sqlType.getComponentType()); + switch (elementType.getType()) { + case LONG: + rewrite = JsonValueReturningArrayBigIntOperatorConversion.FUNCTION.createCall( + SqlParserPos.ZERO, + call.operand(0), + call.operand(1) + ); + break; + case DOUBLE: + rewrite = JsonValueReturningArrayDoubleOperatorConversion.FUNCTION.createCall( + SqlParserPos.ZERO, + call.operand(0), + call.operand(1) + ); + break; + case STRING: + rewrite = JsonValueReturningArrayVarcharOperatorConversion.FUNCTION.createCall( + SqlParserPos.ZERO, + call.operand(0), + call.operand(1) + ); + break; + default: + throw new IAE("Unhandled JSON_VALUE RETURNING ARRAY type [%s]", sqlType.getComponentType()); + } } else { // fallback to json_value_any, e.g. the 'standard' convertlet. rewrite = JsonValueAnyOperatorConversion.FUNCTION.createCall( @@ -298,6 +327,7 @@ public SqlRexConvertlet createConvertlet(PlannerContext plannerContext) ); } + // always cast anyway, to prevent haters from complaining that VARCHAR doesn't match VARCHAR(2000) SqlNode caster = SqlStdOperatorTable.CAST.createCall( SqlParserPos.ZERO, @@ -445,6 +475,135 @@ public JsonValueVarcharOperatorConversion() } } + public abstract static class JsonValueReturningArrayTypeOperatorConversion implements SqlOperatorConversion + { + private final SqlFunction function; + private final ColumnType druidType; + + public JsonValueReturningArrayTypeOperatorConversion(SqlFunction function, ColumnType druidType) + { + this.druidType = druidType; + this.function = function; + } + + @Override + public SqlOperator calciteOperator() + { + return function; + } + + @Nullable + @Override + public DruidExpression toDruidExpression( + PlannerContext plannerContext, + RowSignature rowSignature, + RexNode rexNode + ) + { + final RexCall call = (RexCall) rexNode; + final List druidExpressions = Expressions.toDruidExpressions( + plannerContext, + rowSignature, + call.getOperands() + ); + + if (druidExpressions == null || druidExpressions.size() != 2) { + return null; + } + + final Expr pathExpr = Parser.parse(druidExpressions.get(1).getExpression(), plannerContext.getExprMacroTable()); + if (!pathExpr.isLiteral()) { + return null; + } + // pre-normalize path so that the same expressions with different jq syntax are collapsed + final String path = (String) pathExpr.eval(InputBindings.nilBindings()).value(); + final List parts; + try { + parts = NestedPathFinder.parseJsonPath(path); + } + catch (IllegalArgumentException iae) { + throw new UnsupportedSQLQueryException( + "Cannot use [%s]: [%s]", + call.getOperator().getName(), + iae.getMessage() + ); + } + final String jsonPath = NestedPathFinder.toNormalizedJsonPath(parts); + final DruidExpression.ExpressionGenerator builder = (args) -> + "json_value(" + args.get(0).getExpression() + ",'" + jsonPath + "', '" + druidType.asTypeString() + "')"; + + if (druidExpressions.get(0).isSimpleExtraction()) { + + return DruidExpression.ofVirtualColumn( + druidType, + builder, + ImmutableList.of( + DruidExpression.ofColumn(NestedDataComplexTypeSerde.TYPE, druidExpressions.get(0).getDirectColumn()) + ), + (name, outputType, expression, macroTable) -> new NestedFieldVirtualColumn( + druidExpressions.get(0).getDirectColumn(), + name, + outputType, + parts, + false, + null, + null + ) + ); + } + return DruidExpression.ofExpression(druidType, builder, druidExpressions); + } + + static SqlFunction buildArrayFunction(String functionName, SqlTypeName elementTypeName) + { + return OperatorConversions.operatorBuilder(functionName) + .operandTypeChecker( + OperandTypes.sequence( + "(expr,path)", + OperandTypes.family(SqlTypeFamily.ANY), + OperandTypes.family(SqlTypeFamily.STRING) + ) + ) + .returnTypeInference( + opBinding -> { + return opBinding.getTypeFactory().createTypeWithNullability(Calcites.createSqlArrayTypeWithNullability(opBinding.getTypeFactory(), elementTypeName, false), true); + } + ) + .functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION) + .build(); + } + } + + public static class JsonValueReturningArrayBigIntOperatorConversion extends JsonValueReturningArrayTypeOperatorConversion + { + static final SqlFunction FUNCTION = buildArrayFunction("JSON_VALUE_RETURNING_ARRAY_BIGINT", SqlTypeName.BIGINT); + + public JsonValueReturningArrayBigIntOperatorConversion() + { + super(FUNCTION, ColumnType.LONG_ARRAY); + } + } + + public static class JsonValueReturningArrayDoubleOperatorConversion extends JsonValueReturningArrayTypeOperatorConversion + { + static final SqlFunction FUNCTION = buildArrayFunction("JSON_VALUE_RETURNING_ARRAY_DOUBLE", SqlTypeName.DOUBLE); + + public JsonValueReturningArrayDoubleOperatorConversion() + { + super(FUNCTION, ColumnType.DOUBLE_ARRAY); + } + } + + public static class JsonValueReturningArrayVarcharOperatorConversion extends JsonValueReturningArrayTypeOperatorConversion + { + static final SqlFunction FUNCTION = buildArrayFunction("JSON_VALUE_RETURNING_ARRAY_VARCHAR", SqlTypeName.VARCHAR); + + public JsonValueReturningArrayVarcharOperatorConversion() + { + super(FUNCTION, ColumnType.STRING_ARRAY); + } + } + public static class JsonValueAnyOperatorConversion implements SqlOperatorConversion { private static final SqlFunction FUNCTION = diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java index a5c5fd1361c3..a6d73fc356b3 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java @@ -324,6 +324,9 @@ public class DruidOperatorTable implements SqlOperatorTable .add(new NestedDataOperatorConversions.JsonValueBigintOperatorConversion()) .add(new NestedDataOperatorConversions.JsonValueDoubleOperatorConversion()) .add(new NestedDataOperatorConversions.JsonValueVarcharOperatorConversion()) + .add(new NestedDataOperatorConversions.JsonValueReturningArrayBigIntOperatorConversion()) + .add(new NestedDataOperatorConversions.JsonValueReturningArrayDoubleOperatorConversion()) + .add(new NestedDataOperatorConversions.JsonValueReturningArrayVarcharOperatorConversion()) .add(new NestedDataOperatorConversions.JsonObjectOperatorConversion()) .add(new NestedDataOperatorConversions.ToJsonStringOperatorConversion()) .add(new NestedDataOperatorConversions.ParseJsonOperatorConversion()) 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 ddbe3e533b5c..962f521622cb 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 @@ -704,6 +704,23 @@ public void testQuery( .run(); } + public void testQuery( + final String sql, + final Map queryContext, + final List> expectedQueries, + final List expectedResults, + final RowSignature expectedResultSignature + ) + { + testBuilder() + .sql(sql) + .queryContext(queryContext) + .expectedQueries(expectedQueries) + .expectedResults(expectedResults) + .expectedSignature(expectedResultSignature) + .run(); + } + public void testQuery( final String sql, final Map queryContext, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 692f3830e529..f5bb1cec1361 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -26,8 +26,10 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.ResourceInputSource; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -37,6 +39,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Druids; +import org.apache.druid.query.NestedDataTestUtils; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -44,8 +47,10 @@ import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.LikeDimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery; @@ -71,6 +76,8 @@ import org.junit.Test; import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -79,6 +86,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest private static final String DATA_SOURCE = "nested"; private static final String DATA_SOURCE_MIXED = "nested_mix"; private static final String DATA_SOURCE_MIXED_2 = "nested_mix_2"; + private static final String DATA_SOURCE_ARRAYS = "arrays"; private static final List> RAW_ROWS = ImmutableList.of( ImmutableMap.builder() @@ -212,6 +220,7 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker( .rows(ROWS) .buildMMappedIndex(); + final QueryableIndex indexMix12 = IndexBuilder.create() .tmpDir(temporaryFolder.newFolder()) @@ -260,6 +269,30 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker( .rows(ROWS) .buildMMappedIndex(); + final QueryableIndex indexArrays = + IndexBuilder.create() + .tmpDir(temporaryFolder.newFolder()) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema( + new IncrementalIndexSchema.Builder() + .withTimestampSpec(NestedDataTestUtils.AUTO_SCHEMA.getTimestampSpec()) + .withDimensionsSpec(NestedDataTestUtils.AUTO_SCHEMA.getDimensionsSpec()) + .withMetrics( + new CountAggregatorFactory("cnt") + ) + .withRollup(false) + .build() + ) + .inputSource( + ResourceInputSource.of( + NestedDataTestUtils.class.getClassLoader(), + NestedDataTestUtils.ARRAY_TYPES_DATA_FILE + ) + ) + .inputFormat(JsonInputFormat.DEFAULT) + .inputTmpDir(temporaryFolder.newFolder()) + .buildMMappedIndex(); + SpecificSegmentsQuerySegmentWalker walker = new SpecificSegmentsQuerySegmentWalker(conglomerate); walker.add( DataSegment.builder() @@ -273,7 +306,7 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker( ).add( DataSegment.builder() .dataSource(DATA_SOURCE_MIXED) - .interval(index.getDataInterval()) + .interval(indexMix11.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) .size(0) @@ -282,7 +315,7 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker( ).add( DataSegment.builder() .dataSource(DATA_SOURCE_MIXED) - .interval(index.getDataInterval()) + .interval(indexMix12.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(1)) .size(0) @@ -291,7 +324,7 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker( ).add( DataSegment.builder() .dataSource(DATA_SOURCE_MIXED_2) - .interval(index.getDataInterval()) + .interval(indexMix21.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) .size(0) @@ -306,6 +339,15 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker( .size(0) .build(), indexMix22 + ).add( + DataSegment.builder() + .dataSource(DATA_SOURCE_ARRAYS) + .version("1") + .interval(indexArrays.getDataInterval()) + .shardSpec(new LinearShardSpec(1)) + .size(0) + .build(), + indexArrays ); return walker; @@ -815,6 +857,666 @@ public void testGroupByRootSingleTypeStringMixed2SparseJsonValueNonExistentPath( ); } + @Test + public void testGroupByRootSingleTypeArrayLong() + { + cannotVectorize(); + testQuery( + "SELECT " + + "arrayLong, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayLong", "d0", ColumnType.LONG_ARRAY) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{null, 4L}, + new Object[]{Arrays.asList(1L, 2L, 3L), 4L}, + new Object[]{Arrays.asList(1L, 2L, 3L, 4L), 2L}, + new Object[]{Arrays.asList(1L, 4L), 2L}, + new Object[]{Arrays.asList(2L, 3L), 2L} + ), + RowSignature.builder() + .add("arrayLong", ColumnType.LONG_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayLongNulls() + { + cannotVectorize(); + testQuery( + "SELECT " + + "arrayLongNulls, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayLongNulls", "d0", ColumnType.LONG_ARRAY) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{null, 4L}, + new Object[]{Arrays.asList(NullHandling.defaultLongValue(), 2L, 9L), 2L}, + new Object[]{Collections.singletonList(1L), 2L}, + new Object[]{Arrays.asList(1L, NullHandling.defaultLongValue(), 3L), 2L}, + new Object[]{Arrays.asList(1L, 2L, 3L), 2L}, + new Object[]{Arrays.asList(2L, 3L), 2L} + ), + RowSignature.builder() + .add("arrayLongNulls", ColumnType.LONG_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayLongNullsFiltered() + { + cannotVectorize(); + testQuery( + "SELECT " + + "arrayLongNulls, " + + "SUM(cnt), " + + "SUM(ARRAY_LENGTH(arrayLongNulls)) " + + "FROM druid.arrays " + + "WHERE ARRAY_CONTAINS(arrayLongNulls, 1) " + + "GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayLongNulls", "d0", ColumnType.LONG_ARRAY) + ) + ) + .setVirtualColumns( + new ExpressionVirtualColumn("v0", "array_length(\"arrayLongNulls\")", ColumnType.LONG, queryFramework().macroTable()) + ) + .setDimFilter( + new ExpressionDimFilter("array_contains(\"arrayLongNulls\",1)", queryFramework().macroTable()) + ) + .setAggregatorSpecs( + aggregators( + new LongSumAggregatorFactory("a0", "cnt"), + new LongSumAggregatorFactory("a1", "v0") + ) + ) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{Collections.singletonList(1L), 2L, 2L}, + new Object[]{Arrays.asList(1L, NullHandling.defaultLongValue(), 3L), 2L, 6L}, + new Object[]{Arrays.asList(1L, 2L, 3L), 2L, 6L} + ), + RowSignature.builder() + .add("arrayLongNulls", ColumnType.LONG_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .add("EXPR$2", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayString() + { + cannotVectorize(); + testQuery( + "SELECT " + + "arrayString, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayString", "d0", ColumnType.STRING_ARRAY) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{null, 4L}, + new Object[]{Arrays.asList("a", "b"), 4L}, + new Object[]{Arrays.asList("a", "b", "c"), 2L}, + new Object[]{Arrays.asList("b", "c"), 2L}, + new Object[]{Arrays.asList("d", "e"), 2L} + ), + RowSignature.builder() + .add("arrayString", ColumnType.STRING_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayStringNulls() + { + cannotVectorize(); + testQuery( + "SELECT " + + "arrayStringNulls, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayStringNulls", "d0", ColumnType.STRING_ARRAY) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{null, 4L}, + new Object[]{Arrays.asList(null, "b"), 2L}, + new Object[]{Arrays.asList("a", "b"), 4L}, + new Object[]{Arrays.asList("b", "b"), 2L}, + new Object[]{Arrays.asList("d", null, "b"), 2L} + ), + RowSignature.builder() + .add("arrayStringNulls", ColumnType.STRING_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayStringNullsFiltered() + { + cannotVectorize(); + testQuery( + "SELECT " + + "arrayStringNulls, " + + "SUM(cnt), " + + "SUM(ARRAY_LENGTH(arrayStringNulls)) " + + "FROM druid.arrays " + + "WHERE ARRAY_CONTAINS(arrayStringNulls, 'b') " + + "GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayStringNulls", "d0", ColumnType.STRING_ARRAY) + ) + ) + .setVirtualColumns( + new ExpressionVirtualColumn("v0", "array_length(\"arrayStringNulls\")", ColumnType.LONG, queryFramework().macroTable()) + ) + .setDimFilter( + new ExpressionDimFilter("array_contains(\"arrayStringNulls\",'b')", queryFramework().macroTable()) + ) + .setAggregatorSpecs( + aggregators( + new LongSumAggregatorFactory("a0", "cnt"), + new LongSumAggregatorFactory("a1", "v0") + ) + ) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{Arrays.asList(null, "b"), 2L, 4L}, + new Object[]{Arrays.asList("a", "b"), 4L, 8L}, + new Object[]{Arrays.asList("b", "b"), 2L, 4L}, + new Object[]{Arrays.asList("d", null, "b"), 2L, 6L} + ), + RowSignature.builder() + .add("arrayStringNulls", ColumnType.STRING_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .add("EXPR$2", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayDouble() + { + cannotVectorize(); + testQuery( + "SELECT " + + "arrayDouble, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayDouble", "d0", ColumnType.DOUBLE_ARRAY) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{null, 4L}, + new Object[]{Arrays.asList(1.1, 2.2, 3.3), 4L}, + new Object[]{Arrays.asList(1.1, 3.3), 2L}, + new Object[]{Arrays.asList(2.2, 3.3, 4.0), 2L}, + new Object[]{Arrays.asList(3.3, 4.4, 5.5), 2L} + ), + RowSignature.builder() + .add("arrayDouble", ColumnType.DOUBLE_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayDoubleNulls() + { + cannotVectorize(); + testQuery( + "SELECT " + + "arrayDoubleNulls, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayDoubleNulls", "d0", ColumnType.DOUBLE_ARRAY) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{null, 4L}, + new Object[]{Arrays.asList(NullHandling.defaultDoubleValue(), 1.1), 2L}, + new Object[]{Arrays.asList(NullHandling.defaultDoubleValue(), 2.2, NullHandling.defaultDoubleValue()), 2L}, + new Object[]{Arrays.asList(1.1, 2.2, NullHandling.defaultDoubleValue()), 2L}, + new Object[]{Arrays.asList(999.0, NullHandling.defaultDoubleValue(), 5.5), 2L}, + new Object[]{Arrays.asList(999.0, 5.5, NullHandling.defaultDoubleValue()), 2L} + ), + RowSignature.builder() + .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayDoubleNullsFiltered() + { + cannotVectorize(); + testQuery( + "SELECT " + + "arrayDoubleNulls, " + + "SUM(cnt), " + + "SUM(ARRAY_LENGTH(arrayDoubleNulls)) " + + "FROM druid.arrays " + + "WHERE ARRAY_CONTAINS(arrayDoubleNulls, 2.2)" + + "GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayDoubleNulls", "d0", ColumnType.DOUBLE_ARRAY) + ) + ) + .setVirtualColumns( + new ExpressionVirtualColumn("v0", "array_length(\"arrayDoubleNulls\")", ColumnType.LONG, queryFramework().macroTable()) + ) + .setDimFilter( + new ExpressionDimFilter("array_contains(\"arrayDoubleNulls\",2.2)", queryFramework().macroTable()) + ) + .setAggregatorSpecs( + aggregators( + new LongSumAggregatorFactory("a0", "cnt"), + new LongSumAggregatorFactory("a1", "v0") + ) + ) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{Arrays.asList(NullHandling.defaultDoubleValue(), 2.2, NullHandling.defaultDoubleValue()), 2L, 6L}, + new Object[]{Arrays.asList(1.1, 2.2, NullHandling.defaultDoubleValue()), 2L, 6L} + ), + RowSignature.builder() + .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .add("EXPR$2", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayLongElement() + { + cannotVectorize(); + testQuery( + "SELECT " + + "JSON_VALUE(arrayLong, '$[1]' RETURNING BIGINT)," + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG) + ) + ) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayLong", "$[1]", "v0", ColumnType.LONG) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{NullHandling.defaultLongValue(), 4L}, + new Object[]{2L, 6L}, + new Object[]{3L, 2L}, + new Object[]{4L, 2L} + ), + RowSignature.builder() + .add("EXPR$0", ColumnType.LONG) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayLongElementFiltered() + { + cannotVectorize(); + testQuery( + "SELECT " + + "JSON_VALUE(arrayLong, '$[1]' RETURNING BIGINT)," + + "SUM(cnt) " + + "FROM druid.arrays " + + "WHERE JSON_VALUE(arrayLong, '$[1]' RETURNING BIGINT) = 2" + + "GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG) + ) + ) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayLong", "$[1]", "v0", ColumnType.LONG) + ) + .setDimFilter(new SelectorDimFilter("v0", "2", null)) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{2L, 6L} + ), + RowSignature.builder() + .add("EXPR$0", ColumnType.LONG) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayLongElementDefault() + { + cannotVectorize(); + testQuery( + "SELECT " + + "JSON_VALUE(arrayLong, '$[1]')," + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.STRING) + ) + ) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayLong", "$[1]", "v0", ColumnType.STRING) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{NullHandling.defaultStringValue(), 4L}, + new Object[]{"2", 6L}, + new Object[]{"3", 2L}, + new Object[]{"4", 2L} + ), + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayStringElement() + { + cannotVectorize(); + testQuery( + "SELECT " + + "JSON_VALUE(arrayStringNulls, '$[1]')," + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.STRING) + ) + ) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayStringNulls", "$[1]", "v0", ColumnType.STRING) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{NullHandling.defaultStringValue(), 6L}, + new Object[]{"b", 8L} + ), + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayStringElementFiltered() + { + cannotVectorize(); + testQuery( + "SELECT " + + "JSON_VALUE(arrayStringNulls, '$[1]')," + + "SUM(cnt) " + + "FROM druid.arrays " + + "WHERE JSON_VALUE(arrayStringNulls, '$[1]') = 'b'" + + "GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.STRING) + ) + ) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayStringNulls", "$[1]", "v0", ColumnType.STRING) + ) + .setDimFilter(new SelectorDimFilter("v0", "b", null)) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{"b", 8L} + ), + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayDoubleElement() + { + cannotVectorize(); + testQuery( + "SELECT " + + "JSON_VALUE(arrayDoubleNulls, '$[2]' RETURNING DOUBLE)," + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.DOUBLE) + ) + ) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayDoubleNulls", "$[2]", "v0", ColumnType.DOUBLE) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{NullHandling.defaultDoubleValue(), 12L}, + new Object[]{5.5, 2L} + ), + RowSignature.builder() + .add("EXPR$0", ColumnType.DOUBLE) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByRootSingleTypeArrayDoubleElementFiltered() + { + cannotVectorize(); + testQuery( + "SELECT " + + "JSON_VALUE(arrayDoubleNulls, '$[2]' RETURNING DOUBLE)," + + "SUM(cnt) " + + "FROM druid.arrays " + + "WHERE JSON_VALUE(arrayDoubleNulls, '$[2]' RETURNING DOUBLE) = 5.5" + + "GROUP BY 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.DOUBLE) + ) + ) + .setDimFilter(new SelectorDimFilter("v0", "5.5", null)) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayDoubleNulls", "$[2]", "v0", ColumnType.DOUBLE) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + ImmutableList.of( + new Object[]{5.5, 2L} + ), + RowSignature.builder() + .add("EXPR$0", ColumnType.DOUBLE) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test public void testGroupByJsonValues() { @@ -2754,7 +3456,7 @@ public void testGroupByAllPaths() ), ImmutableList.of( new Object[]{"[\"$\"]", 5L}, - new Object[]{"[\"$.array[1]\",\"$.array[0]\",\"$.n.x\"]", 2L} + new Object[]{"[\"$.array\",\"$.n.x\"]", 2L} ), RowSignature.builder() .add("EXPR$0", ColumnType.STRING_ARRAY) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java index f8547d10c467..6ee01f2d25aa 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java @@ -19,8 +19,6 @@ package org.apache.druid.sql.calcite.util; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -28,18 +26,18 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.ResourceInputSource; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.DoubleDimensionSchema; import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.RE; import org.apache.druid.query.DataSource; import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.InlineDataSource; @@ -70,20 +68,13 @@ import org.joda.time.DateTime; import org.joda.time.chrono.ISOChronology; -import java.io.BufferedReader; import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; -import java.util.zip.GZIPInputStream; /** * Builds a set of test data used by the Calcite query tests. The test data is @@ -615,10 +606,6 @@ public static QueryableIndex makeWikipediaIndex(File tmpDir) new LongDimensionSchema("added"), new LongDimensionSchema("deleted") ); - ArrayList dimensionNames = new ArrayList<>(dimensions.size()); - for (DimensionSchema dimension : dimensions) { - dimensionNames.add(dimension.getName()); - } return IndexBuilder .create() @@ -626,51 +613,18 @@ public static QueryableIndex makeWikipediaIndex(File tmpDir) .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) .schema(new IncrementalIndexSchema.Builder() .withRollup(false) + .withTimestampSpec(new TimestampSpec("time", null, null)) .withDimensionsSpec(new DimensionsSpec(dimensions)) .build() ) - .rows( - () -> { - final InputStream is; - try { - is = new GZIPInputStream( - // The extension ".json.gz" appears to not be included in resource bundles, so name it ".jgz"! - ClassLoader.getSystemResourceAsStream("calcite/tests/wikiticker-2015-09-12-sampled.jgz") - ); - } - catch (IOException e) { - throw new RE(e, "problem loading wikipedia dataset for tests"); - } - - ObjectMapper mapper = new DefaultObjectMapper(); - - // This method is returning an iterator over a BufferedReader, attempts are made to try to close the reader if - // exceptions occur, but this is happening in test setup and failures here should generally fail the tests, so - // leaks are not a primary concern. If anything were to actually try to mimic this code in real life, it should - // do a better job of taking care of resources. - BufferedReader lines = new BufferedReader(new InputStreamReader(is, StandardCharsets.UTF_8)); - return lines - .lines() - .map(line -> { - try { - Map map = mapper.readValue(line, Map.class); - final String time = String.valueOf(map.get("time")); - return (InputRow) new MapBasedInputRow(DateTimes.of(time), dimensionNames, map); - } - catch (JsonProcessingException e) { - final RE toThrow = new RE(e, "Problem reading line setting up wikipedia dataset for tests."); - try { - is.close(); - } - catch (IOException logged) { - toThrow.addSuppressed(logged); - } - throw toThrow; - } - }) - .iterator(); - } + .inputSource( + ResourceInputSource.of( + TestDataBuilder.class.getClassLoader(), + "calcite/tests/wikiticker-2015-09-12-sampled.json.gz" + ) ) + .inputFormat(JsonInputFormat.DEFAULT) + .inputTmpDir(new File(tmpDir, "tmpWikipedia1")) .buildMMappedIndex(); } diff --git a/sql/src/test/resources/calcite/tests/wikiticker-2015-09-12-sampled.jgz b/sql/src/test/resources/calcite/tests/wikiticker-2015-09-12-sampled.json.gz similarity index 100% rename from sql/src/test/resources/calcite/tests/wikiticker-2015-09-12-sampled.jgz rename to sql/src/test/resources/calcite/tests/wikiticker-2015-09-12-sampled.json.gz From f66f0e55383a469c2ad0a14b37e1046e3fff22b4 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 14 Feb 2023 08:49:07 -0800 Subject: [PATCH 02/15] add v4 segment test --- .../nested/NestedDataColumnSupplierTest.java | 50 ++++++++++++++++-- .../resources/nested_segment_v4/index.zip | Bin 0 -> 2062 bytes 2 files changed, 47 insertions(+), 3 deletions(-) create mode 100644 processing/src/test/resources/nested_segment_v4/index.zip diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index 3653dfaf2d98..e07ebb6d9a37 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -303,10 +303,10 @@ public void testLegacyV3ReaderFormat() throws IOException String columnName = "shipTo"; String firstValue = "Cole"; File tmpLocation = tempFolder.newFolder(); - File v3Segment = new File( - NestedDataColumnSupplierTest.class.getClassLoader().getResource("nested_segment_v3/index.zip").getFile() + CompressionUtils.unzip( + NestedDataColumnSupplierTest.class.getClassLoader().getResourceAsStream("nested_segment_v3/index.zip"), + tmpLocation ); - CompressionUtils.unzip(v3Segment, tmpLocation); try (Closer closer = Closer.create()) { QueryableIndex theIndex = closer.register(TestHelper.getTestIndexIO().loadIndex(tmpLocation)); ColumnHolder holder = theIndex.getColumnHolder(columnName); @@ -340,6 +340,50 @@ public void testLegacyV3ReaderFormat() throws IOException } } + @Test + public void testLegacyV4ReaderFormat() throws IOException + { + String columnName = "shipTo"; + // i accidentally didn't use same segment granularity for v3 and v4 segments... so they have different first value + String firstValue = "Beatty"; + File tmpLocation = tempFolder.newFolder(); + CompressionUtils.unzip( + NestedDataColumnSupplierTest.class.getClassLoader().getResourceAsStream("nested_segment_v4/index.zip"), + tmpLocation + ); + try (Closer closer = Closer.create()) { + QueryableIndex theIndex = closer.register(TestHelper.getTestIndexIO().loadIndex(tmpLocation)); + ColumnHolder holder = theIndex.getColumnHolder(columnName); + Assert.assertNotNull(holder); + Assert.assertEquals(NestedDataComplexTypeSerde.TYPE, holder.getCapabilities().toColumnType()); + + NestedDataColumnV4 v4 = closer.register((NestedDataColumnV4) holder.getColumn()); + Assert.assertNotNull(v4); + + List path = ImmutableList.of(new NestedPathField("lastName")); + ColumnHolder nestedColumnHolder = v4.getColumnHolder(path); + Assert.assertNotNull(nestedColumnHolder); + Assert.assertEquals(ColumnType.STRING, nestedColumnHolder.getCapabilities().toColumnType()); + NestedFieldLiteralDictionaryEncodedColumn nestedColumn = + (NestedFieldLiteralDictionaryEncodedColumn) nestedColumnHolder.getColumn(); + + Assert.assertNotNull(nestedColumn); + + ColumnValueSelector selector = nestedColumn.makeColumnValueSelector( + new SimpleAscendingOffset(theIndex.getNumRows()) + ); + + ColumnIndexSupplier indexSupplier = v4.getColumnIndexSupplier(path); + Assert.assertNotNull(indexSupplier); + StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + Assert.assertNotNull(valueSetIndex); + + BitmapColumnIndex indexForValue = valueSetIndex.forValue(firstValue); + Assert.assertEquals(firstValue, selector.getObject()); + Assert.assertTrue(indexForValue.computeBitmapResult(resultFactory).get(0)); + } + } + private void smokeTest(NestedDataComplexColumn column) throws IOException { SimpleAscendingOffset offset = new SimpleAscendingOffset(data.size()); diff --git a/processing/src/test/resources/nested_segment_v4/index.zip b/processing/src/test/resources/nested_segment_v4/index.zip new file mode 100644 index 0000000000000000000000000000000000000000..d2b9fb39be53db5a1bf119e9a478e1cfc0dd9042 GIT binary patch literal 2062 zcmWIWW@Zs#U|`^2nBwdgwtk9uj3f&KLm)o`0}q1?g8>NW73b#X7iWZqa56CO*!DVE zY3J+Y(h6<{MwYLP3=Cl1cf);iq+Lbk@#bbuZc_Ru7$mK`a#fc_QPATZ`%LQnUO%!3 z&0$@lr8W8P#?2Yu+HXdNTzAQuwWWc9b6w1iq9G9C zh*WkT@wi`cZ}v>{_vPt696z_^);Uxjd9}7-rt9WSL3>mDX3RE;>Dy7lbnaS#nMpVA zDQB}pwh)g6>$du?EHAj9z0UXWyxwUk+K1AZj!t>~)$3)|%3TrjRzI4f94mE2!bd;Q z``4mWF@6<}`CP}d4;h^jOYj*jmSN`BUu-ncMhiP?wN z8~Y}osO;Wq_JF7QL_mykob$Y1aarf25Z;q3XM7hDI=V*fOU<2s>T<76Hn0jvd#W8e zxZ}m$IC0%II`M0#+^e*aU$}J6-3H$iD}uylzHd#?m{G{W_jA!-msjbBax7xz+WR|; z?(tP$@%z?{tRt04hsCPSSn$s3Wthi2hc$*}4U<#@#|GvfQXga=$UO-Ea{Y(O*4!&e zmx@we3mosa(d08^wx2RZ|A*P@r_AAYM}&Gj&t_XFP08p|vwXX($@hNp)f0CcOm<9* zo&V*^hW|3VHl9CoMY+uPWvp9ybEy3>*Dk?Y-U7J*rYj7u8MqrB*|FSVtKs|ciqXU- zNo(@ybmN`eKYi8(oanu)A(CcNy)59wnKK9S%XTrm`LpGbf4qf#nP-51-`=wp?tYBh z_Rc(+_PKJ&Gs&s9{(6hP(cUK}8(JnZ4Flvrg}w_9o$4 z%*#t<;ZG0V33{40$F}(ER(~tY;CD0j{?9C*@bAX$gA%$&r``P+r}q8+im%1b|Hb|M z)OayG>^Do&|MRt9bvtF79xwM?&ilamvg%KltV#V#)P6=}O`7lJ`|0Bq6?tFZPXSqz z+Jh(OOz@Ll^JL;dFE#%coBn^Y-}NR}Z6$Mf&Hvl;W^62W7vk?dTzmHKlYhc}M|&P; zJ=Ol`W%4YgsGilPXm|7C+mGHXoPD@Q-Re!@i+A?x^ZH^-A24;@3T2UE;Sm8-7uC zip+(K-@o3T57?ahdv<(*+V}iT0hTE?*Vr59$JVIy+F4xr_w$sQ%Q?DXMQ;;H61~_=YbQhT#OF@7B5`D;)MZRyquo$(^D4624PU~ zl9rfUl3!G*msOmf2QOYgMGRW;Qms@{S&*t^rIhQNSP-0=o|~Fi;s(*HR12~KQG#*& zIkqhcSS;;iW??#qj>uh#Xd<^W`$W%i)QS; zvOXnKgo8E6cFXi_yqDkZ?O*?L(q)sf)B3kWB>KB=-(IKrZQ)WYFDV Date: Tue, 14 Feb 2023 11:17:34 -0800 Subject: [PATCH 03/15] adjustments --- .../data/input/AvroStreamInputFormatTest.java | 1 - .../data/input/AvroStreamInputRowParserTest.java | 1 - .../druid/data/input/orc/OrcReaderTest.java | 1 - .../parquet/NestedColumnParquetReaderTest.java | 4 ++-- .../input/protobuf/ProtobufInputFormatTest.java | 4 +--- .../druid/segment/DimensionHandlerUtils.java | 11 +++++++++++ .../segment/data/FrontCodedIndexedWriter.java | 6 +++--- .../data/FrontCodedIntArrayIndexedWriter.java | 6 +++--- .../nested/CompressedNestedDataComplexColumn.java | 12 ++++++++++++ ...NestedFieldLiteralDictionaryEncodedColumn.java | 11 +++++++++-- .../apache/druid/query/NestedDataTestUtils.java | 15 +++++---------- .../druid/query/scan/NestedDataScanQueryTest.java | 6 ++---- 12 files changed, 48 insertions(+), 30 deletions(-) diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java index 177766510d31..222c5472cd59 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java @@ -106,7 +106,6 @@ public class AvroStreamInputFormatTest extends InitializedNullHandlingTest private static final String TOPIC = "aTopic"; static final List DIMENSIONS = Arrays.asList(EVENT_TYPE, ID, SOME_OTHER_ID, IS_VALID); private static final List DIMENSIONS_SCHEMALESS = Arrays.asList( - "nested", SOME_OTHER_ID, "someIntArray", "someFloat", diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java index 3bcec4e3c168..b4a3514bce5e 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java @@ -86,7 +86,6 @@ public class AvroStreamInputRowParserTest private static final ZonedDateTime DATE_TIME = ZonedDateTime.of(2015, 10, 25, 19, 30, 0, 0, ZoneOffset.UTC); static final List DIMENSIONS = Arrays.asList(EVENT_TYPE, ID, SOME_OTHER_ID, IS_VALID); private static final List DIMENSIONS_SCHEMALESS = Arrays.asList( - "nested", SOME_OTHER_ID, "someIntArray", "someFloat", diff --git a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java index cf8ba19d9b74..b9f87abe8b0e 100644 --- a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java +++ b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java @@ -453,7 +453,6 @@ public void testNestedColumnSchemaless() throws IOException "middle", "list", "map", - "ts", "decimal1" ); try (CloseableIterator iterator = reader.read()) { diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java index 950039d06956..055425aa3d72 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java @@ -181,7 +181,7 @@ public void testNestedColumnSchemalessNestedTestFileNoNested() throws IOExceptio ); List rows = readAllRows(reader); - Assert.assertEquals(ImmutableList.of("dim1", "metric1", "timestamp"), rows.get(0).getDimensions()); + Assert.assertEquals(ImmutableList.of("dim1", "metric1"), rows.get(0).getDimensions()); Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); Assert.assertEquals(ImmutableList.of("d1v1"), rows.get(0).getDimension("dim1")); Assert.assertEquals("d1v1", rows.get(0).getRaw("dim1")); @@ -218,7 +218,7 @@ public void testNestedColumnSchemalessNestedTestFile() throws IOException ); List rows = readAllRows(reader); - Assert.assertEquals(ImmutableList.of("nestedData", "dim1", "metric1", "timestamp"), rows.get(0).getDimensions()); + Assert.assertEquals(ImmutableList.of("nestedData", "dim1", "metric1"), rows.get(0).getDimensions()); Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); Assert.assertEquals(ImmutableList.of("d1v1"), rows.get(0).getDimension("dim1")); Assert.assertEquals("d1v1", rows.get(0).getRaw("dim1")); diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java index 9255e1e7ebd2..c8f9b6575676 100644 --- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java +++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java @@ -243,7 +243,6 @@ public void testParseFlattenDataDiscover() throws Exception .add("someFloatColumn") .add("id") .add("someBytesColumn") - .add("timestamp") .build(), row.getDimensions() ); @@ -380,8 +379,7 @@ public void testParseNestedDataSchemaless() throws Exception "someFloatColumn", "eventType", "id", - "someBytesColumn", - "timestamp" + "someBytesColumn" ), row.getDimensions() ); 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 b509c7f058d6..8009afd654af 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java @@ -624,4 +624,15 @@ public static Float nullToZero(@Nullable Float number) { return number == null ? ZERO_FLOAT : number; } + + public static boolean isNumericNull(@Nullable Object o) + { + if (o instanceof Number) { + return false; + } + if (o instanceof String && Doubles.tryParse((String) o) != null) { + return false; + } + return true; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java index bcbe47db6244..e4888810d2aa 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java @@ -202,11 +202,11 @@ public byte[] get(int index) throws IOException startOffset = getBucketOffset(bucket - 1); } long endOffset = getBucketOffset(bucket); - int bucketSize = Ints.checkedCast(endOffset - startOffset); - if (bucketSize == 0) { + int currentBucketSize = Ints.checkedCast(endOffset - startOffset); + if (currentBucketSize == 0) { return null; } - final ByteBuffer bucketBuffer = ByteBuffer.allocate(bucketSize).order(byteOrder); + final ByteBuffer bucketBuffer = ByteBuffer.allocate(currentBucketSize).order(byteOrder); valuesOut.readFully(startOffset, bucketBuffer); bucketBuffer.clear(); final ByteBuffer valueBuffer = FrontCodedIndexed.getFromBucket(bucketBuffer, relativeIndex); diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java index 39c6f2713f6d..6b298c3472a4 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java @@ -213,11 +213,11 @@ public int[] get(int index) throws IOException startOffset = getBucketOffset(bucket - 1); } long endOffset = getBucketOffset(bucket); - int bucketSize = Ints.checkedCast(endOffset - startOffset); - if (bucketSize == 0) { + int currentBucketSize = Ints.checkedCast(endOffset - startOffset); + if (currentBucketSize == 0) { return null; } - final ByteBuffer bucketBuffer = ByteBuffer.allocate(bucketSize).order(byteOrder); + final ByteBuffer bucketBuffer = ByteBuffer.allocate(currentBucketSize).order(byteOrder); valuesOut.readFully(startOffset, bucketBuffer); bucketBuffer.clear(); return FrontCodedIntArrayIndexed.getFromBucket(bucketBuffer, relativeIndex); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java index 8608ff2c3d5f..11579216f69e 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java @@ -408,6 +408,9 @@ public DimensionSelector makeDimensionSelector( final String arrayField = getField(path.subList(0, path.size() - 1)); if (fields.indexOf(arrayField) >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); + if (elementNumber < 0) { + throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); + } DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset); return new BaseSingleValueDimensionSelector() @@ -456,6 +459,9 @@ public ColumnValueSelector makeColumnValueSelector(List path, final String arrayField = getField(path.subList(0, path.size() - 1)); if (fields.indexOf(arrayField) >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); + if (elementNumber < 0) { + throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); + } DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset); return new ColumnValueSelector() @@ -551,6 +557,9 @@ public VectorObjectSelector makeVectorObjectSelector(List path, final String arrayField = getField(path.subList(0, path.size() - 1)); if (fields.indexOf(arrayField) >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); + if (elementNumber < 0) { + throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); + } DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset); @@ -614,6 +623,9 @@ public VectorValueSelector makeVectorValueSelector(List path, Re final String arrayField = getField(path.subList(0, path.size() - 1)); if (fields.indexOf(arrayField) >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); + if (elementNumber < 0) { + throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); + } DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java index 7fcf9b3884fc..14b203ac1548 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java @@ -33,6 +33,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AbstractDimensionSelector; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.IdLookup; @@ -319,7 +320,10 @@ public long getLong() @Override public boolean isNull() { - return dictionary.get(getRowValue()) == 0; + if (dictionary.get(getRowValue()) == 0) { + return true; + } + return DimensionHandlerUtils.isNumericNull(getObject()); } @Override @@ -646,7 +650,10 @@ public boolean isNull() nullMark = nullIterator.next(); } } - return nullMark == offsetMark; + if (nullMark == offsetMark) { + return true; + } + return DimensionHandlerUtils.isNumericNull(getObject()); } @Override diff --git a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java index 0dcbd97d05b5..5d13e9977f13 100644 --- a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java +++ b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java @@ -174,8 +174,7 @@ public static Segment createSimpleNestedTestDataIncrementalIndex(TemporaryFolder tempFolder, SIMPLE_DATA_FILE, Granularities.NONE, - true, - 1000 + true ); } @@ -201,8 +200,7 @@ public static Segment createIncrementalIndexForJsonInput(TemporaryFolder tempFol tempFolder, fileName, Granularities.NONE, - true, - 1000 + true ); } @@ -210,8 +208,7 @@ public static Segment createIncrementalIndexForJsonInput( TemporaryFolder tempFolder, String file, Granularity granularity, - boolean rollup, - int maxRowCount + boolean rollup ) throws Exception { @@ -224,8 +221,7 @@ public static Segment createIncrementalIndexForJsonInput( TransformSpec.NONE, COUNT, granularity, - rollup, - maxRowCount + rollup ); } @@ -305,8 +301,7 @@ public static Segment createIncrementalIndex( TransformSpec transformSpec, AggregatorFactory[] aggregators, Granularity queryGranularity, - boolean rollup, - int maxRowCount + boolean rollup ) throws Exception { diff --git a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java index 22f2c53cb526..4759791281ae 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java @@ -483,8 +483,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscovery() throws Exception TransformSpec.NONE, NestedDataTestUtils.COUNT, Granularities.DAY, - true, - 1000 + true ) ); List segs = NestedDataTestUtils.createSegments( @@ -543,8 +542,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryArrayTypes() throws TransformSpec.NONE, aggs, Granularities.NONE, - true, - 1000 + true ) ); List segs = NestedDataTestUtils.createSegments( From 524975bf2ca12970ee841b7c646df6c267fa07ab Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sun, 19 Feb 2023 16:50:28 -0800 Subject: [PATCH 04/15] add array element indexes --- .../ArrayOfLiteralsFieldColumnWriter.java | 7 +++- .../CompressedNestedDataComplexColumn.java | 22 ++++++++++- ...balDictionaryEncodedFieldColumnWriter.java | 38 +++++++++++++++++-- ...NestedFieldLiteralColumnIndexSupplier.java | 13 ++++++- .../nested/StringFieldColumnWriter.java | 2 +- .../VariantLiteralFieldColumnWriter.java | 4 +- ...edFieldLiteralColumnIndexSupplierTest.java | 32 ++++++++++++---- 7 files changed, 101 insertions(+), 17 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java index 58d13c84dd47..259aeb510bdc 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java @@ -29,6 +29,7 @@ public class ArrayOfLiteralsFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter { + protected ArrayOfLiteralsFieldColumnWriter( String columnName, String fieldName, @@ -41,7 +42,7 @@ protected ArrayOfLiteralsFieldColumnWriter( } @Override - int[] processValue(Object value) + int[] processValue(int row, Object value) { if (value instanceof Object[]) { Object[] array = (Object[]) value; @@ -59,6 +60,10 @@ int[] processValue(Object value) newIdsWhoDis[i] = -1; } Preconditions.checkArgument(newIdsWhoDis[i] >= 0, "unknown global id [%s] for value [%s]", newIdsWhoDis[i], array[i]); + arrayElements.computeIfAbsent( + newIdsWhoDis[i], + (id) -> indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap() + ).add(row); } return newIdsWhoDis; } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java index 11579216f69e..913213b6b5e9 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java @@ -861,6 +861,24 @@ private ColumnHolder readNestedFieldColumn(String field) metadata.getBitmapSerdeFactory().getObjectStrategy(), columnBuilder.getFileMapper() ); + final Supplier> arrayElementDictionarySupplier; + final GenericIndexed arrayElementBitmaps; + if (dataBuffer.hasRemaining()) { + arrayElementDictionarySupplier = FixedIndexed.read( + dataBuffer, + NestedDataColumnSerializer.INT_TYPE_STRATEGY, + metadata.getByteOrder(), + Integer.BYTES + ); + arrayElementBitmaps = GenericIndexed.read( + dataBuffer, + metadata.getBitmapSerdeFactory().getObjectStrategy(), + columnBuilder.getFileMapper() + ); + } else { + arrayElementDictionarySupplier = null; + arrayElementBitmaps = null; + } final boolean hasNull = localDictionarySupplier.get().get(0) == 0; Supplier> columnSupplier = () -> { FixedIndexed localDict = localDictionarySupplier.get(); @@ -890,7 +908,9 @@ private ColumnHolder readNestedFieldColumn(String field) localDictionarySupplier, stringDictionarySupplier, longDictionarySupplier, - doubleDictionarySupplier + doubleDictionarySupplier, + arrayElementDictionarySupplier, + arrayElementBitmaps ), true, false diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java index 381589271b76..2f85839a82e6 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java @@ -22,6 +22,8 @@ import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap; import it.unimi.dsi.fastutil.ints.IntArrays; import it.unimi.dsi.fastutil.ints.IntIterator; import org.apache.druid.collections.bitmap.ImmutableBitmap; @@ -70,6 +72,8 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter protected final GlobalDictionaryIdLookup globalDictionaryIdLookup; protected final LocalDimensionDictionary localDictionary = new LocalDimensionDictionary(); + protected final Int2ObjectRBTreeMap arrayElements = new Int2ObjectRBTreeMap<>(); + protected FixedIndexedIntWriter intermediateValueWriter; // maybe someday we allow no bitmap indexes or multi-value columns protected int flags = DictionaryEncodedColumnPartSerde.NO_FLAGS; @@ -96,7 +100,7 @@ protected GlobalDictionaryEncodedFieldColumnWriter( /** * Perform any value conversion needed before storing the value in the */ - T processValue(Object value) + T processValue(int row, Object value) { return (T) value; } @@ -133,7 +137,7 @@ public void addValue(int row, Object val) throws IOException if (row > cursorPosition) { fillNull(row); } - final T value = processValue(val); + final T value = processValue(row, val); final int localId; // null is always 0 if (value == null) { @@ -150,7 +154,7 @@ public void addValue(int row, Object val) throws IOException private void fillNull(int row) throws IOException { - final T value = processValue(null); + final T value = processValue(row, null); final int localId = localDictionary.add(0); while (cursorPosition < row) { intermediateValueWriter.write(localId); @@ -184,6 +188,8 @@ public void writeTo(int finalRowCount, FileSmoosher smoosher) throws IOException final SegmentWriteOutMedium tmpWriteoutMedium = segmentWriteOutMedium.makeChildWriteOutMedium(); final FixedIndexedIntWriter sortedDictionaryWriter = new FixedIndexedIntWriter(tmpWriteoutMedium, true); sortedDictionaryWriter.open(); + final FixedIndexedIntWriter arrayElementDictionaryWriter = new FixedIndexedIntWriter(tmpWriteoutMedium, true); + arrayElementDictionaryWriter.open(); GenericIndexedWriter bitmapIndexWriter = new GenericIndexedWriter<>( tmpWriteoutMedium, columnName, @@ -191,6 +197,14 @@ public void writeTo(int finalRowCount, FileSmoosher smoosher) throws IOException ); bitmapIndexWriter.open(); bitmapIndexWriter.setObjectsNotSorted(); + GenericIndexedWriter arrayElementIndexWriter = new GenericIndexedWriter<>( + tmpWriteoutMedium, + columnName, + indexSpec.getBitmapSerdeFactory().getObjectStrategy() + ); + arrayElementIndexWriter.open(); + arrayElementIndexWriter.setObjectsNotSorted(); + final Int2IntOpenHashMap globalToUnsorted = localDictionary.getGlobalIdToLocalId(); final int[] unsortedToGlobal = new int[localDictionary.size()]; for (int key : globalToUnsorted.keySet()) { @@ -210,6 +224,13 @@ public void writeTo(int finalRowCount, FileSmoosher smoosher) throws IOException bitmaps[index] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); } + for (Int2ObjectMap.Entry arrayElement : arrayElements.int2ObjectEntrySet()) { + arrayElementDictionaryWriter.write(arrayElement.getIntKey()); + arrayElementIndexWriter.write( + indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(arrayElement.getValue()) + ); + } + openColumnSerializer(tmpWriteoutMedium, sortedGlobal[sortedGlobal.length - 1]); final IntIterator rows = intermediateValueWriter.getIterator(); int rowCount = 0; @@ -231,9 +252,16 @@ public void writeTo(int finalRowCount, FileSmoosher smoosher) throws IOException @Override public long getSerializedSize() throws IOException { + final long arraySize; + if (arrayElements.size() > 0) { + arraySize = arrayElementDictionaryWriter.getSerializedSize() + arrayElementIndexWriter.getSerializedSize(); + } else { + arraySize = 0; + } return 1 + Integer.BYTES + sortedDictionaryWriter.getSerializedSize() + bitmapIndexWriter.getSerializedSize() + + arraySize + getSerializedColumnSize(); } @@ -245,6 +273,10 @@ public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws I sortedDictionaryWriter.writeTo(channel, smoosher); writeColumnTo(channel, smoosher); bitmapIndexWriter.writeTo(channel, smoosher); + if (arrayElements.size() > 0) { + arrayElementDictionaryWriter.writeTo(channel, smoosher); + arrayElementIndexWriter.writeTo(channel, smoosher); + } } }; final String fieldFileName = NestedDataColumnSerializer.getInternalFileName(columnName, fieldName); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java index 22ba59835e04..2f23554f85c0 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java @@ -83,6 +83,13 @@ public class NestedFieldLiteralColumnIndexSupplier> globalLongDictionarySupplier; private final Supplier> globalDoubleDictionarySupplier; + @SuppressWarnings("FieldCanBeLocal") + @Nullable + private final GenericIndexed arrayElementBitmaps; + @SuppressWarnings("FieldCanBeLocal") + @Nullable + private final Supplier> arrayElementDictionarySupplier; + private final int adjustLongId; private final int adjustDoubleId; @@ -93,7 +100,9 @@ public NestedFieldLiteralColumnIndexSupplier( Supplier> localDictionarySupplier, Supplier globalStringDictionarySupplier, Supplier> globalLongDictionarySupplier, - Supplier> globalDoubleDictionarySupplier + Supplier> globalDoubleDictionarySupplier, + @Nullable Supplier> arrayElementDictionarySupplier, + @Nullable GenericIndexed arrayElementBitmaps ) { this.singleType = types.getSingleType(); @@ -103,6 +112,8 @@ public NestedFieldLiteralColumnIndexSupplier( this.globalStringDictionarySupplier = globalStringDictionarySupplier; this.globalLongDictionarySupplier = globalLongDictionarySupplier; this.globalDoubleDictionarySupplier = globalDoubleDictionarySupplier; + this.arrayElementDictionarySupplier = arrayElementDictionarySupplier; + this.arrayElementBitmaps = arrayElementBitmaps; this.adjustLongId = globalStringDictionarySupplier.get().size(); this.adjustDoubleId = adjustLongId + globalLongDictionarySupplier.get().size(); } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java index c9f10a269c01..5bb2f186b079 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java @@ -44,7 +44,7 @@ public StringFieldColumnWriter( } @Override - String processValue(Object value) + String processValue(int row, Object value) { if (value == null) { return null; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java index 88156202630a..c4690d8c67b4 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java @@ -45,7 +45,7 @@ public VariantLiteralFieldColumnWriter( @Override - Object processValue(Object value) + Object processValue(int row, Object value) { if (value instanceof Object[]) { Object[] array = (Object[]) value; @@ -66,7 +66,7 @@ Object processValue(Object value) } return newIdsWhoDis; } - return super.processValue(value); + return super.processValue(row, value); } @Override diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java index a4dc7bf741fc..0573b7d5fc83 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java @@ -1409,7 +1409,9 @@ public void testEnsureNoImproperSelectionFromAdjustedGlobals() throws IOExceptio dictionarySupplier, stringIndexed, longIndexed, - doubleIndexed + doubleIndexed, + null, + null ); StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); @@ -1505,7 +1507,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringSupplier() dictionarySupplier, globalStrings, globalLongs, - globalDoubles + globalDoubles, + null, + null ); } @@ -1579,7 +1583,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringWithNullsSu dictionarySupplier, globalStrings, globalLongs, - globalDoubles + globalDoubles, + null, + null ); } @@ -1650,7 +1656,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeLongSupplier() th dictionarySupplier, globalStrings, globalLongs, - globalDoubles + globalDoubles, + null, + null ); } @@ -1725,7 +1733,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeLongSupplierWithN dictionarySupplier, globalStrings, globalLongs, - globalDoubles + globalDoubles, + null, + null ); } @@ -1796,7 +1806,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeDoubleSupplier() dictionarySupplier, globalStrings, globalLongs, - globalDoubles + globalDoubles, + null, + null ); } @@ -1871,7 +1883,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeDoubleSupplierWit dictionarySupplier, globalStrings, globalLongs, - globalDoubles + globalDoubles, + null, + null ); } @@ -1957,7 +1971,9 @@ private NestedFieldLiteralColumnIndexSupplier makeVariantSupplierWithNull() t dictionarySupplier, globalStrings, globalLongs, - globalDoubles + globalDoubles, + null, + null ); } From d8e1b3f914b1608de4c7c9bb895419d826172efb Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 1 Mar 2023 16:51:44 -0800 Subject: [PATCH 05/15] fixup --- .../org/apache/druid/math/expr/EvalTest.java | 2 +- .../apache/druid/math/expr/ExprEvalTest.java | 2 +- .../druid/query/NestedDataTestUtils.java | 2 +- .../nested/NestedDataColumnSupplierTest.java | 23 ++++++++++--------- .../calcite/CalciteNestedDataQueryTest.java | 20 ++++++++-------- 5 files changed, 25 insertions(+), 24 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java b/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java index b415c48f59e5..1d8400f42d5c 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java @@ -1102,7 +1102,7 @@ public void testBestEffortOf() eval = ExprEval.bestEffortOf(new Object[] {1L, 2L, null, 3L}); Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type()); - Assert.assertArrayEquals(new Object[] {1L, 2L, NullHandling.defaultLongValue(), 3L}, (Object[]) eval.value()); + Assert.assertArrayEquals(new Object[] {1L, 2L, null, 3L}, (Object[]) eval.value()); eval = ExprEval.bestEffortOf(ImmutableList.of(1L, 2L, 3L)); Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type()); diff --git a/processing/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java b/processing/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java index 2b5bdbfa7e5b..ca807076b4d3 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java @@ -278,7 +278,7 @@ public void test_coerceListToArray() coerced = ExprEval.coerceListToArray(withNumberNulls, false); Assert.assertEquals(ExpressionType.LONG_ARRAY, coerced.lhs); - Assert.assertArrayEquals(new Object[]{1L, NullHandling.defaultLongValue(), 3L}, coerced.rhs); + Assert.assertArrayEquals(new Object[]{1L, null, 3L}, coerced.rhs); List withStringMix = ImmutableList.of(1L, "b", 3L); coerced = ExprEval.coerceListToArray(withStringMix, false); diff --git a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java index 5d13e9977f13..b1e3c7b61646 100644 --- a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java +++ b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java @@ -79,7 +79,7 @@ public class NestedDataTestUtils public static final DimensionsSpec AUTO_DISCOVERY = DimensionsSpec.builder() - .setUseNestedColumnIndexerForSchemaDiscovery(true) + .useSchemaDiscovery(true) .build(); public static final DimensionsSpec TSV_SCHEMA = diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index e07ebb6d9a37..686e0758f0ef 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -116,7 +116,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest "s", new Object[]{null, "b", "c"}, "l", - new Object[]{1L, NullHandling.defaultLongValue(), 3L}, + new Object[]{1L, null, 3L}, "d", new Object[]{2.2, 2.2} ), @@ -124,9 +124,9 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest "s", new Object[]{"b", "c"}, "l", - new Object[]{NullHandling.defaultLongValue(), NullHandling.defaultLongValue()}, + new Object[]{null, null}, "d", - new Object[]{1.1, NullHandling.defaultDoubleValue(), 2.2} + new Object[]{1.1, null, 2.2} ), TestHelper.makeMap("s", new Object[]{"a", "b", "c", "d"}, "l", new Object[]{4L, 2L, 3L}), TestHelper.makeMap("s", new Object[]{"d", "b", "c", "a"}, "d", new Object[]{1.1, 2.2}), @@ -602,14 +602,14 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException } else { Assert.assertEquals(s[1], sElementSelector.getObject()); } - if (l == null || l.length < 1) { + if (l == null || l.length < 1 || l[1] == null) { Assert.assertTrue(lElementSelector.isNull()); Assert.assertNull(lElementSelector.getObject()); } else { Assert.assertEquals(l[1], lElementSelector.getLong()); Assert.assertEquals(l[1], lElementSelector.getObject()); } - if (d == null || d.length < 1) { + if (d == null || d.length < 1 || d[1] == null) { Assert.assertTrue(dElementSelector.isNull()); Assert.assertNull(dElementSelector.getObject()); } else { @@ -655,14 +655,14 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException } else { Assert.assertEquals(s[1], sElementVector[i]); } - if (l == null || l.length < 1) { + if (l == null || l.length < 1 || l[1] == null) { Assert.assertTrue(lElementNulls[i]); Assert.assertNull(lElementObjectVector[i]); } else { Assert.assertEquals(l[1], lElementVector[i]); Assert.assertEquals(l[1], lElementObjectVector[i]); } - if (d == null || d.length < 1) { + if (d == null || d.length < 1 || d[1] == null) { Assert.assertTrue(dElementNulls[i]); Assert.assertNull(dElementObjectVector[i]); } else { @@ -704,12 +704,12 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException } else { Assert.assertEquals(s[1], sElementVector[i]); } - if (l == null || l.length < 1) { + if (l == null || l.length < 1 || l[1] == null) { Assert.assertTrue(lElementNulls[i]); } else { Assert.assertEquals(l[1], lElementVector[i]); } - if (d == null || d.length < 1) { + if (d == null || d.length < 1 || d[1] == null) { Assert.assertTrue(dElementNulls[i]); } else { Assert.assertEquals((Double) d[1], dElementVector[i], 0.0); @@ -741,10 +741,11 @@ private void testPath( Assert.assertEquals(inputValue, valueSelector.getObject()); if (ColumnType.LONG.equals(singleType)) { Assert.assertEquals(inputValue, valueSelector.getLong()); + Assert.assertFalse(path + " is not null", valueSelector.isNull()); } else if (ColumnType.DOUBLE.equals(singleType)) { Assert.assertEquals((double) inputValue, valueSelector.getDouble(), 0.0); + Assert.assertFalse(path + " is not null", valueSelector.isNull()); } - Assert.assertFalse(valueSelector.isNull()); final String theString = String.valueOf(inputValue); Assert.assertEquals(theString, dimSelector.getObject()); @@ -774,7 +775,7 @@ private void testPath( Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches()); } else { Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); + Assert.assertTrue(path, valueSelector.isNull()); Assert.assertEquals(0, dimSelector.getRow().get(0)); Assert.assertNull(dimSelector.getObject()); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index f5bb1cec1361..b0805fe8c5b5 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -921,9 +921,9 @@ public void testGroupByRootSingleTypeArrayLongNulls() ), ImmutableList.of( new Object[]{null, 4L}, - new Object[]{Arrays.asList(NullHandling.defaultLongValue(), 2L, 9L), 2L}, + new Object[]{Arrays.asList(null, 2L, 9L), 2L}, new Object[]{Collections.singletonList(1L), 2L}, - new Object[]{Arrays.asList(1L, NullHandling.defaultLongValue(), 3L), 2L}, + new Object[]{Arrays.asList(1L, null, 3L), 2L}, new Object[]{Arrays.asList(1L, 2L, 3L), 2L}, new Object[]{Arrays.asList(2L, 3L), 2L} ), @@ -974,7 +974,7 @@ public void testGroupByRootSingleTypeArrayLongNullsFiltered() ), ImmutableList.of( new Object[]{Collections.singletonList(1L), 2L, 2L}, - new Object[]{Arrays.asList(1L, NullHandling.defaultLongValue(), 3L), 2L, 6L}, + new Object[]{Arrays.asList(1L, null, 3L), 2L, 6L}, new Object[]{Arrays.asList(1L, 2L, 3L), 2L, 6L} ), RowSignature.builder() @@ -1177,11 +1177,11 @@ public void testGroupByRootSingleTypeArrayDoubleNulls() ), ImmutableList.of( new Object[]{null, 4L}, - new Object[]{Arrays.asList(NullHandling.defaultDoubleValue(), 1.1), 2L}, - new Object[]{Arrays.asList(NullHandling.defaultDoubleValue(), 2.2, NullHandling.defaultDoubleValue()), 2L}, - new Object[]{Arrays.asList(1.1, 2.2, NullHandling.defaultDoubleValue()), 2L}, - new Object[]{Arrays.asList(999.0, NullHandling.defaultDoubleValue(), 5.5), 2L}, - new Object[]{Arrays.asList(999.0, 5.5, NullHandling.defaultDoubleValue()), 2L} + new Object[]{Arrays.asList(null, 1.1), 2L}, + new Object[]{Arrays.asList(null, 2.2, null), 2L}, + new Object[]{Arrays.asList(1.1, 2.2, null), 2L}, + new Object[]{Arrays.asList(999.0, null, 5.5), 2L}, + new Object[]{Arrays.asList(999.0, 5.5, null), 2L} ), RowSignature.builder() .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY) @@ -1229,8 +1229,8 @@ public void testGroupByRootSingleTypeArrayDoubleNullsFiltered() .build() ), ImmutableList.of( - new Object[]{Arrays.asList(NullHandling.defaultDoubleValue(), 2.2, NullHandling.defaultDoubleValue()), 2L, 6L}, - new Object[]{Arrays.asList(1.1, 2.2, NullHandling.defaultDoubleValue()), 2L, 6L} + new Object[]{Arrays.asList(null, 2.2, null), 2L, 6L}, + new Object[]{Arrays.asList(1.1, 2.2, null), 2L, 6L} ), RowSignature.builder() .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY) From 00701578cc7b1ae7a9ef4cf2245bc4d4d5a5033d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 3 Mar 2023 00:51:10 -0800 Subject: [PATCH 06/15] more test --- .../nested/GlobalDictionaryIdLookup.java | 5 -- ...NestedFieldLiteralColumnIndexSupplier.java | 4 +- .../calcite/CalciteNestedDataQueryTest.java | 48 +++++++++++++++++++ 3 files changed, 50 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java index a4f28e20919e..2675b0392c39 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java @@ -59,11 +59,6 @@ public GlobalDictionaryIdLookup() this.arrayLookup.defaultReturnValue(-1); } - public int getSize() - { - return dictionarySize; - } - public void addString(@Nullable String value) { Preconditions.checkState( diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java index 2f23554f85c0..15e6b0a1fee3 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java @@ -83,10 +83,10 @@ public class NestedFieldLiteralColumnIndexSupplier> globalLongDictionarySupplier; private final Supplier> globalDoubleDictionarySupplier; - @SuppressWarnings("FieldCanBeLocal") + @SuppressWarnings({"FieldCanBeLocal", "unused"}) @Nullable private final GenericIndexed arrayElementBitmaps; - @SuppressWarnings("FieldCanBeLocal") + @SuppressWarnings({"FieldCanBeLocal", "unused"}) @Nullable private final Supplier> arrayElementDictionarySupplier; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index b0805fe8c5b5..2db9861dad01 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -857,6 +857,54 @@ public void testGroupByRootSingleTypeStringMixed2SparseJsonValueNonExistentPath( ); } + @Test + public void testJsonValueArrays() + { + testQuery( + "SELECT " + + "JSON_VALUE(arrayString, '$' RETURNING VARCHAR ARRAY), " + + "JSON_VALUE(arrayLong, '$' RETURNING BIGINT ARRAY), " + + "JSON_VALUE(arrayDouble, '$' RETURNING DOUBLE ARRAY) " + + "FROM druid.arrays", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(DATA_SOURCE_ARRAYS) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns( + new NestedFieldVirtualColumn("arrayString", "$", "v0", ColumnType.STRING_ARRAY), + new NestedFieldVirtualColumn("arrayLong", "$", "v1", ColumnType.LONG_ARRAY), + new NestedFieldVirtualColumn("arrayDouble", "$", "v2", ColumnType.DOUBLE_ARRAY) + ) + .columns("v0", "v1", "v2") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .build() + ), + ImmutableList.of( + new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, + new Object[]{null, null, null}, + new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2, 3.3, 4.0)}, + new Object[]{Arrays.asList("a", "b"), null, null}, + new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, + new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1, 3.3)}, + new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3, 4.4, 5.5)}, + new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, + new Object[]{null, null, null}, + new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2, 3.3, 4.0)}, + new Object[]{Arrays.asList("a", "b"), null, null}, + new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, + new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1, 3.3)}, + new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3, 4.4, 5.5)} + ), + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING_ARRAY) + .add("EXPR$1", ColumnType.LONG_ARRAY) + .add("EXPR$2", ColumnType.DOUBLE_ARRAY) + .build() + ); + } + @Test public void testGroupByRootSingleTypeArrayLong() { From 9ea04a3d99c361f910d732a9e9f0223d87f8fe74 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 23 Mar 2023 06:49:09 -0700 Subject: [PATCH 07/15] changes: * FrontCodedIntArrayIndexed is now like v1 of FrontCodedIndexed with totally incremental buckets * add tests for unnest and array columns * fix unnest column value selector cursor handling of null and empty arrays * refactor a bunch of stuff per review --- .../data/input/impl/DelimitedInputFormat.java | 12 - .../data/input/impl/JsonInputFormat.java | 9 - .../data/input/impl/MapInputRowParser.java | 7 +- .../org/apache/druid/math/expr/ExprEval.java | 21 +- .../expression/NestedDataExpressions.java | 13 +- .../druid/query/metadata/SegmentAnalyzer.java | 27 +- .../segment/NestedDataColumnIndexer.java | 97 +- .../druid/segment/NestedDataColumnMerger.java | 94 +- .../UnnestColumnValueSelectorCursor.java | 33 +- .../druid/segment/UnnestStorageAdapter.java | 55 +- .../data/FrontCodedIntArrayIndexed.java | 79 +- .../data/FrontCodedIntArrayIndexedWriter.java | 81 +- ...riter.java => ArrayFieldColumnWriter.java} | 22 +- .../CompressedNestedDataComplexColumn.java | 14 +- .../GlobalDictionarySortedCollector.java | 16 + .../nested/NestedDataColumnSerializer.java | 107 +- .../nested/NestedDataColumnSupplier.java | 8 +- .../segment/nested/NestedDataColumnV3.java | 2 +- .../segment/nested/NestedDataColumnV4.java | 2 +- .../segment/nested/NestedDataColumnV5.java | 2 +- ...va => NestedFieldColumnIndexSupplier.java} | 10 +- ...> NestedFieldDictionaryEncodedColumn.java} | 24 +- ...TypeInfo.java => NestedFieldTypeInfo.java} | 12 +- .../nested/StructuredDataProcessor.java | 114 +- ...ter.java => VariantFieldColumnWriter.java} | 20 +- .../virtual/NestedFieldVirtualColumn.java | 3 +- .../druid/query/NestedDataTestUtils.java | 37 +- .../query/scan/NestedDataScanQueryTest.java | 11 +- .../apache/druid/segment/IndexBuilder.java | 4 +- .../UnnestColumnValueSelectorCursorTest.java | 91 +- .../segment/data/FrontCodedIndexedTest.java | 22 +- .../data/FrontCodedIntArrayIndexedTest.java | 68 +- .../nested/NestedDataColumnSupplierTest.java | 18 +- ...> NestedFieldColumnIndexSupplierTest.java} | 130 +- ...va => NestedFieldColumnSelectorsTest.java} | 9 +- ...Test.java => NestedFieldTypeInfoTest.java} | 36 +- .../resources/nested-array-test-data.json | 12 +- .../org/apache/druid/cli/DumpSegment.java | 6 +- .../sql/calcite/BaseCalciteQueryTest.java | 18 +- .../sql/calcite/CalciteArraysQueryTest.java | 8 +- .../calcite/CalciteNestedDataQueryTest.java | 1628 +++++++++++------ .../sql/calcite/util/TestDataBuilder.java | 11 +- 42 files changed, 1830 insertions(+), 1163 deletions(-) rename processing/src/main/java/org/apache/druid/segment/nested/{ArrayOfLiteralsFieldColumnWriter.java => ArrayFieldColumnWriter.java} (76%) rename processing/src/main/java/org/apache/druid/segment/nested/{NestedFieldLiteralColumnIndexSupplier.java => NestedFieldColumnIndexSupplier.java} (99%) rename processing/src/main/java/org/apache/druid/segment/nested/{NestedFieldLiteralDictionaryEncodedColumn.java => NestedFieldDictionaryEncodedColumn.java} (96%) rename processing/src/main/java/org/apache/druid/segment/nested/{NestedLiteralTypeInfo.java => NestedFieldTypeInfo.java} (95%) rename processing/src/main/java/org/apache/druid/segment/nested/{VariantLiteralFieldColumnWriter.java => VariantFieldColumnWriter.java} (80%) rename processing/src/test/java/org/apache/druid/segment/nested/{NestedFieldLiteralColumnIndexSupplierTest.java => NestedFieldColumnIndexSupplierTest.java} (92%) rename processing/src/test/java/org/apache/druid/segment/nested/{NestedFieldLiteralColumnSelectorsTest.java => NestedFieldColumnSelectorsTest.java} (98%) rename processing/src/test/java/org/apache/druid/segment/nested/{NestedLiteralTypeInfoTest.java => NestedFieldTypeInfoTest.java} (74%) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java b/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java index 7df586d8c3e6..d409f7ef12ed 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java @@ -31,7 +31,6 @@ import javax.annotation.Nullable; import java.io.File; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -43,17 +42,6 @@ public class DelimitedInputFormat extends FlatTextInputFormat { public static final String TYPE_KEY = "tsv"; - public static DelimitedInputFormat ofColumns(String... columns) - { - return new DelimitedInputFormat( - Arrays.asList(columns), - null, - null, - false, - false, - 0 - ); - } private static final String DEFAULT_DELIMITER = "\t"; @JsonCreator diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java b/processing/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java index e886aeb2bfbb..200c621e1389 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java @@ -40,15 +40,6 @@ public class JsonInputFormat extends NestedInputFormat { public static final String TYPE_KEY = "json"; - - public static final JsonInputFormat DEFAULT = new JsonInputFormat( - JSONPathSpec.DEFAULT, - null, - null, - null, - null - ); - private final Map featureSpec; private final ObjectMapper objectMapper; private final boolean keepNullColumns; diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java b/processing/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java index f028131f8720..47ef09e527b6 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java @@ -35,6 +35,7 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Set; public class MapInputRowParser implements InputRowParser> { @@ -85,10 +86,12 @@ private static List findDimensions( Map rawInputRow ) { + final String timestampColumn = timestampSpec.getTimestampColumn(); + final Set exclusions = dimensionsSpec.getDimensionExclusions(); if (dimensionsSpec.isIncludeAllDimensions()) { LinkedHashSet dimensions = new LinkedHashSet<>(dimensionsSpec.getDimensionNames()); for (String field : rawInputRow.keySet()) { - if (timestampSpec.getTimestampColumn().equals(field) || dimensionsSpec.getDimensionExclusions().contains(field)) { + if (timestampColumn.equals(field) || exclusions.contains(field)) { continue; } dimensions.add(field); @@ -100,7 +103,7 @@ private static List findDimensions( } else { List dimensions = new ArrayList<>(); for (String field : rawInputRow.keySet()) { - if (timestampSpec.getTimestampColumn().equals(field) || dimensionsSpec.getDimensionExclusions().contains(field)) { + if (timestampColumn.equals(field) || exclusions.contains(field)) { continue; } dimensions.add(field); diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java index 062ef9805de4..9c105af58ffa 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java @@ -370,6 +370,18 @@ public static ExprEval ofComplex(ExpressionType outputType, @Nullable Object val return new ComplexExprEval(outputType, value); } + public static ExprEval bestEffortArray(@Nullable List theList) + { + // do not convert empty lists to arrays with a single null element here, because that should have been done + // by the selectors preparing their ObjectBindings if necessary. If we get to this point it was legitimately + // empty + NonnullPair coerced = coerceListToArray(theList, false); + if (coerced == null) { + return bestEffortOf(null); + } + return ofArray(coerced.lhs, coerced.rhs); + } + /** * Examine java type to find most appropriate expression type */ @@ -468,14 +480,7 @@ public static ExprEval bestEffortOf(@Nullable Object val) if (val instanceof List || val instanceof Object[]) { final List theList = val instanceof List ? ((List) val) : Arrays.asList((Object[]) val); - // do not convert empty lists to arrays with a single null element here, because that should have been done - // by the selectors preparing their ObjectBindings if necessary. If we get to this point it was legitimately - // empty - NonnullPair coerced = coerceListToArray(theList, false); - if (coerced == null) { - return bestEffortOf(null); - } - return ofArray(coerced.lhs, coerced.rhs); + return bestEffortArray(theList); } // in 'best effort' mode, we couldn't possibly use byte[] as a complex or anything else useful without type diff --git a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java index 57bcf85b7c39..2eabe194dd84 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java +++ b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java @@ -480,22 +480,23 @@ public Expr apply(List args) final StructuredDataProcessor processor = new StructuredDataProcessor() { @Override - public StructuredDataProcessor.ProcessedLiteral processLiteralField(ArrayList fieldPath, @Nullable Object fieldValue) + public ProcessedValue processField(ArrayList fieldPath, @Nullable Object fieldValue) { // do nothing, we only want the list of fields returned by this processor - return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; + return ProcessedValue.NULL_LITERAL; } @Nullable @Override - public ProcessedLiteral processArrayOfLiteralsField( + public ProcessedValue processArrayField( ArrayList fieldPath, - @Nullable Object maybeArrayOfLiterals + @Nullable List array ) { - ExprEval eval = ExprEval.bestEffortOf(maybeArrayOfLiterals); + // we only want to return a non-null value here if the value is an array of primitive values + ExprEval eval = ExprEval.bestEffortArray(array); if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) { - return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; + return ProcessedValue.NULL_LITERAL; } return null; } 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 6834d039d03a..ddd0f8f2b4d2 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 @@ -135,9 +135,8 @@ public Map analyze(Segment segment) } break; case ARRAY: - // todo (clint): this is wack, but works for now because arrays are always nested complex columns... final ColumnHolder arrayHolder = index != null ? index.getColumnHolder(columnName) : null; - analysis = analyzeComplexColumn(capabilities, numRows, arrayHolder); + analysis = analyzeArrayColumn(capabilities, numRows, arrayHolder); break; case COMPLEX: final ColumnHolder columnHolder = index != null ? index.getColumnHolder(columnName) : null; @@ -390,4 +389,28 @@ private ColumnAnalysis analyzeComplexColumn( ); } } + + private ColumnAnalysis analyzeArrayColumn( + @Nullable final ColumnCapabilities capabilities, + final int numCells, + @Nullable final ColumnHolder columnHolder + ) + { + final TypeSignature typeSignature = capabilities == null ? ColumnType.UNKNOWN_COMPLEX : capabilities; + final String typeName = typeSignature.getComplexTypeName(); + final boolean hasMultipleValues = capabilities != null && capabilities.hasMultipleValues().isTrue(); + final boolean hasNulls = capabilities != null && capabilities.hasNulls().isMaybeTrue(); + + return new ColumnAnalysis( + ColumnTypeFactory.ofType(typeSignature), + typeName, + hasMultipleValues, + hasNulls, + 0L, + null, + null, + null, + null + ); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java index 0e40982ebdb9..d3a9503dce11 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java @@ -23,6 +23,7 @@ import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.UOE; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.dimension.DimensionSpec; @@ -36,7 +37,7 @@ import org.apache.druid.segment.nested.GlobalDictionarySortedCollector; import org.apache.druid.segment.nested.GlobalDimensionDictionary; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; -import org.apache.druid.segment.nested.NestedLiteralTypeInfo; +import org.apache.druid.segment.nested.NestedFieldTypeInfo; import org.apache.druid.segment.nested.NestedPathFinder; import org.apache.druid.segment.nested.NestedPathPart; import org.apache.druid.segment.nested.StructuredData; @@ -45,6 +46,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.SortedMap; @@ -54,7 +56,7 @@ public class NestedDataColumnIndexer implements DimensionIndexer fieldIndexers = new TreeMap<>(); + protected SortedMap fieldIndexers = new TreeMap<>(); protected final GlobalDimensionDictionary globalDictionary = new GlobalDimensionDictionary(); int estimatedFieldKeySize = 0; @@ -62,41 +64,40 @@ public class NestedDataColumnIndexer implements DimensionIndexer processLiteralField(ArrayList fieldPath, @Nullable Object fieldValue) + public ProcessedValue processField(ArrayList fieldPath, @Nullable Object fieldValue) { // null value is always added to the global dictionary as id 0, so we can ignore them here if (fieldValue != null) { - // why not final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath); ExprEval eval = ExprEval.bestEffortOf(fieldValue); - LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName); + FieldIndexer fieldIndexer = fieldIndexers.get(fieldName); if (fieldIndexer == null) { estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName); - fieldIndexer = new LiteralFieldIndexer(globalDictionary); + fieldIndexer = new FieldIndexer(globalDictionary); fieldIndexers.put(fieldName, fieldIndexer); } return fieldIndexer.processValue(eval); } - return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; + return ProcessedValue.NULL_LITERAL; } @Nullable @Override - public ProcessedLiteral processArrayOfLiteralsField( + public ProcessedValue processArrayField( ArrayList fieldPath, - Object maybeArrayOfLiterals + @Nullable List array ) { - final ExprEval maybeLiteralArray = ExprEval.bestEffortOf(maybeArrayOfLiterals); - if (maybeLiteralArray.type().isArray() && maybeLiteralArray.type().getElementType().isPrimitive()) { + final ExprEval eval = ExprEval.bestEffortArray(array); + if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) { final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath); - LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName); + FieldIndexer fieldIndexer = fieldIndexers.get(fieldName); if (fieldIndexer == null) { estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName); - fieldIndexer = new LiteralFieldIndexer(globalDictionary); + fieldIndexer = new FieldIndexer(globalDictionary); fieldIndexers.put(fieldName, fieldIndexer); } - return fieldIndexer.processValue(maybeLiteralArray); + return fieldIndexer.processValue(eval); } return null; } @@ -175,9 +176,14 @@ public DimensionSelector makeDimensionSelector( final int dimIndex = desc.getIndex(); final ColumnValueSelector rootLiteralSelector = getRootLiteralValueSelector(currEntry, dimIndex); if (rootLiteralSelector != null) { - final LiteralFieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); - if (root.getTypes().getSingleType().isArray()) { - throw new UnsupportedOperationException("Not supported"); + final FieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); + final ColumnType rootType = root.getTypes().getSingleType(); + if (rootType.isArray()) { + throw new UOE( + "makeDimensionSelector is not supported, column [%s] is [%s] typed and should only use makeColumnValueSelector", + spec.getOutputName(), + rootType + ); } return new BaseSingleValueDimensionSelector() { @@ -199,7 +205,12 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) } }; } - throw new UnsupportedOperationException("Not supported"); + // column has nested data or is of mixed root type, cannot use + throw new UOE( + "makeDimensionSelector is not supported, column [%s] is [%s] typed and should only use makeColumnValueSelector", + spec.getOutputName(), + NestedDataComplexTypeSerde.TYPE + ); } @Override @@ -246,7 +257,7 @@ public Class classOfObject() public ColumnCapabilities getColumnCapabilities() { if (fieldIndexers.size() == 1 && fieldIndexers.containsKey(NestedPathFinder.JSON_PATH_ROOT)) { - LiteralFieldIndexer rootField = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); + FieldIndexer rootField = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); if (rootField.isSingleType()) { return ColumnCapabilitiesImpl.createDefault() .setType(rootField.getTypes().getSingleType()) @@ -299,7 +310,7 @@ public Object convertUnsortedEncodedKeyComponentToActualList(StructuredData key) @Override public ColumnValueSelector convertUnsortedValuesToSorted(ColumnValueSelector selectorWithUnsortedValues) { - final LiteralFieldIndexer rootIndexer = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); + final FieldIndexer rootIndexer = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); if (fieldIndexers.size() == 1 && rootIndexer != null && rootIndexer.isSingleType()) { // for root only literals, makeColumnValueSelector and makeDimensionSelector automatically unwrap StructuredData // we need to do the opposite here, wrapping selector values with a StructuredData so that they are consistently @@ -364,9 +375,9 @@ public void fillBitmapsFromUnsortedEncodedKeyComponent( throw new UnsupportedOperationException("Not supported"); } - public void mergeFields(SortedMap mergedFields) + public void mergeFields(SortedMap mergedFields) { - for (Map.Entry entry : fieldIndexers.entrySet()) { + for (Map.Entry entry : fieldIndexers.entrySet()) { // skip adding the field if no types are in the set, meaning only null values have been processed if (!entry.getValue().getTypes().isEmpty()) { mergedFields.put(entry.getKey(), entry.getValue().getTypes()); @@ -388,7 +399,7 @@ private ColumnValueSelector getRootLiteralValueSelector( if (fieldIndexers.size() > 1) { return null; } - final LiteralFieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); + final FieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); if (root == null || !root.isSingleType()) { return null; } @@ -445,7 +456,7 @@ public Object getObject() if (0 <= dimIndex && dimIndex < dims.length) { final StructuredData data = (StructuredData) dims[dimIndex]; if (data != null) { - return ExprEval.bestEffortOf(data.getValue()).value(); + return ExprEval.bestEffortOf(data.getValue()).valueOrDefault(); } } @@ -460,18 +471,18 @@ public Class classOfObject() }; } - static class LiteralFieldIndexer + static class FieldIndexer { private final GlobalDimensionDictionary globalDimensionDictionary; - private final NestedLiteralTypeInfo.MutableTypeSet typeSet; + private final NestedFieldTypeInfo.MutableTypeSet typeSet; - LiteralFieldIndexer(GlobalDimensionDictionary globalDimensionDictionary) + FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary) { this.globalDimensionDictionary = globalDimensionDictionary; - this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet(); + this.typeSet = new NestedFieldTypeInfo.MutableTypeSet(); } - private StructuredDataProcessor.ProcessedLiteral processValue(ExprEval eval) + private StructuredDataProcessor.ProcessedValue processValue(ExprEval eval) { final ColumnType columnType = ExpressionType.toColumnType(eval.type()); int sizeEstimate; @@ -479,33 +490,39 @@ private StructuredDataProcessor.ProcessedLiteral processValue(ExprEval eva case LONG: typeSet.add(ColumnType.LONG); sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong()); - return new StructuredDataProcessor.ProcessedLiteral<>(eval.asLong(), sizeEstimate); + return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate); case DOUBLE: typeSet.add(ColumnType.DOUBLE); sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble()); - return new StructuredDataProcessor.ProcessedLiteral<>(eval.asDouble(), sizeEstimate); + return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate); case ARRAY: - // skip empty arrays for now, they will always be called 'string' arrays, which isn't very helpful here since - // it will pollute the type set - Preconditions.checkNotNull(columnType.getElementType(), "Array element type must not be null"); + // sanity check, this should never happen + Preconditions.checkNotNull( + columnType.getElementType(), + "Array type [%s] for value [%s] missing element type, how did this possibly happen?", + eval.type(), + eval.valueOrDefault() + ); switch (columnType.getElementType().getType()) { case LONG: typeSet.add(ColumnType.LONG_ARRAY); final Object[] longArray = eval.asArray(); sizeEstimate = globalDimensionDictionary.addLongArray(longArray); - return new StructuredDataProcessor.ProcessedLiteral<>(longArray, sizeEstimate); + return new StructuredDataProcessor.ProcessedValue<>(longArray, sizeEstimate); case DOUBLE: typeSet.add(ColumnType.DOUBLE_ARRAY); final Object[] doubleArray = eval.asArray(); sizeEstimate = globalDimensionDictionary.addDoubleArray(doubleArray); - return new StructuredDataProcessor.ProcessedLiteral<>(doubleArray, sizeEstimate); + return new StructuredDataProcessor.ProcessedValue<>(doubleArray, sizeEstimate); case STRING: final Object[] stringArray = eval.asArray(); - if (!Arrays.stream(stringArray).allMatch(Objects::isNull)) { + // empty arrays and arrays with all nulls are detected as string arrays, but dont count them as part of + // the type set + if (stringArray.length > 0 && !Arrays.stream(stringArray).allMatch(Objects::isNull)) { typeSet.add(ColumnType.STRING_ARRAY); } sizeEstimate = globalDimensionDictionary.addStringArray(stringArray); - return new StructuredDataProcessor.ProcessedLiteral<>(stringArray, sizeEstimate); + return new StructuredDataProcessor.ProcessedValue<>(stringArray, sizeEstimate); default: throw new IAE("Unhandled type: %s", columnType); } @@ -514,11 +531,11 @@ private StructuredDataProcessor.ProcessedLiteral processValue(ExprEval eva typeSet.add(ColumnType.STRING); final String asString = eval.asString(); sizeEstimate = globalDimensionDictionary.addStringValue(asString); - return new StructuredDataProcessor.ProcessedLiteral<>(eval.asString(), sizeEstimate); + return new StructuredDataProcessor.ProcessedValue<>(eval.asString(), sizeEstimate); } } - public NestedLiteralTypeInfo.MutableTypeSet getTypes() + public NestedFieldTypeInfo.MutableTypeSet getTypes() { return typeSet; } diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java index 7199d538d660..436514a6bbe1 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java @@ -39,7 +39,7 @@ import org.apache.druid.segment.nested.GlobalDictionarySortedCollector; import org.apache.druid.segment.nested.NestedDataColumnSerializer; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; -import org.apache.druid.segment.nested.NestedLiteralTypeInfo; +import org.apache.druid.segment.nested.NestedFieldTypeInfo; import org.apache.druid.segment.serde.ComplexColumnPartSerde; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; @@ -104,7 +104,7 @@ public void writeMergedValueDictionary(List adapters) throws I final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()]; final Iterable[] sortedArrayLookups = new Iterable[adapters.size()]; - final SortedMap mergedFields = new TreeMap<>(); + final SortedMap mergedFields = new TreeMap<>(); for (int i = 0; i < adapters.size(); i++) { final IndexableAdapter adapter = adapters.get(i); @@ -114,13 +114,10 @@ public void writeMergedValueDictionary(List adapters) throws I } else if (adapter instanceof QueryableIndexIndexableAdapter) { dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields); } else { - throw new ISE("Unable to merge columns of unsupported adapter %s", adapter.getClass()); + throw new ISE("Unable to merge columns of unsupported adapter [%s]", adapter.getClass()); } - boolean allNulls = dimValues == null || allNull(dimValues.getSortedStrings()) && - allNull(dimValues.getSortedLongs()) && - allNull(dimValues.getSortedDoubles()) && - dimValues.getArrayCardinality() == 0; + boolean allNulls = dimValues == null || dimValues.allNull(); sortedLookup = dimValues; if (!allNulls) { sortedLookups[i] = dimValues.getSortedStrings(); @@ -158,43 +155,46 @@ public void writeMergedValueDictionary(List adapters) throws I int doubleCardinality; int arrayCardinality; if (numMergeIndex == 1) { - defaultSerializer.serializeStringDictionary(sortedLookup.getSortedStrings()); - defaultSerializer.serializeLongDictionary(sortedLookup.getSortedLongs()); - defaultSerializer.serializeDoubleDictionary(sortedLookup.getSortedDoubles()); - defaultSerializer.serializeArrayDictionary(() -> new ArrayDictionaryMergingIterator( - sortedArrayLookups, - defaultSerializer.getGlobalLookup() - )); + defaultSerializer.serializeDictionaries( + sortedLookup.getSortedStrings(), + sortedLookup.getSortedLongs(), + sortedLookup.getSortedDoubles(), + () -> new ArrayDictionaryMergingIterator( + sortedArrayLookups, + defaultSerializer.getGlobalLookup() + ) + ); stringCardinality = sortedLookup.getStringCardinality(); longCardinality = sortedLookup.getLongCardinality(); doubleCardinality = sortedLookup.getDoubleCardinality(); arrayCardinality = sortedLookup.getArrayCardinality(); } else { - SimpleDictionaryMergingIterator dictionaryMergeIterator = new SimpleDictionaryMergingIterator<>( + final SimpleDictionaryMergingIterator stringIterator = new SimpleDictionaryMergingIterator<>( sortedLookups, STRING_MERGING_COMPARATOR ); - SimpleDictionaryMergingIterator longDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>( + final SimpleDictionaryMergingIterator longIterator = new SimpleDictionaryMergingIterator<>( sortedLongLookups, LONG_MERGING_COMPARATOR ); - SimpleDictionaryMergingIterator doubleDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>( + final SimpleDictionaryMergingIterator doubleIterator = new SimpleDictionaryMergingIterator<>( sortedDoubleLookups, DOUBLE_MERGING_COMPARATOR ); - defaultSerializer.serializeStringDictionary(() -> dictionaryMergeIterator); - defaultSerializer.serializeLongDictionary(() -> longDictionaryMergeIterator); - defaultSerializer.serializeDoubleDictionary(() -> doubleDictionaryMergeIterator); - - final ArrayDictionaryMergingIterator arrayDictionaryMergingIterator = new ArrayDictionaryMergingIterator( + final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator( sortedArrayLookups, defaultSerializer.getGlobalLookup() ); - defaultSerializer.serializeArrayDictionary(() -> arrayDictionaryMergingIterator); - stringCardinality = dictionaryMergeIterator.getCardinality(); - longCardinality = longDictionaryMergeIterator.getCardinality(); - doubleCardinality = doubleDictionaryMergeIterator.getCardinality(); - arrayCardinality = arrayDictionaryMergingIterator.getCardinality(); + defaultSerializer.serializeDictionaries( + () -> stringIterator, + () -> longIterator, + () -> doubleIterator, + () -> arrayIterator + ); + stringCardinality = stringIterator.getCardinality(); + longCardinality = longIterator.getCardinality(); + doubleCardinality = doubleIterator.getCardinality(); + arrayCardinality = arrayIterator.getCardinality(); } log.debug( @@ -216,7 +216,7 @@ public void writeMergedValueDictionary(List adapters) throws I @Nullable private GlobalDictionarySortedCollector getSortedIndexFromIncrementalAdapter( IncrementalIndexAdapter adapter, - SortedMap mergedFields + SortedMap mergedFields ) { final IncrementalIndex index = adapter.getIncrementalIndex(); @@ -232,7 +232,7 @@ private GlobalDictionarySortedCollector getSortedIndexFromIncrementalAdapter( @Nullable private GlobalDictionarySortedCollector getSortedIndexesFromQueryableAdapter( QueryableIndexIndexableAdapter adapter, - SortedMap mergedFields + SortedMap mergedFields ) { final ColumnHolder columnHolder = adapter.getQueryableIndex().getColumnHolder(name); @@ -252,7 +252,7 @@ private GlobalDictionarySortedCollector getSortedIndexesFromQueryableAdapter( } private GlobalDictionarySortedCollector getSortedIndexFromV1QueryableAdapterNestedColumn( - SortedMap mergedFields, + SortedMap mergedFields, BaseColumn col ) { @@ -261,10 +261,10 @@ private GlobalDictionarySortedCollector getSortedIndexFromV1QueryableAdapterNest closer.register(column); for (int i = 0; i < column.getFields().size(); i++) { String fieldPath = column.getFields().get(i); - NestedLiteralTypeInfo.TypeSet types = column.getFieldInfo().getTypes(i); + NestedFieldTypeInfo.TypeSet types = column.getFieldInfo().getTypes(i); mergedFields.compute(fieldPath, (k, v) -> { if (v == null) { - return new NestedLiteralTypeInfo.MutableTypeSet(types.getByteValue()); + return new NestedFieldTypeInfo.MutableTypeSet(types.getByteValue()); } return v.merge(types.getByteValue()); }); @@ -311,28 +311,29 @@ public ColumnDescriptor makeColumnDescriptor() return descriptorBuilder.build(); } - private boolean allNull(Indexed dimValues) - { - for (int i = 0, size = dimValues.size(); i < size; i++) { - if (dimValues.get(i) != null) { - return false; - } - } - return true; - } - public static class ArrayDictionaryMergingIterator implements Iterator { private static final Comparator> PEEKING_ITERATOR_COMPARATOR = (lhs, rhs) -> FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR.compare(lhs.peek(), rhs.peek()); protected final PriorityQueue> pQueue; + private final Iterable[] dimValueLookups; + private final GlobalDictionaryIdLookup idLookup; + protected int counter; + private boolean initialized; public ArrayDictionaryMergingIterator(Iterable[] dimValueLookups, GlobalDictionaryIdLookup idLookup) { - pQueue = new PriorityQueue<>(PEEKING_ITERATOR_COMPARATOR); + this.pQueue = new PriorityQueue<>(PEEKING_ITERATOR_COMPARATOR); + this.dimValueLookups = dimValueLookups; + this.idLookup = idLookup; + } + private void initialize() + { + // we initialize lazily because the global id lookup might not be populated because the lower dictionary mergers + // have not been iterated yet, so wait until we iterate this one while serializing to populate it for (Iterable dimValueLookup : dimValueLookups) { if (dimValueLookup == null) { continue; @@ -344,17 +345,24 @@ public ArrayDictionaryMergingIterator(Iterable[] dimValueLookups, Glob pQueue.add(iter); } } + initialized = true; } @Override public boolean hasNext() { + if (!initialized) { + initialize(); + } return !pQueue.isEmpty(); } @Override public int[] next() { + if (!initialized) { + initialize(); + } PeekingIterator smallest = pQueue.remove(); if (smallest == null) { throw new NoSuchElementException(); diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java index 9179d55dc1f8..40a3aeeda22f 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java @@ -187,10 +187,7 @@ public boolean isNull() @Override public Object getObject() { - if (!unnestListForCurrentRow.isEmpty()) { - return unnestListForCurrentRow.get(index); - } - return null; + return unnestListForCurrentRow.get(index); } @Override @@ -279,7 +276,7 @@ private void getNextRow() { currentVal = this.columnValueSelector.getObject(); if (currentVal == null) { - unnestListForCurrentRow = Collections.singletonList(null); + unnestListForCurrentRow = Collections.emptyList(); } else if (currentVal instanceof List) { unnestListForCurrentRow = (List) currentVal; } else if (currentVal instanceof Object[]) { @@ -296,9 +293,25 @@ private void getNextRow() private void initialize() { getNextRow(); + if (unnestListForCurrentRow.isEmpty()) { + moveToNextNonEmptyRow(); + } needInitialization = false; } + private void moveToNextNonEmptyRow() + { + index = 0; + do { + baseCursor.advance(); + if (!baseCursor.isDone()) { + getNextRow(); + } else { + return; + } + } while (unnestListForCurrentRow.isEmpty()); + } + /** * This advances the cursor to move to the next element to be unnested. * When the last element in a row is unnested, it is also responsible @@ -307,14 +320,8 @@ private void initialize() */ private void advanceAndUpdate() { - if (unnestListForCurrentRow.isEmpty() || index >= unnestListForCurrentRow.size() - 1) { - index = 0; - baseCursor.advance(); - if (!baseCursor.isDone()) { - getNextRow(); - } - } else { - index++; + if (++index >= unnestListForCurrentRow.size()) { + moveToNextNonEmptyRow(); } } } diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index f3d3df593d96..12acc586226b 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -75,7 +75,7 @@ public StorageAdapter getBaseAdapter() public UnnestStorageAdapter( final StorageAdapter baseAdapter, final VirtualColumn unnestColumn, - final DimFilter unnestFilter + @Nullable final DimFilter unnestFilter ) { this.baseAdapter = baseAdapter; @@ -100,9 +100,7 @@ public Sequence makeCursors( unnestFilter != null ? unnestFilter.toFilter() : null, virtualColumns, inputColumn, - inputColumn == null || virtualColumns.exists(inputColumn) - ? null - : baseAdapter.getColumnCapabilities(inputColumn) + inputColumn == null ? null : virtualColumns.getColumnCapabilitiesWithFallback(baseAdapter, inputColumn) ); final Sequence baseCursorSequence = baseAdapter.makeCursors( @@ -118,37 +116,29 @@ public Sequence makeCursors( baseCursorSequence, cursor -> { Objects.requireNonNull(cursor); - Cursor retVal = cursor; - ColumnCapabilities capabilities = unnestColumn.capabilities( + final ColumnCapabilities capabilities = unnestColumn.capabilities( cursor.getColumnSelectorFactory(), unnestColumn.getOutputName() ); - if (capabilities != null) { - if (!capabilities.isArray() && capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) { - retVal = new UnnestDimensionCursor( - retVal, - retVal.getColumnSelectorFactory(), - unnestColumn, - outputColumnName - ); - } else { - retVal = new UnnestColumnValueSelectorCursor( - retVal, - retVal.getColumnSelectorFactory(), - unnestColumn, - outputColumnName - ); - } + final Cursor unnestCursor; + + if (useDimensionCursor(capabilities)) { + unnestCursor = new UnnestDimensionCursor( + cursor, + cursor.getColumnSelectorFactory(), + unnestColumn, + outputColumnName + ); } else { - retVal = new UnnestColumnValueSelectorCursor( - retVal, - retVal.getColumnSelectorFactory(), + unnestCursor = new UnnestColumnValueSelectorCursor( + cursor, + cursor.getColumnSelectorFactory(), unnestColumn, outputColumnName ); } return PostJoinCursor.wrap( - retVal, + unnestCursor, virtualColumns, filterPair.rhs ); @@ -478,4 +468,17 @@ private static boolean filterMapsOverMultiValueStrings(final Filter filter) || filter instanceof BoundFilter; } } + + /** + * Array and nested array columns are dictionary encoded, but not correctly for {@link UnnestDimensionCursor} which + * is tailored for scalar logical type values that are {@link ColumnCapabilities#isDictionaryEncoded()} and possibly + * with {@link ColumnCapabilities#hasMultipleValues()} (specifically {@link ValueType#STRING}), so we don't want to + * use this cursor if the capabilities are unknown or if the column type is {@link ValueType#ARRAY}. + */ + private static boolean useDimensionCursor(@Nullable ColumnCapabilities capabilities) + { + return capabilities != null && !capabilities.isArray() && capabilities.isDictionaryEncoded() + .and(capabilities.areDictionaryValuesUnique()) + .isTrue(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java index d950d7a14dcf..011d61e2779c 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java @@ -108,6 +108,8 @@ public static Supplier read(ByteBuffer buffer, ByteOr private final int bucketsPosition; private final boolean hasNull; private final int lastBucketNumValues; + private final int[] unwindPrefixLength; + private final int[] unwindBufferPosition; private FrontCodedIntArrayIndexed( ByteBuffer buffer, @@ -133,6 +135,8 @@ private FrontCodedIntArrayIndexed( this.lastBucketNumValues = (numValues & rem) == 0 ? bucketSize : numValues & rem; this.offsetsPosition = offsetsPosition; this.bucketsPosition = offsetsPosition + ((numBuckets - 1) * Integer.BYTES); + this.unwindPrefixLength = new int[bucketSize]; + this.unwindBufferPosition = new int[bucketSize]; } @Override @@ -379,7 +383,7 @@ private static int compareBucketFirstValue(ByteBuffer bucketBuffer, int length, /** * Finds a value in a bucket among the fragments. The first value is assumed to have been already compared against * and be smaller than the value we are looking for. This comparison is the source of the 'shared prefix', which is - * the length which the value has in common with the first value of the bucket. + * the length which the value has in common with the previous values of the bucket. * * This method uses this shared prefix length to skip more expensive byte by byte full value comparisons when * possible by comparing the shared prefix length with the prefix length of the fragment. Since the bucket is always @@ -394,7 +398,7 @@ private int findValueInBucket( int[] value, int currBucketFirstValueIndex, int bucketSize, - int sharedPrefix + int sharedPrefixLength ) { int relativePosition = 0; @@ -403,23 +407,28 @@ private int findValueInBucket( int insertionPoint = 1; while (++relativePosition < bucketSize) { prefixLength = VByte.readInt(buffer); - if (prefixLength > sharedPrefix) { + if (prefixLength > sharedPrefixLength) { + // bucket value shares more in common with the preceding value, so the value we are looking for comes after final int skip = VByte.readInt(buffer); buffer.position(buffer.position() + (skip * Integer.BYTES)); insertionPoint++; - } else if (prefixLength < sharedPrefix) { - // prefix is smaller, that means this value sorts ahead of it + } else if (prefixLength < sharedPrefixLength) { + // bucket value prefix is smaller, that means the value we are looking for sorts ahead of it break; } else { + // value has the same shared prefix, so compare additional values to find final int fragmentLength = VByte.readInt(buffer); final int common = Math.min(fragmentLength, value.length - prefixLength); int fragmentComparison = 0; + boolean shortCircuit = false; for (int i = 0; i < common; i++) { fragmentComparison = Integer.compare( buffer.getInt(buffer.position() + (i * Integer.BYTES)), value[prefixLength + i] ); if (fragmentComparison != 0) { + sharedPrefixLength = prefixLength + i; + shortCircuit = true; break; } } @@ -430,6 +439,10 @@ private int findValueInBucket( if (fragmentComparison == 0) { return (currBucketFirstValueIndex + adjustIndex) + relativePosition; } else if (fragmentComparison < 0) { + // value we are looking for is longer than the current bucket value, continue on + if (!shortCircuit) { + sharedPrefixLength = prefixLength + common; + } buffer.position(buffer.position() + (fragmentLength * Integer.BYTES)); insertionPoint++; } else { @@ -438,7 +451,7 @@ private int findValueInBucket( } } // (-(insertion point) - 1) - return -(currBucketFirstValueIndex + adjustIndex) + (-(insertionPoint) - 1); + return -(currBucketFirstValueIndex + adjustIndex) + (~insertionPoint); } /** @@ -446,47 +459,64 @@ private int findValueInBucket( * * This method modifies the position of the buffer. */ - static int[] getFromBucket(ByteBuffer buffer, int offset) + int[] getFromBucket(ByteBuffer buffer, int offset) { - int prefixPosition; + // first value is written whole + final int length = VByte.readInt(buffer); if (offset == 0) { - final int length = VByte.readInt(buffer); final int[] firstValue = new int[length]; for (int i = 0; i < length; i++) { firstValue[i] = buffer.getInt(); } return firstValue; - } else { - final int firstLength = VByte.readInt(buffer); - prefixPosition = buffer.position(); - buffer.position(buffer.position() + (firstLength * Integer.BYTES)); } int pos = 0; int prefixLength; int fragmentLength; - int fragmentPosition; - // scan through bucket values until we reach offset + unwindPrefixLength[pos] = 0; + unwindBufferPosition[pos] = buffer.position(); + + buffer.position(buffer.position() + (length * Integer.BYTES)); do { prefixLength = VByte.readInt(buffer); if (++pos < offset) { // not there yet, no need to read anything other than the length to skip ahead final int skipLength = VByte.readInt(buffer); + unwindPrefixLength[pos] = prefixLength; + unwindBufferPosition[pos] = buffer.position(); buffer.position(buffer.position() + (skipLength * Integer.BYTES)); } else { // we've reached our destination fragmentLength = VByte.readInt(buffer); - fragmentPosition = buffer.position(); + if (prefixLength == 0) { + // no prefix, return it directly + final int[] value = new int[fragmentLength]; + for (int i = 0; i < fragmentLength; i++) { + value[i] = buffer.getInt(); + } + return value; + } break; } } while (true); final int valueLength = prefixLength + fragmentLength; final int[] value = new int[valueLength]; - for (int i = 0; i < valueLength; i++) { - if (i < prefixLength) { - value[i] = buffer.getInt(prefixPosition + (i * Integer.BYTES)); - } else { - value[i] = buffer.getInt(fragmentPosition + ((i - prefixLength) * Integer.BYTES)); + for (int i = prefixLength; i < valueLength; i++) { + value[i] = buffer.getInt(); + } + for (int i = prefixLength; i > 0;) { + // previous value had a larger prefix than or the same as the value we are looking for + // skip it since the fragment doesn't have anything we need + if (unwindPrefixLength[--pos] >= i) { + continue; + } + buffer.position(unwindBufferPosition[pos]); + final int prevLength = unwindPrefixLength[pos]; + for (int fragmentOffset = 0; fragmentOffset < i - prevLength; fragmentOffset++) { + value[prevLength + fragmentOffset] = buffer.getInt(); } + + i = unwindPrefixLength[pos]; } return value; } @@ -499,12 +529,14 @@ static int[] getFromBucket(ByteBuffer buffer, int offset) */ private static int[][] readBucket(ByteBuffer bucket, int numValues) { + final int[][] bucketValues = new int[numValues][]; + + // first value is written whole final int length = VByte.readInt(bucket); - final int[] prefix = new int[length]; + int[] prefix = new int[length]; for (int i = 0; i < length; i++) { prefix[i] = bucket.getInt(); } - final int[][] bucketValues = new int[numValues][]; bucketValues[0] = prefix; int pos = 1; while (pos < numValues) { @@ -517,6 +549,7 @@ private static int[][] readBucket(ByteBuffer bucket, int numValues) for (int i = prefixLength; i < value.length; i++) { value[i] = bucket.getInt(); } + prefix = value; bucketValues[pos++] = value; } return bucketValues; diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java index 6b298c3472a4..00cc2079fcbe 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java @@ -220,7 +220,7 @@ public int[] get(int index) throws IOException final ByteBuffer bucketBuffer = ByteBuffer.allocate(currentBucketSize).order(byteOrder); valuesOut.readFully(startOffset, bucketBuffer); bucketBuffer.clear(); - return FrontCodedIntArrayIndexed.getFromBucket(bucketBuffer, relativeIndex); + return getFromBucket(bucketBuffer, relativeIndex); } } @@ -276,13 +276,13 @@ private void growScratch() public static int writeBucket(ByteBuffer buffer, int[][] values, int numValues) { int written = 0; - int[] first = null; + int[] prev = null; while (written < numValues) { int[] next = values[written]; if (written == 0) { - first = next; + prev = next; // the first value in the bucket is written completely as it is - int rem = writeValue(buffer, first); + int rem = writeValue(buffer, prev); // wasn't enough room, bail out if (rem < 0) { return rem; @@ -290,12 +290,13 @@ public static int writeBucket(ByteBuffer buffer, int[][] values, int numValues) } else { // all other values must be partitioned into a prefix length and suffix bytes int prefixLength = 0; - for (; prefixLength < first.length; prefixLength++) { - final int cmp = Integer.compare(first[prefixLength], next[prefixLength]); + for (; prefixLength < prev.length; prefixLength++) { + final int cmp = Integer.compare(prev[prefixLength], next[prefixLength]); if (cmp != 0) { break; } } + // convert to bytes because not every char is a single byte final int[] suffix = new int[next.length - prefixLength]; System.arraycopy(next, prefixLength, suffix, 0, suffix.length); int rem = buffer.remaining() - VByte.computeIntSize(prefixLength); @@ -305,6 +306,7 @@ public static int writeBucket(ByteBuffer buffer, int[][] values, int numValues) } VByte.writeInt(buffer, prefixLength); rem = writeValue(buffer, suffix); + prev = next; // wasn't enough room, bail out if (rem < 0) { return rem; @@ -337,4 +339,71 @@ public static int writeValue(ByteBuffer buffer, int[] ints) } return buffer.position() - pos; } + + /** + * Copy of {@link FrontCodedIntArrayIndexed#getFromBucket(ByteBuffer, int)} but with local declarations of arrays + * for unwinding stuff + */ + int[] getFromBucket(ByteBuffer buffer, int offset) + { + int[] unwindPrefixLength = new int[bucketSize]; + int[] unwindBufferPosition = new int[bucketSize]; + // first value is written whole + final int length = VByte.readInt(buffer); + if (offset == 0) { + final int[] firstValue = new int[length]; + for (int i = 0; i < length; i++) { + firstValue[i] = buffer.getInt(); + } + return firstValue; + } + int pos = 0; + int prefixLength; + int fragmentLength; + unwindPrefixLength[pos] = 0; + unwindBufferPosition[pos] = buffer.position(); + + buffer.position(buffer.position() + (length * Integer.BYTES)); + do { + prefixLength = VByte.readInt(buffer); + if (++pos < offset) { + // not there yet, no need to read anything other than the length to skip ahead + final int skipLength = VByte.readInt(buffer); + unwindPrefixLength[pos] = prefixLength; + unwindBufferPosition[pos] = buffer.position(); + buffer.position(buffer.position() + (skipLength * Integer.BYTES)); + } else { + // we've reached our destination + fragmentLength = VByte.readInt(buffer); + if (prefixLength == 0) { + // no prefix, return it directly + final int[] value = new int[fragmentLength]; + for (int i = 0; i < fragmentLength; i++) { + value[i] = buffer.getInt(); + } + return value; + } + break; + } + } while (true); + final int valueLength = prefixLength + fragmentLength; + final int[] value = new int[valueLength]; + for (int i = prefixLength; i < valueLength; i++) { + value[i] = buffer.getInt(); + } + for (int i = prefixLength; i > 0;) { + // previous value had a larger prefix than or the same as the value we are looking for + // skip it since the fragment doesn't have anything we need + if (unwindPrefixLength[--pos] >= i) { + continue; + } + buffer.position(unwindBufferPosition[pos]); + final int prevLength = unwindPrefixLength[pos]; + for (int fragmentOffset = 0; fragmentOffset < i - prevLength; fragmentOffset++) { + value[prevLength + fragmentOffset] = buffer.getInt(); + } + i = unwindPrefixLength[pos]; + } + return value; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java similarity index 76% rename from processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java rename to processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java index 259aeb510bdc..1ae465d25cd9 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java @@ -27,10 +27,10 @@ import java.io.IOException; import java.nio.channels.WritableByteChannel; -public class ArrayOfLiteralsFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter +public class ArrayFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter { - protected ArrayOfLiteralsFieldColumnWriter( + protected ArrayFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, @@ -46,26 +46,26 @@ int[] processValue(int row, Object value) { if (value instanceof Object[]) { Object[] array = (Object[]) value; - final int[] newIdsWhoDis = new int[array.length]; + final int[] globalIds = new int[array.length]; for (int i = 0; i < array.length; i++) { if (array[i] == null) { - newIdsWhoDis[i] = 0; + globalIds[i] = 0; } else if (array[i] instanceof String) { - newIdsWhoDis[i] = globalDictionaryIdLookup.lookupString((String) array[i]); + globalIds[i] = globalDictionaryIdLookup.lookupString((String) array[i]); } else if (array[i] instanceof Long) { - newIdsWhoDis[i] = globalDictionaryIdLookup.lookupLong((Long) array[i]); + globalIds[i] = globalDictionaryIdLookup.lookupLong((Long) array[i]); } else if (array[i] instanceof Double) { - newIdsWhoDis[i] = globalDictionaryIdLookup.lookupDouble((Double) array[i]); + globalIds[i] = globalDictionaryIdLookup.lookupDouble((Double) array[i]); } else { - newIdsWhoDis[i] = -1; + globalIds[i] = -1; } - Preconditions.checkArgument(newIdsWhoDis[i] >= 0, "unknown global id [%s] for value [%s]", newIdsWhoDis[i], array[i]); + Preconditions.checkArgument(globalIds[i] >= 0, "unknown global id [%s] for value [%s]", globalIds[i], array[i]); arrayElements.computeIfAbsent( - newIdsWhoDis[i], + globalIds[i], (id) -> indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap() ).add(row); } - return newIdsWhoDis; + return globalIds; } return null; } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java index 913213b6b5e9..bbb6dfca3b75 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java @@ -94,7 +94,7 @@ public abstract class CompressedNestedDataComplexColumn fields; - private final NestedLiteralTypeInfo fieldInfo; + private final NestedFieldTypeInfo fieldInfo; private final Supplier stringDictionarySupplier; private final Supplier> longDictionarySupplier; @@ -115,7 +115,7 @@ public CompressedNestedDataComplexColumn( CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, GenericIndexed fields, - NestedLiteralTypeInfo fieldInfo, + NestedFieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, Supplier> doubleDictionarySupplier, @@ -159,7 +159,7 @@ public List> getNestedFields() return fieldParts; } - public NestedLiteralTypeInfo getFieldInfo() + public NestedFieldTypeInfo getFieldInfo() { return fieldInfo; } @@ -749,7 +749,7 @@ public Set getColumnTypes(List path) if (index < 0) { return null; } - return NestedLiteralTypeInfo.convertToSet(fieldInfo.getTypes(index).getByteValue()); + return NestedFieldTypeInfo.convertToSet(fieldInfo.getTypes(index).getByteValue()); } @Nullable @@ -798,7 +798,7 @@ private ColumnHolder readNestedFieldColumn(String field) return null; } final int fieldIndex = fields.indexOf(field); - final NestedLiteralTypeInfo.TypeSet types = fieldInfo.getTypes(fieldIndex); + final NestedFieldTypeInfo.TypeSet types = fieldInfo.getTypes(fieldIndex); final String fieldFileName = getFieldFileName(metadata.getFileNameBase(), field, fieldIndex); final ByteBuffer dataBuffer = fileMapper.mapFile(fieldFileName); if (dataBuffer == null) { @@ -882,7 +882,7 @@ private ColumnHolder readNestedFieldColumn(String field) final boolean hasNull = localDictionarySupplier.get().get(0) == 0; Supplier> columnSupplier = () -> { FixedIndexed localDict = localDictionarySupplier.get(); - return closer.register(new NestedFieldLiteralDictionaryEncodedColumn( + return closer.register(new NestedFieldDictionaryEncodedColumn( types, longs.get(), doubles.get(), @@ -901,7 +901,7 @@ private ColumnHolder readNestedFieldColumn(String field) .setHasNulls(hasNull) .setDictionaryEncodedColumnSupplier(columnSupplier); columnBuilder.setIndexSupplier( - new NestedFieldLiteralColumnIndexSupplier( + new NestedFieldColumnIndexSupplier( types, metadata.getBitmapSerdeFactory().getBitmapFactory(), rBitmaps, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java index 861900ca45dc..70040d7b5286 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java @@ -89,4 +89,20 @@ public int getArrayCardinality() { return arrayCount; } + + public boolean allNull() + { + for (String s : sortedStrings) { + if (s != null) { + return false; + } + } + if (sortedLongs.size() > 0) { + return false; + } + if (sortedDoubles.size() > 0) { + return false; + } + return arrayCount == 0; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java index 0f8c03924c07..aec4641ca73d 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java @@ -25,6 +25,7 @@ import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; @@ -60,6 +61,7 @@ import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.SortedMap; @@ -85,14 +87,14 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer processLiteralField(ArrayList fieldPath, @Nullable Object fieldValue) + public ProcessedValue processField(ArrayList fieldPath, @Nullable Object fieldValue) { final GlobalDictionaryEncodedFieldColumnWriter writer = fieldWriters.get( NestedPathFinder.toNormalizedJsonPath(fieldPath) ); if (writer != null) { try { - ExprEval eval = ExprEval.bestEffortOf(fieldValue); + final ExprEval eval = ExprEval.bestEffortOf(fieldValue); if (eval.type().isPrimitive() || (eval.type().isArray() && eval.type().getElementType().isPrimitive())) { writer.addValue(rowCount, eval.value()); } else { @@ -100,23 +102,23 @@ public StructuredDataProcessor.ProcessedLiteral processLiteralField(ArrayList writer.addValue(rowCount, eval.asString()); } // serializer doesn't use size estimate - return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; + return ProcessedValue.NULL_LITERAL; } catch (IOException e) { - throw new RuntimeException(":("); + throw new RE(e, "Failed to write field [%s] value [%s]", fieldPath, fieldValue); } } - return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; + return ProcessedValue.NULL_LITERAL; } @Nullable @Override - public ProcessedLiteral processArrayOfLiteralsField( + public ProcessedValue processArrayField( ArrayList fieldPath, - @Nullable Object maybeArrayOfLiterals + @Nullable List array ) { - ExprEval eval = ExprEval.bestEffortOf(maybeArrayOfLiterals); + final ExprEval eval = ExprEval.bestEffortArray(array); if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) { final GlobalDictionaryEncodedFieldColumnWriter writer = fieldWriters.get( NestedPathFinder.toNormalizedJsonPath(fieldPath) @@ -125,10 +127,10 @@ public ProcessedLiteral processArrayOfLiteralsField( try { writer.addValue(rowCount, eval.value()); // serializer doesn't use size estimate - return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL; + return ProcessedValue.NULL_LITERAL; } catch (IOException e) { - throw new RuntimeException(":("); + throw new RE(e, "Failed to write field [%s] value [%s]", fieldPath, array); } } } @@ -138,9 +140,9 @@ public ProcessedLiteral processArrayOfLiteralsField( private byte[] metadataBytes; private GlobalDictionaryIdLookup globalDictionaryIdLookup; - private SortedMap fields; + private SortedMap fields; private GenericIndexedWriter fieldsWriter; - private NestedLiteralTypeInfo.Writer fieldsInfoWriter; + private NestedFieldTypeInfo.Writer fieldsInfoWriter; private DictionaryWriter dictionaryWriter; private FixedIndexedWriter longDictionaryWriter; private FixedIndexedWriter doubleDictionaryWriter; @@ -152,10 +154,7 @@ public ProcessedLiteral processArrayOfLiteralsField( private int rowCount = 0; private boolean closedForWrite = false; - private boolean stringDictionarySerialized = false; - private boolean longDictionarySerialized = false; - private boolean doubleDictionarySerialized = false; - private boolean arrayDictionarySerialized = false; + private boolean dictionarySerialized = false; public NestedDataColumnSerializer( String name, @@ -183,7 +182,7 @@ public void open() throws IOException fieldsWriter = new GenericIndexedWriter<>(segmentWriteOutMedium, name, GenericIndexed.STRING_STRATEGY); fieldsWriter.open(); - fieldsInfoWriter = new NestedLiteralTypeInfo.Writer(segmentWriteOutMedium); + fieldsInfoWriter = new NestedFieldTypeInfo.Writer(segmentWriteOutMedium); fieldsInfoWriter.open(); dictionaryWriter = StringEncodingStrategies.getStringDictionaryWriter( @@ -234,12 +233,12 @@ public void open() throws IOException nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); } - public void serializeFields(SortedMap fields) throws IOException + public void serializeFields(SortedMap fields) throws IOException { this.fields = fields; this.fieldWriters = Maps.newHashMapWithExpectedSize(fields.size()); int ctr = 0; - for (Map.Entry field : fields.entrySet()) { + for (Map.Entry field : fields.entrySet()) { final String fieldName = field.getKey(); final String fieldFileName = NESTED_FIELD_PREFIX + ctr++; fieldsWriter.write(fieldName); @@ -272,7 +271,7 @@ public void serializeFields(SortedMap dictionaryValues) throws IOException + public void serializeDictionaries( + Iterable strings, + Iterable longs, + Iterable doubles, + Iterable arrays + ) throws IOException { - if (stringDictionarySerialized) { + if (dictionarySerialized) { throw new ISE("String dictionary already serialized for column [%s], cannot serialize again", name); } + // null is always 0 dictionaryWriter.write(null); globalDictionaryIdLookup.addString(null); - for (String value : dictionaryValues) { + for (String value : strings) { value = NullHandling.emptyToNullIfNeeded(value); if (value == null) { continue; @@ -313,76 +318,38 @@ public void serializeStringDictionary(Iterable dictionaryValues) throws dictionaryWriter.write(value); globalDictionaryIdLookup.addString(value); } - stringDictionarySerialized = true; - } + dictionarySerialized = true; - public void serializeLongDictionary(Iterable dictionaryValues) throws IOException - { - if (!stringDictionarySerialized) { - throw new ISE("Must serialize string value dictionary before serializing long dictionary for column [%s]", name); - } - if (longDictionarySerialized) { - throw new ISE("Long dictionary already serialized for column [%s], cannot serialize again", name); - } - for (Long value : dictionaryValues) { + for (Long value : longs) { if (value == null) { continue; } longDictionaryWriter.write(value); globalDictionaryIdLookup.addLong(value); } - longDictionarySerialized = true; - } - public void serializeDoubleDictionary(Iterable dictionaryValues) throws IOException - { - if (!stringDictionarySerialized) { - throw new ISE("Must serialize string value dictionary before serializing double dictionary for column [%s]", name); - } - if (!longDictionarySerialized) { - throw new ISE("Must serialize long value dictionary before serializing double dictionary for column [%s]", name); - } - if (doubleDictionarySerialized) { - throw new ISE("Double dictionary already serialized for column [%s], cannot serialize again", name); - } - for (Double value : dictionaryValues) { + for (Double value : doubles) { if (value == null) { continue; } doubleDictionaryWriter.write(value); globalDictionaryIdLookup.addDouble(value); } - doubleDictionarySerialized = true; - } - public void serializeArrayDictionary(Iterable dictionaryValues) throws IOException - { - if (!stringDictionarySerialized) { - throw new ISE("Must serialize string value dictionary before serializing array dictionary for column [%s]", name); - } - if (!longDictionarySerialized) { - throw new ISE("Must serialize long value dictionary before serializing array dictionary for column [%s]", name); - } - if (!doubleDictionarySerialized) { - throw new ISE("Must serialize double value dictionary before serializing array dictionary for column [%s]", name); - } - if (arrayDictionarySerialized) { - throw new ISE("Array dictionary already serialized for column [%s], cannot serialize again", name); - } - for (int[] value : dictionaryValues) { + for (int[] value : arrays) { if (value == null) { continue; } arrayDictionaryWriter.write(value); globalDictionaryIdLookup.addArray(value); } - arrayDictionarySerialized = true; + dictionarySerialized = true; } @Override public void serialize(ColumnValueSelector selector) throws IOException { - if (!arrayDictionarySerialized) { + if (!dictionarySerialized) { throw new ISE("Must serialize value dictionaries before serializing values for column [%s]", name); } StructuredData data = StructuredData.wrap(selector.getObject()); @@ -463,7 +430,7 @@ public void writeTo( channel.close(); } - for (Map.Entry field : fields.entrySet()) { + for (Map.Entry field : fields.entrySet()) { // remove writer so that it can be collected when we are done with it GlobalDictionaryEncodedFieldColumnWriter writer = fieldWriters.remove(field.getKey()); writer.writeTo(rowCount, smoosher); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java index 92a29c6d9db5..15dc2d888fe7 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java @@ -78,7 +78,7 @@ public static NestedDataColumnSupplier read( final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); final NestedDataColumnMetadata metadata; final GenericIndexed fields; - final NestedLiteralTypeInfo fieldInfo; + final NestedFieldTypeInfo fieldInfo; final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier; final ImmutableBitmap nullValues; final GenericIndexed stringDictionary; @@ -94,7 +94,7 @@ public static NestedDataColumnSupplier read( NestedDataColumnMetadata.class ); fields = GenericIndexed.read(bb, GenericIndexed.STRING_STRATEGY, mapper); - fieldInfo = NestedLiteralTypeInfo.read(bb, fields.size()); + fieldInfo = NestedFieldTypeInfo.read(bb, fields.size()); if (fields.size() == 0) { // all nulls, in the future we'll deal with this better... but for now lets just call it a string because @@ -227,7 +227,7 @@ public static NestedDataColumnSupplier read( private final byte version; private final NestedDataColumnMetadata metadata; private final GenericIndexed fields; - private final NestedLiteralTypeInfo fieldInfo; + private final NestedFieldTypeInfo fieldInfo; private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier; private final ImmutableBitmap nullValues; private final GenericIndexed stringDictionary; @@ -245,7 +245,7 @@ private NestedDataColumnSupplier( byte version, NestedDataColumnMetadata metadata, GenericIndexed fields, - NestedLiteralTypeInfo fieldInfo, + NestedFieldTypeInfo fieldInfo, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, GenericIndexed stringDictionary, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java index 18527a6ba305..eadfca84ce2f 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java @@ -41,7 +41,7 @@ public NestedDataColumnV3( CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, GenericIndexed fields, - NestedLiteralTypeInfo fieldInfo, + NestedFieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, Supplier> doubleDictionarySupplier, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java index b5f00af308c6..26631ee7fdde 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java @@ -40,7 +40,7 @@ public NestedDataColumnV4( CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, GenericIndexed fields, - NestedLiteralTypeInfo fieldInfo, + NestedFieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, Supplier> doubleDictionarySupplier, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java index cb7463c487ca..755922618106 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java @@ -41,7 +41,7 @@ public NestedDataColumnV5( CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, GenericIndexed fields, - NestedLiteralTypeInfo fieldInfo, + NestedFieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, Supplier> doubleDictionarySupplier, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java similarity index 99% rename from processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java rename to processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java index 15e6b0a1fee3..b4e59d486bc8 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java @@ -68,10 +68,10 @@ import java.util.SortedSet; /** - * Supplies indexes for nested field columns {@link NestedFieldLiteralDictionaryEncodedColumn} of + * Supplies indexes for nested field columns {@link NestedFieldDictionaryEncodedColumn} of * {@link NestedDataComplexColumn}. */ -public class NestedFieldLiteralColumnIndexSupplier> +public class NestedFieldColumnIndexSupplier> implements ColumnIndexSupplier { @Nullable @@ -93,8 +93,8 @@ public class NestedFieldLiteralColumnIndexSupplier bitmaps, Supplier> localDictionarySupplier, @@ -331,7 +331,7 @@ public String getValue(int index) @Override public ImmutableBitmap getBitmap(int idx) { - return NestedFieldLiteralColumnIndexSupplier.this.getBitmap(idx); + return NestedFieldColumnIndexSupplier.this.getBitmap(idx); } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java similarity index 96% rename from processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java rename to processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java index 14b203ac1548..3449d3410a56 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java @@ -71,10 +71,10 @@ import java.nio.ByteBuffer; import java.util.BitSet; -public class NestedFieldLiteralDictionaryEncodedColumn> +public class NestedFieldDictionaryEncodedColumn> implements DictionaryEncodedColumn { - private final NestedLiteralTypeInfo.TypeSet types; + private final NestedFieldTypeInfo.TypeSet types; @Nullable private final ColumnType singleType; private final ColumnarLongs longsColumn; @@ -94,8 +94,8 @@ public class NestedFieldLiteralDictionaryEncodedColumn processLiteralField( - ArrayList fieldPath, - @Nullable Object fieldValue - ); + public abstract ProcessedValue processField(ArrayList fieldPath, @Nullable Object fieldValue); /** - * process a {@link List} or {@link Object[]} that might be an array of literals. If the object was an array of - * literals, returns a {@link ProcessedLiteral}, else returns null. + * Process a {@link List} or {@link Object[]} returning a {@link ProcessedValue} if no further processing should + * be performed by the {@link StructuredDataProcessor}, else a return value of null indicates that each element + * of the array will be processed separately as a new {@link NestedPathArrayElement} part. */ @Nullable - public abstract ProcessedLiteral processArrayOfLiteralsField( - ArrayList fieldPath, - @Nullable Object maybeArrayOfLiterals - ); + public abstract ProcessedValue processArrayField(ArrayList fieldPath, @Nullable List array); /** - * Process fields, returning a list of all paths to literal fields, represented as an ordered sequence of - * {@link NestedPathPart}. + * Process some object, traversing any nested structure and returning a list of all paths which created a + * {@link ProcessedValue} during processing, represented as an ordered sequence of {@link NestedPathPart}. + * + * This method processes plain java objects, for each {@link Map} it adds a {@link MapField} to the path, for + * {@link List} a {@link ArrayField}, {@link Object[]} a {@link ArrayField}, and so on. {@link ArrayField} and + * {@link ArrayField} will be processed by {@link #processArrayField(ArrayList, List)} */ public ProcessResults processFields(Object raw) { @@ -66,11 +66,11 @@ public ProcessResults processFields(Object raw) if (raw instanceof Map) { toProcess.add(new MapField(newPath, (Map) raw)); } else if (raw instanceof List) { - toProcess.add(new ListField(newPath, (List) raw)); + toProcess.add(new ArrayField(newPath, (List) raw)); } else if (raw instanceof Object[]) { - toProcess.add(new ArrayField(newPath, (Object[]) raw)); + toProcess.add(new ArrayField(newPath, Arrays.asList((Object[]) raw))); } else { - return new ProcessResults().addLiteralField(newPath, processLiteralField(newPath, raw).getSize()); + return new ProcessResults().addLiteralField(newPath, processField(newPath, raw).getSize()); } final ProcessResults accumulator = new ProcessResults(); @@ -79,8 +79,6 @@ public ProcessResults processFields(Object raw) Field next = toProcess.poll(); if (next instanceof MapField) { accumulator.merge(processMapField(toProcess, (MapField) next)); - } else if (next instanceof ListField) { - accumulator.merge(processListField(toProcess, (ListField) next)); } else if (next instanceof ArrayField) { accumulator.merge(processArrayField(toProcess, (ArrayField) next)); } @@ -101,26 +99,28 @@ private ProcessResults processMapField(Queue toProcess, MapField map) newPath.add(new NestedPathField(entry.getKey())); if (value instanceof List) { List theList = (List) value; - toProcess.add(new ListField(newPath, theList)); + toProcess.add(new ArrayField(newPath, theList)); + } else if (value instanceof Object[]) { + toProcess.add(new ArrayField(newPath, Arrays.asList((Object[]) value))); } else if (value instanceof Map) { toProcess.add(new MapField(newPath, (Map) value)); } else { // literals get processed - processResults.addLiteralField(newPath, processLiteralField(newPath, value).getSize()); + processResults.addLiteralField(newPath, processField(newPath, value).getSize()); } } return processResults; } - private ProcessResults processListField(Queue toProcess, ListField list) + private ProcessResults processArrayField(Queue toProcess, ArrayField list) { // start with object reference, is probably a bit bigger than this... final ProcessResults results = new ProcessResults().withSize(8); final List theList = list.getList(); - // check to see if the processor treats arrays of literals as literals, if so we can stop processing here - ProcessedLiteral maybeArrayOfLiterals = processArrayOfLiteralsField(list.getPath(), theList); - if (maybeArrayOfLiterals != null) { - results.addLiteralField(list.getPath(), maybeArrayOfLiterals.getSize()); + // check to see if the processor handled the array, indicated by a non-null result, if so we can stop here + final ProcessedValue maybeProcessed = processArrayField(list.getPath(), theList); + if (maybeProcessed != null) { + results.addLiteralField(list.getPath(), maybeProcessed.getSize()); } else { // else we have to dig into the list and process each element for (int i = 0; i < theList.size(); i++) { @@ -131,41 +131,11 @@ private ProcessResults processListField(Queue toProcess, ListField list) if (element instanceof Map) { toProcess.add(new MapField(newPath, (Map) element)); } else if (element instanceof List) { - toProcess.add(new ListField(newPath, (List) element)); + toProcess.add(new ArrayField(newPath, (List) element)); } else if (element instanceof Object[]) { - toProcess.add(new ArrayField(newPath, (Object[]) element)); + toProcess.add(new ArrayField(newPath, Arrays.asList((Object[]) element))); } else { - results.addLiteralField(newPath, processLiteralField(newPath, element).getSize()); - } - } - } - return results; - } - - private ProcessResults processArrayField(Queue toProcess, ArrayField array) - { - // start with object reference, is probably a bit bigger than this... - final ProcessResults results = new ProcessResults().withSize(8); - final Object[] theArray = array.getArray(); - // check to see if the processor treats arrays of literals as literals, if so we can stop processing here - ProcessedLiteral maybeArrayOfLiterals = processArrayOfLiteralsField(array.getPath(), theArray); - if (maybeArrayOfLiterals != null) { - results.addLiteralField(array.getPath(), maybeArrayOfLiterals.getSize()); - } else { - // else we have to dig into the list and process each element - for (int i = 0; i < theArray.length; i++) { - final ArrayList newPath = new ArrayList<>(array.getPath()); - newPath.add(new NestedPathArrayElement(i)); - final Object element = StructuredData.unwrap(theArray[i]); - // maps and lists go back into the queue - if (element instanceof Map) { - toProcess.add(new MapField(newPath, (Map) element)); - } else if (element instanceof List) { - toProcess.add(new ListField(newPath, (List) element)); - } else if (element instanceof Object[]) { - toProcess.add(new ArrayField(newPath, (Object[]) element)); - } else { - results.addLiteralField(newPath, processLiteralField(newPath, element).getSize()); + results.addLiteralField(newPath, processField(newPath, element).getSize()); } } } @@ -187,11 +157,11 @@ public ArrayList getPath() } } - static class ListField extends Field + static class ArrayField extends Field { private final List list; - ListField(ArrayList path, List list) + ArrayField(ArrayList path, List list) { super(path); this.list = list; @@ -203,22 +173,6 @@ public List getList() } } - static class ArrayField extends Field - { - private final Object[] array; - - ArrayField(ArrayList path, Object[] array) - { - super(path); - this.array = array; - } - - public Object[] getArray() - { - return array; - } - } - static class MapField extends Field { private final Map map; @@ -235,14 +189,14 @@ static class MapField extends Field } } - public static class ProcessedLiteral + public static class ProcessedValue { - public static final ProcessedLiteral NULL_LITERAL = new ProcessedLiteral<>(null, 0); + public static final ProcessedValue NULL_LITERAL = new ProcessedValue<>(null, 0); @Nullable private final T value; private final int size; - public ProcessedLiteral(@Nullable T value, int size) + public ProcessedValue(@Nullable T value, int size) { this.value = value; this.size = size; @@ -269,7 +223,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - ProcessedLiteral that = (ProcessedLiteral) o; + ProcessedValue that = (ProcessedValue) o; return size == that.size && Objects.equals(value, that.value); } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantFieldColumnWriter.java similarity index 80% rename from processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java rename to processing/src/main/java/org/apache/druid/segment/nested/VariantFieldColumnWriter.java index c4690d8c67b4..121f388583c9 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantFieldColumnWriter.java @@ -30,9 +30,9 @@ /** * Literal field writer for mixed type nested columns of {@link NestedDataColumnSerializer} */ -public final class VariantLiteralFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter +public final class VariantFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter { - public VariantLiteralFieldColumnWriter( + public VariantFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, @@ -49,22 +49,22 @@ Object processValue(int row, Object value) { if (value instanceof Object[]) { Object[] array = (Object[]) value; - final int[] newIdsWhoDis = new int[array.length]; + final int[] globalIds = new int[array.length]; for (int i = 0; i < array.length; i++) { if (array[i] == null) { - newIdsWhoDis[i] = 0; + globalIds[i] = 0; } else if (array[i] instanceof String) { - newIdsWhoDis[i] = globalDictionaryIdLookup.lookupString((String) array[i]); + globalIds[i] = globalDictionaryIdLookup.lookupString((String) array[i]); } else if (array[i] instanceof Long) { - newIdsWhoDis[i] = globalDictionaryIdLookup.lookupLong((Long) array[i]); + globalIds[i] = globalDictionaryIdLookup.lookupLong((Long) array[i]); } else if (array[i] instanceof Double) { - newIdsWhoDis[i] = globalDictionaryIdLookup.lookupDouble((Double) array[i]); + globalIds[i] = globalDictionaryIdLookup.lookupDouble((Double) array[i]); } else { - newIdsWhoDis[i] = -1; + globalIds[i] = -1; } - Preconditions.checkArgument(newIdsWhoDis[i] >= 0, "unknown global id [%s] for value [%s]", newIdsWhoDis[i], array[i]); + Preconditions.checkArgument(globalIds[i] >= 0, "unknown global id [%s] for value [%s]", globalIds[i], array[i]); } - return newIdsWhoDis; + return globalIds; } return super.processValue(row, value); } diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java index 05d23f5bdc54..f4902946bfb9 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java @@ -57,6 +57,7 @@ import org.apache.druid.segment.nested.CompressedNestedDataComplexColumn; import org.apache.druid.segment.nested.NestedDataComplexColumn; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; +import org.apache.druid.segment.nested.NestedFieldDictionaryEncodedColumn; import org.apache.druid.segment.nested.NestedPathArrayElement; import org.apache.druid.segment.nested.NestedPathFinder; import org.apache.druid.segment.nested.NestedPathPart; @@ -78,7 +79,7 @@ /** * Optimized virtual column that can make direct selectors into a {@link NestedDataComplexColumn} or any associated - * nested fields ({@link org.apache.druid.segment.nested.NestedFieldLiteralDictionaryEncodedColumn}) including using + * nested fields ({@link NestedFieldDictionaryEncodedColumn}) including using * their indexes. *

* This virtual column is used for the SQL operators JSON_VALUE (if {@link #processFromRaw} is set to false) or diff --git a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java index b1e3c7b61646..8e6f3485d46e 100644 --- a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java +++ b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java @@ -37,6 +37,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.expression.TestExprMacroTable; @@ -77,6 +78,14 @@ public class NestedDataTestUtils public static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("timestamp", null, null); + public static final JsonInputFormat DEFAULT_JSON_INPUT_FORMAT = new JsonInputFormat( + JSONPathSpec.DEFAULT, + null, + null, + null, + null + ); + public static final DimensionsSpec AUTO_DISCOVERY = DimensionsSpec.builder() .useSchemaDiscovery(true) @@ -106,14 +115,22 @@ public class NestedDataTestUtils null ); - public static DelimitedInputFormat SIMPLE_DATA_TSV_INPUT_FORMAT = DelimitedInputFormat.ofColumns( - "timestamp", - "dim", - "nest", - "nester", - "variant", - "list" + public static DelimitedInputFormat SIMPLE_DATA_TSV_INPUT_FORMAT = new DelimitedInputFormat( + Arrays.asList( + "timestamp", + "dim", + "nest", + "nester", + "variant", + "list" + ), + null, + null, + false, + false, + 0 ); + public static final TransformSpec SIMPLE_DATA_TSV_TRANSFORM = new TransformSpec( null, Arrays.asList( @@ -215,7 +232,7 @@ public static Segment createIncrementalIndexForJsonInput( return createIncrementalIndex( tempFolder, file, - JsonInputFormat.DEFAULT, + DEFAULT_JSON_INPUT_FORMAT, TIMESTAMP_SPEC, AUTO_DISCOVERY, TransformSpec.NONE, @@ -237,7 +254,7 @@ public static List createSegmentsForJsonInput( tempFolder, closer, inputFile, - JsonInputFormat.DEFAULT, + DEFAULT_JSON_INPUT_FORMAT, TIMESTAMP_SPEC, AUTO_DISCOVERY, TransformSpec.NONE, @@ -266,7 +283,7 @@ public static List createSegmentsWithConcatenatedJsonInput( tempFolder, closer, inputFiles, - JsonInputFormat.DEFAULT, + DEFAULT_JSON_INPUT_FORMAT, TIMESTAMP_SPEC, AUTO_DISCOVERY, TransformSpec.NONE, diff --git a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java index 4759791281ae..726210ad620c 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java @@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -134,7 +133,7 @@ public void testIngestAndScanSegmentsRollup() throws Exception tempFolder, closer, NestedDataTestUtils.NUMERIC_DATA_FILE, - JsonInputFormat.DEFAULT, + NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT, NestedDataTestUtils.TIMESTAMP_SPEC, NestedDataTestUtils.AUTO_DISCOVERY, TransformSpec.NONE, @@ -477,7 +476,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscovery() throws Exception NestedDataTestUtils.createIncrementalIndex( tempFolder, NestedDataTestUtils.TYPES_DATA_FILE, - JsonInputFormat.DEFAULT, + NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT, NestedDataTestUtils.TIMESTAMP_SPEC, NestedDataTestUtils.AUTO_DISCOVERY, TransformSpec.NONE, @@ -490,7 +489,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscovery() throws Exception tempFolder, closer, NestedDataTestUtils.TYPES_DATA_FILE, - JsonInputFormat.DEFAULT, + NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT, NestedDataTestUtils.TIMESTAMP_SPEC, NestedDataTestUtils.AUTO_DISCOVERY, TransformSpec.NONE, @@ -536,7 +535,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryArrayTypes() throws NestedDataTestUtils.createIncrementalIndex( tempFolder, NestedDataTestUtils.ARRAY_TYPES_DATA_FILE, - JsonInputFormat.DEFAULT, + NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT, NestedDataTestUtils.TIMESTAMP_SPEC, NestedDataTestUtils.AUTO_DISCOVERY, TransformSpec.NONE, @@ -549,7 +548,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryArrayTypes() throws tempFolder, closer, NestedDataTestUtils.ARRAY_TYPES_DATA_FILE, - JsonInputFormat.DEFAULT, + NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT, NestedDataTestUtils.TIMESTAMP_SPEC, NestedDataTestUtils.AUTO_DISCOVERY, TransformSpec.NONE, 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 c2871172454e..90a4411c87b1 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -248,10 +248,10 @@ public QueryableIndex buildMMappedMergedIndex() Preconditions.checkNotNull(inputFormat, "inputFormat"); Preconditions.checkNotNull(inputSourceTmpDir, "inputSourceTmpDir"); - TransformSpec tranformer = transformSpec != null ? transformSpec : TransformSpec.NONE; + TransformSpec transformer = transformSpec != null ? transformSpec : TransformSpec.NONE; InputRowSchema rowSchema = new InputRowSchema(schema.getTimestampSpec(), schema.getDimensionsSpec(), null); InputSourceReader reader = inputSource.reader(rowSchema, inputFormat, inputSourceTmpDir); - InputSourceReader transformingReader = tranformer.decorate(reader); + InputSourceReader transformingReader = transformer.decorate(reader); return mergeIndexes(indexMerger, persisted, transformingReader::read); } diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java index bafb50f7089a..70b7d5ab548f 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java @@ -123,7 +123,8 @@ public void test_list_unnest_cursors_user_supplied_list_only_nulls() List inputList = Arrays.asList( Collections.singletonList(null), Arrays.asList(null, null), - Collections.singletonList(null) + Collections.singletonList(null), + Collections.emptyList() ); //Create base cursor @@ -145,6 +146,40 @@ public void test_list_unnest_cursors_user_supplied_list_only_nulls() k++; unnestCursor.advance(); } + // since type is 'STRING', it follows multi-value string rules so single element arrays become scalar values, + // so [null] becomes null, meaning we only have 2 rows + Assert.assertEquals(k, 2); + } + + @Test + public void test_list_unnest_cursors_user_supplied_list_only_nulls_array() + { + List inputList = Arrays.asList( + Collections.singletonList(null), + Arrays.asList(null, null), + Collections.singletonList(null), + Collections.emptyList() + ); + + //Create base cursor + ListCursor listCursor = new ListCursor(inputList); + + //Create unnest cursor + UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor( + listCursor, + listCursor.getColumnSelectorFactory(), + new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING_ARRAY, ExprMacroTable.nil()), + OUTPUT_NAME + ); + ColumnValueSelector unnestColumnValueSelector = unnestCursor.getColumnSelectorFactory() + .makeColumnValueSelector(OUTPUT_NAME); + int k = 0; + while (!unnestCursor.isDone()) { + Object valueSelectorVal = unnestColumnValueSelector.getObject(); + Assert.assertNull(valueSelectorVal); + k++; + unnestCursor.advance(); + } Assert.assertEquals(k, 4); } @@ -155,12 +190,15 @@ public void test_list_unnest_cursors_user_supplied_list_mixed_with_nulls() Arrays.asList("a", "b"), Arrays.asList("b", "c"), "d", + Collections.singletonList(null), + Arrays.asList(null, null), + Collections.emptyList(), null, null, null ); - List expectedResults = Arrays.asList("a", "b", "b", "c", "d", null, null, null); + List expectedResults = Arrays.asList("a", "b", "b", "c", "d", null, null); //Create base cursor ListCursor listCursor = new ListCursor(inputList); @@ -178,7 +216,52 @@ public void test_list_unnest_cursors_user_supplied_list_mixed_with_nulls() while (!unnestCursor.isDone()) { Object valueSelectorVal = unnestColumnValueSelector.getObject(); if (valueSelectorVal == null) { - Assert.assertEquals(expectedResults.get(k), null); + Assert.assertNull(expectedResults.get(k)); + } else { + Assert.assertEquals(expectedResults.get(k), valueSelectorVal.toString()); + } + k++; + unnestCursor.advance(); + } + // since type is 'STRING', it follows multi-value string rules so single element arrays become scalar values, + // so [null] becomes null, meaning we only have 7 rows + Assert.assertEquals(k, 7); + } + + @Test + public void test_list_unnest_cursors_user_supplied_list_mixed_with_nulls_array() + { + List inputList = Arrays.asList( + Arrays.asList("a", "b"), + Arrays.asList("b", "c"), + "d", + Collections.singletonList(null), + Arrays.asList(null, null), + Collections.emptyList(), + null, + null, + null + ); + + List expectedResults = Arrays.asList("a", "b", "b", "c", "d", null, null, null); + + //Create base cursor + ListCursor listCursor = new ListCursor(inputList); + + //Create unnest cursor + UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor( + listCursor, + listCursor.getColumnSelectorFactory(), + new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING_ARRAY, ExprMacroTable.nil()), + OUTPUT_NAME + ); + ColumnValueSelector unnestColumnValueSelector = unnestCursor.getColumnSelectorFactory() + .makeColumnValueSelector(OUTPUT_NAME); + int k = 0; + while (!unnestCursor.isDone()) { + Object valueSelectorVal = unnestColumnValueSelector.getObject(); + if (valueSelectorVal == null) { + Assert.assertNull(expectedResults.get(k)); } else { Assert.assertEquals(expectedResults.get(k), valueSelectorVal.toString()); } @@ -568,7 +651,7 @@ public void test_list_unnest_cursors_dimSelector() k++; unnestCursor.advance(); } - Assert.assertEquals(k, 9); + Assert.assertEquals(k, 8); unnestCursor.reset(); Assert.assertNotNull(unnestDimSelector); } diff --git a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java index f2da1746962f..4c1c51231aaf 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java @@ -68,7 +68,7 @@ public void testFrontCodedIndexed() throws IOException { ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order); List theList = ImmutableList.of("hello", "helloo", "hellooo", "hellooz", "helloozy"); - fillBuffer(buffer, theList, 4, useIncrementalBuckets); + persistToBuffer(buffer, theList, 4, useIncrementalBuckets); buffer.position(0); FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( @@ -99,7 +99,7 @@ public void testFrontCodedIndexedSingleBucket() throws IOException { ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order); List theList = ImmutableList.of("hello", "helloo", "hellooo", "hellooz", "helloozy"); - fillBuffer(buffer, theList, 16, useIncrementalBuckets); + persistToBuffer(buffer, theList, 16, useIncrementalBuckets); FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( buffer, @@ -137,7 +137,7 @@ public void testFrontCodedIndexedBigger() throws IOException for (int i = 0; i < sizeBase + sizeAdjust; i++) { values.add(IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId()); } - fillBuffer(buffer, values, bucketSize, useIncrementalBuckets); + persistToBuffer(buffer, values, bucketSize, useIncrementalBuckets); FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( buffer, @@ -173,7 +173,7 @@ public void testFrontCodedIndexedBiggerWithNulls() throws IOException for (int i = 0; i < sizeBase + sizeAdjust; i++) { values.add(IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId()); } - fillBuffer(buffer, values, bucketSize, useIncrementalBuckets); + persistToBuffer(buffer, values, bucketSize, useIncrementalBuckets); FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( buffer, @@ -207,7 +207,7 @@ public void testFrontCodedIndexedIndexOf() throws IOException ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order); List theList = ImmutableList.of("hello", "helloo", "hellooo", "hellooz", "helloozy"); - fillBuffer(buffer, theList, 4, useIncrementalBuckets); + persistToBuffer(buffer, theList, 4, useIncrementalBuckets); FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( buffer, @@ -231,7 +231,7 @@ public void testFrontCodedIndexedIndexOfWithNull() throws IOException TreeSet values = new TreeSet<>(GenericIndexed.STRING_STRATEGY); values.add(null); values.addAll(theList); - fillBuffer(buffer, values, 4, useIncrementalBuckets); + persistToBuffer(buffer, values, 4, useIncrementalBuckets); FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( buffer, @@ -254,7 +254,7 @@ public void testFrontCodedIndexedUnicodes() throws IOException // "\uD83D\uDCA9" and "(請參見已被刪除版本)" are a regression test for https://github.com/apache/druid/pull/13364 List theList = ImmutableList.of("Győ-Moson-Sopron", "Győr", "\uD83D\uDCA9", "(請參見已被刪除版本)"); - fillBuffer(buffer, theList, 4, useIncrementalBuckets); + persistToBuffer(buffer, theList, 4, useIncrementalBuckets); buffer.position(0); FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( @@ -282,7 +282,7 @@ public void testFrontCodedOnlyNull() throws IOException { ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order); List theList = Collections.singletonList(null); - fillBuffer(buffer, theList, 4, useIncrementalBuckets); + persistToBuffer(buffer, theList, 4, useIncrementalBuckets); buffer.position(0); FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( @@ -308,7 +308,7 @@ public void testFrontCodedEmpty() throws IOException { ByteBuffer buffer = ByteBuffer.allocate(1 << 6).order(order); List theList = Collections.emptyList(); - fillBuffer(buffer, theList, 4); + persistToBuffer(buffer, theList, 4, useIncrementalBuckets); buffer.position(0); FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( @@ -351,7 +351,7 @@ public void testBucketSizes() throws IOException values.add(IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId()); } for (int bucketSize : bucketSizes) { - fillBuffer(buffer, values, bucketSize, useIncrementalBuckets); + persistToBuffer(buffer, values, bucketSize, useIncrementalBuckets); FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( buffer, buffer.order() @@ -414,7 +414,7 @@ public void testBadBucketSize() ); } - private static long fillBuffer(ByteBuffer buffer, Iterable sortedIterable, int bucketSize, boolean useIncrementalBuckets) throws IOException + private static long persistToBuffer(ByteBuffer buffer, Iterable sortedIterable, int bucketSize, boolean useIncrementalBuckets) throws IOException { Iterator sortedStrings = sortedIterable.iterator(); buffer.position(0); diff --git a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java index abc9a2ee60d0..8a08a2a972ac 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.data; import com.google.common.collect.ImmutableList; +import junitparams.converters.Nullable; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; @@ -32,6 +33,7 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Iterator; @@ -66,8 +68,10 @@ public void testFrontCodedIntArrayIndexed() throws IOException values.add(new int[]{1, 2, 4}); values.add(new int[]{1, 3, 4}); values.add(new int[]{1, 2, 1}); + values.add(new int[]{2, 1}); + values.add(new int[]{2, 2, 1}); - fillBuffer(buffer, values, 4); + persistToBuffer(buffer, values, 4); buffer.position(0); FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( @@ -81,8 +85,9 @@ public void testFrontCodedIntArrayIndexed() throws IOException while (expectedIterator.hasNext() && indexedIterator.hasNext()) { final int[] expectedNext = expectedIterator.next(); final int[] next = indexedIterator.next(); - Assert.assertArrayEquals(expectedNext, next); - Assert.assertEquals(ctr, codedIndexed.indexOf(next)); + assertSame(ctr, expectedNext, next); + assertSame(ctr, expectedNext, codedIndexed.get(ctr)); + Assert.assertEquals("row " + ctr, ctr, codedIndexed.indexOf(next)); ctr++; } Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext()); @@ -100,7 +105,7 @@ public void testFrontCodedIntArrayIndexedSingleBucket() throws IOException values.add(new int[]{1, 2, 4}); values.add(new int[]{1, 3, 4}); values.add(new int[]{1, 2, 1}); - fillBuffer(buffer, values, 16); + persistToBuffer(buffer, values, 16); FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( buffer, @@ -113,7 +118,8 @@ public void testFrontCodedIntArrayIndexedSingleBucket() throws IOException while (indexedIterator.hasNext() && expectedIterator.hasNext()) { final int[] expectedNext = expectedIterator.next(); final int[] next = indexedIterator.next(); - Assert.assertArrayEquals(expectedNext, next); + assertSame(ctr, expectedNext, next); + assertSame(ctr, expectedNext, codedIndexed.get(ctr)); Assert.assertEquals(ctr, codedIndexed.indexOf(next)); ctr++; } @@ -136,7 +142,7 @@ public void testFrontCodedIntArrayIndexedBigger() throws IOException } values.add(val); } - fillBuffer(buffer, values, bucketSize); + persistToBuffer(buffer, values, bucketSize); FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( buffer, @@ -149,7 +155,8 @@ public void testFrontCodedIntArrayIndexedBigger() throws IOException while (indexedIterator.hasNext() && expectedIterator.hasNext()) { final int[] expectedNext = expectedIterator.next(); final int[] next = indexedIterator.next(); - Assert.assertArrayEquals(expectedNext, next); + assertSame(ctr, expectedNext, next); + assertSame(ctr, expectedNext, codedIndexed.get(ctr)); Assert.assertEquals(ctr, codedIndexed.indexOf(next)); ctr++; } @@ -175,7 +182,7 @@ public void testFrontCodedIntArrayIndexedBiggerWithNulls() throws IOException } values.add(val); } - fillBuffer(buffer, values, bucketSize); + persistToBuffer(buffer, values, bucketSize); FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( buffer, @@ -188,7 +195,8 @@ public void testFrontCodedIntArrayIndexedBiggerWithNulls() throws IOException while (indexedIterator.hasNext() && expectedIterator.hasNext()) { final int[] expectedNext = expectedIterator.next(); final int[] next = indexedIterator.next(); - Assert.assertArrayEquals(expectedNext, next); + assertSame(ctr, expectedNext, next); + assertSame(ctr, expectedNext, codedIndexed.get(ctr)); Assert.assertEquals(ctr, codedIndexed.indexOf(next)); ctr++; } @@ -209,7 +217,7 @@ public void testFrontCodedIntArrayIndexedIndexOf() throws IOException values.add(new int[]{1, 3}); values.add(new int[]{1, 3, 4}); - fillBuffer(buffer, values, 4); + persistToBuffer(buffer, values, 4); FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( buffer, @@ -237,7 +245,7 @@ public void testFrontCodedIntArrayIndexedIndexOfWithNull() throws IOException values.add(new int[]{1, 2, 4}); values.add(new int[]{1, 3}); values.add(new int[]{1, 3, 4}); - fillBuffer(buffer, values, 4); + persistToBuffer(buffer, values, 4); FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( buffer, @@ -259,7 +267,7 @@ public void testFrontCodedOnlyNull() throws IOException { ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order); List theList = Collections.singletonList(null); - fillBuffer(buffer, theList, 4); + persistToBuffer(buffer, theList, 4); buffer.position(0); FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( @@ -285,7 +293,7 @@ public void testFrontCodedEmpty() throws IOException { ByteBuffer buffer = ByteBuffer.allocate(1 << 6).order(order); List theList = Collections.emptyList(); - fillBuffer(buffer, theList, 4); + persistToBuffer(buffer, theList, 4); buffer.position(0); FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read( @@ -333,7 +341,7 @@ public void testBucketSizes() throws IOException values.add(val); } for (int bucketSize : bucketSizes) { - fillBuffer(buffer, values, bucketSize); + persistToBuffer(buffer, values, bucketSize); FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read( buffer, buffer.order() @@ -345,8 +353,8 @@ public void testBucketSizes() throws IOException while (iterator.hasNext() && expectedIterator.hasNext()) { final int[] expectedNext = expectedIterator.next(); final int[] next = iterator.next(); - Assert.assertArrayEquals(expectedNext, next); - + assertSame(ctr, expectedNext, next); + assertSame(ctr, expectedNext, codedIndexed.get(ctr)); Assert.assertEquals(ctr, codedIndexed.indexOf(next)); ctr++; } @@ -388,7 +396,7 @@ public void testBadBucketSize() ); } - private static long fillBuffer(ByteBuffer buffer, Iterable sortedIterable, int bucketSize) throws IOException + private static long persistToBuffer(ByteBuffer buffer, Iterable sortedIterable, int bucketSize) throws IOException { Iterator sortedInts = sortedIterable.iterator(); buffer.position(0); @@ -403,11 +411,7 @@ private static long fillBuffer(ByteBuffer buffer, Iterable sortedIterable while (sortedInts.hasNext()) { final int[] next = sortedInts.next(); writer.write(next); - if (next == null) { - Assert.assertNull(writer.get(index)); - } else { - Assert.assertArrayEquals(next, writer.get(index)); - } + assertSame(index, next, writer.get(index)); index++; } @@ -415,12 +419,7 @@ private static long fillBuffer(ByteBuffer buffer, Iterable sortedIterable index = 0; sortedInts = sortedIterable.iterator(); while (sortedInts.hasNext()) { - final int[] next = sortedInts.next(); - if (next == null) { - Assert.assertNull("row " + index, writer.get(index)); - } else { - Assert.assertArrayEquals("row " + index, next, writer.get(index)); - } + assertSame(index, sortedInts.next(), writer.get(index)); index++; } @@ -452,4 +451,17 @@ public void close() buffer.position(0); return size; } + + private static void assertSame(int index, @Nullable int[] expected, @Nullable int[] actual) + { + if (expected == null) { + Assert.assertNull("row " + index, actual); + } else { + Assert.assertArrayEquals( + "row " + index + " expected: " + Arrays.toString(expected) + " actual: " + Arrays.toString(actual), + expected, + actual + ); + } + } } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index 686e0758f0ef..ba2be32008e0 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -181,17 +181,17 @@ private SmooshedFileMapper smooshify( for (Object o : data) { indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); } - SortedMap sortedFields = new TreeMap<>(); + SortedMap sortedFields = new TreeMap<>(); indexer.mergeFields(sortedFields); GlobalDictionarySortedCollector globalDictionarySortedCollector = indexer.getSortedCollector(); serializer.open(); serializer.serializeFields(sortedFields); - serializer.serializeStringDictionary(globalDictionarySortedCollector.getSortedStrings()); - serializer.serializeLongDictionary(globalDictionarySortedCollector.getSortedLongs()); - serializer.serializeDoubleDictionary(globalDictionarySortedCollector.getSortedDoubles()); - serializer.serializeArrayDictionary( + serializer.serializeDictionaries( + globalDictionarySortedCollector.getSortedStrings(), + globalDictionarySortedCollector.getSortedLongs(), + globalDictionarySortedCollector.getSortedDoubles(), () -> new NestedDataColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, serializer.getGlobalLookup() @@ -320,8 +320,8 @@ public void testLegacyV3ReaderFormat() throws IOException ColumnHolder nestedColumnHolder = v3.getColumnHolder(path); Assert.assertNotNull(nestedColumnHolder); Assert.assertEquals(ColumnType.STRING, nestedColumnHolder.getCapabilities().toColumnType()); - NestedFieldLiteralDictionaryEncodedColumn nestedColumn = - (NestedFieldLiteralDictionaryEncodedColumn) nestedColumnHolder.getColumn(); + NestedFieldDictionaryEncodedColumn nestedColumn = + (NestedFieldDictionaryEncodedColumn) nestedColumnHolder.getColumn(); Assert.assertNotNull(nestedColumn); @@ -364,8 +364,8 @@ public void testLegacyV4ReaderFormat() throws IOException ColumnHolder nestedColumnHolder = v4.getColumnHolder(path); Assert.assertNotNull(nestedColumnHolder); Assert.assertEquals(ColumnType.STRING, nestedColumnHolder.getCapabilities().toColumnType()); - NestedFieldLiteralDictionaryEncodedColumn nestedColumn = - (NestedFieldLiteralDictionaryEncodedColumn) nestedColumnHolder.getColumn(); + NestedFieldDictionaryEncodedColumn nestedColumn = + (NestedFieldDictionaryEncodedColumn) nestedColumnHolder.getColumn(); Assert.assertNotNull(nestedColumn); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java similarity index 92% rename from processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java rename to processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java index de6dc0d44464..13e5e2b535a6 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java @@ -60,7 +60,7 @@ import java.nio.channels.WritableByteChannel; import java.util.TreeSet; -public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHandlingTest +public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingTest { BitmapSerdeFactory roaringFactory = RoaringBitmapSerdeFactory.getInstance(); BitmapResultFactory bitmapResultFactory = new DefaultBitmapResultFactory( @@ -139,7 +139,7 @@ public void setup() throws IOException @Test public void testSingleTypeStringColumnValueIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); NullValueIndex nullIndex = indexSupplier.as(NullValueIndex.class); Assert.assertNotNull(nullIndex); @@ -161,7 +161,7 @@ public void testSingleTypeStringColumnValueIndex() throws IOException @Test public void testSingleTypeStringColumnValueSetIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); Assert.assertNotNull(valueSetIndex); @@ -194,7 +194,7 @@ public void testSingleTypeStringColumnValueSetIndex() throws IOException @Test public void testSingleTypeStringColumnRangeIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class); Assert.assertNotNull(rangeIndex); @@ -351,7 +351,7 @@ public void testSingleTypeStringColumnRangeIndex() throws IOException @Test public void testSingleTypeStringColumnRangeIndexWithPredicate() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class); Assert.assertNotNull(rangeIndex); @@ -419,7 +419,7 @@ public void testSingleTypeStringColumnRangeIndexWithPredicate() throws IOExcepti @Test public void testSingleTypeStringColumnPredicateIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); Assert.assertNotNull(predicateIndex); @@ -442,7 +442,7 @@ public void testSingleTypeStringColumnPredicateIndex() throws IOException @Test public void testSingleTypeStringColumnWithNullValueIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeStringWithNullsSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringWithNullsSupplier(); NullValueIndex nullIndex = indexSupplier.as(NullValueIndex.class); Assert.assertNotNull(nullIndex); @@ -461,7 +461,7 @@ public void testSingleTypeStringColumnWithNullValueIndex() throws IOException @Test public void testSingleTypeStringColumnWithNullValueSetIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeStringWithNullsSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringWithNullsSupplier(); StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); Assert.assertNotNull(valueSetIndex); @@ -494,7 +494,7 @@ public void testSingleTypeStringColumnWithNullValueSetIndex() throws IOException @Test public void testSingleValueStringWithNullRangeIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeStringWithNullsSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringWithNullsSupplier(); LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class); Assert.assertNotNull(rangeIndex); @@ -584,7 +584,7 @@ public void testSingleValueStringWithNullRangeIndex() throws IOException @Test public void testSingleValueStringWithNullPredicateIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeStringWithNullsSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringWithNullsSupplier(); DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); Assert.assertNotNull(predicateIndex); @@ -607,7 +607,7 @@ public void testSingleValueStringWithNullPredicateIndex() throws IOException @Test public void testSingleTypeLongColumnValueSetIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplier(); StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); Assert.assertNotNull(valueSetIndex); @@ -636,7 +636,7 @@ public void testSingleTypeLongColumnValueSetIndex() throws IOException @Test public void testSingleTypeLongColumnRangeIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplier(); NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class); Assert.assertNotNull(rangeIndex); @@ -705,7 +705,7 @@ public void testSingleTypeLongColumnRangeIndex() throws IOException @Test public void testSingleTypeLongColumnPredicateIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplier(); DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); Assert.assertNotNull(predicateIndex); @@ -728,7 +728,7 @@ public void testSingleTypeLongColumnPredicateIndex() throws IOException @Test public void testSingleTypeLongColumnWithNullValueIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplierWithNull(); NullValueIndex nullIndex = indexSupplier.as(NullValueIndex.class); Assert.assertNotNull(nullIndex); @@ -747,7 +747,7 @@ public void testSingleTypeLongColumnWithNullValueIndex() throws IOException @Test public void testSingleTypeLongColumnWithNullValueSetIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplierWithNull(); StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); Assert.assertNotNull(valueSetIndex); @@ -792,7 +792,7 @@ public void testSingleTypeLongColumnWithNullValueSetIndex() throws IOException @Test public void testSingleValueLongWithNullRangeIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplierWithNull(); NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class); Assert.assertNotNull(rangeIndex); @@ -857,7 +857,7 @@ public void testSingleValueLongWithNullRangeIndex() throws IOException @Test public void testSingleValueLongWithNullPredicateIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplierWithNull(); DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); Assert.assertNotNull(predicateIndex); @@ -880,7 +880,7 @@ public void testSingleValueLongWithNullPredicateIndex() throws IOException @Test public void testSingleTypeDoubleColumnValueSetIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplier(); StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); Assert.assertNotNull(valueSetIndex); @@ -909,7 +909,7 @@ public void testSingleTypeDoubleColumnValueSetIndex() throws IOException @Test public void testSingleTypeDoubleColumnRangeIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplier(); NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class); Assert.assertNotNull(rangeIndex); @@ -1002,7 +1002,7 @@ public void testSingleTypeDoubleColumnRangeIndex() throws IOException @Test public void testSingleTypeDoubleColumnPredicateIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplier(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplier(); DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); Assert.assertNotNull(predicateIndex); @@ -1025,7 +1025,7 @@ public void testSingleTypeDoubleColumnPredicateIndex() throws IOException @Test public void testSingleTypeDoubleColumnWithNullValueIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplierWithNull(); NullValueIndex nullIndex = indexSupplier.as(NullValueIndex.class); Assert.assertNotNull(nullIndex); @@ -1044,7 +1044,7 @@ public void testSingleTypeDoubleColumnWithNullValueIndex() throws IOException @Test public void testSingleTypeDoubleColumnWithNullValueSetIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplierWithNull(); StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); Assert.assertNotNull(valueSetIndex); @@ -1089,7 +1089,7 @@ public void testSingleTypeDoubleColumnWithNullValueSetIndex() throws IOException @Test public void testSingleValueDoubleWithNullRangeIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplierWithNull(); NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class); Assert.assertNotNull(rangeIndex); @@ -1139,7 +1139,7 @@ public void testSingleValueDoubleWithNullRangeIndex() throws IOException @Test public void testSingleValueDoubleWithNullPredicateIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplierWithNull(); DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); Assert.assertNotNull(predicateIndex); @@ -1162,7 +1162,7 @@ public void testSingleValueDoubleWithNullPredicateIndex() throws IOException @Test public void testVariantNullValueIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); NullValueIndex nullIndex = indexSupplier.as(NullValueIndex.class); Assert.assertNotNull(nullIndex); @@ -1184,7 +1184,7 @@ public void testVariantNullValueIndex() throws IOException @Test public void testVariantValueSetIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); Assert.assertNotNull(valueSetIndex); @@ -1242,7 +1242,7 @@ public void testVariantValueSetIndex() throws IOException @Test public void testVariantRangeIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class); Assert.assertNull(rangeIndex); @@ -1254,7 +1254,7 @@ public void testVariantRangeIndex() throws IOException @Test public void testVariantPredicateIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); Assert.assertNotNull(predicateIndex); @@ -1277,7 +1277,7 @@ public void testVariantPredicateIndex() throws IOException @Test public void testDictionaryEncodedStringValueIndex() throws IOException { - NestedFieldLiteralColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); + NestedFieldColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); DictionaryEncodedStringValueIndex lowLevelIndex = indexSupplier.as(DictionaryEncodedStringValueIndex.class); Assert.assertNotNull(lowLevelIndex); @@ -1398,11 +1398,11 @@ public void testEnsureNoImproperSelectionFromAdjustedGlobals() throws IOExceptio GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); - NestedFieldLiteralColumnIndexSupplier indexSupplier = new NestedFieldLiteralColumnIndexSupplier<>( - new NestedLiteralTypeInfo.TypeSet( - new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.STRING) - .add(ColumnType.LONG) - .getByteValue() + NestedFieldColumnIndexSupplier indexSupplier = new NestedFieldColumnIndexSupplier<>( + new NestedFieldTypeInfo.TypeSet( + new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING) + .add(ColumnType.LONG) + .getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1440,7 +1440,7 @@ public void testEnsureNoImproperSelectionFromAdjustedGlobals() throws IOExceptio checkBitmap(bitmap); } - private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringSupplier() throws IOException + private NestedFieldColumnIndexSupplier makeSingleTypeStringSupplier() throws IOException { ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder()); ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12); @@ -1498,9 +1498,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringSupplier() GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); - return new NestedFieldLiteralColumnIndexSupplier<>( - new NestedLiteralTypeInfo.TypeSet( - new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue() + return new NestedFieldColumnIndexSupplier<>( + new NestedFieldTypeInfo.TypeSet( + new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1513,7 +1513,7 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringSupplier() ); } - private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringWithNullsSupplier() throws IOException + private NestedFieldColumnIndexSupplier makeSingleTypeStringWithNullsSupplier() throws IOException { ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder()); ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12); @@ -1574,9 +1574,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringWithNullsSu GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); - return new NestedFieldLiteralColumnIndexSupplier<>( - new NestedLiteralTypeInfo.TypeSet( - new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue() + return new NestedFieldColumnIndexSupplier<>( + new NestedFieldTypeInfo.TypeSet( + new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1589,7 +1589,7 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringWithNullsSu ); } - private NestedFieldLiteralColumnIndexSupplier makeSingleTypeLongSupplier() throws IOException + private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplier() throws IOException { ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder()); ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12); @@ -1647,9 +1647,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeLongSupplier() th GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); - return new NestedFieldLiteralColumnIndexSupplier<>( - new NestedLiteralTypeInfo.TypeSet( - new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue() + return new NestedFieldColumnIndexSupplier<>( + new NestedFieldTypeInfo.TypeSet( + new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1662,7 +1662,7 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeLongSupplier() th ); } - private NestedFieldLiteralColumnIndexSupplier makeSingleTypeLongSupplierWithNull() throws IOException + private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplierWithNull() throws IOException { ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder()); ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12); @@ -1724,9 +1724,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeLongSupplierWithN GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); - return new NestedFieldLiteralColumnIndexSupplier<>( - new NestedLiteralTypeInfo.TypeSet( - new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue() + return new NestedFieldColumnIndexSupplier<>( + new NestedFieldTypeInfo.TypeSet( + new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1739,7 +1739,7 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeLongSupplierWithN ); } - private NestedFieldLiteralColumnIndexSupplier makeSingleTypeDoubleSupplier() throws IOException + private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplier() throws IOException { ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder()); ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12); @@ -1797,9 +1797,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeDoubleSupplier() GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); - return new NestedFieldLiteralColumnIndexSupplier<>( - new NestedLiteralTypeInfo.TypeSet( - new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue() + return new NestedFieldColumnIndexSupplier<>( + new NestedFieldTypeInfo.TypeSet( + new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1812,7 +1812,7 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeDoubleSupplier() ); } - private NestedFieldLiteralColumnIndexSupplier makeSingleTypeDoubleSupplierWithNull() throws IOException + private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplierWithNull() throws IOException { ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder()); ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12); @@ -1874,9 +1874,9 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeDoubleSupplierWit GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); - return new NestedFieldLiteralColumnIndexSupplier<>( - new NestedLiteralTypeInfo.TypeSet( - new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue() + return new NestedFieldColumnIndexSupplier<>( + new NestedFieldTypeInfo.TypeSet( + new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1889,7 +1889,7 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeDoubleSupplierWit ); } - private NestedFieldLiteralColumnIndexSupplier makeVariantSupplierWithNull() throws IOException + private NestedFieldColumnIndexSupplier makeVariantSupplierWithNull() throws IOException { ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder()); ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12); @@ -1959,12 +1959,12 @@ private NestedFieldLiteralColumnIndexSupplier makeVariantSupplierWithNull() t GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); - return new NestedFieldLiteralColumnIndexSupplier<>( - new NestedLiteralTypeInfo.TypeSet( - new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.STRING) - .add(ColumnType.LONG) - .add(ColumnType.DOUBLE) - .getByteValue() + return new NestedFieldColumnIndexSupplier<>( + new NestedFieldTypeInfo.TypeSet( + new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING) + .add(ColumnType.LONG) + .add(ColumnType.DOUBLE) + .getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java similarity index 98% rename from processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnSelectorsTest.java rename to processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java index 0c7e68cd2c96..486bb6df4f69 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.Module; import com.google.common.collect.ImmutableList; -import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.UOE; @@ -60,7 +59,7 @@ import java.io.IOException; import java.util.List; -public class NestedFieldLiteralColumnSelectorsTest +public class NestedFieldColumnSelectorsTest { private static final String NESTED_LONG_FIELD = "long"; private static final String NESTED_DOUBLE_FIELD = "double"; @@ -78,7 +77,7 @@ public class NestedFieldLiteralColumnSelectorsTest private final AggregationTestHelper helper; private final Closer closer; - public NestedFieldLiteralColumnSelectorsTest() + public NestedFieldColumnSelectorsTest() { NestedDataModule.registerHandlersAndSerde(); List mods = NestedDataModule.getJacksonModulesList(); @@ -331,7 +330,7 @@ private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns vir tempFolder, closer, NestedDataTestUtils.NUMERIC_DATA_FILE, - JsonInputFormat.DEFAULT, + NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT, NestedDataTestUtils.TIMESTAMP_SPEC, NestedDataTestUtils.AUTO_DISCOVERY, TransformSpec.NONE, @@ -361,7 +360,7 @@ private VectorColumnSelectorFactory getVectorColumnSelectorFactory(VirtualColumn tempFolder, closer, NestedDataTestUtils.NUMERIC_DATA_FILE, - JsonInputFormat.DEFAULT, + NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT, NestedDataTestUtils.TIMESTAMP_SPEC, NestedDataTestUtils.AUTO_DISCOVERY, TransformSpec.NONE, diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedLiteralTypeInfoTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java similarity index 74% rename from processing/src/test/java/org/apache/druid/segment/nested/NestedLiteralTypeInfoTest.java rename to processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java index bcf5e7c2c356..aa100cf7afd2 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedLiteralTypeInfoTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java @@ -35,7 +35,7 @@ import java.util.List; import java.util.Set; -public class NestedLiteralTypeInfoTest +public class NestedFieldTypeInfoTest { private static final ByteBuffer BUFFER = ByteBuffer.allocate(1024).order(ByteOrder.nativeOrder()); @@ -75,11 +75,11 @@ public void testMultiType() throws IOException @Test public void testEqualsAndHashCode() { - EqualsVerifier.forClass(NestedLiteralTypeInfo.TypeSet.class) + EqualsVerifier.forClass(NestedFieldTypeInfo.TypeSet.class) .usingGetClass() .verify(); - EqualsVerifier.forClass(NestedLiteralTypeInfo.MutableTypeSet.class) + EqualsVerifier.forClass(NestedFieldTypeInfo.MutableTypeSet.class) .suppress(Warning.NONFINAL_FIELDS) .usingGetClass() .verify(); @@ -87,60 +87,60 @@ public void testEqualsAndHashCode() private void testSingleType(ColumnType columnType) throws IOException { - NestedLiteralTypeInfo.MutableTypeSet typeSet = new NestedLiteralTypeInfo.MutableTypeSet(); + NestedFieldTypeInfo.MutableTypeSet typeSet = new NestedFieldTypeInfo.MutableTypeSet(); Assert.assertNull(typeSet.getSingleType()); Assert.assertTrue(typeSet.isEmpty()); typeSet.add(columnType); Assert.assertEquals(columnType, typeSet.getSingleType()); - Assert.assertEquals(ImmutableSet.of(columnType), NestedLiteralTypeInfo.convertToSet(typeSet.getByteValue())); + Assert.assertEquals(ImmutableSet.of(columnType), NestedFieldTypeInfo.convertToSet(typeSet.getByteValue())); writeTypeSet(typeSet); - NestedLiteralTypeInfo info = new NestedLiteralTypeInfo(BUFFER); + NestedFieldTypeInfo info = new NestedFieldTypeInfo(BUFFER); Assert.assertEquals(0, BUFFER.position()); - NestedLiteralTypeInfo.TypeSet roundTrip = info.getTypes(0); + NestedFieldTypeInfo.TypeSet roundTrip = info.getTypes(0); Assert.assertEquals(columnType, roundTrip.getSingleType()); - NestedLiteralTypeInfo info2 = NestedLiteralTypeInfo.read(BUFFER, 1); + NestedFieldTypeInfo info2 = NestedFieldTypeInfo.read(BUFFER, 1); Assert.assertEquals(info.getTypes(0), info2.getTypes(0)); Assert.assertEquals(1, BUFFER.position()); } private void testMultiType(Set columnTypes) throws IOException { - NestedLiteralTypeInfo.MutableTypeSet typeSet = new NestedLiteralTypeInfo.MutableTypeSet(); + NestedFieldTypeInfo.MutableTypeSet typeSet = new NestedFieldTypeInfo.MutableTypeSet(); Assert.assertNull(typeSet.getSingleType()); Assert.assertTrue(typeSet.isEmpty()); - NestedLiteralTypeInfo.MutableTypeSet merge = new NestedLiteralTypeInfo.MutableTypeSet(); + NestedFieldTypeInfo.MutableTypeSet merge = new NestedFieldTypeInfo.MutableTypeSet(); for (ColumnType columnType : columnTypes) { typeSet.add(columnType); - merge.merge(new NestedLiteralTypeInfo.MutableTypeSet().add(columnType).getByteValue()); + merge.merge(new NestedFieldTypeInfo.MutableTypeSet().add(columnType).getByteValue()); } Assert.assertEquals(merge.getByteValue(), typeSet.getByteValue()); Assert.assertNull(typeSet.getSingleType()); - Assert.assertEquals(columnTypes, NestedLiteralTypeInfo.convertToSet(typeSet.getByteValue())); + Assert.assertEquals(columnTypes, NestedFieldTypeInfo.convertToSet(typeSet.getByteValue())); writeTypeSet(typeSet); - NestedLiteralTypeInfo info = new NestedLiteralTypeInfo(BUFFER); + NestedFieldTypeInfo info = new NestedFieldTypeInfo(BUFFER); Assert.assertEquals(0, BUFFER.position()); - NestedLiteralTypeInfo.TypeSet roundTrip = info.getTypes(0); + NestedFieldTypeInfo.TypeSet roundTrip = info.getTypes(0); Assert.assertNull(roundTrip.getSingleType()); - Assert.assertEquals(columnTypes, NestedLiteralTypeInfo.convertToSet(roundTrip.getByteValue())); + Assert.assertEquals(columnTypes, NestedFieldTypeInfo.convertToSet(roundTrip.getByteValue())); - NestedLiteralTypeInfo info2 = NestedLiteralTypeInfo.read(BUFFER, 1); + NestedFieldTypeInfo info2 = NestedFieldTypeInfo.read(BUFFER, 1); Assert.assertEquals(info.getTypes(0), info2.getTypes(0)); Assert.assertEquals(1, BUFFER.position()); } - private static void writeTypeSet(NestedLiteralTypeInfo.MutableTypeSet typeSet) throws IOException + private static void writeTypeSet(NestedFieldTypeInfo.MutableTypeSet typeSet) throws IOException { BUFFER.position(0); - NestedLiteralTypeInfo.Writer writer = new NestedLiteralTypeInfo.Writer(new OnHeapMemorySegmentWriteOutMedium()); + NestedFieldTypeInfo.Writer writer = new NestedFieldTypeInfo.Writer(new OnHeapMemorySegmentWriteOutMedium()); writer.open(); writer.write(typeSet); Assert.assertEquals(1, writer.getSerializedSize()); diff --git a/processing/src/test/resources/nested-array-test-data.json b/processing/src/test/resources/nested-array-test-data.json index 921077103e3a..44e23b84b244 100644 --- a/processing/src/test/resources/nested-array-test-data.json +++ b/processing/src/test/resources/nested-array-test-data.json @@ -2,13 +2,13 @@ {"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayObject":[{"x": 3},{"x":4}]} {"timestamp": "2023-01-01T00:00:00", "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayObject":[null,{"x":2}]} {"timestamp": "2023-01-01T00:00:00", "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayObject":[{"x": null},{"x":2}]} -{"timestamp": "2023-01-01T00:00:00", "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":null, "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayObject":[{"x": 1000},{"y":2000}]} +{"timestamp": "2023-01-01T00:00:00", "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":[], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayObject":[{"x": 1000},{"y":2000}]} {"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"a": 1},{"b":2}]} -{"timestamp": "2023-01-01T00:00:00", "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null, 1.1], "arrayVariant":null, "arrayObject":[{"x": 1},{"x":2}]} -{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"x": 1},{"x":2}]} +{"timestamp": "2023-01-01T00:00:00", "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayObject":[{"x": 1},{"x":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": [], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"x": 1},{"x":2}]} {"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayObject":[{"x": 3},{"x":4}]} {"timestamp": "2023-01-02T00:00:00", "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayObject":[null,{"x":2}]} -{"timestamp": "2023-01-02T00:00:00", "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayObject":[{"x": null},{"x":2}]} -{"timestamp": "2023-01-02T00:00:00", "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":null, "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayObject":[{"x": 1000},{"y":2000}]} -{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"a": 1},{"b":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[null], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayObject":[{"x": null},{"x":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":null, "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[], "arrayObject":[{"x": 1000},{"y":2000}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": [null], "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"a": 1},{"b":2}]} {"timestamp": "2023-01-02T00:00:00", "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null, 1.1], "arrayVariant":null, "arrayObject":[{"x": 1},{"x":2}]} \ No newline at end of file diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 0a59bb0cc2e7..17b02089592b 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -93,7 +93,7 @@ import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.nested.CompressedNestedDataComplexColumn; -import org.apache.druid.segment.nested.NestedFieldLiteralDictionaryEncodedColumn; +import org.apache.druid.segment.nested.NestedFieldDictionaryEncodedColumn; import org.apache.druid.segment.nested.NestedPathFinder; import org.apache.druid.segment.nested.NestedPathPart; import org.apache.druid.timeline.SegmentId; @@ -590,8 +590,8 @@ public static void runDumpNestedColumnPath( final ColumnIndexSupplier indexSupplier = nestedDataColumn.getColumnIndexSupplier(pathParts); final ColumnHolder nestedPathColumnHolder = nestedDataColumn.getColumnHolder(pathParts); - final NestedFieldLiteralDictionaryEncodedColumn nestedPathColumn = - (NestedFieldLiteralDictionaryEncodedColumn) nestedPathColumnHolder.getColumn(); + final NestedFieldDictionaryEncodedColumn nestedPathColumn = + (NestedFieldDictionaryEncodedColumn) nestedPathColumnHolder.getColumn(); final FixedIndexed nestedPathDictionary = nestedPathColumn.getDictionary(); SimpleAscendingOffset offset = new SimpleAscendingOffset(index.getNumRows()); 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 247dc7c9d7f8..ea85780bd7b0 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 @@ -190,6 +190,7 @@ public static void setupNullValues() public static final Map QUERY_CONTEXT_NO_STRINGIFY_ARRAY = DEFAULT_QUERY_CONTEXT_BUILDER.put(QueryContexts.CTX_SQL_STRINGIFY_ARRAYS, false) + .put(PlannerContext.CTX_ENABLE_UNNEST, true) .build(); public static final Map QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS = ImmutableMap.of( @@ -704,23 +705,6 @@ public void testQuery( .run(); } - public void testQuery( - final String sql, - final Map queryContext, - final List> expectedQueries, - final List expectedResults, - final RowSignature expectedResultSignature - ) - { - testBuilder() - .sql(sql) - .queryContext(queryContext) - .expectedQueries(expectedQueries) - .expectedResults(expectedResults) - .expectedSignature(expectedResultSignature) - .run(); - } - public void testQuery( final String sql, final Map queryContext, 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 80d7a5636cc6..2cd15582d318 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 @@ -2830,8 +2830,6 @@ public void testUnnestTwice() .build() ), ImmutableList.of( - new Object[]{"", ImmutableList.of("a", "b"), useDefault ? null : ImmutableList.of(""), "", "a"}, - new Object[]{"", ImmutableList.of("a", "b"), useDefault ? null : ImmutableList.of(""), "", "b"}, new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "10", "b"}, new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "10", "c"}, new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "1", "b"}, @@ -3217,8 +3215,7 @@ public void testUnnestWithFilters() ), ImmutableList.of( new Object[]{"a"}, - new Object[]{"b"}, - new Object[]{""} + new Object[]{"b"} ) ); } @@ -3266,8 +3263,7 @@ public void testUnnestWithFiltersInsideAndOutside() .build() ), ImmutableList.of( - new Object[]{"a"}, - new Object[]{""} + new Object[]{"a"} ) ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 2db9861dad01..6353e6ec4316 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -29,7 +29,6 @@ import org.apache.druid.data.input.ResourceInputSource; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -41,6 +40,8 @@ import org.apache.druid.query.Druids; import org.apache.druid.query.NestedDataTestUtils; import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.UnnestDataSource; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory; @@ -289,7 +290,7 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker( NestedDataTestUtils.ARRAY_TYPES_DATA_FILE ) ) - .inputFormat(JsonInputFormat.DEFAULT) + .inputFormat(TestDataBuilder.DEFAULT_JSON_INPUT_FORMAT) .inputTmpDir(temporaryFolder.newFolder()) .buildMMappedIndex(); @@ -860,708 +861,1189 @@ public void testGroupByRootSingleTypeStringMixed2SparseJsonValueNonExistentPath( @Test public void testJsonValueArrays() { - testQuery( - "SELECT " - + "JSON_VALUE(arrayString, '$' RETURNING VARCHAR ARRAY), " - + "JSON_VALUE(arrayLong, '$' RETURNING BIGINT ARRAY), " - + "JSON_VALUE(arrayDouble, '$' RETURNING DOUBLE ARRAY) " - + "FROM druid.arrays", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - Druids.newScanQueryBuilder() - .dataSource(DATA_SOURCE_ARRAYS) - .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns( - new NestedFieldVirtualColumn("arrayString", "$", "v0", ColumnType.STRING_ARRAY), - new NestedFieldVirtualColumn("arrayLong", "$", "v1", ColumnType.LONG_ARRAY), - new NestedFieldVirtualColumn("arrayDouble", "$", "v2", ColumnType.DOUBLE_ARRAY) - ) - .columns("v0", "v1", "v2") - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) - .build() - ), - ImmutableList.of( - new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, - new Object[]{null, null, null}, - new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2, 3.3, 4.0)}, - new Object[]{Arrays.asList("a", "b"), null, null}, - new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, - new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1, 3.3)}, - new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3, 4.4, 5.5)}, - new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, - new Object[]{null, null, null}, - new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2, 3.3, 4.0)}, - new Object[]{Arrays.asList("a", "b"), null, null}, - new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, - new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1, 3.3)}, - new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3, 4.4, 5.5)} - ), - RowSignature.builder() - .add("EXPR$0", ColumnType.STRING_ARRAY) - .add("EXPR$1", ColumnType.LONG_ARRAY) - .add("EXPR$2", ColumnType.DOUBLE_ARRAY) - .build() - ); + testBuilder() + .sql( + "SELECT " + + "JSON_VALUE(arrayString, '$' RETURNING VARCHAR ARRAY), " + + "JSON_VALUE(arrayLong, '$' RETURNING BIGINT ARRAY), " + + "JSON_VALUE(arrayDouble, '$' RETURNING DOUBLE ARRAY) " + + "FROM druid.arrays" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(DATA_SOURCE_ARRAYS) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns( + new NestedFieldVirtualColumn("arrayString", "$", "v0", ColumnType.STRING_ARRAY), + new NestedFieldVirtualColumn("arrayLong", "$", "v1", ColumnType.LONG_ARRAY), + new NestedFieldVirtualColumn("arrayDouble", "$", "v2", ColumnType.DOUBLE_ARRAY) + ) + .columns("v0", "v1", "v2") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, + new Object[]{null, null, null}, + new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2, 3.3, 4.0)}, + new Object[]{Arrays.asList("a", "b"), null, null}, + new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, + new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1, 3.3)}, + new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3, 4.4, 5.5)}, + new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, + new Object[]{null, null, null}, + new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2, 3.3, 4.0)}, + new Object[]{Arrays.asList("a", "b"), null, null}, + new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, + new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1, 3.3)}, + new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3, 4.4, 5.5)} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING_ARRAY) + .add("EXPR$1", ColumnType.LONG_ARRAY) + .add("EXPR$2", ColumnType.DOUBLE_ARRAY) + .build() + ) + .run(); + } + + @Test + public void testUnnestRootSingleTypeArrayLongNulls() + { + cannotVectorize(); + testBuilder() + .sql("SELECT longs FROM druid.arrays, UNNEST(arrayLongNulls) as u(longs)") + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource( + UnnestDataSource.create( + TableDataSource.create(DATA_SOURCE_ARRAYS), + expressionVirtualColumn("j0.unnest", "\"arrayLongNulls\"", ColumnType.LONG_ARRAY), + null + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("j0.unnest") + .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .legacy(false) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{2L}, + new Object[]{3L}, + new Object[]{1L}, + new Object[]{null}, + new Object[]{2L}, + new Object[]{9L}, + new Object[]{1L}, + new Object[]{null}, + new Object[]{3L}, + new Object[]{1L}, + new Object[]{2L}, + new Object[]{3L}, + new Object[]{2L}, + new Object[]{3L}, + new Object[]{null}, + new Object[]{null}, + new Object[]{2L}, + new Object[]{9L}, + new Object[]{1L}, + new Object[]{null}, + new Object[]{3L}, + new Object[]{1L}, + new Object[]{2L}, + new Object[]{3L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("longs", ColumnType.LONG) + .build() + ) + .run(); } + @Test + public void testUnnestRootSingleTypeArrayStringNulls() + { + cannotVectorize(); + testBuilder() + .sql("SELECT strings FROM druid.arrays, UNNEST(arrayStringNulls) as u(strings)") + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource( + UnnestDataSource.create( + TableDataSource.create(DATA_SOURCE_ARRAYS), + expressionVirtualColumn("j0.unnest", "\"arrayStringNulls\"", ColumnType.STRING_ARRAY), + null + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("j0.unnest") + .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .legacy(false) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{"a"}, + new Object[]{"b"}, + new Object[]{"b"}, + new Object[]{"b"}, + new Object[]{"a"}, + new Object[]{"b"}, + new Object[]{"d"}, + new Object[]{NullHandling.defaultStringValue()}, + new Object[]{"b"}, + new Object[]{NullHandling.defaultStringValue()}, + new Object[]{"b"}, + new Object[]{"a"}, + new Object[]{"b"}, + new Object[]{"b"}, + new Object[]{"b"}, + new Object[]{NullHandling.defaultStringValue()}, + new Object[]{"d"}, + new Object[]{NullHandling.defaultStringValue()}, + new Object[]{"b"}, + new Object[]{NullHandling.defaultStringValue()}, + new Object[]{"b"} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("strings", ColumnType.STRING) + .build() + ) + .run(); + } + + @Test + public void testUnnestRootSingleTypeArrayDoubleNulls() + { + cannotVectorize(); + testBuilder() + .sql("SELECT doubles FROM druid.arrays, UNNEST(arrayDoubleNulls) as u(doubles)") + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource( + UnnestDataSource.create( + TableDataSource.create(DATA_SOURCE_ARRAYS), + expressionVirtualColumn("j0.unnest", "\"arrayDoubleNulls\"", ColumnType.DOUBLE_ARRAY), + null + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("j0.unnest") + .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .legacy(false) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{null}, + new Object[]{999.0D}, + new Object[]{5.5D}, + new Object[]{null}, + new Object[]{1.1D}, + new Object[]{2.2D}, + new Object[]{null}, + new Object[]{null}, + new Object[]{2.2D}, + new Object[]{null}, + new Object[]{999.0D}, + new Object[]{null}, + new Object[]{5.5D}, + new Object[]{null}, + new Object[]{1.1D}, + new Object[]{999.0D}, + new Object[]{5.5D}, + new Object[]{null}, + new Object[]{1.1D}, + new Object[]{2.2D}, + new Object[]{null}, + new Object[]{null}, + new Object[]{2.2D}, + new Object[]{null}, + new Object[]{999.0D}, + new Object[]{null}, + new Object[]{5.5D} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("doubles", ColumnType.DOUBLE) + .build() + ) + .run(); + } + + @Test public void testGroupByRootSingleTypeArrayLong() { cannotVectorize(); - testQuery( - "SELECT " - + "arrayLong, " - + "SUM(cnt) " - + "FROM druid.arrays GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("arrayLong", "d0", ColumnType.LONG_ARRAY) + testBuilder() + .sql( + "SELECT " + + "arrayLong, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayLong", "d0", ColumnType.LONG_ARRAY) + ) ) - ) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{null, 4L}, + new Object[]{Arrays.asList(1L, 2L, 3L), 4L}, + new Object[]{Arrays.asList(1L, 2L, 3L, 4L), 2L}, + new Object[]{Arrays.asList(1L, 4L), 2L}, + new Object[]{Arrays.asList(2L, 3L), 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayLong", ColumnType.LONG_ARRAY) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{null, 4L}, - new Object[]{Arrays.asList(1L, 2L, 3L), 4L}, - new Object[]{Arrays.asList(1L, 2L, 3L, 4L), 2L}, - new Object[]{Arrays.asList(1L, 4L), 2L}, - new Object[]{Arrays.asList(2L, 3L), 2L} - ), - RowSignature.builder() - .add("arrayLong", ColumnType.LONG_ARRAY) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayLongNulls() { cannotVectorize(); - testQuery( - "SELECT " - + "arrayLongNulls, " - + "SUM(cnt) " - + "FROM druid.arrays GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("arrayLongNulls", "d0", ColumnType.LONG_ARRAY) + testBuilder() + .sql( + "SELECT " + + "arrayLongNulls, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayLongNulls", "d0", ColumnType.LONG_ARRAY) + ) ) - ) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{null, 3L}, + new Object[]{Collections.emptyList(), 1L}, + new Object[]{Collections.singletonList(null), 1L}, + new Object[]{Arrays.asList(null, 2L, 9L), 2L}, + new Object[]{Collections.singletonList(1L), 1L}, + new Object[]{Arrays.asList(1L, null, 3L), 2L}, + new Object[]{Arrays.asList(1L, 2L, 3L), 2L}, + new Object[]{Arrays.asList(2L, 3L), 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayLongNulls", ColumnType.LONG_ARRAY) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{null, 4L}, - new Object[]{Arrays.asList(null, 2L, 9L), 2L}, - new Object[]{Collections.singletonList(1L), 2L}, - new Object[]{Arrays.asList(1L, null, 3L), 2L}, - new Object[]{Arrays.asList(1L, 2L, 3L), 2L}, - new Object[]{Arrays.asList(2L, 3L), 2L} - ), - RowSignature.builder() - .add("arrayLongNulls", ColumnType.LONG_ARRAY) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); + } + + @Test + public void testGroupByRootSingleTypeArrayLongNullsUnnest() + { + cannotVectorize(); + testBuilder() + .sql( + "SELECT " + + "longs, " + + "SUM(cnt) " + + "FROM druid.arrays, UNNEST(arrayLongNulls) as u (longs) GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + UnnestDataSource.create( + TableDataSource.create(DATA_SOURCE_ARRAYS), + expressionVirtualColumn("j0.unnest", "\"arrayLongNulls\"", ColumnType.LONG_ARRAY), + null + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("j0.unnest", "d0", ColumnType.LONG) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{NullHandling.defaultLongValue(), 5L}, + new Object[]{1L, 5L}, + new Object[]{2L, 6L}, + new Object[]{3L, 6L}, + new Object[]{9L, 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("longs", ColumnType.LONG) + .add("EXPR$1", ColumnType.LONG) + .build() + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayLongNullsFiltered() { cannotVectorize(); - testQuery( - "SELECT " - + "arrayLongNulls, " - + "SUM(cnt), " - + "SUM(ARRAY_LENGTH(arrayLongNulls)) " - + "FROM druid.arrays " - + "WHERE ARRAY_CONTAINS(arrayLongNulls, 1) " - + "GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("arrayLongNulls", "d0", ColumnType.LONG_ARRAY) + testBuilder() + .sql( + "SELECT " + + "arrayLongNulls, " + + "SUM(cnt), " + + "SUM(ARRAY_LENGTH(arrayLongNulls)) " + + "FROM druid.arrays " + + "WHERE ARRAY_CONTAINS(arrayLongNulls, 1) " + + "GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayLongNulls", "d0", ColumnType.LONG_ARRAY) + ) ) - ) - .setVirtualColumns( - new ExpressionVirtualColumn("v0", "array_length(\"arrayLongNulls\")", ColumnType.LONG, queryFramework().macroTable()) - ) - .setDimFilter( - new ExpressionDimFilter("array_contains(\"arrayLongNulls\",1)", queryFramework().macroTable()) - ) - .setAggregatorSpecs( - aggregators( - new LongSumAggregatorFactory("a0", "cnt"), - new LongSumAggregatorFactory("a1", "v0") + .setVirtualColumns( + new ExpressionVirtualColumn("v0", "array_length(\"arrayLongNulls\")", ColumnType.LONG, queryFramework().macroTable()) ) - ) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setDimFilter( + new ExpressionDimFilter("array_contains(\"arrayLongNulls\",1)", queryFramework().macroTable()) + ) + .setAggregatorSpecs( + aggregators( + new LongSumAggregatorFactory("a0", "cnt"), + new LongSumAggregatorFactory("a1", "v0") + ) + ) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{Collections.singletonList(1L), 1L, 1L}, + new Object[]{Arrays.asList(1L, null, 3L), 2L, 6L}, + new Object[]{Arrays.asList(1L, 2L, 3L), 2L, 6L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayLongNulls", ColumnType.LONG_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .add("EXPR$2", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{Collections.singletonList(1L), 2L, 2L}, - new Object[]{Arrays.asList(1L, null, 3L), 2L, 6L}, - new Object[]{Arrays.asList(1L, 2L, 3L), 2L, 6L} - ), - RowSignature.builder() - .add("arrayLongNulls", ColumnType.LONG_ARRAY) - .add("EXPR$1", ColumnType.LONG) - .add("EXPR$2", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayString() { cannotVectorize(); - testQuery( - "SELECT " - + "arrayString, " - + "SUM(cnt) " - + "FROM druid.arrays GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("arrayString", "d0", ColumnType.STRING_ARRAY) + testBuilder() + .sql( + "SELECT " + + "arrayString, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayString", "d0", ColumnType.STRING_ARRAY) + ) ) - ) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{null, 4L}, + new Object[]{Arrays.asList("a", "b"), 4L}, + new Object[]{Arrays.asList("a", "b", "c"), 2L}, + new Object[]{Arrays.asList("b", "c"), 2L}, + new Object[]{Arrays.asList("d", "e"), 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayString", ColumnType.STRING_ARRAY) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{null, 4L}, - new Object[]{Arrays.asList("a", "b"), 4L}, - new Object[]{Arrays.asList("a", "b", "c"), 2L}, - new Object[]{Arrays.asList("b", "c"), 2L}, - new Object[]{Arrays.asList("d", "e"), 2L} - ), - RowSignature.builder() - .add("arrayString", ColumnType.STRING_ARRAY) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayStringNulls() { cannotVectorize(); - testQuery( - "SELECT " - + "arrayStringNulls, " - + "SUM(cnt) " - + "FROM druid.arrays GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("arrayStringNulls", "d0", ColumnType.STRING_ARRAY) + testBuilder() + .sql( + "SELECT " + + "arrayStringNulls, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayStringNulls", "d0", ColumnType.STRING_ARRAY) + ) ) - ) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{null, 3L}, + new Object[]{Collections.emptyList(), 1L}, + new Object[]{Collections.singletonList(null), 1L}, + new Object[]{Arrays.asList(null, "b"), 2L}, + new Object[]{Arrays.asList("a", "b"), 3L}, + new Object[]{Arrays.asList("b", "b"), 2L}, + new Object[]{Arrays.asList("d", null, "b"), 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayStringNulls", ColumnType.STRING_ARRAY) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{null, 4L}, - new Object[]{Arrays.asList(null, "b"), 2L}, - new Object[]{Arrays.asList("a", "b"), 4L}, - new Object[]{Arrays.asList("b", "b"), 2L}, - new Object[]{Arrays.asList("d", null, "b"), 2L} - ), - RowSignature.builder() - .add("arrayStringNulls", ColumnType.STRING_ARRAY) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); + } + + @Test + public void testGroupByRootSingleTypeArrayStringNullsUnnest() + { + cannotVectorize(); + testBuilder() + .sql( + "SELECT " + + "strings, " + + "SUM(cnt) " + + "FROM druid.arrays, unnest(arrayStringNulls) as u (strings) GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + UnnestDataSource.create( + TableDataSource.create(DATA_SOURCE_ARRAYS), + expressionVirtualColumn("j0.unnest", "\"arrayStringNulls\"", ColumnType.STRING_ARRAY), + null + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("j0.unnest", "d0", ColumnType.STRING) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{NullHandling.defaultStringValue(), 5L}, + new Object[]{"a", 3L}, + new Object[]{"b", 11L}, + new Object[]{"d", 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("strings", ColumnType.STRING) + .add("EXPR$1", ColumnType.LONG) + .build() + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayStringNullsFiltered() { cannotVectorize(); - testQuery( - "SELECT " - + "arrayStringNulls, " - + "SUM(cnt), " - + "SUM(ARRAY_LENGTH(arrayStringNulls)) " - + "FROM druid.arrays " - + "WHERE ARRAY_CONTAINS(arrayStringNulls, 'b') " - + "GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("arrayStringNulls", "d0", ColumnType.STRING_ARRAY) + testBuilder() + .sql( + "SELECT " + + "arrayStringNulls, " + + "SUM(cnt), " + + "SUM(ARRAY_LENGTH(arrayStringNulls)) " + + "FROM druid.arrays " + + "WHERE ARRAY_CONTAINS(arrayStringNulls, 'b') " + + "GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayStringNulls", "d0", ColumnType.STRING_ARRAY) + ) ) - ) - .setVirtualColumns( - new ExpressionVirtualColumn("v0", "array_length(\"arrayStringNulls\")", ColumnType.LONG, queryFramework().macroTable()) - ) - .setDimFilter( - new ExpressionDimFilter("array_contains(\"arrayStringNulls\",'b')", queryFramework().macroTable()) - ) - .setAggregatorSpecs( - aggregators( - new LongSumAggregatorFactory("a0", "cnt"), - new LongSumAggregatorFactory("a1", "v0") + .setVirtualColumns( + new ExpressionVirtualColumn("v0", "array_length(\"arrayStringNulls\")", ColumnType.LONG, queryFramework().macroTable()) ) - ) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setDimFilter( + new ExpressionDimFilter("array_contains(\"arrayStringNulls\",'b')", queryFramework().macroTable()) + ) + .setAggregatorSpecs( + aggregators( + new LongSumAggregatorFactory("a0", "cnt"), + new LongSumAggregatorFactory("a1", "v0") + ) + ) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{Arrays.asList(null, "b"), 2L, 4L}, + new Object[]{Arrays.asList("a", "b"), 3L, 6L}, + new Object[]{Arrays.asList("b", "b"), 2L, 4L}, + new Object[]{Arrays.asList("d", null, "b"), 2L, 6L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayStringNulls", ColumnType.STRING_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .add("EXPR$2", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{Arrays.asList(null, "b"), 2L, 4L}, - new Object[]{Arrays.asList("a", "b"), 4L, 8L}, - new Object[]{Arrays.asList("b", "b"), 2L, 4L}, - new Object[]{Arrays.asList("d", null, "b"), 2L, 6L} - ), - RowSignature.builder() - .add("arrayStringNulls", ColumnType.STRING_ARRAY) - .add("EXPR$1", ColumnType.LONG) - .add("EXPR$2", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayDouble() { cannotVectorize(); - testQuery( - "SELECT " - + "arrayDouble, " - + "SUM(cnt) " - + "FROM druid.arrays GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("arrayDouble", "d0", ColumnType.DOUBLE_ARRAY) + testBuilder() + .sql( + "SELECT " + + "arrayDouble, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayDouble", "d0", ColumnType.DOUBLE_ARRAY) + ) ) - ) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{null, 4L}, + new Object[]{Arrays.asList(1.1, 2.2, 3.3), 4L}, + new Object[]{Arrays.asList(1.1, 3.3), 2L}, + new Object[]{Arrays.asList(2.2, 3.3, 4.0), 2L}, + new Object[]{Arrays.asList(3.3, 4.4, 5.5), 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayDouble", ColumnType.DOUBLE_ARRAY) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{null, 4L}, - new Object[]{Arrays.asList(1.1, 2.2, 3.3), 4L}, - new Object[]{Arrays.asList(1.1, 3.3), 2L}, - new Object[]{Arrays.asList(2.2, 3.3, 4.0), 2L}, - new Object[]{Arrays.asList(3.3, 4.4, 5.5), 2L} - ), - RowSignature.builder() - .add("arrayDouble", ColumnType.DOUBLE_ARRAY) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayDoubleNulls() { cannotVectorize(); - testQuery( - "SELECT " - + "arrayDoubleNulls, " - + "SUM(cnt) " - + "FROM druid.arrays GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("arrayDoubleNulls", "d0", ColumnType.DOUBLE_ARRAY) + testBuilder() + .sql( + "SELECT " + + "arrayDoubleNulls, " + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayDoubleNulls", "d0", ColumnType.DOUBLE_ARRAY) + ) ) - ) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{null, 3L}, + new Object[]{Collections.emptyList(), 1L}, + new Object[]{Collections.singletonList(null), 1L}, + new Object[]{Arrays.asList(null, 1.1), 1L}, + new Object[]{Arrays.asList(null, 2.2, null), 2L}, + new Object[]{Arrays.asList(1.1, 2.2, null), 2L}, + new Object[]{Arrays.asList(999.0, null, 5.5), 2L}, + new Object[]{Arrays.asList(999.0, 5.5, null), 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{null, 4L}, - new Object[]{Arrays.asList(null, 1.1), 2L}, - new Object[]{Arrays.asList(null, 2.2, null), 2L}, - new Object[]{Arrays.asList(1.1, 2.2, null), 2L}, - new Object[]{Arrays.asList(999.0, null, 5.5), 2L}, - new Object[]{Arrays.asList(999.0, 5.5, null), 2L} - ), - RowSignature.builder() - .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); + } + + @Test + public void testGroupByRootSingleTypeArrayDoubleNullsUnnest() + { + cannotVectorize(); + testBuilder() + .sql( + "SELECT " + + "doubles, " + + "SUM(cnt) " + + "FROM druid.arrays, UNNEST(arrayDoubleNulls) as u (doubles) GROUP BY doubles" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + UnnestDataSource.create( + TableDataSource.create(DATA_SOURCE_ARRAYS), + expressionVirtualColumn("j0.unnest", "\"arrayDoubleNulls\"", ColumnType.DOUBLE_ARRAY), + null + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("j0.unnest", "d0", ColumnType.DOUBLE) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{NullHandling.defaultDoubleValue(), 12L}, + new Object[]{1.1D, 3L}, + new Object[]{2.2D, 4L}, + new Object[]{5.5D, 4L}, + new Object[]{999.0D, 4L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("doubles", ColumnType.DOUBLE) + .add("EXPR$1", ColumnType.LONG) + .build() + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayDoubleNullsFiltered() { cannotVectorize(); - testQuery( - "SELECT " - + "arrayDoubleNulls, " - + "SUM(cnt), " - + "SUM(ARRAY_LENGTH(arrayDoubleNulls)) " - + "FROM druid.arrays " - + "WHERE ARRAY_CONTAINS(arrayDoubleNulls, 2.2)" - + "GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("arrayDoubleNulls", "d0", ColumnType.DOUBLE_ARRAY) + testBuilder() + .sql( + "SELECT " + + "arrayDoubleNulls, " + + "SUM(cnt), " + + "SUM(ARRAY_LENGTH(arrayDoubleNulls)) " + + "FROM druid.arrays " + + "WHERE ARRAY_CONTAINS(arrayDoubleNulls, 2.2)" + + "GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayDoubleNulls", "d0", ColumnType.DOUBLE_ARRAY) + ) ) - ) - .setVirtualColumns( - new ExpressionVirtualColumn("v0", "array_length(\"arrayDoubleNulls\")", ColumnType.LONG, queryFramework().macroTable()) - ) - .setDimFilter( - new ExpressionDimFilter("array_contains(\"arrayDoubleNulls\",2.2)", queryFramework().macroTable()) - ) - .setAggregatorSpecs( - aggregators( - new LongSumAggregatorFactory("a0", "cnt"), - new LongSumAggregatorFactory("a1", "v0") + .setVirtualColumns( + new ExpressionVirtualColumn("v0", "array_length(\"arrayDoubleNulls\")", ColumnType.LONG, queryFramework().macroTable()) ) - ) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setDimFilter( + new ExpressionDimFilter("array_contains(\"arrayDoubleNulls\",2.2)", queryFramework().macroTable()) + ) + .setAggregatorSpecs( + aggregators( + new LongSumAggregatorFactory("a0", "cnt"), + new LongSumAggregatorFactory("a1", "v0") + ) + ) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{Arrays.asList(null, 2.2, null), 2L, 6L}, + new Object[]{Arrays.asList(1.1, 2.2, null), 2L, 6L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .add("EXPR$2", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{Arrays.asList(null, 2.2, null), 2L, 6L}, - new Object[]{Arrays.asList(1.1, 2.2, null), 2L, 6L} - ), - RowSignature.builder() - .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY) - .add("EXPR$1", ColumnType.LONG) - .add("EXPR$2", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayLongElement() { cannotVectorize(); - testQuery( - "SELECT " - + "JSON_VALUE(arrayLong, '$[1]' RETURNING BIGINT)," - + "SUM(cnt) " - + "FROM druid.arrays GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("v0", "d0", ColumnType.LONG) + testBuilder() + .sql( + "SELECT " + + "JSON_VALUE(arrayLong, '$[1]' RETURNING BIGINT)," + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG) + ) ) - ) - .setVirtualColumns( - new NestedFieldVirtualColumn("arrayLong", "$[1]", "v0", ColumnType.LONG) - ) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayLong", "$[1]", "v0", ColumnType.LONG) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{NullHandling.defaultLongValue(), 4L}, + new Object[]{2L, 6L}, + new Object[]{3L, 2L}, + new Object[]{4L, 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("EXPR$0", ColumnType.LONG) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{NullHandling.defaultLongValue(), 4L}, - new Object[]{2L, 6L}, - new Object[]{3L, 2L}, - new Object[]{4L, 2L} - ), - RowSignature.builder() - .add("EXPR$0", ColumnType.LONG) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayLongElementFiltered() { cannotVectorize(); - testQuery( - "SELECT " - + "JSON_VALUE(arrayLong, '$[1]' RETURNING BIGINT)," - + "SUM(cnt) " - + "FROM druid.arrays " - + "WHERE JSON_VALUE(arrayLong, '$[1]' RETURNING BIGINT) = 2" - + "GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("v0", "d0", ColumnType.LONG) + testBuilder() + .sql( + "SELECT " + + "JSON_VALUE(arrayLong, '$[1]' RETURNING BIGINT)," + + "SUM(cnt) " + + "FROM druid.arrays " + + "WHERE JSON_VALUE(arrayLong, '$[1]' RETURNING BIGINT) = 2" + + "GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG) + ) ) - ) - .setVirtualColumns( - new NestedFieldVirtualColumn("arrayLong", "$[1]", "v0", ColumnType.LONG) - ) - .setDimFilter(new SelectorDimFilter("v0", "2", null)) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayLong", "$[1]", "v0", ColumnType.LONG) + ) + .setDimFilter(new SelectorDimFilter("v0", "2", null)) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{2L, 6L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("EXPR$0", ColumnType.LONG) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{2L, 6L} - ), - RowSignature.builder() - .add("EXPR$0", ColumnType.LONG) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayLongElementDefault() { cannotVectorize(); - testQuery( - "SELECT " - + "JSON_VALUE(arrayLong, '$[1]')," - + "SUM(cnt) " - + "FROM druid.arrays GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("v0", "d0", ColumnType.STRING) + testBuilder() + .sql( + "SELECT " + + "JSON_VALUE(arrayLong, '$[1]')," + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.STRING) + ) ) - ) - .setVirtualColumns( - new NestedFieldVirtualColumn("arrayLong", "$[1]", "v0", ColumnType.STRING) - ) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayLong", "$[1]", "v0", ColumnType.STRING) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{NullHandling.defaultStringValue(), 4L}, + new Object[]{"2", 6L}, + new Object[]{"3", 2L}, + new Object[]{"4", 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{NullHandling.defaultStringValue(), 4L}, - new Object[]{"2", 6L}, - new Object[]{"3", 2L}, - new Object[]{"4", 2L} - ), - RowSignature.builder() - .add("EXPR$0", ColumnType.STRING) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayStringElement() { cannotVectorize(); - testQuery( - "SELECT " - + "JSON_VALUE(arrayStringNulls, '$[1]')," - + "SUM(cnt) " - + "FROM druid.arrays GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("v0", "d0", ColumnType.STRING) + testBuilder() + .sql( + "SELECT " + + "JSON_VALUE(arrayStringNulls, '$[1]')," + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.STRING) + ) ) - ) - .setVirtualColumns( - new NestedFieldVirtualColumn("arrayStringNulls", "$[1]", "v0", ColumnType.STRING) - ) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayStringNulls", "$[1]", "v0", ColumnType.STRING) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{NullHandling.defaultStringValue(), 7L}, + new Object[]{"b", 7L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{NullHandling.defaultStringValue(), 6L}, - new Object[]{"b", 8L} - ), - RowSignature.builder() - .add("EXPR$0", ColumnType.STRING) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayStringElementFiltered() { cannotVectorize(); - testQuery( - "SELECT " - + "JSON_VALUE(arrayStringNulls, '$[1]')," - + "SUM(cnt) " - + "FROM druid.arrays " - + "WHERE JSON_VALUE(arrayStringNulls, '$[1]') = 'b'" - + "GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("v0", "d0", ColumnType.STRING) + testBuilder() + .sql( + "SELECT " + + "JSON_VALUE(arrayStringNulls, '$[1]')," + + "SUM(cnt) " + + "FROM druid.arrays " + + "WHERE JSON_VALUE(arrayStringNulls, '$[1]') = 'b'" + + "GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.STRING) + ) ) - ) - .setVirtualColumns( - new NestedFieldVirtualColumn("arrayStringNulls", "$[1]", "v0", ColumnType.STRING) - ) - .setDimFilter(new SelectorDimFilter("v0", "b", null)) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayStringNulls", "$[1]", "v0", ColumnType.STRING) + ) + .setDimFilter(new SelectorDimFilter("v0", "b", null)) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{"b", 7L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{"b", 8L} - ), - RowSignature.builder() - .add("EXPR$0", ColumnType.STRING) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayDoubleElement() { cannotVectorize(); - testQuery( - "SELECT " - + "JSON_VALUE(arrayDoubleNulls, '$[2]' RETURNING DOUBLE)," - + "SUM(cnt) " - + "FROM druid.arrays GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("v0", "d0", ColumnType.DOUBLE) + testBuilder() + .sql( + "SELECT " + + "JSON_VALUE(arrayDoubleNulls, '$[2]' RETURNING DOUBLE)," + + "SUM(cnt) " + + "FROM druid.arrays GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.DOUBLE) + ) ) - ) - .setVirtualColumns( - new NestedFieldVirtualColumn("arrayDoubleNulls", "$[2]", "v0", ColumnType.DOUBLE) - ) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayDoubleNulls", "$[2]", "v0", ColumnType.DOUBLE) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{NullHandling.defaultDoubleValue(), 12L}, + new Object[]{5.5, 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("EXPR$0", ColumnType.DOUBLE) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{NullHandling.defaultDoubleValue(), 12L}, - new Object[]{5.5, 2L} - ), - RowSignature.builder() - .add("EXPR$0", ColumnType.DOUBLE) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); } @Test public void testGroupByRootSingleTypeArrayDoubleElementFiltered() { cannotVectorize(); - testQuery( - "SELECT " - + "JSON_VALUE(arrayDoubleNulls, '$[2]' RETURNING DOUBLE)," - + "SUM(cnt) " - + "FROM druid.arrays " - + "WHERE JSON_VALUE(arrayDoubleNulls, '$[2]' RETURNING DOUBLE) = 5.5" - + "GROUP BY 1", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(DATA_SOURCE_ARRAYS) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new DefaultDimensionSpec("v0", "d0", ColumnType.DOUBLE) + testBuilder() + .sql( + "SELECT " + + "JSON_VALUE(arrayDoubleNulls, '$[2]' RETURNING DOUBLE)," + + "SUM(cnt) " + + "FROM druid.arrays " + + "WHERE JSON_VALUE(arrayDoubleNulls, '$[2]' RETURNING DOUBLE) = 5.5" + + "GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.DOUBLE) + ) ) - ) - .setDimFilter(new SelectorDimFilter("v0", "5.5", null)) - .setVirtualColumns( - new NestedFieldVirtualColumn("arrayDoubleNulls", "$[2]", "v0", ColumnType.DOUBLE) - ) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setDimFilter(new SelectorDimFilter("v0", "5.5", null)) + .setVirtualColumns( + new NestedFieldVirtualColumn("arrayDoubleNulls", "$[2]", "v0", ColumnType.DOUBLE) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{5.5, 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("EXPR$0", ColumnType.DOUBLE) + .add("EXPR$1", ColumnType.LONG) .build() - ), - ImmutableList.of( - new Object[]{5.5, 2L} - ), - RowSignature.builder() - .add("EXPR$0", ColumnType.DOUBLE) - .add("EXPR$1", ColumnType.LONG) - .build() - ); + ) + .run(); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java index 6ee01f2d25aa..16c1e62462fa 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java @@ -38,6 +38,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.query.DataSource; import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.InlineDataSource; @@ -108,6 +109,14 @@ public Optional build( } }; + public static final JsonInputFormat DEFAULT_JSON_INPUT_FORMAT = new JsonInputFormat( + JSONPathSpec.DEFAULT, + null, + null, + null, + null + ); + private static final InputRowSchema FOO_SCHEMA = new InputRowSchema( new TimestampSpec(TIMESTAMP_COLUMN, "iso", null), new DimensionsSpec( @@ -623,7 +632,7 @@ public static QueryableIndex makeWikipediaIndex(File tmpDir) "calcite/tests/wikiticker-2015-09-12-sampled.json.gz" ) ) - .inputFormat(JsonInputFormat.DEFAULT) + .inputFormat(DEFAULT_JSON_INPUT_FORMAT) .inputTmpDir(new File(tmpDir, "tmpWikipedia1")) .buildMMappedIndex(); } From 82f01aab0dd178b8ff0aca62e01e502e3b7de202 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 23 Mar 2023 07:04:44 -0700 Subject: [PATCH 08/15] reuse field index for stuff --- .../CompressedNestedDataComplexColumn.java | 88 +++++++++++-------- 1 file changed, 49 insertions(+), 39 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java index bbb6dfca3b75..1d4a68338486 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java @@ -105,7 +105,7 @@ public abstract class CompressedNestedDataComplexColumn columns = new ConcurrentHashMap<>(); + private final ConcurrentHashMap columns = new ConcurrentHashMap<>(); private static final ObjectStrategy STRATEGY = NestedDataComplexTypeSerde.INSTANCE.getObjectStrategy(); @@ -398,20 +398,21 @@ public DimensionSelector makeDimensionSelector( { final String field = getField(path); Preconditions.checkNotNull(field, "Null field"); - - if (fields.indexOf(field) >= 0) { - DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(field).getColumn(); + final int fieldIndex = fields.indexOf(field); + if (fieldIndex >= 0) { + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(field, fieldIndex).getColumn(); return col.makeDimensionSelector(readableOffset, fn); } if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { final NestedPathPart lastPath = path.get(path.size() - 1); final String arrayField = getField(path.subList(0, path.size() - 1)); - if (fields.indexOf(arrayField) >= 0) { + final int arrayFieldIndex = fields.indexOf(arrayField); + if (arrayFieldIndex >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); if (elementNumber < 0) { throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); } - DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField, arrayFieldIndex).getColumn(); ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset); return new BaseSingleValueDimensionSelector() { @@ -450,19 +451,21 @@ public ColumnValueSelector makeColumnValueSelector(List path, final String field = getField(path); Preconditions.checkNotNull(field, "Null field"); - if (fields.indexOf(field) >= 0) { - BaseColumn col = getColumnHolder(field).getColumn(); + final int fieldIndex = fields.indexOf(field); + if (fieldIndex >= 0) { + BaseColumn col = getColumnHolder(field, fieldIndex).getColumn(); return col.makeColumnValueSelector(readableOffset); } if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { final NestedPathPart lastPath = path.get(path.size() - 1); final String arrayField = getField(path.subList(0, path.size() - 1)); - if (fields.indexOf(arrayField) >= 0) { + final int arrayFieldIndex = fields.indexOf(arrayField); + if (arrayFieldIndex >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); if (elementNumber < 0) { throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); } - DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField, arrayFieldIndex).getColumn(); ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset); return new ColumnValueSelector() { @@ -533,9 +536,9 @@ public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector { final String field = getField(path); Preconditions.checkNotNull(field, "Null field"); - - if (fields.indexOf(field) >= 0) { - DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(field).getColumn(); + final int fieldIndex = fields.indexOf(field); + if (fieldIndex >= 0) { + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(field, fieldIndex).getColumn(); return col.makeSingleValueDimensionVectorSelector(readableOffset); } else { return NilVectorSelector.create(readableOffset); @@ -547,20 +550,21 @@ public VectorObjectSelector makeVectorObjectSelector(List path, { final String field = getField(path); Preconditions.checkNotNull(field, "Null field"); - - if (fields.indexOf(field) >= 0) { - BaseColumn col = getColumnHolder(field).getColumn(); + final int fieldIndex = fields.indexOf(field); + if (fieldIndex >= 0) { + BaseColumn col = getColumnHolder(field, fieldIndex).getColumn(); return col.makeVectorObjectSelector(readableOffset); } if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { final NestedPathPart lastPath = path.get(path.size() - 1); final String arrayField = getField(path.subList(0, path.size() - 1)); - if (fields.indexOf(arrayField) >= 0) { + final int arrayFieldIndex = fields.indexOf(arrayField); + if (arrayFieldIndex >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); if (elementNumber < 0) { throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); } - DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField, arrayFieldIndex).getColumn(); VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset); return new VectorObjectSelector() @@ -613,20 +617,21 @@ public VectorValueSelector makeVectorValueSelector(List path, Re { final String field = getField(path); Preconditions.checkNotNull(field, "Null field"); - - if (fields.indexOf(field) >= 0) { - BaseColumn col = getColumnHolder(field).getColumn(); + final int fieldIndex = fields.indexOf(field); + if (fieldIndex >= 0) { + BaseColumn col = getColumnHolder(field, fieldIndex).getColumn(); return col.makeVectorValueSelector(readableOffset); } if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { final NestedPathPart lastPath = path.get(path.size() - 1); final String arrayField = getField(path.subList(0, path.size() - 1)); - if (fields.indexOf(arrayField) >= 0) { + final int arrayFieldIndex = fields.indexOf(arrayField); + if (arrayFieldIndex >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); if (elementNumber < 0) { throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); } - DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField).getColumn(); + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField, arrayFieldIndex).getColumn(); VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset); return new VectorValueSelector() @@ -756,7 +761,9 @@ public Set getColumnTypes(List path) @Override public ColumnHolder getColumnHolder(List path) { - return getColumnHolder(getField(path)); + final String field = getField(path); + final int fieldIndex = fields.indexOf(field); + return getColumnHolder(field, fieldIndex); } @Nullable @@ -764,40 +771,43 @@ public ColumnHolder getColumnHolder(List path) public ColumnIndexSupplier getColumnIndexSupplier(List path) { final String field = getField(path); - if (fields.indexOf(field) < 0) { - if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { - final String arrayField = getField(path.subList(0, path.size() - 1)); - if (fields.indexOf(arrayField) >= 0) { - return NoIndexesColumnIndexSupplier.getInstance(); - } + int fieldIndex = fields.indexOf(field); + if (fieldIndex >= 0) { + return getColumnHolder(field, fieldIndex).getIndexSupplier(); + } + if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { + final String arrayField = getField(path.subList(0, path.size() - 1)); + final int arrayFieldIndex = fields.indexOf(arrayField); + if (arrayFieldIndex >= 0) { + return NoIndexesColumnIndexSupplier.getInstance(); } - return null; } - return getColumnHolder(field).getIndexSupplier(); + return null; } @Override public boolean isNumeric(List path) { final String field = getField(path); - if (fields.indexOf(field) < 0) { + final int fieldIndex = fields.indexOf(field); + if (fieldIndex < 0) { return true; } - return getColumnHolder(field).getCapabilities().isNumeric(); + return getColumnHolder(field, fieldIndex).getCapabilities().isNumeric(); } - private ColumnHolder getColumnHolder(String field) + private ColumnHolder getColumnHolder(String field, int fieldIndex) { - return columns.computeIfAbsent(field, this::readNestedFieldColumn); + return columns.computeIfAbsent(fieldIndex, (f) -> readNestedFieldColumn(field, fieldIndex)); } - private ColumnHolder readNestedFieldColumn(String field) + @Nullable + private ColumnHolder readNestedFieldColumn(String field, int fieldIndex) { try { - if (fields.indexOf(field) < 0) { + if (fieldIndex < 0) { return null; } - final int fieldIndex = fields.indexOf(field); final NestedFieldTypeInfo.TypeSet types = fieldInfo.getTypes(fieldIndex); final String fieldFileName = getFieldFileName(metadata.getFileNameBase(), field, fieldIndex); final ByteBuffer dataBuffer = fileMapper.mapFile(fieldFileName); From 810649c2387ab000c005c5a0c6c4f09055b050a6 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 23 Mar 2023 13:17:01 -0700 Subject: [PATCH 09/15] fix tests --- .../query/scan/NestedDataScanQueryTest.java | 5 +--- .../sql/calcite/CalciteArraysQueryTest.java | 24 +++++++++++++++++++ 2 files changed, 25 insertions(+), 4 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java index 726210ad620c..8593f746cc1a 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java @@ -75,10 +75,7 @@ public NestedDataScanQueryTest() { NestedDataModule.registerHandlersAndSerde(); List mods = NestedDataModule.getJacksonModulesList(); - this.helper = AggregationTestHelper.createScanQueryAggregationTestHelper( - mods, - tempFolder - ); + this.helper = AggregationTestHelper.createScanQueryAggregationTestHelper(mods, tempFolder); this.closer = Closer.create(); } 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 2cd15582d318..96dc7fd18125 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 @@ -2829,6 +2829,7 @@ public void testUnnestTwice() .columns(ImmutableList.of("_j0.unnest", "dim1", "j0.unnest", "v0", "v1")) .build() ), + NullHandling.replaceWithDefault() ? ImmutableList.of( new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "10", "b"}, new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "10", "c"}, @@ -2838,6 +2839,18 @@ public void testUnnestTwice() new Object[]{"1", useDefault ? null : ImmutableList.of(""), ImmutableList.of("1"), "1", ""}, new Object[]{"def", null, ImmutableList.of("def"), "def", NullHandling.defaultStringValue()}, new Object[]{"abc", null, ImmutableList.of("abc"), "abc", NullHandling.defaultStringValue()} + ) : + ImmutableList.of( + new Object[]{"", ImmutableList.of("a", "b"), ImmutableList.of(""), "", "a"}, + new Object[]{"", ImmutableList.of("a", "b"), ImmutableList.of(""), "", "b"}, + new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "10", "b"}, + new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "10", "c"}, + new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "1", "b"}, + new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "1", "c"}, + new Object[]{"2", ImmutableList.of("d"), ImmutableList.of("2"), "2", "d"}, + new Object[]{"1", ImmutableList.of(""), ImmutableList.of("1"), "1", ""}, + new Object[]{"def", null, ImmutableList.of("def"), "def", null}, + new Object[]{"abc", null, ImmutableList.of("abc"), "abc", null} ) ); } @@ -3213,9 +3226,15 @@ public void testUnnestWithFilters() .columns(ImmutableList.of("j0.unnest")) .build() ), + NullHandling.replaceWithDefault() ? ImmutableList.of( new Object[]{"a"}, new Object[]{"b"} + ) : + ImmutableList.of( + new Object[]{"a"}, + new Object[]{"b"}, + new Object[]{""} ) ); } @@ -3262,8 +3281,13 @@ public void testUnnestWithFiltersInsideAndOutside() .columns(ImmutableList.of("j0.unnest")) .build() ), + NullHandling.replaceWithDefault() ? ImmutableList.of( new Object[]{"a"} + ) : + ImmutableList.of( + new Object[]{"a"}, + new Object[]{""} ) ); } From 76da0fbcf17d54ceca3480674951edfa72e4107d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 23 Mar 2023 15:02:47 -0700 Subject: [PATCH 10/15] adjust --- .../druid/query/metadata/SegmentAnalyzer.java | 21 ++++++------------- .../druid/segment/NestedDataColumnMerger.java | 18 ++++++++-------- 2 files changed, 15 insertions(+), 24 deletions(-) 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 ddd0f8f2b4d2..58edcb99d06e 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 @@ -136,7 +136,7 @@ public Map analyze(Segment segment) break; case ARRAY: final ColumnHolder arrayHolder = index != null ? index.getColumnHolder(columnName) : null; - analysis = analyzeArrayColumn(capabilities, numRows, arrayHolder); + analysis = analyzeArrayColumn(capabilities); break; case COMPLEX: final ColumnHolder columnHolder = index != null ? index.getColumnHolder(columnName) : null; @@ -390,22 +390,13 @@ private ColumnAnalysis analyzeComplexColumn( } } - private ColumnAnalysis analyzeArrayColumn( - @Nullable final ColumnCapabilities capabilities, - final int numCells, - @Nullable final ColumnHolder columnHolder - ) + private ColumnAnalysis analyzeArrayColumn(final ColumnCapabilities capabilities) { - final TypeSignature typeSignature = capabilities == null ? ColumnType.UNKNOWN_COMPLEX : capabilities; - final String typeName = typeSignature.getComplexTypeName(); - final boolean hasMultipleValues = capabilities != null && capabilities.hasMultipleValues().isTrue(); - final boolean hasNulls = capabilities != null && capabilities.hasNulls().isMaybeTrue(); - return new ColumnAnalysis( - ColumnTypeFactory.ofType(typeSignature), - typeName, - hasMultipleValues, - hasNulls, + capabilities.toColumnType(), + capabilities.getType().name(), + false, + capabilities.hasNulls().isTrue(), 0L, null, null, diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java index 436514a6bbe1..df0fe0506d1c 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java @@ -423,27 +423,27 @@ public int[] next() if (next == null) { return null; } - final int[] newIdsWhoDis = new int[next.length]; + final int[] globalIds = new int[next.length]; for (int i = 0; i < next.length; i++) { if (next[i] == null) { - newIdsWhoDis[i] = 0; + globalIds[i] = 0; } else if (next[i] instanceof String) { - newIdsWhoDis[i] = idLookup.lookupString((String) next[i]); + globalIds[i] = idLookup.lookupString((String) next[i]); } else if (next[i] instanceof Long) { - newIdsWhoDis[i] = idLookup.lookupLong((Long) next[i]); + globalIds[i] = idLookup.lookupLong((Long) next[i]); } else if (next[i] instanceof Double) { - newIdsWhoDis[i] = idLookup.lookupDouble((Double) next[i]); + globalIds[i] = idLookup.lookupDouble((Double) next[i]); } else { - newIdsWhoDis[i] = -1; + globalIds[i] = -1; } Preconditions.checkArgument( - newIdsWhoDis[i] >= 0, + globalIds[i] >= 0, "unknown global id [%s] for value [%s]", - newIdsWhoDis[i], + globalIds[i], next[i] ); } - return newIdsWhoDis; + return globalIds; } } } From 313f48681b6acfb06c59930fd6bc60f3239abded Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 23 Mar 2023 15:03:08 -0700 Subject: [PATCH 11/15] adjust again --- .../nested/GlobalDimensionDictionary.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java index 31690b32f2db..48b06d4e45c7 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java @@ -286,28 +286,28 @@ private int[] convertArray(Object[] array) if (array == null) { return null; } - final int[] newIdsWhoDis = new int[array.length]; + final int[] globalIds = new int[array.length]; for (int i = 0; i < array.length; i++) { if (array[i] == null) { - newIdsWhoDis[i] = 0; + globalIds[i] = 0; } else if (array[i] instanceof String) { // offset by 1 because nulls are ignored by the indexer, but always global id 0 - newIdsWhoDis[i] = 1 + strings.indexOf((String) array[i]); + globalIds[i] = 1 + strings.indexOf((String) array[i]); } else if (array[i] instanceof Long) { - newIdsWhoDis[i] = longs.indexOf((Long) array[i]) + adjustLongs; + globalIds[i] = longs.indexOf((Long) array[i]) + adjustLongs; } else if (array[i] instanceof Double) { - newIdsWhoDis[i] = doubles.indexOf((Double) array[i]) + adjustDoubles; + globalIds[i] = doubles.indexOf((Double) array[i]) + adjustDoubles; } else { - newIdsWhoDis[i] = -1; + globalIds[i] = -1; } Preconditions.checkArgument( - newIdsWhoDis[i] >= 0, + globalIds[i] >= 0, "unknown global id [%s] for value [%s]", - newIdsWhoDis[i], + globalIds[i], array[i] ); } - return newIdsWhoDis; + return globalIds; } }); sortedArrays.addAll(stringArrays); From 665e0bcca0f3a8bb14dc1ac364b52577a2b38b33 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 23 Mar 2023 23:25:18 -0700 Subject: [PATCH 12/15] nested array tests --- .../druid/data/input/orc/OrcReaderTest.java | 5 - .../protobuf/ProtobufInputFormatTest.java | 2 +- .../apache/druid/math/expr/ConstantExpr.java | 1 - .../org/apache/druid/math/expr/ExprEval.java | 1 - .../druid/math/expr/ExpressionProcessing.java | 20 +- .../math/expr/ExpressionProcessingConfig.java | 11 - .../druid/math/expr/ExpressionType.java | 8 - .../org/apache/druid/math/expr/Function.java | 1 - .../druid/query/metadata/SegmentAnalyzer.java | 1 - .../org/apache/druid/math/expr/EvalTest.java | 194 +++++++------- .../druid/math/expr/OutputTypeTest.java | 8 +- .../apache/druid/math/expr/ParserTest.java | 43 ++-- .../druid/math/expr/VectorExprSanityTest.java | 2 +- .../druid/query/MultiValuedDimensionTest.java | 2 +- .../VectorExpressionsSanityTest.java | 2 +- .../UnnestColumnValueSelectorCursorTest.java | 37 ++- .../segment/filter/ExpressionFilterTest.java | 2 +- .../segment/transform/TransformSpecTest.java | 4 +- .../testing/InitializedNullHandlingTest.java | 2 +- .../resources/nested-array-test-data.json | 28 +-- .../sql/calcite/CalciteArraysQueryTest.java | 236 ++++++++---------- .../CalciteMultiValueStringQueryTest.java | 2 +- .../calcite/CalciteNestedDataQueryTest.java | 38 +-- .../SqlVectorizedExpressionSanityTest.java | 2 +- .../sql/calcite/util/CalciteTestBase.java | 2 +- 25 files changed, 304 insertions(+), 350 deletions(-) diff --git a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java index 51358b06fe80..6a395af8e993 100644 --- a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java +++ b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java @@ -36,7 +36,6 @@ import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; import org.apache.druid.java.util.common.parsers.JSONPathFieldType; import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.math.expr.ExpressionProcessing; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.segment.NestedDataDimensionSchema; import org.apache.druid.segment.transform.ExpressionTransform; @@ -592,7 +591,6 @@ public void testListMap() throws IOException @Test public void testNestedArray() throws IOException { - ExpressionProcessing.initializeForTests(true); final InputFormat inputFormat = new OrcInputFormat( new JSONPathSpec( true, @@ -668,9 +666,6 @@ public void testNestedArray() throws IOException Assert.assertArrayEquals(new Object[]{1L, 2L}, (Object[]) row.getRaw("t_d_0")); Assert.assertFalse(iterator.hasNext()); } - finally { - ExpressionProcessing.initializeForTests(null); - } } @Test diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java index 510505d17463..86b5ae4edc48 100644 --- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java +++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java @@ -75,7 +75,7 @@ public class ProtobufInputFormatTest public void setUp() throws Exception { NullHandling.initializeForTests(); - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); timestampSpec = new TimestampSpec("timestamp", "iso", null); dimensionsSpec = new DimensionsSpec(Lists.newArrayList( new StringDimensionSchema("event"), diff --git a/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java b/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java index ff4cedbecbb0..fdf6f080ee9c 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java @@ -379,7 +379,6 @@ public ArrayExpr(ExpressionType outputType, @Nullable Object[] value) { super(outputType, value); Preconditions.checkArgument(outputType.isArray(), "Output type %s is not an array", outputType); - ExpressionType.checkNestedArrayAllowed(outputType); } @Override diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java index 9c105af58ffa..1b337bbcbce9 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java @@ -1114,7 +1114,6 @@ private ArrayExprEval(ExpressionType arrayType, @Nullable Object[] value) super(value); this.arrayType = arrayType; Preconditions.checkArgument(arrayType.isArray(), "Output type %s is not an array", arrayType); - ExpressionType.checkNestedArrayAllowed(arrayType); } @Override diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java index 905e0850f712..4e9e81b096d7 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java @@ -22,8 +22,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; -import javax.annotation.Nullable; - /** * Like {@link org.apache.druid.common.config.NullHandling}, except for expressions processing configs */ @@ -43,33 +41,23 @@ public class ExpressionProcessing /** * Many unit tests do not setup modules for this value to be injected, this method provides a manual way to initialize * {@link #INSTANCE} - * @param allowNestedArrays */ @VisibleForTesting - public static void initializeForTests(@Nullable Boolean allowNestedArrays) + public static void initializeForTests() { - INSTANCE = new ExpressionProcessingConfig(allowNestedArrays, null, null, null); + INSTANCE = new ExpressionProcessingConfig( null, null, null); } @VisibleForTesting public static void initializeForStrictBooleansTests(boolean useStrict) { - INSTANCE = new ExpressionProcessingConfig(null, useStrict, null, null); + INSTANCE = new ExpressionProcessingConfig(useStrict, null, null); } @VisibleForTesting public static void initializeForHomogenizeNullMultiValueStrings() { - INSTANCE = new ExpressionProcessingConfig(null, null, null, true); - } - - /** - * [['is expression support for'],['nested arrays'],['enabled?']] - */ - public static boolean allowNestedArrays() - { - checkInitialized(); - return INSTANCE.allowNestedArrays(); + INSTANCE = new ExpressionProcessingConfig( null, null, true); } /** diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java index b832578fe319..44e3b9409d6b 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java @@ -26,7 +26,6 @@ public class ExpressionProcessingConfig { - public static final String NESTED_ARRAYS_CONFIG_STRING = "druid.expressions.allowNestedArrays"; public static final String NULL_HANDLING_LEGACY_LOGICAL_OPS_STRING = "druid.expressions.useStrictBooleans"; // Coerce arrays to multi value strings public static final String PROCESS_ARRAYS_AS_MULTIVALUE_STRINGS_CONFIG_STRING = @@ -35,9 +34,6 @@ public class ExpressionProcessingConfig public static final String HOMOGENIZE_NULL_MULTIVALUE_STRING_ARRAYS = "druid.expressions.homogenizeNullMultiValueStringArrays"; - @JsonProperty("allowNestedArrays") - private final boolean allowNestedArrays; - @JsonProperty("useStrictBooleans") private final boolean useStrictBooleans; @@ -49,13 +45,11 @@ public class ExpressionProcessingConfig @JsonCreator public ExpressionProcessingConfig( - @JsonProperty("allowNestedArrays") @Nullable Boolean allowNestedArrays, @JsonProperty("useStrictBooleans") @Nullable Boolean useStrictBooleans, @JsonProperty("processArraysAsMultiValueStrings") @Nullable Boolean processArraysAsMultiValueStrings, @JsonProperty("homogenizeNullMultiValueStringArrays") @Nullable Boolean homogenizeNullMultiValueStringArrays ) { - this.allowNestedArrays = getWithPropertyFallbackFalse(allowNestedArrays, NESTED_ARRAYS_CONFIG_STRING); this.useStrictBooleans = getWithPropertyFallbackFalse(useStrictBooleans, NULL_HANDLING_LEGACY_LOGICAL_OPS_STRING); this.processArraysAsMultiValueStrings = getWithPropertyFallbackFalse( processArraysAsMultiValueStrings, @@ -67,11 +61,6 @@ public ExpressionProcessingConfig( ); } - public boolean allowNestedArrays() - { - return allowNestedArrays; - } - public boolean isUseStrictBooleans() { return useStrictBooleans; diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionType.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionType.java index 6f6b7c302509..f2f17f2ef732 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionType.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionType.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.column.BaseTypeSignature; import org.apache.druid.segment.column.ColumnType; @@ -210,11 +209,4 @@ public static ColumnType toColumnType(ExpressionType exprType) throw new ISE("Unsupported expression type[%s]", exprType); } } - - public static void checkNestedArrayAllowed(ExpressionType outputType) - { - if (outputType.isArray() && outputType.getElementType().isArray() && !ExpressionProcessing.allowNestedArrays()) { - throw new IAE("Cannot create a nested array type [%s], 'druid.expressions.allowNestedArrays' must be set to true", outputType); - } - } } diff --git a/processing/src/main/java/org/apache/druid/math/expr/Function.java b/processing/src/main/java/org/apache/druid/math/expr/Function.java index 70cd0f8f278a..03eb95820fa9 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/Function.java +++ b/processing/src/main/java/org/apache/druid/math/expr/Function.java @@ -3054,7 +3054,6 @@ static ExpressionType setArrayOutput(@Nullable ExpressionType arrayType, Object[ if (arrayType == null) { arrayType = ExpressionTypeFactory.getInstance().ofArray(evaluated.type()); } - ExpressionType.checkNestedArrayAllowed(arrayType); if (arrayType.getElementType().isNumeric() && evaluated.isNumericNull()) { out[i] = null; } else if (!evaluated.asArrayType().equals(arrayType)) { 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 58edcb99d06e..7704a8e4c85a 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 @@ -135,7 +135,6 @@ public Map analyze(Segment segment) } break; case ARRAY: - final ColumnHolder arrayHolder = index != null ? index.getColumnHolder(columnName) : null; analysis = analyzeArrayColumn(capabilities); break; case COMPLEX: diff --git a/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java b/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java index 8ec892947829..a2ddb78dc3a3 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java @@ -102,7 +102,7 @@ public void testDoubleEval() Assert.assertFalse(evalDouble("!2.0", bindings) > 0.0); } finally { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } try { ExpressionProcessing.initializeForStrictBooleansTests(true); @@ -137,7 +137,7 @@ public void testDoubleEval() assertEquals(3.0, evalDouble("if(0.0, 2.0, 3.0)", bindings), 0.0001); } finally { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } } @@ -435,7 +435,7 @@ public void testBooleanReturn() assertEquals(ExpressionType.DOUBLE, eval.type()); } finally { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } try { ExpressionProcessing.initializeForStrictBooleansTests(true); @@ -464,7 +464,7 @@ public void testBooleanReturn() assertEquals(ExpressionType.LONG, eval.type()); } finally { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } } @@ -532,7 +532,7 @@ public void testLogicalOperators() } finally { // reset - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } try { @@ -607,7 +607,7 @@ public void testLogicalOperators() } finally { // reset - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } } @@ -664,7 +664,7 @@ public void testBooleanInputs() } finally { // reset - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } try { @@ -703,7 +703,7 @@ public void testBooleanInputs() } finally { // reset - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } } @@ -948,103 +948,95 @@ public void testEvalOfType() Assert.assertArrayEquals(new Object[] {"1.0", "2", "3", "true", "false"}, (Object[]) eval.value()); // nested arrays - try { - ExpressionProcessing.initializeForTests(true); - - ExpressionType nestedLongArray = ExpressionTypeFactory.getInstance().ofArray(ExpressionType.LONG_ARRAY); - final Object[] expectedLongArray = new Object[]{ - new Object[] {1L, 2L, 3L}, - new Object[] {5L, null, 9L}, - null, - new Object[] {2L, 4L, 6L} - }; - - List longArrayInputs = Arrays.asList( - new Object[]{ - new Object[] {1L, 2L, 3L}, - new Object[] {5L, null, 9L}, - null, - new Object[] {2L, 4L, 6L} - }, - Arrays.asList( - new Object[] {1L, 2L, 3L}, - new Object[] {5L, null, 9L}, - null, - new Object[] {2L, 4L, 6L} - ), - Arrays.asList( - Arrays.asList(1L, 2L, 3L), - Arrays.asList(5L, null, 9L), - null, - Arrays.asList(2L, 4L, 6L) - ), - Arrays.asList( - Arrays.asList(1L, 2L, 3L), - Arrays.asList("5", "hello", "9"), - null, - new Object[]{2.2, 4.4, 6.6} - ) - ); - - for (Object o : longArrayInputs) { - eval = ExprEval.ofType(nestedLongArray, o); - Assert.assertEquals(nestedLongArray, eval.type()); - Object[] val = (Object[]) eval.value(); - Assert.assertEquals(expectedLongArray.length, val.length); - for (int i = 0; i < expectedLongArray.length; i++) { - Assert.assertArrayEquals((Object[]) expectedLongArray[i], (Object[]) val[i]); - } + ExpressionType nestedLongArray = ExpressionTypeFactory.getInstance().ofArray(ExpressionType.LONG_ARRAY); + final Object[] expectedLongArray = new Object[]{ + new Object[] {1L, 2L, 3L}, + new Object[] {5L, null, 9L}, + null, + new Object[] {2L, 4L, 6L} + }; + + List longArrayInputs = Arrays.asList( + new Object[]{ + new Object[] {1L, 2L, 3L}, + new Object[] {5L, null, 9L}, + null, + new Object[] {2L, 4L, 6L} + }, + Arrays.asList( + new Object[] {1L, 2L, 3L}, + new Object[] {5L, null, 9L}, + null, + new Object[] {2L, 4L, 6L} + ), + Arrays.asList( + Arrays.asList(1L, 2L, 3L), + Arrays.asList(5L, null, 9L), + null, + Arrays.asList(2L, 4L, 6L) + ), + Arrays.asList( + Arrays.asList(1L, 2L, 3L), + Arrays.asList("5", "hello", "9"), + null, + new Object[]{2.2, 4.4, 6.6} + ) + ); + + for (Object o : longArrayInputs) { + eval = ExprEval.ofType(nestedLongArray, o); + Assert.assertEquals(nestedLongArray, eval.type()); + Object[] val = (Object[]) eval.value(); + Assert.assertEquals(expectedLongArray.length, val.length); + for (int i = 0; i < expectedLongArray.length; i++) { + Assert.assertArrayEquals((Object[]) expectedLongArray[i], (Object[]) val[i]); } + } - ExpressionType nestedDoubleArray = ExpressionTypeFactory.getInstance().ofArray(ExpressionType.DOUBLE_ARRAY); - final Object[] expectedDoubleArray = new Object[]{ - new Object[] {1.1, 2.2, 3.3}, - new Object[] {5.5, null, 9.9}, - null, - new Object[] {2.2, 4.4, 6.6} - }; - - List doubleArrayInputs = Arrays.asList( - new Object[]{ - new Object[] {1.1, 2.2, 3.3}, - new Object[] {5.5, null, 9.9}, - null, - new Object[] {2.2, 4.4, 6.6} - }, - new Object[]{ - Arrays.asList(1.1, 2.2, 3.3), - Arrays.asList(5.5, null, 9.9), - null, - Arrays.asList(2.2, 4.4, 6.6) - }, - Arrays.asList( - Arrays.asList(1.1, 2.2, 3.3), - Arrays.asList(5.5, null, 9.9), - null, - Arrays.asList(2.2, 4.4, 6.6) - ), - new Object[]{ - new Object[] {"1.1", "2.2", "3.3"}, - Arrays.asList("5.5", null, "9.9"), - null, - new String[] {"2.2", "4.4", "6.6"} - } - ); - - for (Object o : doubleArrayInputs) { - eval = ExprEval.ofType(nestedDoubleArray, o); - Assert.assertEquals(nestedDoubleArray, eval.type()); - Object[] val = (Object[]) eval.value(); - Assert.assertEquals(expectedLongArray.length, val.length); - for (int i = 0; i < expectedLongArray.length; i++) { - Assert.assertArrayEquals((Object[]) expectedDoubleArray[i], (Object[]) val[i]); + ExpressionType nestedDoubleArray = ExpressionTypeFactory.getInstance().ofArray(ExpressionType.DOUBLE_ARRAY); + final Object[] expectedDoubleArray = new Object[]{ + new Object[] {1.1, 2.2, 3.3}, + new Object[] {5.5, null, 9.9}, + null, + new Object[] {2.2, 4.4, 6.6} + }; + + List doubleArrayInputs = Arrays.asList( + new Object[]{ + new Object[] {1.1, 2.2, 3.3}, + new Object[] {5.5, null, 9.9}, + null, + new Object[] {2.2, 4.4, 6.6} + }, + new Object[]{ + Arrays.asList(1.1, 2.2, 3.3), + Arrays.asList(5.5, null, 9.9), + null, + Arrays.asList(2.2, 4.4, 6.6) + }, + Arrays.asList( + Arrays.asList(1.1, 2.2, 3.3), + Arrays.asList(5.5, null, 9.9), + null, + Arrays.asList(2.2, 4.4, 6.6) + ), + new Object[]{ + new Object[] {"1.1", "2.2", "3.3"}, + Arrays.asList("5.5", null, "9.9"), + null, + new String[] {"2.2", "4.4", "6.6"} } + ); + + for (Object o : doubleArrayInputs) { + eval = ExprEval.ofType(nestedDoubleArray, o); + Assert.assertEquals(nestedDoubleArray, eval.type()); + Object[] val = (Object[]) eval.value(); + Assert.assertEquals(expectedLongArray.length, val.length); + for (int i = 0; i < expectedLongArray.length; i++) { + Assert.assertArrayEquals((Object[]) expectedDoubleArray[i], (Object[]) val[i]); } } - finally { - // reset - ExpressionProcessing.initializeForTests(null); - } } @Test @@ -1073,7 +1065,7 @@ public void testBestEffortOf() } finally { // reset - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } // doubles diff --git a/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java b/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java index e2f14dd712df..00682b9632d8 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java @@ -86,7 +86,7 @@ public void testUnaryOperators() } finally { // reset - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } try { @@ -96,7 +96,7 @@ public void testUnaryOperators() } finally { // reset - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } } @@ -161,7 +161,7 @@ public void testBinaryMathOperators() assertOutputType("z_ || z", inspector, ExpressionType.LONG); } finally { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } try { ExpressionProcessing.initializeForStrictBooleansTests(false); @@ -184,7 +184,7 @@ public void testBinaryMathOperators() assertOutputType("z_ || z", inspector, ExpressionType.DOUBLE); } finally { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } assertOutputType("1*(2 + 3.0)", inspector, ExpressionType.DOUBLE); } diff --git a/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java b/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java index 87cc6f1b647f..2479b7486fc2 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java @@ -353,36 +353,28 @@ public void testLiteralArraysExplicitTypesMixedElements() @Test public void testLiteralExplicitTypedArrays() { - ExpressionProcessing.initializeForTests(true); - - try { - validateConstantExpression("ARRAY[1.0, 2.0, null, 3.0]", new Object[]{1.0, 2.0, null, 3.0}); - validateConstantExpression("ARRAY[1, 2, null, 3]", new Object[]{1L, 2L, null, 3L}); - validateConstantExpression("ARRAY['1', '2', null, '3.0']", new Object[]{"1", "2", null, "3.0"}); - - // mixed type tests - validateConstantExpression("ARRAY[3, null, 4, 2.345]", new Object[]{3.0, null, 4.0, 2.345}); - validateConstantExpression("ARRAY[1.0, null, 2000.0]", new Object[]{1L, null, 2000L}); - - // explicit typed string arrays should accept any literal and convert - validateConstantExpression("ARRAY['1', null, 2000, 1.1]", new Object[]{"1", null, "2000", "1.1"}); - validateConstantExpression("ARRAY['1', null, 2000, 1.1]", new Object[]{1L, null, 2000L, 1L}); - validateConstantExpression("ARRAY['1', null, 2000, 1.1]", new Object[]{1.0, null, 2000.0, 1.1}); - - // the gramar isn't cool enough yet to parse populated nested-arrays or complex arrays..., but empty ones can - // be defined... - validateConstantExpression("ARRAY>[]", new Object[]{}); - validateConstantExpression("ARRAY>[]", new Object[]{}); - } - finally { - ExpressionProcessing.initializeForTests(null); - } + validateConstantExpression("ARRAY[1.0, 2.0, null, 3.0]", new Object[]{1.0, 2.0, null, 3.0}); + validateConstantExpression("ARRAY[1, 2, null, 3]", new Object[]{1L, 2L, null, 3L}); + validateConstantExpression("ARRAY['1', '2', null, '3.0']", new Object[]{"1", "2", null, "3.0"}); + + // mixed type tests + validateConstantExpression("ARRAY[3, null, 4, 2.345]", new Object[]{3.0, null, 4.0, 2.345}); + validateConstantExpression("ARRAY[1.0, null, 2000.0]", new Object[]{1L, null, 2000L}); + + // explicit typed string arrays should accept any literal and convert + validateConstantExpression("ARRAY['1', null, 2000, 1.1]", new Object[]{"1", null, "2000", "1.1"}); + validateConstantExpression("ARRAY['1', null, 2000, 1.1]", new Object[]{1L, null, 2000L, 1L}); + validateConstantExpression("ARRAY['1', null, 2000, 1.1]", new Object[]{1.0, null, 2000.0, 1.1}); + + // the gramar isn't cool enough yet to parse populated nested-arrays or complex arrays..., but empty ones can + // be defined... + validateConstantExpression("ARRAY>[]", new Object[]{}); + validateConstantExpression("ARRAY>[]", new Object[]{}); } @Test public void testConstantComplexAndNestedArrays() { - ExpressionProcessing.initializeForTests(true); // they can be built with array builder functions though... validateConstantExpression( "array(['foo', 'bar', 'baz'], ['baz','foo','bar'])", @@ -426,7 +418,6 @@ public void testConstantComplexAndNestedArrays() StringUtils.format("array(%s,%s)", l1String, l2String), new Object[]{l1, l2} ); - ExpressionProcessing.initializeForTests(null); } @Test diff --git a/processing/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java b/processing/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java index 2e20a486fab9..a8724db8a01e 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java @@ -73,7 +73,7 @@ public static void setupTests() @AfterClass public static void teardownTests() { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } @Test 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 152e7c908ab3..dafa7ff9a82d 100644 --- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java @@ -550,7 +550,7 @@ public void testGroupByExpressionMultiMultiBackwardsCompat0dot22andOlder() TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-multi-multi"); } finally { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } } diff --git a/processing/src/test/java/org/apache/druid/query/expression/VectorExpressionsSanityTest.java b/processing/src/test/java/org/apache/druid/query/expression/VectorExpressionsSanityTest.java index 87ae80fa1016..71e977909f61 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/VectorExpressionsSanityTest.java +++ b/processing/src/test/java/org/apache/druid/query/expression/VectorExpressionsSanityTest.java @@ -71,7 +71,7 @@ public static void setupTests() @AfterClass public static void teardownTests() { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } static void testExpression(String expr, Expr parsed, Map types) diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java index 70b7d5ab548f..475f26833823 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java @@ -21,7 +21,6 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExpressionProcessing; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.monomorphicprocessing.StringRuntimeShape; import org.apache.druid.segment.column.ColumnType; @@ -45,13 +44,11 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling public static void setUpClass() { NullHandling.initializeForTests(); - ExpressionProcessing.initializeForTests(true); // Allow nested arrays } @AfterClass public static void tearDownClass() { - ExpressionProcessing.initializeForTests(null); // Clear special expression-processing config. } @Test @@ -151,6 +148,40 @@ public void test_list_unnest_cursors_user_supplied_list_only_nulls() Assert.assertEquals(k, 2); } + @Test + public void test_list_unnest_cursors_user_supplied_list_only_nulls_mv_to_array() + { + List inputList = Arrays.asList( + Collections.singletonList(null), + Arrays.asList(null, null), + Collections.singletonList(null), + Collections.emptyList() + ); + + //Create base cursor + ListCursor listCursor = new ListCursor(inputList); + + //Create unnest cursor + UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor( + listCursor, + listCursor.getColumnSelectorFactory(), + new ExpressionVirtualColumn("__unnest__", "mv_to_array(\"dummy\")", ColumnType.STRING, ExprMacroTable.nil()), + OUTPUT_NAME + ); + ColumnValueSelector unnestColumnValueSelector = unnestCursor.getColumnSelectorFactory() + .makeColumnValueSelector(OUTPUT_NAME); + int k = 0; + while (!unnestCursor.isDone()) { + Object valueSelectorVal = unnestColumnValueSelector.getObject(); + Assert.assertNull(valueSelectorVal); + k++; + unnestCursor.advance(); + } + // since type is 'STRING', it follows multi-value string rules so single element arrays become scalar values, + // so [null] becomes null, meaning we only have 2 rows + Assert.assertEquals(k, 2); + } + @Test public void test_list_unnest_cursors_user_supplied_list_only_nulls_array() { diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java index d0e089a7f784..289843f6006f 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java @@ -128,7 +128,7 @@ public void setup() @After public void teardown() { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } @AfterClass diff --git a/processing/src/test/java/org/apache/druid/segment/transform/TransformSpecTest.java b/processing/src/test/java/org/apache/druid/segment/transform/TransformSpecTest.java index b9244b3ae109..3ed0a51bf039 100644 --- a/processing/src/test/java/org/apache/druid/segment/transform/TransformSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/transform/TransformSpecTest.java @@ -237,7 +237,7 @@ public void testBoolTransforms() Assert.assertEquals(0L, row2.getRaw("truthy2")); } finally { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } try { ExpressionProcessing.initializeForStrictBooleansTests(false); @@ -268,7 +268,7 @@ public void testBoolTransforms() Assert.assertEquals(0L, row2.getRaw("truthy2")); } finally { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } } diff --git a/processing/src/test/java/org/apache/druid/testing/InitializedNullHandlingTest.java b/processing/src/test/java/org/apache/druid/testing/InitializedNullHandlingTest.java index 1ba482d38a5b..b0334757001a 100644 --- a/processing/src/test/java/org/apache/druid/testing/InitializedNullHandlingTest.java +++ b/processing/src/test/java/org/apache/druid/testing/InitializedNullHandlingTest.java @@ -26,6 +26,6 @@ public class InitializedNullHandlingTest { static { NullHandling.initializeForTests(); - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } } diff --git a/processing/src/test/resources/nested-array-test-data.json b/processing/src/test/resources/nested-array-test-data.json index 44e23b84b244..b8ae3ace3847 100644 --- a/processing/src/test/resources/nested-array-test-data.json +++ b/processing/src/test/resources/nested-array-test-data.json @@ -1,14 +1,14 @@ -{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"x": 1},{"x":2}]} -{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayObject":[{"x": 3},{"x":4}]} -{"timestamp": "2023-01-01T00:00:00", "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayObject":[null,{"x":2}]} -{"timestamp": "2023-01-01T00:00:00", "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayObject":[{"x": null},{"x":2}]} -{"timestamp": "2023-01-01T00:00:00", "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":[], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayObject":[{"x": 1000},{"y":2000}]} -{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"a": 1},{"b":2}]} -{"timestamp": "2023-01-01T00:00:00", "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayObject":[{"x": 1},{"x":2}]} -{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": [], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"x": 1},{"x":2}]} -{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayObject":[{"x": 3},{"x":4}]} -{"timestamp": "2023-01-02T00:00:00", "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayObject":[null,{"x":2}]} -{"timestamp": "2023-01-02T00:00:00", "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[null], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayObject":[{"x": null},{"x":2}]} -{"timestamp": "2023-01-02T00:00:00", "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":null, "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[], "arrayObject":[{"x": 1000},{"y":2000}]} -{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": [null], "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayObject":[{"a": 1},{"b":2}]} -{"timestamp": "2023-01-02T00:00:00", "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null, 1.1], "arrayVariant":null, "arrayObject":[{"x": 1},{"x":2}]} \ No newline at end of file +{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayNestedLong":[[1, 2, null], [3, 4]], "arrayObject":[{"x": 1},{"x":2}]} +{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayNestedLong":[null, [null], []], "arrayObject":[{"x": 3},{"x":4}]} +{"timestamp": "2023-01-01T00:00:00", "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayNestedLong":[[1], null, [1, 2, 3]], "arrayObject":[null,{"x":2}]} +{"timestamp": "2023-01-01T00:00:00", "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}]} +{"timestamp": "2023-01-01T00:00:00", "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":[], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayObject":[{"x": 1000},{"y":2000}]} +{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayNestedLong":[[1], [1, 2, null]], "arrayObject":[{"a": 1},{"b":2}]} +{"timestamp": "2023-01-01T00:00:00", "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayNestedLong":null, "arrayObject":[{"x": 1},{"x":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": [], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayNestedLong":[[2, 3], [1, 5]], "arrayObject":[{"x": 1},{"x":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayNestedLong":[null], "arrayObject":[{"x": 3},{"x":4}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayNestedLong":[[1], null, [1]], "arrayObject":[null,{"x":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[null], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":null, "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[], "arrayObject":[{"x": 1000},{"y":2000}]} +{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"], "arrayStringNulls": [null], "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayNestedLong":[], "arrayObject":[{"a": 1},{"b":2}]} +{"timestamp": "2023-01-02T00:00:00", "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null, 1.1], "arrayVariant":null, "arrayNestedLong":null, "arrayObject":[{"x": 1},{"x":2}]} \ No newline at end of file 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 96dc7fd18125..914a89ea96f2 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 @@ -28,7 +28,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExpressionProcessing; import org.apache.druid.query.Druids; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.Query; @@ -181,43 +180,26 @@ public void testSelectNonConstantArrayExpressionFromTableForMultival() .context(QUERY_CONTEXT_DEFAULT) .build(); + // dim3 is a multi-valued string column, so the automatic translation will turn this + // expression into + // + // `map((dim3) -> array(concat(dim3,'word'),'up'), dim3)` + // + // this works, but we still translate the output into a string since that is the current output type + // in some future this might not auto-convert to a string type (when we support grouping on arrays maybe?) - try { - ExpressionProcessing.initializeForTests(true); - // if nested arrays are allowed, dim3 is a multi-valued string column, so the automatic translation will turn this - // expression into - // - // `map((dim3) -> array(concat(dim3,'word'),'up'), dim3)` - // - // this works, but we still translate the output into a string since that is the current output type - // in some future this might not auto-convert to a string type (when we support grouping on arrays maybe?) - - testQuery( - sql, - ImmutableList.of(scanQuery), - ImmutableList.of( - new Object[]{"[[\"aword\",\"up\"],[\"bword\",\"up\"]]", ""}, - new Object[]{"[[\"bword\",\"up\"],[\"cword\",\"up\"]]", "10.1"}, - new Object[]{"[[\"dword\",\"up\"]]", "2"}, - new Object[]{"[[\"word\",\"up\"]]", "1"}, - useDefault ? new Object[]{"[[\"word\",\"up\"]]", "def"} : new Object[]{"[[null,\"up\"]]", "def"} - ) - ); - } - finally { - ExpressionProcessing.initializeForTests(null); - } - - // if nested arrays are not enabled, this doesn't work - expectedException.expect(IAE.class); - expectedException.expectMessage( - "Cannot create a nested array type [ARRAY>], 'druid.expressions.allowNestedArrays' must be set to true" - ); testQuery( sql, ImmutableList.of(scanQuery), - ImmutableList.of() + ImmutableList.of( + new Object[]{"[[\"aword\",\"up\"],[\"bword\",\"up\"]]", ""}, + new Object[]{"[[\"bword\",\"up\"],[\"cword\",\"up\"]]", "10.1"}, + new Object[]{"[[\"dword\",\"up\"]]", "2"}, + new Object[]{"[[\"word\",\"up\"]]", "1"}, + useDefault ? new Object[]{"[[\"word\",\"up\"]]", "def"} : new Object[]{"[[null,\"up\"]]", "def"} + ) ); + } @Test @@ -1747,103 +1729,97 @@ public void testArrayAggQuantile() @Test public void testArrayAggArrays() { - try { - ExpressionProcessing.initializeForTests(true); - cannotVectorize(); - testQuery( - "SELECT ARRAY_AGG(ARRAY[l1, l2]), ARRAY_AGG(DISTINCT ARRAY[l1, l2]) FROM numfoo", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE3) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .virtualColumns( - expressionVirtualColumn("v0", "array(\"l1\",\"l2\")", ColumnType.LONG_ARRAY) - ) - .aggregators( - aggregators( - new ExpressionLambdaAggregatorFactory( - "a0", - ImmutableSet.of("v0"), - "__acc", - "ARRAY>[]", - "ARRAY>[]", - true, - true, - false, - "array_append(\"__acc\", \"v0\")", - "array_concat(\"__acc\", \"a0\")", - null, - null, - ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, - TestExprMacroTable.INSTANCE - ), - new ExpressionLambdaAggregatorFactory( - "a1", - ImmutableSet.of("v0"), - "__acc", - "ARRAY>[]", - "ARRAY>[]", - true, - true, - false, - "array_set_add(\"__acc\", \"v0\")", - "array_set_add_all(\"__acc\", \"a1\")", - null, - null, - ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, - TestExprMacroTable.INSTANCE - ) - ) - ) - .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) - .build() - ), - (sql, results) -> { - // ordering is not stable in array_agg and array_concat_agg - List expected = ImmutableList.of( - useDefault ? - new Object[]{ - Arrays.asList( - Arrays.asList(7L, 0L), - Arrays.asList(325323L, 325323L), - Arrays.asList(0L, 0L), - Arrays.asList(0L, 0L), - Arrays.asList(0L, 0L), - Arrays.asList(0L, 0L) - ), - Arrays.asList( - Arrays.asList(0L, 0L), - Arrays.asList(7L, 0L), - Arrays.asList(325323L, 325323L) - ) - } - : - new Object[]{ - Arrays.asList( - Arrays.asList(7L, null), - Arrays.asList(325323L, 325323L), - Arrays.asList(0L, 0L), - Arrays.asList(null, null), - Arrays.asList(null, null), - Arrays.asList(null, null) - ), - Arrays.asList( - Arrays.asList(null, null), - Arrays.asList(0L, 0L), - Arrays.asList(7L, null), - Arrays.asList(325323L, 325323L) - ) - } - ); - assertResultsDeepEquals(sql, expected, results); - } - ); - } - finally { - ExpressionProcessing.initializeForTests(null); - } + cannotVectorize(); + testQuery( + "SELECT ARRAY_AGG(ARRAY[l1, l2]), ARRAY_AGG(DISTINCT ARRAY[l1, l2]) FROM numfoo", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY, + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE3) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .virtualColumns( + expressionVirtualColumn("v0", "array(\"l1\",\"l2\")", ColumnType.LONG_ARRAY) + ) + .aggregators( + aggregators( + new ExpressionLambdaAggregatorFactory( + "a0", + ImmutableSet.of("v0"), + "__acc", + "ARRAY>[]", + "ARRAY>[]", + true, + true, + false, + "array_append(\"__acc\", \"v0\")", + "array_concat(\"__acc\", \"a0\")", + null, + null, + ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, + TestExprMacroTable.INSTANCE + ), + new ExpressionLambdaAggregatorFactory( + "a1", + ImmutableSet.of("v0"), + "__acc", + "ARRAY>[]", + "ARRAY>[]", + true, + true, + false, + "array_set_add(\"__acc\", \"v0\")", + "array_set_add_all(\"__acc\", \"a1\")", + null, + null, + ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, + TestExprMacroTable.INSTANCE + ) + ) + ) + .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ), + (sql, results) -> { + // ordering is not stable in array_agg and array_concat_agg + List expected = ImmutableList.of( + useDefault ? + new Object[]{ + Arrays.asList( + Arrays.asList(7L, 0L), + Arrays.asList(325323L, 325323L), + Arrays.asList(0L, 0L), + Arrays.asList(0L, 0L), + Arrays.asList(0L, 0L), + Arrays.asList(0L, 0L) + ), + Arrays.asList( + Arrays.asList(0L, 0L), + Arrays.asList(7L, 0L), + Arrays.asList(325323L, 325323L) + ) + } + : + new Object[]{ + Arrays.asList( + Arrays.asList(7L, null), + Arrays.asList(325323L, 325323L), + Arrays.asList(0L, 0L), + Arrays.asList(null, null), + Arrays.asList(null, null), + Arrays.asList(null, null) + ), + Arrays.asList( + Arrays.asList(null, null), + Arrays.asList(0L, 0L), + Arrays.asList(7L, null), + Arrays.asList(325323L, 325323L) + ) + } + ); + assertResultsDeepEquals(sql, expected, results); + } + ); } @Test diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java index 1deaa18121bc..2311372e75e3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java @@ -745,7 +745,7 @@ public void testMultiValueStringConcatBackwardsCompat0dot22andOlder() ); } finally { - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 6353e6ec4316..4545bc09cddd 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -866,7 +866,8 @@ public void testJsonValueArrays() "SELECT " + "JSON_VALUE(arrayString, '$' RETURNING VARCHAR ARRAY), " + "JSON_VALUE(arrayLong, '$' RETURNING BIGINT ARRAY), " - + "JSON_VALUE(arrayDouble, '$' RETURNING DOUBLE ARRAY) " + + "JSON_VALUE(arrayDouble, '$' RETURNING DOUBLE ARRAY), " + + "JSON_VALUE(arrayNestedLong, '$[0]' RETURNING BIGINT ARRAY) " + "FROM druid.arrays" ) .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) @@ -878,9 +879,10 @@ public void testJsonValueArrays() .virtualColumns( new NestedFieldVirtualColumn("arrayString", "$", "v0", ColumnType.STRING_ARRAY), new NestedFieldVirtualColumn("arrayLong", "$", "v1", ColumnType.LONG_ARRAY), - new NestedFieldVirtualColumn("arrayDouble", "$", "v2", ColumnType.DOUBLE_ARRAY) + new NestedFieldVirtualColumn("arrayDouble", "$", "v2", ColumnType.DOUBLE_ARRAY), + new NestedFieldVirtualColumn("arrayNestedLong", "$[0]", "v3", ColumnType.LONG_ARRAY) ) - .columns("v0", "v1", "v2") + .columns("v0", "v1", "v2", "v3") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .legacy(false) .build() @@ -888,20 +890,21 @@ public void testJsonValueArrays() ) .expectedResults( ImmutableList.of( - new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, - new Object[]{null, null, null}, - new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2, 3.3, 4.0)}, - new Object[]{Arrays.asList("a", "b"), null, null}, - new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, - new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1, 3.3)}, - new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3, 4.4, 5.5)}, - new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, - new Object[]{null, null, null}, - new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2, 3.3, 4.0)}, - new Object[]{Arrays.asList("a", "b"), null, null}, - new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1, 2.2, 3.3)}, - new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1, 3.3)}, - new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3, 4.4, 5.5)} + new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1D, 2.2D, 3.3D), null}, + new Object[]{null, null, null, null}, + new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2D, 3.3D, 4.0D), Arrays.asList(1L, 2L)}, + new Object[]{Arrays.asList("a", "b"), null, null, Collections.singletonList(1L)}, + new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1D, 2.2D, 3.3D), Arrays.asList(1L, 2L, null)}, + new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1D, 3.3D), Collections.singletonList(1L)}, + new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3D, 4.4D, 5.5D), null}, + new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1D, 2.2D, 3.3D), null}, + new Object[]{null, null, null, null}, + new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2D, 3.3D, 4.0D), Arrays.asList(1L, 2L)}, + new Object[]{Arrays.asList("a", "b"), null, null, null}, + new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1D, 2.2D, 3.3D), Arrays.asList(2L, 3L)}, + new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1D, 3.3D), Collections.singletonList(1L)}, + new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3D, 4.4D, 5.5D), null} + ) ) .expectedSignature( @@ -909,6 +912,7 @@ public void testJsonValueArrays() .add("EXPR$0", ColumnType.STRING_ARRAY) .add("EXPR$1", ColumnType.LONG_ARRAY) .add("EXPR$2", ColumnType.DOUBLE_ARRAY) + .add("EXPR$3", ColumnType.LONG_ARRAY) .build() ) .run(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java index dc3a1367b9d5..5483fc8141af 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java @@ -162,7 +162,7 @@ public static void setupClass() public static void teardownClass() throws IOException { CLOSER.close(); - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } @Parameterized.Parameters(name = "query = {0}") diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestBase.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestBase.java index 81a7cbb36d77..e18342c64a19 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestBase.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestBase.java @@ -43,7 +43,7 @@ public abstract class CalciteTestBase public static void setupCalciteProperties() { NullHandling.initializeForTests(); - ExpressionProcessing.initializeForTests(null); + ExpressionProcessing.initializeForTests(); } /** From e3f48a72fc8f1d54ceb9fc3163e42bb11e37c986 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 24 Mar 2023 01:20:28 -0700 Subject: [PATCH 13/15] style and comments --- .../druid/math/expr/ExpressionProcessing.java | 4 ++-- .../druid/segment/UnnestStorageAdapter.java | 20 ++++++++++++++++--- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java index 4e9e81b096d7..ae1d5fb297bb 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java @@ -45,7 +45,7 @@ public class ExpressionProcessing @VisibleForTesting public static void initializeForTests() { - INSTANCE = new ExpressionProcessingConfig( null, null, null); + INSTANCE = new ExpressionProcessingConfig(null, null, null); } @VisibleForTesting @@ -57,7 +57,7 @@ public static void initializeForStrictBooleansTests(boolean useStrict) @VisibleForTesting public static void initializeForHomogenizeNullMultiValueStrings() { - INSTANCE = new ExpressionProcessingConfig( null, null, true); + INSTANCE = new ExpressionProcessingConfig(null, null, true); } /** diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index 12acc586226b..74ea48ad4eb6 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -477,8 +477,22 @@ private static boolean filterMapsOverMultiValueStrings(final Filter filter) */ private static boolean useDimensionCursor(@Nullable ColumnCapabilities capabilities) { - return capabilities != null && !capabilities.isArray() && capabilities.isDictionaryEncoded() - .and(capabilities.areDictionaryValuesUnique()) - .isTrue(); + if (capabilities == null) { + // capabilities being null here should be indicative of the column not existing or being a virtual column with + // no type information, chances are it is not going to be using a very cool dimension selector and so wont work + // with this, which requires real dictionary ids for the value matcher to work correctly + return false; + } + // the column needs real, unique value dictionary so that the value matcher id lookup works correctly, otherwise + // we must not use the dimension selector + if (capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) { + // if we got here, we only actually want to do this for dictionary encoded strings, since no other dictionary + // encoded column type should ever have multiple values set. nested and array columns are also dictionary encoded, + // but for arrays, the row is always a single dictionary id which maps to the entire array instead of an array + // of ids for each element, so we don't want to ever use the dimension selector cursor for that + return capabilities.is(ValueType.STRING); + } + // wasn't a dictionary encoded string, use the value selector + return false; } } From e988d05a0e0e2c5437dfe1e190bde566a5af013e Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 24 Mar 2023 01:38:33 -0700 Subject: [PATCH 14/15] remove negative tests --- .../apache/druid/math/expr/ParserTest.java | 10 --- .../sql/calcite/CalciteArraysQueryTest.java | 63 ------------------- 2 files changed, 73 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java b/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java index 2479b7486fc2..f6f5eccd10e9 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.column.TypeStrategies; @@ -420,15 +419,6 @@ public void testConstantComplexAndNestedArrays() ); } - @Test - public void nestedArraysExplodeIfNotEnabled() - { - expectedException.expect(IAE.class); - expectedException.expectMessage("Cannot create a nested array type [ARRAY>], 'druid.expressions.allowNestedArrays' must be set to true"); - validateConstantExpression("ARRAY>[]", new Object[]{}); - } - - @Test public void testLiteralArrayImplicitStringParseException() { 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 914a89ea96f2..8bba413f04c0 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 @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; @@ -1822,68 +1821,6 @@ public void testArrayAggArrays() ); } - @Test - public void testArrayAggArraysNoNest() - { - cannotVectorize(); - testQueryThrows( - "SELECT ARRAY_AGG(ARRAY[l1, l2]), ARRAY_AGG(DISTINCT ARRAY[l1, l2]) FROM numfoo", - QUERY_CONTEXT_NO_STRINGIFY_ARRAY, - ImmutableList.of( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE3) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .virtualColumns( - expressionVirtualColumn("v0", "array(\"l1\",\"l2\")", ColumnType.LONG_ARRAY) - ) - .aggregators( - aggregators( - new ExpressionLambdaAggregatorFactory( - "a0", - ImmutableSet.of("v0"), - "__acc", - "ARRAY>[]", - "ARRAY>[]", - true, - true, - false, - "array_append(\"__acc\", \"v0\")", - "array_concat(\"__acc\", \"a0\")", - null, - null, - ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, - TestExprMacroTable.INSTANCE - ), - new ExpressionLambdaAggregatorFactory( - "a1", - ImmutableSet.of("v0"), - "__acc", - "ARRAY>[]", - "ARRAY>[]", - true, - true, - false, - "array_set_add(\"__acc\", \"v0\")", - "array_set_add_all(\"__acc\", \"a1\")", - null, - null, - ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, - TestExprMacroTable.INSTANCE - ) - ) - ) - .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) - .build() - ), - expected -> { - expected.expect(IAE.class); - expected.expectMessage( - "Cannot create a nested array type [ARRAY>], 'druid.expressions.allowNestedArrays' must be set to true"); - } - ); - } - @Test public void testArrayConcatAggArrays() { From 151526747a436f0c9818c02d33fc67162b0974f4 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sun, 26 Mar 2023 23:30:46 -0700 Subject: [PATCH 15/15] fix stuff --- .../org/apache/druid/math/expr/ExprEval.java | 11 +++++------ .../druid/segment/NestedDataColumnIndexer.java | 18 +++++++++--------- .../druid/segment/NestedDataColumnMerger.java | 2 +- .../druid/segment/data/FrontCodedIndexed.java | 5 +++-- .../data/FrontCodedIntArrayIndexed.java | 5 +++-- .../nested/NestedDataColumnSerializer.java | 2 +- 6 files changed, 22 insertions(+), 21 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java index 1b337bbcbce9..cca67bc12aeb 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java @@ -387,6 +387,9 @@ public static ExprEval bestEffortArray(@Nullable List theList) */ public static ExprEval bestEffortOf(@Nullable Object val) { + if (val == null) { + return new StringExprEval(null); + } if (val instanceof ExprEval) { return (ExprEval) val; } @@ -490,12 +493,8 @@ public static ExprEval bestEffortOf(@Nullable Object val) return new StringExprEval(StringUtils.encodeBase64String((byte[]) val)); } - if (val != null) { - // is this cool? - return new ComplexExprEval(ExpressionType.UNKNOWN_COMPLEX, val); - } - - return new StringExprEval(null); + // is this cool? + return new ComplexExprEval(ExpressionType.UNKNOWN_COMPLEX, val); } public static ExprEval ofType(@Nullable ExpressionType type, @Nullable Object value) diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java index d3a9503dce11..9b0de5fcdcb7 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java @@ -19,7 +19,6 @@ package org.apache.druid.segment; -import com.google.common.base.Preconditions; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.java.util.common.IAE; @@ -497,12 +496,12 @@ private StructuredDataProcessor.ProcessedValue processValue(ExprEval eval) return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate); case ARRAY: // sanity check, this should never happen - Preconditions.checkNotNull( - columnType.getElementType(), - "Array type [%s] for value [%s] missing element type, how did this possibly happen?", - eval.type(), - eval.valueOrDefault() - ); + if (columnType.getElementType() == null) { + throw new IAE( + "Array type [%s] missing element type, how did this possibly happen?", + eval.type() + ); + } switch (columnType.getElementType().getType()) { case LONG: typeSet.add(ColumnType.LONG_ARRAY); @@ -527,11 +526,12 @@ private StructuredDataProcessor.ProcessedValue processValue(ExprEval eval) throw new IAE("Unhandled type: %s", columnType); } case STRING: - default: typeSet.add(ColumnType.STRING); final String asString = eval.asString(); sizeEstimate = globalDimensionDictionary.addStringValue(asString); - return new StructuredDataProcessor.ProcessedValue<>(eval.asString(), sizeEstimate); + return new StructuredDataProcessor.ProcessedValue<>(asString, sizeEstimate); + default: + throw new IAE("Unhandled type: %s", columnType); } } diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java index df0fe0506d1c..95a52b83c2ad 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java @@ -207,7 +207,7 @@ public void writeMergedValueDictionary(List adapters) throws I System.currentTimeMillis() - dimStartTime ); } - catch (Throwable ioe) { + catch (IOException ioe) { log.error(ioe, "Failed to merge dictionary for column [%s]", name); throw ioe; } diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java index 2d292db0ecaa..27a84b5df310 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java @@ -179,8 +179,9 @@ public ByteBuffer get(int index) } Indexed.checkIndex(index, adjustedNumValues); - // due to vbyte encoding, the null value is not actually stored in the bucket (no negative values), so we adjust - // the index + // due to vbyte encoding, the null value is not actually stored in the bucket. we would typically represent it as a + // length of -1, since 0 is the empty string, but VByte encoding cannot have negative values, so if the null value + // is present, we adjust the index by 1 since it is always stored as position 0 due to sorting first final int adjustedIndex = index - adjustIndex; // find the bucket which contains the value with maths final int bucket = adjustedIndex >> div; diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java index 011d61e2779c..de7ce2d977f7 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java @@ -154,8 +154,9 @@ public int[] get(int index) } Indexed.checkIndex(index, adjustedNumValues); - // due to vbyte encoding, the null value is not actually stored in the bucket (no negative values), so we adjust - // the index + // due to vbyte encoding, the null value is not actually stored in the bucket. we would typically represent it as a + // length of -1, since 0 is the empty array, but VByte encoding cannot have negative values, so if the null value + // is present, we adjust the index by 1 since it is always stored as position 0 due to sorting first final int adjustedIndex = index - adjustIndex; // find the bucket which contains the value with maths final int bucket = adjustedIndex >> div; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java index aec4641ca73d..020dc10ad0f3 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java @@ -105,7 +105,7 @@ public ProcessedValue processField(ArrayList fieldPath, @Null return ProcessedValue.NULL_LITERAL; } catch (IOException e) { - throw new RE(e, "Failed to write field [%s] value [%s]", fieldPath, fieldValue); + throw new RE(e, "Failed to write field [%s], unhandled value", fieldPath); } } return ProcessedValue.NULL_LITERAL;