You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by cw...@apache.org on 2023/03/27 19:42:47 UTC
[druid] branch master updated: nested columns + arrays = array columns! (#13803)
This is an automated email from the ASF dual-hosted git repository.
cwylie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git
The following commit(s) were added to refs/heads/master by this push:
new d5b1b5bc8e nested columns + arrays = array columns! (#13803)
d5b1b5bc8e is described below
commit d5b1b5bc8e751b8a45b2a2bf5a02ee2018968a9f
Author: Clint Wylie <cw...@apache.org>
AuthorDate: Mon Mar 27 12:42:35 2023 -0700
nested columns + arrays = array columns! (#13803)
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 type mimic behavior means that columns ingested with only root arrays of primitive values will be ARRAY typed columns
* neat test refactor stuff
* add v4 segment test
* add array element indexes
* add tests for unnest and array columns
* fix unnest column value selector cursor handling of null and empty arrays
---
.../data/input/AvroStreamInputFormatTest.java | 1 -
.../data/input/AvroStreamInputRowParserTest.java | 1 -
.../apache/druid/data/input/orc/OrcReaderTest.java | 6 -
.../parquet/NestedColumnParquetReaderTest.java | 4 +-
.../input/protobuf/ProtobufInputFormatTest.java | 6 +-
.../data/input/impl/DelimitedInputFormat.java | 1 +
.../druid/data/input/impl/JsonInputFormat.java | 1 -
.../druid/data/input/impl/MapInputRowParser.java | 26 +-
.../org/apache/druid/math/expr/ConstantExpr.java | 1 -
.../java/org/apache/druid/math/expr/ExprEval.java | 33 +-
.../druid/math/expr/ExpressionProcessing.java | 20 +-
.../math/expr/ExpressionProcessingConfig.java | 11 -
.../org/apache/druid/math/expr/ExpressionType.java | 8 -
.../java/org/apache/druid/math/expr/Function.java | 1 -
.../query/expression/NestedDataExpressions.java | 19 +-
...ngleValueStringGroupByVectorColumnSelector.java | 3 +-
.../druid/query/metadata/SegmentAnalyzer.java | 18 +
.../druid/segment/DimensionHandlerUtils.java | 11 +
.../druid/segment/NestedDataColumnIndexer.java | 169 ++-
.../druid/segment/NestedDataColumnMerger.java | 348 ++++--
.../segment/UnnestColumnValueSelectorCursor.java | 33 +-
.../apache/druid/segment/UnnestStorageAdapter.java | 69 +-
.../druid/segment/data/FrontCodedIndexed.java | 12 +-
...Indexed.java => FrontCodedIntArrayIndexed.java} | 413 ++-----
.../data/FrontCodedIntArrayIndexedWriter.java | 409 +++++++
.../incremental/IncrementalIndexSchema.java | 5 +
...lumnWriter.java => ArrayFieldColumnWriter.java} | 42 +-
.../nested/CompressedNestedDataComplexColumn.java | 478 +++++++-
.../GlobalDictionaryEncodedFieldColumnWriter.java | 38 +-
.../segment/nested/GlobalDictionaryIdLookup.java | 20 +
.../nested/GlobalDictionarySortedCollector.java | 48 +-
.../segment/nested/GlobalDimensionDictionary.java | 143 ++-
.../segment/nested/LocalDimensionDictionary.java | 4 +-
.../segment/nested/NestedDataColumnSerializer.java | 125 +-
.../segment/nested/NestedDataColumnSupplier.java | 68 +-
.../druid/segment/nested/NestedDataColumnV3.java | 3 +-
.../druid/segment/nested/NestedDataColumnV4.java | 3 +-
...edDataColumnV4.java => NestedDataColumnV5.java} | 9 +-
...er.java => NestedFieldColumnIndexSupplier.java} | 23 +-
...ava => NestedFieldDictionaryEncodedColumn.java} | 128 +-
...teralTypeInfo.java => NestedFieldTypeInfo.java} | 95 +-
.../segment/nested/StringFieldColumnWriter.java | 2 +-
.../segment/nested/StructuredDataProcessor.java | 91 +-
...mnWriter.java => VariantFieldColumnWriter.java} | 33 +-
.../segment/virtual/NestedFieldVirtualColumn.java | 12 +-
.../druid/data/input/ResourceInputSource.java | 110 ++
.../data/input/impl/InputRowParserSerdeTest.java | 1 -
.../druid/data/input/impl/JsonLineReaderTest.java | 6 +-
.../java/org/apache/druid/math/expr/EvalTest.java | 194 ++-
.../org/apache/druid/math/expr/OutputTypeTest.java | 8 +-
.../org/apache/druid/math/expr/ParserTest.java | 47 +-
.../druid/math/expr/VectorExprSanityTest.java | 2 +-
.../druid/query/MultiValuedDimensionTest.java | 2 +-
.../apache/druid/query/NestedDataTestUtils.java | 633 +++++-----
.../query/aggregation/AggregationTestHelper.java | 46 -
.../expression/NestedDataExpressionsTest.java | 2 +-
.../expression/VectorExpressionsSanityTest.java | 2 +-
.../query/groupby/NestedDataGroupByQueryTest.java | 64 +-
.../druid/query/scan/NestedDataScanQueryTest.java | 137 ++-
.../druid/query/topn/NestedDataTopNQueryTest.java | 32 +-
.../org/apache/druid/segment/IndexBuilder.java | 200 +++-
.../druid/segment/NestedDataColumnIndexerTest.java | 29 +-
.../UnnestColumnValueSelectorCursorTest.java | 128 +-
.../druid/segment/data/FrontCodedIndexedTest.java | 46 +-
.../data/FrontCodedIntArrayIndexedTest.java | 467 ++++++++
.../druid/segment/filter/ExpressionFilterTest.java | 2 +-
.../nested/NestedDataColumnSupplierTest.java | 432 ++++++-
...ava => NestedFieldColumnIndexSupplierTest.java} | 162 +--
...st.java => NestedFieldColumnSelectorsTest.java} | 31 +-
.../segment/nested/NestedFieldTypeInfoTest.java | 174 +++
.../druid/segment/transform/TransformSpecTest.java | 4 +-
.../druid/testing/InitializedNullHandlingTest.java | 2 +-
.../src/test/resources/nested-array-test-data.json | 14 +
.../test/resources/nested-numeric-test-data.json | 10 +
.../test/resources/nested-simple-test-data.json | 8 +
.../src/test/resources/nested-simple-test-data.tsv | 8 +
.../src/test/resources/nested-types-test-data.json | 8 +
.../src/test/resources/nested_segment_v4/index.zip | Bin 0 -> 2062 bytes
.../resources/numeric-nested-test-data-parser.json | 20 -
.../test/resources/numeric-nested-test-data.json | 10 -
.../resources/simple-nested-test-data-aggs.json | 6 -
.../resources/simple-nested-test-data-parser.json | 33 -
.../simple-nested-test-data-tsv-parser.json | 41 -
.../simple-nested-test-data-tsv-transform.json | 24 -
.../test/resources/simple-nested-test-data.json | 8 -
.../src/test/resources/simple-nested-test-data.tsv | 8 -
processing/src/test/resources/types-test-data.json | 8 -
.../java/org/apache/druid/cli/DumpSegment.java | 6 +-
.../builtin/ArrayContainsOperatorConversion.java | 2 +-
.../builtin/NestedDataOperatorConversions.java | 159 +++
.../sql/calcite/planner/DruidOperatorTable.java | 3 +
.../druid/sql/calcite/BaseCalciteQueryTest.java | 1 +
.../druid/sql/calcite/CalciteArraysQueryTest.java | 219 ++--
.../calcite/CalciteMultiValueStringQueryTest.java | 2 +-
.../sql/calcite/CalciteNestedDataQueryTest.java | 1244 +++++++++++++++++++-
.../calcite/SqlVectorizedExpressionSanityTest.java | 2 +-
.../druid/sql/calcite/util/CalciteTestBase.java | 2 +-
.../druid/sql/calcite/util/TestDataBuilder.java | 75 +-
...d.jgz => wikiticker-2015-09-12-sampled.json.gz} | Bin
99 files changed, 6032 insertions(+), 1846 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 177766510d..222c5472cd 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<String> DIMENSIONS = Arrays.asList(EVENT_TYPE, ID, SOME_OTHER_ID, IS_VALID);
private static final List<String> 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 3bcec4e3c1..b4a3514bce 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<String> DIMENSIONS = Arrays.asList(EVENT_TYPE, ID, SOME_OTHER_ID, IS_VALID);
private static final List<String> 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 3013b4439e..6a395af8e9 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.CloseableIterator;
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;
@@ -453,7 +452,6 @@ public class OrcReaderTest extends InitializedNullHandlingTest
"middle",
"list",
"map",
- "ts",
"decimal1"
);
try (CloseableIterator<InputRow> iterator = reader.read()) {
@@ -593,7 +591,6 @@ public class OrcReaderTest extends InitializedNullHandlingTest
@Test
public void testNestedArray() throws IOException
{
- ExpressionProcessing.initializeForTests(true);
final InputFormat inputFormat = new OrcInputFormat(
new JSONPathSpec(
true,
@@ -669,9 +666,6 @@ public class OrcReaderTest extends InitializedNullHandlingTest
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/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 cd587f444f..96d2a8c578 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 class NestedColumnParquetReaderTest extends BaseParquetReaderTest
);
List<InputRow> 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 class NestedColumnParquetReaderTest extends BaseParquetReaderTest
);
List<InputRow> 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 498174852b..86b5ae4edc 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"),
@@ -243,7 +243,6 @@ public class ProtobufInputFormatTest
.add("someFloatColumn")
.add("id")
.add("someBytesColumn")
- .add("timestamp")
.build(),
row.getDimensions()
);
@@ -380,8 +379,7 @@ public class ProtobufInputFormatTest
"someFloatColumn",
"eventType",
"id",
- "someBytesColumn",
- "timestamp"
+ "someBytesColumn"
),
row.getDimensions()
);
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 f1a0a48522..d409f7ef12 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
@@ -41,6 +41,7 @@ import java.util.List;
public class DelimitedInputFormat extends FlatTextInputFormat
{
public static final String TYPE_KEY = "tsv";
+
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 821c5bf435..200c621e13 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,7 +40,6 @@ import java.util.Objects;
public class JsonInputFormat extends NestedInputFormat
{
public static final String TYPE_KEY = "json";
-
private final Map<String, Boolean> 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 58930804d0..47ef09e527 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
@@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
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;
@@ -36,6 +35,7 @@ import java.util.ArrayList;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
+import java.util.Set;
public class MapInputRowParser implements InputRowParser<Map<String, Object>>
{
@@ -77,22 +77,38 @@ public class MapInputRowParser implements InputRowParser<Map<String, Object>>
* 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<String> findDimensions(
+ TimestampSpec timestampSpec,
DimensionsSpec dimensionsSpec,
Map<String, Object> rawInputRow
)
{
+ final String timestampColumn = timestampSpec.getTimestampColumn();
+ final Set<String> exclusions = dimensionsSpec.getDimensionExclusions();
if (dimensionsSpec.isIncludeAllDimensions()) {
LinkedHashSet<String> dimensions = new LinkedHashSet<>(dimensionsSpec.getDimensionNames());
- dimensions.addAll(Sets.difference(rawInputRow.keySet(), dimensionsSpec.getDimensionExclusions()));
+ for (String field : rawInputRow.keySet()) {
+ if (timestampColumn.equals(field) || exclusions.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<String> dimensions = new ArrayList<>();
+ for (String field : rawInputRow.keySet()) {
+ if (timestampColumn.equals(field) || exclusions.contains(field)) {
+ continue;
+ }
+ dimensions.add(field);
+ }
+ return dimensions;
}
}
}
@@ -104,7 +120,7 @@ public class MapInputRowParser implements InputRowParser<Map<String, Object>>
Map<String, Object> theMap
) throws ParseException
{
- final List<String> dimensionsToUse = findDimensions(dimensionsSpec, theMap);
+ final List<String> dimensionsToUse = findDimensions(timestampSpec, dimensionsSpec, theMap);
final DateTime timestamp;
try {
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 ff4cedbecb..fdf6f080ee 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 @@ class ArrayExpr extends ConstantExpr<Object[]>
{
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 062ef9805d..cca67bc12a 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,11 +370,26 @@ public abstract class ExprEval<T>
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<ExpressionType, Object[]> 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
*/
public static ExprEval bestEffortOf(@Nullable Object val)
{
+ if (val == null) {
+ return new StringExprEval(null);
+ }
if (val instanceof ExprEval) {
return (ExprEval) val;
}
@@ -468,14 +483,7 @@ public abstract class ExprEval<T>
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<ExpressionType, Object[]> 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
@@ -485,12 +493,8 @@ public abstract class ExprEval<T>
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)
@@ -1109,7 +1113,6 @@ public abstract class ExprEval<T>
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 905e0850f7..ae1d5fb297 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 @@ package org.apache.druid.math.expr;
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 b832578fe3..44e3b9409d 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 @@ import javax.annotation.Nullable;
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 class 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 6f6b7c3025..f2f17f2ef7 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.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
-import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.column.BaseTypeSignature;
import org.apache.druid.segment.column.ColumnType;
@@ -210,11 +209,4 @@ public class ExpressionType extends BaseTypeSignature<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 70cd0f8f27..03eb95820f 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 @@ public interface Function extends NamedFunction
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/expression/NestedDataExpressions.java b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java
index 833f46428b..2eabe194dd 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,10 +480,25 @@ public class NestedDataExpressions
final StructuredDataProcessor processor = new StructuredDataProcessor()
{
@Override
- public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+ public ProcessedValue<?> processField(ArrayList<NestedPathPart> 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 ProcessedValue<?> processArrayField(
+ ArrayList<NestedPathPart> fieldPath,
+ @Nullable List<?> array
+ )
+ {
+ // 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 ProcessedValue.NULL_LITERAL;
+ }
+ return null;
}
};
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 937ddc339c..83f49e1c83 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.common.config.NullHandling;
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 class DictionaryBuildingSingleValueStringGroupByVectorColumnSelector impl
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 859396170b..7704a8e4c8 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,9 @@ public class SegmentAnalyzer
analysis = analyzeStringColumn(capabilities, storageAdapter, columnName);
}
break;
+ case ARRAY:
+ analysis = analyzeArrayColumn(capabilities);
+ break;
case COMPLEX:
final ColumnHolder columnHolder = index != null ? index.getColumnHolder(columnName) : null;
analysis = analyzeComplexColumn(capabilities, numRows, columnHolder);
@@ -385,4 +388,19 @@ public class SegmentAnalyzer
);
}
}
+
+ private ColumnAnalysis analyzeArrayColumn(final ColumnCapabilities capabilities)
+ {
+ return new ColumnAnalysis(
+ capabilities.toColumnType(),
+ capabilities.getType().name(),
+ false,
+ capabilities.hasNulls().isTrue(),
+ 0L,
+ null,
+ null,
+ null,
+ null
+ );
+ }
}
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 b509c7f058..8009afd654 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 final class DimensionHandlerUtils
{
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/NestedDataColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java
index df64897eb7..9b0de5fcdc 100644
--- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java
+++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java
@@ -21,6 +21,8 @@ package org.apache.druid.segment;
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;
@@ -34,7 +36,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexRowHolder;
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;
@@ -42,6 +44,8 @@ import org.apache.druid.segment.nested.StructuredDataProcessor;
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;
@@ -51,7 +55,7 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
{
protected volatile boolean hasNulls = false;
- protected SortedMap<String, LiteralFieldIndexer> fieldIndexers = new TreeMap<>();
+ protected SortedMap<String, FieldIndexer> fieldIndexers = new TreeMap<>();
protected final GlobalDimensionDictionary globalDictionary = new GlobalDimensionDictionary();
int estimatedFieldKeySize = 0;
@@ -59,16 +63,42 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
protected final StructuredDataProcessor indexerProcessor = new StructuredDataProcessor()
{
@Override
- public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+ public ProcessedValue<?> processField(ArrayList<NestedPathPart> 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) {
+ final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
+ ExprEval<?> eval = ExprEval.bestEffortOf(fieldValue);
+ FieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
+ if (fieldIndexer == null) {
+ estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
+ fieldIndexer = new FieldIndexer(globalDictionary);
+ fieldIndexers.put(fieldName, fieldIndexer);
+ }
+ return fieldIndexer.processValue(eval);
}
- return fieldIndexer.processValue(fieldValue);
+ return ProcessedValue.NULL_LITERAL;
+ }
+
+ @Nullable
+ @Override
+ public ProcessedValue<?> processArrayField(
+ ArrayList<NestedPathPart> fieldPath,
+ @Nullable List<?> array
+ )
+ {
+ final ExprEval<?> eval = ExprEval.bestEffortArray(array);
+ if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+ final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
+ FieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
+ if (fieldIndexer == null) {
+ estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
+ fieldIndexer = new FieldIndexer(globalDictionary);
+ fieldIndexers.put(fieldName, fieldIndexer);
+ }
+ return fieldIndexer.processValue(eval);
+ }
+ return null;
}
};
@@ -145,6 +175,15 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
final int dimIndex = desc.getIndex();
final ColumnValueSelector<?> rootLiteralSelector = getRootLiteralValueSelector(currEntry, dimIndex);
if (rootLiteralSelector != null) {
+ 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()
{
@Nullable
@@ -165,7 +204,12 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
}
};
}
- 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
@@ -212,7 +256,7 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
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())
@@ -265,7 +309,7 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
@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
@@ -330,9 +374,9 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
throw new UnsupportedOperationException("Not supported");
}
- public void mergeFields(SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields)
+ public void mergeFields(SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> mergedFields)
{
- for (Map.Entry<String, NestedDataColumnIndexer.LiteralFieldIndexer> entry : fieldIndexers.entrySet()) {
+ for (Map.Entry<String, FieldIndexer> 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());
@@ -354,7 +398,7 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
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;
}
@@ -411,7 +455,7 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
if (0 <= dimIndex && dimIndex < dims.length) {
final StructuredData data = (StructuredData) dims[dimIndex];
if (data != null) {
- return data.getValue();
+ return ExprEval.bestEffortOf(data.getValue()).valueOrDefault();
}
}
@@ -426,55 +470,72 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
};
}
- 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(@Nullable Object value)
+ private StructuredDataProcessor.ProcessedValue<?> 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()
+ 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.ProcessedValue<>(eval.asLong(), sizeEstimate);
+ case DOUBLE:
+ typeSet.add(ColumnType.DOUBLE);
+ sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+ return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate);
+ case ARRAY:
+ // sanity check, this should never happen
+ if (columnType.getElementType() == null) {
+ throw new IAE(
+ "Array type [%s] missing element type, how did this possibly happen?",
+ eval.type()
);
- case STRING:
- default:
- final String asString = eval.asString();
- globalDimensionDictionary.addStringValue(asString);
- typeSet.add(ColumnType.STRING);
- return new StructuredDataProcessor.ProcessedLiteral<>(
- eval.asString(),
- StructuredDataProcessor.estimateStringSize(asString)
- );
- }
+ }
+ switch (columnType.getElementType().getType()) {
+ case LONG:
+ typeSet.add(ColumnType.LONG_ARRAY);
+ final Object[] longArray = eval.asArray();
+ sizeEstimate = globalDimensionDictionary.addLongArray(longArray);
+ 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.ProcessedValue<>(doubleArray, sizeEstimate);
+ case STRING:
+ final Object[] stringArray = eval.asArray();
+ // 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.ProcessedValue<>(stringArray, sizeEstimate);
+ default:
+ throw new IAE("Unhandled type: %s", columnType);
+ }
+ case STRING:
+ typeSet.add(ColumnType.STRING);
+ final String asString = eval.asString();
+ sizeEstimate = globalDimensionDictionary.addStringValue(asString);
+ return new StructuredDataProcessor.ProcessedValue<>(asString, sizeEstimate);
+ default:
+ throw new IAE("Unhandled type: %s", columnType);
}
- return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
}
- 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 bf672c3522..95a52b83c2 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,22 +30,28 @@ import org.apache.druid.segment.column.ColumnDescriptor;
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;
-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;
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,99 +94,129 @@ public class NestedDataColumnMerger implements DimensionMergerV9
@Override
public void writeMergedValueDictionary(List<IndexableAdapter> 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<Object[]>[] sortedArrayLookups = new Iterable[adapters.size()];
+
+ final SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> 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 || dimValues.allNull();
+ sortedLookup = dimValues;
+ if (!allNulls) {
+ sortedLookups[i] = dimValues.getSortedStrings();
+ sortedLongLookups[i] = dimValues.getSortedLongs();
+ sortedDoubleLookups[i] = dimValues.getSortedDoubles();
+ sortedArrayLookups[i] = dimValues.getSortedArrays();
+ numMergeIndex++;
+ }
+ }
- final SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> 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.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 {
- throw new ISE("Unable to merge columns of unsupported adapter %s", adapter.getClass());
+ final SimpleDictionaryMergingIterator<String> stringIterator = new SimpleDictionaryMergingIterator<>(
+ sortedLookups,
+ STRING_MERGING_COMPARATOR
+ );
+ final SimpleDictionaryMergingIterator<Long> longIterator = new SimpleDictionaryMergingIterator<>(
+ sortedLongLookups,
+ LONG_MERGING_COMPARATOR
+ );
+ final SimpleDictionaryMergingIterator<Double> doubleIterator = new SimpleDictionaryMergingIterator<>(
+ sortedDoubleLookups,
+ DOUBLE_MERGING_COMPARATOR
+ );
+ final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator(
+ sortedArrayLookups,
+ defaultSerializer.getGlobalLookup()
+ );
+ defaultSerializer.serializeDictionaries(
+ () -> stringIterator,
+ () -> longIterator,
+ () -> doubleIterator,
+ () -> arrayIterator
+ );
+ stringCardinality = stringIterator.getCardinality();
+ longCardinality = longIterator.getCardinality();
+ doubleCardinality = doubleIterator.getCardinality();
+ arrayCardinality = arrayIterator.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<String> 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<Long> longDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>(
- sortedLongLookups,
- LONG_MERGING_COMPARATOR
- );
- SimpleDictionaryMergingIterator<Double> 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 (IOException ioe) {
+ log.error(ioe, "Failed to merge dictionary for column [%s]", name);
+ throw ioe;
+ }
}
@Nullable
private GlobalDictionarySortedCollector getSortedIndexFromIncrementalAdapter(
IncrementalIndexAdapter adapter,
- SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields
+ SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> mergedFields
)
{
final IncrementalIndex index = adapter.getIncrementalIndex();
@@ -194,7 +232,7 @@ public class NestedDataColumnMerger implements DimensionMergerV9
@Nullable
private GlobalDictionarySortedCollector getSortedIndexesFromQueryableAdapter(
QueryableIndexIndexableAdapter adapter,
- SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields
+ SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> mergedFields
)
{
final ColumnHolder columnHolder = adapter.getQueryableIndex().getColumnHolder(name);
@@ -214,7 +252,7 @@ public class NestedDataColumnMerger implements DimensionMergerV9
}
private GlobalDictionarySortedCollector getSortedIndexFromV1QueryableAdapterNestedColumn(
- SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields,
+ SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> mergedFields,
BaseColumn col
)
{
@@ -223,10 +261,10 @@ public class NestedDataColumnMerger implements DimensionMergerV9
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());
});
@@ -234,7 +272,9 @@ public class NestedDataColumnMerger implements DimensionMergerV9
return new GlobalDictionarySortedCollector(
new StringEncodingStrategies.Utf8ToStringIndexed(column.getStringDictionary()),
column.getLongDictionary(),
- column.getDoubleDictionary()
+ column.getDoubleDictionary(),
+ column.getArraysIterable(),
+ column.getArrayDictionary().size()
);
}
@@ -271,13 +311,139 @@ public class NestedDataColumnMerger implements DimensionMergerV9
return descriptorBuilder.build();
}
- private <T> boolean allNull(Indexed<T> dimValues)
+ public static class ArrayDictionaryMergingIterator implements Iterator<int[]>
{
- for (int i = 0, size = dimValues.size(); i < size; i++) {
- if (dimValues.get(i) != null) {
- return false;
+ private static final Comparator<PeekingIterator<int[]>> PEEKING_ITERATOR_COMPARATOR =
+ (lhs, rhs) -> FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR.compare(lhs.peek(), rhs.peek());
+
+ protected final PriorityQueue<PeekingIterator<int[]>> pQueue;
+ private final Iterable<Object[]>[] dimValueLookups;
+ private final GlobalDictionaryIdLookup idLookup;
+
+ protected int counter;
+ private boolean initialized;
+
+ public ArrayDictionaryMergingIterator(Iterable<Object[]>[] dimValueLookups, GlobalDictionaryIdLookup idLookup)
+ {
+ 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<Object[]> dimValueLookup : dimValueLookups) {
+ if (dimValueLookup == null) {
+ continue;
+ }
+ final PeekingIterator<int[]> iter = Iterators.peekingIterator(
+ new IdLookupArrayIterator(idLookup, dimValueLookup.iterator())
+ );
+ if (iter.hasNext()) {
+ pQueue.add(iter);
+ }
+ }
+ initialized = true;
+ }
+
+ @Override
+ public boolean hasNext()
+ {
+ if (!initialized) {
+ initialize();
+ }
+ return !pQueue.isEmpty();
+ }
+
+ @Override
+ public int[] next()
+ {
+ if (!initialized) {
+ initialize();
+ }
+ PeekingIterator<int[]> 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<int[]> 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<int[]>
+ {
+ private final GlobalDictionaryIdLookup idLookup;
+ private final Iterator<Object[]> delegate;
+
+ public IdLookupArrayIterator(
+ GlobalDictionaryIdLookup idLookup,
+ Iterator<Object[]> 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[] globalIds = new int[next.length];
+ for (int i = 0; i < next.length; i++) {
+ if (next[i] == null) {
+ globalIds[i] = 0;
+ } else if (next[i] instanceof String) {
+ globalIds[i] = idLookup.lookupString((String) next[i]);
+ } else if (next[i] instanceof Long) {
+ globalIds[i] = idLookup.lookupLong((Long) next[i]);
+ } else if (next[i] instanceof Double) {
+ globalIds[i] = idLookup.lookupDouble((Double) next[i]);
+ } else {
+ globalIds[i] = -1;
+ }
+ Preconditions.checkArgument(
+ globalIds[i] >= 0,
+ "unknown global id [%s] for value [%s]",
+ globalIds[i],
+ next[i]
+ );
}
+ return globalIds;
}
- return true;
}
}
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 9179d55dc1..40a3aeeda2 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 class UnnestColumnValueSelectorCursor implements Cursor
@Override
public Object getObject()
{
- if (!unnestListForCurrentRow.isEmpty()) {
- return unnestListForCurrentRow.get(index);
- }
- return null;
+ return unnestListForCurrentRow.get(index);
}
@Override
@@ -279,7 +276,7 @@ public class UnnestColumnValueSelectorCursor implements Cursor
{
currentVal = this.columnValueSelector.getObject();
if (currentVal == null) {
- unnestListForCurrentRow = Collections.singletonList(null);
+ unnestListForCurrentRow = Collections.emptyList();
} else if (currentVal instanceof List) {
unnestListForCurrentRow = (List<Object>) currentVal;
} else if (currentVal instanceof Object[]) {
@@ -296,9 +293,25 @@ public class UnnestColumnValueSelectorCursor implements Cursor
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 @@ public class UnnestColumnValueSelectorCursor implements Cursor
*/
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 8506a99d02..74ea48ad4e 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 class UnnestStorageAdapter implements StorageAdapter
public UnnestStorageAdapter(
final StorageAdapter baseAdapter,
final VirtualColumn unnestColumn,
- final DimFilter unnestFilter
+ @Nullable final DimFilter unnestFilter
)
{
this.baseAdapter = baseAdapter;
@@ -100,9 +100,7 @@ public class UnnestStorageAdapter implements StorageAdapter
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<Cursor> baseCursorSequence = baseAdapter.makeCursors(
@@ -118,37 +116,29 @@ public class UnnestStorageAdapter implements StorageAdapter
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.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,31 @@ public class UnnestStorageAdapter implements StorageAdapter
|| 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)
+ {
+ 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;
+ }
}
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 cc0f280405..27a84b5df3 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 final class FrontCodedIndexed implements Indexed<ByteBuffer>
}
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;
@@ -199,6 +200,10 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
return hasNull ? 0 : -1;
}
+ if (numBuckets == 0) {
+ return hasNull ? -2 : -1;
+ }
+
int minBucketIndex = 0;
int maxBucketIndex = numBuckets - 1;
while (minBucketIndex < maxBucketIndex) {
@@ -298,6 +303,9 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
@Override
public Iterator<ByteBuffer> iterator()
{
+ if (adjustedNumValues == 0) {
+ return Collections.emptyIterator();
+ }
if (hasNull && adjustedNumValues == 1) {
return Collections.<ByteBuffer>singletonList(null).iterator();
}
diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java
similarity index 58%
copy from processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java
copy to processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java
index cc0f280405..de7ce2d977 100644
--- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java
+++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java
@@ -21,7 +21,6 @@ package org.apache.druid.segment.data;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
-import org.apache.druid.annotations.SuppressFBWarnings;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
@@ -35,52 +34,50 @@ import java.util.Iterator;
import java.util.NoSuchElementException;
/**
- * {@link Indexed} specialized for storing variable-width binary values (such as utf8 encoded strings), which must be
- * sorted and unique, using 'front coding'. Front coding is a type of delta encoding for byte arrays, 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 byte array of the bucket to use as a prefix, followed
- * by the remaining bytes after the prefix to complete the value. If using 'incremental' buckets, instead of using the
- * prefix of the first bucket value, instead the prefix is computed against the immediately preceding value in the
- * bucket.
- * <p>
+ * {@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[] |
- * <p>
+ *
* "offsets" are the ending offsets of each bucket stored in order, stored as plain integers for easy random access.
- * <p>
+ *
* bucket layout:
* | first value | prefix length | fragment | ... | prefix length | fragment |
* | ----------- | ------------- | -------- | --- | ------------- | -------- |
- * | blob | vbyte int | blob | ... | vbyte int | blob |
- * <p>
- * blob layout:
- * | blob length | blob bytes |
- * | ----------- | ---------- |
- * | vbyte int | byte[] |
- * <p>
- * <p>
+ * | 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.
- * <p>
+ *
* 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).
- * <p>
+ *
* 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.
- * <p>
+ *
* This class is not thread-safe since during operation modifies positions of a shared buffer.
*/
-public final class FrontCodedIndexed implements Indexed<ByteBuffer>
+public final class FrontCodedIntArrayIndexed implements Indexed<int[]>
{
- public static Supplier<FrontCodedIndexed> read(ByteBuffer buffer, ByteOrder ordering)
+ public static Supplier<FrontCodedIntArrayIndexed> read(ByteBuffer buffer, ByteOrder ordering)
{
final ByteBuffer orderedBuffer = buffer.asReadOnlyBuffer().order(ordering);
final byte version = orderedBuffer.get();
- Preconditions.checkArgument(version == 0 || version == 1, "only V0 and V1 exist, encountered " + version);
+ 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);
@@ -90,14 +87,13 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
// move position to end of buffer
buffer.position(offsetsPosition + size);
- return () -> new FrontCodedIndexed(
+ return () -> new FrontCodedIntArrayIndexed(
buffer,
ordering,
bucketSize,
numValues,
hasNull,
- offsetsPosition,
- version
+ offsetsPosition
);
}
@@ -105,8 +101,6 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
private final int adjustedNumValues;
private final int adjustIndex;
private final int bucketSize;
- private final int[] unwindPrefixLength;
- private final int[] unwindBufferPosition;
private final int numBuckets;
private final int div;
private final int rem;
@@ -114,20 +108,16 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
private final int bucketsPosition;
private final boolean hasNull;
private final int lastBucketNumValues;
+ private final int[] unwindPrefixLength;
+ private final int[] unwindBufferPosition;
- private final GetBucketValue getBucketValueFn;
- private final ReadBucket readBucketFn;
- private final FindInBucket findInBucketFn;
-
- @SuppressFBWarnings(value = "NP_STORE_INTO_NONNULL_FIELD", justification = "V0 does not use unwindPrefixLength or unwindBufferPosition")
- private FrontCodedIndexed(
+ private FrontCodedIntArrayIndexed(
ByteBuffer buffer,
ByteOrder order,
int bucketSize,
int numValues,
boolean hasNull,
- int offsetsPosition,
- byte version
+ int offsetsPosition
)
{
if (Integer.bitCount(bucketSize) != 1) {
@@ -145,23 +135,8 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
this.lastBucketNumValues = (numValues & rem) == 0 ? bucketSize : numValues & rem;
this.offsetsPosition = offsetsPosition;
this.bucketsPosition = offsetsPosition + ((numBuckets - 1) * Integer.BYTES);
- if (version == 0) {
- // version zero, all prefixes are computed against the first value in the bucket
- this.getBucketValueFn = FrontCodedIndexed::getFromBucketV0;
- this.readBucketFn = FrontCodedIndexed::readBucketV0;
- this.findInBucketFn = this::findValueInBucketV0;
- //noinspection DataFlowIssue
- this.unwindPrefixLength = null;
- //noinspection DataFlowIssue
- this.unwindBufferPosition = null;
- } else {
- // version one uses 'incremental' buckets, where the prefix is computed against the previous value
- this.unwindPrefixLength = new int[bucketSize];
- this.unwindBufferPosition = new int[bucketSize];
- this.getBucketValueFn = this::getFromBucketV1;
- this.readBucketFn = this::readBucketV1;
- this.findInBucketFn = this::findValueInBucketV1;
- }
+ this.unwindPrefixLength = new int[bucketSize];
+ this.unwindBufferPosition = new int[bucketSize];
}
@Override
@@ -172,26 +147,27 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
@Nullable
@Override
- public ByteBuffer get(int index)
+ 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
+ // 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;
final int bucketIndex = adjustedIndex & rem;
final int offset = getBucketOffset(bucket);
buffer.position(offset);
- return getBucketValueFn.get(buffer, bucketIndex);
+ return getFromBucket(buffer, bucketIndex);
}
@Override
- public int indexOf(@Nullable ByteBuffer value)
+ 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
@@ -199,6 +175,10 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
return hasNull ? 0 : -1;
}
+ if (numBuckets == 0) {
+ return hasNull ? -2 : -1;
+ }
+
int minBucketIndex = 0;
int maxBucketIndex = numBuckets - 1;
while (minBucketIndex < maxBucketIndex) {
@@ -214,13 +194,13 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
// 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;
+ final int sharedPrefix = (buffer.position() - firstOffset) / Integer.BYTES;
if (comparison == 0) {
- if (firstLength == value.remaining()) {
+ 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.remaining());
+ comparison = Integer.compare(firstLength, value.length);
}
}
@@ -231,21 +211,21 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
final int nextLength = VByte.readInt(buffer);
int comparisonNext = compareBucketFirstValue(buffer, nextLength, value);
if (comparisonNext == 0) {
- if (nextLength == value.remaining()) {
+ 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.remaining());
+ 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);
+ buffer.position(firstOffset + (firstLength * Integer.BYTES));
- return findInBucketFn.find(value, currBucketFirstValueIndex, bucketSize, sharedPrefix);
+ return findValueInBucket(value, currBucketFirstValueIndex, bucketSize, sharedPrefix);
} else if (comparison < 0) {
minBucketIndex = currentBucket + 1;
} else {
@@ -268,24 +248,24 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
final int firstLength = VByte.readInt(buffer);
final int firstOffset = buffer.position();
int comparison = compareBucketFirstValue(buffer, firstLength, value);
- final int sharedPrefix = buffer.position() - firstOffset;
+ final int sharedPrefix = (buffer.position() - firstOffset) / Integer.BYTES;
if (comparison == 0) {
- if (firstLength == value.remaining()) {
+ 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.remaining());
+ comparison = Integer.compare(firstLength, value.length);
}
}
if (comparison > 0) {
// value preceedes bucket, so bail out
- return ~(bucketIndexBase + adjustIndex);
+ return -(bucketIndexBase + adjustIndex) - 1;
}
- buffer.position(firstOffset + firstLength);
+ buffer.position(firstOffset + (firstLength * Integer.BYTES));
- return findInBucketFn.find(value, bucketIndexBase, numValuesInBucket, sharedPrefix);
+ return findValueInBucket(value, bucketIndexBase, numValuesInBucket, sharedPrefix);
}
@Override
@@ -296,20 +276,24 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
}
@Override
- public Iterator<ByteBuffer> iterator()
+ public Iterator<int[]> iterator()
{
+ if (adjustedNumValues == 0) {
+ return Collections.emptyIterator();
+ }
if (hasNull && adjustedNumValues == 1) {
- return Collections.<ByteBuffer>singletonList(null).iterator();
+ return Collections.<int[]>singletonList(null).iterator();
}
+
ByteBuffer copy = buffer.asReadOnlyBuffer().order(buffer.order());
copy.position(bucketsPosition);
- final ByteBuffer[] firstBucket = readBucketFn.readBucket(copy, numBuckets > 1 ? bucketSize : lastBucketNumValues);
+ 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<ByteBuffer>()
+ return new Iterator<int[]>()
{
private int currIndex = 0;
private int currentBucketIndex = 0;
- private ByteBuffer[] currentBucket = firstBucket;
+ private int[][] currentBucket = firstBucket;
@Override
public boolean hasNext()
@@ -318,7 +302,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
}
@Override
- public ByteBuffer next()
+ public int[] next()
{
// null is handled special
if (hasNull && currIndex == 0) {
@@ -334,7 +318,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
if (bucketNum != currentBucketIndex) {
final int offset = copy.getInt(offsetsPosition + ((bucketNum - 1) * Integer.BYTES));
copy.position(bucketsPosition + offset);
- currentBucket = readBucketFn.readBucket(
+ currentBucket = readBucket(
copy,
bucketNum < (numBuckets - 1) ? bucketSize : lastBucketNumValues
);
@@ -374,23 +358,23 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
* 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.
- * <p>
+ *
* 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, ByteBuffer value)
+ private static int compareBucketFirstValue(ByteBuffer bucketBuffer, int length, int[] value)
{
final int startOffset = bucketBuffer.position();
- final int commonLength = Math.min(length, value.remaining());
+ 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 = StringUtils.compareUtf8UsingJavaStringOrdering(bucketBuffer.get(), value.get(sharedPrefix));
+ comparison = Integer.compare(bucketBuffer.getInt(), value[sharedPrefix]);
if (comparison != 0) {
- bucketBuffer.position(startOffset + sharedPrefix);
+ bucketBuffer.position(startOffset + (sharedPrefix * Integer.BYTES));
break;
}
}
@@ -400,162 +384,19 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
/**
* 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.
- * <p>
+ * 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
* 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
- * <p>
+ *
* this method modifies the position of {@link #buffer}
*/
- private int findValueInBucketV0(
- ByteBuffer 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) {
- // this value shares more in common with the first value, so the value we are looking for comes after
- final int skip = VByte.readInt(buffer);
- buffer.position(buffer.position() + skip);
- 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.remaining() - prefixLength);
- int fragmentComparison = 0;
- for (int i = 0; i < common; i++) {
- fragmentComparison = StringUtils.compareUtf8UsingJavaStringOrdering(
- buffer.get(buffer.position() + i),
- value.get(prefixLength + i)
- );
- if (fragmentComparison != 0) {
- break;
- }
- }
- if (fragmentComparison == 0) {
- fragmentComparison = Integer.compare(prefixLength + fragmentLength, value.remaining());
- }
-
- if (fragmentComparison == 0) {
- return (currBucketFirstValueIndex + adjustIndex) + relativePosition;
- } else if (fragmentComparison < 0) {
- buffer.position(buffer.position() + fragmentLength);
- insertionPoint++;
- } else {
- break;
- }
- }
- }
- // (-(insertion point) - 1)
- return -(currBucketFirstValueIndex + adjustIndex) + (~insertionPoint);
- }
-
- /**
- * Get a value from a bucket at a relative position.
- * <p>
- * This method modifies the position of the buffer.
- */
- static ByteBuffer getFromBucketV0(ByteBuffer buffer, int offset)
- {
- int prefixPosition;
- if (offset == 0) {
- final int length = VByte.readInt(buffer);
- final ByteBuffer firstValue = buffer.asReadOnlyBuffer();
- firstValue.limit(firstValue.position() + length);
- return firstValue;
- } else {
- final int firstLength = VByte.readInt(buffer);
- prefixPosition = buffer.position();
- buffer.position(buffer.position() + firstLength);
- }
- 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);
- } else {
- // we've reached our destination
- fragmentLength = VByte.readInt(buffer);
- fragmentPosition = buffer.position();
- break;
- }
- } while (true);
- final int valueLength = prefixLength + fragmentLength;
- ByteBuffer value = ByteBuffer.allocate(valueLength);
- for (int i = 0; i < valueLength; i++) {
- if (i < prefixLength) {
- value.put(buffer.get(prefixPosition + i));
- } else {
- value.put(buffer.get(fragmentPosition + i - prefixLength));
- }
- }
- value.flip();
- return value;
- }
-
-
- /**
- * Read an entire bucket from a {@link ByteBuffer}, returning an array of reconstructed value bytes.
- * <p>
- * This method modifies the position of the buffer.
- */
- private static ByteBuffer[] readBucketV0(ByteBuffer bucket, int numValues)
- {
- final int length = VByte.readInt(bucket);
- final byte[] prefixBytes = new byte[length];
- bucket.get(prefixBytes, 0, length);
- final ByteBuffer[] bucketBuffers = new ByteBuffer[numValues];
- bucketBuffers[0] = ByteBuffer.wrap(prefixBytes);
- int pos = 1;
- while (pos < numValues) {
- final int prefixLength = VByte.readInt(bucket);
- final int fragmentLength = VByte.readInt(bucket);
- final byte[] fragment = new byte[fragmentLength];
- bucket.get(fragment, 0, fragmentLength);
- final ByteBuffer value = ByteBuffer.allocate(prefixLength + fragmentLength);
- value.put(prefixBytes, 0, prefixLength);
- value.put(fragment);
- value.flip();
- bucketBuffers[pos++] = value;
- }
- return bucketBuffers;
- }
-
- /**
- * 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 previous value of the bucket.
- * <p>
- * 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
- * <p>
- * this method modifies the position of {@link #buffer}
- */
- private int findValueInBucketV1(
- ByteBuffer value,
+ private int findValueInBucket(
+ int[] value,
int currBucketFirstValueIndex,
int bucketSize,
int sharedPrefixLength
@@ -570,7 +411,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
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);
+ buffer.position(buffer.position() + (skip * Integer.BYTES));
insertionPoint++;
} else if (prefixLength < sharedPrefixLength) {
// bucket value prefix is smaller, that means the value we are looking for sorts ahead of it
@@ -578,13 +419,13 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
} 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.remaining() - prefixLength);
+ final int common = Math.min(fragmentLength, value.length - prefixLength);
int fragmentComparison = 0;
boolean shortCircuit = false;
for (int i = 0; i < common; i++) {
- fragmentComparison = StringUtils.compareUtf8UsingJavaStringOrdering(
- buffer.get(buffer.position() + i),
- value.get(prefixLength + i)
+ fragmentComparison = Integer.compare(
+ buffer.getInt(buffer.position() + (i * Integer.BYTES)),
+ value[prefixLength + i]
);
if (fragmentComparison != 0) {
sharedPrefixLength = prefixLength + i;
@@ -593,7 +434,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
}
}
if (fragmentComparison == 0) {
- fragmentComparison = Integer.compare(prefixLength + fragmentLength, value.remaining());
+ fragmentComparison = Integer.compare(prefixLength + fragmentLength, value.length);
}
if (fragmentComparison == 0) {
@@ -603,7 +444,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
if (!shortCircuit) {
sharedPrefixLength = prefixLength + common;
}
- buffer.position(buffer.position() + fragmentLength);
+ buffer.position(buffer.position() + (fragmentLength * Integer.BYTES));
insertionPoint++;
} else {
break;
@@ -614,15 +455,21 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
return -(currBucketFirstValueIndex + adjustIndex) + (~insertionPoint);
}
- private ByteBuffer getFromBucketV1(ByteBuffer buffer, int offset)
+ /**
+ * Get a value from a bucket at a relative position.
+ *
+ * This method modifies the position of the buffer.
+ */
+ int[] getFromBucket(ByteBuffer buffer, int offset)
{
// first value is written whole
final int length = VByte.readInt(buffer);
if (offset == 0) {
- // return first value directly from underlying buffer since it is stored whole
- final ByteBuffer value = buffer.asReadOnlyBuffer();
- value.limit(value.position() + length);
- return value;
+ final int[] firstValue = new int[length];
+ for (int i = 0; i < length; i++) {
+ firstValue[i] = buffer.getInt();
+ }
+ return firstValue;
}
int pos = 0;
int prefixLength;
@@ -630,7 +477,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
unwindPrefixLength[pos] = 0;
unwindBufferPosition[pos] = buffer.position();
- buffer.position(buffer.position() + length);
+ buffer.position(buffer.position() + (length * Integer.BYTES));
do {
prefixLength = VByte.readInt(buffer);
if (++pos < offset) {
@@ -638,22 +485,26 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
final int skipLength = VByte.readInt(buffer);
unwindPrefixLength[pos] = prefixLength;
unwindBufferPosition[pos] = buffer.position();
- buffer.position(buffer.position() + skipLength);
+ 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 from the underlying buffer
- final ByteBuffer value = buffer.asReadOnlyBuffer();
- value.limit(value.position() + fragmentLength);
+ // 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 byte[] valueBytes = new byte[valueLength];
- buffer.get(valueBytes, 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
@@ -661,55 +512,47 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
continue;
}
buffer.position(unwindBufferPosition[pos]);
- buffer.get(valueBytes, unwindPrefixLength[pos], i - unwindPrefixLength[pos]);
+ final int prevLength = unwindPrefixLength[pos];
+ for (int fragmentOffset = 0; fragmentOffset < i - prevLength; fragmentOffset++) {
+ value[prevLength + fragmentOffset] = buffer.getInt();
+ }
+
i = unwindPrefixLength[pos];
}
- return ByteBuffer.wrap(valueBytes);
+ return value;
}
+
/**
* Read an entire bucket from a {@link ByteBuffer}, returning an array of reconstructed value bytes.
- * <p>
+ *
* This method modifies the position of the buffer.
*/
- private ByteBuffer[] readBucketV1(ByteBuffer bucket, int numValues)
+ private static int[][] readBucket(ByteBuffer bucket, int numValues)
{
- final ByteBuffer[] bucketBuffers = new ByteBuffer[numValues];
+ final int[][] bucketValues = new int[numValues][];
// first value is written whole
final int length = VByte.readInt(bucket);
- byte[] prefixBytes = new byte[length];
- bucket.get(prefixBytes, 0, length);
- bucketBuffers[0] = ByteBuffer.wrap(prefixBytes);
+ int[] prefix = new int[length];
+ for (int i = 0; i < length; i++) {
+ prefix[i] = bucket.getInt();
+ }
+ bucketValues[0] = prefix;
int pos = 1;
while (pos < numValues) {
final int prefixLength = VByte.readInt(bucket);
final int fragmentLength = VByte.readInt(bucket);
- byte[] nextValueBytes = new byte[prefixLength + fragmentLength];
- System.arraycopy(prefixBytes, 0, nextValueBytes, 0, prefixLength);
- bucket.get(nextValueBytes, prefixLength, fragmentLength);
- final ByteBuffer value = ByteBuffer.wrap(nextValueBytes);
- prefixBytes = nextValueBytes;
- bucketBuffers[pos++] = value;
+ 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();
+ }
+ prefix = value;
+ bucketValues[pos++] = value;
}
- return bucketBuffers;
- }
-
- @FunctionalInterface
- interface GetBucketValue
- {
- ByteBuffer get(ByteBuffer buffer, int offset);
- }
-
- @FunctionalInterface
- interface ReadBucket
- {
- ByteBuffer[] readBucket(ByteBuffer buffer, int bucketSize);
- }
-
- @FunctionalInterface
- interface FindInBucket
- {
- int find(ByteBuffer value, int currBucketFirstValueIndex, int bucketSize, int sharedPrefixLength);
+ 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 0000000000..00cc2079fc
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java
@@ -0,0 +1,409 @@
+/*
+ * 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<int[]>
+{
+ private static final int MAX_LOG_BUFFER_SIZE = 26;
+
+ public static final Comparator<int[]> 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 currentBucketSize = Ints.checkedCast(endOffset - startOffset);
+ if (currentBucketSize == 0) {
+ return null;
+ }
+ final ByteBuffer bucketBuffer = ByteBuffer.allocate(currentBucketSize).order(byteOrder);
+ valuesOut.readFully(startOffset, bucketBuffer);
+ bucketBuffer.clear();
+ return 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[] prev = null;
+ while (written < numValues) {
+ int[] next = values[written];
+ if (written == 0) {
+ prev = next;
+ // the first value in the bucket is written completely as it is
+ int rem = writeValue(buffer, prev);
+ // 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 < 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);
+ // wasn't enough room, bail out
+ if (rem < 0) {
+ return rem;
+ }
+ VByte.writeInt(buffer, prefixLength);
+ rem = writeValue(buffer, suffix);
+ prev = next;
+ // 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;
+ }
+
+ /**
+ * 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/incremental/IncrementalIndexSchema.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java
index 2931eaeb2b..d127f3d89b 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 @@ import javax.annotation.Nullable;
*/
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/StringFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java
similarity index 55%
copy from processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java
copy to processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java
index c9f10a269c..1ae465d25c 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java
@@ -19,7 +19,7 @@
package org.apache.druid.segment.nested;
-import org.apache.druid.common.config.NullHandling;
+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;
@@ -27,12 +27,10 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.IOException;
import java.nio.channels.WritableByteChannel;
-/**
- * Literal field writer for string type nested columns of {@link NestedDataColumnSerializer}
- */
-public final class StringFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<String>
+public class ArrayFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<int[]>
{
- public StringFieldColumnWriter(
+
+ protected ArrayFieldColumnWriter(
String columnName,
String fieldName,
SegmentWriteOutMedium segmentWriteOutMedium,
@@ -44,18 +42,38 @@ public final class StringFieldColumnWriter extends GlobalDictionaryEncodedFieldC
}
@Override
- String processValue(Object value)
+ int[] processValue(int row, Object value)
{
- if (value == null) {
- return null;
+ if (value instanceof Object[]) {
+ Object[] array = (Object[]) value;
+ final int[] globalIds = new int[array.length];
+ for (int i = 0; i < array.length; i++) {
+ if (array[i] == null) {
+ globalIds[i] = 0;
+ } else if (array[i] instanceof String) {
+ globalIds[i] = globalDictionaryIdLookup.lookupString((String) array[i]);
+ } else if (array[i] instanceof Long) {
+ globalIds[i] = globalDictionaryIdLookup.lookupLong((Long) array[i]);
+ } else if (array[i] instanceof Double) {
+ globalIds[i] = globalDictionaryIdLookup.lookupDouble((Double) array[i]);
+ } else {
+ globalIds[i] = -1;
+ }
+ Preconditions.checkArgument(globalIds[i] >= 0, "unknown global id [%s] for value [%s]", globalIds[i], array[i]);
+ arrayElements.computeIfAbsent(
+ globalIds[i],
+ (id) -> indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap()
+ ).add(row);
+ }
+ return globalIds;
}
- return NullHandling.emptyToNullIfNeeded(String.valueOf(value));
+ return null;
}
@Override
- int lookupGlobalId(String value)
+ int lookupGlobalId(int[] value)
{
- return globalDictionaryIdLookup.lookupString(value);
+ return globalDictionaryIdLookup.lookupArray(value);
}
@Override
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 77919c4bff..1d4a683384 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 @@ package org.apache.druid.segment.nested;
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.Closer;
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.ColumnHolder;
import org.apache.druid.segment.column.ColumnIndexSupplier;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.DictionaryEncodedColumn;
-import org.apache.druid.segment.column.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.CompressedVSizeColumnarIntsSupplier;
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.ReadableOffset;
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.io.IOException;
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;
@@ -89,16 +94,18 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
private final ImmutableBitmap nullValues;
private final GenericIndexed<String> fields;
- private final NestedLiteralTypeInfo fieldInfo;
+ private final NestedFieldTypeInfo fieldInfo;
private final Supplier<TStringDictionary> stringDictionarySupplier;
private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
+ private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
+
private final SmooshedFileMapper fileMapper;
private final String rootFieldPath;
- private final ConcurrentHashMap<String, ColumnHolder> columns = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap<Integer, ColumnHolder> columns = new ConcurrentHashMap<>();
private static final ObjectStrategy<Object> STRATEGY = NestedDataComplexTypeSerde.INSTANCE.getObjectStrategy();
@@ -108,10 +115,11 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
ImmutableBitmap nullValues,
GenericIndexed<String> fields,
- NestedLiteralTypeInfo fieldInfo,
+ NestedFieldTypeInfo fieldInfo,
Supplier<TStringDictionary> stringDictionary,
Supplier<FixedIndexed<Long>> longDictionarySupplier,
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
+ Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
SmooshedFileMapper fileMapper,
String rootFieldPath
)
@@ -123,6 +131,7 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
this.stringDictionarySupplier = stringDictionary;
this.longDictionarySupplier = longDictionarySupplier;
this.doubleDictionarySupplier = doubleDictionarySupplier;
+ this.arrayDictionarySupplier = arrayDictionarySupplier;
this.fileMapper = fileMapper;
this.closer = Closer.create();
this.compressedRawColumnSupplier = compressedRawColumnSupplier;
@@ -150,7 +159,7 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
return fieldParts;
}
- public NestedLiteralTypeInfo getFieldInfo()
+ public NestedFieldTypeInfo getFieldInfo()
{
return fieldInfo;
}
@@ -170,6 +179,60 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
return doubleDictionarySupplier.get();
}
+ public FrontCodedIntArrayIndexed getArrayDictionary()
+ {
+ return arrayDictionarySupplier.get();
+ }
+
+ public Iterable<Object[]> getArraysIterable()
+ {
+ Iterable<Object[]> arrays = () -> {
+ final TStringDictionary stringDictionary = stringDictionarySupplier.get();
+ final FixedIndexed<Long> longDictionary = longDictionarySupplier.get();
+ final FixedIndexed<Double> doubleDictionary = doubleDictionarySupplier.get();
+
+ return new Iterator<Object[]>()
+ {
+ final Iterator<int[]> 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,17 +390,59 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
}
@Override
- public DimensionSelector makeDimensionSelector(List<NestedPathPart> path, ReadableOffset readableOffset, ExtractionFn fn)
+ public DimensionSelector makeDimensionSelector(
+ List<NestedPathPart> path,
+ ReadableOffset readableOffset,
+ ExtractionFn fn
+ )
{
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);
- } 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));
+ 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, arrayFieldIndex).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
@@ -346,12 +451,81 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
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);
- } 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));
+ 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, arrayFieldIndex).getColumn();
+ ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset);
+ return new ColumnValueSelector<Object>()
+ {
+ @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
@@ -362,9 +536,9 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
{
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);
@@ -376,13 +550,66 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
{
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);
- } 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));
+ 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, arrayFieldIndex).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
@@ -390,15 +617,132 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
{
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);
- } 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));
+ 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, arrayFieldIndex).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
@@ -410,14 +754,16 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
if (index < 0) {
return null;
}
- return NestedLiteralTypeInfo.convertToSet(fieldInfo.getTypes(index).getByteValue());
+ return NestedFieldTypeInfo.convertToSet(fieldInfo.getTypes(index).getByteValue());
}
@Nullable
@Override
public ColumnHolder getColumnHolder(List<NestedPathPart> path)
{
- return getColumnHolder(getField(path));
+ final String field = getField(path);
+ final int fieldIndex = fields.indexOf(field);
+ return getColumnHolder(field, fieldIndex);
}
@Nullable
@@ -425,35 +771,44 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
public ColumnIndexSupplier getColumnIndexSupplier(List<NestedPathPart> path)
{
final String field = getField(path);
- if (fields.indexOf(field) < 0) {
- return null;
+ 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 getColumnHolder(field).getIndexSupplier();
+ return null;
}
@Override
public boolean isNumeric(List<NestedPathPart> 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 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) {
@@ -491,10 +846,16 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
int doublesLength = bb.getInt();
dataBuffer.position(dataBuffer.position() + Integer.BYTES + Integer.BYTES);
int pos = dataBuffer.position();
- final Supplier<ColumnarLongs> longs = longsLength > 0 ? CompressedColumnarLongsSupplier.fromByteBuffer(dataBuffer, metadata.getByteOrder()) : () -> null;
+ final Supplier<ColumnarLongs> longs = longsLength > 0 ? CompressedColumnarLongsSupplier.fromByteBuffer(
+ dataBuffer,
+ metadata.getByteOrder()
+ ) : () -> null;
dataBuffer.position(pos + longsLength);
pos = dataBuffer.position();
- final Supplier<ColumnarDoubles> doubles = doublesLength > 0 ? CompressedColumnarDoublesSuppliers.fromByteBuffer(dataBuffer, metadata.getByteOrder()) : () -> null;
+ final Supplier<ColumnarDoubles> doubles = doublesLength > 0 ? CompressedColumnarDoublesSuppliers.fromByteBuffer(
+ dataBuffer,
+ metadata.getByteOrder()
+ ) : () -> null;
dataBuffer.position(pos + doublesLength);
final WritableSupplier<ColumnarInts> ints;
if (version == DictionaryEncodedColumnPartSerde.VERSION.COMPRESSED) {
@@ -503,17 +864,35 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
ints = VSizeColumnarInts.readFromByteBuffer(dataBuffer);
}
ColumnType theType = types.getSingleType();
- columnBuilder.setType(theType == null ? ValueType.STRING : theType.getType());
+ columnBuilder.setType(theType == null ? ColumnType.STRING : theType);
GenericIndexed<ImmutableBitmap> rBitmaps = GenericIndexed.read(
dataBuffer,
metadata.getBitmapSerdeFactory().getObjectStrategy(),
columnBuilder.getFileMapper()
);
+ final Supplier<FixedIndexed<Integer>> arrayElementDictionarySupplier;
+ final GenericIndexed<ImmutableBitmap> 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<DictionaryEncodedColumn<?>> columnSupplier = () -> {
FixedIndexed<Integer> localDict = localDictionarySupplier.get();
- return closer.register(new NestedFieldLiteralDictionaryEncodedColumn(
+ return closer.register(new NestedFieldDictionaryEncodedColumn(
types,
longs.get(),
doubles.get(),
@@ -521,6 +900,7 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
stringDictionarySupplier.get(),
longDictionarySupplier.get(),
doubleDictionarySupplier.get(),
+ arrayDictionarySupplier != null ? arrayDictionarySupplier.get() : null,
localDict,
hasNull
? rBitmaps.get(0)
@@ -531,14 +911,16 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
.setHasNulls(hasNull)
.setDictionaryEncodedColumnSupplier(columnSupplier);
columnBuilder.setIndexSupplier(
- new NestedFieldLiteralColumnIndexSupplier(
+ new NestedFieldColumnIndexSupplier(
types,
metadata.getBitmapSerdeFactory().getBitmapFactory(),
rBitmaps,
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 def741a107..0c308ff6b4 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 @@ package org.apache.druid.segment.nested;
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<T>
protected final GlobalDictionaryIdLookup globalDictionaryIdLookup;
protected final LocalDimensionDictionary localDictionary = new LocalDimensionDictionary();
+ protected final Int2ObjectRBTreeMap<MutableBitmap> 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 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
/**
* 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 abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
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 abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
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 abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
final SegmentWriteOutMedium tmpWriteoutMedium = segmentWriteOutMedium.makeChildWriteOutMedium();
final FixedIndexedIntWriter sortedDictionaryWriter = new FixedIndexedIntWriter(tmpWriteoutMedium, true);
sortedDictionaryWriter.open();
+ final FixedIndexedIntWriter arrayElementDictionaryWriter = new FixedIndexedIntWriter(tmpWriteoutMedium, true);
+ arrayElementDictionaryWriter.open();
GenericIndexedWriter<ImmutableBitmap> bitmapIndexWriter = new GenericIndexedWriter<>(
tmpWriteoutMedium,
columnName,
@@ -191,6 +197,14 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
);
bitmapIndexWriter.open();
bitmapIndexWriter.setObjectsNotSorted();
+ GenericIndexedWriter<ImmutableBitmap> 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 abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
bitmaps[index] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap();
}
+ for (Int2ObjectMap.Entry<MutableBitmap> 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;
@@ -233,9 +254,16 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
@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();
}
@@ -247,6 +275,10 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
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/GlobalDictionaryIdLookup.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java
index 821f3ecc26..2675b0392c 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.Double2IntLinkedOpenHashMap;
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<int[]> arrayLookup;
+
private int dictionarySize;
public GlobalDictionaryIdLookup()
@@ -51,6 +55,8 @@ public class GlobalDictionaryIdLookup
longLookup.defaultReturnValue(-1);
this.doubleLookup = new Double2IntLinkedOpenHashMap();
doubleLookup.defaultReturnValue(-1);
+ this.arrayLookup = new Object2IntAVLTreeMap<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+ this.arrayLookup.defaultReturnValue(-1);
}
public void addString(@Nullable String value)
@@ -99,4 +105,18 @@ public class GlobalDictionaryIdLookup
}
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 7cb775cc54..70040d7b52 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<String> sortedStrings;
private final Indexed<Long> sortedLongs;
private final Indexed<Double> sortedDoubles;
+ private final Iterable<Object[]> sortedArrays;
+ private final int arrayCount;
public GlobalDictionarySortedCollector(
Indexed<String> sortedStrings,
Indexed<Long> sortedLongs,
- Indexed<Double> sortedDoubles
+ Indexed<Double> sortedDoubles,
+ Iterable<Object[]> sortedArrays,
+ int arrayCount
)
{
this.sortedStrings = sortedStrings;
this.sortedLongs = sortedLongs;
this.sortedDoubles = sortedDoubles;
+ this.sortedArrays = sortedArrays;
+ this.arrayCount = arrayCount;
}
public Indexed<String> getSortedStrings()
@@ -59,8 +65,44 @@ public class GlobalDictionarySortedCollector
return sortedDoubles;
}
- public int size()
+ public Iterable<Object[]> 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;
+ }
+
+ 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/GlobalDimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java
index 4f22577b41..48b06d4e45 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<String> stringDictionary;
private final ComparatorDimensionDictionary<Long> longDictionary;
private final ComparatorDimensionDictionary<Double> doubleDictionary;
+ private final Set<Object[]> stringArrays;
+ private final Set<Object[]> longArrays;
+ private final Set<Object[]> doubleArrays;
+
+ private int arrayBytesSizeEstimate;
public GlobalDimensionDictionary()
{
- this.stringDictionary = new ComparatorDimensionDictionary<String>(GenericIndexed.STRING_STRATEGY) {
+ this.stringDictionary = new ComparatorDimensionDictionary<String>(GenericIndexed.STRING_STRATEGY)
+ {
@Override
public long estimateSizeOfValue(String value)
{
return StructuredDataProcessor.estimateStringSize(value);
}
};
- this.longDictionary = new ComparatorDimensionDictionary<Long>(ColumnType.LONG.getNullableStrategy()) {
+ this.longDictionary = new ComparatorDimensionDictionary<Long>(ColumnType.LONG.getNullableStrategy())
+ {
@Override
public long estimateSizeOfValue(Long value)
{
return StructuredDataProcessor.getLongObjectEstimateSize();
}
};
- this.doubleDictionary = new ComparatorDimensionDictionary<Double>(ColumnType.DOUBLE.getNullableStrategy()) {
+ this.doubleDictionary = new ComparatorDimensionDictionary<Double>(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 class GlobalDimensionDictionary
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 class GlobalDimensionDictionary
// 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<Object[]> sortedArrays = new TreeSet<>(new Comparator<Object[]>()
+ {
+ @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[] globalIds = new int[array.length];
+ for (int i = 0; i < array.length; i++) {
+ if (array[i] == null) {
+ globalIds[i] = 0;
+ } else if (array[i] instanceof String) {
+ // offset by 1 because nulls are ignored by the indexer, but always global id 0
+ globalIds[i] = 1 + strings.indexOf((String) array[i]);
+ } else if (array[i] instanceof Long) {
+ globalIds[i] = longs.indexOf((Long) array[i]) + adjustLongs;
+ } else if (array[i] instanceof Double) {
+ globalIds[i] = doubles.indexOf((Double) array[i]) + adjustDoubles;
+ } else {
+ globalIds[i] = -1;
+ }
+ Preconditions.checkArgument(
+ globalIds[i] >= 0,
+ "unknown global id [%s] for value [%s]",
+ globalIds[i],
+ array[i]
+ );
+ }
+ return globalIds;
+ }
+ });
+ 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 97b447dadd..21a5467818 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 class LocalDimensionDictionary
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 f73b72209c..020dc10ad0 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,8 @@ import com.google.common.collect.Maps;
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.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;
@@ -46,17 +48,20 @@ import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSerializer
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;
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;
@@ -67,6 +72,7 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
public static final String STRING_DICTIONARY_FILE_NAME = "__stringDictionary";
public static final String LONG_DICTIONARY_FILE_NAME = "__longDictionary";
public static final String DOUBLE_DICTIONARY_FILE_NAME = "__doubleDictionary";
+ public static final String ARRAY_DICTIONARY_FILE_NAME = "__arrayDictionary";
public static final String RAW_FILE_NAME = "__raw";
public static final String NULL_BITMAP_FILE_NAME = "__nullIndex";
@@ -81,14 +87,14 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
private final StructuredDataProcessor fieldProcessor = new StructuredDataProcessor()
{
@Override
- public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+ public ProcessedValue<?> processField(ArrayList<NestedPathPart> 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 {
@@ -96,24 +102,51 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
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], unhandled value", fieldPath);
}
}
- return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+ return ProcessedValue.NULL_LITERAL;
+ }
+
+ @Nullable
+ @Override
+ public ProcessedValue<?> processArrayField(
+ ArrayList<NestedPathPart> fieldPath,
+ @Nullable List<?> array
+ )
+ {
+ final ExprEval<?> eval = ExprEval.bestEffortArray(array);
+ 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 ProcessedValue.NULL_LITERAL;
+ }
+ catch (IOException e) {
+ throw new RE(e, "Failed to write field [%s] value [%s]", fieldPath, array);
+ }
+ }
+ }
+ return null;
}
};
private byte[] metadataBytes;
private GlobalDictionaryIdLookup globalDictionaryIdLookup;
- private SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> fields;
+ private SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> fields;
private GenericIndexedWriter<String> fieldsWriter;
- private NestedLiteralTypeInfo.Writer fieldsInfoWriter;
+ private NestedFieldTypeInfo.Writer fieldsInfoWriter;
private DictionaryWriter<String> dictionaryWriter;
private FixedIndexedWriter<Long> longDictionaryWriter;
private FixedIndexedWriter<Double> doubleDictionaryWriter;
+ private FrontCodedIntArrayIndexedWriter arrayDictionaryWriter;
private CompressedVariableSizedBlobColumnSerializer rawWriter;
private ByteBufferWriter<ImmutableBitmap> nullBitmapWriter;
private MutableBitmap nullRowsBitmap;
@@ -121,6 +154,8 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
private int rowCount = 0;
private boolean closedForWrite = false;
+ private boolean dictionarySerialized = false;
+
public NestedDataColumnSerializer(
String name,
IndexSpec indexSpec,
@@ -136,13 +171,18 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
this.globalDictionaryIdLookup = new GlobalDictionaryIdLookup();
}
+ public GlobalDictionaryIdLookup getGlobalLookup()
+ {
+ return globalDictionaryIdLookup;
+ }
+
@Override
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(
@@ -170,6 +210,13 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
);
doubleDictionaryWriter.open();
+ arrayDictionaryWriter = new FrontCodedIntArrayIndexedWriter(
+ segmentWriteOutMedium,
+ ByteOrder.nativeOrder(),
+ 4
+ );
+ arrayDictionaryWriter.open();
+
rawWriter = new CompressedVariableSizedBlobColumnSerializer(
getInternalFileName(name, RAW_FILE_NAME),
segmentWriteOutMedium,
@@ -186,12 +233,12 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap();
}
- public void serializeFields(SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> fields) throws IOException
+ public void serializeFields(SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> fields) throws IOException
{
this.fields = fields;
this.fieldWriters = Maps.newHashMapWithExpectedSize(fields.size());
int ctr = 0;
- for (Map.Entry<String, NestedLiteralTypeInfo.MutableTypeSet> field : fields.entrySet()) {
+ for (Map.Entry<String, NestedFieldTypeInfo.MutableTypeSet> field : fields.entrySet()) {
final String fieldName = field.getKey();
final String fieldFileName = NESTED_FIELD_PREFIX + ctr++;
fieldsWriter.write(fieldName);
@@ -215,7 +262,7 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
indexSpec,
globalDictionaryIdLookup
);
- } else {
+ } else if (Types.is(type, ValueType.DOUBLE)) {
writer = new DoubleFieldColumnWriter(
name,
fieldFileName,
@@ -223,9 +270,19 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
indexSpec,
globalDictionaryIdLookup
);
+ } else if (Types.is(type, ValueType.ARRAY)) {
+ writer = new ArrayFieldColumnWriter(
+ name,
+ fieldFileName,
+ segmentWriteOutMedium,
+ indexSpec,
+ globalDictionaryIdLookup
+ );
+ } else {
+ throw new ISE("Invalid field type [%s], how did this happen?", type);
}
} else {
- writer = new VariantLiteralFieldColumnWriter(
+ writer = new VariantFieldColumnWriter(
name,
fieldFileName,
segmentWriteOutMedium,
@@ -238,11 +295,21 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
}
}
- public void serializeStringDictionary(Iterable<String> dictionaryValues) throws IOException
+ public void serializeDictionaries(
+ Iterable<String> strings,
+ Iterable<Long> longs,
+ Iterable<Double> doubles,
+ Iterable<int[]> arrays
+ ) throws IOException
{
+ 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;
@@ -251,33 +318,40 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
dictionaryWriter.write(value);
globalDictionaryIdLookup.addString(value);
}
- }
+ dictionarySerialized = true;
- public void serializeLongDictionary(Iterable<Long> dictionaryValues) throws IOException
- {
- for (Long value : dictionaryValues) {
+ for (Long value : longs) {
if (value == null) {
continue;
}
longDictionaryWriter.write(value);
globalDictionaryIdLookup.addLong(value);
}
- }
- public void serializeDoubleDictionary(Iterable<Double> dictionaryValues) throws IOException
- {
- for (Double value : dictionaryValues) {
+ for (Double value : doubles) {
if (value == null) {
continue;
}
doubleDictionaryWriter.write(value);
globalDictionaryIdLookup.addDouble(value);
}
+
+ for (int[] value : arrays) {
+ if (value == null) {
+ continue;
+ }
+ arrayDictionaryWriter.write(value);
+ globalDictionaryIdLookup.addArray(value);
+ }
+ dictionarySerialized = true;
}
@Override
public void serialize(ColumnValueSelector<? extends StructuredData> selector) throws IOException
{
+ if (!dictionarySerialized) {
+ 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 +404,8 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
{
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 +414,7 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
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);
@@ -355,7 +430,7 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
channel.close();
}
- for (Map.Entry<String, NestedLiteralTypeInfo.MutableTypeSet> field : fields.entrySet()) {
+ for (Map.Entry<String, NestedFieldTypeInfo.MutableTypeSet> 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 507a531f4b..15dc2d888f 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.CompressedVariableSizedBlobColumnSupplier;
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,18 +73,20 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
{
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;
final GenericIndexed<String> fields;
- final NestedLiteralTypeInfo fieldInfo;
+ final NestedFieldTypeInfo fieldInfo;
final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
final ImmutableBitmap nullValues;
final GenericIndexed<ByteBuffer> stringDictionary;
final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
final Supplier<FixedIndexed<Long>> longDictionarySupplier;
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
+ final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
+
ColumnType simpleType;
metadata = jsonMapper.readValue(
@@ -91,7 +94,7 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
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
@@ -99,7 +102,7 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
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 class NestedDataColumnSupplier implements Supplier<ComplexColumn>
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 class NestedDataColumnSupplier implements Supplier<ComplexColumn>
frontCodedStringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
+ arrayDictionarySupplier,
columnConfig,
mapper,
simpleType
@@ -210,13 +227,14 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
private final byte version;
private final NestedDataColumnMetadata metadata;
private final GenericIndexed<String> fields;
- private final NestedLiteralTypeInfo fieldInfo;
+ private final NestedFieldTypeInfo fieldInfo;
private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
private final ImmutableBitmap nullValues;
private final GenericIndexed<ByteBuffer> stringDictionary;
private final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
+ private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
private final ColumnConfig columnConfig;
private final SmooshedFileMapper fileMapper;
@@ -227,13 +245,14 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
byte version,
NestedDataColumnMetadata metadata,
GenericIndexed<String> fields,
- NestedLiteralTypeInfo fieldInfo,
+ NestedFieldTypeInfo fieldInfo,
CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
ImmutableBitmap nullValues,
GenericIndexed<ByteBuffer> stringDictionary,
Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier,
Supplier<FixedIndexed<Long>> longDictionarySupplier,
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
+ Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
ColumnConfig columnConfig,
SmooshedFileMapper fileMapper,
@Nullable ColumnType simpleType
@@ -249,6 +268,7 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
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 class NestedDataColumnSupplier implements Supplier<ComplexColumn>
{
if (version == 0x03) {
return makeV3();
+ } else if (version == 0x04) {
+ return makeV4();
}
- return makeV4();
+ return makeV5();
}
@Nullable
@@ -329,6 +351,38 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
);
}
+ 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 8c2cc91963..eadfca84ce 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 final class NestedDataColumnV3<TStringDictionary extends Indexed<ByteBuff
CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
ImmutableBitmap nullValues,
GenericIndexed<String> fields,
- NestedLiteralTypeInfo fieldInfo,
+ NestedFieldTypeInfo fieldInfo,
Supplier<TStringDictionary> stringDictionary,
Supplier<FixedIndexed<Long>> longDictionarySupplier,
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
@@ -58,6 +58,7 @@ public final class NestedDataColumnV3<TStringDictionary extends Indexed<ByteBuff
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 96c5f56e2b..26631ee7fd 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 final class NestedDataColumnV4<TStringDictionary extends Indexed<ByteBuff
CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
ImmutableBitmap nullValues,
GenericIndexed<String> fields,
- NestedLiteralTypeInfo fieldInfo,
+ NestedFieldTypeInfo fieldInfo,
Supplier<TStringDictionary> stringDictionary,
Supplier<FixedIndexed<Long>> longDictionarySupplier,
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
@@ -57,6 +57,7 @@ public final class NestedDataColumnV4<TStringDictionary extends Indexed<ByteBuff
stringDictionary,
longDictionarySupplier,
doubleDictionarySupplier,
+ null,
fileMapper,
NestedPathFinder.JSON_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/NestedDataColumnV5.java
similarity index 89%
copy from processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java
copy to processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java
index 96c5f56e2b..7559226181 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java
@@ -25,25 +25,27 @@ 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 final class NestedDataColumnV4<TStringDictionary extends Indexed<ByteBuffer>>
+public class NestedDataColumnV5<TStringDictionary extends Indexed<ByteBuffer>>
extends CompressedNestedDataComplexColumn<TStringDictionary>
{
- public NestedDataColumnV4(
+ public NestedDataColumnV5(
NestedDataColumnMetadata metadata,
ColumnConfig columnConfig,
CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
ImmutableBitmap nullValues,
GenericIndexed<String> fields,
- NestedLiteralTypeInfo fieldInfo,
+ NestedFieldTypeInfo fieldInfo,
Supplier<TStringDictionary> stringDictionary,
Supplier<FixedIndexed<Long>> longDictionarySupplier,
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
+ Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
SmooshedFileMapper fileMapper
)
{
@@ -57,6 +59,7 @@ public final class NestedDataColumnV4<TStringDictionary extends Indexed<ByteBuff
stringDictionary,
longDictionarySupplier,
doubleDictionarySupplier,
+ arrayDictionarySupplier,
fileMapper,
NestedPathFinder.JSON_PATH_ROOT
);
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 97%
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 22ba59835e..b4e59d486b 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.NoSuchElementException;
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<TStringDictionary extends Indexed<ByteBuffer>>
+public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<ByteBuffer>>
implements ColumnIndexSupplier
{
@Nullable
@@ -83,17 +83,26 @@ public class NestedFieldLiteralColumnIndexSupplier<TStringDictionary extends Ind
private final Supplier<FixedIndexed<Long>> globalLongDictionarySupplier;
private final Supplier<FixedIndexed<Double>> globalDoubleDictionarySupplier;
+ @SuppressWarnings({"FieldCanBeLocal", "unused"})
+ @Nullable
+ private final GenericIndexed<ImmutableBitmap> arrayElementBitmaps;
+ @SuppressWarnings({"FieldCanBeLocal", "unused"})
+ @Nullable
+ private final Supplier<FixedIndexed<Integer>> arrayElementDictionarySupplier;
+
private final int adjustLongId;
private final int adjustDoubleId;
- public NestedFieldLiteralColumnIndexSupplier(
- NestedLiteralTypeInfo.TypeSet types,
+ public NestedFieldColumnIndexSupplier(
+ NestedFieldTypeInfo.TypeSet types,
BitmapFactory bitmapFactory,
GenericIndexed<ImmutableBitmap> bitmaps,
Supplier<FixedIndexed<Integer>> localDictionarySupplier,
Supplier<TStringDictionary> globalStringDictionarySupplier,
Supplier<FixedIndexed<Long>> globalLongDictionarySupplier,
- Supplier<FixedIndexed<Double>> globalDoubleDictionarySupplier
+ Supplier<FixedIndexed<Double>> globalDoubleDictionarySupplier,
+ @Nullable Supplier<FixedIndexed<Integer>> arrayElementDictionarySupplier,
+ @Nullable GenericIndexed<ImmutableBitmap> arrayElementBitmaps
)
{
this.singleType = types.getSingleType();
@@ -103,6 +112,8 @@ public class NestedFieldLiteralColumnIndexSupplier<TStringDictionary extends Ind
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();
}
@@ -320,7 +331,7 @@ public class NestedFieldLiteralColumnIndexSupplier<TStringDictionary extends Ind
@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 85%
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 be942a9dfd..3449d3410a 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
@@ -33,6 +33,7 @@ import org.apache.druid.query.filter.ValueMatcher;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.AbstractDimensionSelector;
import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.DoubleColumnSelector;
import org.apache.druid.segment.IdLookup;
@@ -46,6 +47,7 @@ import org.apache.druid.segment.data.ColumnarDoubles;
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;
@@ -69,10 +71,10 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.BitSet;
-public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends Indexed<ByteBuffer>>
+public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexed<ByteBuffer>>
implements DictionaryEncodedColumn<String>
{
- private final NestedLiteralTypeInfo.TypeSet types;
+ private final NestedFieldTypeInfo.TypeSet types;
@Nullable
private final ColumnType singleType;
private final ColumnarLongs longsColumn;
@@ -82,20 +84,25 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
private final FixedIndexed<Long> globalLongDictionary;
private final FixedIndexed<Double> globalDoubleDictionary;
+ private final FrontCodedIntArrayIndexed globalArrayDictionary;
+
private final FixedIndexed<Integer> dictionary;
private final ImmutableBitmap nullBitmap;
private final int adjustLongId;
private final int adjustDoubleId;
+ private final int adjustArrayId;
+
- public NestedFieldLiteralDictionaryEncodedColumn(
- NestedLiteralTypeInfo.TypeSet types,
+ public NestedFieldDictionaryEncodedColumn(
+ NestedFieldTypeInfo.TypeSet types,
ColumnarLongs longsColumn,
ColumnarDoubles doublesColumn,
ColumnarInts column,
TStringDictionary globalDictionary,
FixedIndexed<Long> globalLongDictionary,
FixedIndexed<Double> globalDoubleDictionary,
+ @Nullable FrontCodedIntArrayIndexed globalArrayDictionary,
FixedIndexed<Integer> dictionary,
ImmutableBitmap nullBitmap
)
@@ -108,10 +115,12 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
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 +224,18 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
}
}
+ 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,
@@ -299,7 +320,10 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
@Override
public boolean isNull()
{
- return dictionary.get(getRowValue()) == 0;
+ if (dictionary.get(getRowValue()) == 0) {
+ return true;
+ }
+ return DimensionHandlerUtils.isNumericNull(getObject());
}
@Override
@@ -332,7 +356,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
- inspector.visit("column", NestedFieldLiteralDictionaryEncodedColumn.this);
+ inspector.visit("column", NestedFieldDictionaryEncodedColumn.this);
}
};
} else {
@@ -373,7 +397,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
- inspector.visit("column", NestedFieldLiteralDictionaryEncodedColumn.this);
+ inspector.visit("column", NestedFieldDictionaryEncodedColumn.this);
}
};
}
@@ -381,7 +405,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
@Override
public Object getObject()
{
- return NestedFieldLiteralDictionaryEncodedColumn.this.lookupName(getRowValue());
+ return NestedFieldDictionaryEncodedColumn.this.lookupName(getRowValue());
}
@Override
@@ -407,7 +431,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
@Override
public String lookupName(int id)
{
- final String value = NestedFieldLiteralDictionaryEncodedColumn.this.lookupName(id);
+ final String value = NestedFieldDictionaryEncodedColumn.this.lookupName(id);
return extractionFn == null ? value : extractionFn.apply(value);
}
@@ -428,7 +452,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
public int lookupId(String name)
{
if (extractionFn == null) {
- return NestedFieldLiteralDictionaryEncodedColumn.this.lookupId(name);
+ return NestedFieldDictionaryEncodedColumn.this.lookupId(name);
}
throw new UnsupportedOperationException("cannot perform lookup when applying an extraction function");
}
@@ -524,7 +548,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
};
}
}
- if (singleType == null) {
+ if (singleType == null || singleType.isArray()) {
return new ColumnValueSelector<Object>()
{
@@ -538,12 +562,18 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
{
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;
}
}
@@ -620,7 +650,10 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
nullMark = nullIterator.next();
}
}
- return nullMark == offsetMark;
+ if (nullMark == offsetMark) {
+ return true;
+ }
+ return DimensionHandlerUtils.isNumericNull(getObject());
}
@Override
@@ -662,7 +695,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
@Override
public String lookupName(final int id)
{
- return NestedFieldLiteralDictionaryEncodedColumn.this.lookupName(id);
+ return NestedFieldDictionaryEncodedColumn.this.lookupName(id);
}
@Nullable
@@ -682,7 +715,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
@Override
public int lookupId(@Nullable String name)
{
- return NestedFieldLiteralDictionaryEncodedColumn.this.lookupId(name);
+ return NestedFieldDictionaryEncodedColumn.this.lookupId(name);
}
}
@@ -698,6 +731,61 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
@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()
@@ -709,7 +797,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
@Override
public String lookupName(int id)
{
- return NestedFieldLiteralDictionaryEncodedColumn.this.lookupName(id);
+ return NestedFieldDictionaryEncodedColumn.this.lookupName(id);
}
}
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedLiteralTypeInfo.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.java
similarity index 68%
rename from processing/src/main/java/org/apache/druid/segment/nested/NestedLiteralTypeInfo.java
rename to processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.java
index 4b4b89df83..1d6d251fab 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/NestedLiteralTypeInfo.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.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,21 +32,28 @@ import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.WritableByteChannel;
+import java.util.Objects;
import java.util.Set;
/**
* Binary serialization for nested field type info, translated into this compact format for storage in segments.
* The index of the type info here is the same as the field index in {@link CompressedNestedDataComplexColumn#fields}
*/
-public class NestedLiteralTypeInfo
+public class NestedFieldTypeInfo
{
private static final byte STRING_MASK = 1;
private static final byte LONG_MASK = 1 << 2;
private static final byte DOUBLE_MASK = 1 << 3;
- public static NestedLiteralTypeInfo read(ByteBuffer buffer, int length)
+ 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 NestedFieldTypeInfo read(ByteBuffer buffer, int length)
{
- NestedLiteralTypeInfo typeInfo = new NestedLiteralTypeInfo(buffer);
+ NestedFieldTypeInfo typeInfo = new NestedFieldTypeInfo(buffer);
buffer.position(buffer.position() + length);
return typeInfo;
}
@@ -53,7 +61,7 @@ public class NestedLiteralTypeInfo
private final ByteBuffer buffer;
private final int startOffset;
- public NestedLiteralTypeInfo(ByteBuffer buffer)
+ public NestedFieldTypeInfo(ByteBuffer buffer)
{
this.buffer = buffer;
this.startOffset = buffer.position();
@@ -79,7 +87,7 @@ public class NestedLiteralTypeInfo
@Nullable
public ColumnType getSingleType()
{
- return NestedLiteralTypeInfo.getSingleType(types);
+ return NestedFieldTypeInfo.getSingleType(types);
}
public byte getByteValue()
@@ -92,6 +100,25 @@ public class NestedLiteralTypeInfo
{
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 class NestedLiteralTypeInfo
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());
}
@@ -135,7 +178,7 @@ public class NestedLiteralTypeInfo
@Nullable
public ColumnType getSingleType()
{
- return NestedLiteralTypeInfo.getSingleType(types);
+ return NestedFieldTypeInfo.getSingleType(types);
}
public boolean isEmpty()
@@ -154,6 +197,25 @@ public class NestedLiteralTypeInfo
{
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 @@ public class NestedLiteralTypeInfo
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 class NestedLiteralTypeInfo
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/StringFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java
index c9f10a269c..5bb2f186b0 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 final class StringFieldColumnWriter extends GlobalDictionaryEncodedFieldC
}
@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/StructuredDataProcessor.java b/processing/src/main/java/org/apache/druid/segment/nested/StructuredDataProcessor.java
index 8549deda44..e517bdc0a0 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
@@ -22,6 +22,7 @@ package org.apache.druid.segment.nested;
import javax.annotation.Nullable;
import java.util.ArrayDeque;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -31,11 +32,31 @@ import java.util.Set;
public abstract class StructuredDataProcessor
{
- public abstract ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue);
+ protected StructuredDataProcessor()
+ {
+ }
+
+ /**
+ * process a value that is definitely not a {@link Map}, {@link List}, or {@link Object[]}
+ */
+ public abstract ProcessedValue<?> processField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue);
+
+ /**
+ * 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 ProcessedValue<?> processArrayField(ArrayList<NestedPathPart> 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)
{
@@ -45,9 +66,11 @@ public abstract class StructuredDataProcessor
if (raw instanceof Map) {
toProcess.add(new MapField(newPath, (Map<String, ?>) 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, 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();
@@ -56,8 +79,8 @@ public abstract class StructuredDataProcessor
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));
}
}
return accumulator;
@@ -76,40 +99,50 @@ public abstract class StructuredDataProcessor
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<String, ?>) 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<Field> toProcess, ListField list)
+ private ProcessResults processArrayField(Queue<Field> 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();
- for (int i = 0; i < theList.size(); i++) {
- final ArrayList<NestedPathPart> 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<String, ?>) 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 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++) {
+ final ArrayList<NestedPathPart> 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<String, ?>) element));
+ } else if (element instanceof List) {
+ toProcess.add(new ArrayField(newPath, (List<?>) element));
+ } else if (element instanceof Object[]) {
+ toProcess.add(new ArrayField(newPath, Arrays.asList((Object[]) element)));
+ } else {
+ results.addLiteralField(newPath, processField(newPath, element).getSize());
+ }
}
}
return results;
}
- abstract static class Field
+ private abstract static class Field
{
private final ArrayList<NestedPathPart> path;
@@ -124,11 +157,11 @@ public abstract class StructuredDataProcessor
}
}
- static class ListField extends Field
+ static class ArrayField extends Field
{
private final List<?> list;
- ListField(ArrayList<NestedPathPart> path, List<?> list)
+ ArrayField(ArrayList<NestedPathPart> path, List<?> list)
{
super(path);
this.list = list;
@@ -156,14 +189,14 @@ public abstract class StructuredDataProcessor
}
}
- public static class ProcessedLiteral<T>
+ public static class ProcessedValue<T>
{
- 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;
@@ -190,7 +223,7 @@ public abstract class StructuredDataProcessor
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 63%
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 7440c12004..121f388583 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
@@ -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;
@@ -29,9 +30,9 @@ import java.nio.channels.WritableByteChannel;
/**
* Literal field writer for mixed type nested columns of {@link NestedDataColumnSerializer}
*/
-public final class VariantLiteralFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<Object>
+public final class VariantFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<Object>
{
- public VariantLiteralFieldColumnWriter(
+ public VariantFieldColumnWriter(
String columnName,
String fieldName,
SegmentWriteOutMedium segmentWriteOutMedium,
@@ -42,6 +43,32 @@ public final class VariantLiteralFieldColumnWriter extends GlobalDictionaryEncod
super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup);
}
+
+ @Override
+ Object processValue(int row, Object value)
+ {
+ if (value instanceof Object[]) {
+ Object[] array = (Object[]) value;
+ final int[] globalIds = new int[array.length];
+ for (int i = 0; i < array.length; i++) {
+ if (array[i] == null) {
+ globalIds[i] = 0;
+ } else if (array[i] instanceof String) {
+ globalIds[i] = globalDictionaryIdLookup.lookupString((String) array[i]);
+ } else if (array[i] instanceof Long) {
+ globalIds[i] = globalDictionaryIdLookup.lookupLong((Long) array[i]);
+ } else if (array[i] instanceof Double) {
+ globalIds[i] = globalDictionaryIdLookup.lookupDouble((Double) array[i]);
+ } else {
+ globalIds[i] = -1;
+ }
+ Preconditions.checkArgument(globalIds[i] >= 0, "unknown global id [%s] for value [%s]", globalIds[i], array[i]);
+ }
+ return globalIds;
+ }
+ return super.processValue(row, value);
+ }
+
@Override
int lookupGlobalId(Object value)
{
@@ -52,6 +79,8 @@ public final class VariantLiteralFieldColumnWriter extends GlobalDictionaryEncod
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 6541e1e79c..f4902946bf 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.data.ReadableOffset;
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 @@ import java.util.Objects;
/**
* 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.
* <p>
* This virtual column is used for the SQL operators JSON_VALUE (if {@link #processFromRaw} is set to false) or
@@ -655,12 +656,13 @@ public class NestedFieldVirtualColumn implements VirtualColumn
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 0000000000..44aa52615d
--- /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/data/input/impl/InputRowParserSerdeTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java
index 7e1b487cae..34a3722fc2 100644
--- a/processing/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java
+++ b/processing/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java
@@ -233,7 +233,6 @@ public class InputRowParserSerdeTest
"hey0barx",
"metA",
"missing",
- "timestamp",
"foo.bar1",
"blah",
"newmet",
diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java
index 5b0a3a391a..dea2efb019 100644
--- a/processing/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java
+++ b/processing/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java
@@ -188,7 +188,7 @@ public class JsonLineReaderTest
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 class JsonLineReaderTest
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 class JsonLineReaderTest
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/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java b/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java
index 8ec8929478..a2ddb78dc3 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 class EvalTest extends InitializedNullHandlingTest
Assert.assertFalse(evalDouble("!2.0", bindings) > 0.0);
}
finally {
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
try {
ExpressionProcessing.initializeForStrictBooleansTests(true);
@@ -137,7 +137,7 @@ public class EvalTest extends InitializedNullHandlingTest
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 class EvalTest extends InitializedNullHandlingTest
assertEquals(ExpressionType.DOUBLE, eval.type());
}
finally {
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
try {
ExpressionProcessing.initializeForStrictBooleansTests(true);
@@ -464,7 +464,7 @@ public class EvalTest extends InitializedNullHandlingTest
assertEquals(ExpressionType.LONG, eval.type());
}
finally {
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
}
@@ -532,7 +532,7 @@ public class EvalTest extends InitializedNullHandlingTest
}
finally {
// reset
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
try {
@@ -607,7 +607,7 @@ public class EvalTest extends InitializedNullHandlingTest
}
finally {
// reset
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
}
@@ -664,7 +664,7 @@ public class EvalTest extends InitializedNullHandlingTest
}
finally {
// reset
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
try {
@@ -703,7 +703,7 @@ public class EvalTest extends InitializedNullHandlingTest
}
finally {
// reset
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
}
@@ -948,103 +948,95 @@ public class EvalTest extends InitializedNullHandlingTest
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 class EvalTest extends InitializedNullHandlingTest
}
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 e2f14dd712..00682b9632 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 class OutputTypeTest extends InitializedNullHandlingTest
}
finally {
// reset
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
try {
@@ -96,7 +96,7 @@ public class OutputTypeTest extends InitializedNullHandlingTest
}
finally {
// reset
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
}
@@ -161,7 +161,7 @@ public class OutputTypeTest extends InitializedNullHandlingTest
assertOutputType("z_ || z", inspector, ExpressionType.LONG);
}
finally {
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
try {
ExpressionProcessing.initializeForStrictBooleansTests(false);
@@ -184,7 +184,7 @@ public class OutputTypeTest extends InitializedNullHandlingTest
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 87cc6f1b64..f6f5eccd10 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 @@ package org.apache.druid.math.expr;
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;
@@ -353,36 +352,28 @@ public class ParserTest extends InitializedNullHandlingTest
@Test
public void testLiteralExplicitTypedArrays()
{
- ExpressionProcessing.initializeForTests(true);
+ validateConstantExpression("ARRAY<DOUBLE>[1.0, 2.0, null, 3.0]", new Object[]{1.0, 2.0, null, 3.0});
+ validateConstantExpression("ARRAY<LONG>[1, 2, null, 3]", new Object[]{1L, 2L, null, 3L});
+ validateConstantExpression("ARRAY<STRING>['1', '2', null, '3.0']", new Object[]{"1", "2", null, "3.0"});
- try {
- validateConstantExpression("ARRAY<DOUBLE>[1.0, 2.0, null, 3.0]", new Object[]{1.0, 2.0, null, 3.0});
- validateConstantExpression("ARRAY<LONG>[1, 2, null, 3]", new Object[]{1L, 2L, null, 3L});
- validateConstantExpression("ARRAY<STRING>['1', '2', null, '3.0']", new Object[]{"1", "2", null, "3.0"});
-
- // mixed type tests
- validateConstantExpression("ARRAY<DOUBLE>[3, null, 4, 2.345]", new Object[]{3.0, null, 4.0, 2.345});
- validateConstantExpression("ARRAY<LONG>[1.0, null, 2000.0]", new Object[]{1L, null, 2000L});
+ // mixed type tests
+ validateConstantExpression("ARRAY<DOUBLE>[3, null, 4, 2.345]", new Object[]{3.0, null, 4.0, 2.345});
+ validateConstantExpression("ARRAY<LONG>[1.0, null, 2000.0]", new Object[]{1L, null, 2000L});
- // explicit typed string arrays should accept any literal and convert
- validateConstantExpression("ARRAY<STRING>['1', null, 2000, 1.1]", new Object[]{"1", null, "2000", "1.1"});
- validateConstantExpression("ARRAY<LONG>['1', null, 2000, 1.1]", new Object[]{1L, null, 2000L, 1L});
- validateConstantExpression("ARRAY<DOUBLE>['1', null, 2000, 1.1]", new Object[]{1.0, null, 2000.0, 1.1});
+ // explicit typed string arrays should accept any literal and convert
+ validateConstantExpression("ARRAY<STRING>['1', null, 2000, 1.1]", new Object[]{"1", null, "2000", "1.1"});
+ validateConstantExpression("ARRAY<LONG>['1', null, 2000, 1.1]", new Object[]{1L, null, 2000L, 1L});
+ validateConstantExpression("ARRAY<DOUBLE>['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<COMPLEX<nullableLongPair>>[]", new Object[]{});
- validateConstantExpression("ARRAY<ARRAY<LONG>>[]", new Object[]{});
- }
- finally {
- ExpressionProcessing.initializeForTests(null);
- }
+ // the gramar isn't cool enough yet to parse populated nested-arrays or complex arrays..., but empty ones can
+ // be defined...
+ validateConstantExpression("ARRAY<COMPLEX<nullableLongPair>>[]", new Object[]{});
+ validateConstantExpression("ARRAY<ARRAY<LONG>>[]", 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,18 +417,8 @@ public class ParserTest extends InitializedNullHandlingTest
StringUtils.format("array(%s,%s)", l1String, l2String),
new Object[]{l1, l2}
);
- ExpressionProcessing.initializeForTests(null);
}
- @Test
- public void nestedArraysExplodeIfNotEnabled()
- {
- expectedException.expect(IAE.class);
- expectedException.expectMessage("Cannot create a nested array type [ARRAY<ARRAY<LONG>>], 'druid.expressions.allowNestedArrays' must be set to true");
- validateConstantExpression("ARRAY<ARRAY<LONG>>[]", new Object[]{});
- }
-
-
@Test
public void testLiteralArrayImplicitStringParseException()
{
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 2e20a486fa..a8724db8a0 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 class VectorExprSanityTest extends InitializedNullHandlingTest
@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 152e7c908a..dafa7ff9a8 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 class MultiValuedDimensionTest extends InitializedNullHandlingTest
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/NestedDataTestUtils.java b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java
index d75f92e7f1..8e6f3485d4 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,456 @@
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.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;
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 JsonInputFormat DEFAULT_JSON_INPUT_FORMAT = new JsonInputFormat(
+ JSONPathSpec.DEFAULT,
+ null,
+ null,
+ null,
+ null
+ );
+
+ public static final DimensionsSpec AUTO_DISCOVERY =
+ DimensionsSpec.builder()
+ .useSchemaDiscovery(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 = 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(
+ 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<Segment> createSegments(
- AggregationTestHelper helper,
+ public static List<Segment> 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<Segment> createTsvSegments(
- AggregationTestHelper helper,
+ public static List<Segment> 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
);
}
- public static List<Segment> createSegments(
- AggregationTestHelper helper,
+ public static List<Segment> 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<Segment> 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
+ );
}
- public static List<Segment> 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
+ boolean rollup
+ )
+ 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,
+ DEFAULT_JSON_INPUT_FORMAT,
+ TIMESTAMP_SPEC,
+ AUTO_DISCOVERY,
+ TransformSpec.NONE,
+ COUNT,
granularity,
- maxRowCount,
rollup
);
- inputDataStream.close();
-
- final List<Segment> 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 segments;
}
- public static List<Segment> createSegmentsWithConcatenatedInput(
- AggregationTestHelper helper,
+ public static List<Segment> 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,
+ DEFAULT_JSON_INPUT_FORMAT,
+ 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<Segment> createSegmentsWithConcatenatedInput(
- AggregationTestHelper helper,
+ public static List<Segment> 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<File> segmentDirs = Lists.newArrayListWithCapacity(numSegments);
+ List<InputSource> inputFiles = Lists.newArrayListWithCapacity(numSegments);
for (int i = 0; i < numSegments; i++) {
- List<InputStream> 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<Segment> 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<AggregatorFactory> aggregatorSpecs = JSON_MAPPER.readValue(
- aggJson,
- new TypeReference<List<AggregatorFactory>>()
- {
- }
- );
- IncrementalIndex index = AggregationTestHelper.createIncrementalIndex(
- iter,
- parser,
- parser.getParseSpec().getDimensionsSpec().getDimensions(),
- aggregatorSpecs.toArray(new AggregatorFactory[0]),
- 0,
+ return createSegments(
+ tempFolder,
+ closer,
+ inputFiles,
+ DEFAULT_JSON_INPUT_FORMAT,
+ 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<Segment> createDefaultHourlySegments(
- AggregationTestHelper helper,
+ public static List<Segment> 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<Segment> 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
)
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<Segment> createDefaultDaySegments(
- AggregationTestHelper helper,
+ public static List<Segment> 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<Segment> createSegments(
+ TemporaryFolder tempFolder,
+ Closer closer,
+ List<InputSource> 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<Segment> 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<InputStream> 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<TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>> getSegmentGenerators()
+ public static List<BiFunction<TemporaryFolder, Closer, List<Segment>>> getSegmentGenerators(
+ String jsonInputFile
+ )
{
- final List<TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>> segmentsGenerators = new ArrayList<>();
- segmentsGenerators.add(new TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>()
+ final List<BiFunction<TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
+ new ArrayList<>();
+ segmentsGenerators.add(new BiFunction<TemporaryFolder, Closer, List<Segment>>()
{
@Override
- public List<Segment> apply(AggregationTestHelper helper, TemporaryFolder tempFolder, Closer closer)
+ public List<Segment> apply(TemporaryFolder tempFolder, Closer closer)
{
try {
return ImmutableList.<Segment>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 +482,15 @@ public class NestedDataTestUtils
return "mixed";
}
});
- segmentsGenerators.add(new TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>()
+ segmentsGenerators.add(new BiFunction<TemporaryFolder, Closer, List<Segment>>()
{
@Override
- public List<Segment> apply(AggregationTestHelper helper, TemporaryFolder tempFolder, Closer closer)
+ public List<Segment> 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 +504,26 @@ public class NestedDataTestUtils
return "incremental";
}
});
- segmentsGenerators.add(new TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>()
+ segmentsGenerators.add(new BiFunction<TemporaryFolder, Closer, List<Segment>>()
{
@Override
- public List<Segment> apply(AggregationTestHelper helper, TemporaryFolder tempFolder, Closer closer)
+ public List<Segment> apply(TemporaryFolder tempFolder, Closer closer)
{
try {
return ImmutableList.<Segment>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 5cad59dffb..65a59fc528 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.column.ColumnConfig;
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 class AggregationTestHelper implements Closeable
}
}
- 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<AggregatorFactory> aggregatorSpecs = mapper.readValue(
- aggregators,
- new TypeReference<List<AggregatorFactory>>()
- {
- }
- );
-
- 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 1b7e899509..235579b19f 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
@@ -131,7 +131,7 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
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/expression/VectorExpressionsSanityTest.java b/processing/src/test/java/org/apache/druid/query/expression/VectorExpressionsSanityTest.java
index 87ae80fa10..71e977909f 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 class VectorExpressionsSanityTest extends InitializedNullHandlingTest
@AfterClass
public static void teardownTests()
{
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
static void testExpression(String expr, Expr parsed, Map<String, ExpressionType> types)
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 3038ccaedf..030a40b29c 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.common.config.NullHandling;
import org.apache.druid.guice.NestedDataModule;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.java.util.common.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.ArrayList;
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<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
+ private final BiFunction<TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
private final String segmentsName;
public NestedDataGroupByQueryTest(
GroupByQueryConfig config,
- TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> segmentGenerator,
+ BiFunction<TemporaryFolder, Closer, List<Segment>> segmentGenerator,
String vectorize
)
{
@@ -107,11 +107,11 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
public static Collection<?> constructorFeeder()
{
final List<Object[]> constructors = new ArrayList<>();
- final List<TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
- NestedDataTestUtils.getSegmentGenerators();
+ final List<BiFunction<TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
+ NestedDataTestUtils.getSegmentGenerators(NestedDataTestUtils.SIMPLE_DATA_FILE);
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
- for (TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> generatorFn : segmentsGenerators) {
+ for (BiFunction<TemporaryFolder, Closer, List<Segment>> generatorFn : segmentsGenerators) {
for (String vectorize : new String[]{"false", "true", "force"}) {
constructors.add(new Object[]{config, generatorFn, vectorize});
}
@@ -444,8 +444,8 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
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 class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
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 class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
runResults(
groupQuery,
ImmutableList.of(
- new Object[]{1609459200000L, 8L}
+ new Object[]{1672531200000L, 8L}
),
false,
true
@@ -495,7 +495,7 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
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 class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
runResults(
groupQuery,
ImmutableList.of(
- new Object[]{"1609459200000", 8L}
+ new Object[]{"1672531200000", 8L}
),
true,
false
@@ -542,14 +542,18 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
runResults(groupQuery, expectedResults, false, false);
}
- private void runResults(GroupByQuery groupQuery, List<Object[]> expectedResults, boolean hasUnknownCardinality, boolean hasNonStringOutput)
+ private void runResults(
+ GroupByQuery groupQuery,
+ List<Object[]> expectedResults,
+ boolean hasUnknownCardinality,
+ boolean hasNonStringOutput
+ )
{
+ Supplier<List<ResultRow>> 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 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
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 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
}
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 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
}
}
- Sequence<ResultRow> seq = helper.runQueryOnSegmentsObjs(
- segmentsGenerator.apply(helper, tempFolder, closer),
- groupQuery
- );
-
- List<ResultRow> results = seq.toList();
+ List<ResultRow> 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 268c19bf91..8593f746cc 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,7 @@ package org.apache.druid.query.scan;
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.guice.NestedDataModule;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
@@ -32,6 +33,8 @@ import org.apache.druid.query.Druids;
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 +42,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
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;
@@ -71,10 +75,7 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
{
NestedDataModule.registerHandlersAndSerde();
List<? extends Module> mods = NestedDataModule.getJacksonModulesList();
- this.helper = AggregationTestHelper.createScanQueryAggregationTestHelper(
- mods,
- tempFolder
- );
+ this.helper = AggregationTestHelper.createScanQueryAggregationTestHelper(mods, tempFolder);
this.closer = Closer.create();
}
@@ -97,7 +98,7 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
.limit(100)
.context(ImmutableMap.of())
.build();
- List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer);
+ List<Segment> segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer);
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
@@ -126,15 +127,16 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
.build();
List<Segment> segs = ImmutableList.<Segment>builder().addAll(
NestedDataTestUtils.createSegments(
- helper,
tempFolder,
closer,
NestedDataTestUtils.NUMERIC_DATA_FILE,
- NestedDataTestUtils.NUMERIC_PARSER_FILE,
- NestedDataTestUtils.SIMPLE_AGG_FILE,
+ NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+ NestedDataTestUtils.TIMESTAMP_SPEC,
+ NestedDataTestUtils.AUTO_DISCOVERY,
+ TransformSpec.NONE,
+ NestedDataTestUtils.COUNT,
Granularities.YEAR,
- true,
- 1000
+ true
)
).build();
@@ -171,9 +173,9 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
.context(ImmutableMap.of())
.build();
List<Segment> realtimeSegs = ImmutableList.of(
- NestedDataTestUtils.createDefaultHourlyIncrementalIndex()
+ NestedDataTestUtils.createSimpleNestedTestDataIncrementalIndex(tempFolder)
);
- List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer);
+ List<Segment> segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer);
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(realtimeSegs, scanQuery);
@@ -242,9 +244,9 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
.context(ImmutableMap.of())
.build();
List<Segment> realtimeSegs = ImmutableList.of(
- NestedDataTestUtils.createDefaultHourlyIncrementalIndex()
+ NestedDataTestUtils.createSimpleNestedTestDataIncrementalIndex(tempFolder)
);
- List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer);
+ List<Segment> segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer);
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(realtimeSegs, scanQuery);
@@ -278,7 +280,7 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
.limit(100)
.context(ImmutableMap.of())
.build();
- List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegmentsTsv(helper, tempFolder, closer);
+ List<Segment> segs = NestedDataTestUtils.createSimpleSegmentsTsv(tempFolder, closer);
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
@@ -302,13 +304,12 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
.limit(100)
.context(ImmutableMap.of())
.build();
- List<Segment> segs = NestedDataTestUtils.createSegments(
- helper,
+ List<Segment> segs = NestedDataTestUtils.createSegmentsForJsonInput(
tempFolder,
closer,
+ NestedDataTestUtils.SIMPLE_DATA_FILE,
Granularities.HOUR,
- true,
- 3
+ true
);
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
@@ -334,13 +335,12 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
.build();
- List<Segment> segs = NestedDataTestUtils.createSegmentsWithConcatenatedInput(
- helper,
+ List<Segment> segs = NestedDataTestUtils.createSegmentsWithConcatenatedJsonInput(
tempFolder,
closer,
+ NestedDataTestUtils.SIMPLE_DATA_FILE,
Granularities.HOUR,
false,
- 5,
10,
1
);
@@ -369,13 +369,12 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
// same rows over and over so expect same 8 rows after rollup
- List<Segment> segs = NestedDataTestUtils.createSegmentsWithConcatenatedInput(
- helper,
+ List<Segment> segs = NestedDataTestUtils.createSegmentsWithConcatenatedJsonInput(
tempFolder,
closer,
- Granularities.HOUR,
+ NestedDataTestUtils.SIMPLE_DATA_FILE,
+ Granularities.YEAR,
true,
- 5,
100,
1
);
@@ -407,7 +406,7 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
.limit(100)
.context(ImmutableMap.of())
.build();
- List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer);
+ List<Segment> segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer);
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
@@ -446,7 +445,7 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
.limit(100)
.context(ImmutableMap.of())
.build();
- List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer);
+ List<Segment> segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer);
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
@@ -472,25 +471,87 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
.build();
List<Segment> realtimeSegs = ImmutableList.of(
NestedDataTestUtils.createIncrementalIndex(
+ tempFolder,
NestedDataTestUtils.TYPES_DATA_FILE,
- NestedDataTestUtils.TYPES_PARSER_FILE,
- NestedDataTestUtils.SIMPLE_AGG_FILE,
+ NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+ NestedDataTestUtils.TIMESTAMP_SPEC,
+ NestedDataTestUtils.AUTO_DISCOVERY,
+ TransformSpec.NONE,
+ NestedDataTestUtils.COUNT,
Granularities.DAY,
- true,
- false,
- 1000
+ true
)
);
List<Segment> segs = NestedDataTestUtils.createSegments(
- helper,
tempFolder,
closer,
NestedDataTestUtils.TYPES_DATA_FILE,
- NestedDataTestUtils.TYPES_PARSER_FILE,
- NestedDataTestUtils.SIMPLE_AGG_FILE,
+ NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+ NestedDataTestUtils.TIMESTAMP_SPEC,
+ NestedDataTestUtils.AUTO_DISCOVERY,
+ TransformSpec.NONE,
+ NestedDataTestUtils.COUNT,
Granularities.DAY,
- true,
- 100
+ true
+ );
+
+
+ final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(realtimeSegs, scanQuery);
+ final Sequence<ScanResultValue> seq2 = helper.runQueryOnSegmentsObjs(segs, scanQuery);
+
+ List<ScanResultValue> resultsRealtime = seq.toList();
+ List<ScanResultValue> 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<ScanResultValue> scanQuery = builder.build();
+ final AggregatorFactory[] aggs = new AggregatorFactory[]{new CountAggregatorFactory("count")};
+ List<Segment> realtimeSegs = ImmutableList.of(
+ NestedDataTestUtils.createIncrementalIndex(
+ tempFolder,
+ NestedDataTestUtils.ARRAY_TYPES_DATA_FILE,
+ NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+ NestedDataTestUtils.TIMESTAMP_SPEC,
+ NestedDataTestUtils.AUTO_DISCOVERY,
+ TransformSpec.NONE,
+ aggs,
+ Granularities.NONE,
+ true
+ )
+ );
+ List<Segment> segs = NestedDataTestUtils.createSegments(
+ tempFolder,
+ closer,
+ NestedDataTestUtils.ARRAY_TYPES_DATA_FILE,
+ NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+ 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 58cbf46c5c..392df70e1d 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.Intervals;
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.Arrays;
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<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
+ private final BiFunction<TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
private final Closer closer;
public NestedDataTopNQueryTest(
- TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> segmentGenerator
+ BiFunction<TemporaryFolder, Closer, List<Segment>> segmentGenerator
)
{
NestedDataModule.registerHandlersAndSerde();
@@ -85,10 +85,10 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest
public static Collection<?> constructorFeeder()
{
final List<Object[]> constructors = new ArrayList<>();
- final List<TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
- NestedDataTestUtils.getSegmentGenerators();
+ final List<BiFunction<TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
+ NestedDataTestUtils.getSegmentGenerators(NestedDataTestUtils.SIMPLE_DATA_FILE);
- for (TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> generatorFn : segmentsGenerators) {
+ for (BiFunction<TemporaryFolder, Closer, List<Segment>> generatorFn : segmentsGenerators) {
constructors.add(new Object[]{generatorFn});
}
return constructors;
@@ -114,7 +114,7 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest
.build();
- Sequence<Result<TopNResultValue>> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), topN);
+ Sequence<Result<TopNResultValue>> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(tempFolder, closer), topN);
Sequence<Object[]> resultsSeq = new TopNQueryQueryToolChest(new TopNQueryConfig()).resultsAsArrays(topN, seq);
@@ -123,10 +123,10 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest
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 class NestedDataTopNQueryTest extends InitializedNullHandlingTest
.build();
- Sequence<Result<TopNResultValue>> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), topN);
+ Sequence<Result<TopNResultValue>> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(tempFolder, closer), topN);
Sequence<Object[]> resultsSeq = new TopNQueryQueryToolChest(new TopNQueryConfig()).resultsAsArrays(topN, seq);
@@ -157,10 +157,10 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest
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 f0d9b1fc32..90a4411c87 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 @@ package org.apache.druid.segment;
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.IncrementalIndex;
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<InputRow> 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<InputRow> 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 class IndexBuilder
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<InputRow> rowIterator = transformingReader.read()) {
+ while (rowIterator.hasNext()) {
+ rows.add(rowIterator.next());
+ }
+ }
+ return this;
+ }
+
public IndexBuilder rows(Iterable<InputRow> rows)
{
this.rows.clear();
@@ -128,8 +190,30 @@ public class IndexBuilder
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 class IndexBuilder
public QueryableIndex buildMMappedMergedIndex()
{
- IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory);
Preconditions.checkNotNull(tmpDir, "tmpDir");
-
final List<QueryableIndex> 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 transformer = transformSpec != null ? transformSpec : TransformSpec.NONE;
+ InputRowSchema rowSchema = new InputRowSchema(schema.getTimestampSpec(), schema.getDimensionsSpec(), null);
+ InputSourceReader reader = inputSource.reader(rowSchema, inputFormat, inputSourceTmpDir);
+ InputSourceReader transformingReader = transformer.decorate(reader);
+ return mergeIndexes(indexMerger, persisted, transformingReader::read);
+ }
+
+ return mergeIndexes(indexMerger, persisted, () -> CloseableIterators.withEmptyBaggage(rows.iterator()));
+ }
+
+ @Nonnull
+ private QueryableIndex mergeIndexes(
+ IndexMerger indexMerger,
+ List<QueryableIndex> persisted,
+ IteratorSupplier iteratorSupplier
+ )
+ {
+ try (CloseableIterator<InputRow> 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 class IndexBuilder
)
);
}
+
final QueryableIndex merged = TestHelper.getTestIndexIO().loadIndex(
indexMerger.mergeQueryableIndex(
persisted,
@@ -267,4 +395,44 @@ public class IndexBuilder
}
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<InputRow> rowIterator = transformingReader.read()) {
+ while (rowIterator.hasNext()) {
+ incrementalIndex.add(rowIterator.next());
+ }
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ return incrementalIndex;
+ }
+
+
+ @FunctionalInterface
+ interface IteratorSupplier
+ {
+ CloseableIterator<InputRow> 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 d489bb5c1c..ce28ed0066 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 class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
// 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 class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
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 class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
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 class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
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/UnnestColumnValueSelectorCursorTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java
index bafb50f708..475f268338 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 @@ package org.apache.druid.segment;
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
@@ -123,7 +120,8 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
List<Object> inputList = Arrays.asList(
Collections.singletonList(null),
Arrays.asList(null, null),
- Collections.singletonList(null)
+ Collections.singletonList(null),
+ Collections.emptyList()
);
//Create base cursor
@@ -145,6 +143,74 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
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_mv_to_array()
+ {
+ List<Object> 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()
+ {
+ List<Object> 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 +221,15 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
Arrays.asList("a", "b"),
Arrays.asList("b", "c"),
"d",
+ Collections.singletonList(null),
+ Arrays.asList(null, null),
+ Collections.emptyList(),
null,
null,
null
);
- List<String> expectedResults = Arrays.asList("a", "b", "b", "c", "d", null, null, null);
+ List<String> expectedResults = Arrays.asList("a", "b", "b", "c", "d", null, null);
//Create base cursor
ListCursor listCursor = new ListCursor(inputList);
@@ -178,7 +247,52 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
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<Object> 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<String> 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 +682,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
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 88a4a5d3ac..4c1c51231a 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 class FrontCodedIndexedTest extends InitializedNullHandlingTest
{
ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
List<String> 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 class FrontCodedIndexedTest extends InitializedNullHandlingTest
{
ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
List<String> 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 class FrontCodedIndexedTest extends InitializedNullHandlingTest
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 class FrontCodedIndexedTest extends InitializedNullHandlingTest
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 class FrontCodedIndexedTest extends InitializedNullHandlingTest
ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
List<String> 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 class FrontCodedIndexedTest extends InitializedNullHandlingTest
TreeSet<String> 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 class FrontCodedIndexedTest extends InitializedNullHandlingTest
// "\uD83D\uDCA9" and "(請參見已被刪除版本)" are a regression test for https://github.com/apache/druid/pull/13364
List<String> 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 class FrontCodedIndexedTest extends InitializedNullHandlingTest
{
ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
List<String> theList = Collections.singletonList(null);
- fillBuffer(buffer, theList, 4, useIncrementalBuckets);
+ persistToBuffer(buffer, theList, 4, useIncrementalBuckets);
buffer.position(0);
FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
@@ -303,6 +303,32 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
Assert.assertFalse(utf8Iterator.hasNext());
}
+ @Test
+ public void testFrontCodedEmpty() throws IOException
+ {
+ ByteBuffer buffer = ByteBuffer.allocate(1 << 6).order(order);
+ List<String> theList = Collections.emptyList();
+ persistToBuffer(buffer, theList, 4, useIncrementalBuckets);
+
+ 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<ByteBuffer> utf8Iterator = codedUtf8Indexed.iterator();
+ Assert.assertFalse(utf8Iterator.hasNext());
+ }
+
@Test
public void testBucketSizes() throws IOException
{
@@ -325,7 +351,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
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()
@@ -388,7 +414,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
);
}
- private static long fillBuffer(ByteBuffer buffer, Iterable<String> sortedIterable, int bucketSize, boolean useIncrementalBuckets) throws IOException
+ private static long persistToBuffer(ByteBuffer buffer, Iterable<String> sortedIterable, int bucketSize, boolean useIncrementalBuckets) throws IOException
{
Iterator<String> 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
new file mode 100644
index 0000000000..8a08a2a972
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java
@@ -0,0 +1,467 @@
+/*
+ * 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 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;
+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.Arrays;
+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<Object[]> 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<int[]> 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});
+ values.add(new int[]{2, 1});
+ values.add(new int[]{2, 2, 1});
+
+ persistToBuffer(buffer, values, 4);
+
+ buffer.position(0);
+ FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+ buffer,
+ buffer.order()
+ ).get();
+
+ Iterator<int[]> indexedIterator = codedIndexed.iterator();
+ Iterator<int[]> expectedIterator = values.iterator();
+ int ctr = 0;
+ while (expectedIterator.hasNext() && indexedIterator.hasNext()) {
+ final int[] expectedNext = expectedIterator.next();
+ final int[] next = indexedIterator.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());
+ }
+
+
+ @Test
+ public void testFrontCodedIntArrayIndexedSingleBucket() throws IOException
+ {
+ ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+ TreeSet<int[]> 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});
+ persistToBuffer(buffer, values, 16);
+
+ FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+ buffer,
+ buffer.order()
+ ).get();
+
+ Iterator<int[]> expectedIterator = values.iterator();
+ Iterator<int[]> indexedIterator = codedIndexed.iterator();
+ int ctr = 0;
+ while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+ final int[] expectedNext = expectedIterator.next();
+ final int[] next = indexedIterator.next();
+ assertSame(ctr, expectedNext, next);
+ assertSame(ctr, expectedNext, codedIndexed.get(ctr));
+ 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<int[]> 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);
+ }
+ persistToBuffer(buffer, values, bucketSize);
+
+ FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+ buffer,
+ buffer.order()
+ ).get();
+
+ Iterator<int[]> expectedIterator = values.iterator();
+ Iterator<int[]> indexedIterator = codedIndexed.iterator();
+ int ctr = 0;
+ while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+ final int[] expectedNext = expectedIterator.next();
+ final int[] next = indexedIterator.next();
+ assertSame(ctr, expectedNext, next);
+ assertSame(ctr, expectedNext, codedIndexed.get(ctr));
+ 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<int[]> 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);
+ }
+ persistToBuffer(buffer, values, bucketSize);
+
+ FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+ buffer,
+ buffer.order()
+ ).get();
+
+ Iterator<int[]> expectedIterator = values.iterator();
+ Iterator<int[]> indexedIterator = codedIndexed.iterator();
+ int ctr = 0;
+ while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+ final int[] expectedNext = expectedIterator.next();
+ final int[] next = indexedIterator.next();
+ assertSame(ctr, expectedNext, next);
+ assertSame(ctr, expectedNext, codedIndexed.get(ctr));
+ 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<int[]> 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});
+
+ persistToBuffer(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<int[]> 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});
+ persistToBuffer(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<int[]> theList = Collections.singletonList(null);
+ persistToBuffer(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<int[]> 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<int[]> theList = Collections.emptyList();
+ persistToBuffer(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<ByteBuffer> 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<int[]> 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) {
+ persistToBuffer(buffer, values, bucketSize);
+ FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+ buffer,
+ buffer.order()
+ ).get();
+
+ Iterator<int[]> expectedIterator = values.iterator();
+ Iterator<int[]> iterator = codedIndexed.iterator();
+ int ctr = 0;
+ while (iterator.hasNext() && expectedIterator.hasNext()) {
+ final int[] expectedNext = expectedIterator.next();
+ final int[] next = iterator.next();
+ assertSame(ctr, expectedNext, next);
+ assertSame(ctr, expectedNext, codedIndexed.get(ctr));
+ 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 persistToBuffer(ByteBuffer buffer, Iterable<int[]> sortedIterable, int bucketSize) throws IOException
+ {
+ Iterator<int[]> 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);
+ assertSame(index, 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()) {
+ assertSame(index, sortedInts.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;
+ }
+
+ 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/filter/ExpressionFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java
index d0e089a7f7..289843f600 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 class ExpressionFilterTest extends BaseFilterTest
@After
public void teardown()
{
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
@AfterClass
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 38764b093a..ba2be32008 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.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.druid.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.ColumnValueSelector;
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.DruidPredicateIndex;
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<Map<String, Object>> 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, null, 3L},
+ "d",
+ new Object[]{2.2, 2.2}
+ ),
+ TestHelper.makeMap(
+ "s",
+ new Object[]{"b", "c"},
+ "l",
+ new Object[]{null, null},
+ "d",
+ 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}),
+ 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 class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
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<Map<String, Object>> data
+ )
+ throws IOException
+ {
+ SegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance();
+ try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) {
NestedDataColumnSerializer serializer = new NestedDataColumnSerializer(
fileNameBase,
new IndexSpec(),
@@ -135,16 +181,22 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
for (Object o : data) {
indexer.processRowValsToUnsortedEncodedKeyComponent(o, false);
}
- SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> sortedFields = new TreeMap<>();
+ SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> 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.serializeDictionaries(
+ globalDictionarySortedCollector.getSortedStrings(),
+ globalDictionarySortedCollector.getSortedLongs(),
+ globalDictionarySortedCollector.getSortedDoubles(),
+ () -> new NestedDataColumnMerger.ArrayDictionaryMergingIterator(
+ new Iterable[]{globalDictionarySortedCollector.getSortedArrays()},
+ serializer.getGlobalLookup()
+ )
+ );
SettableSelector valueSelector = new SettableSelector();
for (Object o : data) {
@@ -156,8 +208,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
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 class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
}
}
+ @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
{
@@ -234,10 +303,10 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
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);
@@ -251,8 +320,8 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
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);
@@ -271,6 +340,50 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
}
}
+ @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<NestedPathPart> path = ImmutableList.of(new NestedPathField("lastName"));
+ ColumnHolder nestedColumnHolder = v4.getColumnHolder(path);
+ Assert.assertNotNull(nestedColumnHolder);
+ Assert.assertEquals(ColumnType.STRING, nestedColumnHolder.getCapabilities().toColumnType());
+ NestedFieldDictionaryEncodedColumn<?> nestedColumn =
+ (NestedFieldDictionaryEncodedColumn<?>) 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());
@@ -310,7 +423,10 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
NullValueIndex zNulls = zIndexSupplier.as(NullValueIndex.class);
final List<NestedPathPart> 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,12 +460,265 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
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<NestedPathPart> 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<NestedPathPart> 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<NestedPathPart> 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<NestedPathPart> 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<NestedPathPart> 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<NestedPathPart> 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 || 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 || d[1] == null) {
+ 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 || 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 || d[1] == null) {
+ 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 || l[1] == null) {
+ Assert.assertTrue(lElementNulls[i]);
+ } else {
+ Assert.assertEquals(l[1], lElementVector[i]);
+ }
+ if (d == null || d.length < 1 || d[1] == null) {
+ Assert.assertTrue(dElementNulls[i]);
+ } else {
+ Assert.assertEquals((Double) d[1], dElementVector[i], 0.0);
+ }
+ }
+ bitmapVectorOffset.advance();
+ }
+ }
+
private void testPath(
Map row,
int rowNumber,
@@ -372,10 +741,11 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
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());
@@ -384,11 +754,19 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
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());
@@ -397,7 +775,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
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());
@@ -406,9 +784,13 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
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/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 46e881a9cb..13e5e2b535 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.ByteOrder;
import java.nio.channels.WritableByteChannel;
import java.util.TreeSet;
-public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHandlingTest
+public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingTest
{
BitmapSerdeFactory roaringFactory = RoaringBitmapSerdeFactory.getInstance();
BitmapResultFactory<ImmutableBitmap> bitmapResultFactory = new DefaultBitmapResultFactory(
@@ -139,7 +139,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@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 class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
@Test
public void testDictionaryEncodedStringValueIndex() throws IOException
{
- NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
+ NestedFieldColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
DictionaryEncodedStringValueIndex lowLevelIndex = indexSupplier.as(DictionaryEncodedStringValueIndex.class);
Assert.assertNotNull(lowLevelIndex);
@@ -1398,18 +1398,20 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
GenericIndexed<ImmutableBitmap> 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,
dictionarySupplier,
stringIndexed,
longIndexed,
- doubleIndexed
+ doubleIndexed,
+ null,
+ null
);
StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
@@ -1438,7 +1440,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
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);
@@ -1496,20 +1498,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
GenericIndexed<ImmutableBitmap> 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,
dictionarySupplier,
globalStrings,
globalLongs,
- globalDoubles
+ globalDoubles,
+ null,
+ null
);
}
- 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);
@@ -1570,20 +1574,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
GenericIndexed<ImmutableBitmap> 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,
dictionarySupplier,
globalStrings,
globalLongs,
- globalDoubles
+ globalDoubles,
+ null,
+ null
);
}
- 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);
@@ -1641,20 +1647,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
GenericIndexed<ImmutableBitmap> 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,
dictionarySupplier,
globalStrings,
globalLongs,
- globalDoubles
+ globalDoubles,
+ null,
+ null
);
}
- 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);
@@ -1716,20 +1724,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
GenericIndexed<ImmutableBitmap> 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,
dictionarySupplier,
globalStrings,
globalLongs,
- globalDoubles
+ globalDoubles,
+ null,
+ null
);
}
- 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);
@@ -1787,20 +1797,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
GenericIndexed<ImmutableBitmap> 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,
dictionarySupplier,
globalStrings,
globalLongs,
- globalDoubles
+ globalDoubles,
+ null,
+ null
);
}
- 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);
@@ -1862,20 +1874,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
GenericIndexed<ImmutableBitmap> 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,
dictionarySupplier,
globalStrings,
globalLongs,
- globalDoubles
+ globalDoubles,
+ null,
+ null
);
}
- 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);
@@ -1945,19 +1959,21 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
GenericIndexed<ImmutableBitmap> 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,
dictionarySupplier,
globalStrings,
globalLongs,
- globalDoubles
+ globalDoubles,
+ null,
+ null
);
}
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 95%
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 cf65e483ba..486bb6df4f 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
@@ -41,6 +41,7 @@ import org.apache.druid.segment.Segment;
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;
@@ -58,7 +59,7 @@ import org.junit.rules.TemporaryFolder;
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";
@@ -76,7 +77,7 @@ public class NestedFieldLiteralColumnSelectorsTest
private final AggregationTestHelper helper;
private final Closer closer;
- public NestedFieldLiteralColumnSelectorsTest()
+ public NestedFieldColumnSelectorsTest()
{
NestedDataModule.registerHandlersAndSerde();
List<? extends Module> mods = NestedDataModule.getJacksonModulesList();
@@ -326,15 +327,16 @@ public class NestedFieldLiteralColumnSelectorsTest
private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns virtualColumns) throws Exception
{
List<Segment> segments = NestedDataTestUtils.createSegments(
- helper,
tempFolder,
closer,
NestedDataTestUtils.NUMERIC_DATA_FILE,
- NestedDataTestUtils.NUMERIC_PARSER_FILE,
- NestedDataTestUtils.SIMPLE_AGG_FILE,
- Granularities.DAY,
- true,
- 1000
+ NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+ 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 +357,16 @@ public class NestedFieldLiteralColumnSelectorsTest
private VectorColumnSelectorFactory getVectorColumnSelectorFactory(VirtualColumns virtualColumns) throws Exception
{
List<Segment> segments = NestedDataTestUtils.createSegments(
- helper,
tempFolder,
closer,
NestedDataTestUtils.NUMERIC_DATA_FILE,
- NestedDataTestUtils.NUMERIC_PARSER_FILE,
- NestedDataTestUtils.SIMPLE_AGG_FILE,
- Granularities.DAY,
- true,
- 1000
+ NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+ 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/NestedFieldTypeInfoTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java
new file mode 100644
index 0000000000..aa100cf7af
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.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 NestedFieldTypeInfoTest
+{
+ private static final ByteBuffer BUFFER = ByteBuffer.allocate(1024).order(ByteOrder.nativeOrder());
+
+ @Test
+ public void testSingleType() throws IOException
+ {
+ List<ColumnType> 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<Set<ColumnType>> 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<ColumnType> typeSet : tests) {
+ testMultiType(typeSet);
+ }
+ }
+
+ @Test
+ public void testEqualsAndHashCode()
+ {
+ EqualsVerifier.forClass(NestedFieldTypeInfo.TypeSet.class)
+ .usingGetClass()
+ .verify();
+
+ EqualsVerifier.forClass(NestedFieldTypeInfo.MutableTypeSet.class)
+ .suppress(Warning.NONFINAL_FIELDS)
+ .usingGetClass()
+ .verify();
+ }
+
+ private void testSingleType(ColumnType columnType) throws IOException
+ {
+ 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), NestedFieldTypeInfo.convertToSet(typeSet.getByteValue()));
+
+ writeTypeSet(typeSet);
+ NestedFieldTypeInfo info = new NestedFieldTypeInfo(BUFFER);
+ Assert.assertEquals(0, BUFFER.position());
+
+ NestedFieldTypeInfo.TypeSet roundTrip = info.getTypes(0);
+ Assert.assertEquals(columnType, roundTrip.getSingleType());
+
+ NestedFieldTypeInfo info2 = NestedFieldTypeInfo.read(BUFFER, 1);
+ Assert.assertEquals(info.getTypes(0), info2.getTypes(0));
+ Assert.assertEquals(1, BUFFER.position());
+ }
+
+ private void testMultiType(Set<ColumnType> columnTypes) throws IOException
+ {
+ NestedFieldTypeInfo.MutableTypeSet typeSet = new NestedFieldTypeInfo.MutableTypeSet();
+ Assert.assertNull(typeSet.getSingleType());
+ Assert.assertTrue(typeSet.isEmpty());
+
+ NestedFieldTypeInfo.MutableTypeSet merge = new NestedFieldTypeInfo.MutableTypeSet();
+ for (ColumnType columnType : columnTypes) {
+ typeSet.add(columnType);
+ merge.merge(new NestedFieldTypeInfo.MutableTypeSet().add(columnType).getByteValue());
+ }
+
+ Assert.assertEquals(merge.getByteValue(), typeSet.getByteValue());
+ Assert.assertNull(typeSet.getSingleType());
+ Assert.assertEquals(columnTypes, NestedFieldTypeInfo.convertToSet(typeSet.getByteValue()));
+
+ writeTypeSet(typeSet);
+ NestedFieldTypeInfo info = new NestedFieldTypeInfo(BUFFER);
+ Assert.assertEquals(0, BUFFER.position());
+
+ NestedFieldTypeInfo.TypeSet roundTrip = info.getTypes(0);
+ Assert.assertNull(roundTrip.getSingleType());
+ Assert.assertEquals(columnTypes, NestedFieldTypeInfo.convertToSet(roundTrip.getByteValue()));
+
+ NestedFieldTypeInfo info2 = NestedFieldTypeInfo.read(BUFFER, 1);
+ Assert.assertEquals(info.getTypes(0), info2.getTypes(0));
+ Assert.assertEquals(1, BUFFER.position());
+ }
+
+ private static void writeTypeSet(NestedFieldTypeInfo.MutableTypeSet typeSet) throws IOException
+ {
+ BUFFER.position(0);
+ NestedFieldTypeInfo.Writer writer = new NestedFieldTypeInfo.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/java/org/apache/druid/segment/transform/TransformSpecTest.java b/processing/src/test/java/org/apache/druid/segment/transform/TransformSpecTest.java
index b9244b3ae1..3ed0a51bf0 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 class TransformSpecTest extends InitializedNullHandlingTest
Assert.assertEquals(0L, row2.getRaw("truthy2"));
}
finally {
- ExpressionProcessing.initializeForTests(null);
+ ExpressionProcessing.initializeForTests();
}
try {
ExpressionProcessing.initializeForStrictBooleansTests(false);
@@ -268,7 +268,7 @@ public class TransformSpecTest extends InitializedNullHandlingTest
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 1ba482d38a..b033475700 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
new file mode 100644
index 0000000000..b8ae3ace38
--- /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], "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/processing/src/test/resources/nested-numeric-test-data.json b/processing/src/test/resources/nested-numeric-test-data.json
new file mode 100644
index 0000000000..ba79dca0e9
--- /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}}
... 2350 lines suppressed ...
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org