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 86b5ae4edc48..6327b2dd780c 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 @@ -42,7 +42,7 @@ 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.AutoTypeColumnSchema; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.transform.TransformingInputEntityReader; @@ -269,14 +269,14 @@ public void testParseNestedData() throws Exception timestampSpec, new DimensionsSpec( Lists.newArrayList( - new StringDimensionSchema("event"), - new StringDimensionSchema("id"), - new StringDimensionSchema("someOtherId"), - new StringDimensionSchema("isValid"), - new StringDimensionSchema("eventType"), - new NestedDataDimensionSchema("foo"), - new NestedDataDimensionSchema("bar"), - new StringDimensionSchema("someBytesColumn") + new AutoTypeColumnSchema("event"), + new AutoTypeColumnSchema("id"), + new AutoTypeColumnSchema("someOtherId"), + new AutoTypeColumnSchema("isValid"), + new AutoTypeColumnSchema("eventType"), + new AutoTypeColumnSchema("foo"), + new AutoTypeColumnSchema("bar"), + new AutoTypeColumnSchema("someBytesColumn") ) ), null diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index f9ab426d13c2..967b55665c85 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -47,7 +47,7 @@ import org.apache.druid.segment.BaseProgressIndicator; import org.apache.druid.segment.ProgressIndicator; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.timeline.DataSegment; @@ -292,7 +292,7 @@ private static IncrementalIndex makeIncrementalIndex( AggregatorFactory[] aggs, HadoopDruidIndexerConfig config, Iterable oldDimOrder, - Map oldCapabilities + Map oldCapabilities ) { final HadoopTuningConfig tuningConfig = config.getSchema().getTuningConfig(); @@ -456,7 +456,7 @@ protected void reduce(final BytesWritable key, Iterable values, f dimOrder.addAll(index.getDimensionOrder()); log.info("current index full due to [%s]. creating new index.", index.getOutOfRowsReason()); flushIndexToContextAndClose(key, index, context); - index = makeIncrementalIndex(bucket, combiningAggs, config, dimOrder, index.getColumnHandlerCapabilities()); + index = makeIncrementalIndex(bucket, combiningAggs, config, dimOrder, index.getColumnFormats()); } index.add(value); @@ -752,7 +752,7 @@ public void doRun() combiningAggs, config, allDimensionNames, - persistIndex.getColumnHandlerCapabilities() + persistIndex.getColumnFormats() ); startTime = System.currentTimeMillis(); ++indexCount; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 9dc559657f06..8387d137f11e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -41,10 +41,6 @@ import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.DoubleDimensionSchema; -import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.impl.LongDimensionSchema; -import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.Checks; import org.apache.druid.indexer.Property; @@ -80,11 +76,9 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.DimensionHandler; -import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.indexing.DataSchema; @@ -869,47 +863,6 @@ public void close() ); } - @VisibleForTesting - static DimensionSchema createDimensionSchema( - String name, - ColumnCapabilities capabilities, - DimensionSchema.MultiValueHandling multiValueHandling - ) - { - switch (capabilities.getType()) { - case FLOAT: - Preconditions.checkArgument( - multiValueHandling == null, - "multi-value dimension [%s] is not supported for float type yet", - name - ); - return new FloatDimensionSchema(name); - case LONG: - Preconditions.checkArgument( - multiValueHandling == null, - "multi-value dimension [%s] is not supported for long type yet", - name - ); - return new LongDimensionSchema(name); - case DOUBLE: - Preconditions.checkArgument( - multiValueHandling == null, - "multi-value dimension [%s] is not supported for double type yet", - name - ); - return new DoubleDimensionSchema(name); - case STRING: - return new StringDimensionSchema(name, multiValueHandling, capabilities.hasBitmapIndexes()); - default: - DimensionHandler handler = DimensionHandlerUtils.getHandlerFromCapabilities( - name, - capabilities, - multiValueHandling - ); - return handler.getDimensionSchema(capabilities); - } - } - /** * Class for fetching and analyzing existing segments in order to generate reingestion specs. */ @@ -1109,7 +1062,7 @@ private void processDimensionsSpec(final QueryableIndex index) ); if (!uniqueDims.containsKey(dimension)) { - final DimensionHandler dimensionHandler = Preconditions.checkNotNull( + Preconditions.checkNotNull( dimensionHandlerMap.get(dimension), "Cannot find dimensionHandler for dimension[%s]", dimension @@ -1118,11 +1071,7 @@ private void processDimensionsSpec(final QueryableIndex index) uniqueDims.put(dimension, uniqueDims.size()); dimensionSchemaMap.put( dimension, - createDimensionSchema( - dimension, - columnHolder.getHandlerCapabilities(), - dimensionHandler.getMultivalueHandling() - ) + columnHolder.getColumnFormat().getColumnSchema(dimension) ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java index be3b31be3b70..32033fe17a7e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java @@ -247,11 +247,12 @@ public SamplerResponse sample( if (!SamplerInputRow.SAMPLER_ORDERING_COLUMN.equals(dimensionDesc.getName())) { final ColumnType columnType = dimensionDesc.getCapabilities().toColumnType(); signatureBuilder.add(dimensionDesc.getName(), columnType); + // for now, use legacy types instead of standard type logicalDimensionSchemas.add( DimensionSchema.getDefaultSchemaForBuiltInType(dimensionDesc.getName(), dimensionDesc.getCapabilities()) ); physicalDimensionSchemas.add( - dimensionDesc.getHandler().getDimensionSchema(dimensionDesc.getCapabilities()) + dimensionDesc.getIndexer().getFormat().getColumnSchema(dimensionDesc.getName()) ); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 01a3f8475d39..19b178caa349 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -92,7 +92,6 @@ import org.apache.druid.query.aggregation.first.FloatFirstAggregatorFactory; import org.apache.druid.query.aggregation.last.DoubleLastAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DoubleDimensionHandler; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; @@ -1670,107 +1669,6 @@ public void testChooseFinestGranularityAllNulls() Assert.assertNull(chooseFinestGranularityHelper(input)); } - @Test - public void testCreateDimensionSchema() - { - final String dimensionName = "dim"; - DimensionHandlerUtils.registerDimensionHandlerProvider( - ExtensionDimensionHandler.TYPE_NAME, - d -> new ExtensionDimensionHandler(d) - ); - DimensionSchema stringSchema = CompactionTask.createDimensionSchema( - dimensionName, - ColumnCapabilitiesImpl.createSimpleSingleValueStringColumnCapabilities() - .setHasBitmapIndexes(true) - .setDictionaryEncoded(true) - .setDictionaryValuesUnique(true) - .setDictionaryValuesUnique(true), - DimensionSchema.MultiValueHandling.SORTED_SET - ); - - Assert.assertTrue(stringSchema instanceof StringDimensionSchema); - - DimensionSchema floatSchema = CompactionTask.createDimensionSchema( - dimensionName, - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), - null - ); - Assert.assertTrue(floatSchema instanceof FloatDimensionSchema); - - DimensionSchema doubleSchema = CompactionTask.createDimensionSchema( - dimensionName, - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE), - null - ); - Assert.assertTrue(doubleSchema instanceof DoubleDimensionSchema); - - DimensionSchema longSchema = CompactionTask.createDimensionSchema( - dimensionName, - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG), - null - ); - Assert.assertTrue(longSchema instanceof LongDimensionSchema); - - DimensionSchema extensionSchema = CompactionTask.createDimensionSchema( - dimensionName, - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities( - ColumnType.ofComplex(ExtensionDimensionHandler.TYPE_NAME) - ), - null - ); - Assert.assertTrue(extensionSchema instanceof ExtensionDimensionSchema); - } - - @Test - public void testCreateDimensionSchemaIllegalFloat() - { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("multi-value dimension [foo] is not supported for float type yet"); - CompactionTask.createDimensionSchema( - "foo", - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), - DimensionSchema.MultiValueHandling.SORTED_SET - ); - } - - @Test - public void testCreateDimensionSchemaIllegalDouble() - { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("multi-value dimension [foo] is not supported for double type yet"); - CompactionTask.createDimensionSchema( - "foo", - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE), - DimensionSchema.MultiValueHandling.SORTED_SET - ); - } - - @Test - public void testCreateDimensionSchemaIllegalLong() - { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("multi-value dimension [foo] is not supported for long type yet"); - CompactionTask.createDimensionSchema( - "foo", - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG), - DimensionSchema.MultiValueHandling.SORTED_SET - ); - } - - @Test - public void testCreateDimensionSchemaIllegalComplex() - { - expectedException.expect(ISE.class); - expectedException.expectMessage("Can't find DimensionHandlerProvider for typeName [unknown]"); - CompactionTask.createDimensionSchema( - "foo", - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities( - ColumnType.ofComplex("unknown") - ), - DimensionSchema.MultiValueHandling.SORTED_SET - ); - } - private Granularity chooseFinestGranularityHelper(List granularities) { SettableSupplier queryGranularity = new SettableSupplier<>(); diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java index bb07c669ab79..5c1e372c2751 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java @@ -29,10 +29,14 @@ import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.AutoTypeColumnSchema; +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.NestedDataDimensionSchema; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import java.util.Objects; @@ -47,7 +51,8 @@ @JsonSubTypes.Type(name = DimensionSchema.FLOAT_TYPE_NAME, value = FloatDimensionSchema.class), @JsonSubTypes.Type(name = DimensionSchema.DOUBLE_TYPE_NAME, value = DoubleDimensionSchema.class), @JsonSubTypes.Type(name = DimensionSchema.SPATIAL_TYPE_NAME, value = NewSpatialDimensionSchema.class), - @JsonSubTypes.Type(name = NestedDataComplexTypeSerde.TYPE_NAME, value = NestedDataDimensionSchema.class) + @JsonSubTypes.Type(name = NestedDataComplexTypeSerde.TYPE_NAME, value = NestedDataDimensionSchema.class), + @JsonSubTypes.Type(name = AutoTypeColumnSchema.TYPE, value = AutoTypeColumnSchema.class) }) public abstract class DimensionSchema { @@ -150,6 +155,17 @@ public boolean hasBitmapIndex() @JsonIgnore public abstract ColumnType getColumnType(); + @JsonIgnore + public DimensionHandler getDimensionHandler() + { + // default implementation for backwards compatibility + return DimensionHandlerUtils.getHandlerFromCapabilities( + name, + IncrementalIndex.makeDefaultCapabilitiesFromValueType(getColumnType()), + multiValueHandling + ); + } + @Override public boolean equals(final Object o) { diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DoubleDimensionSchema.java b/processing/src/main/java/org/apache/druid/data/input/impl/DoubleDimensionSchema.java index 01c42d031529..2bde08122149 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/DoubleDimensionSchema.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/DoubleDimensionSchema.java @@ -21,6 +21,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.DoubleDimensionHandler; import org.apache.druid.segment.column.ColumnType; public class DoubleDimensionSchema extends DimensionSchema @@ -42,4 +44,10 @@ public ColumnType getColumnType() { return ColumnType.DOUBLE; } + + @Override + public DimensionHandler getDimensionHandler() + { + return new DoubleDimensionHandler(getName()); + } } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/FloatDimensionSchema.java b/processing/src/main/java/org/apache/druid/data/input/impl/FloatDimensionSchema.java index 9514107a1728..1a9d6e73fe39 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/FloatDimensionSchema.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/FloatDimensionSchema.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.FloatDimensionHandler; import org.apache.druid.segment.column.ColumnType; public class FloatDimensionSchema extends DimensionSchema @@ -46,4 +48,10 @@ public ColumnType getColumnType() { return ColumnType.FLOAT; } + + @Override + public DimensionHandler getDimensionHandler() + { + return new FloatDimensionHandler(getName()); + } } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/LongDimensionSchema.java b/processing/src/main/java/org/apache/druid/data/input/impl/LongDimensionSchema.java index a9533f279906..e8a90a7a4818 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/LongDimensionSchema.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/LongDimensionSchema.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.LongDimensionHandler; import org.apache.druid.segment.column.ColumnType; @@ -47,4 +49,10 @@ public ColumnType getColumnType() { return ColumnType.LONG; } + + @Override + public DimensionHandler getDimensionHandler() + { + return new LongDimensionHandler(getName()); + } } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/NewSpatialDimensionSchema.java b/processing/src/main/java/org/apache/druid/data/input/impl/NewSpatialDimensionSchema.java index 3357a30d5cf4..bd1fd8292aea 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/NewSpatialDimensionSchema.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/NewSpatialDimensionSchema.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.StringDimensionHandler; import org.apache.druid.segment.column.ColumnType; import java.util.List; @@ -67,6 +69,12 @@ public ColumnType getColumnType() return ColumnType.STRING; } + @Override + public DimensionHandler getDimensionHandler() + { + return new StringDimensionHandler(getName(), getMultiValueHandling(), hasBitmapIndex(), true); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/StringDimensionSchema.java b/processing/src/main/java/org/apache/druid/data/input/impl/StringDimensionSchema.java index 10de88d5e26a..2af2fbbaac23 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/StringDimensionSchema.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/StringDimensionSchema.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.StringDimensionHandler; import org.apache.druid.segment.column.ColumnType; public class StringDimensionSchema extends DimensionSchema @@ -61,4 +63,10 @@ public ColumnType getColumnType() { return ColumnType.STRING; } + + @Override + public DimensionHandler getDimensionHandler() + { + return new StringDimensionHandler(getName(), getMultiValueHandling(), hasBitmapIndex(), false); + } } diff --git a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java index 2eabe194dd84..d19845548106 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 @@ -30,7 +30,7 @@ import org.apache.druid.math.expr.ExprType; import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.NamedFunction; -import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.nested.NestedPathFinder; import org.apache.druid.segment.nested.NestedPathPart; import org.apache.druid.segment.nested.StructuredData; @@ -47,7 +47,7 @@ public class NestedDataExpressions { public static final ExpressionType TYPE = Preconditions.checkNotNull( - ExpressionType.fromColumnType(NestedDataComplexTypeSerde.TYPE) + ExpressionType.fromColumnType(ColumnType.NESTED_DATA) ); public static class JsonObjectExprMacro implements ExprMacroTable.ExprMacro diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java new file mode 100644 index 000000000000..ee9331d31894 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java @@ -0,0 +1,612 @@ +/* + * 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; + +import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.collections.bitmap.MutableBitmap; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +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; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnFormat; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.CloseableIndexed; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexRowHolder; +import org.apache.druid.segment.nested.FieldTypeInfo; +import org.apache.druid.segment.nested.NestedPathFinder; +import org.apache.druid.segment.nested.NestedPathPart; +import org.apache.druid.segment.nested.SortedValueDictionary; +import org.apache.druid.segment.nested.StructuredData; +import org.apache.druid.segment.nested.StructuredDataProcessor; +import org.apache.druid.segment.nested.ValueDictionary; + +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; +import java.util.TreeMap; + +public class AutoTypeColumnIndexer implements DimensionIndexer +{ + protected volatile boolean hasNulls = false; + + protected SortedMap fieldIndexers = new TreeMap<>(); + protected final ValueDictionary globalDictionary = new ValueDictionary(); + + int estimatedFieldKeySize = 0; + + protected final StructuredDataProcessor indexerProcessor = new StructuredDataProcessor() + { + @Override + public ProcessedValue processField(ArrayList fieldPath, @Nullable Object fieldValue) + { + // null value is always added to the global dictionary as id 0, so we can ignore them here + if (fieldValue != null) { + 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 ProcessedValue.NULL_LITERAL; + } + + @Nullable + @Override + public ProcessedValue processArrayField( + ArrayList 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; + } + }; + + @Override + public EncodedKeyComponent processRowValsToUnsortedEncodedKeyComponent( + @Nullable Object dimValues, + boolean reportParseExceptions + ) + { + final long oldDictSizeInBytes = globalDictionary.sizeInBytes(); + final int oldFieldKeySize = estimatedFieldKeySize; + final StructuredData data; + if (dimValues == null) { + hasNulls = true; + data = null; + } else if (dimValues instanceof StructuredData) { + data = (StructuredData) dimValues; + } else { + data = new StructuredData(dimValues); + } + StructuredDataProcessor.ProcessResults info = indexerProcessor.processFields(data == null ? null : data.getValue()); + // 'raw' data is currently preserved 'as-is', and not replaced with object references to the global dictionaries + long effectiveSizeBytes = info.getEstimatedSize(); + // then, we add the delta of size change to the global dictionaries to account for any new space added by the + // 'raw' data + effectiveSizeBytes += (globalDictionary.sizeInBytes() - oldDictSizeInBytes); + effectiveSizeBytes += (estimatedFieldKeySize - oldFieldKeySize); + return new EncodedKeyComponent<>(data, effectiveSizeBytes); + } + + @Override + public void setSparseIndexed() + { + this.hasNulls = true; + } + + @Override + public StructuredData getUnsortedEncodedValueFromSorted(StructuredData sortedIntermediateValue) + { + return sortedIntermediateValue; + } + + @Override + public CloseableIndexed getSortedIndexedValues() + { + throw new UnsupportedOperationException("Not supported"); + } + + public SortedValueDictionary getSortedValueLookups() + { + return globalDictionary.getSortedCollector(); + } + + public SortedMap getFieldTypeInfo() + { + final TreeMap fields = new TreeMap<>(); + for (Map.Entry entry : fieldIndexers.entrySet()) { + // skip adding the field if no types are in the set, meaning only null values have been processed + if (!entry.getValue().getTypes().isEmpty()) { + fields.put(entry.getKey(), entry.getValue().getTypes()); + } + } + return fields; + } + + @Override + public StructuredData getMinValue() + { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public StructuredData getMaxValue() + { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public int getCardinality() + { + return globalDictionary.getCardinality(); + } + + @Override + public DimensionSelector makeDimensionSelector( + DimensionSpec spec, + IncrementalIndexRowHolder currEntry, + IncrementalIndex.DimensionDesc desc + ) + { + 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 + @Override + protected String getValue() + { + final Object o = rootLiteralSelector.getObject(); + if (o == null) { + return null; + } + return o.toString(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + }; + } + // 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(), + ColumnType.NESTED_DATA + ); + } + + @Override + public ColumnValueSelector makeColumnValueSelector( + IncrementalIndexRowHolder currEntry, + IncrementalIndex.DimensionDesc desc + ) + { + final int dimIndex = desc.getIndex(); + final ColumnValueSelector rootLiteralSelector = getRootLiteralValueSelector(currEntry, dimIndex); + if (rootLiteralSelector != null) { + return rootLiteralSelector; + } + + return new ObjectColumnSelector() + { + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + + @Nullable + @Override + public StructuredData getObject() + { + final Object[] dims = currEntry.get().getDims(); + if (0 <= dimIndex && dimIndex < dims.length) { + return (StructuredData) dims[dimIndex]; + } else { + return null; + } + } + + @Override + public Class classOfObject() + { + return StructuredData.class; + } + }; + } + + @Override + public ColumnCapabilities getColumnCapabilities() + { + return ColumnCapabilitiesImpl.createDefault() + .setType(getLogicalType()) + .setHasNulls(hasNulls); + } + + private ColumnType getLogicalType() + { + if (fieldIndexers.size() == 1 && fieldIndexers.containsKey(NestedPathFinder.JSON_PATH_ROOT)) { + FieldIndexer rootField = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); + ColumnType singleType = rootField.getTypes().getSingleType(); + return singleType == null ? ColumnType.NESTED_DATA : singleType; + } + return ColumnType.NESTED_DATA; + } + + @Override + public ColumnFormat getFormat() + { + return new Format(getLogicalType(), hasNulls); + } + + @Override + public int compareUnsortedEncodedKeyComponents( + @Nullable StructuredData lhs, + @Nullable StructuredData rhs + ) + { + return StructuredData.COMPARATOR.compare(lhs, rhs); + } + + @Override + public boolean checkUnsortedEncodedKeyComponentsEqual( + @Nullable StructuredData lhs, + @Nullable StructuredData rhs + ) + { + return Objects.equals(lhs, rhs); + } + + @Override + public int getUnsortedEncodedKeyComponentHashCode(@Nullable StructuredData key) + { + return Objects.hash(key); + } + + @Override + public Object convertUnsortedEncodedKeyComponentToActualList(StructuredData key) + { + return key; + } + + @Override + public ColumnValueSelector convertUnsortedValuesToSorted(ColumnValueSelector selectorWithUnsortedValues) + { + 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 + // typed for the merger + return new ColumnValueSelector() + { + @Override + public boolean isNull() + { + return selectorWithUnsortedValues.isNull(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + selectorWithUnsortedValues.inspectRuntimeShape(inspector); + } + + @Nullable + @Override + public StructuredData getObject() + { + return StructuredData.wrap(selectorWithUnsortedValues.getObject()); + } + + @Override + public float getFloat() + { + return selectorWithUnsortedValues.getFloat(); + } + + @Override + public double getDouble() + { + return selectorWithUnsortedValues.getDouble(); + } + + @Override + public long getLong() + { + return selectorWithUnsortedValues.getLong(); + } + + @Override + public Class classOfObject() + { + return StructuredData.class; + } + }; + } + return selectorWithUnsortedValues; + } + + @Override + public void fillBitmapsFromUnsortedEncodedKeyComponent( + StructuredData key, + int rowNum, + MutableBitmap[] bitmapIndexes, + BitmapFactory factory + ) + { + throw new UnsupportedOperationException("Not supported"); + } + + @Nullable + private ColumnValueSelector getRootLiteralValueSelector( + IncrementalIndexRowHolder currEntry, + int dimIndex + ) + { + if (fieldIndexers.size() > 1) { + return null; + } + final FieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); + if (root == null || !root.isSingleType()) { + return null; + } + return new ColumnValueSelector() + { + @Override + public boolean isNull() + { + final Object o = getObject(); + return !(o instanceof Number); + } + + @Override + public float getFloat() + { + Object value = getObject(); + if (value == null) { + return 0; + } + return ((Number) value).floatValue(); + } + + @Override + public double getDouble() + { + Object value = getObject(); + if (value == null) { + return 0; + } + return ((Number) value).doubleValue(); + } + + @Override + public long getLong() + { + Object value = getObject(); + if (value == null) { + return 0; + } + return ((Number) value).longValue(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + + @Nullable + @Override + public Object getObject() + { + final Object[] dims = currEntry.get().getDims(); + if (0 <= dimIndex && dimIndex < dims.length) { + final StructuredData data = (StructuredData) dims[dimIndex]; + if (data != null) { + return ExprEval.bestEffortOf(data.getValue()).valueOrDefault(); + } + } + + return null; + } + + @Override + public Class classOfObject() + { + return Object.class; + } + }; + } + + static class FieldIndexer + { + private final ValueDictionary valueDictionary; + private final FieldTypeInfo.MutableTypeSet typeSet; + + FieldIndexer(ValueDictionary valueDictionary) + { + this.valueDictionary = valueDictionary; + this.typeSet = new FieldTypeInfo.MutableTypeSet(); + } + + private StructuredDataProcessor.ProcessedValue processValue(ExprEval eval) + { + final ColumnType columnType = ExpressionType.toColumnType(eval.type()); + int sizeEstimate; + switch (columnType.getType()) { + case LONG: + typeSet.add(ColumnType.LONG); + sizeEstimate = valueDictionary.addLongValue(eval.asLong()); + return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate); + case DOUBLE: + typeSet.add(ColumnType.DOUBLE); + sizeEstimate = valueDictionary.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() + ); + } + switch (columnType.getElementType().getType()) { + case LONG: + typeSet.add(ColumnType.LONG_ARRAY); + final Object[] longArray = eval.asArray(); + sizeEstimate = valueDictionary.addLongArray(longArray); + return new StructuredDataProcessor.ProcessedValue<>(longArray, sizeEstimate); + case DOUBLE: + typeSet.add(ColumnType.DOUBLE_ARRAY); + final Object[] doubleArray = eval.asArray(); + sizeEstimate = valueDictionary.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 = valueDictionary.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 = valueDictionary.addStringValue(asString); + return new StructuredDataProcessor.ProcessedValue<>(asString, sizeEstimate); + default: + throw new IAE("Unhandled type: %s", columnType); + } + } + + public FieldTypeInfo.MutableTypeSet getTypes() + { + return typeSet; + } + + public boolean isSingleType() + { + return typeSet.getSingleType() != null; + } + } + + static class Format implements ColumnFormat + { + private final ColumnType logicalType; + private final boolean hasNulls; + + Format(ColumnType logicalType, boolean hasNulls) + { + this.logicalType = logicalType; + this.hasNulls = hasNulls; + } + + @Override + public ColumnType getLogicalType() + { + return logicalType; + } + + @Override + public DimensionHandler getColumnHandler(String columnName) + { + return new NestedCommonFormatColumnHandler(columnName); + } + + @Override + public DimensionSchema getColumnSchema(String columnName) + { + return new AutoTypeColumnSchema(columnName); + } + + @Override + public ColumnFormat merge(@Nullable ColumnFormat otherFormat) + { + if (otherFormat == null) { + return this; + } + if (otherFormat instanceof Format) { + final boolean otherHasNulls = ((Format) otherFormat).hasNulls; + if (!getLogicalType().equals(otherFormat.getLogicalType())) { + return new Format(ColumnType.NESTED_DATA, hasNulls || otherHasNulls); + } + return new Format(logicalType, hasNulls || otherHasNulls); + } + throw new ISE( + "Cannot merge columns of type[%s] and format[%s] and with [%s] and [%s]", + logicalType, + this.getClass().getName(), + otherFormat.getLogicalType(), + otherFormat.getClass().getName() + ); + } + + @Override + public ColumnCapabilities toColumnCapabilities() + { + return ColumnCapabilitiesImpl.createDefault() + .setType(logicalType) + .setHasNulls(hasNulls); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java new file mode 100644 index 000000000000..060f6b68ccfb --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java @@ -0,0 +1,457 @@ +/* + * 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; + +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; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.column.ColumnDescriptor; +import org.apache.druid.segment.column.ColumnType; +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.nested.DictionaryIdLookup; +import org.apache.druid.segment.nested.FieldTypeInfo; +import org.apache.druid.segment.nested.NestedCommonFormatColumn; +import org.apache.druid.segment.nested.NestedCommonFormatColumnSerializer; +import org.apache.druid.segment.nested.NestedDataColumnSerializer; +import org.apache.druid.segment.nested.NestedPathFinder; +import org.apache.druid.segment.nested.ScalarDoubleColumnSerializer; +import org.apache.druid.segment.nested.ScalarLongColumnSerializer; +import org.apache.druid.segment.nested.ScalarStringColumnSerializer; +import org.apache.druid.segment.nested.SortedValueDictionary; +import org.apache.druid.segment.nested.VariantArrayColumnSerializer; +import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteOrder; +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; + +/** + * Column merger for {@link AutoTypeColumnIndexer} to eventually produce some form of + * {@link NestedCommonFormatColumn}. + *

+ * Depending on the types of values encountered + */ +public class AutoTypeColumnMerger implements DimensionMergerV9 +{ + private static final Logger log = new Logger(NestedDataColumnMerger.class); + + public static final Comparator> STRING_MERGING_COMPARATOR = + SimpleDictionaryMergingIterator.makePeekingComparator(); + public static final Comparator> LONG_MERGING_COMPARATOR = + SimpleDictionaryMergingIterator.makePeekingComparator(); + public static final Comparator> DOUBLE_MERGING_COMPARATOR = + SimpleDictionaryMergingIterator.makePeekingComparator(); + + private final String name; + private final IndexSpec indexSpec; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private final ProgressIndicator progressIndicator; + private final Closer closer; + private NestedCommonFormatColumnSerializer serializer; + + private ColumnType logicalType; + + public AutoTypeColumnMerger( + String name, + IndexSpec indexSpec, + SegmentWriteOutMedium segmentWriteOutMedium, + ProgressIndicator progressIndicator, + Closer closer + ) + { + + this.name = name; + this.indexSpec = indexSpec; + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.progressIndicator = progressIndicator; + this.closer = closer; + } + + @Override + public void writeMergedValueDictionary(List adapters) throws IOException + { + try { + long dimStartTime = System.currentTimeMillis(); + + int numMergeIndex = 0; + SortedValueDictionary sortedLookup = null; + final Indexed[] sortedLookups = new Indexed[adapters.size()]; + final Indexed[] sortedLongLookups = new Indexed[adapters.size()]; + final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()]; + final Indexed[] sortedArrayLookups = new Indexed[adapters.size()]; + + final SortedMap mergedFields = new TreeMap<>(); + + for (int i = 0; i < adapters.size(); i++) { + final IndexableAdapter adapter = adapters.get(i); + final IndexableAdapter.NestedColumnMergable mergable = closer.register( + adapter.getNestedColumnMergeables(name) + ); + final SortedValueDictionary dimValues = mergable.getValueDictionary(); + + boolean allNulls = dimValues == null || dimValues.allNull(); + sortedLookup = dimValues; + if (!allNulls) { + mergable.mergeFieldsInto(mergedFields); + sortedLookups[i] = dimValues.getSortedStrings(); + sortedLongLookups[i] = dimValues.getSortedLongs(); + sortedDoubleLookups[i] = dimValues.getSortedDoubles(); + sortedArrayLookups[i] = dimValues.getSortedArrays(); + numMergeIndex++; + } + } + + // check to see if we can specialize the serializer after merging all the adapters + if (isSingleTypeRoot(mergedFields)) { + logicalType = mergedFields.get(NestedPathFinder.JSON_PATH_ROOT).getSingleType(); + switch (logicalType.getType()) { + case LONG: + final ScalarLongColumnSerializer longSerializer = new ScalarLongColumnSerializer( + name, + indexSpec, + segmentWriteOutMedium, + progressIndicator, + closer + ); + serializer = longSerializer; + break; + case DOUBLE: + final ScalarDoubleColumnSerializer doubleSerializer = new ScalarDoubleColumnSerializer( + name, + indexSpec, + segmentWriteOutMedium, + progressIndicator, + closer + ); + serializer = doubleSerializer; + break; + case STRING: + final ScalarStringColumnSerializer stringSerializer = new ScalarStringColumnSerializer( + name, + indexSpec, + segmentWriteOutMedium, + progressIndicator, + closer + ); + serializer = stringSerializer; + break; + case ARRAY: + final VariantArrayColumnSerializer arraySerializer = new VariantArrayColumnSerializer( + name, + indexSpec, + segmentWriteOutMedium, + progressIndicator, + closer + ); + serializer = arraySerializer; + break; + default: + throw new ISE( + "How did we get here? Column [%s] with type [%s] does not have specialized serializer", + name, + logicalType + ); + } + } else { + // all the bells and whistles + logicalType = ColumnType.NESTED_DATA; + final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer( + name, + indexSpec, + segmentWriteOutMedium, + progressIndicator, + closer + ); + serializer = defaultSerializer; + } + + serializer.openDictionaryWriter(); + serializer.serializeFields(mergedFields); + + int stringCardinality; + int longCardinality; + int doubleCardinality; + int arrayCardinality; + if (numMergeIndex == 1) { + serializer.serializeDictionaries( + sortedLookup.getSortedStrings(), + sortedLookup.getSortedLongs(), + sortedLookup.getSortedDoubles(), + () -> new ArrayDictionaryMergingIterator( + sortedArrayLookups, + serializer.getGlobalLookup() + ) + ); + stringCardinality = sortedLookup.getStringCardinality(); + longCardinality = sortedLookup.getLongCardinality(); + doubleCardinality = sortedLookup.getDoubleCardinality(); + arrayCardinality = sortedLookup.getArrayCardinality(); + } else { + final SimpleDictionaryMergingIterator stringIterator = new SimpleDictionaryMergingIterator<>( + sortedLookups, + STRING_MERGING_COMPARATOR + ); + final SimpleDictionaryMergingIterator longIterator = new SimpleDictionaryMergingIterator<>( + sortedLongLookups, + LONG_MERGING_COMPARATOR + ); + final SimpleDictionaryMergingIterator doubleIterator = new SimpleDictionaryMergingIterator<>( + sortedDoubleLookups, + DOUBLE_MERGING_COMPARATOR + ); + final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator( + sortedArrayLookups, + serializer.getGlobalLookup() + ); + serializer.serializeDictionaries( + () -> stringIterator, + () -> longIterator, + () -> doubleIterator, + () -> arrayIterator + ); + stringCardinality = stringIterator.getCardinality(); + longCardinality = longIterator.getCardinality(); + doubleCardinality = doubleIterator.getCardinality(); + arrayCardinality = arrayIterator.getCardinality(); + } + // open main serializer after dictionaries have been serialized. we can't do this earlier since we don't know + // dictionary cardinalities until after merging them, and we need to know that to configure compression and such + // which depend on knowing the highest value + serializer.open(); + + 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 + ); + } + catch (IOException ioe) { + log.error(ioe, "Failed to merge dictionary for column [%s]", name); + throw ioe; + } + } + + private static boolean isSingleTypeRoot(SortedMap mergedFields) + { + return mergedFields.size() == 1 + && mergedFields.get(NestedPathFinder.JSON_PATH_ROOT) != null + && mergedFields.get(NestedPathFinder.JSON_PATH_ROOT).getSingleType() != null; + } + + @Override + public ColumnValueSelector convertSortedSegmentRowValuesToMergedRowValues( + int segmentIndex, + ColumnValueSelector source + ) + { + return source; + } + + @Override + public void processMergedRow(ColumnValueSelector selector) throws IOException + { + serializer.serialize(selector); + } + + @Override + public void writeIndexes(@Nullable List segmentRowNumConversions) + { + // fields write their own indexes + } + + @Override + public boolean hasOnlyNulls() + { + return false; + } + + @Override + public ColumnDescriptor makeColumnDescriptor() + { + ColumnDescriptor.Builder descriptorBuilder = new ColumnDescriptor.Builder(); + + final NestedCommonFormatColumnPartSerde partSerde = NestedCommonFormatColumnPartSerde.serializerBuilder() + .withLogicalType(logicalType) + .withHasNulls(serializer.hasNulls()) + .withByteOrder(ByteOrder.nativeOrder()) + .withBitmapSerdeFactory(indexSpec.getBitmapSerdeFactory()) + .withSerializer(serializer) + .build(); + descriptorBuilder.setValueType(ValueType.COMPLEX) // this doesn't really matter... you could say.. its complicated.. + .setHasMultipleValues(false) + .addSerde(partSerde); + return descriptorBuilder.build(); + } + + public static class ArrayDictionaryMergingIterator implements Iterator + { + private static final Comparator> PEEKING_ITERATOR_COMPARATOR = + (lhs, rhs) -> FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR.compare(lhs.peek(), rhs.peek()); + + protected final PriorityQueue> pQueue; + private final Iterable[] dimValueLookups; + private final DictionaryIdLookup idLookup; + + protected int counter; + private boolean initialized; + + public ArrayDictionaryMergingIterator(Iterable[] dimValueLookups, DictionaryIdLookup 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 dimValueLookup : dimValueLookups) { + if (dimValueLookup == null) { + continue; + } + final PeekingIterator 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 smallest = pQueue.remove(); + if (smallest == null) { + throw new NoSuchElementException(); + } + final int[] value = smallest.next(); + if (smallest.hasNext()) { + pQueue.add(smallest); + } + + while (!pQueue.isEmpty() && Arrays.equals(value, pQueue.peek().peek())) { + PeekingIterator same = pQueue.remove(); + same.next(); + if (same.hasNext()) { + pQueue.add(same); + } + } + counter++; + + return value; + } + + public int getCardinality() + { + return counter; + } + + @Override + public void remove() + { + throw new UnsupportedOperationException("remove"); + } + } + + public static class IdLookupArrayIterator implements Iterator + { + private final DictionaryIdLookup idLookup; + private final Iterator delegate; + + public IdLookupArrayIterator( + DictionaryIdLookup idLookup, + Iterator delegate + ) + { + this.idLookup = idLookup; + this.delegate = delegate; + } + + @Override + public boolean hasNext() + { + return delegate.hasNext(); + } + + @Override + public int[] next() + { + final Object[] next = delegate.next(); + if (next == null) { + return null; + } + final int[] 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; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java new file mode 100644 index 000000000000..dff680d8ae6e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java @@ -0,0 +1,90 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.nested.NestedCommonFormatColumn; +import org.apache.druid.segment.nested.NestedCommonFormatColumnSerializer; +import org.apache.druid.segment.nested.NestedDataColumnSerializer; +import org.apache.druid.segment.nested.ScalarDoubleColumnSerializer; +import org.apache.druid.segment.nested.ScalarLongColumnSerializer; +import org.apache.druid.segment.nested.ScalarStringColumnSerializer; +import org.apache.druid.segment.nested.StructuredData; +import org.apache.druid.segment.nested.VariantArrayColumnSerializer; +import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; + +/** + * Common {@link DimensionSchema} for ingestion of 'standard' Druid built-in {@link ColumnType} datatypes. + * + * Automatically determines the most appropriate type for the given input data, able to produce columns of type: + * {@link ColumnType#STRING} + * {@link ColumnType#STRING_ARRAY} + * {@link ColumnType#LONG} + * {@link ColumnType#LONG_ARRAY} + * {@link ColumnType#DOUBLE} + * {@link ColumnType#DOUBLE_ARRAY} + * {@link ColumnType#NESTED_DATA} + * + * and includes bitmap value set indexes. Input of mixed type will be stored as {@link ColumnType#NESTED_DATA}. + * + * @see AutoTypeColumnIndexer + * @see AutoTypeColumnMerger + * @see NestedCommonFormatColumnSerializer + * @see VariantArrayColumnSerializer + * @see ScalarDoubleColumnSerializer + * @see ScalarLongColumnSerializer + * @see NestedDataColumnSerializer + * @see ScalarStringColumnSerializer + * @see NestedCommonFormatColumnPartSerde + * @see NestedCommonFormatColumn + */ +public class AutoTypeColumnSchema extends DimensionSchema +{ + public static final String TYPE = "auto"; + + @JsonCreator + public AutoTypeColumnSchema( + @JsonProperty("name") String name + ) + { + super(name, null, true); + } + + @Override + public String getTypeName() + { + return TYPE; + } + + @Override + public ColumnType getColumnType() + { + return ColumnType.NESTED_DATA; + } + + @Override + public DimensionHandler getDimensionHandler() + { + return new NestedCommonFormatColumnHandler(getName()); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java index 21e637727343..634ee0d5fc62 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java @@ -22,7 +22,9 @@ import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.column.CapabilitiesBasedFormat; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexRowHolder; @@ -174,7 +176,6 @@ EncodedKeyComponent processRowValsToUnsortedEncodedKeyC */ CloseableIndexed getSortedIndexedValues(); - /** * Get the minimum dimension value seen by this indexer. * @@ -237,9 +238,9 @@ ColumnValueSelector makeColumnValueSelector( ColumnCapabilities getColumnCapabilities(); - default ColumnCapabilities getHandlerCapabilities() + default ColumnFormat getFormat() { - return getColumnCapabilities(); + return CapabilitiesBasedFormat.forColumnIndexer(getColumnCapabilities()); } /** diff --git a/processing/src/main/java/org/apache/druid/segment/IndexIO.java b/processing/src/main/java/org/apache/druid/segment/IndexIO.java index 5477e8b616fb..c88edbb122d3 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexIO.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexIO.java @@ -794,9 +794,7 @@ ColumnHolder deserializeColumn( SmooshedFileMapper smooshedFiles ) throws IOException { - ColumnDescriptor serde = mapper.readValue( - SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class - ); + ColumnDescriptor serde = mapper.readValue(SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class); return serde.read(byteBuffer, columnConfig, smooshedFiles); } } diff --git a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java index bae6d94fbbc6..4c7d7f757c03 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java @@ -40,10 +40,8 @@ import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ColumnCapabilities.CoercionLogic; -import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnDescriptor; +import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.ValueType; @@ -82,7 +80,6 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.TreeMap; import java.util.function.Function; @@ -93,75 +90,6 @@ public class IndexMergerV9 implements IndexMerger { private static final Logger log = new Logger(IndexMergerV9.class); - // merge logic for the state capabilities will be in after incremental index is persisted - public static final ColumnCapabilities.CoercionLogic DIMENSION_CAPABILITY_MERGE_LOGIC = - new ColumnCapabilities.CoercionLogic() - { - @Override - public boolean dictionaryEncoded() - { - return true; - } - - @Override - public boolean dictionaryValuesSorted() - { - return true; - } - - @Override - public boolean dictionaryValuesUnique() - { - return true; - } - - @Override - public boolean multipleValues() - { - return false; - } - - @Override - public boolean hasNulls() - { - return false; - } - }; - - public static final ColumnCapabilities.CoercionLogic METRIC_CAPABILITY_MERGE_LOGIC = - new ColumnCapabilities.CoercionLogic() - { - @Override - public boolean dictionaryEncoded() - { - return false; - } - - @Override - public boolean dictionaryValuesSorted() - { - return false; - } - - @Override - public boolean dictionaryValuesUnique() - { - return false; - } - - @Override - public boolean multipleValues() - { - return false; - } - - @Override - public boolean hasNulls() - { - return false; - } - }; - private final ObjectMapper mapper; private final IndexIO indexIO; private final SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory; @@ -263,11 +191,11 @@ private File makeIndexFiles( log.debug("Completed factory.json in %,d millis", System.currentTimeMillis() - startTime); progress.progress(); - final Map> metricTypes = new TreeMap<>(Comparators.naturalNullsFirst()); - final List dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size()); - mergeCapabilities(adapters, mergedDimensions, metricTypes, dimCapabilities); + final Map metricFormats = new TreeMap<>(Comparators.naturalNullsFirst()); + final List dimFormats = Lists.newArrayListWithCapacity(mergedDimensions.size()); + mergeFormat(adapters, mergedDimensions, metricFormats, dimFormats); - final Map handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities); + final Map handlers = makeDimensionHandlers(mergedDimensions, dimFormats); final List mergers = new ArrayList<>(); for (int i = 0; i < mergedDimensions.size(); i++) { DimensionHandler handler = handlers.get(mergedDimensions.get(i)); @@ -275,7 +203,7 @@ private File makeIndexFiles( handler.makeMerger( indexSpec, segmentWriteOutMedium, - dimCapabilities.get(i), + dimFormats.get(i).toColumnCapabilities(), progress, closer ) @@ -301,7 +229,7 @@ private File makeIndexFiles( closer.register(timeAndDimsIterator); final GenericColumnSerializer timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec); final ArrayList metricWriters = - setupMetricsWriters(segmentWriteOutMedium, mergedMetrics, metricTypes, indexSpec); + setupMetricsWriters(segmentWriteOutMedium, mergedMetrics, metricFormats, indexSpec); IndexMergeResult indexMergeResult = mergeIndexesAndWriteColumns( adapters, progress, @@ -320,7 +248,7 @@ private File makeIndexFiles( v9Smoosher, progress, mergedMetrics, - metricTypes, + metricFormats, metricWriters, indexSpec ); @@ -335,7 +263,7 @@ private File makeIndexFiles( // shouldStore AND hasOnlyNulls ColumnDescriptor columnDesc = ColumnDescriptor .builder() - .setValueType(dimCapabilities.get(i).getType()) + .setValueType(dimFormats.get(i).getLogicalType().getType()) .addSerde(new NullColumnPartSerde(indexMergeResult.rowCount, indexSpec.getBitmapSerdeFactory())) .build(); makeColumn(v9Smoosher, mergedDimensions.get(i), columnDesc); @@ -493,7 +421,7 @@ private void makeMetricsColumns( final FileSmoosher v9Smoosher, final ProgressIndicator progress, final List mergedMetrics, - final Map> metricsTypes, + final Map metricsTypes, final List metWriters, final IndexSpec indexSpec ) throws IOException @@ -508,7 +436,7 @@ private void makeMetricsColumns( GenericColumnSerializer writer = metWriters.get(i); final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); - TypeSignature type = metricsTypes.get(metric); + TypeSignature type = metricsTypes.get(metric).getLogicalType(); switch (type.getType()) { case LONG: builder.setValueType(ValueType.LONG); @@ -761,14 +689,14 @@ private GenericColumnSerializer setupTimeWriter( private ArrayList setupMetricsWriters( final SegmentWriteOutMedium segmentWriteOutMedium, final List mergedMetrics, - final Map> metricsTypes, + final Map metricsTypes, final IndexSpec indexSpec ) throws IOException { ArrayList metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); for (String metric : mergedMetrics) { - TypeSignature type = metricsTypes.get(metric); + TypeSignature type = metricsTypes.get(metric).getLogicalType(); GenericColumnSerializer writer; switch (type.getType()) { case LONG: @@ -890,100 +818,30 @@ private void writeDimValuesAndSetupDimConversion( progress.stopSection(section); } - private void mergeCapabilities( + private void mergeFormat( final List adapters, final List mergedDimensions, - final Map> metricTypes, - final List dimCapabilities + final Map metricTypes, + final List dimFormats ) { - final Map capabilitiesMap = new HashMap<>(); + final Map columnFormats = new HashMap<>(); for (IndexableAdapter adapter : adapters) { for (String dimension : adapter.getDimensionNames()) { - ColumnCapabilities capabilities = adapter.getCapabilities(dimension); - capabilitiesMap.compute(dimension, (d, existingCapabilities) -> - mergeCapabilities(capabilities, existingCapabilities, DIMENSION_CAPABILITY_MERGE_LOGIC) - ); + ColumnFormat format = adapter.getFormat(dimension); + columnFormats.compute(dimension, (d, existingFormat) -> existingFormat == null ? format : format.merge(existingFormat)); } for (String metric : adapter.getMetricNames()) { - final ColumnCapabilities capabilities = adapter.getCapabilities(metric); - final ColumnCapabilities merged = capabilitiesMap.compute(metric, (m, existingCapabilities) -> - mergeCapabilities(capabilities, existingCapabilities, METRIC_CAPABILITY_MERGE_LOGIC) + final ColumnFormat format = adapter.getFormat(metric); + final ColumnFormat merged = columnFormats.compute(metric, (m, existingFormat) -> + existingFormat == null ? format : format.merge(existingFormat) ); + metricTypes.put(metric, merged); } } for (String dim : mergedDimensions) { - dimCapabilities.add(capabilitiesMap.get(dim)); - } - } - - /** - * Creates a merged columnCapabilities to merge two queryableIndexes. - * This method first snapshots a pair of capabilities and then merges them. - */ - @Nullable - private static ColumnCapabilitiesImpl mergeCapabilities( - @Nullable final ColumnCapabilities capabilities, - @Nullable final ColumnCapabilities other, - CoercionLogic coercionLogic - ) - { - ColumnCapabilitiesImpl merged = ColumnCapabilitiesImpl.snapshot(capabilities, coercionLogic); - ColumnCapabilitiesImpl otherSnapshot = ColumnCapabilitiesImpl.snapshot(other, coercionLogic); - if (merged == null) { - return otherSnapshot; - } else if (otherSnapshot == null) { - return merged; - } - - throwIfTypeNotMatchToMerge(merged, otherSnapshot); - - merged.setDictionaryEncoded(merged.isDictionaryEncoded().or(otherSnapshot.isDictionaryEncoded()).isTrue()); - merged.setHasMultipleValues(merged.hasMultipleValues().or(otherSnapshot.hasMultipleValues()).isTrue()); - merged.setDictionaryValuesSorted( - merged.areDictionaryValuesSorted().and(otherSnapshot.areDictionaryValuesSorted()).isTrue() - ); - merged.setDictionaryValuesUnique( - merged.areDictionaryValuesUnique().and(otherSnapshot.areDictionaryValuesUnique()).isTrue() - ); - merged.setHasNulls(merged.hasNulls().or(other.hasNulls()).isTrue()); - // When merging persisted queryableIndexes in the same ingestion job, - // all queryableIndexes should have the exact same hasBitmapIndexes flag set which is set in the ingestionSpec. - // One exception is null-only columns as they always do NOT have bitmap indexes no matter whether the flag is set - // in the ingestionSpec. As a result, the mismatch checked in the if clause below can happen - // when one of the columnCapability is from a real column and another is from a null-only column. - // See NullColumnPartSerde for how columnCapability is created for null-only columns. - // When the mismatch is found, we prefer the flag set in the ingestionSpec over - // the columnCapability of null-only columns. - if (merged.hasBitmapIndexes() != otherSnapshot.hasBitmapIndexes()) { - merged.setHasBitmapIndexes(false); - } - if (merged.hasSpatialIndexes() != otherSnapshot.hasSpatialIndexes()) { - merged.setHasSpatialIndexes(merged.hasSpatialIndexes() || otherSnapshot.hasSpatialIndexes()); - } - merged.setFilterable(merged.isFilterable() && otherSnapshot.isFilterable()); - - return merged; - } - - private static void throwIfTypeNotMatchToMerge(ColumnCapabilitiesImpl c1, ColumnCapabilitiesImpl c2) - { - if (!Objects.equals(c1.getType(), c2.getType()) - || !Objects.equals(c1.getElementType(), c2.getElementType())) { - final String mergedType = c1.getType() == null ? null : c1.asTypeString(); - final String otherType = c2.getType() == null ? null : c2.asTypeString(); - throw new ISE( - "Cannot merge columns of type[%s] and [%s]", - mergedType, - otherType - ); - } else if (!Objects.equals(c1.getComplexTypeName(), c2.getComplexTypeName())) { - throw new ISE( - "Cannot merge columns of type[%s] and [%s]", - c1.getComplexTypeName(), - c2.getComplexTypeName() - ); + dimFormats.add(columnFormats.get(dim)); } } @@ -1311,17 +1169,13 @@ private File merge( private Map makeDimensionHandlers( final List mergedDimensions, - final List dimCapabilities + final List dimFormats ) { Map handlers = new LinkedHashMap<>(); for (int i = 0; i < mergedDimensions.size(); i++) { - ColumnCapabilities capabilities = ColumnCapabilitiesImpl.snapshot( - dimCapabilities.get(i), - DIMENSION_CAPABILITY_MERGE_LOGIC - ); String dimName = mergedDimensions.get(i); - DimensionHandler handler = DimensionHandlerUtils.getHandlerFromCapabilities(dimName, capabilities, null); + DimensionHandler handler = dimFormats.get(i).getColumnHandler(dimName); handlers.put(dimName, handler); } return handlers; diff --git a/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java index 96bc27be5812..c6eff05f49fb 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java @@ -20,13 +20,21 @@ package org.apache.druid.segment; import com.google.errorprone.annotations.MustBeClosed; +import org.apache.druid.segment.column.CapabilitiesBasedFormat; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.data.BitmapValues; import org.apache.druid.segment.data.CloseableIndexed; +import org.apache.druid.segment.nested.FieldTypeInfo; +import org.apache.druid.segment.nested.SortedValueDictionary; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; import java.util.List; +import java.util.Map; +import java.util.SortedMap; /** * An adapter to an index @@ -45,11 +53,60 @@ public interface IndexableAdapter @Nullable > CloseableIndexed getDimValueLookup(String dimension); + @Nullable + NestedColumnMergable getNestedColumnMergeables(String column); + TransformableRowIterator getRows(); BitmapValues getBitmapValues(String dimension, int dictId); ColumnCapabilities getCapabilities(String column); + default ColumnFormat getFormat(String column) + { + return new CapabilitiesBasedFormat(getCapabilities(column)); + } + Metadata getMetadata(); + + class NestedColumnMergable implements Closeable + { + private final SortedValueDictionary valueDictionary; + private final SortedMap fields; + + public NestedColumnMergable( + SortedValueDictionary valueDictionary, + SortedMap fields + ) + { + this.valueDictionary = valueDictionary; + this.fields = fields; + } + + @Nullable + public SortedValueDictionary getValueDictionary() + { + return valueDictionary; + } + + public void mergeFieldsInto(SortedMap mergeInto) + { + for (Map.Entry entry : fields.entrySet()) { + final String fieldPath = entry.getKey(); + final FieldTypeInfo.MutableTypeSet types = entry.getValue(); + mergeInto.compute(fieldPath, (k, v) -> { + if (v == null) { + return new FieldTypeInfo.MutableTypeSet(types.getByteValue()); + } + return v.merge(types.getByteValue()); + }); + } + } + + @Override + public void close() throws IOException + { + valueDictionary.close(); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java b/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java new file mode 100644 index 000000000000..9f6bd439efad --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java @@ -0,0 +1,105 @@ +/* + * 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; + +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.nested.StructuredData; +import org.apache.druid.segment.selector.settable.SettableColumnValueSelector; +import org.apache.druid.segment.selector.settable.SettableObjectColumnValueSelector; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import java.util.Comparator; + +public class NestedCommonFormatColumnHandler implements DimensionHandler +{ + private static Comparator COMPARATOR = (s1, s2) -> + StructuredData.COMPARATOR.compare( + StructuredData.wrap(s1.getObject()), + StructuredData.wrap(s2.getObject()) + ); + + private final String name; + + public NestedCommonFormatColumnHandler(String name) + { + this.name = name; + } + + @Override + public String getDimensionName() + { + return name; + } + + @Override + public DimensionSpec getDimensionSpec() + { + return new DefaultDimensionSpec(name, name, ColumnType.NESTED_DATA); + } + + @Override + public DimensionSchema getDimensionSchema(ColumnCapabilities capabilities) + { + return new AutoTypeColumnSchema(name); + } + + @Override + public DimensionIndexer makeIndexer(boolean useMaxMemoryEstimates) + { + return new AutoTypeColumnIndexer(); + } + + @Override + public DimensionMergerV9 makeMerger( + IndexSpec indexSpec, + SegmentWriteOutMedium segmentWriteOutMedium, + ColumnCapabilities capabilities, + ProgressIndicator progress, + Closer closer + ) + { + return new AutoTypeColumnMerger(name, indexSpec, segmentWriteOutMedium, progress, closer); + } + + @Override + public int getLengthOfEncodedKeyComponent(StructuredData dimVals) + { + // this is called in one place, OnheapIncrementalIndex, where returning 0 here means the value is null + // so the actual value we return here doesn't matter. we should consider refactoring this to a boolean + return 1; + } + + @Override + public Comparator getEncodedValueSelectorComparator() + { + return COMPARATOR; + } + + @Override + public SettableColumnValueSelector makeNewSettableEncodedValueSelector() + { + return new SettableObjectColumnValueSelector(); + } +} 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 9b0de5fcdcb7..ec17ef6c8d75 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java @@ -29,22 +29,22 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexRowHolder; -import org.apache.druid.segment.nested.GlobalDictionarySortedCollector; -import org.apache.druid.segment.nested.GlobalDimensionDictionary; +import org.apache.druid.segment.nested.FieldTypeInfo; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; -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.SortedValueDictionary; import org.apache.druid.segment.nested.StructuredData; import org.apache.druid.segment.nested.StructuredDataProcessor; +import org.apache.druid.segment.nested.ValueDictionary; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -53,10 +53,12 @@ public class NestedDataColumnIndexer implements DimensionIndexer { + private static final ColumnFormat FORMAT = new NestedDataComplexTypeSerde.NestedColumnFormatV4(); + protected volatile boolean hasNulls = false; protected SortedMap fieldIndexers = new TreeMap<>(); - protected final GlobalDimensionDictionary globalDictionary = new GlobalDimensionDictionary(); + protected final ValueDictionary globalDictionary = new ValueDictionary(); int estimatedFieldKeySize = 0; @@ -87,17 +89,7 @@ public ProcessedValue processArrayField( @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); - } + // classic nested data column indexer does not handle arrays return null; } }; @@ -175,15 +167,6 @@ public DimensionSelector makeDimensionSelector( 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 @@ -208,7 +191,7 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) throw new UOE( "makeDimensionSelector is not supported, column [%s] is [%s] typed and should only use makeColumnValueSelector", spec.getOutputName(), - NestedDataComplexTypeSerde.TYPE + ColumnType.NESTED_DATA ); } @@ -251,31 +234,47 @@ public Class classOfObject() } }; } - - @Override - public ColumnCapabilities getColumnCapabilities() + private ColumnType getLogicalType() { if (fieldIndexers.size() == 1 && fieldIndexers.containsKey(NestedPathFinder.JSON_PATH_ROOT)) { FieldIndexer rootField = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); - if (rootField.isSingleType()) { - return ColumnCapabilitiesImpl.createDefault() - .setType(rootField.getTypes().getSingleType()) - .setHasNulls(hasNulls); - } + ColumnType singleType = rootField.getTypes().getSingleType(); + return singleType == null ? ColumnType.NESTED_DATA : singleType; } - return ColumnCapabilitiesImpl.createDefault() - .setType(NestedDataComplexTypeSerde.TYPE) - .setHasNulls(hasNulls); + return ColumnType.NESTED_DATA; } @Override - public ColumnCapabilities getHandlerCapabilities() + public ColumnCapabilities getColumnCapabilities() { return ColumnCapabilitiesImpl.createDefault() - .setType(NestedDataComplexTypeSerde.TYPE) + .setType(getLogicalType()) .setHasNulls(hasNulls); } + @Override + public ColumnFormat getFormat() + { + return FORMAT; + } + + public SortedValueDictionary getSortedValueLookups() + { + return globalDictionary.getSortedCollector(); + } + + public SortedMap getFieldTypeInfo() + { + TreeMap fields = new TreeMap<>(); + for (Map.Entry entry : fieldIndexers.entrySet()) { + // skip adding the field if no types are in the set, meaning only null values have been processed + if (!entry.getValue().getTypes().isEmpty()) { + fields.put(entry.getKey(), entry.getValue().getTypes()); + } + } + return fields; + } + @Override public int compareUnsortedEncodedKeyComponents( @Nullable StructuredData lhs, @@ -374,21 +373,6 @@ public void fillBitmapsFromUnsortedEncodedKeyComponent( throw new UnsupportedOperationException("Not supported"); } - public void mergeFields(SortedMap mergedFields) - { - for (Map.Entry entry : fieldIndexers.entrySet()) { - // skip adding the field if no types are in the set, meaning only null values have been processed - if (!entry.getValue().getTypes().isEmpty()) { - mergedFields.put(entry.getKey(), entry.getValue().getTypes()); - } - } - } - - public GlobalDictionarySortedCollector getSortedCollector() - { - return globalDictionary.getSortedCollector(); - } - @Nullable private ColumnValueSelector getRootLiteralValueSelector( IncrementalIndexRowHolder currEntry, @@ -472,13 +456,13 @@ public Class classOfObject() static class FieldIndexer { - private final GlobalDimensionDictionary globalDimensionDictionary; - private final NestedFieldTypeInfo.MutableTypeSet typeSet; + private final ValueDictionary valueDictionary; + private final FieldTypeInfo.MutableTypeSet typeSet; - FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary) + FieldIndexer(ValueDictionary valueDictionary) { - this.globalDimensionDictionary = globalDimensionDictionary; - this.typeSet = new NestedFieldTypeInfo.MutableTypeSet(); + this.valueDictionary = valueDictionary; + this.typeSet = new FieldTypeInfo.MutableTypeSet(); } private StructuredDataProcessor.ProcessedValue processValue(ExprEval eval) @@ -488,54 +472,23 @@ private StructuredDataProcessor.ProcessedValue processValue(ExprEval eval) switch (columnType.getType()) { case LONG: typeSet.add(ColumnType.LONG); - sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong()); + sizeEstimate = valueDictionary.addLongValue(eval.asLong()); return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate); case DOUBLE: typeSet.add(ColumnType.DOUBLE); - sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble()); + sizeEstimate = valueDictionary.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() - ); - } - 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); + sizeEstimate = valueDictionary.addStringValue(asString); return new StructuredDataProcessor.ProcessedValue<>(asString, sizeEstimate); default: throw new IAE("Unhandled type: %s", columnType); } } - public NestedFieldTypeInfo.MutableTypeSet getTypes() + public FieldTypeInfo.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 95a52b83c2ad..caa6552d53f9 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java @@ -19,39 +19,24 @@ 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; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.segment.column.BaseColumn; 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.FieldTypeInfo; +import org.apache.druid.segment.nested.NestedDataColumnSerializerV4; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; -import org.apache.druid.segment.nested.NestedFieldTypeInfo; +import org.apache.druid.segment.nested.SortedValueDictionary; 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; @@ -98,39 +83,35 @@ public void writeMergedValueDictionary(List adapters) throws I long dimStartTime = System.currentTimeMillis(); int numMergeIndex = 0; - GlobalDictionarySortedCollector sortedLookup = null; + SortedValueDictionary sortedLookup = null; final Indexed[] sortedLookups = new Indexed[adapters.size()]; final Indexed[] sortedLongLookups = new Indexed[adapters.size()]; final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()]; - final Iterable[] sortedArrayLookups = new Iterable[adapters.size()]; - final SortedMap mergedFields = new TreeMap<>(); + final SortedMap mergedFields = new TreeMap<>(); for (int i = 0; i < adapters.size(); i++) { final IndexableAdapter adapter = adapters.get(i); - final GlobalDictionarySortedCollector dimValues; - if (adapter instanceof IncrementalIndexAdapter) { - dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields); - } else if (adapter instanceof QueryableIndexIndexableAdapter) { - dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields); - } else { - throw new ISE("Unable to merge columns of unsupported adapter [%s]", adapter.getClass()); - } + + final IndexableAdapter.NestedColumnMergable mergable = closer.register( + adapter.getNestedColumnMergeables(name) + ); + final SortedValueDictionary dimValues = mergable.getValueDictionary(); boolean allNulls = dimValues == null || dimValues.allNull(); sortedLookup = dimValues; if (!allNulls) { + mergable.mergeFieldsInto(mergedFields); sortedLookups[i] = dimValues.getSortedStrings(); sortedLongLookups[i] = dimValues.getSortedLongs(); sortedDoubleLookups[i] = dimValues.getSortedDoubles(); - sortedArrayLookups[i] = dimValues.getSortedArrays(); numMergeIndex++; } } descriptorBuilder = new ColumnDescriptor.Builder(); - final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer( + final NestedDataColumnSerializerV4 defaultSerializer = new NestedDataColumnSerializerV4( name, indexSpec, segmentWriteOutMedium, @@ -153,21 +134,15 @@ public void writeMergedValueDictionary(List adapters) throws I int stringCardinality; int longCardinality; int doubleCardinality; - int arrayCardinality; if (numMergeIndex == 1) { defaultSerializer.serializeDictionaries( sortedLookup.getSortedStrings(), sortedLookup.getSortedLongs(), - sortedLookup.getSortedDoubles(), - () -> new ArrayDictionaryMergingIterator( - sortedArrayLookups, - defaultSerializer.getGlobalLookup() - ) + sortedLookup.getSortedDoubles() ); stringCardinality = sortedLookup.getStringCardinality(); longCardinality = sortedLookup.getLongCardinality(); doubleCardinality = sortedLookup.getDoubleCardinality(); - arrayCardinality = sortedLookup.getArrayCardinality(); } else { final SimpleDictionaryMergingIterator stringIterator = new SimpleDictionaryMergingIterator<>( sortedLookups, @@ -181,29 +156,22 @@ public void writeMergedValueDictionary(List adapters) throws I sortedDoubleLookups, DOUBLE_MERGING_COMPARATOR ); - final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator( - sortedArrayLookups, - defaultSerializer.getGlobalLookup() - ); defaultSerializer.serializeDictionaries( () -> stringIterator, () -> longIterator, - () -> doubleIterator, - () -> arrayIterator + () -> doubleIterator ); stringCardinality = stringIterator.getCardinality(); longCardinality = longIterator.getCardinality(); doubleCardinality = doubleIterator.getCardinality(); - arrayCardinality = arrayIterator.getCardinality(); } log.debug( - "Completed dim[%s] conversions with string cardinality[%,d], long cardinality[%,d], double cardinality[%,d], array cardinality[%,d] in %,d millis.", + "Completed dim[%s] conversions with string cardinality[%,d], long cardinality[%,d], double cardinality[%,d] in %,d millis.", name, stringCardinality, longCardinality, doubleCardinality, - arrayCardinality, System.currentTimeMillis() - dimStartTime ); } @@ -213,71 +181,6 @@ public void writeMergedValueDictionary(List adapters) throws I } } - @Nullable - private GlobalDictionarySortedCollector getSortedIndexFromIncrementalAdapter( - IncrementalIndexAdapter adapter, - SortedMap mergedFields - ) - { - final IncrementalIndex index = adapter.getIncrementalIndex(); - final IncrementalIndex.DimensionDesc dim = index.getDimension(name); - if (dim == null || !(dim.getIndexer() instanceof NestedDataColumnIndexer)) { - return null; - } - final NestedDataColumnIndexer indexer = (NestedDataColumnIndexer) dim.getIndexer(); - indexer.mergeFields(mergedFields); - return indexer.getSortedCollector(); - } - - @Nullable - private GlobalDictionarySortedCollector getSortedIndexesFromQueryableAdapter( - QueryableIndexIndexableAdapter adapter, - SortedMap mergedFields - ) - { - final ColumnHolder columnHolder = adapter.getQueryableIndex().getColumnHolder(name); - - if (columnHolder == null) { - return null; - } - - final BaseColumn col = columnHolder.getColumn(); - - closer.register(col); - - if (col instanceof CompressedNestedDataComplexColumn) { - return getSortedIndexFromV1QueryableAdapterNestedColumn(mergedFields, col); - } - return null; - } - - private GlobalDictionarySortedCollector getSortedIndexFromV1QueryableAdapterNestedColumn( - SortedMap mergedFields, - BaseColumn col - ) - { - @SuppressWarnings("unchecked") - CompressedNestedDataComplexColumn column = (CompressedNestedDataComplexColumn) col; - closer.register(column); - for (int i = 0; i < column.getFields().size(); i++) { - String fieldPath = column.getFields().get(i); - NestedFieldTypeInfo.TypeSet types = column.getFieldInfo().getTypes(i); - mergedFields.compute(fieldPath, (k, v) -> { - if (v == null) { - return new NestedFieldTypeInfo.MutableTypeSet(types.getByteValue()); - } - return v.merge(types.getByteValue()); - }); - } - return new GlobalDictionarySortedCollector( - new StringEncodingStrategies.Utf8ToStringIndexed(column.getStringDictionary()), - column.getLongDictionary(), - column.getDoubleDictionary(), - column.getArraysIterable(), - column.getArrayDictionary().size() - ); - } - @Override public ColumnValueSelector convertSortedSegmentRowValuesToMergedRowValues( int segmentIndex, @@ -311,139 +214,4 @@ public ColumnDescriptor makeColumnDescriptor() return descriptorBuilder.build(); } - public static class ArrayDictionaryMergingIterator implements Iterator - { - private static final Comparator> PEEKING_ITERATOR_COMPARATOR = - (lhs, rhs) -> FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR.compare(lhs.peek(), rhs.peek()); - - protected final PriorityQueue> pQueue; - private final Iterable[] dimValueLookups; - private final GlobalDictionaryIdLookup idLookup; - - protected int counter; - private boolean initialized; - - public ArrayDictionaryMergingIterator(Iterable[] dimValueLookups, GlobalDictionaryIdLookup idLookup) - { - this.pQueue = new PriorityQueue<>(PEEKING_ITERATOR_COMPARATOR); - this.dimValueLookups = dimValueLookups; - this.idLookup = idLookup; - } - - private void initialize() - { - // we initialize lazily because the global id lookup might not be populated because the lower dictionary mergers - // have not been iterated yet, so wait until we iterate this one while serializing to populate it - for (Iterable dimValueLookup : dimValueLookups) { - if (dimValueLookup == null) { - continue; - } - final PeekingIterator 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 smallest = pQueue.remove(); - if (smallest == null) { - throw new NoSuchElementException(); - } - final int[] value = smallest.next(); - if (smallest.hasNext()) { - pQueue.add(smallest); - } - - while (!pQueue.isEmpty() && Arrays.equals(value, pQueue.peek().peek())) { - PeekingIterator same = pQueue.remove(); - same.next(); - if (same.hasNext()) { - pQueue.add(same); - } - } - counter++; - - return value; - } - - public int getCardinality() - { - return counter; - } - - @Override - public void remove() - { - throw new UnsupportedOperationException("remove"); - } - } - - public static class IdLookupArrayIterator implements Iterator - { - private final GlobalDictionaryIdLookup idLookup; - private final Iterator delegate; - - public IdLookupArrayIterator( - GlobalDictionaryIdLookup idLookup, - Iterator delegate - ) - { - this.idLookup = idLookup; - this.delegate = delegate; - } - - @Override - public boolean hasNext() - { - return delegate.hasNext(); - } - - @Override - public int[] next() - { - final Object[] next = delegate.next(); - if (next == null) { - return null; - } - final int[] 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; - } - } } diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/NestedDataDimensionHandler.java index 5b6afa5ee941..3eda293a7af7 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataDimensionHandler.java @@ -24,7 +24,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.nested.StructuredData; import org.apache.druid.segment.selector.settable.SettableColumnValueSelector; import org.apache.druid.segment.selector.settable.SettableObjectColumnValueSelector; @@ -56,7 +56,7 @@ public String getDimensionName() @Override public DimensionSpec getDimensionSpec() { - return new DefaultDimensionSpec(name, name, NestedDataComplexTypeSerde.TYPE); + return new DefaultDimensionSpec(name, name, ColumnType.NESTED_DATA); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataDimensionSchema.java b/processing/src/main/java/org/apache/druid/segment/NestedDataDimensionSchema.java index d660e674a38b..215fb2f73ead 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataDimensionSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataDimensionSchema.java @@ -44,7 +44,13 @@ public String getTypeName() @Override public ColumnType getColumnType() { - return NestedDataComplexTypeSerde.TYPE; + return ColumnType.NESTED_DATA; + } + + @Override + public DimensionHandler getDimensionHandler() + { + return new NestedDataDimensionHandler(getName()); } } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java index a3ec8f3b9cb5..28e26c02ded7 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java @@ -25,6 +25,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.DictionaryEncodedColumn; @@ -33,6 +34,8 @@ import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.data.ImmutableBitmapValues; import org.apache.druid.segment.data.IndexedIterable; +import org.apache.druid.segment.nested.NestedCommonFormatColumn; +import org.apache.druid.segment.nested.SortedValueDictionary; import org.apache.druid.segment.selector.settable.SettableColumnValueSelector; import org.apache.druid.segment.selector.settable.SettableLongColumnValueSelector; import org.apache.druid.utils.CloseableUtils; @@ -108,6 +111,13 @@ public > CloseableIndexed getDimValueLookup(S final BaseColumn col = columnHolder.getColumn(); if (!(col instanceof DictionaryEncodedColumn)) { + // this shouldn't happen, but if it does, try to close to prevent a leak + try { + col.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } return null; } @@ -155,6 +165,44 @@ public void close() throws IOException }; } + @Nullable + @Override + public NestedColumnMergable getNestedColumnMergeables(String columnName) + { + final ColumnHolder columnHolder = input.getColumnHolder(columnName); + + if (columnHolder == null) { + return null; + } + if (!(columnHolder.getColumnFormat() instanceof NestedCommonFormatColumn.Format)) { + return null; + } + + final BaseColumn col = columnHolder.getColumn(); + if (col instanceof NestedCommonFormatColumn) { + NestedCommonFormatColumn column = (NestedCommonFormatColumn) col; + return new NestedColumnMergable( + new SortedValueDictionary( + column.getStringDictionary(), + column.getLongDictionary(), + column.getDoubleDictionary(), + column.getArrayDictionary(), + column + ), + column.getFieldTypeInfo() + ); + } + + // this shouldn't happen because of the format check, but if it does try to close the column just in case + try { + col.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + return null; + } + @Override public RowIteratorImpl getRows() { @@ -340,7 +388,13 @@ public void mark() @Override public ColumnCapabilities getCapabilities(String column) { - return input.getColumnHolder(column).getHandlerCapabilities(); + return input.getColumnHolder(column).getCapabilities(); + } + + @Override + public ColumnFormat getFormat(String column) + { + return input.getColumnHolder(column).getColumnFormat(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java index 1c1e0b614850..37245a8fbb73 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java @@ -27,7 +27,6 @@ import com.google.common.collect.Maps; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; -import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.Indexed; import org.joda.time.Interval; @@ -164,8 +163,7 @@ private Map initDimensionHandlers(Indexed avai Map dimensionHandlerMap = Maps.newLinkedHashMap(); for (String dim : availableDimensions) { final ColumnHolder columnHolder = getColumnHolder(dim); - ColumnCapabilities capabilities = columnHolder.getHandlerCapabilities(); - DimensionHandler handler = DimensionHandlerUtils.getHandlerFromCapabilities(dim, capabilities, null); + final DimensionHandler handler = columnHolder.getColumnFormat().getColumnHandler(dim); dimensionHandlerMap.put(dim, handler); } return dimensionHandlerMap; diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java index 31ca53d1585b..b3ff66394032 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java @@ -124,7 +124,7 @@ public String getDimensionName() @Override public DimensionSchema getDimensionSchema(ColumnCapabilities capabilities) { - return new StringDimensionSchema(dimensionName); + return new StringDimensionSchema(dimensionName, multiValueHandling, hasBitmapIndexes); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/column/CapabilitiesBasedFormat.java b/processing/src/main/java/org/apache/druid/segment/column/CapabilitiesBasedFormat.java new file mode 100644 index 000000000000..d04123d46536 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/column/CapabilitiesBasedFormat.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.column; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.DimensionHandlerUtils; + +import javax.annotation.Nullable; +import java.util.Objects; + +public class CapabilitiesBasedFormat implements ColumnFormat +{ + // merge logic for the state capabilities will be in after incremental index is persisted + @VisibleForTesting + public static final ColumnCapabilities.CoercionLogic DIMENSION_CAPABILITY_MERGE_LOGIC = + new ColumnCapabilities.CoercionLogic() + { + @Override + public boolean dictionaryEncoded() + { + return true; + } + + @Override + public boolean dictionaryValuesSorted() + { + return true; + } + + @Override + public boolean dictionaryValuesUnique() + { + return true; + } + + @Override + public boolean multipleValues() + { + return false; + } + + @Override + public boolean hasNulls() + { + return false; + } + }; + private final ColumnCapabilities capabilities; + + public static CapabilitiesBasedFormat forColumnIndexer(ColumnCapabilities capabilities) + { + return new CapabilitiesBasedFormat(ColumnCapabilitiesImpl.snapshot(capabilities, DIMENSION_CAPABILITY_MERGE_LOGIC)); + } + + public CapabilitiesBasedFormat(ColumnCapabilities capabilities) + { + this.capabilities = capabilities; + } + + @Override + public DimensionHandler getColumnHandler(String columnName) + { + return DimensionHandlerUtils.getHandlerFromCapabilities(columnName, capabilities, null); + } + + @Override + public DimensionSchema getColumnSchema(String columnName) + { + return getColumnHandler(columnName).getDimensionSchema(capabilities); + } + + @Override + public ColumnFormat merge(@Nullable ColumnFormat otherFormat) + { + if (otherFormat == null) { + return this; + } + + ColumnCapabilitiesImpl merged = ColumnCapabilitiesImpl.copyOf(this.toColumnCapabilities()); + ColumnCapabilitiesImpl otherSnapshot = ColumnCapabilitiesImpl.copyOf(otherFormat.toColumnCapabilities()); + + if (!Objects.equals(merged.getType(), otherSnapshot.getType()) + || !Objects.equals(merged.getElementType(), otherSnapshot.getElementType())) { + final String mergedType = merged.getType() == null ? null : merged.asTypeString(); + final String otherType = otherSnapshot.getType() == null ? null : otherSnapshot.asTypeString(); + throw new ISE( + "Cannot merge columns of type[%s] and [%s]", + mergedType, + otherType + ); + } else if (!Objects.equals(merged.getComplexTypeName(), otherSnapshot.getComplexTypeName())) { + throw new ISE( + "Cannot merge columns of type[%s] and [%s]", + merged.getComplexTypeName(), + otherSnapshot.getComplexTypeName() + ); + } + + merged.setDictionaryEncoded(merged.isDictionaryEncoded().or(otherSnapshot.isDictionaryEncoded()).isTrue()); + merged.setHasMultipleValues(merged.hasMultipleValues().or(otherSnapshot.hasMultipleValues()).isTrue()); + merged.setDictionaryValuesSorted( + merged.areDictionaryValuesSorted().and(otherSnapshot.areDictionaryValuesSorted()).isTrue() + ); + merged.setDictionaryValuesUnique( + merged.areDictionaryValuesUnique().and(otherSnapshot.areDictionaryValuesUnique()).isTrue() + ); + merged.setHasNulls(merged.hasNulls().or(otherSnapshot.hasNulls()).isTrue()); + // when merging persisted queryableIndexes in the same ingestion job, all queryableIndexes should have the exact + // same hasBitmapIndexes flag set which is set in the ingestionSpec.. the exception is explicit null value + // columns which always report as having bitmap indexes because they can always resolve to all-true or all-false + // depending on whether or not the filter matches the null value. choosing false here picks what is most likely + // to be correct since explicit falses mean that the ingestion spec does not have bitmap index creation set + if (merged.hasBitmapIndexes() != otherSnapshot.hasBitmapIndexes()) { + merged.setHasBitmapIndexes(false); + } + if (merged.hasSpatialIndexes() != otherSnapshot.hasSpatialIndexes()) { + merged.setHasSpatialIndexes(merged.hasSpatialIndexes() || otherSnapshot.hasSpatialIndexes()); + } + merged.setFilterable(merged.isFilterable() && otherSnapshot.isFilterable()); + + return new CapabilitiesBasedFormat(merged); + } + + @Override + public ColumnType getLogicalType() + { + return capabilities.toColumnType(); + } + + @Override + public ColumnCapabilities toColumnCapabilities() + { + return capabilities; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java index 7176ad89debf..e142a3bd49cd 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java @@ -24,6 +24,7 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.segment.nested.NestedCommonFormatColumn; import org.apache.druid.segment.serde.NoIndexesColumnIndexSupplier; import org.apache.druid.segment.serde.NullValueIndexSupplier; @@ -35,9 +36,6 @@ public class ColumnBuilder { private final ColumnCapabilitiesImpl capabilitiesBuilder = ColumnCapabilitiesImpl.createDefault(); - @Nullable - private ColumnCapabilities handlerCapabilities = null; - @Nullable private Supplier columnSupplier = null; @Nullable @@ -45,6 +43,9 @@ public class ColumnBuilder @Nullable private SmooshedFileMapper fileMapper = null; + @Nullable + private ColumnFormat columnFormat = null; + public ColumnCapabilitiesImpl getCapabilitiesBuilder() { return capabilitiesBuilder; @@ -116,6 +117,13 @@ public ColumnBuilder setNumericColumnSupplier(Supplier return this; } + public ColumnBuilder setStandardTypeColumnSupplier(Supplier columnSupplier) + { + checkColumnSupplierNotSet(); + this.columnSupplier = columnSupplier; + return this; + } + public ColumnBuilder setIndexSupplier( @Nullable ColumnIndexSupplier indexSupplier, boolean hasBitmapIndex, @@ -147,9 +155,9 @@ public ColumnBuilder setHasNulls(ColumnCapabilities.Capable nullable) return this; } - public ColumnBuilder setHandlerCapabilities(ColumnCapabilities handlerCapabilities) + public ColumnBuilder setColumnFormat(ColumnFormat columnFormat) { - this.handlerCapabilities = handlerCapabilities; + this.columnFormat = columnFormat; return this; } @@ -157,7 +165,7 @@ public ColumnHolder build() { Preconditions.checkState(capabilitiesBuilder.getType() != null, "Type must be set."); - return new SimpleColumnHolder(capabilitiesBuilder, handlerCapabilities, columnSupplier, indexSupplier); + return new SimpleColumnHolder(capabilitiesBuilder, columnFormat, columnSupplier, indexSupplier); } private void checkColumnSupplierNotSet() diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java index f3e6d175d762..74d090be8c4b 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java @@ -32,7 +32,7 @@ */ public class ColumnCapabilitiesImpl implements ColumnCapabilities { - private static final CoercionLogic ALL_FALSE = new CoercionLogic() + public static final CoercionLogic ALL_FALSE = new CoercionLogic() { @Override public boolean dictionaryEncoded() diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnFormat.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnFormat.java new file mode 100644 index 000000000000..5b4dd5067795 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnFormat.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.column; + +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.segment.DimensionHandler; + +import javax.annotation.Nullable; + +public interface ColumnFormat +{ + ColumnType getLogicalType(); + + ColumnCapabilities toColumnCapabilities(); + + DimensionHandler getColumnHandler(String columnName); + + DimensionSchema getColumnSchema(String columnName); + + ColumnFormat merge(@Nullable ColumnFormat otherFormat); +} diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java index 0a8cae83684e..11b472c26d30 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java @@ -39,9 +39,9 @@ static boolean storeDoubleAsFloat() ColumnCapabilities getCapabilities(); - default ColumnCapabilities getHandlerCapabilities() + default ColumnFormat getColumnFormat() { - return getCapabilities(); + return new CapabilitiesBasedFormat(getCapabilities()); } int getLength(); diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnType.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnType.java index 1f80ccf9ad1d..500a7deab642 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnType.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnType.java @@ -23,6 +23,7 @@ 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.segment.nested.NestedDataComplexTypeSerde; import javax.annotation.Nullable; @@ -70,25 +71,27 @@ public class ColumnType extends BaseTypeSignature * @see ValueType#ARRAY * @see ValueType#STRING */ - public static final ColumnType STRING_ARRAY = new ColumnType(ValueType.ARRAY, null, STRING); + public static final ColumnType STRING_ARRAY = ofArray(STRING); /** * An array of Longs. Values will be represented as Object[] or long[] * @see ValueType#ARRAY * @see ValueType#LONG */ - public static final ColumnType LONG_ARRAY = new ColumnType(ValueType.ARRAY, null, LONG); + public static final ColumnType LONG_ARRAY = ofArray(LONG); /** * An array of Doubles. Values will be represented as Object[] or double[]. * @see ValueType#ARRAY * @see ValueType#DOUBLE */ - public static final ColumnType DOUBLE_ARRAY = new ColumnType(ValueType.ARRAY, null, DOUBLE); + public static final ColumnType DOUBLE_ARRAY = ofArray(DOUBLE); /** * An array of Floats. Values will be represented as Object[] or float[]. * @see ValueType#ARRAY * @see ValueType#FLOAT */ - public static final ColumnType FLOAT_ARRAY = new ColumnType(ValueType.ARRAY, null, FLOAT); + public static final ColumnType FLOAT_ARRAY = ofArray(FLOAT); + + public static final ColumnType NESTED_DATA = ofComplex(NestedDataComplexTypeSerde.TYPE_NAME); /** * Placeholder type for an "unknown" complex, which is used when the complex type name was "lost" or unavailable for * whatever reason, to indicate an opaque type that cannot be generically handled with normal complex type handling @@ -97,7 +100,7 @@ public class ColumnType extends BaseTypeSignature * * @see ValueType#COMPLEX */ - public static final ColumnType UNKNOWN_COMPLEX = new ColumnType(ValueType.COMPLEX, null, null); + public static final ColumnType UNKNOWN_COMPLEX = ofComplex(null); @JsonCreator public ColumnType( diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java index 31ff30b3f800..3eea9193cc37 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java @@ -149,16 +149,6 @@ public ColumnType ofLong() @Override public ColumnType ofArray(ColumnType elementType) { - if (elementType.isPrimitive()) { - switch (elementType.getType()) { - case STRING: - return ColumnType.STRING_ARRAY; - case DOUBLE: - return ColumnType.DOUBLE_ARRAY; - case LONG: - return ColumnType.LONG_ARRAY; - } - } // i guess this is potentially unbounded if we ever support arbitrarily deep nested arrays return INTERNER.intern(new ColumnType(ValueType.ARRAY, null, elementType)); } @@ -166,9 +156,6 @@ public ColumnType ofArray(ColumnType elementType) @Override public ColumnType ofComplex(@Nullable String complexTypeName) { - if (complexTypeName == null) { - return ColumnType.UNKNOWN_COMPLEX; - } return INTERNER.intern(new ColumnType(ValueType.COMPLEX, complexTypeName, null)); } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java b/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java index 9de07005afb5..30dc4a464ca2 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java @@ -33,8 +33,7 @@ class SimpleColumnHolder implements ColumnHolder { private final ColumnCapabilities capabilities; - - private final ColumnCapabilities handlerCapabilities; + private final ColumnFormat columnFormat; @Nullable private final Supplier columnSupplier; @@ -47,7 +46,7 @@ class SimpleColumnHolder implements ColumnHolder SimpleColumnHolder( ColumnCapabilities capabilities, - @Nullable ColumnCapabilities handlerCapabilities, + @Nullable ColumnFormat columnFormat, @Nullable Supplier columnSupplier, @Nullable ColumnIndexSupplier indexSupplier ) @@ -55,7 +54,7 @@ class SimpleColumnHolder implements ColumnHolder this.capabilities = capabilities; this.columnSupplier = columnSupplier; this.indexSupplier = indexSupplier; - this.handlerCapabilities = handlerCapabilities == null ? capabilities : handlerCapabilities; + this.columnFormat = columnFormat == null ? new CapabilitiesBasedFormat(capabilities) : columnFormat; // ColumnSupplier being null is sort of a rare case but can happen when a segment // was created, for example, using an aggregator that was removed in later versions. @@ -79,9 +78,9 @@ public ColumnCapabilities getCapabilities() } @Override - public ColumnCapabilities getHandlerCapabilities() + public ColumnFormat getColumnFormat() { - return handlerCapabilities; + return columnFormat; } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/column/StringFrontCodedDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/column/StringFrontCodedDictionaryEncodedColumn.java index 0df381914bb7..c447e9b63651 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/StringFrontCodedDictionaryEncodedColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/column/StringFrontCodedDictionaryEncodedColumn.java @@ -31,12 +31,14 @@ import org.apache.druid.segment.data.ColumnarInts; import org.apache.druid.segment.data.ColumnarMultiInts; import org.apache.druid.segment.data.FrontCodedIndexed; +import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.ReadableOffset; import org.apache.druid.segment.data.SingleIndexedInt; import org.apache.druid.segment.filter.BooleanValueMatcher; import org.apache.druid.segment.historical.HistoricalDimensionSelector; import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector; +import org.apache.druid.segment.nested.NestedCommonFormatColumn; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; import org.apache.druid.segment.vector.ReadableVectorOffset; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; @@ -56,7 +58,8 @@ * This class is otherwise nearly identical to {@link StringDictionaryEncodedColumn} other than the dictionary * difference. */ -public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncodedColumn +public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncodedColumn, + NestedCommonFormatColumn { @Nullable private final ColumnarInts column; @@ -479,4 +482,16 @@ public void close() throws IOException { CloseableUtils.closeAll(column, multiValueColumn); } + + @Override + public ColumnType getLogicalType() + { + return ColumnType.STRING; + } + + @Override + public Indexed getStringDictionary() + { + return new StringEncodingStrategies.Utf8ToStringIndexed(utf8Dictionary); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/DictionaryWriter.java b/processing/src/main/java/org/apache/druid/segment/data/DictionaryWriter.java index a9cd5892ff44..170f6975f28e 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/DictionaryWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/DictionaryWriter.java @@ -34,4 +34,6 @@ public interface DictionaryWriter extends Serializer @Nullable T get(int dictId) throws IOException; + + int getCardinality(); } diff --git a/processing/src/main/java/org/apache/druid/segment/data/EncodedStringDictionaryWriter.java b/processing/src/main/java/org/apache/druid/segment/data/EncodedStringDictionaryWriter.java index 0ea3c809d0ff..371a73bebd74 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/EncodedStringDictionaryWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/EncodedStringDictionaryWriter.java @@ -74,6 +74,12 @@ public String get(int dictId) throws IOException return StringUtils.fromUtf8(bytes); } + @Override + public int getCardinality() + { + return delegate.getCardinality(); + } + @Override public long getSerializedSize() throws IOException { diff --git a/processing/src/main/java/org/apache/druid/segment/data/FixedIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/FixedIndexed.java index 4f4fc92d2261..bbde2bb0cfe7 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FixedIndexed.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FixedIndexed.java @@ -21,6 +21,8 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import it.unimi.dsi.fastutil.ints.IntIntImmutablePair; +import it.unimi.dsi.fastutil.ints.IntIntPair; import org.apache.druid.common.config.NullHandling; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.TypeStrategy; @@ -73,7 +75,7 @@ public static Supplier> read(ByteBuffer bb, TypeStrategy valuesOffset ); - bb.position(buffer.position() + (width * size)); + bb.position(buffer.position() + (width * (hasNull ? size - 1 : size))); return fixedIndexed; } @@ -168,6 +170,43 @@ public Iterator iterator() return IndexedIterable.create(this).iterator(); } + + + public IntIntPair getRange( + @Nullable T startValue, + boolean startStrict, + @Nullable T endValue, + boolean endStrict + ) + { + final int firstValue = hasNull ? 1 : 0; + int startIndex, endIndex; + if (startValue == null) { + startIndex = firstValue; + } else { + final int found = indexOf(startValue); + if (found >= firstValue) { + startIndex = startStrict ? found + 1 : found; + } else { + startIndex = -(found + 1); + } + } + + if (endValue == null) { + endIndex = size(); + } else { + final int found = indexOf(endValue); + if (found >= firstValue) { + endIndex = endStrict ? found : found + 1; + } else { + endIndex = -(found + 1); + } + } + + endIndex = Math.max(startIndex, endIndex); + return new IntIntImmutablePair(startIndex, endIndex); + } + @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { diff --git a/processing/src/main/java/org/apache/druid/segment/data/FixedIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/FixedIndexedWriter.java index 9246ca78820d..76944d2dc499 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FixedIndexedWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FixedIndexedWriter.java @@ -78,6 +78,11 @@ public void open() throws IOException this.valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); } + public int getCardinality() + { + return hasNulls ? numWritten + 1 : numWritten; + } + @Override public long getSerializedSize() { diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java index 8f711df610d0..34e81dfc72e5 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java @@ -224,6 +224,12 @@ public byte[] get(int index) throws IOException } } + @Override + public int getCardinality() + { + return numWritten; + } + private long getBucketOffset(int index) throws IOException { getOffsetBuffer.clear(); diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java index 00cc2079fcbe..05802fdad72b 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java @@ -224,6 +224,12 @@ public int[] get(int index) throws IOException } } + @Override + public int getCardinality() + { + return numWritten; + } + private long getBucketOffset(int index) throws IOException { getOffsetBuffer.clear(); diff --git a/processing/src/main/java/org/apache/druid/segment/data/GenericIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/GenericIndexedWriter.java index d62a7e782417..394fccba06b8 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/GenericIndexedWriter.java @@ -301,6 +301,12 @@ public T get(int index) throws IOException return strategy.fromByteBuffer(bb, valueSize); } + @Override + public int getCardinality() + { + return numWritten; + } + private long getOffset(int index) throws IOException { if (!requireMultipleFiles) { diff --git a/processing/src/main/java/org/apache/druid/segment/data/Indexed.java b/processing/src/main/java/org/apache/druid/segment/data/Indexed.java index 272ff6a21209..f85e1a84168e 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/Indexed.java +++ b/processing/src/main/java/org/apache/druid/segment/data/Indexed.java @@ -23,8 +23,11 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop; import org.apache.druid.query.monomorphicprocessing.HotLoopCallee; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import javax.annotation.Nullable; +import java.util.Collections; +import java.util.Iterator; /** * Indexed is a fixed-size, immutable, indexed set of values which allows @@ -36,6 +39,43 @@ @PublicApi public interface Indexed extends Iterable, HotLoopCallee { + static Indexed empty() + { + return new Indexed() + { + @Override + public int size() + { + return 0; + } + + @Nullable + @Override + public T get(int index) + { + Indexed.checkIndex(index, 0); + return null; + } + + @Override + public int indexOf(@Nullable T value) + { + return -1; + } + + @Override + public Iterator iterator() + { + return Collections.emptyIterator(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } + }; + } /** * Number of elements in the value set diff --git a/processing/src/main/java/org/apache/druid/segment/data/RoaringBitmapSerdeFactory.java b/processing/src/main/java/org/apache/druid/segment/data/RoaringBitmapSerdeFactory.java index eff47eec91ae..ea6bb9bd994d 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/RoaringBitmapSerdeFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/data/RoaringBitmapSerdeFactory.java @@ -73,6 +73,9 @@ public Class getClazz() @Nullable public ImmutableBitmap fromByteBuffer(ByteBuffer buffer, int numBytes) { + if (numBytes == 0) { + return INSTANCE.bitmapFactory.makeEmptyImmutableBitmap(); + } buffer.limit(buffer.position() + numBytes); return new WrappedImmutableRoaringBitmap(new ImmutableRoaringBitmap(buffer)); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java index a3289592c59a..08076f9f8cab 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java @@ -59,22 +59,23 @@ import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.EncodedKeyComponent; import org.apache.druid.segment.FloatColumnSelector; -import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.Metadata; +import org.apache.druid.segment.NestedCommonFormatColumnHandler; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.ObjectColumnSelector; import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.CapabilitiesBasedFormat; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.apache.druid.segment.serde.ComplexMetricExtractor; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; @@ -238,6 +239,7 @@ public ColumnCapabilities getColumnCapabilities(String columnName) private final List dimensionDescsList; // dimension capabilities are provided by the indexers private final Map timeAndMetricsColumnCapabilities; + private final Map timeAndMetricsColumnFormats; private final AtomicInteger numEntries = new AtomicInteger(); private final AtomicLong bytesInMemory = new AtomicLong(); private final boolean useMaxMemoryEstimates; @@ -292,6 +294,7 @@ protected IncrementalIndex( .useSchemaDiscovery(); this.timeAndMetricsColumnCapabilities = new HashMap<>(); + this.timeAndMetricsColumnFormats = new HashMap<>(); this.metricDescs = Maps.newLinkedHashMap(); this.dimensionDescs = Maps.newLinkedHashMap(); this.metadata = new Metadata( @@ -307,32 +310,34 @@ protected IncrementalIndex( for (AggregatorFactory metric : metrics) { MetricDesc metricDesc = new MetricDesc(metricDescs.size(), metric); metricDescs.put(metricDesc.getName(), metricDesc); - timeAndMetricsColumnCapabilities.put(metricDesc.getName(), metricDesc.getCapabilities()); + final ColumnCapabilities capabilities = metricDesc.getCapabilities(); + timeAndMetricsColumnCapabilities.put(metricDesc.getName(), capabilities); + if (capabilities.is(ValueType.COMPLEX)) { + timeAndMetricsColumnFormats.put( + metricDesc.getName(), + new CapabilitiesBasedFormat( + ColumnCapabilitiesImpl.snapshot( + ColumnCapabilitiesImpl.copyOf(capabilities).setType(ColumnType.ofComplex(metricDesc.getType())), + ColumnCapabilitiesImpl.ALL_FALSE + ) + ) + ); + } else { + timeAndMetricsColumnFormats.put( + metricDesc.getName(), + new CapabilitiesBasedFormat( + ColumnCapabilitiesImpl.snapshot(capabilities, ColumnCapabilitiesImpl.ALL_FALSE) + ) + ); + } + } this.dimensionsSpec = incrementalIndexSchema.getDimensionsSpec(); this.dimensionDescsList = new ArrayList<>(); for (DimensionSchema dimSchema : dimensionsSpec.getDimensions()) { - ColumnType type = dimSchema.getColumnType(); - String dimName = dimSchema.getName(); - - // Note: Things might be simpler if DimensionSchema had a method "getColumnCapabilities()" which could return - // type specific capabilities by itself. However, for various reasons, DimensionSchema currently lives in druid-core - // while ColumnCapabilities lives in druid-processing which makes that approach difficult. - ColumnCapabilitiesImpl capabilities = makeDefaultCapabilitiesFromValueType(type); - - capabilities.setHasBitmapIndexes(dimSchema.hasBitmapIndex()); - - if (dimSchema.getTypeName().equals(DimensionSchema.SPATIAL_TYPE_NAME)) { - capabilities.setHasSpatialIndexes(true); - } - DimensionHandler handler = DimensionHandlerUtils.getHandlerFromCapabilities( - dimName, - capabilities, - dimSchema.getMultiValueHandling() - ); - addNewDimension(dimName, handler); + addNewDimension(dimSchema.getName(), dimSchema.getDimensionHandler()); } //__time capabilities @@ -459,13 +464,13 @@ public InputRow formatRow(InputRow row) return row; } - public Map getColumnHandlerCapabilities() + public Map getColumnFormats() { - ImmutableMap.Builder builder = - ImmutableMap.builder().putAll(timeAndMetricsColumnCapabilities); + ImmutableMap.Builder builder = ImmutableMap.builder(); synchronized (dimensionDescs) { - dimensionDescs.forEach((dimension, desc) -> builder.put(dimension, desc.getIndexer().getHandlerCapabilities())); + timeAndMetricsColumnFormats.forEach(builder::put); + dimensionDescs.forEach((dimension, desc) -> builder.put(dimension, desc.getIndexer().getFormat())); } return builder.build(); } @@ -484,26 +489,15 @@ public ColumnCapabilities getColumnCapabilities(String columnName) } @Nullable - public ColumnCapabilities getColumnHandlerCapabilities(String columnName) + public ColumnFormat getColumnFormat(String columnName) { - if (timeAndMetricsColumnCapabilities.containsKey(columnName)) { - final ColumnCapabilities capabilities = timeAndMetricsColumnCapabilities.get(columnName); - if (capabilities.is(ValueType.COMPLEX)) { - // normalize complex type name for these capabilities. the values in timeAndMetricsColumnCapabilities - // are direct from the AggregatorFactory, so might be too specific (think build vs merge aggs) - // for this method though, we want the 'normal' type name for the capabilities, since this is the true 'output' - // type of the column, so use the type from the MetricDesc instead, which is computed by round-tripping through - // something like ComplexMetrics.getSerdeForType(valueType.getComplexTypeName()).getTypeName() - return ColumnCapabilitiesImpl.copyOf(capabilities) - .setType(ColumnType.ofComplex(metricDescs.get(columnName).getType())); - } - return capabilities; + if (timeAndMetricsColumnFormats.containsKey(columnName)) { + return timeAndMetricsColumnFormats.get(columnName); } - synchronized (dimensionDescs) { final DimensionDesc desc = dimensionDescs.get(columnName); - return desc != null ? desc.getIndexer().getHandlerCapabilities() : null; + return desc != null ? desc.getIndexer().getFormat() : null; } } @@ -592,11 +586,7 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) wasNewDim = true; final DimensionHandler handler; if (useSchemaDiscovery) { - handler = DimensionHandlerUtils.getHandlerFromCapabilities( - dimension, - makeDefaultCapabilitiesFromValueType(NestedDataComplexTypeSerde.TYPE), - null - ); + handler = new NestedCommonFormatColumnHandler(dimension); } else { // legacy behavior: for schemaless type discovery, everything is a String handler = DimensionHandlerUtils.getHandlerFromCapabilities( @@ -872,7 +862,7 @@ public List getDimensionOrder() } } - private ColumnCapabilitiesImpl makeDefaultCapabilitiesFromValueType(ColumnType type) + public static ColumnCapabilitiesImpl makeDefaultCapabilitiesFromValueType(ColumnType type) { switch (type.getType()) { case STRING: @@ -896,7 +886,7 @@ private ColumnCapabilitiesImpl makeDefaultCapabilitiesFromValueType(ColumnType t */ public void loadDimensionIterable( Iterable oldDimensionOrder, - Map oldColumnCapabilities + Map oldColumnCapabilities ) { synchronized (dimensionDescs) { @@ -905,12 +895,8 @@ public void loadDimensionIterable( } for (String dim : oldDimensionOrder) { if (dimensionDescs.get(dim) == null) { - ColumnCapabilitiesImpl capabilities = ColumnCapabilitiesImpl.snapshot( - oldColumnCapabilities.get(dim), - IndexMergerV9.DIMENSION_CAPABILITY_MERGE_LOGIC - ); - DimensionHandler handler = DimensionHandlerUtils.getHandlerFromCapabilities(dim, capabilities, null); - addNewDimension(dim, handler); + ColumnFormat format = oldColumnCapabilities.get(dim); + addNewDimension(dim, format.getColumnHandler(dim)); } } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java index 2799bfc3cd65..06a2408d60e7 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java @@ -22,12 +22,15 @@ import it.unimi.dsi.fastutil.ints.IntIterator; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.MutableBitmap; +import org.apache.druid.segment.AutoTypeColumnIndexer; import org.apache.druid.segment.DimensionIndexer; import org.apache.druid.segment.IndexableAdapter; import org.apache.druid.segment.IntIteratorUtils; import org.apache.druid.segment.Metadata; +import org.apache.druid.segment.NestedDataColumnIndexer; import org.apache.druid.segment.TransformableRowIterator; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.data.BitmapValues; import org.apache.druid.segment.data.CloseableIndexed; import org.joda.time.Interval; @@ -99,11 +102,6 @@ private void processRows( } } - public IncrementalIndex getIncrementalIndex() - { - return index; - } - @Override public Interval getDataInterval() { @@ -142,6 +140,34 @@ public > CloseableIndexed getDimValueLookup(S return indexer.getSortedIndexedValues(); } + @Nullable + @Override + public NestedColumnMergable getNestedColumnMergeables(String column) + { + final DimensionAccessor accessor = accessors.get(column); + if (accessor == null) { + return null; + } + + final DimensionIndexer indexer = accessor.dimensionDesc.getIndexer(); + if (indexer instanceof NestedDataColumnIndexer) { + NestedDataColumnIndexer nestedDataColumnIndexer = (NestedDataColumnIndexer) indexer; + + return new NestedColumnMergable( + nestedDataColumnIndexer.getSortedValueLookups(), + nestedDataColumnIndexer.getFieldTypeInfo() + ); + } + if (indexer instanceof AutoTypeColumnIndexer) { + AutoTypeColumnIndexer standardIndexer = (AutoTypeColumnIndexer) indexer; + return new NestedColumnMergable( + standardIndexer.getSortedValueLookups(), + standardIndexer.getFieldTypeInfo() + ); + } + return null; + } + @Override public TransformableRowIterator getRows() { @@ -179,7 +205,13 @@ public BitmapValues getBitmapValues(String dimension, int index) @Override public ColumnCapabilities getCapabilities(String column) { - return index.getColumnHandlerCapabilities(column); + return index.getColumnCapabilities(column); + } + + @Override + public ColumnFormat getFormat(String column) + { + return index.getColumnFormat(column); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 4e5fd8000b4a..5c2271744d46 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -40,10 +40,10 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.segment.filter.BooleanValueMatcher; -import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -213,7 +213,7 @@ public ColumnCapabilities getColumnCapabilities(String column) // nested column indexer is a liar, and behaves like any type if it only processes unnested literals of a single type // so keep it in the family so to speak if (desc != null && desc.getIndexer() instanceof NestedDataColumnIndexer) { - return ColumnCapabilitiesImpl.createDefault().setType(NestedDataComplexTypeSerde.TYPE); + return ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA); } // Different from index.getColumnCapabilities because, in a way, IncrementalIndex's string-typed dimensions // are always potentially multi-valued at query time. (Missing / null values for a row can potentially be 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 1d4a68338486..fbee9f8d6099 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 @@ -44,6 +44,10 @@ 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.StringEncodingStrategies; +import org.apache.druid.segment.column.TypeStrategies; +import org.apache.druid.segment.column.TypeStrategy; +import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.ColumnarDoubles; import org.apache.druid.segment.data.ColumnarInts; import org.apache.druid.segment.data.ColumnarLongs; @@ -73,11 +77,14 @@ import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; /** @@ -85,16 +92,16 @@ * 'raw' {@link StructuredData} values and provides selectors for nested 'literal' field columns. */ public abstract class CompressedNestedDataComplexColumn> - extends NestedDataComplexColumn + extends NestedDataComplexColumn implements NestedCommonFormatColumn { - private final NestedDataColumnMetadata metadata; + public static final IntTypeStrategy INT_TYPE_STRATEGY = new IntTypeStrategy(); private final Closer closer; private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier; private CompressedVariableSizedBlobColumn compressedRawColumn; private final ImmutableBitmap nullValues; private final GenericIndexed fields; - private final NestedFieldTypeInfo fieldInfo; + private final FieldTypeInfo fieldInfo; private final Supplier stringDictionarySupplier; private final Supplier> longDictionarySupplier; @@ -109,22 +116,32 @@ public abstract class CompressedNestedDataComplexColumn STRATEGY = NestedDataComplexTypeSerde.INSTANCE.getObjectStrategy(); + private final ColumnType logicalType; + + private final String columnName; + private final BitmapSerdeFactory bitmapSerdeFactory; + private final ByteOrder byteOrder; + public CompressedNestedDataComplexColumn( - NestedDataColumnMetadata metadata, + String columnName, + ColumnType logicalType, @SuppressWarnings("unused") ColumnConfig columnConfig, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, GenericIndexed fields, - NestedFieldTypeInfo fieldInfo, + FieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, Supplier> doubleDictionarySupplier, Supplier arrayDictionarySupplier, SmooshedFileMapper fileMapper, + BitmapSerdeFactory bitmapSerdeFactory, + ByteOrder byteOrder, String rootFieldPath ) { - this.metadata = metadata; + this.columnName = columnName; + this.logicalType = logicalType; this.nullValues = nullValues; this.fields = fields; this.fieldInfo = fieldInfo; @@ -135,6 +152,8 @@ public CompressedNestedDataComplexColumn( this.fileMapper = fileMapper; this.closer = Closer.create(); this.compressedRawColumnSupplier = compressedRawColumnSupplier; + this.bitmapSerdeFactory = bitmapSerdeFactory; + this.byteOrder = byteOrder; this.rootFieldPath = rootFieldPath; } @@ -144,9 +163,22 @@ public CompressedNestedDataComplexColumn( public abstract String getFieldFileName(String fileNameBase, String field, int fieldIndex); - public GenericIndexed getFields() + @Override + public SortedMap getFieldTypeInfo() + { + SortedMap fieldMap = new TreeMap<>(); + for (int i = 0; i < fields.size(); i++) { + String fieldPath = fields.get(i); + FieldTypeInfo.TypeSet types = fieldInfo.getTypes(i); + fieldMap.put(fieldPath, new FieldTypeInfo.MutableTypeSet(types.getByteValue())); + } + return fieldMap; + } + + @Override + public ColumnType getLogicalType() { - return fields; + return logicalType; } @Override @@ -159,32 +191,32 @@ public List> getNestedFields() return fieldParts; } - public NestedFieldTypeInfo getFieldInfo() - { - return fieldInfo; - } - public TStringDictionary getStringDictionary() + public TStringDictionary getUtf8BytesDictionary() { return stringDictionarySupplier.get(); } - public FixedIndexed getLongDictionary() + @Override + public Indexed getStringDictionary() { - return longDictionarySupplier.get(); + return new StringEncodingStrategies.Utf8ToStringIndexed(stringDictionarySupplier.get()); } - public FixedIndexed getDoubleDictionary() + @Override + public Indexed getLongDictionary() { - return doubleDictionarySupplier.get(); + return longDictionarySupplier.get(); } - public FrontCodedIntArrayIndexed getArrayDictionary() + @Override + public Indexed getDoubleDictionary() { - return arrayDictionarySupplier.get(); + return doubleDictionarySupplier.get(); } - public Iterable getArraysIterable() + @Override + public Indexed getArrayDictionary() { Iterable arrays = () -> { final TStringDictionary stringDictionary = stringDictionarySupplier.get(); @@ -230,7 +262,39 @@ private Object lookupId(int globalId) } }; }; - return arrays; + return new Indexed() + { + @Override + public int size() + { + return arrayDictionarySupplier.get().size(); + } + + @Nullable + @Override + public Object[] get(int index) + { + throw new UnsupportedOperationException("get not supported"); + } + + @Override + public int indexOf(@Nullable Object[] value) + { + throw new UnsupportedOperationException("indexOf not supported"); + } + + @Override + public Iterator iterator() + { + return arrays.iterator(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // meh + } + }; } public ImmutableBitmap getNullValues() @@ -754,7 +818,7 @@ public Set getColumnTypes(List path) if (index < 0) { return null; } - return NestedFieldTypeInfo.convertToSet(fieldInfo.getTypes(index).getByteValue()); + return FieldTypeInfo.convertToSet(fieldInfo.getTypes(index).getByteValue()); } @Nullable @@ -808,15 +872,15 @@ private ColumnHolder readNestedFieldColumn(String field, int fieldIndex) if (fieldIndex < 0) { return null; } - final NestedFieldTypeInfo.TypeSet types = fieldInfo.getTypes(fieldIndex); - final String fieldFileName = getFieldFileName(metadata.getFileNameBase(), field, fieldIndex); + final FieldTypeInfo.TypeSet types = fieldInfo.getTypes(fieldIndex); + final String fieldFileName = getFieldFileName(columnName, field, fieldIndex); final ByteBuffer dataBuffer = fileMapper.mapFile(fieldFileName); if (dataBuffer == null) { throw new ISE( "Can't find field [%s] with name [%s] in [%s] file.", field, fieldFileName, - metadata.getFileNameBase() + columnName ); } @@ -837,29 +901,29 @@ private ColumnHolder readNestedFieldColumn(String field, int fieldIndex) final Supplier> localDictionarySupplier = FixedIndexed.read( dataBuffer, - NestedDataColumnSerializer.INT_TYPE_STRATEGY, - metadata.getByteOrder(), + INT_TYPE_STRATEGY, + byteOrder, Integer.BYTES ); - ByteBuffer bb = dataBuffer.asReadOnlyBuffer().order(metadata.getByteOrder()); + ByteBuffer bb = dataBuffer.asReadOnlyBuffer().order(byteOrder); int longsLength = bb.getInt(); int doublesLength = bb.getInt(); dataBuffer.position(dataBuffer.position() + Integer.BYTES + Integer.BYTES); int pos = dataBuffer.position(); final Supplier longs = longsLength > 0 ? CompressedColumnarLongsSupplier.fromByteBuffer( dataBuffer, - metadata.getByteOrder() + byteOrder ) : () -> null; dataBuffer.position(pos + longsLength); pos = dataBuffer.position(); final Supplier doubles = doublesLength > 0 ? CompressedColumnarDoublesSuppliers.fromByteBuffer( dataBuffer, - metadata.getByteOrder() + byteOrder ) : () -> null; dataBuffer.position(pos + doublesLength); final WritableSupplier ints; if (version == DictionaryEncodedColumnPartSerde.VERSION.COMPRESSED) { - ints = CompressedVSizeColumnarIntsSupplier.fromByteBuffer(dataBuffer, metadata.getByteOrder()); + ints = CompressedVSizeColumnarIntsSupplier.fromByteBuffer(dataBuffer, byteOrder); } else { ints = VSizeColumnarInts.readFromByteBuffer(dataBuffer); } @@ -868,7 +932,7 @@ private ColumnHolder readNestedFieldColumn(String field, int fieldIndex) GenericIndexed rBitmaps = GenericIndexed.read( dataBuffer, - metadata.getBitmapSerdeFactory().getObjectStrategy(), + bitmapSerdeFactory.getObjectStrategy(), columnBuilder.getFileMapper() ); final Supplier> arrayElementDictionarySupplier; @@ -876,13 +940,13 @@ private ColumnHolder readNestedFieldColumn(String field, int fieldIndex) if (dataBuffer.hasRemaining()) { arrayElementDictionarySupplier = FixedIndexed.read( dataBuffer, - NestedDataColumnSerializer.INT_TYPE_STRATEGY, - metadata.getByteOrder(), + INT_TYPE_STRATEGY, + byteOrder, Integer.BYTES ); arrayElementBitmaps = GenericIndexed.read( dataBuffer, - metadata.getBitmapSerdeFactory().getObjectStrategy(), + bitmapSerdeFactory.getObjectStrategy(), columnBuilder.getFileMapper() ); } else { @@ -904,7 +968,7 @@ private ColumnHolder readNestedFieldColumn(String field, int fieldIndex) localDict, hasNull ? rBitmaps.get(0) - : metadata.getBitmapSerdeFactory().getBitmapFactory().makeEmptyImmutableBitmap() + : bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap() )); }; columnBuilder.setHasMultipleValues(false) @@ -913,7 +977,7 @@ private ColumnHolder readNestedFieldColumn(String field, int fieldIndex) columnBuilder.setIndexSupplier( new NestedFieldColumnIndexSupplier( types, - metadata.getBitmapSerdeFactory().getBitmapFactory(), + bitmapSerdeFactory.getBitmapFactory(), rBitmaps, localDictionarySupplier, stringDictionarySupplier, @@ -931,4 +995,50 @@ private ColumnHolder readNestedFieldColumn(String field, int fieldIndex) throw new RE(ex, "Failed to read data for [%s]", field); } } + + private static final class IntTypeStrategy implements TypeStrategy + { + @Override + public int estimateSizeBytes(Integer value) + { + return Integer.BYTES; + } + + @Override + public Integer read(ByteBuffer buffer) + { + return buffer.getInt(); + } + + @Override + public Integer read(ByteBuffer buffer, int offset) + { + return buffer.getInt(offset); + } + + @Override + public boolean readRetainsBufferReference() + { + return false; + } + + @Override + public int write(ByteBuffer buffer, Integer value, int maxSizeBytes) + { + TypeStrategies.checkMaxSize(buffer.remaining(), maxSizeBytes, ColumnType.LONG); + final int sizeBytes = Integer.BYTES; + final int remaining = maxSizeBytes - sizeBytes; + if (remaining >= 0) { + buffer.putInt(value); + return sizeBytes; + } + return remaining; + } + + @Override + public int compare(Object o1, Object o2) + { + return Integer.compare(((Number) o1).intValue(), ((Number) o2).intValue()); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java similarity index 88% rename from processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java rename to processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java index 2675b0392c39..2daa5b97350b 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java @@ -32,10 +32,10 @@ import javax.annotation.Nullable; /** - * Ingestion time dictionary identifier lookup, used by {@link NestedDataColumnSerializer} to build a global dictionary + * Ingestion time dictionary identifier lookup, used by {@link NestedDataColumnSerializerV4} to build a global dictionary * id to value mapping for the 'stacked' global value dictionaries. */ -public class GlobalDictionaryIdLookup +public class DictionaryIdLookup { private final Object2IntMap stringLookup; @@ -47,7 +47,7 @@ public class GlobalDictionaryIdLookup private int dictionarySize; - public GlobalDictionaryIdLookup() + public DictionaryIdLookup() { this.stringLookup = new Object2IntLinkedOpenHashMap<>(); stringLookup.defaultReturnValue(-1); @@ -69,6 +69,16 @@ public void addString(@Nullable String value) stringLookup.put(value, id); } + // used when there are no string values to ensure that 0 is used for the null value + public void addNumericNull() + { + Preconditions.checkState( + stringLookup.size() == 0 && longLookup.size() == 0 && doubleLookup.size() == 0, + "Lookup must be empty to add implicit null" + ); + dictionarySize++; + } + public int lookupString(@Nullable String value) { return stringLookup.getInt(value); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.java b/processing/src/main/java/org/apache/druid/segment/nested/FieldTypeInfo.java similarity index 95% rename from processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.java rename to processing/src/main/java/org/apache/druid/segment/nested/FieldTypeInfo.java index 1d6d251fabac..4471fdab11aa 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/FieldTypeInfo.java @@ -39,7 +39,7 @@ * 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 NestedFieldTypeInfo +public class FieldTypeInfo { private static final byte STRING_MASK = 1; private static final byte LONG_MASK = 1 << 2; @@ -51,9 +51,9 @@ public class NestedFieldTypeInfo private static final byte DOUBLE_ARRAY_MASK = 1 << 6; - public static NestedFieldTypeInfo read(ByteBuffer buffer, int length) + public static FieldTypeInfo read(ByteBuffer buffer, int length) { - NestedFieldTypeInfo typeInfo = new NestedFieldTypeInfo(buffer); + FieldTypeInfo typeInfo = new FieldTypeInfo(buffer); buffer.position(buffer.position() + length); return typeInfo; } @@ -61,7 +61,7 @@ public static NestedFieldTypeInfo read(ByteBuffer buffer, int length) private final ByteBuffer buffer; private final int startOffset; - public NestedFieldTypeInfo(ByteBuffer buffer) + public FieldTypeInfo(ByteBuffer buffer) { this.buffer = buffer; this.startOffset = buffer.position(); @@ -87,7 +87,7 @@ public TypeSet(byte types) @Nullable public ColumnType getSingleType() { - return NestedFieldTypeInfo.getSingleType(types); + return FieldTypeInfo.getSingleType(types); } public byte getByteValue() @@ -178,7 +178,7 @@ public MutableTypeSet merge(byte other) @Nullable public ColumnType getSingleType() { - return NestedFieldTypeInfo.getSingleType(types); + return FieldTypeInfo.getSingleType(types); } public boolean isEmpty() 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 0c308ff6b488..4800d211771a 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 @@ -50,11 +50,11 @@ import java.nio.channels.WritableByteChannel; /** - * Base class for writer of global dictionary encoded nested literal columns for {@link NestedDataColumnSerializer}. - * {@link NestedDataColumnSerializer} while processing the 'raw' nested data will call {@link #addValue(int, Object)} - * for all literal writers, which for this type of writer entails building a local dictionary to map into to the global - * dictionary ({@link #localDictionary}) and writes this unsorted localId to an intermediate integer column, - * {@link #intermediateValueWriter}. + * Base class for writer of global dictionary encoded nested literal columns for {@link NestedDataColumnSerializerV4} and + * {@link NestedDataColumnSerializer}. While processing the 'raw' nested data, the + * serializers will call {@link #addValue(int, Object)} for writers, which for this type of writer entails building a + * local dictionary to map into to the global dictionary ({@link #localDictionary}) and writes this unsorted localId to + * an intermediate integer column, {@link #intermediateValueWriter}. *

* When processing the 'raw' value column is complete, the {@link #writeTo(int, FileSmoosher)} method will sort the * local ids and write them out to a local sorted dictionary, iterate over {@link #intermediateValueWriter} swapping @@ -69,7 +69,7 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter protected final String columnName; protected final String fieldName; protected final IndexSpec indexSpec; - protected final GlobalDictionaryIdLookup globalDictionaryIdLookup; + protected final DictionaryIdLookup globalDictionaryIdLookup; protected final LocalDimensionDictionary localDictionary = new LocalDimensionDictionary(); protected final Int2ObjectRBTreeMap arrayElements = new Int2ObjectRBTreeMap<>(); @@ -87,7 +87,7 @@ protected GlobalDictionaryEncodedFieldColumnWriter( String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec, - GlobalDictionaryIdLookup globalDictionaryIdLookup + DictionaryIdLookup globalDictionaryIdLookup ) { this.columnName = columnName; @@ -281,7 +281,7 @@ public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws I } } }; - final String fieldFileName = NestedDataColumnSerializer.getInternalFileName(columnName, fieldName); + final String fieldFileName = NestedDataColumnSerializerV4.getInternalFileName(columnName, fieldName); final long size = fieldSerializer.getSerializedSize(); log.debug("Column [%s] serializing [%s] field of size [%d].", columnName, fieldName, size); try (SmooshedWriter smooshChannel = smoosher.addWithSmooshedWriter(fieldFileName, size)) { diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java new file mode 100644 index 000000000000..b52670c148f6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java @@ -0,0 +1,144 @@ +/* + * 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 org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.AutoTypeColumnMerger; +import org.apache.druid.segment.AutoTypeColumnSchema; +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.NestedCommonFormatColumnHandler; +import org.apache.druid.segment.column.BaseColumn; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnFormat; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.Indexed; + +import javax.annotation.Nullable; +import java.util.SortedMap; +import java.util.TreeMap; + +/** + * Base implementation for columns created with {@link AutoTypeColumnSchema} and handled with + * {@link NestedCommonFormatColumnHandler} to allow ease of merge via + * {@link AutoTypeColumnMerger} + */ +public interface NestedCommonFormatColumn extends BaseColumn +{ + default Indexed getStringDictionary() + { + return Indexed.empty(); + } + + default Indexed getLongDictionary() + { + return Indexed.empty(); + } + + default Indexed getDoubleDictionary() + { + return Indexed.empty(); + } + + default Indexed getArrayDictionary() + { + return Indexed.empty(); + } + + + default SortedMap getFieldTypeInfo() + { + FieldTypeInfo.MutableTypeSet rootOnlyType = new FieldTypeInfo.MutableTypeSet().add(getLogicalType()); + SortedMap fields = new TreeMap<>(); + fields.put(NestedPathFinder.JSON_PATH_ROOT, rootOnlyType); + return fields; + } + + ColumnType getLogicalType(); + + class Format implements ColumnFormat + { + private final ColumnType logicalType; + private final boolean hasNulls; + + public Format(ColumnType logicalType, boolean hasNulls) + { + this.logicalType = logicalType; + this.hasNulls = hasNulls; + } + + @Override + public ColumnType getLogicalType() + { + return logicalType; + } + + @Override + public DimensionHandler getColumnHandler(String columnName) + { + return new NestedCommonFormatColumnHandler(columnName); + } + + @Override + public DimensionSchema getColumnSchema(String columnName) + { + return new AutoTypeColumnSchema(columnName); + } + + @Override + public ColumnFormat merge(@Nullable ColumnFormat otherFormat) + { + if (otherFormat == null) { + return this; + } + + if (otherFormat instanceof Format) { + final boolean otherHasNulls = ((Format) otherFormat).hasNulls; + if (!getLogicalType().equals(otherFormat.getLogicalType())) { + return new Format(ColumnType.NESTED_DATA, hasNulls || otherHasNulls); + } + return new Format(logicalType, hasNulls || otherHasNulls); + } + throw new ISE( + "Cannot merge columns of type[%s] and format[%s] and with [%s] and [%s]", + logicalType, + this.getClass().getName(), + otherFormat.getLogicalType(), + otherFormat.getClass().getName() + ); + } + + @Override + public ColumnCapabilities toColumnCapabilities() + { + if (logicalType.isPrimitive() || logicalType.isArray()) { + return ColumnCapabilitiesImpl.createDefault() + .setType(logicalType) + .setDictionaryEncoded(true) + .setDictionaryValuesSorted(true) + .setDictionaryValuesUnique(true) + .setHasBitmapIndexes(true) + .setHasNulls(hasNulls); + } + return ColumnCapabilitiesImpl.createDefault().setType(logicalType).setHasNulls(hasNulls); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnSerializer.java new file mode 100644 index 000000000000..7ccc0d31c4dd --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnSerializer.java @@ -0,0 +1,102 @@ +/* + * 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 org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; +import org.apache.druid.segment.GenericColumnSerializer; +import org.apache.druid.segment.data.VByte; +import org.apache.druid.segment.serde.Serializer; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; +import java.util.SortedMap; + +public abstract class NestedCommonFormatColumnSerializer implements GenericColumnSerializer +{ + public static final byte V0 = 0x00; + 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 ARRAY_ELEMENT_DICTIONARY_FILE_NAME = "__arrayElementDictionary"; + public static final String ENCODED_VALUE_COLUMN_FILE_NAME = "__encodedColumn"; + public static final String LONG_VALUE_COLUMN_FILE_NAME = "__longColumn"; + public static final String DOUBLE_VALUE_COLUMN_FILE_NAME = "__doubleColumn"; + public static final String BITMAP_INDEX_FILE_NAME = "__valueIndexes"; + public static final String ARRAY_ELEMENT_BITMAP_INDEX_FILE_NAME = "__arrayElementIndexes"; + public static final String RAW_FILE_NAME = "__raw"; + public static final String NULL_BITMAP_FILE_NAME = "__nullIndex"; + public static final String NESTED_FIELD_PREFIX = "__field_"; + + public abstract void openDictionaryWriter() throws IOException; + + public void serializeFields(SortedMap fields) throws IOException + { + // nothing to do unless we actually have nested fields + } + + public abstract void serializeDictionaries( + Iterable strings, + Iterable longs, + Iterable doubles, + Iterable arrays + ) throws IOException; + + public abstract String getColumnName(); + + public abstract DictionaryIdLookup getGlobalLookup(); + + public abstract boolean hasNulls(); + + protected void writeInternal(FileSmoosher smoosher, Serializer serializer, String fileName) throws IOException + { + final String internalName = getInternalFileName(getColumnName(), fileName); + try (SmooshedWriter smooshChannel = smoosher.addWithSmooshedWriter(internalName, serializer.getSerializedSize())) { + serializer.writeTo(smooshChannel, smoosher); + } + } + + protected void writeV0Header(WritableByteChannel channel, ByteBuffer columnNameBuffer) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{NestedCommonFormatColumnSerializer.V0})); + channel.write(columnNameBuffer); + } + + protected ByteBuffer computeFilenameBytes() + { + final String columnName = getColumnName(); + final byte[] bytes = StringUtils.toUtf8(columnName); + final int length = VByte.computeIntSize(bytes.length); + final ByteBuffer buffer = ByteBuffer.allocate(length + bytes.length).order(ByteOrder.nativeOrder()); + VByte.writeInt(buffer, bytes.length); + buffer.put(bytes); + buffer.flip(); + return buffer; + } + + public static String getInternalFileName(String fileNameBase, String field) + { + return StringUtils.format("%s.%s", fileNameBase, field); + } +} 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 020dc10ad0f3..526e9f6e6ef1 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 @@ -26,21 +26,17 @@ 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; import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.GenericColumnSerializer; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.ProgressIndicator; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.StringEncodingStrategies; -import org.apache.druid.segment.column.TypeStrategies; -import org.apache.druid.segment.column.TypeStrategy; import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.ByteBufferWriter; @@ -51,7 +47,6 @@ 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; @@ -65,18 +60,9 @@ import java.util.Map; import java.util.SortedMap; -public class NestedDataColumnSerializer implements GenericColumnSerializer +public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializer { private static final Logger log = new Logger(NestedDataColumnSerializer.class); - public static final IntTypeStrategy INT_TYPE_STRATEGY = new IntTypeStrategy(); - 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"; - - public static final String NESTED_FIELD_PREFIX = "__field_"; private final String name; private final SegmentWriteOutMedium segmentWriteOutMedium; @@ -138,11 +124,10 @@ public ProcessedValue processArrayField( } }; - private byte[] metadataBytes; - private GlobalDictionaryIdLookup globalDictionaryIdLookup; - private SortedMap fields; + private DictionaryIdLookup globalDictionaryIdLookup; + private SortedMap fields; private GenericIndexedWriter fieldsWriter; - private NestedFieldTypeInfo.Writer fieldsInfoWriter; + private FieldTypeInfo.Writer fieldsInfoWriter; private DictionaryWriter dictionaryWriter; private FixedIndexedWriter longDictionaryWriter; private FixedIndexedWriter doubleDictionaryWriter; @@ -155,6 +140,7 @@ public ProcessedValue processArrayField( private boolean closedForWrite = false; private boolean dictionarySerialized = false; + private ByteBuffer columnNameBytes = null; public NestedDataColumnSerializer( String name, @@ -168,21 +154,34 @@ public NestedDataColumnSerializer( this.segmentWriteOutMedium = segmentWriteOutMedium; this.indexSpec = indexSpec; this.closer = closer; - this.globalDictionaryIdLookup = new GlobalDictionaryIdLookup(); + this.globalDictionaryIdLookup = new DictionaryIdLookup(); + } + + @Override + public String getColumnName() + { + return name; } - public GlobalDictionaryIdLookup getGlobalLookup() + @Override + public DictionaryIdLookup getGlobalLookup() { return globalDictionaryIdLookup; } @Override - public void open() throws IOException + public boolean hasNulls() + { + return !nullRowsBitmap.isEmpty(); + } + + @Override + public void openDictionaryWriter() throws IOException { fieldsWriter = new GenericIndexedWriter<>(segmentWriteOutMedium, name, GenericIndexed.STRING_STRATEGY); fieldsWriter.open(); - fieldsInfoWriter = new NestedFieldTypeInfo.Writer(segmentWriteOutMedium); + fieldsInfoWriter = new FieldTypeInfo.Writer(segmentWriteOutMedium); fieldsInfoWriter.open(); dictionaryWriter = StringEncodingStrategies.getStringDictionaryWriter( @@ -216,7 +215,11 @@ public void open() throws IOException 4 ); arrayDictionaryWriter.open(); + } + @Override + public void open() throws IOException + { rawWriter = new CompressedVariableSizedBlobColumnSerializer( getInternalFileName(name, RAW_FILE_NAME), segmentWriteOutMedium, @@ -233,12 +236,13 @@ public void open() throws IOException nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); } - public void serializeFields(SortedMap fields) throws IOException + @Override + public void serializeFields(SortedMap fields) throws IOException { this.fields = fields; this.fieldWriters = Maps.newHashMapWithExpectedSize(fields.size()); int ctr = 0; - for (Map.Entry field : fields.entrySet()) { + for (Map.Entry field : fields.entrySet()) { final String fieldName = field.getKey(); final String fieldFileName = NESTED_FIELD_PREFIX + ctr++; fieldsWriter.write(fieldName); @@ -247,7 +251,7 @@ public void serializeFields(SortedMap strings, Iterable longs, @@ -363,8 +368,7 @@ public void serialize(ColumnValueSelector selector) th rowCount++; } - @Override - public long getSerializedSize() throws IOException + private void closeForWrite() throws IOException { if (!closedForWrite) { closedForWrite = true; @@ -380,12 +384,17 @@ public long getSerializedSize() throws IOException ) ) ); - this.metadataBytes = baos.toByteArray(); - this.nullBitmapWriter.write(nullRowsBitmap); + nullBitmapWriter.write(nullRowsBitmap); + columnNameBytes = computeFilenameBytes(); } + } - long size = 1; - size += metadataBytes.length; + @Override + public long getSerializedSize() throws IOException + { + closeForWrite(); + + long size = 1 + columnNameBytes.capacity(); if (fieldsWriter != null) { size += fieldsWriter.getSerializedSize(); } @@ -404,13 +413,10 @@ public void writeTo( { Preconditions.checkState(closedForWrite, "Not closed yet!"); Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); - // version 5 - channel.write(ByteBuffer.wrap(new byte[]{0x05})); - channel.write(ByteBuffer.wrap(metadataBytes)); + + writeV0Header(channel, columnNameBytes); fieldsWriter.writeTo(channel, smoosher); fieldsInfoWriter.writeTo(channel, smoosher); - - // version 3 stores large components in separate files to prevent exceeding smoosh file limit (int max) writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME); writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME); writeInternal(smoosher, doubleDictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME); @@ -430,70 +436,11 @@ public void writeTo( channel.close(); } - for (Map.Entry field : fields.entrySet()) { + for (Map.Entry field : fields.entrySet()) { // remove writer so that it can be collected when we are done with it GlobalDictionaryEncodedFieldColumnWriter writer = fieldWriters.remove(field.getKey()); writer.writeTo(rowCount, smoosher); } log.info("Column [%s] serialized successfully with [%d] nested columns.", name, fields.size()); } - - private void writeInternal(FileSmoosher smoosher, Serializer serializer, String fileName) throws IOException - { - final String internalName = getInternalFileName(name, fileName); - try (SmooshedWriter smooshChannel = smoosher.addWithSmooshedWriter(internalName, serializer.getSerializedSize())) { - serializer.writeTo(smooshChannel, smoosher); - } - } - - public static String getInternalFileName(String fileNameBase, String field) - { - return StringUtils.format("%s.%s", fileNameBase, field); - } - - private static final class IntTypeStrategy implements TypeStrategy - { - @Override - public int estimateSizeBytes(Integer value) - { - return Integer.BYTES; - } - - @Override - public Integer read(ByteBuffer buffer) - { - return buffer.getInt(); - } - - @Override - public Integer read(ByteBuffer buffer, int offset) - { - return buffer.getInt(offset); - } - - @Override - public boolean readRetainsBufferReference() - { - return false; - } - - @Override - public int write(ByteBuffer buffer, Integer value, int maxSizeBytes) - { - TypeStrategies.checkMaxSize(buffer.remaining(), maxSizeBytes, ColumnType.LONG); - final int sizeBytes = Integer.BYTES; - final int remaining = maxSizeBytes - sizeBytes; - if (remaining >= 0) { - buffer.putInt(value); - return sizeBytes; - } - return remaining; - } - - @Override - public int compare(Object o1, Object o2) - { - return Integer.compare(((Number) o1).intValue(), ((Number) o2).intValue()); - } - } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializerV4.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializerV4.java new file mode 100644 index 000000000000..881147a1cd8b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializerV4.java @@ -0,0 +1,401 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Preconditions; +import 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; +import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.GenericColumnSerializer; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.ProgressIndicator; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.StringEncodingStrategies; +import org.apache.druid.segment.column.Types; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.data.ByteBufferWriter; +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.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; + +public class NestedDataColumnSerializerV4 implements GenericColumnSerializer +{ + private static final Logger log = new Logger(NestedDataColumnSerializerV4.class); + 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"; + + public static final String NESTED_FIELD_PREFIX = "__field_"; + + private final String name; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private final IndexSpec indexSpec; + @SuppressWarnings("unused") + private final Closer closer; + + private final StructuredDataProcessor fieldProcessor = new StructuredDataProcessor() + { + @Override + public ProcessedValue processField(ArrayList fieldPath, @Nullable Object fieldValue) + { + final GlobalDictionaryEncodedFieldColumnWriter writer = fieldWriters.get( + NestedPathFinder.toNormalizedJsonPath(fieldPath) + ); + if (writer != null) { + try { + final ExprEval eval = ExprEval.bestEffortOf(fieldValue); + if (eval.type().isPrimitive() || (eval.type().isArray() && eval.type().getElementType().isPrimitive())) { + writer.addValue(rowCount, eval.value()); + } else { + // behave consistently with nested column indexer, which defaults to string + writer.addValue(rowCount, eval.asString()); + } + // serializer doesn't use size estimate + return ProcessedValue.NULL_LITERAL; + } + catch (IOException e) { + throw new RE(e, "Failed to write field [%s], unhandled value", fieldPath); + } + } + return ProcessedValue.NULL_LITERAL; + } + + @Nullable + @Override + public ProcessedValue processArrayField( + ArrayList fieldPath, + @Nullable List array + ) + { + // classic nested column ingestion does not support array fields + return null; + } + }; + + private byte[] metadataBytes; + private DictionaryIdLookup globalDictionaryIdLookup; + private SortedMap fields; + private GenericIndexedWriter fieldsWriter; + private FieldTypeInfo.Writer fieldsInfoWriter; + private DictionaryWriter dictionaryWriter; + private FixedIndexedWriter longDictionaryWriter; + private FixedIndexedWriter doubleDictionaryWriter; + private CompressedVariableSizedBlobColumnSerializer rawWriter; + private ByteBufferWriter nullBitmapWriter; + private MutableBitmap nullRowsBitmap; + private Map> fieldWriters; + private int rowCount = 0; + private boolean closedForWrite = false; + + private boolean dictionarySerialized = false; + + public NestedDataColumnSerializerV4( + String name, + IndexSpec indexSpec, + SegmentWriteOutMedium segmentWriteOutMedium, + @SuppressWarnings("unused") ProgressIndicator progressIndicator, + Closer closer + ) + { + this.name = name; + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.indexSpec = indexSpec; + this.closer = closer; + this.globalDictionaryIdLookup = new DictionaryIdLookup(); + } + + @Override + public void open() throws IOException + { + fieldsWriter = new GenericIndexedWriter<>(segmentWriteOutMedium, name, GenericIndexed.STRING_STRATEGY); + fieldsWriter.open(); + + fieldsInfoWriter = new FieldTypeInfo.Writer(segmentWriteOutMedium); + fieldsInfoWriter.open(); + + dictionaryWriter = StringEncodingStrategies.getStringDictionaryWriter( + indexSpec.getStringDictionaryEncoding(), + segmentWriteOutMedium, + name + ); + dictionaryWriter.open(); + + longDictionaryWriter = new FixedIndexedWriter<>( + segmentWriteOutMedium, + ColumnType.LONG.getStrategy(), + ByteOrder.nativeOrder(), + Long.BYTES, + true + ); + longDictionaryWriter.open(); + + doubleDictionaryWriter = new FixedIndexedWriter<>( + segmentWriteOutMedium, + ColumnType.DOUBLE.getStrategy(), + ByteOrder.nativeOrder(), + Double.BYTES, + true + ); + doubleDictionaryWriter.open(); + + rawWriter = new CompressedVariableSizedBlobColumnSerializer( + getInternalFileName(name, RAW_FILE_NAME), + segmentWriteOutMedium, + indexSpec.getJsonCompression() != null ? indexSpec.getJsonCompression() : CompressionStrategy.LZ4 + ); + rawWriter.open(); + + nullBitmapWriter = new ByteBufferWriter<>( + segmentWriteOutMedium, + indexSpec.getBitmapSerdeFactory().getObjectStrategy() + ); + nullBitmapWriter.open(); + + nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); + } + + public void serializeFields(SortedMap fields) throws IOException + { + this.fields = fields; + this.fieldWriters = Maps.newHashMapWithExpectedSize(fields.size()); + int ctr = 0; + for (Map.Entry field : fields.entrySet()) { + final String fieldName = field.getKey(); + final String fieldFileName = NESTED_FIELD_PREFIX + ctr++; + fieldsWriter.write(fieldName); + fieldsInfoWriter.write(field.getValue()); + final GlobalDictionaryEncodedFieldColumnWriter writer; + final ColumnType type = field.getValue().getSingleType(); + if (type != null) { + if (Types.is(type, ValueType.STRING)) { + writer = new ScalarStringFieldColumnWriter( + name, + fieldFileName, + segmentWriteOutMedium, + indexSpec, + globalDictionaryIdLookup + ); + } else if (Types.is(type, ValueType.LONG)) { + writer = new ScalarLongFieldColumnWriter( + name, + fieldFileName, + segmentWriteOutMedium, + indexSpec, + globalDictionaryIdLookup + ); + } else if (Types.is(type, ValueType.DOUBLE)) { + writer = new ScalarDoubleFieldColumnWriter( + name, + fieldFileName, + segmentWriteOutMedium, + indexSpec, + globalDictionaryIdLookup + ); + } else { + throw new ISE("Invalid field type [%s], how did this happen?", type); + } + } else { + writer = new VariantFieldColumnWriter( + name, + fieldFileName, + segmentWriteOutMedium, + indexSpec, + globalDictionaryIdLookup + ); + } + writer.open(); + fieldWriters.put(fieldName, writer); + } + } + + public void serializeDictionaries( + Iterable strings, + Iterable longs, + Iterable doubles + ) 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 : strings) { + value = NullHandling.emptyToNullIfNeeded(value); + if (value == null) { + continue; + } + + dictionaryWriter.write(value); + globalDictionaryIdLookup.addString(value); + } + dictionarySerialized = true; + + for (Long value : longs) { + if (value == null) { + continue; + } + longDictionaryWriter.write(value); + globalDictionaryIdLookup.addLong(value); + } + + for (Double value : doubles) { + if (value == null) { + continue; + } + doubleDictionaryWriter.write(value); + globalDictionaryIdLookup.addDouble(value); + } + dictionarySerialized = true; + } + + @Override + public void serialize(ColumnValueSelector selector) throws IOException + { + final StructuredData data = StructuredData.wrap(selector.getObject()); + if (data == null) { + nullRowsBitmap.add(rowCount); + } + rawWriter.addValue(NestedDataComplexTypeSerde.INSTANCE.toBytes(data)); + if (data != null) { + fieldProcessor.processFields(data.getValue()); + } + rowCount++; + } + + @Override + public long getSerializedSize() throws IOException + { + if (!closedForWrite) { + closedForWrite = true; + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + IndexMerger.SERIALIZER_UTILS.writeString( + baos, + NestedDataComplexTypeSerde.OBJECT_MAPPER.writeValueAsString( + new NestedDataColumnMetadata( + ByteOrder.nativeOrder(), + indexSpec.getBitmapSerdeFactory(), + name, + !nullRowsBitmap.isEmpty() + ) + ) + ); + this.metadataBytes = baos.toByteArray(); + this.nullBitmapWriter.write(nullRowsBitmap); + } + + long size = 1; + size += metadataBytes.length; + if (fieldsWriter != null) { + size += fieldsWriter.getSerializedSize(); + } + if (fieldsInfoWriter != null) { + size += fieldsInfoWriter.getSerializedSize(); + } + // the value dictionaries, raw column, and null index are all stored in separate files + return size; + } + + @Override + public void writeTo( + WritableByteChannel channel, + FileSmoosher smoosher + ) throws IOException + { + Preconditions.checkState(closedForWrite, "Not closed yet!"); + Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + // version 5 + channel.write(ByteBuffer.wrap(new byte[]{0x04})); + channel.write(ByteBuffer.wrap(metadataBytes)); + fieldsWriter.writeTo(channel, smoosher); + fieldsInfoWriter.writeTo(channel, smoosher); + + // version 3 stores large components in separate files to prevent exceeding smoosh file limit (int max) + writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME); + writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME); + writeInternal(smoosher, doubleDictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME); + writeInternal(smoosher, rawWriter, RAW_FILE_NAME); + if (!nullRowsBitmap.isEmpty()) { + writeInternal(smoosher, nullBitmapWriter, NULL_BITMAP_FILE_NAME); + } + + + // close the SmooshedWriter since we are done here, so we don't write to a temporary file per sub-column + // In the future, it would be best if the writeTo() itself didn't take a channel but was expected to actually + // open its own channels on the FileSmoosher object itself. Or some other thing that give this Serializer + // total control over when resources are opened up and when they are closed. Until then, we are stuck + // with a very tight coupling of this code with how the external "driver" is working. + if (channel instanceof SmooshedWriter) { + channel.close(); + } + + for (Map.Entry field : fields.entrySet()) { + // remove writer so that it can be collected when we are done with it + GlobalDictionaryEncodedFieldColumnWriter writer = fieldWriters.remove(field.getKey()); + writer.writeTo(rowCount, smoosher); + } + log.info("Column [%s] serialized successfully with [%d] nested columns.", name, fields.size()); + } + + private void writeInternal(FileSmoosher smoosher, Serializer serializer, String fileName) throws IOException + { + final String internalName = getInternalFileName(name, fileName); + try (SmooshedWriter smooshChannel = smoosher.addWithSmooshedWriter(internalName, serializer.getSerializedSize())) { + serializer.writeTo(smooshChannel, smoosher); + } + } + + public static String getInternalFileName(String fileNameBase, String field) + { + return StringUtils.format("%s.%s", fileNameBase, field); + } + +} 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 15dc2d888fe7..0a572577d087 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 @@ -19,66 +19,51 @@ package org.apache.druid.segment.nested; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; import org.apache.druid.collections.bitmap.ImmutableBitmap; 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.smoosh.SmooshedFileMapper; -import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.ComplexColumn; import org.apache.druid.segment.column.StringEncodingStrategy; -import org.apache.druid.segment.column.TypeStrategy; +import org.apache.druid.segment.data.BitmapSerdeFactory; 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 org.apache.druid.segment.data.VByte; +import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.ByteOrder; -public class NestedDataColumnSupplier implements Supplier +public class NestedDataColumnSupplier implements Supplier { public static NestedDataColumnSupplier read( + boolean hasNulls, ByteBuffer bb, ColumnBuilder columnBuilder, ColumnConfig columnConfig, - ObjectMapper jsonMapper - ) - { - return read( - bb, - columnBuilder, - columnConfig, - jsonMapper, - ColumnType.LONG.getStrategy(), - ColumnType.DOUBLE.getStrategy() - ); - } - - public static NestedDataColumnSupplier read( - ByteBuffer bb, - ColumnBuilder columnBuilder, - ColumnConfig columnConfig, - ObjectMapper jsonMapper, - TypeStrategy longTypeStrategy, - TypeStrategy doubleTypeStrategy + BitmapSerdeFactory bitmapSerdeFactory, + ByteOrder byteOrder ) { final byte version = bb.get(); + final int columnNameLength = VByte.readInt(bb); + final String columnName = StringUtils.fromUtf8(bb, columnNameLength); - if (version == 0x03 || version == 0x04 || version == 0x05) { + if (version == NestedCommonFormatColumnSerializer.V0) { try { final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); - final NestedDataColumnMetadata metadata; final GenericIndexed fields; - final NestedFieldTypeInfo fieldInfo; + final FieldTypeInfo fieldInfo; final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier; final ImmutableBitmap nullValues; final GenericIndexed stringDictionary; @@ -88,31 +73,23 @@ public static NestedDataColumnSupplier read( final Supplier arrayDictionarySupplier; ColumnType simpleType; - - metadata = jsonMapper.readValue( - IndexMerger.SERIALIZER_UTILS.readString(bb), - NestedDataColumnMetadata.class - ); fields = GenericIndexed.read(bb, GenericIndexed.STRING_STRATEGY, mapper); - fieldInfo = NestedFieldTypeInfo.read(bb, fields.size()); + fieldInfo = FieldTypeInfo.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 // it is the most permissive (besides json) simpleType = ColumnType.STRING; - } else if (fields.size() == 1 && - ((version == 0x03 && NestedPathFinder.JQ_PATH_ROOT.equals(fields.get(0))) || - ((version == 0x04 || version == 0x05) && NestedPathFinder.JSON_PATH_ROOT.equals(fields.get(0)))) - ) { + } else if (fields.size() == 1 && NestedPathFinder.JSON_PATH_ROOT.equals(fields.get(0))) { simpleType = fieldInfo.getTypes(0).getSingleType(); } else { simpleType = null; } - final ByteBuffer stringDictionaryBuffer = loadInternalFile( + final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, - metadata, - NestedDataColumnSerializer.STRING_DICTIONARY_FILE_NAME + columnName, + NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME ); final int dictionaryStartPosition = stringDictionaryBuffer.position(); @@ -123,7 +100,7 @@ public static NestedDataColumnSupplier read( if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) { frontCodedStringDictionarySupplier = FrontCodedIndexed.read( stringDictionaryBuffer, - metadata.getByteOrder() + byteOrder ); stringDictionary = null; } else if (encodingId == StringEncodingStrategy.UTF8_ID) { @@ -143,65 +120,65 @@ public static NestedDataColumnSupplier read( stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper); frontCodedStringDictionarySupplier = null; } - final ByteBuffer longDictionaryBuffer = loadInternalFile( + final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, - metadata, - NestedDataColumnSerializer.LONG_DICTIONARY_FILE_NAME + columnName, + NestedCommonFormatColumnSerializer.LONG_DICTIONARY_FILE_NAME ); longDictionarySupplier = FixedIndexed.read( longDictionaryBuffer, - longTypeStrategy, - metadata.getByteOrder(), + ColumnType.LONG.getStrategy(), + byteOrder, Long.BYTES ); - final ByteBuffer doubleDictionaryBuffer = loadInternalFile( + final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, - metadata, - NestedDataColumnSerializer.DOUBLE_DICTIONARY_FILE_NAME + columnName, + NestedCommonFormatColumnSerializer.DOUBLE_DICTIONARY_FILE_NAME ); doubleDictionarySupplier = FixedIndexed.read( doubleDictionaryBuffer, - doubleTypeStrategy, - metadata.getByteOrder(), + ColumnType.DOUBLE.getStrategy(), + byteOrder, 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); + final ByteBuffer arrayDictionarybuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.ARRAY_DICTIONARY_FILE_NAME + ); + arrayDictionarySupplier = FrontCodedIntArrayIndexed.read( + arrayDictionarybuffer, + byteOrder + ); + final ByteBuffer rawBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.RAW_FILE_NAME + ); compressedRawColumnSupplier = CompressedVariableSizedBlobColumnSupplier.fromByteBuffer( - NestedDataColumnSerializer.getInternalFileName( - metadata.getFileNameBase(), NestedDataColumnSerializer.RAW_FILE_NAME + NestedCommonFormatColumnSerializer.getInternalFileName( + columnName, + NestedCommonFormatColumnSerializer.RAW_FILE_NAME ), rawBuffer, - metadata.getByteOrder(), + byteOrder, mapper ); - if (metadata.hasNulls()) { + if (hasNulls) { columnBuilder.setHasNulls(true); - final ByteBuffer nullIndexBuffer = loadInternalFile( + final ByteBuffer nullIndexBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, - metadata, - NestedDataColumnSerializer.NULL_BITMAP_FILE_NAME + columnName, + NestedCommonFormatColumnSerializer.NULL_BITMAP_FILE_NAME ); - nullValues = metadata.getBitmapSerdeFactory().getObjectStrategy().fromByteBufferWithSize(nullIndexBuffer); + nullValues = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(nullIndexBuffer); } else { - nullValues = metadata.getBitmapSerdeFactory().getBitmapFactory().makeEmptyImmutableBitmap(); + nullValues = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); } return new NestedDataColumnSupplier( - version, - metadata, + columnName, fields, fieldInfo, compressedRawColumnSupplier, @@ -213,6 +190,8 @@ public static NestedDataColumnSupplier read( arrayDictionarySupplier, columnConfig, mapper, + bitmapSerdeFactory, + byteOrder, simpleType ); } @@ -224,10 +203,9 @@ public static NestedDataColumnSupplier read( } } - private final byte version; - private final NestedDataColumnMetadata metadata; + private final String columnName; private final GenericIndexed fields; - private final NestedFieldTypeInfo fieldInfo; + private final FieldTypeInfo fieldInfo; private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier; private final ImmutableBitmap nullValues; private final GenericIndexed stringDictionary; @@ -237,15 +215,16 @@ public static NestedDataColumnSupplier read( private final Supplier arrayDictionarySupplier; private final ColumnConfig columnConfig; private final SmooshedFileMapper fileMapper; + private final BitmapSerdeFactory bitmapSerdeFactory; + private final ByteOrder byteOrder; @Nullable private final ColumnType simpleType; private NestedDataColumnSupplier( - byte version, - NestedDataColumnMetadata metadata, + String columnName, GenericIndexed fields, - NestedFieldTypeInfo fieldInfo, + FieldTypeInfo fieldInfo, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, GenericIndexed stringDictionary, @@ -255,11 +234,12 @@ private NestedDataColumnSupplier( Supplier arrayDictionarySupplier, ColumnConfig columnConfig, SmooshedFileMapper fileMapper, + BitmapSerdeFactory bitmapSerdeFactory, + ByteOrder byteOrder, @Nullable ColumnType simpleType ) { - this.version = version; - this.metadata = metadata; + this.columnName = columnName; this.fields = fields; this.fieldInfo = fieldInfo; this.compressedRawColumnSupplier = compressedRawColumnSupplier; @@ -271,91 +251,18 @@ private NestedDataColumnSupplier( this.arrayDictionarySupplier = arrayDictionarySupplier; this.columnConfig = columnConfig; this.fileMapper = fileMapper; + this.bitmapSerdeFactory = bitmapSerdeFactory; + this.byteOrder = byteOrder; this.simpleType = simpleType; } @Override - public ComplexColumn get() - { - if (version == 0x03) { - return makeV3(); - } else if (version == 0x04) { - return makeV4(); - } - return makeV5(); - } - - @Nullable - public ColumnType getSimpleType() - { - return simpleType; - } - - private NestedDataColumnV3 makeV3() - { - if (frontCodedStringDictionarySupplier != null) { - return new NestedDataColumnV3<>( - metadata, - columnConfig, - compressedRawColumnSupplier, - nullValues, - fields, - fieldInfo, - frontCodedStringDictionarySupplier, - longDictionarySupplier, - doubleDictionarySupplier, - fileMapper - ); - } - return new NestedDataColumnV3<>( - metadata, - columnConfig, - compressedRawColumnSupplier, - nullValues, - fields, - fieldInfo, - stringDictionary::singleThreaded, - longDictionarySupplier, - doubleDictionarySupplier, - fileMapper - ); - } - - private NestedDataColumnV4 makeV4() - { - if (frontCodedStringDictionarySupplier != null) { - return new NestedDataColumnV4<>( - metadata, - columnConfig, - compressedRawColumnSupplier, - nullValues, - fields, - fieldInfo, - frontCodedStringDictionarySupplier, - longDictionarySupplier, - doubleDictionarySupplier, - fileMapper - ); - } - return new NestedDataColumnV4<>( - metadata, - columnConfig, - compressedRawColumnSupplier, - nullValues, - fields, - fieldInfo, - stringDictionary::singleThreaded, - longDictionarySupplier, - doubleDictionarySupplier, - fileMapper - ); - } - - private NestedDataColumnV5 makeV5() + public NestedCommonFormatColumn get() { if (frontCodedStringDictionarySupplier != null) { return new NestedDataColumnV5<>( - metadata, + columnName, + getLogicalType(), columnConfig, compressedRawColumnSupplier, nullValues, @@ -365,11 +272,14 @@ private NestedDataColumnV5 makeV5() longDictionarySupplier, doubleDictionarySupplier, arrayDictionarySupplier, - fileMapper + fileMapper, + bitmapSerdeFactory, + byteOrder ); } return new NestedDataColumnV5<>( - metadata, + columnName, + getLogicalType(), columnConfig, compressedRawColumnSupplier, nullValues, @@ -379,18 +289,14 @@ private NestedDataColumnV5 makeV5() longDictionarySupplier, doubleDictionarySupplier, arrayDictionarySupplier, - fileMapper + fileMapper, + bitmapSerdeFactory, + byteOrder ); } - private static ByteBuffer loadInternalFile( - SmooshedFileMapper fileMapper, - NestedDataColumnMetadata metadata, - String internalFileName - ) throws IOException + public ColumnType getLogicalType() { - return fileMapper.mapFile( - NestedDataColumnSerializer.getInternalFileName(metadata.getFileNameBase(), internalFileName) - ); + return simpleType == null ? ColumnType.NESTED_DATA : simpleType; } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4.java new file mode 100644 index 000000000000..721da12ce0ac --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4.java @@ -0,0 +1,428 @@ +/* + * 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.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.ComplexColumn; +import org.apache.druid.segment.column.StringEncodingStrategy; +import org.apache.druid.segment.column.TypeStrategy; +import org.apache.druid.segment.data.BitmapSerdeFactory; +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; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class NestedDataColumnSupplierV4 implements Supplier +{ + public static NestedDataColumnSupplierV4 read( + ByteBuffer bb, + ColumnBuilder columnBuilder, + ColumnConfig columnConfig, + ObjectMapper jsonMapper + ) + { + return read( + bb, + columnBuilder, + columnConfig, + jsonMapper, + ColumnType.LONG.getStrategy(), + ColumnType.DOUBLE.getStrategy() + ); + } + + public static NestedDataColumnSupplierV4 read( + ByteBuffer bb, + ColumnBuilder columnBuilder, + ColumnConfig columnConfig, + ObjectMapper jsonMapper, + TypeStrategy longTypeStrategy, + TypeStrategy doubleTypeStrategy + ) + { + final byte version = bb.get(); + + // v5 was never actually released, but it existed for a short time in the master branch, and doesn't hurt to be here + if (version == 0x03 || version == 0x04 || version == 0x05) { + try { + final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); + final NestedDataColumnMetadata metadata; + final GenericIndexed fields; + final FieldTypeInfo fieldInfo; + final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier; + final ImmutableBitmap nullValues; + final GenericIndexed stringDictionary; + final Supplier frontCodedStringDictionarySupplier; + final Supplier> longDictionarySupplier; + final Supplier> doubleDictionarySupplier; + final Supplier arrayDictionarySupplier; + + ColumnType simpleType; + + metadata = jsonMapper.readValue( + IndexMerger.SERIALIZER_UTILS.readString(bb), + NestedDataColumnMetadata.class + ); + fields = GenericIndexed.read(bb, GenericIndexed.STRING_STRATEGY, mapper); + fieldInfo = FieldTypeInfo.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 + // it is the most permissive (besides json) + simpleType = ColumnType.STRING; + } else if (fields.size() == 1 && + ((version == 0x03 && NestedPathFinder.JQ_PATH_ROOT.equals(fields.get(0))) || + ((version == 0x04 || version == 0x05) && NestedPathFinder.JSON_PATH_ROOT.equals(fields.get(0)))) + ) { + simpleType = fieldInfo.getTypes(0).getSingleType(); + } else { + simpleType = null; + } + + final ByteBuffer stringDictionaryBuffer = loadInternalFile( + mapper, + metadata, + NestedDataColumnSerializerV4.STRING_DICTIONARY_FILE_NAME + ); + + final int dictionaryStartPosition = stringDictionaryBuffer.position(); + final byte dictionaryVersion = stringDictionaryBuffer.get(); + + if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) { + final byte encodingId = stringDictionaryBuffer.get(); + if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) { + frontCodedStringDictionarySupplier = FrontCodedIndexed.read( + stringDictionaryBuffer, + metadata.getByteOrder() + ); + stringDictionary = null; + } else if (encodingId == StringEncodingStrategy.UTF8_ID) { + // this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but + // this provides backwards compatibility should we switch at some point in the future to always + // writing dictionaryVersion + stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper); + frontCodedStringDictionarySupplier = null; + } else { + throw new ISE("impossible, unknown encoding strategy id: %s", encodingId); + } + } else { + // legacy format that only supports plain utf8 enoding stored in GenericIndexed and the byte we are reading + // as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the + // GenericIndexed version can be correctly read + stringDictionaryBuffer.position(dictionaryStartPosition); + stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper); + frontCodedStringDictionarySupplier = null; + } + final ByteBuffer longDictionaryBuffer = loadInternalFile( + mapper, + metadata, + NestedDataColumnSerializerV4.LONG_DICTIONARY_FILE_NAME + ); + longDictionarySupplier = FixedIndexed.read( + longDictionaryBuffer, + longTypeStrategy, + metadata.getByteOrder(), + Long.BYTES + ); + final ByteBuffer doubleDictionaryBuffer = loadInternalFile( + mapper, + metadata, + NestedDataColumnSerializerV4.DOUBLE_DICTIONARY_FILE_NAME + ); + doubleDictionarySupplier = FixedIndexed.read( + doubleDictionaryBuffer, + doubleTypeStrategy, + metadata.getByteOrder(), + Double.BYTES + ); + if (version == 0x05) { + final ByteBuffer arrayDictionarybuffer = loadInternalFile( + mapper, + metadata, + NestedDataColumnSerializerV4.ARRAY_DICTIONARY_FILE_NAME + ); + arrayDictionarySupplier = FrontCodedIntArrayIndexed.read( + arrayDictionarybuffer, + metadata.getByteOrder() + ); + } else { + arrayDictionarySupplier = null; + } + final ByteBuffer rawBuffer = loadInternalFile(mapper, metadata, NestedDataColumnSerializerV4.RAW_FILE_NAME); + compressedRawColumnSupplier = CompressedVariableSizedBlobColumnSupplier.fromByteBuffer( + NestedDataColumnSerializerV4.getInternalFileName( + metadata.getFileNameBase(), NestedDataColumnSerializerV4.RAW_FILE_NAME + ), + rawBuffer, + metadata.getByteOrder(), + mapper + ); + if (metadata.hasNulls()) { + columnBuilder.setHasNulls(true); + final ByteBuffer nullIndexBuffer = loadInternalFile( + mapper, + metadata, + NestedDataColumnSerializerV4.NULL_BITMAP_FILE_NAME + ); + nullValues = metadata.getBitmapSerdeFactory().getObjectStrategy().fromByteBufferWithSize(nullIndexBuffer); + } else { + nullValues = metadata.getBitmapSerdeFactory().getBitmapFactory().makeEmptyImmutableBitmap(); + } + + return new NestedDataColumnSupplierV4( + version, + metadata.getFileNameBase(), + columnConfig, + fields, + fieldInfo, + compressedRawColumnSupplier, + nullValues, + stringDictionary, + frontCodedStringDictionarySupplier, + longDictionarySupplier, + doubleDictionarySupplier, + arrayDictionarySupplier, + mapper, + metadata.getBitmapSerdeFactory(), + metadata.getByteOrder(), + simpleType + ); + } + catch (IOException ex) { + throw new RE(ex, "Failed to deserialize V%s column.", version); + } + } else { + throw new RE("Unknown version " + version); + } + } + + private final byte version; + private final String columnName; + private final ColumnConfig columnConfig; + private final GenericIndexed fields; + private final FieldTypeInfo fieldInfo; + private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier; + private final ImmutableBitmap nullValues; + private final GenericIndexed stringDictionary; + private final Supplier frontCodedStringDictionarySupplier; + private final Supplier> longDictionarySupplier; + private final Supplier> doubleDictionarySupplier; + private final Supplier arrayDictionarySupplier; + private final SmooshedFileMapper fileMapper; + + @Nullable + private final ColumnType simpleType; + private final ColumnType logicalType; + private final BitmapSerdeFactory bitmapSerdeFactory; + private final ByteOrder byteOrder; + + private NestedDataColumnSupplierV4( + byte version, + String columnName, + ColumnConfig columnConfig, + GenericIndexed fields, + FieldTypeInfo fieldInfo, + CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, + ImmutableBitmap nullValues, + GenericIndexed stringDictionary, + Supplier frontCodedStringDictionarySupplier, + Supplier> longDictionarySupplier, + Supplier> doubleDictionarySupplier, + Supplier arrayDictionarySupplier, + SmooshedFileMapper fileMapper, + BitmapSerdeFactory bitmapSerdeFactory, + ByteOrder byteOrder, + @Nullable ColumnType simpleType + ) + { + this.version = version; + this.columnName = columnName; + this.columnConfig = columnConfig; + this.fields = fields; + this.fieldInfo = fieldInfo; + this.compressedRawColumnSupplier = compressedRawColumnSupplier; + this.nullValues = nullValues; + this.stringDictionary = stringDictionary; + this.frontCodedStringDictionarySupplier = frontCodedStringDictionarySupplier; + this.longDictionarySupplier = longDictionarySupplier; + this.doubleDictionarySupplier = doubleDictionarySupplier; + this.arrayDictionarySupplier = arrayDictionarySupplier; + this.fileMapper = fileMapper; + this.bitmapSerdeFactory = bitmapSerdeFactory; + this.byteOrder = byteOrder; + this.simpleType = simpleType; + this.logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType; + } + + @Override + public ComplexColumn get() + { + if (version == 0x03) { + return makeV3(); + } else if (version == 0x04) { + return makeV4(); + } + // v5 was never actually released, but it existed for a short time in the master branch, and doesn't hurt to be here + return makeV5(); + } + + @Nullable + public ColumnType getSimpleType() + { + return simpleType; + } + + private NestedDataColumnV3 makeV3() + { + if (frontCodedStringDictionarySupplier != null) { + return new NestedDataColumnV3<>( + columnName, + logicalType, + columnConfig, + compressedRawColumnSupplier, + nullValues, + fields, + fieldInfo, + frontCodedStringDictionarySupplier, + longDictionarySupplier, + doubleDictionarySupplier, + fileMapper, + bitmapSerdeFactory, + byteOrder + ); + } + return new NestedDataColumnV3<>( + columnName, + logicalType, + columnConfig, + compressedRawColumnSupplier, + nullValues, + fields, + fieldInfo, + stringDictionary::singleThreaded, + longDictionarySupplier, + doubleDictionarySupplier, + fileMapper, + bitmapSerdeFactory, + byteOrder + ); + } + + private NestedDataColumnV4 makeV4() + { + if (frontCodedStringDictionarySupplier != null) { + return new NestedDataColumnV4<>( + columnName, + logicalType, + columnConfig, + compressedRawColumnSupplier, + nullValues, + fields, + fieldInfo, + frontCodedStringDictionarySupplier, + longDictionarySupplier, + doubleDictionarySupplier, + fileMapper, + bitmapSerdeFactory, + byteOrder + ); + } + return new NestedDataColumnV4<>( + columnName, + logicalType, + columnConfig, + compressedRawColumnSupplier, + nullValues, + fields, + fieldInfo, + stringDictionary::singleThreaded, + longDictionarySupplier, + doubleDictionarySupplier, + fileMapper, + bitmapSerdeFactory, + byteOrder + ); + } + + private NestedDataColumnV5 makeV5() + { + if (frontCodedStringDictionarySupplier != null) { + return new NestedDataColumnV5<>( + columnName, + logicalType, + columnConfig, + compressedRawColumnSupplier, + nullValues, + fields, + fieldInfo, + frontCodedStringDictionarySupplier, + longDictionarySupplier, + doubleDictionarySupplier, + arrayDictionarySupplier, + fileMapper, + bitmapSerdeFactory, + byteOrder + ); + } + return new NestedDataColumnV5<>( + columnName, + logicalType, + columnConfig, + compressedRawColumnSupplier, + nullValues, + fields, + fieldInfo, + stringDictionary::singleThreaded, + longDictionarySupplier, + doubleDictionarySupplier, + arrayDictionarySupplier, + fileMapper, + bitmapSerdeFactory, + byteOrder + ); + } + + private static ByteBuffer loadInternalFile( + SmooshedFileMapper fileMapper, + NestedDataColumnMetadata metadata, + String internalFileName + ) throws IOException + { + return fileMapper.mapFile( + NestedDataColumnSerializerV4.getInternalFileName(metadata.getFileNameBase(), internalFileName) + ); + } +} 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 eadfca84ce2f..906ffe821e26 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 @@ -24,32 +24,39 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.Indexed; import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.List; public final class NestedDataColumnV3> extends CompressedNestedDataComplexColumn { public NestedDataColumnV3( - NestedDataColumnMetadata metadata, + String columnName, + ColumnType logicalType, ColumnConfig columnConfig, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, GenericIndexed fields, - NestedFieldTypeInfo fieldInfo, + FieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, Supplier> doubleDictionarySupplier, - SmooshedFileMapper fileMapper + SmooshedFileMapper fileMapper, + BitmapSerdeFactory bitmapSerdeFactory, + ByteOrder byteOrder ) { super( - metadata, + columnName, + logicalType, columnConfig, compressedRawColumnSupplier, nullValues, @@ -60,6 +67,8 @@ public NestedDataColumnV3( doubleDictionarySupplier, null, fileMapper, + bitmapSerdeFactory, + byteOrder, 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 26631ee7fdde..b721090dd103 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 @@ -23,32 +23,39 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.Indexed; import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.List; public final class NestedDataColumnV4> extends CompressedNestedDataComplexColumn { public NestedDataColumnV4( - NestedDataColumnMetadata metadata, + String columnName, + ColumnType logicalType, ColumnConfig columnConfig, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, GenericIndexed fields, - NestedFieldTypeInfo fieldInfo, + FieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, Supplier> doubleDictionarySupplier, - SmooshedFileMapper fileMapper + SmooshedFileMapper fileMapper, + BitmapSerdeFactory bitmapSerdeFactory, + ByteOrder byteOrder ) { super( - metadata, + columnName, + logicalType, columnConfig, compressedRawColumnSupplier, nullValues, @@ -59,6 +66,8 @@ public NestedDataColumnV4( doubleDictionarySupplier, null, fileMapper, + bitmapSerdeFactory, + byteOrder, NestedPathFinder.JSON_PATH_ROOT ); } @@ -72,9 +81,9 @@ public List parsePath(String path) @Override public String getFieldFileName(String fileNameBase, String field, int fieldIndex) { - return NestedDataColumnSerializer.getInternalFileName( + return NestedDataColumnSerializerV4.getInternalFileName( fileNameBase, - NestedDataColumnSerializer.NESTED_FIELD_PREFIX + fieldIndex + NestedDataColumnSerializerV4.NESTED_FIELD_PREFIX + fieldIndex ); } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java index 755922618106..a6d5843d2385 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java @@ -23,6 +23,8 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.FrontCodedIntArrayIndexed; @@ -30,27 +32,41 @@ import org.apache.druid.segment.data.Indexed; import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.List; +/** + * Nested data column with optimized support for simple arrays. Not actually v5 in the segment since columns are now + * serialized using {@link org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde} instead of the generic + * complex type system. + * + * Not really stored in a segment as V5 since instead of V5 we migrated to {@link NestedCommonFormatColumn} which + * specializes physical format based on the types of data encountered during processing, and so versions are now + * {@link NestedCommonFormatColumnSerializer#V0} for all associated specializations. + */ public class NestedDataColumnV5> extends CompressedNestedDataComplexColumn { public NestedDataColumnV5( - NestedDataColumnMetadata metadata, + String columnName, + ColumnType logicalType, ColumnConfig columnConfig, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, GenericIndexed fields, - NestedFieldTypeInfo fieldInfo, + FieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, Supplier> doubleDictionarySupplier, Supplier arrayDictionarySupplier, - SmooshedFileMapper fileMapper + SmooshedFileMapper fileMapper, + BitmapSerdeFactory bitmapSerdeFactory, + ByteOrder byteOrder ) { super( - metadata, + columnName, + logicalType, columnConfig, compressedRawColumnSupplier, nullValues, @@ -61,6 +77,8 @@ public NestedDataColumnV5( doubleDictionarySupplier, arrayDictionarySupplier, fileMapper, + bitmapSerdeFactory, + byteOrder, NestedPathFinder.JSON_PATH_ROOT ); } @@ -74,9 +92,9 @@ public List parsePath(String path) @Override public String getFieldFileName(String fileNameBase, String field, int fieldIndex) { - return NestedDataColumnSerializer.getInternalFileName( + return NestedCommonFormatColumnSerializer.getInternalFileName( fileNameBase, - NestedDataColumnSerializer.NESTED_FIELD_PREFIX + fieldIndex + NestedCommonFormatColumnSerializer.NESTED_FIELD_PREFIX + fieldIndex ); } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java index 868b735e213f..a35aa93c25b7 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java @@ -23,13 +23,19 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.fasterxml.jackson.dataformat.smile.SmileGenerator; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Comparators; +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.NestedDataDimensionHandler; +import org.apache.druid.segment.NestedDataDimensionSchema; import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.ObjectStrategy; import org.apache.druid.segment.serde.ComplexMetricExtractor; @@ -42,7 +48,6 @@ public class NestedDataComplexTypeSerde extends ComplexMetricSerde { public static final String TYPE_NAME = "json"; - public static final ColumnType TYPE = ColumnType.ofComplex(TYPE_NAME); public static final ObjectMapper OBJECT_MAPPER; @@ -83,28 +88,24 @@ public void deserializeColumn( ColumnConfig columnConfig ) { - NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read(buffer, builder, columnConfig, OBJECT_MAPPER); - ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + final NestedDataColumnSupplierV4 supplier = NestedDataColumnSupplierV4.read( + buffer, + builder, + columnConfig, + OBJECT_MAPPER + ); + final ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); capabilitiesBuilder.setDictionaryEncoded(true); capabilitiesBuilder.setDictionaryValuesSorted(true); capabilitiesBuilder.setDictionaryValuesUnique(true); - ColumnType simpleType = supplier.getSimpleType(); + final ColumnType simpleType = supplier.getSimpleType(); if (simpleType != null) { builder.setType(simpleType); } else { builder.setComplexTypeName(TYPE_NAME); - } builder.setComplexColumnSupplier(supplier); - - // always use the nested column dimension handler, regardless what we claim our query time type is - builder.setHandlerCapabilities( - new ColumnCapabilitiesImpl().setType(TYPE) - .setDictionaryEncoded(true) - .setDictionaryValuesUnique(true) - .setDictionaryValuesSorted(true) - .setHasNulls(capabilitiesBuilder.hasNulls()) - ); + builder.setColumnFormat(new NestedColumnFormatV4()); } @Override @@ -112,7 +113,6 @@ public ObjectStrategy getObjectStrategy() { return new ObjectStrategy() { - @Override public int compare(Object o1, Object o2) { @@ -156,4 +156,38 @@ public byte[] toBytes(@Nullable Object val) } }; } + + public static class NestedColumnFormatV4 implements ColumnFormat + { + @Override + public ColumnType getLogicalType() + { + return ColumnType.NESTED_DATA; + } + + @Override + public DimensionHandler getColumnHandler(String columnName) + { + return new NestedDataDimensionHandler(columnName); + } + + @Override + public DimensionSchema getColumnSchema(String columnName) + { + return new NestedDataDimensionSchema(columnName); + } + + @Override + public ColumnFormat merge(@Nullable ColumnFormat otherFormat) + { + // we don't care, we are anything, there is no configurability + return this; + } + + @Override + public ColumnCapabilities toColumnCapabilities() + { + return ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA).setHasNulls(true); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java index b4e59d486bc8..7cf2655440c8 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java @@ -94,7 +94,7 @@ public class NestedFieldColumnIndexSupplier bitmaps, Supplier> localDictionarySupplier, @@ -132,36 +132,36 @@ public T as(Class clazz) } return (T) (NullValueIndex) () -> nullIndex; } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) || clazz.equals(DictionaryEncodedValueIndex.class)) { - return (T) new NestedLiteralDictionaryEncodedStringValueIndex(); + return (T) new NestedFieldDictionaryEncodedStringValueIndex(); } if (singleType != null) { switch (singleType.getType()) { case STRING: if (clazz.equals(StringValueSetIndex.class)) { - return (T) new NestedStringLiteralValueSetIndex(); + return (T) new NestedStringValueSetIndex(); } else if (clazz.equals(LexicographicalRangeIndex.class)) { - return (T) new NestedStringLiteralLexicographicalRangeIndex(); + return (T) new NestedStringLexicographicalRangeIndex(); } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new NestedStringLiteralPredicateIndex(); + return (T) new NestedStringPredicateIndex(); } return null; case LONG: if (clazz.equals(StringValueSetIndex.class)) { - return (T) new NestedLongLiteralValueSetIndex(); + return (T) new NestedLongValueSetIndex(); } else if (clazz.equals(NumericRangeIndex.class)) { - return (T) new NestedLongLiteralNumericRangeIndex(); + return (T) new NestedLongNumericRangeIndex(); } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new NestedLongLiteralPredicateIndex(); + return (T) new NestedLongPredicateIndex(); } return null; case DOUBLE: if (clazz.equals(StringValueSetIndex.class)) { - return (T) new NestedDoubleLiteralValueSetIndex(); + return (T) new NestedDoubleValueSetIndex(); } else if (clazz.equals(NumericRangeIndex.class)) { - return (T) new NestedDoubleLiteralNumericRangeIndex(); + return (T) new NestedDoubleNumericRangeIndex(); } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new NestedDoubleLiteralPredicateIndex(); + return (T) new NestedDoublePredicateIndex(); } return null; default: @@ -169,9 +169,9 @@ public T as(Class clazz) } } if (clazz.equals(StringValueSetIndex.class)) { - return (T) new NestedVariantLiteralValueSetIndex(); + return (T) new NestedVariantValueSetIndex(); } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new NestedVariantLiteralPredicateIndex(); + return (T) new NestedVariantPredicateIndex(); } return null; } @@ -301,7 +301,7 @@ public ImmutableBitmap next() }; } - private class NestedLiteralDictionaryEncodedStringValueIndex implements DictionaryEncodedStringValueIndex + private class NestedFieldDictionaryEncodedStringValueIndex implements DictionaryEncodedStringValueIndex { final FixedIndexed localDictionary = localDictionarySupplier.get(); final Indexed stringDictionary = globalStringDictionarySupplier.get(); @@ -335,7 +335,7 @@ public ImmutableBitmap getBitmap(int idx) } } - private class NestedStringLiteralValueSetIndex implements StringValueSetIndex + private class NestedStringValueSetIndex implements StringValueSetIndex { @Override public BitmapColumnIndex forValue(@Nullable String value) @@ -417,7 +417,7 @@ private void findNext() } } - private class NestedStringLiteralLexicographicalRangeIndex implements LexicographicalRangeIndex + private class NestedStringLexicographicalRangeIndex implements LexicographicalRangeIndex { @Override public BitmapColumnIndex forRange( @@ -511,7 +511,7 @@ public ImmutableBitmap next() } } - private class NestedStringLiteralPredicateIndex implements DruidPredicateIndex + private class NestedStringPredicateIndex implements DruidPredicateIndex { @Override public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory) @@ -574,7 +574,7 @@ private void findNext() } } - private class NestedLongLiteralValueSetIndex implements StringValueSetIndex + private class NestedLongValueSetIndex implements StringValueSetIndex { @Override public BitmapColumnIndex forValue(@Nullable String value) @@ -694,7 +694,7 @@ private void findNext() } } - private class NestedLongLiteralNumericRangeIndex implements NumericRangeIndex + private class NestedLongNumericRangeIndex implements NumericRangeIndex { @Override public BitmapColumnIndex forRange( @@ -716,7 +716,7 @@ public BitmapColumnIndex forRange( } } - private class NestedLongLiteralPredicateIndex implements DruidPredicateIndex + private class NestedLongPredicateIndex implements DruidPredicateIndex { @Override public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory) @@ -782,7 +782,7 @@ private void findNext() } } - private class NestedDoubleLiteralValueSetIndex implements StringValueSetIndex + private class NestedDoubleValueSetIndex implements StringValueSetIndex { @Override public BitmapColumnIndex forValue(@Nullable String value) @@ -901,7 +901,7 @@ private void findNext() } } - private class NestedDoubleLiteralNumericRangeIndex implements NumericRangeIndex + private class NestedDoubleNumericRangeIndex implements NumericRangeIndex { @Override public BitmapColumnIndex forRange( @@ -923,7 +923,7 @@ public BitmapColumnIndex forRange( } } - private class NestedDoubleLiteralPredicateIndex implements DruidPredicateIndex + private class NestedDoublePredicateIndex implements DruidPredicateIndex { @Override public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory) @@ -1039,7 +1039,7 @@ IntList getIndexes(@Nullable String value) /** * {@link StringValueSetIndex} but for variant typed nested literal columns */ - private class NestedVariantLiteralValueSetIndex extends NestedVariantLiteralIndex implements StringValueSetIndex + private class NestedVariantValueSetIndex extends NestedVariantLiteralIndex implements StringValueSetIndex { @Override public BitmapColumnIndex forValue(@Nullable String value) @@ -1118,7 +1118,7 @@ private void findNext() /** * {@link DruidPredicateIndex} but for variant typed nested literal columns */ - private class NestedVariantLiteralPredicateIndex extends NestedVariantLiteralIndex implements DruidPredicateIndex + private class NestedVariantPredicateIndex extends NestedVariantLiteralIndex implements DruidPredicateIndex { @Override public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java index 3449d3410a56..6dd627d652fa 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java @@ -74,7 +74,7 @@ public class NestedFieldDictionaryEncodedColumn> implements DictionaryEncodedColumn { - private final NestedFieldTypeInfo.TypeSet types; + private final FieldTypeInfo.TypeSet types; @Nullable private final ColumnType singleType; private final ColumnarLongs longsColumn; @@ -95,7 +95,7 @@ public class NestedFieldDictionaryEncodedColumn doubleDictionary; + private final ColumnarDoubles valueColumn; + private final ImmutableBitmap nullValueIndex; + + public ScalarDoubleColumn( + FixedIndexed doubleDictionary, + ColumnarDoubles valueColumn, + ImmutableBitmap nullValueIndex + ) + { + this.doubleDictionary = doubleDictionary; + this.valueColumn = valueColumn; + this.nullValueIndex = nullValueIndex; + } + + @Override + public Indexed getDoubleDictionary() + { + return doubleDictionary; + } + + @Override + public ColumnType getLogicalType() + { + return ColumnType.DOUBLE; + } + + @Override + public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + return new DoubleColumnSelector() + { + private PeekableIntIterator nullIterator = nullValueIndex.peekableIterator(); + private int nullMark = -1; + private int offsetMark = -1; + + @Override + public double getDouble() + { + return valueColumn.get(offset.getOffset()); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("doubleColumn", valueColumn); + inspector.visit("nullBitmap", nullValueIndex); + } + + @Override + public boolean isNull() + { + final int i = offset.getOffset(); + if (i < offsetMark) { + // offset was reset, reset iterator state + nullMark = -1; + nullIterator = nullValueIndex.peekableIterator(); + } + offsetMark = i; + if (nullMark < i) { + nullIterator.advanceIfNeeded(offsetMark); + if (nullIterator.hasNext()) { + nullMark = nullIterator.next(); + } + } + return nullMark == offsetMark; + } + }; + } + + @Override + public VectorValueSelector makeVectorValueSelector(ReadableVectorOffset offset) + { + return new BaseDoubleVectorValueSelector(offset) + { + private final double[] valueVector = new double[offset.getMaxVectorSize()]; + @Nullable + private boolean[] nullVector = null; + private int id = ReadableVectorInspector.NULL_ID; + + @Nullable + private PeekableIntIterator nullIterator = nullValueIndex != null ? nullValueIndex.peekableIterator() : null; + private int offsetMark = -1; + + @Override + public double[] getDoubleVector() + { + computeVectorsIfNeeded(); + return valueVector; + } + + @Nullable + @Override + public boolean[] getNullVector() + { + computeVectorsIfNeeded(); + return nullVector; + } + + private void computeVectorsIfNeeded() + { + if (id == offset.getId()) { + return; + } + + if (offset.isContiguous()) { + if (offset.getStartOffset() < offsetMark) { + nullIterator = nullValueIndex.peekableIterator(); + } + offsetMark = offset.getStartOffset() + offset.getCurrentVectorSize(); + valueColumn.get(valueVector, offset.getStartOffset(), offset.getCurrentVectorSize()); + } else { + final int[] offsets = offset.getOffsets(); + if (offsets[offsets.length - 1] < offsetMark) { + nullIterator = nullValueIndex.peekableIterator(); + } + offsetMark = offsets[offsets.length - 1]; + valueColumn.get(valueVector, offsets, offset.getCurrentVectorSize()); + } + + nullVector = VectorSelectorUtils.populateNullVector(nullVector, offset, nullIterator); + + id = offset.getId(); + } + }; + } + + @Override + public void close() + { + valueColumn.close(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java new file mode 100644 index 000000000000..e7862bb645f0 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java @@ -0,0 +1,447 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Strings; +import com.google.common.base.Supplier; +import com.google.common.primitives.Doubles; +import it.unimi.dsi.fastutil.doubles.DoubleArraySet; +import it.unimi.dsi.fastutil.doubles.DoubleIterator; +import it.unimi.dsi.fastutil.doubles.DoubleSet; +import it.unimi.dsi.fastutil.ints.IntIntPair; +import it.unimi.dsi.fastutil.ints.IntIterator; +import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.query.BitmapResultFactory; +import org.apache.druid.query.filter.DruidDoublePredicate; +import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.segment.IntListUtils; +import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnIndexSupplier; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.column.DictionaryEncodedValueIndex; +import org.apache.druid.segment.column.DruidPredicateIndex; +import org.apache.druid.segment.column.NullValueIndex; +import org.apache.druid.segment.column.NumericRangeIndex; +import org.apache.druid.segment.column.SimpleBitmapColumnIndex; +import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; +import org.apache.druid.segment.column.SimpleImmutableBitmapIterableIndex; +import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.ColumnarDoubles; +import org.apache.druid.segment.data.CompressedColumnarDoublesSuppliers; +import org.apache.druid.segment.data.FixedIndexed; +import org.apache.druid.segment.data.GenericIndexed; +import org.apache.druid.segment.data.VByte; +import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.SortedSet; + +public class ScalarDoubleColumnAndIndexSupplier implements Supplier, ColumnIndexSupplier +{ + public static ScalarDoubleColumnAndIndexSupplier read( + ByteOrder byteOrder, + BitmapSerdeFactory bitmapSerdeFactory, + ByteBuffer bb, + ColumnBuilder columnBuilder + ) + { + final byte version = bb.get(); + final int columnNameLength = VByte.readInt(bb); + final String columnName = StringUtils.fromUtf8(bb, columnNameLength); + + if (version == NestedCommonFormatColumnSerializer.V0) { + try { + + final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); + + final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.DOUBLE_DICTIONARY_FILE_NAME + ); + final ByteBuffer doublesValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.DOUBLE_VALUE_COLUMN_FILE_NAME + ); + + final Supplier> doubleDictionarySupplier = FixedIndexed.read( + doubleDictionaryBuffer, + ColumnType.DOUBLE.getStrategy(), + byteOrder, + Double.BYTES + ); + + final Supplier doubles = CompressedColumnarDoublesSuppliers.fromByteBuffer( + doublesValueColumn, + byteOrder + ); + final ByteBuffer valueIndexBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.BITMAP_INDEX_FILE_NAME + ); + GenericIndexed rBitmaps = GenericIndexed.read( + valueIndexBuffer, + bitmapSerdeFactory.getObjectStrategy(), + columnBuilder.getFileMapper() + ); + return new ScalarDoubleColumnAndIndexSupplier( + doubleDictionarySupplier, + doubles, + rBitmaps, + bitmapSerdeFactory.getBitmapFactory() + ); + } + catch (IOException ex) { + throw new RE(ex, "Failed to deserialize V%s column.", version); + } + } else { + throw new RE("Unknown version " + version); + } + } + + private final Supplier> doubleDictionarySupplier; + + private final Supplier valueColumnSupplier; + + private final GenericIndexed valueIndexes; + + private final BitmapFactory bitmapFactory; + private final ImmutableBitmap nullValueBitmap; + + private ScalarDoubleColumnAndIndexSupplier( + Supplier> longDictionary, + Supplier valueColumnSupplier, + GenericIndexed valueIndexes, + BitmapFactory bitmapFactory + ) + { + this.doubleDictionarySupplier = longDictionary; + this.valueColumnSupplier = valueColumnSupplier; + this.valueIndexes = valueIndexes; + this.bitmapFactory = bitmapFactory; + this.nullValueBitmap = valueIndexes.get(0) == null ? bitmapFactory.makeEmptyImmutableBitmap() : valueIndexes.get(0); + } + + @Override + public NestedCommonFormatColumn get() + { + return new ScalarDoubleColumn( + doubleDictionarySupplier.get(), + valueColumnSupplier.get(), + nullValueBitmap + ); + } + + @Nullable + @Override + public T as(Class clazz) + { + if (clazz.equals(NullValueIndex.class)) { + final BitmapColumnIndex nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap); + return (T) (NullValueIndex) () -> nullIndex; + } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) + || clazz.equals(DictionaryEncodedValueIndex.class)) { + return (T) new DoubleDictionaryEncodedValueSetIndex(); + } else if (clazz.equals(StringValueSetIndex.class)) { + return (T) new DoubleValueSetIndex(); + } else if (clazz.equals(NumericRangeIndex.class)) { + return (T) new DoubleNumericRangeIndex(); + } else if (clazz.equals(DruidPredicateIndex.class)) { + return (T) new DoublePredicateIndex(); + } + + return null; + } + + private ImmutableBitmap getBitmap(int idx) + { + if (idx < 0) { + return bitmapFactory.makeEmptyImmutableBitmap(); + } + + final ImmutableBitmap bitmap = valueIndexes.get(idx); + return bitmap == null ? bitmapFactory.makeEmptyImmutableBitmap() : bitmap; + } + + private class DoubleValueSetIndex implements StringValueSetIndex + { + @Override + public BitmapColumnIndex forValue(@Nullable String value) + { + final boolean inputNull = value == null; + final Double doubleValue = Strings.isNullOrEmpty(value) ? null : Doubles.tryParse(value); + return new SimpleBitmapColumnIndex() + { + final FixedIndexed dictionary = doubleDictionarySupplier.get(); + + @Override + public double estimateSelectivity(int totalRows) + { + if (doubleValue == null) { + if (inputNull) { + return (double) getBitmap(0).size() / totalRows; + } else { + return 0.0; + } + } + final int id = dictionary.indexOf(doubleValue); + if (id < 0) { + return 0.0; + } + return (double) getBitmap(id).size() / totalRows; + } + + @Override + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + { + if (doubleValue == null) { + if (inputNull) { + return bitmapResultFactory.wrapDimensionValue(getBitmap(0)); + } else { + return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); + } + } + final int id = dictionary.indexOf(doubleValue); + if (id < 0) { + return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); + } + return bitmapResultFactory.wrapDimensionValue(getBitmap(id)); + } + }; + } + + @Override + public BitmapColumnIndex forSortedValues(SortedSet values) + { + return new SimpleImmutableBitmapIterableIndex() + { + @Override + public Iterable getBitmapIterable() + { + DoubleSet doubles = new DoubleArraySet(values.size()); + boolean needNullCheck = false; + for (String value : values) { + if (value == null) { + needNullCheck = true; + } else { + Double theValue = Doubles.tryParse(value); + if (theValue != null) { + doubles.add(theValue.doubleValue()); + } + } + } + final boolean doNullCheck = needNullCheck; + return () -> new Iterator() + { + final FixedIndexed dictionary = doubleDictionarySupplier.get(); + final DoubleIterator iterator = doubles.iterator(); + int next = -1; + boolean nullChecked = false; + + @Override + public boolean hasNext() + { + if (doNullCheck && !nullChecked) { + return true; + } + if (next < 0) { + findNext(); + } + return next >= 0; + } + + @Override + public ImmutableBitmap next() + { + if (doNullCheck && !nullChecked) { + nullChecked = true; + return getBitmap(0); + } + if (next < 0) { + findNext(); + if (next < 0) { + throw new NoSuchElementException(); + } + } + final int swap = next; + next = -1; + return getBitmap(swap); + } + + private void findNext() + { + while (next < 0 && iterator.hasNext()) { + double nextValue = iterator.nextDouble(); + next = dictionary.indexOf(nextValue); + } + } + }; + } + }; + } + } + + private class DoubleNumericRangeIndex implements NumericRangeIndex + { + @Override + public BitmapColumnIndex forRange( + @Nullable Number startValue, + boolean startStrict, + @Nullable Number endValue, + boolean endStrict + ) + { + final FixedIndexed dictionary = doubleDictionarySupplier.get(); + IntIntPair range = dictionary.getRange( + startValue == null ? null : startValue.doubleValue(), + startStrict, + endValue == null ? null : endValue.doubleValue(), + endStrict + ); + + final int startIndex = range.leftInt(); + final int endIndex = range.rightInt(); + return new SimpleImmutableBitmapIterableIndex() + { + @Override + public Iterable getBitmapIterable() + { + return () -> new Iterator() + { + final IntIterator rangeIterator = IntListUtils.fromTo(startIndex, endIndex).iterator(); + + @Override + public boolean hasNext() + { + return rangeIterator.hasNext(); + } + + @Override + public ImmutableBitmap next() + { + return getBitmap(rangeIterator.nextInt()); + } + }; + } + }; + } + } + + private class DoublePredicateIndex implements DruidPredicateIndex + { + @Override + public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory) + { + return new SimpleImmutableBitmapIterableIndex() + { + @Override + public Iterable getBitmapIterable() + { + return () -> new Iterator() + { + final Iterator iterator = doubleDictionarySupplier.get().iterator(); + final DruidDoublePredicate doublePredicate = matcherFactory.makeDoublePredicate(); + + int next; + int index = 0; + boolean nextSet = false; + + @Override + public boolean hasNext() + { + if (!nextSet) { + findNext(); + } + return nextSet; + } + + @Override + public ImmutableBitmap next() + { + if (!nextSet) { + findNext(); + if (!nextSet) { + throw new NoSuchElementException(); + } + } + nextSet = false; + return getBitmap(next); + } + + private void findNext() + { + while (!nextSet && iterator.hasNext()) { + Double nextValue = iterator.next(); + if (nextValue == null) { + nextSet = doublePredicate.applyNull(); + } else { + nextSet = doublePredicate.applyDouble(nextValue); + } + if (nextSet) { + next = index; + } + index++; + } + } + }; + } + }; + } + } + + private class DoubleDictionaryEncodedValueSetIndex implements DictionaryEncodedStringValueIndex + { + private final FixedIndexed dictionary = doubleDictionarySupplier.get(); + + @Override + public ImmutableBitmap getBitmap(int idx) + { + return ScalarDoubleColumnAndIndexSupplier.this.getBitmap(idx); + } + + @Override + public int getCardinality() + { + return dictionary.size(); + } + + @Nullable + @Override + public String getValue(int index) + { + final Double value = dictionary.get(index); + return value == null ? null : String.valueOf(value); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java new file mode 100644 index 000000000000..1cd7e16c58a9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Preconditions; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.collections.bitmap.MutableBitmap; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.ProgressIndicator; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.ColumnarDoublesSerializer; +import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSerializer; +import org.apache.druid.segment.data.CompressionFactory; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.FixedIndexedWriter; +import org.apache.druid.segment.data.GenericIndexedWriter; +import org.apache.druid.segment.data.SingleValueColumnarIntsSerializer; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +public class ScalarDoubleColumnSerializer extends NestedCommonFormatColumnSerializer +{ + private static final Logger log = new Logger(ScalarDoubleColumnSerializer.class); + + private final String name; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private final IndexSpec indexSpec; + @SuppressWarnings("unused") + private final Closer closer; + private DictionaryIdLookup dictionaryIdLookup; + private FixedIndexedWriter doubleDictionaryWriter; + private int rowCount = 0; + private boolean closedForWrite = false; + private boolean dictionarySerialized = false; + + private SingleValueColumnarIntsSerializer encodedValueSerializer; + private ColumnarDoublesSerializer doublesSerializer; + private GenericIndexedWriter bitmapIndexWriter; + private MutableBitmap[] bitmaps; + private ByteBuffer columnNameBytes = null; + + public ScalarDoubleColumnSerializer( + String name, + IndexSpec indexSpec, + SegmentWriteOutMedium segmentWriteOutMedium, + @SuppressWarnings("unused") ProgressIndicator progressIndicator, + Closer closer + ) + { + this.name = name; + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.indexSpec = indexSpec; + this.closer = closer; + this.dictionaryIdLookup = new DictionaryIdLookup(); + } + + @Override + public String getColumnName() + { + return name; + } + + @Override + public DictionaryIdLookup getGlobalLookup() + { + return dictionaryIdLookup; + } + + @Override + public boolean hasNulls() + { + return !bitmaps[0].isEmpty(); + } + + @Override + public void open() throws IOException + { + if (!dictionarySerialized) { + throw new IllegalStateException("Dictionary not serialized, cannot open value serializer"); + } + String filenameBase = StringUtils.format("%s.forward_dim", name); + final CompressionStrategy compression = indexSpec.getDimensionCompression(); + final CompressionStrategy compressionToUse; + if (compression != CompressionStrategy.UNCOMPRESSED && compression != CompressionStrategy.NONE) { + compressionToUse = compression; + } else { + compressionToUse = CompressionStrategy.LZ4; + } + encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create( + name, + segmentWriteOutMedium, + filenameBase, + doubleDictionaryWriter.getCardinality(), + compressionToUse + ); + encodedValueSerializer.open(); + + doublesSerializer = CompressionFactory.getDoubleSerializer( + name, + segmentWriteOutMedium, + StringUtils.format("%s.double_column", name), + ByteOrder.nativeOrder(), + indexSpec.getDimensionCompression() + ); + doublesSerializer.open(); + + bitmapIndexWriter = new GenericIndexedWriter<>( + segmentWriteOutMedium, + name, + indexSpec.getBitmapSerdeFactory().getObjectStrategy() + ); + bitmapIndexWriter.open(); + bitmapIndexWriter.setObjectsNotSorted(); + bitmaps = new MutableBitmap[doubleDictionaryWriter.getCardinality()]; + for (int i = 0; i < bitmaps.length; i++) { + bitmaps[i] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); + } + } + + @Override + public void openDictionaryWriter() throws IOException + { + doubleDictionaryWriter = new FixedIndexedWriter<>( + segmentWriteOutMedium, + ColumnType.DOUBLE.getStrategy(), + ByteOrder.nativeOrder(), + Long.BYTES, + true + ); + doubleDictionaryWriter.open(); + } + + @Override + public void serializeDictionaries( + Iterable strings, + Iterable longs, + Iterable doubles, + Iterable arrays + ) throws IOException + { + if (dictionarySerialized) { + throw new ISE("String dictionary already serialized for column [%s], cannot serialize again", name); + } + + // null is always 0 + doubleDictionaryWriter.write(null); + // put a dummy string in there + dictionaryIdLookup.addNumericNull(); + for (Double value : doubles) { + if (value == null) { + continue; + } + doubleDictionaryWriter.write(value); + dictionaryIdLookup.addDouble(value); + } + dictionarySerialized = true; + } + + @Override + public void serialize(ColumnValueSelector selector) throws IOException + { + if (!dictionarySerialized) { + throw new ISE("Must serialize value dictionaries before serializing values for column [%s]", name); + } + + final Object value = StructuredData.unwrap(selector.getObject()); + final ExprEval eval = ExprEval.bestEffortOf(value); + + final double val = eval.asDouble(); + final int dictId = eval.isNumericNull() ? 0 : dictionaryIdLookup.lookupDouble(val); + encodedValueSerializer.addValue(dictId); + doublesSerializer.add(dictId == 0 ? 0.0 : val); + bitmaps[dictId].add(rowCount); + rowCount++; + } + + + private void closeForWrite() throws IOException + { + if (!closedForWrite) { + for (int i = 0; i < bitmaps.length; i++) { + final MutableBitmap bitmap = bitmaps[i]; + bitmapIndexWriter.write( + indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(bitmap) + ); + bitmaps[i] = null; // Reclaim memory + } + columnNameBytes = computeFilenameBytes(); + closedForWrite = true; + } + } + + @Override + public long getSerializedSize() throws IOException + { + closeForWrite(); + + long size = 1 + columnNameBytes.capacity(); + // the value dictionaries, raw column, and null index are all stored in separate files + return size; + } + + @Override + public void writeTo( + WritableByteChannel channel, + FileSmoosher smoosher + ) throws IOException + { + Preconditions.checkState(closedForWrite, "Not closed yet!"); + + writeV0Header(channel, columnNameBytes); + writeInternal(smoosher, doubleDictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME); + writeInternal(smoosher, encodedValueSerializer, ENCODED_VALUE_COLUMN_FILE_NAME); + writeInternal(smoosher, doublesSerializer, DOUBLE_VALUE_COLUMN_FILE_NAME); + writeInternal(smoosher, bitmapIndexWriter, BITMAP_INDEX_FILE_NAME); + + log.info("Column [%s] serialized successfully.", name); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/DoubleFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleFieldColumnWriter.java similarity index 92% rename from processing/src/main/java/org/apache/druid/segment/nested/DoubleFieldColumnWriter.java rename to processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleFieldColumnWriter.java index 623eb1490ff7..eb7c52ff3ff1 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/DoubleFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleFieldColumnWriter.java @@ -33,20 +33,20 @@ import java.nio.channels.WritableByteChannel; /** - * Literal field writer for double type nested columns of {@link NestedDataColumnSerializer}. In addition to the normal + * Literal field writer for double type nested columns of {@link NestedDataColumnSerializerV4}. In addition to the normal * dictionary encoded column, this writer also writes an additional double value column with {@link #doublesSerializer}, * which is written to during {@link #addValue}. */ -public final class DoubleFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter +public final class ScalarDoubleFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter { private ColumnarDoublesSerializer doublesSerializer; - protected DoubleFieldColumnWriter( + public ScalarDoubleFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec, - GlobalDictionaryIdLookup globalDictionaryIdLookup + DictionaryIdLookup globalDictionaryIdLookup ) { super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumn.java new file mode 100644 index 000000000000..b8664a572ce3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumn.java @@ -0,0 +1,175 @@ +/* + * 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 org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.LongColumnSelector; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.ColumnarLongs; +import org.apache.druid.segment.data.FixedIndexed; +import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.data.ReadableOffset; +import org.apache.druid.segment.vector.BaseLongVectorValueSelector; +import org.apache.druid.segment.vector.ReadableVectorInspector; +import org.apache.druid.segment.vector.ReadableVectorOffset; +import org.apache.druid.segment.vector.VectorSelectorUtils; +import org.apache.druid.segment.vector.VectorValueSelector; +import org.roaringbitmap.PeekableIntIterator; + +import javax.annotation.Nullable; + +public class ScalarLongColumn implements NestedCommonFormatColumn +{ + private final FixedIndexed longDictionary; + private final ColumnarLongs valueColumn; + private final ImmutableBitmap nullValueIndex; + + public ScalarLongColumn( + FixedIndexed longDictionary, + ColumnarLongs valueColumn, + ImmutableBitmap nullValueIndex + ) + { + this.longDictionary = longDictionary; + this.valueColumn = valueColumn; + this.nullValueIndex = nullValueIndex; + } + + + @Override + public Indexed getLongDictionary() + { + return longDictionary; + } + + @Override + public ColumnType getLogicalType() + { + return ColumnType.LONG; + } + + @Override + public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + return new LongColumnSelector() + { + private PeekableIntIterator nullIterator = nullValueIndex.peekableIterator(); + private int nullMark = -1; + private int offsetMark = -1; + + @Override + public long getLong() + { + return valueColumn.get(offset.getOffset()); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("longColumn", valueColumn); + inspector.visit("nullBitmap", nullValueIndex); + } + + @Override + public boolean isNull() + { + final int i = offset.getOffset(); + if (i < offsetMark) { + // offset was reset, reset iterator state + nullMark = -1; + nullIterator = nullValueIndex.peekableIterator(); + } + offsetMark = i; + if (nullMark < i) { + nullIterator.advanceIfNeeded(offsetMark); + if (nullIterator.hasNext()) { + nullMark = nullIterator.next(); + } + } + return nullMark == offsetMark; + } + }; + } + + @Override + public VectorValueSelector makeVectorValueSelector(ReadableVectorOffset offset) + { + return new BaseLongVectorValueSelector(offset) + { + private final long[] valueVector = new long[offset.getMaxVectorSize()]; + @Nullable + private boolean[] nullVector = null; + private int id = ReadableVectorInspector.NULL_ID; + + @Nullable + private PeekableIntIterator nullIterator = nullValueIndex.peekableIterator(); + private int offsetMark = -1; + + @Override + public long[] getLongVector() + { + computeVectorsIfNeeded(); + return valueVector; + } + + @Nullable + @Override + public boolean[] getNullVector() + { + computeVectorsIfNeeded(); + return nullVector; + } + + private void computeVectorsIfNeeded() + { + if (id == offset.getId()) { + return; + } + + if (offset.isContiguous()) { + if (offset.getStartOffset() < offsetMark) { + nullIterator = nullValueIndex.peekableIterator(); + } + offsetMark = offset.getStartOffset() + offset.getCurrentVectorSize(); + valueColumn.get(valueVector, offset.getStartOffset(), offset.getCurrentVectorSize()); + } else { + final int[] offsets = offset.getOffsets(); + if (offsets[offsets.length - 1] < offsetMark) { + nullIterator = nullValueIndex.peekableIterator(); + } + offsetMark = offsets[offsets.length - 1]; + valueColumn.get(valueVector, offsets, offset.getCurrentVectorSize()); + } + + nullVector = VectorSelectorUtils.populateNullVector(nullVector, offset, nullIterator); + + id = offset.getId(); + } + }; + } + + @Override + public void close() + { + valueColumn.close(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java new file mode 100644 index 000000000000..583ef677cbff --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java @@ -0,0 +1,448 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Supplier; +import it.unimi.dsi.fastutil.ints.IntIntPair; +import it.unimi.dsi.fastutil.ints.IntIterator; +import it.unimi.dsi.fastutil.longs.LongArraySet; +import it.unimi.dsi.fastutil.longs.LongIterator; +import it.unimi.dsi.fastutil.longs.LongSet; +import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.common.guava.GuavaUtils; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.query.BitmapResultFactory; +import org.apache.druid.query.filter.DruidLongPredicate; +import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.segment.IntListUtils; +import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnIndexSupplier; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.column.DictionaryEncodedValueIndex; +import org.apache.druid.segment.column.DruidPredicateIndex; +import org.apache.druid.segment.column.NullValueIndex; +import org.apache.druid.segment.column.NumericRangeIndex; +import org.apache.druid.segment.column.SimpleBitmapColumnIndex; +import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; +import org.apache.druid.segment.column.SimpleImmutableBitmapIterableIndex; +import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.ColumnarLongs; +import org.apache.druid.segment.data.CompressedColumnarLongsSupplier; +import org.apache.druid.segment.data.FixedIndexed; +import org.apache.druid.segment.data.GenericIndexed; +import org.apache.druid.segment.data.VByte; +import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.SortedSet; + +public class ScalarLongColumnAndIndexSupplier implements Supplier, ColumnIndexSupplier +{ + public static ScalarLongColumnAndIndexSupplier read( + ByteOrder byteOrder, + BitmapSerdeFactory bitmapSerdeFactory, + ByteBuffer bb, + ColumnBuilder columnBuilder + ) + { + final byte version = bb.get(); + final int columnNameLength = VByte.readInt(bb); + final String columnName = StringUtils.fromUtf8(bb, columnNameLength); + + if (version == NestedCommonFormatColumnSerializer.V0) { + try { + + final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); + + final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.LONG_DICTIONARY_FILE_NAME + ); + final ByteBuffer longsValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.LONG_VALUE_COLUMN_FILE_NAME + ); + final ByteBuffer valueIndexBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.BITMAP_INDEX_FILE_NAME + ); + GenericIndexed rBitmaps = GenericIndexed.read( + valueIndexBuffer, + bitmapSerdeFactory.getObjectStrategy(), + columnBuilder.getFileMapper() + ); + + final Supplier> longDictionarySupplier = FixedIndexed.read( + longDictionaryBuffer, + ColumnType.LONG.getStrategy(), + byteOrder, + Long.BYTES + ); + + final Supplier longs = CompressedColumnarLongsSupplier.fromByteBuffer( + longsValueColumn, + byteOrder + ); + return new ScalarLongColumnAndIndexSupplier( + longDictionarySupplier, + longs, + rBitmaps, + bitmapSerdeFactory.getBitmapFactory() + ); + } + catch (IOException ex) { + throw new RE(ex, "Failed to deserialize V%s column.", version); + } + } else { + throw new RE("Unknown version " + version); + } + } + + private final Supplier> longDictionarySupplier; + + private final Supplier valueColumnSupplier; + + private final GenericIndexed valueIndexes; + + private final BitmapFactory bitmapFactory; + + private final ImmutableBitmap nullValueBitmap; + + private ScalarLongColumnAndIndexSupplier( + Supplier> longDictionarySupplier, + Supplier valueColumnSupplier, + GenericIndexed valueIndexes, + BitmapFactory bitmapFactory + ) + { + this.longDictionarySupplier = longDictionarySupplier; + this.valueColumnSupplier = valueColumnSupplier; + this.valueIndexes = valueIndexes; + this.bitmapFactory = bitmapFactory; + this.nullValueBitmap = valueIndexes.get(0) == null ? bitmapFactory.makeEmptyImmutableBitmap() : valueIndexes.get(0); + } + + @Override + public NestedCommonFormatColumn get() + { + return new ScalarLongColumn( + longDictionarySupplier.get(), + valueColumnSupplier.get(), + nullValueBitmap + ); + } + + @Nullable + @Override + public T as(Class clazz) + { + if (clazz.equals(NullValueIndex.class)) { + final BitmapColumnIndex nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap); + return (T) (NullValueIndex) () -> nullIndex; + } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) + || clazz.equals(DictionaryEncodedValueIndex.class)) { + return (T) new LongDictionaryEncodedValueSetIndex(); + } else if (clazz.equals(StringValueSetIndex.class)) { + return (T) new LongValueSetIndex(); + } else if (clazz.equals(NumericRangeIndex.class)) { + return (T) new LongNumericRangeIndex(); + } else if (clazz.equals(DruidPredicateIndex.class)) { + return (T) new LongPredicateIndex(); + } + + return null; + } + + private ImmutableBitmap getBitmap(int idx) + { + if (idx < 0) { + return bitmapFactory.makeEmptyImmutableBitmap(); + } + + final ImmutableBitmap bitmap = valueIndexes.get(idx); + return bitmap == null ? bitmapFactory.makeEmptyImmutableBitmap() : bitmap; + } + + private class LongValueSetIndex implements StringValueSetIndex + { + @Override + public BitmapColumnIndex forValue(@Nullable String value) + { + final boolean inputNull = value == null; + final Long longValue = GuavaUtils.tryParseLong(value); + return new SimpleBitmapColumnIndex() + { + final FixedIndexed dictionary = longDictionarySupplier.get(); + + @Override + public double estimateSelectivity(int totalRows) + { + if (longValue == null) { + if (inputNull) { + return (double) getBitmap(0).size() / totalRows; + } else { + return 0.0; + } + } + final int id = dictionary.indexOf(longValue); + if (id < 0) { + return 0.0; + } + return (double) getBitmap(id).size() / totalRows; + } + + @Override + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + { + if (longValue == null) { + if (inputNull) { + return bitmapResultFactory.wrapDimensionValue(getBitmap(0)); + } else { + return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); + } + } + final int id = dictionary.indexOf(longValue); + if (id < 0) { + return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); + } + return bitmapResultFactory.wrapDimensionValue(getBitmap(id)); + } + }; + } + + @Override + public BitmapColumnIndex forSortedValues(SortedSet values) + { + return new SimpleImmutableBitmapIterableIndex() + { + @Override + public Iterable getBitmapIterable() + { + LongSet longs = new LongArraySet(values.size()); + boolean needNullCheck = false; + for (String value : values) { + if (value == null) { + needNullCheck = true; + } else { + Long theValue = GuavaUtils.tryParseLong(value); + if (theValue != null) { + longs.add(theValue.longValue()); + } + } + } + final boolean doNullCheck = needNullCheck; + return () -> new Iterator() + { + final FixedIndexed dictionary = longDictionarySupplier.get(); + final LongIterator iterator = longs.iterator(); + int next = -1; + boolean nullChecked = false; + + @Override + public boolean hasNext() + { + if (doNullCheck && !nullChecked) { + return true; + } + if (next < 0) { + findNext(); + } + return next >= 0; + } + + @Override + public ImmutableBitmap next() + { + if (doNullCheck && !nullChecked) { + nullChecked = true; + return getBitmap(0); + } + if (next < 0) { + findNext(); + if (next < 0) { + throw new NoSuchElementException(); + } + } + final int swap = next; + next = -1; + return getBitmap(swap); + } + + private void findNext() + { + while (next < 0 && iterator.hasNext()) { + long nextValue = iterator.nextLong(); + next = dictionary.indexOf(nextValue); + } + } + }; + } + }; + } + } + + private class LongNumericRangeIndex implements NumericRangeIndex + { + @Override + public BitmapColumnIndex forRange( + @Nullable Number startValue, + boolean startStrict, + @Nullable Number endValue, + boolean endStrict + ) + { + final FixedIndexed dictionary = longDictionarySupplier.get(); + IntIntPair range = dictionary.getRange( + startValue == null ? null : startValue.longValue(), + startStrict, + endValue == null ? null : endValue.longValue(), + endStrict + ); + + final int startIndex = range.leftInt(); + final int endIndex = range.rightInt(); + return new SimpleImmutableBitmapIterableIndex() + { + @Override + public Iterable getBitmapIterable() + { + return () -> new Iterator() + { + final IntIterator rangeIterator = IntListUtils.fromTo(startIndex, endIndex).iterator(); + + @Override + public boolean hasNext() + { + return rangeIterator.hasNext(); + } + + @Override + public ImmutableBitmap next() + { + return getBitmap(rangeIterator.nextInt()); + } + }; + } + }; + } + } + + private class LongPredicateIndex implements DruidPredicateIndex + { + @Override + public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory) + { + return new SimpleImmutableBitmapIterableIndex() + { + @Override + public Iterable getBitmapIterable() + { + return () -> new Iterator() + { + final Iterator iterator = longDictionarySupplier.get().iterator(); + final DruidLongPredicate longPredicate = matcherFactory.makeLongPredicate(); + + int next; + int index = 0; + boolean nextSet = false; + + @Override + public boolean hasNext() + { + if (!nextSet) { + findNext(); + } + return nextSet; + } + + @Override + public ImmutableBitmap next() + { + if (!nextSet) { + findNext(); + if (!nextSet) { + throw new NoSuchElementException(); + } + } + nextSet = false; + + return getBitmap(next); + } + + private void findNext() + { + while (!nextSet && iterator.hasNext()) { + Long nextValue = iterator.next(); + if (nextValue == null) { + nextSet = longPredicate.applyNull(); + } else { + nextSet = longPredicate.applyLong(nextValue); + } + if (nextSet) { + next = index; + } + index++; + } + } + }; + } + }; + } + } + + private class LongDictionaryEncodedValueSetIndex implements DictionaryEncodedStringValueIndex + { + private final FixedIndexed dictionary = longDictionarySupplier.get(); + + @Override + public ImmutableBitmap getBitmap(int idx) + { + return ScalarLongColumnAndIndexSupplier.this.getBitmap(idx); + } + + @Override + public int getCardinality() + { + return dictionary.size(); + } + + @Nullable + @Override + public String getValue(int index) + { + final Long value = dictionary.get(index); + return value == null ? null : String.valueOf(value); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java new file mode 100644 index 000000000000..6148d183b83a --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Preconditions; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.collections.bitmap.MutableBitmap; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.ProgressIndicator; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.ColumnarLongsSerializer; +import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSerializer; +import org.apache.druid.segment.data.CompressionFactory; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.FixedIndexedWriter; +import org.apache.druid.segment.data.GenericIndexedWriter; +import org.apache.druid.segment.data.SingleValueColumnarIntsSerializer; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +public class ScalarLongColumnSerializer extends NestedCommonFormatColumnSerializer +{ + private static final Logger log = new Logger(ScalarLongColumnSerializer.class); + + private final String name; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private final IndexSpec indexSpec; + @SuppressWarnings("unused") + private final Closer closer; + private DictionaryIdLookup dictionaryIdLookup; + private FixedIndexedWriter longDictionaryWriter; + private int rowCount = 0; + private boolean closedForWrite = false; + private boolean dictionarySerialized = false; + + private SingleValueColumnarIntsSerializer encodedValueSerializer; + private ColumnarLongsSerializer longsSerializer; + private GenericIndexedWriter bitmapIndexWriter; + private MutableBitmap[] bitmaps; + private ByteBuffer columnNameBytes = null; + + public ScalarLongColumnSerializer( + String name, + IndexSpec indexSpec, + SegmentWriteOutMedium segmentWriteOutMedium, + @SuppressWarnings("unused") ProgressIndicator progressIndicator, + Closer closer + ) + { + this.name = name; + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.indexSpec = indexSpec; + this.closer = closer; + this.dictionaryIdLookup = new DictionaryIdLookup(); + } + + @Override + public String getColumnName() + { + return name; + } + + @Override + public DictionaryIdLookup getGlobalLookup() + { + return dictionaryIdLookup; + } + + @Override + public boolean hasNulls() + { + return !bitmaps[0].isEmpty(); + } + + @Override + public void open() throws IOException + { + if (!dictionarySerialized) { + throw new IllegalStateException("Dictionary not serialized, cannot open value serializer"); + } + String filenameBase = StringUtils.format("%s.forward_dim", name); + final CompressionStrategy compression = indexSpec.getDimensionCompression(); + final CompressionStrategy compressionToUse; + if (compression != CompressionStrategy.UNCOMPRESSED && compression != CompressionStrategy.NONE) { + compressionToUse = compression; + } else { + compressionToUse = CompressionStrategy.LZ4; + } + encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create( + name, + segmentWriteOutMedium, + filenameBase, + longDictionaryWriter.getCardinality(), + compressionToUse + ); + encodedValueSerializer.open(); + + longsSerializer = CompressionFactory.getLongSerializer( + name, + segmentWriteOutMedium, + StringUtils.format("%s.long_column", name), + ByteOrder.nativeOrder(), + indexSpec.getLongEncoding(), + indexSpec.getDimensionCompression() + ); + longsSerializer.open(); + + bitmapIndexWriter = new GenericIndexedWriter<>( + segmentWriteOutMedium, + name, + indexSpec.getBitmapSerdeFactory().getObjectStrategy() + ); + bitmapIndexWriter.open(); + bitmapIndexWriter.setObjectsNotSorted(); + bitmaps = new MutableBitmap[longDictionaryWriter.getCardinality()]; + for (int i = 0; i < bitmaps.length; i++) { + bitmaps[i] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); + } + } + + @Override + public void openDictionaryWriter() throws IOException + { + longDictionaryWriter = new FixedIndexedWriter<>( + segmentWriteOutMedium, + ColumnType.LONG.getStrategy(), + ByteOrder.nativeOrder(), + Long.BYTES, + true + ); + longDictionaryWriter.open(); + } + + + @Override + public void serializeDictionaries( + Iterable strings, + Iterable longs, + Iterable doubles, + Iterable arrays + ) throws IOException + { + if (dictionarySerialized) { + throw new ISE("String dictionary already serialized for column [%s], cannot serialize again", name); + } + + // null is always 0 + longDictionaryWriter.write(null); + dictionaryIdLookup.addNumericNull(); + for (Long value : longs) { + if (value == null) { + continue; + } + longDictionaryWriter.write(value); + dictionaryIdLookup.addLong(value); + } + dictionarySerialized = true; + } + + @Override + public void serialize(ColumnValueSelector selector) throws IOException + { + if (!dictionarySerialized) { + throw new ISE("Must serialize value dictionaries before serializing values for column [%s]", name); + } + + final Object value = StructuredData.unwrap(selector.getObject()); + final ExprEval eval = ExprEval.bestEffortOf(value); + + final long val = eval.asLong(); + final int dictId = eval.isNumericNull() ? 0 : dictionaryIdLookup.lookupLong(val); + encodedValueSerializer.addValue(dictId); + longsSerializer.add(dictId == 0 ? 0L : val); + bitmaps[dictId].add(rowCount); + rowCount++; + } + + + private void closeForWrite() throws IOException + { + if (!closedForWrite) { + for (int i = 0; i < bitmaps.length; i++) { + final MutableBitmap bitmap = bitmaps[i]; + bitmapIndexWriter.write( + indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(bitmap) + ); + bitmaps[i] = null; // Reclaim memory + } + columnNameBytes = computeFilenameBytes(); + closedForWrite = true; + } + } + + @Override + public long getSerializedSize() throws IOException + { + closeForWrite(); + + long size = 1 + columnNameBytes.capacity(); + // the value dictionaries, raw column, and null index are all stored in separate files + return size; + } + + @Override + public void writeTo( + WritableByteChannel channel, + FileSmoosher smoosher + ) throws IOException + { + Preconditions.checkState(closedForWrite, "Not closed yet!"); + + writeV0Header(channel, columnNameBytes); + writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME); + writeInternal(smoosher, encodedValueSerializer, ENCODED_VALUE_COLUMN_FILE_NAME); + writeInternal(smoosher, longsSerializer, LONG_VALUE_COLUMN_FILE_NAME); + writeInternal(smoosher, bitmapIndexWriter, BITMAP_INDEX_FILE_NAME); + + log.info("Column [%s] serialized successfully.", name); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/LongFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongFieldColumnWriter.java similarity index 92% rename from processing/src/main/java/org/apache/druid/segment/nested/LongFieldColumnWriter.java rename to processing/src/main/java/org/apache/druid/segment/nested/ScalarLongFieldColumnWriter.java index 1321a00023b3..88d7d9a5cfa6 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/LongFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongFieldColumnWriter.java @@ -33,20 +33,20 @@ import java.nio.channels.WritableByteChannel; /** - * Literal field writer for long type nested columns of {@link NestedDataColumnSerializer}. In addition to the normal + * Literal field writer for long type nested columns of {@link NestedDataColumnSerializerV4}. In addition to the normal * dictionary encoded column, this writer also writes an additional long value column with {@link #longsSerializer}, * which is written to during {@link #addValue}. */ -public final class LongFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter +public final class ScalarLongFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter { private ColumnarLongsSerializer longsSerializer; - protected LongFieldColumnWriter( + public ScalarLongFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec, - GlobalDictionaryIdLookup globalDictionaryIdLookup + DictionaryIdLookup globalDictionaryIdLookup ) { super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java new file mode 100644 index 000000000000..a1842adf3447 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Supplier; +import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +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.smoosh.SmooshedFileMapper; +import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnIndexSupplier; +import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.column.DictionaryEncodedValueIndex; +import org.apache.druid.segment.column.DruidPredicateIndex; +import org.apache.druid.segment.column.IndexedStringDictionaryEncodedStringValueIndex; +import org.apache.druid.segment.column.IndexedStringDruidPredicateIndex; +import org.apache.druid.segment.column.IndexedUtf8LexicographicalRangeIndex; +import org.apache.druid.segment.column.IndexedUtf8ValueSetIndex; +import org.apache.druid.segment.column.LexicographicalRangeIndex; +import org.apache.druid.segment.column.NullValueIndex; +import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; +import org.apache.druid.segment.column.StringEncodingStrategies; +import org.apache.druid.segment.column.StringEncodingStrategy; +import org.apache.druid.segment.column.StringFrontCodedDictionaryEncodedColumn; +import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.ColumnarInts; +import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier; +import org.apache.druid.segment.data.EncodedStringDictionaryWriter; +import org.apache.druid.segment.data.FrontCodedIndexed; +import org.apache.druid.segment.data.GenericIndexed; +import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.data.VByte; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class ScalarStringColumnAndIndexSupplier implements Supplier, ColumnIndexSupplier +{ + public static ScalarStringColumnAndIndexSupplier read( + ByteOrder byteOrder, + BitmapSerdeFactory bitmapSerdeFactory, + ByteBuffer bb, + ColumnBuilder columnBuilder + ) + { + final byte version = bb.get(); + final int columnNameLength = VByte.readInt(bb); + final String columnName = StringUtils.fromUtf8(bb, columnNameLength); + + + if (version == NestedCommonFormatColumnSerializer.V0) { + try { + final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); + final GenericIndexed stringDictionary; + final Supplier frontCodedStringDictionarySupplier; + + final ByteBuffer stringDictionaryBuffer = loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME + ); + + final int dictionaryStartPosition = stringDictionaryBuffer.position(); + final byte dictionaryVersion = stringDictionaryBuffer.get(); + + if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) { + final byte encodingId = stringDictionaryBuffer.get(); + if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) { + frontCodedStringDictionarySupplier = FrontCodedIndexed.read( + stringDictionaryBuffer, + byteOrder + ); + stringDictionary = null; + } else if (encodingId == StringEncodingStrategy.UTF8_ID) { + // this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but + // this provides backwards compatibility should we switch at some point in the future to always + // writing dictionaryVersion + stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper); + frontCodedStringDictionarySupplier = null; + } else { + throw new ISE("impossible, unknown encoding strategy id: %s", encodingId); + } + } else { + // legacy format that only supports plain utf8 enoding stored in GenericIndexed and the byte we are reading + // as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the + // GenericIndexed version can be correctly read + stringDictionaryBuffer.position(dictionaryStartPosition); + stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper); + frontCodedStringDictionarySupplier = null; + } + final ByteBuffer encodedValueColumn = loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.ENCODED_VALUE_COLUMN_FILE_NAME + ); + final CompressedVSizeColumnarIntsSupplier ints = CompressedVSizeColumnarIntsSupplier.fromByteBuffer( + encodedValueColumn, + byteOrder + ); + final ByteBuffer valueIndexBuffer = loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.BITMAP_INDEX_FILE_NAME + ); + GenericIndexed valueIndexes = GenericIndexed.read( + valueIndexBuffer, + bitmapSerdeFactory.getObjectStrategy(), + columnBuilder.getFileMapper() + ); + return new ScalarStringColumnAndIndexSupplier( + stringDictionary, + frontCodedStringDictionarySupplier, + ints, + valueIndexes, + bitmapSerdeFactory + ); + } + catch (IOException ex) { + throw new RE(ex, "Failed to deserialize V%s column.", version); + } + } else { + throw new RE("Unknown version " + version); + } + } + + + private final GenericIndexed stringDictionary; + private final Supplier frontCodedStringDictionarySupplier; + private final Supplier encodedColumnSupplier; + private final GenericIndexed valueIndexes; + private final ImmutableBitmap nullValueBitmap; + private final BitmapFactory bitmapFactory; + + private ScalarStringColumnAndIndexSupplier( + GenericIndexed stringDictionary, + Supplier frontCodedStringDictionarySupplier, + Supplier encodedColumnSupplier, + GenericIndexed valueIndexes, + BitmapSerdeFactory serdeFactory + ) + { + this.stringDictionary = stringDictionary; + this.frontCodedStringDictionarySupplier = frontCodedStringDictionarySupplier; + this.encodedColumnSupplier = encodedColumnSupplier; + this.valueIndexes = valueIndexes; + this.bitmapFactory = serdeFactory.getBitmapFactory(); + this.nullValueBitmap = valueIndexes.get(0) == null ? bitmapFactory.makeEmptyImmutableBitmap() : valueIndexes.get(0); + } + + @Override + public NestedCommonFormatColumn get() + { + if (frontCodedStringDictionarySupplier != null) { + return new StringFrontCodedDictionaryEncodedColumn( + encodedColumnSupplier.get(), + null, + frontCodedStringDictionarySupplier.get() + ); + } + return new ScalarStringDictionaryEncodedColumn<>(encodedColumnSupplier.get(), stringDictionary.singleThreaded()); + } + + private static ByteBuffer loadInternalFile( + SmooshedFileMapper fileMapper, + String filenameBase, + String internalFileName + ) throws IOException + { + return fileMapper.mapFile( + NestedCommonFormatColumnSerializer.getInternalFileName(filenameBase, internalFileName) + ); + } + + @Nullable + @Override + public T as(Class clazz) + { + if (valueIndexes != null) { + final Indexed singleThreadedBitmaps = valueIndexes.singleThreaded(); + final Indexed utf8Dictionary = frontCodedStringDictionarySupplier == null + ? stringDictionary.singleThreaded() + : frontCodedStringDictionarySupplier.get(); + if (clazz.equals(NullValueIndex.class)) { + final BitmapColumnIndex nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap); + return (T) (NullValueIndex) () -> nullIndex; + } else if (clazz.equals(StringValueSetIndex.class)) { + return (T) new IndexedUtf8ValueSetIndex<>( + bitmapFactory, + utf8Dictionary, + singleThreadedBitmaps + ); + } else if (clazz.equals(DruidPredicateIndex.class)) { + return (T) new IndexedStringDruidPredicateIndex<>( + bitmapFactory, + new StringEncodingStrategies.Utf8ToStringIndexed(utf8Dictionary), + singleThreadedBitmaps + ); + } else if (clazz.equals(LexicographicalRangeIndex.class)) { + return (T) new IndexedUtf8LexicographicalRangeIndex<>( + bitmapFactory, + utf8Dictionary, + singleThreadedBitmaps, + utf8Dictionary.get(0) == null + ); + } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) + || clazz.equals(DictionaryEncodedValueIndex.class)) { + return (T) new IndexedStringDictionaryEncodedStringValueIndex<>( + bitmapFactory, + new StringEncodingStrategies.Utf8ToStringIndexed(utf8Dictionary), + valueIndexes + ); + } + } + return null; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java new file mode 100644 index 000000000000..788880934cb7 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Preconditions; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.collections.bitmap.MutableBitmap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.ProgressIndicator; +import org.apache.druid.segment.column.StringEncodingStrategies; +import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSerializer; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.DictionaryWriter; +import org.apache.druid.segment.data.GenericIndexedWriter; +import org.apache.druid.segment.data.SingleValueColumnarIntsSerializer; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +public class ScalarStringColumnSerializer extends NestedCommonFormatColumnSerializer +{ + private static final Logger log = new Logger(ScalarStringColumnSerializer.class); + + private final String name; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private final IndexSpec indexSpec; + @SuppressWarnings("unused") + private final Closer closer; + private DictionaryIdLookup dictionaryIdLookup; + private DictionaryWriter dictionaryWriter; + private int rowCount = 0; + private boolean closedForWrite = false; + private boolean dictionarySerialized = false; + + private SingleValueColumnarIntsSerializer encodedValueSerializer; + private GenericIndexedWriter bitmapIndexWriter; + private MutableBitmap[] bitmaps; + private ByteBuffer columnNameBytes = null; + + public ScalarStringColumnSerializer( + String name, + IndexSpec indexSpec, + SegmentWriteOutMedium segmentWriteOutMedium, + @SuppressWarnings("unused") ProgressIndicator progressIndicator, + Closer closer + ) + { + this.name = name; + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.indexSpec = indexSpec; + this.closer = closer; + this.dictionaryIdLookup = new DictionaryIdLookup(); + } + + @Override + public String getColumnName() + { + return name; + } + + @Override + public DictionaryIdLookup getGlobalLookup() + { + return dictionaryIdLookup; + } + + @Override + public boolean hasNulls() + { + return !bitmaps[0].isEmpty(); + } + + @Override + public void openDictionaryWriter() throws IOException + { + dictionaryWriter = StringEncodingStrategies.getStringDictionaryWriter( + indexSpec.getStringDictionaryEncoding(), + segmentWriteOutMedium, + name + ); + dictionaryWriter.open(); + } + + @Override + public void open() throws IOException + { + if (!dictionarySerialized) { + throw new IllegalStateException("Dictionary not serialized, cannot open value serializer"); + } + String filenameBase = StringUtils.format("%s.forward_dim", name); + final CompressionStrategy compression = indexSpec.getDimensionCompression(); + final CompressionStrategy compressionToUse; + if (compression != CompressionStrategy.UNCOMPRESSED && compression != CompressionStrategy.NONE) { + compressionToUse = compression; + } else { + // always compress + compressionToUse = CompressionStrategy.LZ4; + } + encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create( + name, + segmentWriteOutMedium, + filenameBase, + dictionaryWriter.getCardinality(), + compressionToUse + ); + encodedValueSerializer.open(); + + bitmapIndexWriter = new GenericIndexedWriter<>( + segmentWriteOutMedium, + name, + indexSpec.getBitmapSerdeFactory().getObjectStrategy() + ); + bitmapIndexWriter.open(); + bitmapIndexWriter.setObjectsNotSorted(); + bitmaps = new MutableBitmap[dictionaryWriter.getCardinality()]; + for (int i = 0; i < bitmaps.length; i++) { + bitmaps[i] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); + } + } + + @Override + public void serializeDictionaries( + Iterable strings, + Iterable longs, + Iterable doubles, + Iterable 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); + dictionaryIdLookup.addString(null); + for (String value : strings) { + value = NullHandling.emptyToNullIfNeeded(value); + if (value == null) { + continue; + } + + dictionaryWriter.write(value); + dictionaryIdLookup.addString(value); + } + dictionarySerialized = true; + } + + @Override + public void serialize(ColumnValueSelector selector) throws IOException + { + if (!dictionarySerialized) { + throw new ISE("Must serialize value dictionaries before serializing values for column [%s]", name); + } + + final Object value = StructuredData.unwrap(selector.getObject()); + final ExprEval eval = ExprEval.bestEffortOf(value); + final String s = eval.castTo(ExpressionType.STRING).asString(); + final int dictId = dictionaryIdLookup.lookupString(s); + encodedValueSerializer.addValue(dictId); + bitmaps[dictId].add(rowCount); + rowCount++; + } + + private void closeForWrite() throws IOException + { + if (!closedForWrite) { + for (int i = 0; i < bitmaps.length; i++) { + final MutableBitmap bitmap = bitmaps[i]; + bitmapIndexWriter.write( + indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(bitmap) + ); + bitmaps[i] = null; // Reclaim memory + } + columnNameBytes = computeFilenameBytes(); + closedForWrite = true; + } + } + + @Override + public long getSerializedSize() throws IOException + { + closeForWrite(); + + // standard string version + long size = 1 + columnNameBytes.capacity(); + // the value dictionaries, raw column, and null index are all stored in separate files + return size; + } + + @Override + public void writeTo( + WritableByteChannel channel, + FileSmoosher smoosher + ) throws IOException + { + Preconditions.checkState(closedForWrite, "Not closed yet!"); + Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + + writeV0Header(channel, columnNameBytes); + writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME); + writeInternal(smoosher, encodedValueSerializer, ENCODED_VALUE_COLUMN_FILE_NAME); + writeInternal(smoosher, bitmapIndexWriter, BITMAP_INDEX_FILE_NAME); + + log.info("Column [%s] serialized successfully.", name); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringDictionaryEncodedColumn.java new file mode 100644 index 000000000000..e9fd15123686 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringDictionaryEncodedColumn.java @@ -0,0 +1,376 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.AbstractDimensionSelector; +import org.apache.druid.segment.IdLookup; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.DictionaryEncodedColumn; +import org.apache.druid.segment.column.StringDictionaryEncodedColumn; +import org.apache.druid.segment.column.StringEncodingStrategies; +import org.apache.druid.segment.data.ColumnarInts; +import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.data.ReadableOffset; +import org.apache.druid.segment.data.SingleIndexedInt; +import org.apache.druid.segment.filter.BooleanValueMatcher; +import org.apache.druid.segment.historical.HistoricalDimensionSelector; +import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector; +import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; +import org.apache.druid.segment.vector.ReadableVectorOffset; +import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; +import org.apache.druid.segment.vector.VectorObjectSelector; +import org.apache.druid.utils.CloseableUtils; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.BitSet; + +public class ScalarStringDictionaryEncodedColumn> + implements DictionaryEncodedColumn, NestedCommonFormatColumn +{ + private final ColumnarInts column; + private final TIndexed utf8Dictionary; + + public ScalarStringDictionaryEncodedColumn( + ColumnarInts singleValueColumn, + TIndexed utf8Dictionary + ) + { + this.column = singleValueColumn; + this.utf8Dictionary = utf8Dictionary; + } + + @Override + public int length() + { + return column.size(); + } + + @Override + public boolean hasMultipleValues() + { + return false; + } + + @Override + public int getSingleValueRow(int rowNum) + { + return column.get(rowNum); + } + + @Override + public IndexedInts getMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException("Column is not multi-valued"); + } + + @Override + @Nullable + public String lookupName(int id) + { + final ByteBuffer buffer = utf8Dictionary.get(id); + if (buffer == null) { + return null; + } + return StringUtils.fromUtf8(buffer); + } + + @Override + public int lookupId(String name) + { + return utf8Dictionary.indexOf(StringUtils.toUtf8ByteBuffer(name)); + } + + @Override + public int getCardinality() + { + return utf8Dictionary.size(); + } + + @Override + public HistoricalDimensionSelector makeDimensionSelector( + final ReadableOffset offset, + @Nullable final ExtractionFn extractionFn + ) + { + class SingleValueQueryableDimensionSelector extends AbstractDimensionSelector + implements SingleValueHistoricalDimensionSelector, IdLookup, HistoricalDimensionSelector + { + private final SingleIndexedInt row = new SingleIndexedInt(); + + @Override + public int getValueCardinality() + { + /* + This is technically wrong if + extractionFn != null && (extractionFn.getExtractionType() != ExtractionFn.ExtractionType.ONE_TO_ONE || + !extractionFn.preservesOrdering()) + However current behavior allows some GroupBy-V1 queries to work that wouldn't work otherwise and doesn't + cause any problems due to special handling of extractionFn everywhere. + See https://github.com/apache/druid/pull/8433 + */ + return getCardinality(); + } + + @Override + public String lookupName(int id) + { + final String value = ScalarStringDictionaryEncodedColumn.this.lookupName(id); + return extractionFn == null ? value : extractionFn.apply(value); + } + + @Nullable + @Override + public ByteBuffer lookupNameUtf8(int id) + { + return utf8Dictionary.get(id); + } + + @Override + public boolean supportsLookupNameUtf8() + { + return true; + } + + @Override + public boolean nameLookupPossibleInAdvance() + { + return true; + } + + @Nullable + @Override + public IdLookup idLookup() + { + return extractionFn == null ? this : null; + } + + @Override + public int lookupId(String name) + { + if (extractionFn != null) { + throw new UnsupportedOperationException("cannot perform lookup when applying an extraction function"); + } + return ScalarStringDictionaryEncodedColumn.this.lookupId(name); + } + + @Override + public IndexedInts getRow() + { + row.setValue(getRowValue()); + return row; + } + + public int getRowValue() + { + return column.get(offset.getOffset()); + } + + @Override + public IndexedInts getRow(int offset) + { + row.setValue(getRowValue(offset)); + return row; + } + + @Override + public int getRowValue(int offset) + { + return column.get(offset); + } + + @Override + public ValueMatcher makeValueMatcher(final @Nullable String value) + { + if (extractionFn == null) { + final int valueId = lookupId(value); + if (valueId >= 0) { + return new ValueMatcher() + { + @Override + public boolean matches() + { + return getRowValue() == valueId; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", ScalarStringDictionaryEncodedColumn.this); + } + }; + } else { + return BooleanValueMatcher.of(false); + } + } else { + // Employ caching BitSet optimization + return makeValueMatcher(Predicates.equalTo(value)); + } + } + + @Override + public ValueMatcher makeValueMatcher(final Predicate predicate) + { + final BitSet checkedIds = new BitSet(getCardinality()); + final BitSet matchingIds = new BitSet(getCardinality()); + + // Lazy matcher; only check an id if matches() is called. + return new ValueMatcher() + { + @Override + public boolean matches() + { + final int id = getRowValue(); + + if (checkedIds.get(id)) { + return matchingIds.get(id); + } else { + final boolean matches = predicate.apply(lookupName(id)); + checkedIds.set(id); + if (matches) { + matchingIds.set(id); + } + return matches; + } + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", ScalarStringDictionaryEncodedColumn.this); + } + }; + } + + @Override + public Object getObject() + { + return lookupName(getRowValue()); + } + + @Override + public Class classOfObject() + { + return String.class; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", column); + inspector.visit("offset", offset); + inspector.visit("extractionFn", extractionFn); + } + } + return new SingleValueQueryableDimensionSelector(); + } + + @Override + public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(final ReadableVectorOffset offset) + { + final class StringVectorSelector extends StringDictionaryEncodedColumn.StringSingleValueDimensionVectorSelector + { + public StringVectorSelector() + { + super(column, offset); + } + + @Override + public int getValueCardinality() + { + return getCardinality(); + } + + @Nullable + @Override + public String lookupName(final int id) + { + return ScalarStringDictionaryEncodedColumn.this.lookupName(id); + } + + @Nullable + @Override + public ByteBuffer lookupNameUtf8(int id) + { + return utf8Dictionary.get(id); + } + + @Override + public int lookupId(@Nullable String name) + { + return ScalarStringDictionaryEncodedColumn.this.lookupId(name); + } + } + + return new StringVectorSelector(); + } + + @Override + public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(ReadableVectorOffset vectorOffset) + { + throw new UnsupportedOperationException("Column not multi-valued"); + } + + @Override + public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset) + { + final class StringVectorSelector extends StringDictionaryEncodedColumn.StringVectorObjectSelector + { + public StringVectorSelector() + { + super(column, offset); + } + + @Nullable + @Override + public String lookupName(int id) + { + return ScalarStringDictionaryEncodedColumn.this.lookupName(id); + } + } + return new StringVectorSelector(); + } + + @Override + public void close() throws IOException + { + CloseableUtils.closeAll(column); + } + + @Override + public ColumnType getLogicalType() + { + return ColumnType.STRING; + } + + @Override + public Indexed getStringDictionary() + { + return new StringEncodingStrategies.Utf8ToStringIndexed(utf8Dictionary); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringFieldColumnWriter.java similarity index 85% rename from processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java rename to processing/src/main/java/org/apache/druid/segment/nested/ScalarStringFieldColumnWriter.java index 5bb2f186b079..bf4c5f0bca3b 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringFieldColumnWriter.java @@ -28,16 +28,17 @@ import java.nio.channels.WritableByteChannel; /** - * Literal field writer for string type nested columns of {@link NestedDataColumnSerializer} + * Field writer for string type nested columns of {@link NestedDataColumnSerializerV4} and + * {@link NestedDataColumnSerializer} */ -public final class StringFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter +public final class ScalarStringFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter { - public StringFieldColumnWriter( + public ScalarStringFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec, - GlobalDictionaryIdLookup globalDictionaryIdLookup + DictionaryIdLookup globalDictionaryIdLookup ) { super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java b/processing/src/main/java/org/apache/druid/segment/nested/SortedValueDictionary.java similarity index 79% rename from processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java rename to processing/src/main/java/org/apache/druid/segment/nested/SortedValueDictionary.java index 70040d7b5286..b81637747aa0 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/SortedValueDictionary.java @@ -22,32 +22,37 @@ import org.apache.druid.segment.DictionaryMergingIterator; import org.apache.druid.segment.data.Indexed; +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; + /** * Container to collect a set of sorted {@link Indexed} representing the global value dictionaries of some * {@link NestedDataComplexColumn}, to later use with {@link DictionaryMergingIterator} * to merge into a new global dictionary */ -public class GlobalDictionarySortedCollector +public class SortedValueDictionary implements Closeable { private final Indexed sortedStrings; private final Indexed sortedLongs; private final Indexed sortedDoubles; - private final Iterable sortedArrays; - private final int arrayCount; + private final Indexed sortedArrays; + @Nullable + private final Closeable closeable; - public GlobalDictionarySortedCollector( + public SortedValueDictionary( Indexed sortedStrings, Indexed sortedLongs, Indexed sortedDoubles, - Iterable sortedArrays, - int arrayCount + Indexed sortedArrays, + @Nullable Closeable closeable ) { this.sortedStrings = sortedStrings; this.sortedLongs = sortedLongs; this.sortedDoubles = sortedDoubles; this.sortedArrays = sortedArrays; - this.arrayCount = arrayCount; + this.closeable = closeable; } public Indexed getSortedStrings() @@ -65,7 +70,7 @@ public Indexed getSortedDoubles() return sortedDoubles; } - public Iterable getSortedArrays() + public Indexed getSortedArrays() { return sortedArrays; } @@ -87,7 +92,7 @@ public int getDoubleCardinality() public int getArrayCardinality() { - return arrayCount; + return sortedArrays.size(); } public boolean allNull() @@ -103,6 +108,14 @@ public boolean allNull() if (sortedDoubles.size() > 0) { return false; } - return arrayCount == 0; + return sortedArrays.size() == 0; + } + + @Override + public void close() throws IOException + { + if (closeable != null) { + closeable.close(); + } } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/nested/ValueDictionary.java similarity index 88% rename from processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java rename to processing/src/main/java/org/apache/druid/segment/nested/ValueDictionary.java index 48b06d4e45c7..b8ffb0711a73 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ValueDictionary.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.AutoTypeColumnIndexer; import org.apache.druid.segment.ComparatorDimensionDictionary; import org.apache.druid.segment.ComparatorSortedDimensionDictionary; import org.apache.druid.segment.DimensionDictionary; @@ -38,11 +39,11 @@ import java.util.TreeSet; /** - * Used by {@link NestedDataColumnIndexer} to build the global value dictionary, which can be converted into a - * {@link GlobalDictionarySortedCollector} to sort and write out the values to a segment with - * {@link #getSortedCollector()}. + * Used by {@link AutoTypeColumnIndexer} and {@link NestedDataColumnIndexer} to build the + * value dictionary, which can be converted into a {@link SortedValueDictionary} to sort and write out the values to a + * segment with {@link #getSortedCollector()}. */ -public class GlobalDimensionDictionary +public class ValueDictionary { private final ComparatorDimensionDictionary stringDictionary; private final ComparatorDimensionDictionary longDictionary; @@ -53,7 +54,7 @@ public class GlobalDimensionDictionary private int arrayBytesSizeEstimate; - public GlobalDimensionDictionary() + public ValueDictionary() { this.stringDictionary = new ComparatorDimensionDictionary(GenericIndexed.STRING_STRATEGY) { @@ -150,7 +151,7 @@ public int addDoubleArray(@Nullable Object[] value) return sizeEstimate; } - public GlobalDictionarySortedCollector getSortedCollector() + public SortedValueDictionary getSortedCollector() { final ComparatorSortedDimensionDictionary sortedStringDimensionDictionary = stringDictionary.sort(); @@ -313,8 +314,41 @@ private int[] convertArray(Object[] array) sortedArrays.addAll(stringArrays); sortedArrays.addAll(longArrays); sortedArrays.addAll(doubleArrays); + Indexed sortedArraysIndexed = new Indexed() + { + @Override + public Iterator iterator() + { + return sortedArrays.iterator(); + } + + @Override + public int size() + { + return sortedArrays.size(); + } + + @Nullable + @Override + public Object[] get(int index) + { + return new Object[0]; + } + + @Override + public int indexOf(@Nullable Object[] value) + { + throw new UnsupportedOperationException("indexOf not supported"); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } + }; - return new GlobalDictionarySortedCollector(strings, longs, doubles, sortedArrays, sortedArrays.size()); + return new SortedValueDictionary(strings, longs, doubles, sortedArraysIndexed, null); } public long sizeInBytes() diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayColumn.java new file mode 100644 index 000000000000..f2ec4c2b3954 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayColumn.java @@ -0,0 +1,382 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.primitives.Doubles; +import com.google.common.primitives.Floats; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.common.guava.GuavaUtils; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.StringEncodingStrategies; +import org.apache.druid.segment.data.ColumnarInts; +import org.apache.druid.segment.data.FixedIndexed; +import org.apache.druid.segment.data.FrontCodedIntArrayIndexed; +import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.data.ReadableOffset; +import org.apache.druid.segment.vector.ReadableVectorInspector; +import org.apache.druid.segment.vector.ReadableVectorOffset; +import org.apache.druid.segment.vector.VectorObjectSelector; +import org.roaringbitmap.PeekableIntIterator; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; + +public class VariantArrayColumn> implements NestedCommonFormatColumn +{ + private final TStringDictionary stringDictionary; + private final FixedIndexed longDictionary; + private final FixedIndexed doubleDictionary; + private final FrontCodedIntArrayIndexed arrayDictionary; + private final ColumnarInts encodedValueColumn; + private final ImmutableBitmap nullValueBitmap; + private final ColumnType logicalType; + private final int adjustLongId; + private final int adjustDoubleId; + private final int adjustArrayId; + + public VariantArrayColumn( + TStringDictionary stringDictionary, + FixedIndexed longDictionary, + FixedIndexed doubleDictionary, + FrontCodedIntArrayIndexed arrayDictionary, + ColumnarInts encodedValueColumn, + ImmutableBitmap nullValueBitmap, + ColumnType logicalType + ) + { + this.stringDictionary = stringDictionary; + this.longDictionary = longDictionary; + this.doubleDictionary = doubleDictionary; + this.arrayDictionary = arrayDictionary; + this.encodedValueColumn = encodedValueColumn; + this.nullValueBitmap = nullValueBitmap; + this.logicalType = logicalType; + this.adjustLongId = stringDictionary.size(); + this.adjustDoubleId = adjustLongId + longDictionary.size(); + this.adjustArrayId = adjustDoubleId + doubleDictionary.size(); + } + + @Override + public ColumnType getLogicalType() + { + return logicalType; + } + + @Override + public Indexed getStringDictionary() + { + return new StringEncodingStrategies.Utf8ToStringIndexed(stringDictionary); + } + + @Override + public Indexed getLongDictionary() + { + return longDictionary; + } + + @Override + public Indexed getDoubleDictionary() + { + return doubleDictionary; + } + + @Override + public Indexed getArrayDictionary() + { + Iterable arrays = () -> { + + return new Iterator() + { + final Iterator delegate = arrayDictionary.iterator(); + + @Override + public boolean hasNext() + { + return delegate.hasNext(); + } + + @Override + public Object[] next() + { + final int[] next = delegate.next(); + final Object[] nextArray = new Object[next.length]; + for (int i = 0; i < nextArray.length; i++) { + nextArray[i] = lookupId(next[i]); + } + return nextArray; + } + + @Nullable + private Object lookupId(int globalId) + { + if (globalId == 0) { + return null; + } + final int adjustLongId = stringDictionary.size(); + final int adjustDoubleId = stringDictionary.size() + longDictionary.size(); + if (globalId < adjustLongId) { + return StringUtils.fromUtf8Nullable(stringDictionary.get(globalId)); + } else if (globalId < adjustDoubleId) { + return longDictionary.get(globalId - adjustLongId); + } else if (globalId < adjustDoubleId + doubleDictionary.size()) { + return doubleDictionary.get(globalId - adjustDoubleId); + } + throw new IAE("Unknown globalId [%s]", globalId); + } + }; + }; + return new Indexed() + { + @Override + public int size() + { + return arrayDictionary.size(); + } + + @Nullable + @Override + public Object[] get(int index) + { + throw new UnsupportedOperationException("get not supported"); + } + + @Override + public int indexOf(@Nullable Object[] value) + { + throw new UnsupportedOperationException("indexOf not supported"); + } + + @Override + public Iterator iterator() + { + return arrays.iterator(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // meh + } + }; + } + + @Override + public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + return new ColumnValueSelector() + { + + private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator(); + private int nullMark = -1; + private int offsetMark = -1; + + @Nullable + @Override + public Object getObject() + { + final int id = encodedValueColumn.get(offset.getOffset()); + if (id < adjustArrayId) { + return lookupScalarValue(id); + } else { + int[] arr = arrayDictionary.get(id - adjustArrayId); + if (arr == null) { + return null; + } + final Object[] array = new Object[arr.length]; + for (int i = 0; i < arr.length; i++) { + array[i] = lookupScalarValue(arr[i]); + } + return array; + } + } + + @Override + public float getFloat() + { + final int id = encodedValueColumn.get(offset.getOffset()); + if (id == 0) { + // zero + return 0f; + } else if (id < adjustLongId) { + // try to convert string to float + Float f = Floats.tryParse(StringUtils.fromUtf8(stringDictionary.get(id))); + return f == null ? 0f : f; + } else if (id < adjustDoubleId) { + return longDictionary.get(id - adjustLongId).floatValue(); + } else { + return doubleDictionary.get(id - adjustDoubleId).floatValue(); + } + } + + @Override + public double getDouble() + { + final int id = encodedValueColumn.get(offset.getOffset()); + if (id == 0) { + // zero + return 0.0; + } else if (id < adjustLongId) { + // try to convert string to double + Double d = Doubles.tryParse(StringUtils.fromUtf8(stringDictionary.get(id))); + return d == null ? 0.0 : d; + } else if (id < adjustDoubleId) { + return longDictionary.get(id - adjustLongId).doubleValue(); + } else { + return doubleDictionary.get(id - adjustDoubleId); + } + } + + @Override + public long getLong() + { + final int id = encodedValueColumn.get(offset.getOffset()); + if (id == 0) { + // zero + return 0L; + } else if (id < adjustLongId) { + // try to convert string to long + Long l = GuavaUtils.tryParseLong(StringUtils.fromUtf8(stringDictionary.get(id))); + return l == null ? 0L : l; + } else if (id < adjustDoubleId) { + return longDictionary.get(id - adjustLongId); + } else { + return doubleDictionary.get(id - adjustDoubleId).longValue(); + } + } + + @Override + public boolean isNull() + { + final int i = offset.getOffset(); + if (i < offsetMark) { + // offset was reset, reset iterator state + nullMark = -1; + nullIterator = nullValueBitmap.peekableIterator(); + } + offsetMark = i; + if (nullMark < i) { + nullIterator.advanceIfNeeded(offsetMark); + if (nullIterator.hasNext()) { + nullMark = nullIterator.next(); + } + } + if (nullMark == offsetMark) { + return true; + } + return DimensionHandlerUtils.isNumericNull(getObject()); + } + + @Override + public Class classOfObject() + { + return Object.class; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("encodedValueColumn", encodedValueColumn); + } + }; + } + + @Override + public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset) + { + return new VectorObjectSelector() + { + private final int[] vector = new int[offset.getMaxVectorSize()]; + private final Object[] objects = new Object[offset.getMaxVectorSize()]; + private int id = ReadableVectorInspector.NULL_ID; + + @Override + + public Object[] getObjectVector() + { + if (id == offset.getId()) { + return objects; + } + + if (offset.isContiguous()) { + encodedValueColumn.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize()); + } else { + encodedValueColumn.get(vector, offset.getOffsets(), offset.getCurrentVectorSize()); + } + for (int i = 0; i < offset.getCurrentVectorSize(); i++) { + final int globalId = vector[i]; + if (globalId < adjustArrayId) { + objects[i] = lookupScalarValue(globalId); + } else { + int[] arr = arrayDictionary.get(globalId - adjustArrayId); + if (arr == null) { + objects[i] = null; + } else { + final Object[] array = new Object[arr.length]; + for (int j = 0; j < arr.length; j++) { + array[j] = lookupScalarValue(arr[j]); + } + objects[i] = array; + } + } + } + id = offset.getId(); + + return objects; + } + + @Override + public int getMaxVectorSize() + { + return offset.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return offset.getCurrentVectorSize(); + } + }; + } + + @Override + public void close() throws IOException + { + encodedValueColumn.close(); + } + + private Object lookupScalarValue(int globalId) + { + if (globalId < adjustLongId) { + return StringUtils.fromUtf8Nullable(stringDictionary.get(globalId)); + } else if (globalId < adjustDoubleId) { + return longDictionary.get(globalId - adjustLongId); + } else if (globalId < adjustArrayId) { + return doubleDictionary.get(globalId - adjustDoubleId); + } + throw new IllegalArgumentException("not a scalar in the dictionary"); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayColumnAndIndexSupplier.java new file mode 100644 index 000000000000..3a2dcb2eb278 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayColumnAndIndexSupplier.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Supplier; +import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +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.smoosh.SmooshedFileMapper; +import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnIndexSupplier; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.NullValueIndex; +import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; +import org.apache.druid.segment.column.StringEncodingStrategy; +import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.ColumnarInts; +import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier; +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 org.apache.druid.segment.data.VByte; +import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class VariantArrayColumnAndIndexSupplier implements Supplier, ColumnIndexSupplier +{ + public static VariantArrayColumnAndIndexSupplier read( + ColumnType logicalType, + ByteOrder byteOrder, + BitmapSerdeFactory bitmapSerdeFactory, + ByteBuffer bb, + ColumnBuilder columnBuilder + ) + { + final byte version = bb.get(); + final int columnNameLength = VByte.readInt(bb); + final String columnName = StringUtils.fromUtf8(bb, columnNameLength); + + if (version == NestedCommonFormatColumnSerializer.V0) { + try { + final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); + final GenericIndexed stringDictionary; + final Supplier frontCodedStringDictionarySupplier; + final Supplier> longDictionarySupplier; + final Supplier> doubleDictionarySupplier; + final Supplier arrayDictionarySupplier; + + final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME + ); + + final int dictionaryStartPosition = stringDictionaryBuffer.position(); + final byte dictionaryVersion = stringDictionaryBuffer.get(); + + if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) { + final byte encodingId = stringDictionaryBuffer.get(); + if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) { + frontCodedStringDictionarySupplier = FrontCodedIndexed.read( + stringDictionaryBuffer, + byteOrder + ); + stringDictionary = null; + } else if (encodingId == StringEncodingStrategy.UTF8_ID) { + // this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but + // this provides backwards compatibility should we switch at some point in the future to always + // writing dictionaryVersion + stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper); + frontCodedStringDictionarySupplier = null; + } else { + throw new ISE("impossible, unknown encoding strategy id: %s", encodingId); + } + } else { + // legacy format that only supports plain utf8 enoding stored in GenericIndexed and the byte we are reading + // as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the + // GenericIndexed version can be correctly read + stringDictionaryBuffer.position(dictionaryStartPosition); + stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper); + frontCodedStringDictionarySupplier = null; + } + final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.ENCODED_VALUE_COLUMN_FILE_NAME + ); + final CompressedVSizeColumnarIntsSupplier ints = CompressedVSizeColumnarIntsSupplier.fromByteBuffer( + encodedValueColumn, + byteOrder + ); + final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.LONG_DICTIONARY_FILE_NAME + ); + final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.DOUBLE_DICTIONARY_FILE_NAME + ); + final ByteBuffer valueIndexBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.BITMAP_INDEX_FILE_NAME + ); + GenericIndexed valueIndexes = GenericIndexed.read( + valueIndexBuffer, + bitmapSerdeFactory.getObjectStrategy(), + columnBuilder.getFileMapper() + ); + + longDictionarySupplier = FixedIndexed.read( + longDictionaryBuffer, + ColumnType.LONG.getStrategy(), + byteOrder, + Long.BYTES + ); + doubleDictionarySupplier = FixedIndexed.read( + doubleDictionaryBuffer, + ColumnType.DOUBLE.getStrategy(), + byteOrder, + Double.BYTES + ); + final ByteBuffer arrayDictionarybuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.ARRAY_DICTIONARY_FILE_NAME + ); + arrayDictionarySupplier = FrontCodedIntArrayIndexed.read( + arrayDictionarybuffer, + byteOrder + ); + return new VariantArrayColumnAndIndexSupplier( + logicalType, + stringDictionary, + frontCodedStringDictionarySupplier, + longDictionarySupplier, + doubleDictionarySupplier, + arrayDictionarySupplier, + ints, + valueIndexes, + bitmapSerdeFactory.getBitmapFactory() + ); + } + catch (IOException ex) { + throw new RE(ex, "Failed to deserialize V%s column.", version); + } + } else { + throw new RE("Unknown version " + version); + } + } + + + private final ColumnType logicalType; + + private final GenericIndexed stringDictionary; + private final Supplier frontCodedStringDictionarySupplier; + private final Supplier> longDictionarySupplier; + private final Supplier> doubleDictionarySupplier; + private final Supplier arrayDictionarySupplier; + private final Supplier encodedValueColumnSupplier; + @SuppressWarnings("unused") + private final GenericIndexed valueIndexes; + private final ImmutableBitmap nullValueBitmap; + + public VariantArrayColumnAndIndexSupplier( + ColumnType logicalType, + GenericIndexed stringDictionary, + Supplier frontCodedStringDictionarySupplier, + Supplier> longDictionarySupplier, + Supplier> doubleDictionarySupplier, + Supplier arrayDictionarySupplier, + Supplier encodedValueColumnSupplier, + GenericIndexed valueIndexes, + @SuppressWarnings("unused") BitmapFactory bitmapFactory + ) + { + this.logicalType = logicalType; + this.stringDictionary = stringDictionary; + this.frontCodedStringDictionarySupplier = frontCodedStringDictionarySupplier; + this.longDictionarySupplier = longDictionarySupplier; + this.doubleDictionarySupplier = doubleDictionarySupplier; + this.arrayDictionarySupplier = arrayDictionarySupplier; + this.encodedValueColumnSupplier = encodedValueColumnSupplier; + this.valueIndexes = valueIndexes; + this.nullValueBitmap = valueIndexes.get(0) == null ? bitmapFactory.makeEmptyImmutableBitmap() : valueIndexes.get(0); + } + + @Override + public NestedCommonFormatColumn get() + { + if (frontCodedStringDictionarySupplier != null) { + return new VariantArrayColumn<>( + frontCodedStringDictionarySupplier.get(), + longDictionarySupplier.get(), + doubleDictionarySupplier.get(), + arrayDictionarySupplier.get(), + encodedValueColumnSupplier.get(), + nullValueBitmap, + logicalType + ); + } + return new VariantArrayColumn<>( + stringDictionary.singleThreaded(), + longDictionarySupplier.get(), + doubleDictionarySupplier.get(), + arrayDictionarySupplier.get(), + encodedValueColumnSupplier.get(), + nullValueBitmap, + logicalType + ); + } + + @Nullable + @Override + public T as(Class clazz) + { + if (clazz.equals(NullValueIndex.class)) { + final BitmapColumnIndex nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap); + return (T) (NullValueIndex) () -> nullIndex; + } + // coming soon... + return null; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayColumnSerializer.java new file mode 100644 index 000000000000..541c3552206c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayColumnSerializer.java @@ -0,0 +1,341 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.nested; + +import com.google.common.base.Preconditions; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.collections.bitmap.MutableBitmap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.ProgressIndicator; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.StringEncodingStrategies; +import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSerializer; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.DictionaryWriter; +import org.apache.druid.segment.data.FixedIndexedIntWriter; +import org.apache.druid.segment.data.FixedIndexedWriter; +import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter; +import org.apache.druid.segment.data.GenericIndexedWriter; +import org.apache.druid.segment.data.SingleValueColumnarIntsSerializer; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +public class VariantArrayColumnSerializer extends NestedCommonFormatColumnSerializer +{ + private static final Logger log = new Logger(VariantArrayColumnSerializer.class); + + private final String name; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private final IndexSpec indexSpec; + @SuppressWarnings("unused") + private final Closer closer; + private DictionaryIdLookup dictionaryIdLookup; + private DictionaryWriter dictionaryWriter; + private FixedIndexedWriter longDictionaryWriter; + private FixedIndexedWriter doubleDictionaryWriter; + private FrontCodedIntArrayIndexedWriter arrayDictionaryWriter; + private FixedIndexedIntWriter arrayElementDictionaryWriter; + + private int rowCount = 0; + private boolean closedForWrite = false; + + private boolean dictionarySerialized = false; + + private SingleValueColumnarIntsSerializer encodedValueSerializer; + private GenericIndexedWriter bitmapIndexWriter; + private GenericIndexedWriter arrayElementIndexWriter; + private MutableBitmap[] bitmaps; + private ByteBuffer columnNameBytes = null; + private final Int2ObjectRBTreeMap arrayElements = new Int2ObjectRBTreeMap<>(); + + public VariantArrayColumnSerializer( + String name, + IndexSpec indexSpec, + SegmentWriteOutMedium segmentWriteOutMedium, + @SuppressWarnings("unused") ProgressIndicator progressIndicator, + Closer closer + ) + { + this.name = name; + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.indexSpec = indexSpec; + this.closer = closer; + this.dictionaryIdLookup = new DictionaryIdLookup(); + } + + @Override + public String getColumnName() + { + return name; + } + + @Override + public DictionaryIdLookup getGlobalLookup() + { + return dictionaryIdLookup; + } + + @Override + public boolean hasNulls() + { + return !bitmaps[0].isEmpty(); + } + + @Override + public void openDictionaryWriter() throws IOException + { + dictionaryWriter = StringEncodingStrategies.getStringDictionaryWriter( + indexSpec.getStringDictionaryEncoding(), + segmentWriteOutMedium, + name + ); + dictionaryWriter.open(); + + longDictionaryWriter = new FixedIndexedWriter<>( + segmentWriteOutMedium, + ColumnType.LONG.getStrategy(), + ByteOrder.nativeOrder(), + Long.BYTES, + true + ); + longDictionaryWriter.open(); + + doubleDictionaryWriter = new FixedIndexedWriter<>( + segmentWriteOutMedium, + ColumnType.DOUBLE.getStrategy(), + ByteOrder.nativeOrder(), + Double.BYTES, + true + ); + doubleDictionaryWriter.open(); + + arrayDictionaryWriter = new FrontCodedIntArrayIndexedWriter( + segmentWriteOutMedium, + ByteOrder.nativeOrder(), + 4 + ); + arrayDictionaryWriter.open(); + arrayElementDictionaryWriter = new FixedIndexedIntWriter(segmentWriteOutMedium, true); + arrayElementDictionaryWriter.open(); + } + + @Override + public void open() throws IOException + { + if (!dictionarySerialized) { + throw new IllegalStateException("Dictionary not serialized, cannot open value serializer"); + } + String filenameBase = StringUtils.format("%s.forward_dim", name); + final int cardinality = dictionaryWriter.getCardinality() + + longDictionaryWriter.getCardinality() + + doubleDictionaryWriter.getCardinality() + + arrayDictionaryWriter.getCardinality(); + final CompressionStrategy compression = indexSpec.getDimensionCompression(); + final CompressionStrategy compressionToUse; + if (compression != CompressionStrategy.UNCOMPRESSED && compression != CompressionStrategy.NONE) { + compressionToUse = compression; + } else { + compressionToUse = CompressionStrategy.LZ4; + } + encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create( + name, + segmentWriteOutMedium, + filenameBase, + cardinality, + compressionToUse + ); + encodedValueSerializer.open(); + + bitmapIndexWriter = new GenericIndexedWriter<>( + segmentWriteOutMedium, + name, + indexSpec.getBitmapSerdeFactory().getObjectStrategy() + ); + bitmapIndexWriter.open(); + bitmapIndexWriter.setObjectsNotSorted(); + bitmaps = new MutableBitmap[cardinality]; + for (int i = 0; i < bitmaps.length; i++) { + bitmaps[i] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); + } + arrayElementIndexWriter = new GenericIndexedWriter<>( + segmentWriteOutMedium, + name + "_arrays", + indexSpec.getBitmapSerdeFactory().getObjectStrategy() + ); + arrayElementIndexWriter.open(); + arrayElementIndexWriter.setObjectsNotSorted(); + } + + @Override + public void serializeDictionaries( + Iterable strings, + Iterable longs, + Iterable doubles, + Iterable 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); + dictionaryIdLookup.addString(null); + for (String value : strings) { + value = NullHandling.emptyToNullIfNeeded(value); + if (value == null) { + continue; + } + + dictionaryWriter.write(value); + dictionaryIdLookup.addString(value); + } + + for (Long value : longs) { + if (value == null) { + continue; + } + longDictionaryWriter.write(value); + dictionaryIdLookup.addLong(value); + } + + for (Double value : doubles) { + if (value == null) { + continue; + } + doubleDictionaryWriter.write(value); + dictionaryIdLookup.addDouble(value); + } + + for (int[] value : arrays) { + if (value == null) { + continue; + } + arrayDictionaryWriter.write(value); + dictionaryIdLookup.addArray(value); + } + dictionarySerialized = true; + } + + @Override + public void serialize(ColumnValueSelector selector) throws IOException + { + if (!dictionarySerialized) { + throw new ISE("Must serialize value dictionaries before serializing values for column [%s]", name); + } + + ExprEval eval = ExprEval.bestEffortOf(StructuredData.unwrap(selector.getObject())); + int[] globalIds = null; + if (eval.isArray()) { + Object[] array = eval.asArray(); + 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] = dictionaryIdLookup.lookupString((String) array[i]); + } else if (array[i] instanceof Long) { + globalIds[i] = dictionaryIdLookup.lookupLong((Long) array[i]); + } else if (array[i] instanceof Double) { + globalIds[i] = dictionaryIdLookup.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(rowCount); + } + } + final int dictId = globalIds == null ? 0 : dictionaryIdLookup.lookupArray(globalIds); + encodedValueSerializer.addValue(dictId); + bitmaps[dictId].add(rowCount); + rowCount++; + } + + private void closeForWrite() throws IOException + { + if (!closedForWrite) { + for (int i = 0; i < bitmaps.length; i++) { + final MutableBitmap bitmap = bitmaps[i]; + bitmapIndexWriter.write( + indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(bitmap) + ); + bitmaps[i] = null; // Reclaim memory + } + for (Int2ObjectMap.Entry arrayElement : arrayElements.int2ObjectEntrySet()) { + arrayElementDictionaryWriter.write(arrayElement.getIntKey()); + arrayElementIndexWriter.write( + indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(arrayElement.getValue()) + ); + } + columnNameBytes = computeFilenameBytes(); + closedForWrite = true; + } + } + + @Override + public long getSerializedSize() throws IOException + { + closeForWrite(); + + long size = 1 + columnNameBytes.capacity(); + // the value dictionaries, raw column, and null index are all stored in separate files + return size; + } + + @Override + public void writeTo( + WritableByteChannel channel, + FileSmoosher smoosher + ) throws IOException + { + Preconditions.checkState(closedForWrite, "Not closed yet!"); + Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + + writeV0Header(channel, columnNameBytes); + + 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, arrayElementDictionaryWriter, ARRAY_ELEMENT_DICTIONARY_FILE_NAME); + writeInternal(smoosher, encodedValueSerializer, ENCODED_VALUE_COLUMN_FILE_NAME); + writeInternal(smoosher, bitmapIndexWriter, BITMAP_INDEX_FILE_NAME); + writeInternal(smoosher, arrayElementIndexWriter, ARRAY_ELEMENT_BITMAP_INDEX_FILE_NAME); + + log.info("Column [%s] serialized successfully.", name); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayFieldColumnWriter.java similarity index 93% rename from processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java rename to processing/src/main/java/org/apache/druid/segment/nested/VariantArrayFieldColumnWriter.java index 1ae465d25cd9..147f2b04474f 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayFieldColumnWriter.java @@ -27,15 +27,15 @@ import java.io.IOException; import java.nio.channels.WritableByteChannel; -public class ArrayFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter +public class VariantArrayFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter { - protected ArrayFieldColumnWriter( + public VariantArrayFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec, - GlobalDictionaryIdLookup globalDictionaryIdLookup + DictionaryIdLookup globalDictionaryIdLookup ) { super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantFieldColumnWriter.java index 121f388583c9..e0cd92dd6b96 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantFieldColumnWriter.java @@ -28,7 +28,7 @@ import java.nio.channels.WritableByteChannel; /** - * Literal field writer for mixed type nested columns of {@link NestedDataColumnSerializer} + * Literal field writer for mixed type nested columns of {@link NestedDataColumnSerializerV4} */ public final class VariantFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter { @@ -37,7 +37,7 @@ public VariantFieldColumnWriter( String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec, - GlobalDictionaryIdLookup globalDictionaryIdLookup + DictionaryIdLookup globalDictionaryIdLookup ) { super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); diff --git a/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java index 8fffc0427fde..19008a36526b 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java @@ -39,7 +39,8 @@ @JsonSubTypes.Type(name = "floatV2", value = FloatNumericColumnPartSerdeV2.class), @JsonSubTypes.Type(name = "longV2", value = LongNumericColumnPartSerdeV2.class), @JsonSubTypes.Type(name = "doubleV2", value = DoubleNumericColumnPartSerdeV2.class), - @JsonSubTypes.Type(name = "null", value = NullColumnPartSerde.class) + @JsonSubTypes.Type(name = "null", value = NullColumnPartSerde.class), + @JsonSubTypes.Type(name = "nestedCommonFormat", value = NestedCommonFormatColumnPartSerde.class) }) public interface ColumnPartSerde { diff --git a/processing/src/main/java/org/apache/druid/segment/serde/ComplexMetricSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/ComplexMetricSerde.java index 5442383c34cf..694009ca8fd7 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/ComplexMetricSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/ComplexMetricSerde.java @@ -60,6 +60,7 @@ public void deserializeColumn( deserializeColumn(buffer, builder); } + /** * {@link ComplexMetricSerde#deserializeColumn(ByteBuffer, ColumnBuilder, ColumnConfig)} should be used instead of this. * This method is left for backward compatibility. diff --git a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java new file mode 100644 index 000000000000..7c1fea26abe6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java @@ -0,0 +1,268 @@ +/* + * 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.serde; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.nested.NestedCommonFormatColumn; +import org.apache.druid.segment.nested.NestedCommonFormatColumnSerializer; +import org.apache.druid.segment.nested.NestedDataColumnSupplier; +import org.apache.druid.segment.nested.ScalarDoubleColumnAndIndexSupplier; +import org.apache.druid.segment.nested.ScalarLongColumnAndIndexSupplier; +import org.apache.druid.segment.nested.ScalarStringColumnAndIndexSupplier; +import org.apache.druid.segment.nested.VariantArrayColumnAndIndexSupplier; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde +{ + public static SerializerBuilder serializerBuilder() + { + return new SerializerBuilder(); + } + + public static ByteBuffer loadInternalFile( + SmooshedFileMapper fileMapper, + String fileNameBase, + String internalFileName + ) throws IOException + { + return fileMapper.mapFile( + NestedCommonFormatColumnSerializer.getInternalFileName(fileNameBase, internalFileName) + ); + } + + @JsonCreator + public static NestedCommonFormatColumnPartSerde createDeserializer( + @JsonProperty("logicalType") ColumnType logicalType, + @JsonProperty("hasNulls") boolean hasNulls, + @JsonProperty("byteOrder") ByteOrder byteOrder, + @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory + ) + { + return new NestedCommonFormatColumnPartSerde(logicalType, hasNulls, byteOrder, bitmapSerdeFactory, null); + } + + private final ColumnType logicalType; + private final boolean hasNulls; + private final ByteOrder byteOrder; + private final BitmapSerdeFactory bitmapSerdeFactory; + + @Nullable + private final Serializer serializer; + + + private NestedCommonFormatColumnPartSerde( + ColumnType logicalType, + boolean hasNulls, + ByteOrder byteOrder, + BitmapSerdeFactory bitmapSerdeFactory, + @Nullable Serializer serializer + ) + { + this.logicalType = logicalType; + this.hasNulls = hasNulls; + this.byteOrder = byteOrder; + this.bitmapSerdeFactory = bitmapSerdeFactory; + this.serializer = serializer; + } + + @JsonIgnore + @Nullable + @Override + public Serializer getSerializer() + { + return serializer; + } + + @Override + public Deserializer getDeserializer() + { + if (logicalType.is(ValueType.STRING)) { + return ((buffer, builder, columnConfig) -> { + ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read( + byteOrder, + bitmapSerdeFactory, + buffer, + builder + ); + ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + capabilitiesBuilder.setDictionaryEncoded(true); + capabilitiesBuilder.setDictionaryValuesSorted(true); + capabilitiesBuilder.setDictionaryValuesUnique(true); + builder.setType(logicalType); + builder.setStandardTypeColumnSupplier(supplier); + builder.setIndexSupplier(supplier, true, false); + builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); + }); + } + if (logicalType.is(ValueType.LONG)) { + return ((buffer, builder, columnConfig) -> { + ScalarLongColumnAndIndexSupplier supplier = ScalarLongColumnAndIndexSupplier.read( + byteOrder, + bitmapSerdeFactory, + buffer, + builder + ); + ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + capabilitiesBuilder.setDictionaryEncoded(true); + capabilitiesBuilder.setDictionaryValuesSorted(true); + capabilitiesBuilder.setDictionaryValuesUnique(true); + builder.setType(logicalType); + builder.setStandardTypeColumnSupplier(supplier); + builder.setIndexSupplier(supplier, true, false); + builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); + }); + } + if (logicalType.is(ValueType.DOUBLE)) { + return ((buffer, builder, columnConfig) -> { + ScalarDoubleColumnAndIndexSupplier supplier = ScalarDoubleColumnAndIndexSupplier.read( + byteOrder, + bitmapSerdeFactory, + buffer, + builder + ); + ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + capabilitiesBuilder.setDictionaryEncoded(true); + capabilitiesBuilder.setDictionaryValuesSorted(true); + capabilitiesBuilder.setDictionaryValuesUnique(true); + builder.setType(logicalType); + builder.setStandardTypeColumnSupplier(supplier); + builder.setIndexSupplier(supplier, true, false); + builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); + }); + } + if (logicalType.isArray()) { + return ((buffer, builder, columnConfig) -> { + VariantArrayColumnAndIndexSupplier supplier = VariantArrayColumnAndIndexSupplier.read( + logicalType, + byteOrder, + bitmapSerdeFactory, + buffer, + builder + ); + ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + capabilitiesBuilder.setDictionaryEncoded(true); + capabilitiesBuilder.setDictionaryValuesSorted(true); + capabilitiesBuilder.setDictionaryValuesUnique(true); + builder.setType(logicalType); + builder.setStandardTypeColumnSupplier(supplier); + builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); + }); + } + return (buffer, builder, columnConfig) -> { + NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read( + hasNulls, + buffer, + builder, + columnConfig, + bitmapSerdeFactory, + byteOrder + ); + ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + capabilitiesBuilder.setDictionaryEncoded(true); + capabilitiesBuilder.setDictionaryValuesSorted(true); + capabilitiesBuilder.setDictionaryValuesUnique(true); + ColumnType simpleType = supplier.getLogicalType(); + ColumnType logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType; + builder.setType(logicalType); + builder.setStandardTypeColumnSupplier(supplier); + builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, hasNulls)); + }; + } + + @JsonProperty + public ColumnType getLogicalType() + { + return logicalType; + } + + @JsonProperty + public boolean isHasNulls() + { + return hasNulls; + } + + @JsonProperty + public ByteOrder getByteOrder() + { + return byteOrder; + } + + @JsonProperty + public BitmapSerdeFactory getBitmapSerdeFactory() + { + return bitmapSerdeFactory; + } + + public static class SerializerBuilder + { + private ColumnType logicalType; + private boolean hasNulls; + private ByteOrder byteOrder = ByteOrder.nativeOrder(); + BitmapSerdeFactory bitmapSerdeFactory; + @Nullable + private Serializer serializer = null; + + public SerializerBuilder withLogicalType(ColumnType logicalType) + { + this.logicalType = logicalType; + return this; + } + + public SerializerBuilder withSerializer(final Serializer serializer) + { + this.serializer = serializer; + return this; + } + + public SerializerBuilder withByteOrder(final ByteOrder byteOrder) + { + this.byteOrder = byteOrder; + return this; + } + + public SerializerBuilder withBitmapSerdeFactory(BitmapSerdeFactory serdeFactory) + { + this.bitmapSerdeFactory = serdeFactory; + return this; + } + + public SerializerBuilder withHasNulls(boolean hasNulls) + { + this.hasNulls = hasNulls; + return this; + } + + public NestedCommonFormatColumnPartSerde build() + { + return new NestedCommonFormatColumnPartSerde(logicalType, hasNulls, byteOrder, bitmapSerdeFactory, serializer); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedDictionaryEncodedColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedDictionaryEncodedColumnSupplier.java index 622b554d3dae..d67730de4363 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedDictionaryEncodedColumnSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedDictionaryEncodedColumnSupplier.java @@ -21,6 +21,7 @@ import com.google.common.base.Supplier; import org.apache.druid.segment.column.DictionaryEncodedColumn; +import org.apache.druid.segment.column.StringDictionaryEncodedColumn; import org.apache.druid.segment.column.StringFrontCodedDictionaryEncodedColumn; import org.apache.druid.segment.data.ColumnarInts; import org.apache.druid.segment.data.ColumnarMultiInts; @@ -30,7 +31,7 @@ /** * {@link DictionaryEncodedColumnSupplier} but for columns using a {@link StringFrontCodedDictionaryEncodedColumn} - * instead of the traditional {@link org.apache.druid.segment.column.StringDictionaryEncodedColumn} + * instead of the traditional {@link StringDictionaryEncodedColumn} */ public class StringFrontCodedDictionaryEncodedColumnSupplier implements Supplier> { 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 f4902946bfb9..617e1befe77f 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 @@ -28,6 +28,7 @@ import com.google.common.primitives.Doubles; import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.guava.GuavaUtils; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Numbers; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.dimension.DimensionSpec; @@ -62,6 +63,7 @@ import org.apache.druid.segment.nested.NestedPathFinder; import org.apache.druid.segment.nested.NestedPathPart; import org.apache.druid.segment.nested.StructuredData; +import org.apache.druid.segment.nested.VariantArrayColumn; import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector; import org.apache.druid.segment.vector.BaseLongVectorValueSelector; import org.apache.druid.segment.vector.NilVectorSelector; @@ -292,6 +294,41 @@ private DimensionSelector makeDimensionSelectorUndecorated( ); } + if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantArrayColumn) { + final VariantArrayColumn arrayColumn = (VariantArrayColumn) theColumn; + ColumnValueSelector arraySelector = arrayColumn.makeColumnValueSelector(offset); + final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); + if (elementNumber < 0) { + throw new IAE("Cannot make array element selector, negative array index not supported"); + } + return new 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); + } + }; + } + // we are not a nested column and are being asked for a path that will never exist, so we are nil selector return DimensionSelector.constant(null, extractionFn); } @@ -337,6 +374,71 @@ public ColumnValueSelector makeColumnValueSelector( return theColumn.makeColumnValueSelector(offset); } + if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantArrayColumn) { + final VariantArrayColumn arrayColumn = (VariantArrayColumn) theColumn; + ColumnValueSelector arraySelector = arrayColumn.makeColumnValueSelector(offset); + final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); + if (elementNumber < 0) { + throw new IAE("Cannot make array element selector, negative array index not supported"); + } + return new ColumnValueSelector() + { + @Override + public boolean isNull() + { + Object o = getObject(); + return !(o instanceof Number); + } + + @Override + public long getLong() + { + Object o = getObject(); + return o instanceof Number ? ((Number) o).longValue() : 0L; + } + + @Override + public float getFloat() + { + Object o = getObject(); + return o instanceof Number ? ((Number) o).floatValue() : 0f; + } + + @Override + public double getDouble() + { + Object o = getObject(); + return o instanceof Number ? ((Number) o).doubleValue() : 0.0; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + arraySelector.inspectRuntimeShape(inspector); + } + + @Nullable + @Override + public Object getObject() + { + Object o = arraySelector.getObject(); + if (o instanceof Object[]) { + Object[] array = (Object[]) o; + if (elementNumber < array.length) { + return array[elementNumber]; + } + } + return null; + } + + @Override + public Class classOfObject() + { + return Object.class; + } + }; + } + // we are not a nested column and are being asked for a path that will never exist, so we are nil selector return NilColumnValueSelector.instance(); } @@ -398,33 +500,85 @@ public VectorObjectSelector makeVectorObjectSelector( if (holder == null) { return NilVectorSelector.create(offset); } - BaseColumn theColumn = holder.getColumn(); + BaseColumn column = holder.getColumn(); // processFromRaw is true, that means JSON_QUERY, which can return partial results, otherwise this virtual column // is JSON_VALUE which only returns literals, so we can use the nested columns value selector - if (theColumn instanceof NestedDataComplexColumn) { - final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn; + if (column instanceof NestedDataComplexColumn) { + final NestedDataComplexColumn complexColumn = (NestedDataComplexColumn) column; if (processFromRaw) { - return new RawFieldVectorObjectSelector(column.makeVectorObjectSelector(offset), parts); + return new RawFieldVectorObjectSelector(complexColumn.makeVectorObjectSelector(offset), parts); } - return column.makeVectorObjectSelector(parts, offset); + return complexColumn.makeVectorObjectSelector(parts, offset); } // not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts if (parts.isEmpty()) { ColumnCapabilities capabilities = holder.getCapabilities(); // expectedType shouldn't possibly be null if we are being asked for an object selector and the underlying column // is numeric, else we would have been asked for a value selector - Preconditions.checkArgument(expectedType != null, "Asked for a VectorObjectSelector on a numeric column, 'expectedType' must not be null"); + Preconditions.checkArgument( + expectedType != null, + "Asked for a VectorObjectSelector on a numeric column, 'expectedType' must not be null" + ); if (capabilities.isNumeric()) { return ExpressionVectorSelectors.castValueSelectorToObject( offset, this.columnName, - theColumn.makeVectorValueSelector(offset), + column.makeVectorValueSelector(offset), capabilities.toColumnType(), expectedType ); } - return theColumn.makeVectorObjectSelector(offset); + return column.makeVectorObjectSelector(offset); + } + + if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && column instanceof VariantArrayColumn) { + final VariantArrayColumn arrayColumn = (VariantArrayColumn) column; + VectorObjectSelector arraySelector = arrayColumn.makeVectorObjectSelector(offset); + final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); + if (elementNumber < 0) { + throw new IAE("Cannot make array element selector, negative array index not supported"); + } + return new VectorObjectSelector() + { + private final Object[] elements = new Object[arraySelector.getMaxVectorSize()]; + private int id = ReadableVectorInspector.NULL_ID; + + @Override + public Object[] getObjectVector() + { + if (offset.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 = offset.getId(); + } + return elements; + } + + @Override + public int getMaxVectorSize() + { + return arraySelector.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return arraySelector.getCurrentVectorSize(); + } + }; } // we are not a nested column and are being asked for a path that will never exist, so we are nil selector @@ -459,6 +613,120 @@ public VectorValueSelector makeVectorValueSelector( } return theColumn.makeVectorValueSelector(offset); } + if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantArrayColumn) { + final VariantArrayColumn arrayColumn = (VariantArrayColumn) theColumn; + VectorObjectSelector arraySelector = arrayColumn.makeVectorObjectSelector(offset); + final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); + if (elementNumber < 0) { + throw new IAE("Cannot make array element selector, negative array index not supported"); + } + + return new VectorValueSelector() + { + private final long[] longs = new long[offset.getMaxVectorSize()]; + private final double[] doubles = new double[offset.getMaxVectorSize()]; + private final float[] floats = new float[offset.getMaxVectorSize()]; + private final boolean[] nulls = new boolean[offset.getMaxVectorSize()]; + private int id = ReadableVectorInspector.NULL_ID; + + private void computeNumbers() + { + if (offset.getId() != id) { + final Object[] maybeArrays = arraySelector.getObjectVector(); + for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) { + Object maybeArray = maybeArrays[i]; + if (maybeArray instanceof Object[]) { + Object[] anArray = (Object[]) maybeArray; + if (elementNumber < anArray.length) { + if (anArray[elementNumber] instanceof Number) { + Number n = (Number) anArray[elementNumber]; + longs[i] = n.longValue(); + doubles[i] = n.doubleValue(); + floats[i] = n.floatValue(); + nulls[i] = false; + } else { + Double d = anArray[elementNumber] instanceof String + ? Doubles.tryParse((String) anArray[elementNumber]) + : null; + if (d != null) { + longs[i] = d.longValue(); + doubles[i] = d; + floats[i] = d.floatValue(); + nulls[i] = false; + } else { + nullElement(i); + } + } + } else { + nullElement(i); + } + } else { + // not an array? + nullElement(i); + } + } + id = offset.getId(); + } + } + + private void nullElement(int i) + { + longs[i] = 0L; + doubles[i] = 0L; + floats[i] = 0L; + nulls[i] = true; + } + + @Override + public long[] getLongVector() + { + if (offset.getId() != id) { + computeNumbers(); + } + return longs; + } + + @Override + public float[] getFloatVector() + { + if (offset.getId() != id) { + computeNumbers(); + } + return floats; + } + + @Override + public double[] getDoubleVector() + { + if (offset.getId() != id) { + computeNumbers(); + } + return doubles; + } + + @Nullable + @Override + public boolean[] getNullVector() + { + if (offset.getId() != id) { + computeNumbers(); + } + return nulls; + } + + @Override + public int getMaxVectorSize() + { + return arraySelector.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return arraySelector.getCurrentVectorSize(); + } + }; + } return NilVectorSelector.create(offset); } @@ -616,6 +884,9 @@ public ColumnIndexSupplier getIndexSupplier( if (parts.isEmpty()) { return holder.getIndexSupplier(); } + if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && !hasNegativeArrayIndex && theColumn instanceof VariantArrayColumn) { + return holder.getIndexSupplier(); + } return null; } @@ -625,7 +896,7 @@ public ColumnCapabilities capabilities(String columnName) if (processFromRaw) { // JSON_QUERY always returns a StructuredData return ColumnCapabilitiesImpl.createDefault() - .setType(NestedDataComplexTypeSerde.TYPE) + .setType(ColumnType.NESTED_DATA) .setHasMultipleValues(false) .setHasNulls(true); } @@ -643,7 +914,7 @@ public ColumnCapabilities capabilities(ColumnInspector inspector, String columnN if (processFromRaw) { // JSON_QUERY always returns a StructuredData return ColumnCapabilitiesImpl.createDefault() - .setType(NestedDataComplexTypeSerde.TYPE) + .setType(ColumnType.NESTED_DATA) .setHasMultipleValues(false) .setHasNulls(true); } 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 eeedef5fd21c..9d214978f8a9 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 @@ -21,7 +21,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; @@ -539,7 +539,7 @@ public void testOperatorAutoConversion() ExpressionTypeConversion.operator(ExpressionType.STRING_ARRAY, ExpressionType.STRING_ARRAY) ); - ExpressionType nested = ExpressionType.fromColumnType(NestedDataComplexTypeSerde.TYPE); + ExpressionType nested = ExpressionType.fromColumnType(ColumnType.NESTED_DATA); Assert.assertEquals( nested, ExpressionTypeConversion.operator(nested, nested) @@ -616,7 +616,7 @@ public void testFunctionAutoConversion() ExpressionType.STRING_ARRAY, ExpressionTypeConversion.function(ExpressionType.STRING_ARRAY, ExpressionType.STRING_ARRAY) ); - ExpressionType nested = ExpressionType.fromColumnType(NestedDataComplexTypeSerde.TYPE); + ExpressionType nested = ExpressionType.fromColumnType(ColumnType.NESTED_DATA); Assert.assertEquals( nested, ExpressionTypeConversion.function(nested, nested) 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 8e6f3485d46e..818e2ee48f0e 100644 --- a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java +++ b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java @@ -30,7 +30,6 @@ 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; @@ -41,9 +40,9 @@ 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.AutoTypeColumnSchema; 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; @@ -95,11 +94,11 @@ public class NestedDataTestUtils DimensionsSpec.builder() .setDimensions( Arrays.asList( - new StringDimensionSchema("dim"), - new NestedDataDimensionSchema("nest_json"), - new NestedDataDimensionSchema("nester_json"), - new NestedDataDimensionSchema("variant_json"), - new NestedDataDimensionSchema("list_json") + new AutoTypeColumnSchema("dim"), + new AutoTypeColumnSchema("nest_json"), + new AutoTypeColumnSchema("nester_json"), + new AutoTypeColumnSchema("variant_json"), + new AutoTypeColumnSchema("list_json") ) ) .build(); 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 8593f746cc1a..6e70d4cf53cf 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 @@ -41,7 +41,6 @@ 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; @@ -214,7 +213,7 @@ public void testIngestAndScanSegmentsRealtimeWithFallback() throws Exception new NestedFieldVirtualColumn( "nester", "x_0", - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, null, true, "$.x[0]", @@ -223,7 +222,7 @@ public void testIngestAndScanSegmentsRealtimeWithFallback() throws Exception new NestedFieldVirtualColumn( "nester", "y_c_1", - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, null, true, "$.y.c[1]", @@ -232,7 +231,7 @@ public void testIngestAndScanSegmentsRealtimeWithFallback() throws Exception new NestedFieldVirtualColumn( "nester", "nester_root", - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, null, true, "$.", @@ -505,7 +504,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscovery() throws Exception 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()); + Assert.assertEquals(resultsRealtime.get(0).getEvents().toString(), resultsSegments.get(0).getEvents().toString()); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java new file mode 100644 index 000000000000..33c93bb818a7 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -0,0 +1,566 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +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.query.aggregation.AggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; +import org.apache.druid.segment.nested.StructuredData; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import javax.annotation.Nonnull; +import java.util.List; +import java.util.Map; + +public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest +{ + private static final String TIME_COL = "time"; + private static final String STRING_COL = "string"; + private static final String STRING_ARRAY_COL = "string_array"; + private static final String LONG_COL = "long"; + private static final String DOUBLE_COL = "double"; + private static final String VARIANT_COL = "variant"; + private static final String NESTED_COL = "nested"; + + @BeforeClass + public static void setup() + { + NestedDataModule.registerHandlersAndSerde(); + } + + @Test + public void testKeySizeEstimation() + { + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer(); + Assert.assertEquals(0, indexer.getCardinality()); + + EncodedKeyComponent key; + // new raw value, new field, new dictionary entry + key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of("x", "foo"), false); + Assert.assertEquals(228, key.getEffectiveSizeBytes()); + Assert.assertEquals(1, indexer.getCardinality()); + // adding same value only adds estimated size of value itself + key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of("x", "foo"), false); + Assert.assertEquals(112, key.getEffectiveSizeBytes()); + Assert.assertEquals(1, indexer.getCardinality()); + // new raw value, new field, new dictionary entry + key = indexer.processRowValsToUnsortedEncodedKeyComponent(10L, false); + Assert.assertEquals(94, key.getEffectiveSizeBytes()); + Assert.assertEquals(2, indexer.getCardinality()); + // adding same value only adds estimated size of value itself + key = indexer.processRowValsToUnsortedEncodedKeyComponent(10L, false); + Assert.assertEquals(16, key.getEffectiveSizeBytes()); + Assert.assertEquals(2, indexer.getCardinality()); + // new raw value, new dictionary entry + key = indexer.processRowValsToUnsortedEncodedKeyComponent(11L, false); + Assert.assertEquals(48, key.getEffectiveSizeBytes()); + Assert.assertEquals(3, indexer.getCardinality()); + + // new raw value, new fields + key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false); + Assert.assertEquals(168, key.getEffectiveSizeBytes()); + Assert.assertEquals(6, indexer.getCardinality()); + // new raw value, re-use fields and dictionary + key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false); + Assert.assertEquals(104, key.getEffectiveSizeBytes()); + Assert.assertEquals(6, indexer.getCardinality()); + // new raw value, new fields + key = indexer.processRowValsToUnsortedEncodedKeyComponent( + ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)), + false + ); + Assert.assertEquals(166, key.getEffectiveSizeBytes()); + Assert.assertEquals(6, indexer.getCardinality()); + // new raw value + key = indexer.processRowValsToUnsortedEncodedKeyComponent( + ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)), + false + ); + Assert.assertEquals(166, key.getEffectiveSizeBytes()); + Assert.assertEquals(6, indexer.getCardinality()); + + key = indexer.processRowValsToUnsortedEncodedKeyComponent("", false); + if (NullHandling.replaceWithDefault()) { + Assert.assertEquals(0, key.getEffectiveSizeBytes()); + Assert.assertEquals(7, indexer.getCardinality()); + } else { + Assert.assertEquals(104, key.getEffectiveSizeBytes()); + Assert.assertEquals(7, indexer.getCardinality()); + } + } + + @Test + public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeExceededException + { + long minTimestamp = System.currentTimeMillis(); + IncrementalIndex index = makeIncrementalIndex(minTimestamp); + + index.add(makeInputRow(minTimestamp + 1, true, STRING_COL, "a")); + index.add(makeInputRow(minTimestamp + 2, true, STRING_COL, "b")); + index.add(makeInputRow(minTimestamp + 3, true, STRING_COL, "c")); + index.add(makeInputRow(minTimestamp + 4, true, STRING_COL, null)); + index.add(makeInputRow(minTimestamp + 5, false, STRING_COL, null)); + + IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); + Sequence cursorSequence = storageAdapter.makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.NONE, + false, + null + ); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); + List cursorList = cursorSequence.toList(); + ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("a", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("a", dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals("b", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("b", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("b", dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals("c", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("c", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("c", dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } + + @Test + public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExceededException + { + long minTimestamp = System.currentTimeMillis(); + IncrementalIndex index = makeIncrementalIndex(minTimestamp); + + index.add(makeInputRow(minTimestamp + 1, true, LONG_COL, 1L)); + index.add(makeInputRow(minTimestamp + 2, true, LONG_COL, 2L)); + index.add(makeInputRow(minTimestamp + 3, true, LONG_COL, 3L)); + index.add(makeInputRow(minTimestamp + 4, true, LONG_COL, null)); + index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); + + IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); + Sequence cursorSequence = storageAdapter.makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.NONE, + false, + null + ); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); + List cursorList = cursorSequence.toList(); + ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(1L, valueSelector.getObject()); + Assert.assertEquals(1L, valueSelector.getLong()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("1", dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(2L, valueSelector.getObject()); + Assert.assertEquals(2L, valueSelector.getLong()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("2", dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(3L, valueSelector.getObject()); + Assert.assertEquals(3L, valueSelector.getLong()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("3", dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } + + @Test + public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeExceededException + { + long minTimestamp = System.currentTimeMillis(); + IncrementalIndex index = makeIncrementalIndex(minTimestamp); + + index.add(makeInputRow(minTimestamp + 1, true, DOUBLE_COL, 1.1)); + index.add(makeInputRow(minTimestamp + 2, true, DOUBLE_COL, 2.2)); + index.add(makeInputRow(minTimestamp + 3, true, DOUBLE_COL, 3.3)); + index.add(makeInputRow(minTimestamp + 4, true, DOUBLE_COL, null)); + index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); + + IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); + Sequence cursorSequence = storageAdapter.makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.NONE, + false, + null + ); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); + List cursorList = cursorSequence.toList(); + ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(1.1, valueSelector.getObject()); + Assert.assertEquals(1.1, valueSelector.getDouble(), 0.0); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("1.1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("1.1", dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(2.2, valueSelector.getObject()); + Assert.assertEquals(2.2, valueSelector.getDouble(), 0.0); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("2.2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("2.2", dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(3.3, valueSelector.getObject()); + Assert.assertEquals(3.3, valueSelector.getDouble(), 0.0); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("3.3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("3.3", dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + + columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); + dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } + + @Test + public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws IndexSizeExceededException + { + long minTimestamp = System.currentTimeMillis(); + IncrementalIndex index = makeIncrementalIndex(minTimestamp); + + index.add(makeInputRow(minTimestamp + 1, true, STRING_ARRAY_COL, new String[]{"a"})); + index.add(makeInputRow(minTimestamp + 2, true, STRING_ARRAY_COL, new Object[]{"b", "c"})); + index.add(makeInputRow(minTimestamp + 3, true, STRING_ARRAY_COL, ImmutableList.of("d", "e"))); + index.add(makeInputRow(minTimestamp + 4, true, STRING_ARRAY_COL, null)); + index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); + + IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); + Sequence cursorSequence = storageAdapter.makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.NONE, + false, + null + ); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_ARRAY_COL, STRING_ARRAY_COL, ColumnType.STRING); + List cursorList = cursorSequence.toList(); + ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertArrayEquals(new Object[]{"a"}, (Object[]) valueSelector.getObject()); + + columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertArrayEquals(new Object[]{"b", "c"}, (Object[]) valueSelector.getObject()); + + columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertArrayEquals(new Object[]{"d", "e"}, (Object[]) valueSelector.getObject()); + + columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertNull(valueSelector.getObject()); + + columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertNull(valueSelector.getObject()); + } + + @Test + public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSizeExceededException + { + long minTimestamp = System.currentTimeMillis(); + IncrementalIndex index = makeIncrementalIndex(minTimestamp); + + index.add(makeInputRow(minTimestamp + 1, true, VARIANT_COL, "a")); + index.add(makeInputRow(minTimestamp + 2, true, VARIANT_COL, 2L)); + index.add(makeInputRow(minTimestamp + 3, true, VARIANT_COL, 3.3)); + index.add(makeInputRow(minTimestamp + 4, true, VARIANT_COL, null)); + index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); + + IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); + Sequence cursorSequence = storageAdapter.makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.NONE, + false, + null + ); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); + List cursorList = cursorSequence.toList(); + ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); + + columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + + columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertEquals(StructuredData.wrap(3.3), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + + columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertNull(valueSelector.getObject()); + + columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertNull(valueSelector.getObject()); + } + + @Test + public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExceededException + { + long minTimestamp = System.currentTimeMillis(); + IncrementalIndex index = makeIncrementalIndex(minTimestamp); + + index.add(makeInputRow(minTimestamp + 1, true, NESTED_COL, "a")); + index.add(makeInputRow(minTimestamp + 2, true, NESTED_COL, 2L)); + index.add(makeInputRow(minTimestamp + 3, true, NESTED_COL, ImmutableMap.of("x", 1.1, "y", 2L))); + index.add(makeInputRow(minTimestamp + 4, true, NESTED_COL, null)); + index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); + + IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); + Sequence cursorSequence = storageAdapter.makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.NONE, + false, + null + ); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); + List cursorList = cursorSequence.toList(); + ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); + + columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + + columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertEquals(StructuredData.wrap(ImmutableMap.of("x", 1.1, "y", 2L)), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + + columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertNull(valueSelector.getObject()); + + columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); + valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertNull(valueSelector.getObject()); + } + + @Nonnull + private static IncrementalIndex makeIncrementalIndex(long minTimestamp) + { + IncrementalIndex index = new OnheapIncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema( + minTimestamp, + new TimestampSpec(TIME_COL, "millis", null), + Granularities.NONE, + VirtualColumns.EMPTY, + DimensionsSpec.builder().useSchemaDiscovery(true).build(), + new AggregatorFactory[0], + false + ) + ) + .setMaxRowCount(1000) + .build(); + return index; + } + + private MapBasedInputRow makeInputRow( + long timestamp, + boolean explicitNull, + Object... kv + ) + { + final Map event = TestHelper.makeMap(explicitNull, kv); + event.put("time", timestamp); + return new MapBasedInputRow(timestamp, ImmutableList.copyOf(event.keySet()), event); + } +} 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 ce28ed006651..92da314dfc7c 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java @@ -94,34 +94,34 @@ public void testKeySizeEstimation() // new raw value, new fields key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false); - Assert.assertEquals(168, key.getEffectiveSizeBytes()); - Assert.assertEquals(6, indexer.getCardinality()); + Assert.assertEquals(276, key.getEffectiveSizeBytes()); + Assert.assertEquals(5, indexer.getCardinality()); // new raw value, re-use fields and dictionary key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false); - Assert.assertEquals(104, key.getEffectiveSizeBytes()); - Assert.assertEquals(6, indexer.getCardinality()); + Assert.assertEquals(56, key.getEffectiveSizeBytes()); + Assert.assertEquals(5, indexer.getCardinality()); // new raw value, new fields key = indexer.processRowValsToUnsortedEncodedKeyComponent( ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)), false ); - Assert.assertEquals(166, key.getEffectiveSizeBytes()); - Assert.assertEquals(6, indexer.getCardinality()); + Assert.assertEquals(286, key.getEffectiveSizeBytes()); + Assert.assertEquals(5, indexer.getCardinality()); // new raw value key = indexer.processRowValsToUnsortedEncodedKeyComponent( ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)), false ); - Assert.assertEquals(166, key.getEffectiveSizeBytes()); - Assert.assertEquals(6, indexer.getCardinality()); + Assert.assertEquals(118, key.getEffectiveSizeBytes()); + Assert.assertEquals(5, indexer.getCardinality()); key = indexer.processRowValsToUnsortedEncodedKeyComponent("", false); if (NullHandling.replaceWithDefault()) { Assert.assertEquals(0, key.getEffectiveSizeBytes()); - Assert.assertEquals(7, indexer.getCardinality()); + Assert.assertEquals(6, indexer.getCardinality()); } else { Assert.assertEquals(104, key.getEffectiveSizeBytes()); - Assert.assertEquals(7, indexer.getCardinality()); + Assert.assertEquals(6, indexer.getCardinality()); } } diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java index f7b47f6ee5ba..0832796c4fdb 100644 --- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java +++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java @@ -38,6 +38,7 @@ import org.apache.druid.query.aggregation.FloatSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import org.apache.druid.segment.column.CapabilitiesBasedFormat; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; @@ -240,7 +241,7 @@ public void testStringColumn() Assert.assertFalse( ColumnCapabilitiesImpl.snapshot( caps, - IndexMergerV9.DIMENSION_CAPABILITY_MERGE_LOGIC + CapabilitiesBasedFormat.DIMENSION_CAPABILITY_MERGE_LOGIC ).hasMultipleValues().isMaybeTrue() ); Assert.assertFalse(caps.hasSpatialIndexes()); @@ -274,7 +275,7 @@ public void testStringColumnWithNulls() Assert.assertFalse( ColumnCapabilitiesImpl.snapshot( caps, - IndexMergerV9.DIMENSION_CAPABILITY_MERGE_LOGIC + CapabilitiesBasedFormat.DIMENSION_CAPABILITY_MERGE_LOGIC ).hasMultipleValues().isMaybeTrue() ); Assert.assertFalse(caps.hasSpatialIndexes()); 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 ba2be32008e0..31a1afdf6dde 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 @@ -27,7 +27,6 @@ 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; @@ -39,25 +38,24 @@ import org.apache.druid.query.DefaultBitmapResultFactory; import org.apache.druid.query.filter.SelectorPredicateFactory; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.AutoTypeColumnIndexer; +import org.apache.druid.segment.AutoTypeColumnMerger; import org.apache.druid.segment.BaseProgressIndicator; 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.IndexableAdapter; import org.apache.druid.segment.ObjectColumnSelector; -import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnBuilder; -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.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.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.vector.BitmapVectorOffset; import org.apache.druid.segment.vector.NoFilterVectorOffset; import org.apache.druid.segment.vector.VectorObjectSelector; @@ -65,7 +63,6 @@ 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; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -78,6 +75,7 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -99,7 +97,8 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); - DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(new RoaringBitmapFactory()); + BitmapSerdeFactory bitmapSerdeFactory = RoaringBitmapSerdeFactory.getInstance(); + DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(bitmapSerdeFactory.getBitmapFactory()); List> data = ImmutableList.of( TestHelper.makeMap("x", 1L, "y", 1.0, "z", "a", "v", "100", "nullish", "notnull"), @@ -177,26 +176,30 @@ private SmooshedFileMapper smooshify( closer ); - NestedDataColumnIndexer indexer = new NestedDataColumnIndexer(); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer(); for (Object o : data) { indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); } - SortedMap sortedFields = new TreeMap<>(); - indexer.mergeFields(sortedFields); + SortedMap sortedFields = new TreeMap<>(); - GlobalDictionarySortedCollector globalDictionarySortedCollector = indexer.getSortedCollector(); + IndexableAdapter.NestedColumnMergable mergable = closer.register( + new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo()) + ); + SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary(); + mergable.mergeFieldsInto(sortedFields); - serializer.open(); + serializer.openDictionaryWriter(); serializer.serializeFields(sortedFields); serializer.serializeDictionaries( globalDictionarySortedCollector.getSortedStrings(), globalDictionarySortedCollector.getSortedLongs(), globalDictionarySortedCollector.getSortedDoubles(), - () -> new NestedDataColumnMerger.ArrayDictionaryMergingIterator( + () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, serializer.getGlobalLookup() ) ); + serializer.open(); SettableSelector valueSelector = new SettableSelector(); for (Object o : data) { @@ -224,12 +227,12 @@ public void testBasicFunctionality() throws IOException ColumnBuilder bob = new ColumnBuilder(); bob.setFileMapper(fileMapper); NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read( + false, baseBuffer, bob, () -> 0, - NestedDataComplexTypeSerde.OBJECT_MAPPER, - new OnlyPositionalReadsTypeStrategy<>(ColumnType.LONG.getStrategy()), - new OnlyPositionalReadsTypeStrategy<>(ColumnType.DOUBLE.getStrategy()) + bitmapSerdeFactory, + ByteOrder.nativeOrder() ); try (NestedDataComplexColumn column = (NestedDataComplexColumn) supplier.get()) { smokeTest(column); @@ -242,12 +245,12 @@ public void testArrayFunctionality() throws IOException ColumnBuilder bob = new ColumnBuilder(); bob.setFileMapper(arrayFileMapper); NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read( + false, arrayBaseBuffer, bob, () -> 0, - NestedDataComplexTypeSerde.OBJECT_MAPPER, - new OnlyPositionalReadsTypeStrategy<>(ColumnType.LONG.getStrategy()), - new OnlyPositionalReadsTypeStrategy<>(ColumnType.DOUBLE.getStrategy()) + bitmapSerdeFactory, + ByteOrder.nativeOrder() ); try (NestedDataComplexColumn column = (NestedDataComplexColumn) supplier.get()) { smokeTestArrays(column); @@ -261,17 +264,19 @@ public void testConcurrency() throws ExecutionException, InterruptedException ColumnBuilder bob = new ColumnBuilder(); bob.setFileMapper(fileMapper); NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read( + false, baseBuffer, bob, () -> 0, - NestedDataComplexTypeSerde.OBJECT_MAPPER + bitmapSerdeFactory, + ByteOrder.nativeOrder() ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); final int threads = 10; ListeningExecutorService executorService = MoreExecutors.listeningDecorator( - Execs.multiThreaded(threads, "NestedDataColumnSupplierTest-%d") + Execs.multiThreaded(threads, "StandardNestedColumnSupplierTest-%d") ); Collection> futures = new ArrayList<>(threads); final CountDownLatch threadsStartLatch = new CountDownLatch(1); @@ -297,93 +302,6 @@ public void testConcurrency() throws ExecutionException, InterruptedException Assert.assertEquals(expectedReason, failureReason.get()); } - @Test - public void testLegacyV3ReaderFormat() throws IOException - { - String columnName = "shipTo"; - String firstValue = "Cole"; - File tmpLocation = tempFolder.newFolder(); - CompressionUtils.unzip( - NestedDataColumnSupplierTest.class.getClassLoader().getResourceAsStream("nested_segment_v3/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()); - - NestedDataColumnV3 v3 = closer.register((NestedDataColumnV3) holder.getColumn()); - Assert.assertNotNull(v3); - - List path = ImmutableList.of(new NestedPathField("lastName")); - ColumnHolder nestedColumnHolder = v3.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 = v3.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)); - } - } - - @Test - public void testLegacyV4ReaderFormat() throws IOException - { - String columnName = "shipTo"; - // i accidentally didn't use same segment granularity for v3 and v4 segments... so they have different first value - String firstValue = "Beatty"; - File tmpLocation = tempFolder.newFolder(); - CompressionUtils.unzip( - NestedDataColumnSupplierTest.class.getClassLoader().getResourceAsStream("nested_segment_v4/index.zip"), - tmpLocation - ); - try (Closer closer = Closer.create()) { - QueryableIndex theIndex = closer.register(TestHelper.getTestIndexIO().loadIndex(tmpLocation)); - ColumnHolder holder = theIndex.getColumnHolder(columnName); - Assert.assertNotNull(holder); - Assert.assertEquals(NestedDataComplexTypeSerde.TYPE, holder.getCapabilities().toColumnType()); - - NestedDataColumnV4 v4 = closer.register((NestedDataColumnV4) holder.getColumn()); - Assert.assertNotNull(v4); - - List path = ImmutableList.of(new NestedPathField("lastName")); - ColumnHolder nestedColumnHolder = v4.getColumnHolder(path); - Assert.assertNotNull(nestedColumnHolder); - Assert.assertEquals(ColumnType.STRING, nestedColumnHolder.getCapabilities().toColumnType()); - 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()); @@ -799,7 +717,7 @@ private void testPath( } } - private static class SettableSelector extends ObjectColumnSelector + static class SettableSelector extends ObjectColumnSelector { private StructuredData data; @@ -827,56 +745,4 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) } } - - private static class OnlyPositionalReadsTypeStrategy implements TypeStrategy - { - private final TypeStrategy delegate; - - private OnlyPositionalReadsTypeStrategy(TypeStrategy delegate) - { - this.delegate = delegate; - } - - @Override - public int estimateSizeBytes(T value) - { - return delegate.estimateSizeBytes(value); - } - - @Override - public T read(ByteBuffer buffer) - { - throw new IllegalStateException("non-positional read"); - } - - @Override - public boolean readRetainsBufferReference() - { - return delegate.readRetainsBufferReference(); - } - - @Override - public int write(ByteBuffer buffer, T value, int maxSizeBytes) - { - return delegate.write(buffer, value, maxSizeBytes); - } - - @Override - public T read(ByteBuffer buffer, int offset) - { - return delegate.read(buffer, offset); - } - - @Override - public int write(ByteBuffer buffer, int offset, T value, int maxSizeBytes) - { - return delegate.write(buffer, offset, value, maxSizeBytes); - } - - @Override - public int compare(Object o1, Object o2) - { - return delegate.compare(o1, o2); - } - } } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java new file mode 100644 index 000000000000..d335a041b13f --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java @@ -0,0 +1,614 @@ +/* + * 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.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +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.RoaringBitmapFactory; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; +import org.apache.druid.query.DefaultBitmapResultFactory; +import org.apache.druid.query.filter.SelectorPredicateFactory; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.BaseProgressIndicator; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.IndexableAdapter; +import org.apache.druid.segment.NestedDataColumnIndexer; +import org.apache.druid.segment.ObjectColumnSelector; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.SimpleAscendingOffset; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.column.ColumnBuilder; +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.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.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.utils.CompressionUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicReference; + +public class NestedDataColumnSupplierV4Test extends InitializedNullHandlingTest +{ + private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); + + private static final String NO_MATCH = "no"; + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(new RoaringBitmapFactory()); + + List> data = ImmutableList.of( + TestHelper.makeMap("x", 1L, "y", 1.0, "z", "a", "v", "100", "nullish", "notnull"), + TestHelper.makeMap("y", 3.0, "z", "d", "v", 1000L, "nullish", null), + TestHelper.makeMap("x", 5L, "y", 5.0, "z", "b", "nullish", ""), + TestHelper.makeMap("x", 3L, "y", 4.0, "z", "c", "v", 3000.333, "nullish", "null"), + TestHelper.makeMap("x", 2L, "v", "40000"), + TestHelper.makeMap("x", 4L, "y", 2.0, "z", "e", "v", 11111L, "nullish", null) + ); + + List> arrayTestData = ImmutableList.of( + TestHelper.makeMap("s", new Object[]{"a", "b", "c"}, "l", new Object[]{1L, 2L, 3L}, "d", new Object[]{1.1, 2.2}), + TestHelper.makeMap( + "s", + new Object[]{null, "b", "c"}, + "l", + new Object[]{1L, 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() + { + NestedDataModule.registerHandlersAndSerde(); + } + + @Before + public void setup() throws IOException + { + final String fileNameBase = "test"; + final String arrayFileNameBase = "array"; + fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), data); + baseBuffer = fileMapper.mapFile(fileNameBase); + arrayFileMapper = smooshify(arrayFileNameBase, tempFolder.newFolder(), arrayTestData); + arrayBaseBuffer = arrayFileMapper.mapFile(arrayFileNameBase); + } + + private SmooshedFileMapper smooshify( + String fileNameBase, + File tmpFile, + List> data + ) + throws IOException + { + SegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance(); + try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { + NestedDataColumnSerializerV4 serializer = new NestedDataColumnSerializerV4( + fileNameBase, + new IndexSpec(), + writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), + new BaseProgressIndicator(), + closer + ); + + NestedDataColumnIndexer indexer = new NestedDataColumnIndexer(); + for (Object o : data) { + indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); + } + SortedMap sortedFields = new TreeMap<>(); + + IndexableAdapter.NestedColumnMergable mergable = closer.register( + new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo()) + ); + SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary(); + mergable.mergeFieldsInto(sortedFields); + + serializer.open(); + serializer.serializeFields(sortedFields); + serializer.serializeDictionaries( + globalDictionarySortedCollector.getSortedStrings(), + globalDictionarySortedCollector.getSortedLongs(), + globalDictionarySortedCollector.getSortedDoubles() + ); + + SettableSelector valueSelector = new SettableSelector(); + for (Object o : data) { + valueSelector.setObject(StructuredData.wrap(o)); + serializer.serialize(valueSelector); + } + + try (SmooshedWriter writer = smoosher.addWithSmooshedWriter(fileNameBase, serializer.getSerializedSize())) { + serializer.writeTo(writer, smoosher); + } + smoosher.close(); + return closer.register(SmooshedFileMapper.load(tmpFile)); + } + } + + @After + public void teardown() throws IOException + { + closer.close(); + } + + @Test + public void testBasicFunctionality() throws IOException + { + ColumnBuilder bob = new ColumnBuilder(); + bob.setFileMapper(fileMapper); + NestedDataColumnSupplierV4 supplier = NestedDataColumnSupplierV4.read( + baseBuffer, + bob, + () -> 0, + NestedDataComplexTypeSerde.OBJECT_MAPPER, + new OnlyPositionalReadsTypeStrategy<>(ColumnType.LONG.getStrategy()), + new OnlyPositionalReadsTypeStrategy<>(ColumnType.DOUBLE.getStrategy()) + ); + try (NestedDataComplexColumn column = (NestedDataComplexColumn) supplier.get()) { + smokeTest(column); + } + } + + @Test + public void testConcurrency() throws ExecutionException, InterruptedException + { + // if this test ever starts being to be a flake, there might be thread safety issues + ColumnBuilder bob = new ColumnBuilder(); + bob.setFileMapper(fileMapper); + NestedDataColumnSupplierV4 supplier = NestedDataColumnSupplierV4.read( + baseBuffer, + bob, + () -> 0, + NestedDataComplexTypeSerde.OBJECT_MAPPER + ); + final String expectedReason = "none"; + final AtomicReference failureReason = new AtomicReference<>(expectedReason); + + final int threads = 10; + ListeningExecutorService executorService = MoreExecutors.listeningDecorator( + Execs.multiThreaded(threads, "NestedDataColumnSupplierTest-%d") + ); + Collection> futures = new ArrayList<>(threads); + final CountDownLatch threadsStartLatch = new CountDownLatch(1); + for (int i = 0; i < threads; ++i) { + futures.add( + executorService.submit(() -> { + try { + threadsStartLatch.await(); + for (int iter = 0; iter < 5000; iter++) { + try (NestedDataComplexColumn column = (NestedDataComplexColumn) supplier.get()) { + smokeTest(column); + } + } + } + catch (Throwable ex) { + failureReason.set(ex.getMessage()); + } + }) + ); + } + threadsStartLatch.countDown(); + Futures.allAsList(futures).get(); + Assert.assertEquals(expectedReason, failureReason.get()); + } + + @Test + public void testLegacyV3ReaderFormat() throws IOException + { + String columnName = "shipTo"; + String firstValue = "Cole"; + File tmpLocation = tempFolder.newFolder(); + CompressionUtils.unzip( + NestedDataColumnSupplierV4Test.class.getClassLoader().getResourceAsStream("nested_segment_v3/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(ColumnType.NESTED_DATA, holder.getCapabilities().toColumnType()); + + NestedDataColumnV3 v3 = closer.register((NestedDataColumnV3) holder.getColumn()); + Assert.assertNotNull(v3); + + List path = ImmutableList.of(new NestedPathField("lastName")); + ColumnHolder nestedColumnHolder = v3.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 = v3.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)); + } + } + + @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( + NestedDataColumnSupplierV4Test.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(ColumnType.NESTED_DATA, holder.getCapabilities().toColumnType()); + + NestedDataColumnV4 v4 = closer.register((NestedDataColumnV4) holder.getColumn()); + Assert.assertNotNull(v4); + + List path = ImmutableList.of(new NestedPathField("lastName")); + ColumnHolder nestedColumnHolder = v4.getColumnHolder(path); + Assert.assertNotNull(nestedColumnHolder); + Assert.assertEquals(ColumnType.STRING, nestedColumnHolder.getCapabilities().toColumnType()); + 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()); + ColumnValueSelector rawSelector = column.makeColumnValueSelector(offset); + + final List xPath = NestedPathFinder.parseJsonPath("$.x"); + Assert.assertEquals(ImmutableSet.of(ColumnType.LONG), column.getColumnTypes(xPath)); + Assert.assertEquals(ColumnType.LONG, column.getColumnHolder(xPath).getCapabilities().toColumnType()); + ColumnValueSelector xSelector = column.makeColumnValueSelector(xPath, offset); + DimensionSelector xDimSelector = column.makeDimensionSelector(xPath, offset, null); + ColumnIndexSupplier xIndexSupplier = column.getColumnIndexSupplier(xPath); + Assert.assertNotNull(xIndexSupplier); + StringValueSetIndex xValueIndex = xIndexSupplier.as(StringValueSetIndex.class); + DruidPredicateIndex xPredicateIndex = xIndexSupplier.as(DruidPredicateIndex.class); + NullValueIndex xNulls = xIndexSupplier.as(NullValueIndex.class); + + final List yPath = NestedPathFinder.parseJsonPath("$.y"); + Assert.assertEquals(ImmutableSet.of(ColumnType.DOUBLE), column.getColumnTypes(yPath)); + Assert.assertEquals(ColumnType.DOUBLE, column.getColumnHolder(yPath).getCapabilities().toColumnType()); + ColumnValueSelector ySelector = column.makeColumnValueSelector(yPath, offset); + DimensionSelector yDimSelector = column.makeDimensionSelector(yPath, offset, null); + ColumnIndexSupplier yIndexSupplier = column.getColumnIndexSupplier(yPath); + Assert.assertNotNull(yIndexSupplier); + StringValueSetIndex yValueIndex = yIndexSupplier.as(StringValueSetIndex.class); + DruidPredicateIndex yPredicateIndex = yIndexSupplier.as(DruidPredicateIndex.class); + NullValueIndex yNulls = yIndexSupplier.as(NullValueIndex.class); + + final List zPath = NestedPathFinder.parseJsonPath("$.z"); + Assert.assertEquals(ImmutableSet.of(ColumnType.STRING), column.getColumnTypes(zPath)); + Assert.assertEquals(ColumnType.STRING, column.getColumnHolder(zPath).getCapabilities().toColumnType()); + ColumnValueSelector zSelector = column.makeColumnValueSelector(zPath, offset); + DimensionSelector zDimSelector = column.makeDimensionSelector(zPath, offset, null); + ColumnIndexSupplier zIndexSupplier = column.getColumnIndexSupplier(zPath); + Assert.assertNotNull(zIndexSupplier); + StringValueSetIndex zValueIndex = zIndexSupplier.as(StringValueSetIndex.class); + DruidPredicateIndex zPredicateIndex = zIndexSupplier.as(DruidPredicateIndex.class); + NullValueIndex zNulls = zIndexSupplier.as(NullValueIndex.class); + + final List vPath = NestedPathFinder.parseJsonPath("$.v"); + Assert.assertEquals( + ImmutableSet.of(ColumnType.STRING, ColumnType.LONG, ColumnType.DOUBLE), + column.getColumnTypes(vPath) + ); + Assert.assertEquals(ColumnType.STRING, column.getColumnHolder(vPath).getCapabilities().toColumnType()); + ColumnValueSelector vSelector = column.makeColumnValueSelector(vPath, offset); + DimensionSelector vDimSelector = column.makeDimensionSelector(vPath, offset, null); + ColumnIndexSupplier vIndexSupplier = column.getColumnIndexSupplier(vPath); + Assert.assertNotNull(vIndexSupplier); + StringValueSetIndex vValueIndex = vIndexSupplier.as(StringValueSetIndex.class); + DruidPredicateIndex vPredicateIndex = vIndexSupplier.as(DruidPredicateIndex.class); + NullValueIndex vNulls = vIndexSupplier.as(NullValueIndex.class); + + final List nullishPath = NestedPathFinder.parseJsonPath("$.nullish"); + Assert.assertEquals(ImmutableSet.of(ColumnType.STRING), column.getColumnTypes(nullishPath)); + Assert.assertEquals(ColumnType.STRING, column.getColumnHolder(nullishPath).getCapabilities().toColumnType()); + ColumnValueSelector nullishSelector = column.makeColumnValueSelector(nullishPath, offset); + DimensionSelector nullishDimSelector = column.makeDimensionSelector(nullishPath, offset, null); + ColumnIndexSupplier nullishIndexSupplier = column.getColumnIndexSupplier(nullishPath); + Assert.assertNotNull(nullishIndexSupplier); + StringValueSetIndex nullishValueIndex = nullishIndexSupplier.as(StringValueSetIndex.class); + DruidPredicateIndex nullishPredicateIndex = nullishIndexSupplier.as(DruidPredicateIndex.class); + NullValueIndex nullishNulls = nullishIndexSupplier.as(NullValueIndex.class); + + Assert.assertEquals(ImmutableList.of(nullishPath, vPath, xPath, yPath, zPath), column.getNestedFields()); + + for (int i = 0; i < data.size(); i++) { + Map row = data.get(i); + Assert.assertEquals( + JSON_MAPPER.writeValueAsString(row), + JSON_MAPPER.writeValueAsString(StructuredData.unwrap(rawSelector.getObject())) + ); + + testPath(row, i, "v", vSelector, vDimSelector, vValueIndex, vPredicateIndex, vNulls, null); + 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 + ); + + offset.increment(); + } + } + + private void testPath( + Map row, + int rowNumber, + String path, + ColumnValueSelector valueSelector, + DimensionSelector dimSelector, + StringValueSetIndex valueSetIndex, + DruidPredicateIndex predicateIndex, + NullValueIndex nullValueIndex, + @Nullable ColumnType singleType + ) + { + final Object inputValue = row.get(path); + // in default value mode, even though the input row had an empty string, the selector spits out null, so we want + // to take the null checking path + final boolean isStringAndNullEquivalent = + inputValue instanceof String && NullHandling.isNullOrEquivalent((String) inputValue); + + if (row.containsKey(path) && inputValue != null && !isStringAndNullEquivalent) { + 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()); + } + + final String theString = String.valueOf(inputValue); + Assert.assertEquals(theString, dimSelector.getObject()); + String dimSelectorLookupVal = dimSelector.lookupName(dimSelector.getRow().get(0)); + Assert.assertEquals(theString, dimSelectorLookupVal); + 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.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(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber)); + + Assert.assertTrue(dimSelector.makeValueMatcher(theString).matches()); + Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); + Assert.assertTrue(dimSelector.makeValueMatcher(x -> Objects.equals(x, theString)).matches()); + Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches()); + } else { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(path, valueSelector.isNull()); + + Assert.assertEquals(0, dimSelector.getRow().get(0)); + Assert.assertNull(dimSelector.getObject()); + Assert.assertNull(dimSelector.lookupName(dimSelector.getRow().get(0))); + + Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertTrue(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.assertTrue(dimSelector.makeValueMatcher((String) null).matches()); + Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); + Assert.assertTrue(dimSelector.makeValueMatcher(x -> x == null).matches()); + Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches()); + } + } + + private static class SettableSelector extends ObjectColumnSelector + { + private StructuredData data; + + public void setObject(StructuredData o) + { + this.data = o; + } + + @Nullable + @Override + public StructuredData getObject() + { + return data; + } + + @Override + public Class classOfObject() + { + return StructuredData.class; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + } + + private static class OnlyPositionalReadsTypeStrategy implements TypeStrategy + { + private final TypeStrategy delegate; + + private OnlyPositionalReadsTypeStrategy(TypeStrategy delegate) + { + this.delegate = delegate; + } + + @Override + public int estimateSizeBytes(T value) + { + return delegate.estimateSizeBytes(value); + } + + @Override + public T read(ByteBuffer buffer) + { + throw new IllegalStateException("non-positional read"); + } + + @Override + public boolean readRetainsBufferReference() + { + return delegate.readRetainsBufferReference(); + } + + @Override + public int write(ByteBuffer buffer, T value, int maxSizeBytes) + { + return delegate.write(buffer, value, maxSizeBytes); + } + + @Override + public T read(ByteBuffer buffer, int offset) + { + return delegate.read(buffer, offset); + } + + @Override + public int write(ByteBuffer buffer, int offset, T value, int maxSizeBytes) + { + return delegate.write(buffer, offset, value, maxSizeBytes); + } + + @Override + public int compare(Object o1, Object o2) + { + return delegate.compare(o1, o2); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java index 13e5e2b535a6..27dfd79bbcd5 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java @@ -1351,7 +1351,7 @@ public void testEnsureNoImproperSelectionFromAdjustedGlobals() throws IOExceptio FixedIndexedWriter localDictionaryWriter = new FixedIndexedWriter<>( new OnHeapMemorySegmentWriteOutMedium(), - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES, true @@ -1391,7 +1391,7 @@ public void testEnsureNoImproperSelectionFromAdjustedGlobals() throws IOExceptio Supplier> dictionarySupplier = FixedIndexed.read( localDictionaryBuffer, - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES ); @@ -1399,10 +1399,10 @@ public void testEnsureNoImproperSelectionFromAdjustedGlobals() throws IOExceptio GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); NestedFieldColumnIndexSupplier indexSupplier = new NestedFieldColumnIndexSupplier<>( - new NestedFieldTypeInfo.TypeSet( - new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING) - .add(ColumnType.LONG) - .getByteValue() + new FieldTypeInfo.TypeSet( + new FieldTypeInfo.MutableTypeSet().add(ColumnType.STRING) + .add(ColumnType.LONG) + .getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1447,7 +1447,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeStringSupplier() throws FixedIndexedWriter localDictionaryWriter = new FixedIndexedWriter<>( new OnHeapMemorySegmentWriteOutMedium(), - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES, true @@ -1491,7 +1491,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeStringSupplier() throws Supplier> dictionarySupplier = FixedIndexed.read( localDictionaryBuffer, - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES ); @@ -1499,8 +1499,8 @@ private NestedFieldColumnIndexSupplier makeSingleTypeStringSupplier() throws GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); return new NestedFieldColumnIndexSupplier<>( - new NestedFieldTypeInfo.TypeSet( - new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue() + new FieldTypeInfo.TypeSet( + new FieldTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1520,7 +1520,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeStringWithNullsSupplier( FixedIndexedWriter localDictionaryWriter = new FixedIndexedWriter<>( new OnHeapMemorySegmentWriteOutMedium(), - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES, true @@ -1567,7 +1567,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeStringWithNullsSupplier( Supplier> dictionarySupplier = FixedIndexed.read( localDictionaryBuffer, - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES ); @@ -1575,8 +1575,8 @@ private NestedFieldColumnIndexSupplier makeSingleTypeStringWithNullsSupplier( GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); return new NestedFieldColumnIndexSupplier<>( - new NestedFieldTypeInfo.TypeSet( - new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue() + new FieldTypeInfo.TypeSet( + new FieldTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1596,7 +1596,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplier() throws IO FixedIndexedWriter localDictionaryWriter = new FixedIndexedWriter<>( new OnHeapMemorySegmentWriteOutMedium(), - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES, true @@ -1640,7 +1640,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplier() throws IO Supplier> dictionarySupplier = FixedIndexed.read( localDictionaryBuffer, - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES ); @@ -1648,8 +1648,8 @@ private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplier() throws IO GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); return new NestedFieldColumnIndexSupplier<>( - new NestedFieldTypeInfo.TypeSet( - new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue() + new FieldTypeInfo.TypeSet( + new FieldTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1669,7 +1669,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplierWithNull() t FixedIndexedWriter localDictionaryWriter = new FixedIndexedWriter<>( new OnHeapMemorySegmentWriteOutMedium(), - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES, true @@ -1717,7 +1717,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplierWithNull() t Supplier> dictionarySupplier = FixedIndexed.read( localDictionaryBuffer, - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES ); @@ -1725,8 +1725,8 @@ private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplierWithNull() t GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); return new NestedFieldColumnIndexSupplier<>( - new NestedFieldTypeInfo.TypeSet( - new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue() + new FieldTypeInfo.TypeSet( + new FieldTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1746,7 +1746,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplier() throws FixedIndexedWriter localDictionaryWriter = new FixedIndexedWriter<>( new OnHeapMemorySegmentWriteOutMedium(), - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES, true @@ -1790,7 +1790,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplier() throws Supplier> dictionarySupplier = FixedIndexed.read( localDictionaryBuffer, - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES ); @@ -1798,8 +1798,8 @@ private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplier() throws GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); return new NestedFieldColumnIndexSupplier<>( - new NestedFieldTypeInfo.TypeSet( - new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue() + new FieldTypeInfo.TypeSet( + new FieldTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1819,7 +1819,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplierWithNull() FixedIndexedWriter localDictionaryWriter = new FixedIndexedWriter<>( new OnHeapMemorySegmentWriteOutMedium(), - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES, true @@ -1867,7 +1867,7 @@ private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplierWithNull() Supplier> dictionarySupplier = FixedIndexed.read( localDictionaryBuffer, - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES ); @@ -1875,8 +1875,8 @@ private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplierWithNull() GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); return new NestedFieldColumnIndexSupplier<>( - new NestedFieldTypeInfo.TypeSet( - new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue() + new FieldTypeInfo.TypeSet( + new FieldTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, @@ -1896,7 +1896,7 @@ private NestedFieldColumnIndexSupplier makeVariantSupplierWithNull() throws I FixedIndexedWriter localDictionaryWriter = new FixedIndexedWriter<>( new OnHeapMemorySegmentWriteOutMedium(), - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES, true @@ -1952,7 +1952,7 @@ private NestedFieldColumnIndexSupplier makeVariantSupplierWithNull() throws I Supplier> dictionarySupplier = FixedIndexed.read( localDictionaryBuffer, - NestedDataColumnSerializer.INT_TYPE_STRATEGY, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, ByteOrder.nativeOrder(), Integer.BYTES ); @@ -1960,11 +1960,11 @@ private NestedFieldColumnIndexSupplier makeVariantSupplierWithNull() throws I GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); return new NestedFieldColumnIndexSupplier<>( - new NestedFieldTypeInfo.TypeSet( - new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING) - .add(ColumnType.LONG) - .add(ColumnType.DOUBLE) - .getByteValue() + new FieldTypeInfo.TypeSet( + new FieldTypeInfo.MutableTypeSet().add(ColumnType.STRING) + .add(ColumnType.LONG) + .add(ColumnType.DOUBLE) + .getByteValue() ), roaringFactory.getBitmapFactory(), bitmaps, diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java index aa100cf7afd2..600c687b6d8a 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java @@ -75,11 +75,11 @@ public void testMultiType() throws IOException @Test public void testEqualsAndHashCode() { - EqualsVerifier.forClass(NestedFieldTypeInfo.TypeSet.class) + EqualsVerifier.forClass(FieldTypeInfo.TypeSet.class) .usingGetClass() .verify(); - EqualsVerifier.forClass(NestedFieldTypeInfo.MutableTypeSet.class) + EqualsVerifier.forClass(FieldTypeInfo.MutableTypeSet.class) .suppress(Warning.NONFINAL_FIELDS) .usingGetClass() .verify(); @@ -87,60 +87,60 @@ public void testEqualsAndHashCode() private void testSingleType(ColumnType columnType) throws IOException { - NestedFieldTypeInfo.MutableTypeSet typeSet = new NestedFieldTypeInfo.MutableTypeSet(); + FieldTypeInfo.MutableTypeSet typeSet = new FieldTypeInfo.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())); + Assert.assertEquals(ImmutableSet.of(columnType), FieldTypeInfo.convertToSet(typeSet.getByteValue())); writeTypeSet(typeSet); - NestedFieldTypeInfo info = new NestedFieldTypeInfo(BUFFER); + FieldTypeInfo info = new FieldTypeInfo(BUFFER); Assert.assertEquals(0, BUFFER.position()); - NestedFieldTypeInfo.TypeSet roundTrip = info.getTypes(0); + FieldTypeInfo.TypeSet roundTrip = info.getTypes(0); Assert.assertEquals(columnType, roundTrip.getSingleType()); - NestedFieldTypeInfo info2 = NestedFieldTypeInfo.read(BUFFER, 1); + FieldTypeInfo info2 = FieldTypeInfo.read(BUFFER, 1); Assert.assertEquals(info.getTypes(0), info2.getTypes(0)); Assert.assertEquals(1, BUFFER.position()); } private void testMultiType(Set columnTypes) throws IOException { - NestedFieldTypeInfo.MutableTypeSet typeSet = new NestedFieldTypeInfo.MutableTypeSet(); + FieldTypeInfo.MutableTypeSet typeSet = new FieldTypeInfo.MutableTypeSet(); Assert.assertNull(typeSet.getSingleType()); Assert.assertTrue(typeSet.isEmpty()); - NestedFieldTypeInfo.MutableTypeSet merge = new NestedFieldTypeInfo.MutableTypeSet(); + FieldTypeInfo.MutableTypeSet merge = new FieldTypeInfo.MutableTypeSet(); for (ColumnType columnType : columnTypes) { typeSet.add(columnType); - merge.merge(new NestedFieldTypeInfo.MutableTypeSet().add(columnType).getByteValue()); + merge.merge(new FieldTypeInfo.MutableTypeSet().add(columnType).getByteValue()); } Assert.assertEquals(merge.getByteValue(), typeSet.getByteValue()); Assert.assertNull(typeSet.getSingleType()); - Assert.assertEquals(columnTypes, NestedFieldTypeInfo.convertToSet(typeSet.getByteValue())); + Assert.assertEquals(columnTypes, FieldTypeInfo.convertToSet(typeSet.getByteValue())); writeTypeSet(typeSet); - NestedFieldTypeInfo info = new NestedFieldTypeInfo(BUFFER); + FieldTypeInfo info = new FieldTypeInfo(BUFFER); Assert.assertEquals(0, BUFFER.position()); - NestedFieldTypeInfo.TypeSet roundTrip = info.getTypes(0); + FieldTypeInfo.TypeSet roundTrip = info.getTypes(0); Assert.assertNull(roundTrip.getSingleType()); - Assert.assertEquals(columnTypes, NestedFieldTypeInfo.convertToSet(roundTrip.getByteValue())); + Assert.assertEquals(columnTypes, FieldTypeInfo.convertToSet(roundTrip.getByteValue())); - NestedFieldTypeInfo info2 = NestedFieldTypeInfo.read(BUFFER, 1); + FieldTypeInfo info2 = FieldTypeInfo.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 + private static void writeTypeSet(FieldTypeInfo.MutableTypeSet typeSet) throws IOException { BUFFER.position(0); - NestedFieldTypeInfo.Writer writer = new NestedFieldTypeInfo.Writer(new OnHeapMemorySegmentWriteOutMedium()); + FieldTypeInfo.Writer writer = new FieldTypeInfo.Writer(new OnHeapMemorySegmentWriteOutMedium()); writer.open(); writer.write(typeSet); Assert.assertEquals(1, writer.getSerializedSize()); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java new file mode 100644 index 000000000000..bf2ec04890de --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java @@ -0,0 +1,296 @@ +/* + * 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.ImmutableMap; +import com.google.common.collect.ImmutableSet; +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.guice.NestedDataModule; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; +import org.apache.druid.query.DefaultBitmapResultFactory; +import org.apache.druid.query.filter.SelectorPredicateFactory; +import org.apache.druid.segment.AutoTypeColumnIndexer; +import org.apache.druid.segment.AutoTypeColumnMerger; +import org.apache.druid.segment.BaseProgressIndicator; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.IndexableAdapter; +import org.apache.druid.segment.SimpleAscendingOffset; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.DruidPredicateIndex; +import org.apache.druid.segment.column.NullValueIndex; +import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicReference; + +public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest +{ + private static final String NO_MATCH = "no"; + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + BitmapSerdeFactory bitmapSerdeFactory = RoaringBitmapSerdeFactory.getInstance(); + DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(bitmapSerdeFactory.getBitmapFactory()); + + List data = Arrays.asList( + 1.0, + 0.0, + null, + 2.0, + 3.3, + 9.9 + ); + + Closer closer = Closer.create(); + + SmooshedFileMapper fileMapper; + + ByteBuffer baseBuffer; + + @BeforeClass + public static void staticSetup() + { + NestedDataModule.registerHandlersAndSerde(); + } + + @Before + public void setup() throws IOException + { + final String fileNameBase = "test"; + fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), data); + baseBuffer = fileMapper.mapFile(fileNameBase); + } + + private SmooshedFileMapper smooshify( + String fileNameBase, + File tmpFile, + List data + ) + throws IOException + { + SegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance(); + try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { + ScalarDoubleColumnSerializer serializer = new ScalarDoubleColumnSerializer( + fileNameBase, + new IndexSpec(), + writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), + new BaseProgressIndicator(), + closer + ); + + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer(); + for (Object o : data) { + indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); + } + SortedMap sortedFields = new TreeMap<>(); + + IndexableAdapter.NestedColumnMergable mergable = closer.register( + new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo()) + ); + SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary(); + mergable.mergeFieldsInto(sortedFields); + + serializer.openDictionaryWriter(); + serializer.serializeDictionaries( + globalDictionarySortedCollector.getSortedStrings(), + globalDictionarySortedCollector.getSortedLongs(), + globalDictionarySortedCollector.getSortedDoubles(), + () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( + new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, + serializer.getGlobalLookup() + ) + ); + serializer.open(); + + NestedDataColumnSupplierTest.SettableSelector valueSelector = new NestedDataColumnSupplierTest.SettableSelector(); + for (Object o : data) { + valueSelector.setObject(StructuredData.wrap(o)); + serializer.serialize(valueSelector); + } + + try (SmooshedWriter writer = smoosher.addWithSmooshedWriter(fileNameBase, serializer.getSerializedSize())) { + serializer.writeTo(writer, smoosher); + } + smoosher.close(); + return closer.register(SmooshedFileMapper.load(tmpFile)); + } + } + + @After + public void teardown() throws IOException + { + closer.close(); + } + + @Test + public void testBasicFunctionality() + { + ColumnBuilder bob = new ColumnBuilder(); + bob.setFileMapper(fileMapper); + ScalarDoubleColumnAndIndexSupplier supplier = ScalarDoubleColumnAndIndexSupplier.read( + ByteOrder.nativeOrder(), + bitmapSerdeFactory, + baseBuffer, + bob + ); + try (ScalarDoubleColumn column = (ScalarDoubleColumn) supplier.get()) { + smokeTest(supplier, column); + } + } + + @Test + public void testConcurrency() throws ExecutionException, InterruptedException + { + // if this test ever starts being to be a flake, there might be thread safety issues + ColumnBuilder bob = new ColumnBuilder(); + bob.setFileMapper(fileMapper); + ScalarDoubleColumnAndIndexSupplier supplier = ScalarDoubleColumnAndIndexSupplier.read( + ByteOrder.nativeOrder(), + bitmapSerdeFactory, + baseBuffer, + bob + ); + final String expectedReason = "none"; + final AtomicReference failureReason = new AtomicReference<>(expectedReason); + + final int threads = 10; + ListeningExecutorService executorService = MoreExecutors.listeningDecorator( + Execs.multiThreaded(threads, "StandardNestedColumnSupplierTest-%d") + ); + Collection> futures = new ArrayList<>(threads); + final CountDownLatch threadsStartLatch = new CountDownLatch(1); + for (int i = 0; i < threads; ++i) { + futures.add( + executorService.submit(() -> { + try { + threadsStartLatch.await(); + for (int iter = 0; iter < 5000; iter++) { + try (ScalarDoubleColumn column = (ScalarDoubleColumn) supplier.get()) { + smokeTest(supplier, column); + } + } + } + catch (Throwable ex) { + failureReason.set(ex.getMessage()); + } + }) + ); + } + threadsStartLatch.countDown(); + Futures.allAsList(futures).get(); + Assert.assertEquals(expectedReason, failureReason.get()); + } + + private void smokeTest(ScalarDoubleColumnAndIndexSupplier supplier, ScalarDoubleColumn column) + { + SimpleAscendingOffset offset = new SimpleAscendingOffset(data.size()); + ColumnValueSelector valueSelector = column.makeColumnValueSelector(offset); + + StringValueSetIndex valueSetIndex = supplier.as(StringValueSetIndex.class); + DruidPredicateIndex predicateIndex = supplier.as(DruidPredicateIndex.class); + NullValueIndex nullValueIndex = supplier.as(NullValueIndex.class); + + SortedMap fields = column.getFieldTypeInfo(); + Assert.assertEquals( + ImmutableMap.of(NestedPathFinder.JSON_PATH_ROOT, new FieldTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE)), + fields + ); + + for (int i = 0; i < data.size(); i++) { + Double row = data.get(i); + + // in default value mode, even though the input row had an empty string, the selector spits out null, so we want + // to take the null checking path + + if (row != null) { + Assert.assertEquals(row, valueSelector.getObject()); + Assert.assertEquals((double) row, valueSelector.getDouble(), 0.0); + Assert.assertFalse(valueSelector.isNull()); + + Assert.assertTrue(valueSetIndex.forValue(String.valueOf(row)).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(String.valueOf(row)))) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(String.valueOf(row))) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH))) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + + } else { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + + Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) + .computeBitmapResult(resultFactory) + .get(i)); + } + + offset.increment(); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java new file mode 100644 index 000000000000..eb834da77e4d --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java @@ -0,0 +1,296 @@ +/* + * 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.ImmutableMap; +import com.google.common.collect.ImmutableSet; +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.guice.NestedDataModule; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; +import org.apache.druid.query.DefaultBitmapResultFactory; +import org.apache.druid.query.filter.SelectorPredicateFactory; +import org.apache.druid.segment.AutoTypeColumnIndexer; +import org.apache.druid.segment.AutoTypeColumnMerger; +import org.apache.druid.segment.BaseProgressIndicator; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.IndexableAdapter; +import org.apache.druid.segment.SimpleAscendingOffset; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.DruidPredicateIndex; +import org.apache.druid.segment.column.NullValueIndex; +import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicReference; + +public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest +{ + private static final String NO_MATCH = "no"; + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + BitmapSerdeFactory bitmapSerdeFactory = RoaringBitmapSerdeFactory.getInstance(); + DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(bitmapSerdeFactory.getBitmapFactory()); + + List data = Arrays.asList( + 1L, + 0L, + null, + 2L, + 3L, + 4L + ); + + Closer closer = Closer.create(); + + SmooshedFileMapper fileMapper; + + ByteBuffer baseBuffer; + + @BeforeClass + public static void staticSetup() + { + NestedDataModule.registerHandlersAndSerde(); + } + + @Before + public void setup() throws IOException + { + final String fileNameBase = "test"; + fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), data); + baseBuffer = fileMapper.mapFile(fileNameBase); + } + + private SmooshedFileMapper smooshify( + String fileNameBase, + File tmpFile, + List data + ) + throws IOException + { + SegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance(); + try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { + ScalarLongColumnSerializer serializer = new ScalarLongColumnSerializer( + fileNameBase, + new IndexSpec(), + writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), + new BaseProgressIndicator(), + closer + ); + + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer(); + for (Object o : data) { + indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); + } + SortedMap sortedFields = new TreeMap<>(); + + IndexableAdapter.NestedColumnMergable mergable = closer.register( + new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo()) + ); + SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary(); + mergable.mergeFieldsInto(sortedFields); + + serializer.openDictionaryWriter(); + serializer.serializeDictionaries( + globalDictionarySortedCollector.getSortedStrings(), + globalDictionarySortedCollector.getSortedLongs(), + globalDictionarySortedCollector.getSortedDoubles(), + () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( + new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, + serializer.getGlobalLookup() + ) + ); + serializer.open(); + + NestedDataColumnSupplierTest.SettableSelector valueSelector = new NestedDataColumnSupplierTest.SettableSelector(); + for (Object o : data) { + valueSelector.setObject(StructuredData.wrap(o)); + serializer.serialize(valueSelector); + } + + try (SmooshedWriter writer = smoosher.addWithSmooshedWriter(fileNameBase, serializer.getSerializedSize())) { + serializer.writeTo(writer, smoosher); + } + smoosher.close(); + return closer.register(SmooshedFileMapper.load(tmpFile)); + } + } + + @After + public void teardown() throws IOException + { + closer.close(); + } + + @Test + public void testBasicFunctionality() + { + ColumnBuilder bob = new ColumnBuilder(); + bob.setFileMapper(fileMapper); + ScalarLongColumnAndIndexSupplier supplier = ScalarLongColumnAndIndexSupplier.read( + ByteOrder.nativeOrder(), + bitmapSerdeFactory, + baseBuffer, + bob + ); + try (ScalarLongColumn column = (ScalarLongColumn) supplier.get()) { + smokeTest(supplier, column); + } + } + + @Test + public void testConcurrency() throws ExecutionException, InterruptedException + { + // if this test ever starts being to be a flake, there might be thread safety issues + ColumnBuilder bob = new ColumnBuilder(); + bob.setFileMapper(fileMapper); + ScalarLongColumnAndIndexSupplier supplier = ScalarLongColumnAndIndexSupplier.read( + ByteOrder.nativeOrder(), + bitmapSerdeFactory, + baseBuffer, + bob + ); + final String expectedReason = "none"; + final AtomicReference failureReason = new AtomicReference<>(expectedReason); + + final int threads = 10; + ListeningExecutorService executorService = MoreExecutors.listeningDecorator( + Execs.multiThreaded(threads, "StandardNestedColumnSupplierTest-%d") + ); + Collection> futures = new ArrayList<>(threads); + final CountDownLatch threadsStartLatch = new CountDownLatch(1); + for (int i = 0; i < threads; ++i) { + futures.add( + executorService.submit(() -> { + try { + threadsStartLatch.await(); + for (int iter = 0; iter < 5000; iter++) { + try (ScalarLongColumn column = (ScalarLongColumn) supplier.get()) { + smokeTest(supplier, column); + } + } + } + catch (Throwable ex) { + failureReason.set(ex.getMessage()); + } + }) + ); + } + threadsStartLatch.countDown(); + Futures.allAsList(futures).get(); + Assert.assertEquals(expectedReason, failureReason.get()); + } + + private void smokeTest(ScalarLongColumnAndIndexSupplier supplier, ScalarLongColumn column) + { + SimpleAscendingOffset offset = new SimpleAscendingOffset(data.size()); + ColumnValueSelector valueSelector = column.makeColumnValueSelector(offset); + + StringValueSetIndex valueSetIndex = supplier.as(StringValueSetIndex.class); + DruidPredicateIndex predicateIndex = supplier.as(DruidPredicateIndex.class); + NullValueIndex nullValueIndex = supplier.as(NullValueIndex.class); + + SortedMap fields = column.getFieldTypeInfo(); + Assert.assertEquals( + ImmutableMap.of(NestedPathFinder.JSON_PATH_ROOT, new FieldTypeInfo.MutableTypeSet().add(ColumnType.LONG)), + fields + ); + + for (int i = 0; i < data.size(); i++) { + Long row = data.get(i); + + // in default value mode, even though the input row had an empty string, the selector spits out null, so we want + // to take the null checking path + + if (row != null) { + Assert.assertEquals(row, valueSelector.getObject()); + Assert.assertEquals((long) row, valueSelector.getLong()); + Assert.assertFalse(valueSelector.isNull()); + + Assert.assertTrue(valueSetIndex.forValue(String.valueOf(row)).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(String.valueOf(row)))) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(String.valueOf(row))) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH))) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + + } else { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + + Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) + .computeBitmapResult(resultFactory) + .get(i)); + } + + offset.increment(); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java new file mode 100644 index 000000000000..d3a0e9093a16 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -0,0 +1,312 @@ +/* + * 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.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; +import org.apache.druid.query.DefaultBitmapResultFactory; +import org.apache.druid.query.filter.SelectorPredicateFactory; +import org.apache.druid.segment.AutoTypeColumnIndexer; +import org.apache.druid.segment.AutoTypeColumnMerger; +import org.apache.druid.segment.BaseProgressIndicator; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.IndexableAdapter; +import org.apache.druid.segment.SimpleAscendingOffset; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.DruidPredicateIndex; +import org.apache.druid.segment.column.NullValueIndex; +import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Objects; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicReference; + +public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest +{ + private static final String NO_MATCH = "no"; + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + BitmapSerdeFactory bitmapSerdeFactory = RoaringBitmapSerdeFactory.getInstance(); + DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(bitmapSerdeFactory.getBitmapFactory()); + + List data = Arrays.asList( + "a", + "b", + null, + "", + "null", + "c" + ); + + Closer closer = Closer.create(); + + SmooshedFileMapper fileMapper; + + ByteBuffer baseBuffer; + + @BeforeClass + public static void staticSetup() + { + NestedDataModule.registerHandlersAndSerde(); + } + + @Before + public void setup() throws IOException + { + final String fileNameBase = "test"; + fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), data); + baseBuffer = fileMapper.mapFile(fileNameBase); + } + + private SmooshedFileMapper smooshify( + String fileNameBase, + File tmpFile, + List data + ) + throws IOException + { + SegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance(); + try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { + ScalarStringColumnSerializer serializer = new ScalarStringColumnSerializer( + fileNameBase, + new IndexSpec(), + writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), + new BaseProgressIndicator(), + closer + ); + + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer(); + for (Object o : data) { + indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); + } + SortedMap sortedFields = new TreeMap<>(); + + IndexableAdapter.NestedColumnMergable mergable = closer.register( + new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo()) + ); + SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary(); + mergable.mergeFieldsInto(sortedFields); + + serializer.openDictionaryWriter(); + serializer.serializeDictionaries( + globalDictionarySortedCollector.getSortedStrings(), + globalDictionarySortedCollector.getSortedLongs(), + globalDictionarySortedCollector.getSortedDoubles(), + () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( + new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, + serializer.getGlobalLookup() + ) + ); + serializer.open(); + + NestedDataColumnSupplierTest.SettableSelector valueSelector = new NestedDataColumnSupplierTest.SettableSelector(); + for (Object o : data) { + valueSelector.setObject(StructuredData.wrap(o)); + serializer.serialize(valueSelector); + } + + try (SmooshedWriter writer = smoosher.addWithSmooshedWriter(fileNameBase, serializer.getSerializedSize())) { + serializer.writeTo(writer, smoosher); + } + smoosher.close(); + return closer.register(SmooshedFileMapper.load(tmpFile)); + } + } + + @After + public void teardown() throws IOException + { + closer.close(); + } + + @Test + public void testBasicFunctionality() throws IOException + { + ColumnBuilder bob = new ColumnBuilder(); + bob.setFileMapper(fileMapper); + ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read( + ByteOrder.nativeOrder(), + bitmapSerdeFactory, + baseBuffer, + bob + ); + try (ScalarStringDictionaryEncodedColumn column = (ScalarStringDictionaryEncodedColumn) supplier.get()) { + smokeTest(supplier, column); + } + } + + @Test + public void testConcurrency() throws ExecutionException, InterruptedException + { + // if this test ever starts being to be a flake, there might be thread safety issues + ColumnBuilder bob = new ColumnBuilder(); + bob.setFileMapper(fileMapper); + ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read( + ByteOrder.nativeOrder(), + bitmapSerdeFactory, + baseBuffer, + bob + ); + final String expectedReason = "none"; + final AtomicReference failureReason = new AtomicReference<>(expectedReason); + + final int threads = 10; + ListeningExecutorService executorService = MoreExecutors.listeningDecorator( + Execs.multiThreaded(threads, "StandardNestedColumnSupplierTest-%d") + ); + Collection> futures = new ArrayList<>(threads); + final CountDownLatch threadsStartLatch = new CountDownLatch(1); + for (int i = 0; i < threads; ++i) { + futures.add( + executorService.submit(() -> { + try { + threadsStartLatch.await(); + for (int iter = 0; iter < 5000; iter++) { + try (ScalarStringDictionaryEncodedColumn column = (ScalarStringDictionaryEncodedColumn) supplier.get()) { + smokeTest(supplier, column); + } + } + } + catch (Throwable ex) { + failureReason.set(ex.getMessage()); + } + }) + ); + } + threadsStartLatch.countDown(); + Futures.allAsList(futures).get(); + Assert.assertEquals(expectedReason, failureReason.get()); + } + + private void smokeTest(ScalarStringColumnAndIndexSupplier supplier, ScalarStringDictionaryEncodedColumn column) + { + SimpleAscendingOffset offset = new SimpleAscendingOffset(data.size()); + ColumnValueSelector valueSelector = column.makeColumnValueSelector(offset); + DimensionSelector dimSelector = column.makeDimensionSelector(offset, null); + + StringValueSetIndex valueSetIndex = supplier.as(StringValueSetIndex.class); + DruidPredicateIndex predicateIndex = supplier.as(DruidPredicateIndex.class); + NullValueIndex nullValueIndex = supplier.as(NullValueIndex.class); + + SortedMap fields = column.getFieldTypeInfo(); + Assert.assertEquals(ImmutableMap.of(NestedPathFinder.JSON_PATH_ROOT, new FieldTypeInfo.MutableTypeSet().add(ColumnType.STRING)), fields); + + for (int i = 0; i < data.size(); i++) { + String row = data.get(i); + + // in default value mode, even though the input row had an empty string, the selector spits out null, so we want + // to take the null checking path + final boolean isStringAndNullEquivalent = NullHandling.isNullOrEquivalent(row); + + if (row != null && !isStringAndNullEquivalent) { + Assert.assertEquals(row, valueSelector.getObject()); + Assert.assertEquals(row, dimSelector.getObject()); + String dimSelectorLookupVal = dimSelector.lookupName(dimSelector.getRow().get(0)); + Assert.assertEquals(row, dimSelectorLookupVal); + Assert.assertEquals(dimSelector.idLookup().lookupId(dimSelectorLookupVal), dimSelector.getRow().get(0)); + + Assert.assertTrue(valueSetIndex.forValue(row).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(row))) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(row)) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH))) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + + Assert.assertTrue(dimSelector.makeValueMatcher(row).matches()); + Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); + Assert.assertTrue(dimSelector.makeValueMatcher(x -> Objects.equals(x, row)).matches()); + Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches()); + } else { + Assert.assertNull(valueSelector.getObject()); + + Assert.assertEquals(0, dimSelector.getRow().get(0)); + Assert.assertNull(dimSelector.getObject()); + Assert.assertNull(dimSelector.lookupName(dimSelector.getRow().get(0))); + + Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) + .computeBitmapResult(resultFactory) + .get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) + .computeBitmapResult(resultFactory) + .get(i)); + + Assert.assertTrue(dimSelector.makeValueMatcher((String) null).matches()); + Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); + Assert.assertTrue(dimSelector.makeValueMatcher(x -> x == null).matches()); + Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches()); + } + + offset.increment(); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/nested/VariantArrayColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/VariantArrayColumnSupplierTest.java new file mode 100644 index 000000000000..f70b8c79ddea --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/nested/VariantArrayColumnSupplierTest.java @@ -0,0 +1,270 @@ +/* + * 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.ImmutableMap; +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.guice.NestedDataModule; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; +import org.apache.druid.query.DefaultBitmapResultFactory; +import org.apache.druid.segment.AutoTypeColumnIndexer; +import org.apache.druid.segment.AutoTypeColumnMerger; +import org.apache.druid.segment.BaseProgressIndicator; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.IndexableAdapter; +import org.apache.druid.segment.SimpleAscendingOffset; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.DruidPredicateIndex; +import org.apache.druid.segment.column.NullValueIndex; +import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicReference; + +public class VariantArrayColumnSupplierTest extends InitializedNullHandlingTest +{ + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + BitmapSerdeFactory bitmapSerdeFactory = RoaringBitmapSerdeFactory.getInstance(); + DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(bitmapSerdeFactory.getBitmapFactory()); + + List> data = Arrays.asList( + Collections.emptyList(), + Arrays.asList(1L, null, 2L), + null, + Collections.singletonList(null), + Arrays.asList(3L, 4L), + Arrays.asList(null, null) + ); + + Closer closer = Closer.create(); + + SmooshedFileMapper fileMapper; + + ByteBuffer baseBuffer; + + @BeforeClass + public static void staticSetup() + { + NestedDataModule.registerHandlersAndSerde(); + } + + @Before + public void setup() throws IOException + { + final String fileNameBase = "test"; + fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), data); + baseBuffer = fileMapper.mapFile(fileNameBase); + } + + private SmooshedFileMapper smooshify( + String fileNameBase, + File tmpFile, + List data + ) + throws IOException + { + SegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance(); + try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { + VariantArrayColumnSerializer serializer = new VariantArrayColumnSerializer( + fileNameBase, + new IndexSpec(), + writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), + new BaseProgressIndicator(), + closer + ); + + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer(); + for (Object o : data) { + indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); + } + SortedMap sortedFields = new TreeMap<>(); + + IndexableAdapter.NestedColumnMergable mergable = closer.register( + new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo()) + ); + SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary(); + mergable.mergeFieldsInto(sortedFields); + + serializer.openDictionaryWriter(); + serializer.serializeDictionaries( + globalDictionarySortedCollector.getSortedStrings(), + globalDictionarySortedCollector.getSortedLongs(), + globalDictionarySortedCollector.getSortedDoubles(), + () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( + new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, + serializer.getGlobalLookup() + ) + ); + serializer.open(); + + NestedDataColumnSupplierTest.SettableSelector valueSelector = new NestedDataColumnSupplierTest.SettableSelector(); + for (Object o : data) { + valueSelector.setObject(StructuredData.wrap(o)); + serializer.serialize(valueSelector); + } + + try (SmooshedWriter writer = smoosher.addWithSmooshedWriter(fileNameBase, serializer.getSerializedSize())) { + serializer.writeTo(writer, smoosher); + } + smoosher.close(); + return closer.register(SmooshedFileMapper.load(tmpFile)); + } + } + + @After + public void teardown() throws IOException + { + closer.close(); + } + + @Test + public void testBasicFunctionality() throws IOException + { + ColumnBuilder bob = new ColumnBuilder(); + bob.setFileMapper(fileMapper); + VariantArrayColumnAndIndexSupplier supplier = VariantArrayColumnAndIndexSupplier.read( + ColumnType.LONG_ARRAY, + ByteOrder.nativeOrder(), + bitmapSerdeFactory, + baseBuffer, + bob + ); + try (VariantArrayColumn column = (VariantArrayColumn) supplier.get()) { + smokeTest(supplier, column); + } + } + + @Test + public void testConcurrency() throws ExecutionException, InterruptedException + { + // if this test ever starts being to be a flake, there might be thread safety issues + ColumnBuilder bob = new ColumnBuilder(); + bob.setFileMapper(fileMapper); + VariantArrayColumnAndIndexSupplier supplier = VariantArrayColumnAndIndexSupplier.read( + ColumnType.LONG_ARRAY, + ByteOrder.nativeOrder(), + bitmapSerdeFactory, + baseBuffer, + bob + ); + final String expectedReason = "none"; + final AtomicReference failureReason = new AtomicReference<>(expectedReason); + + final int threads = 10; + ListeningExecutorService executorService = MoreExecutors.listeningDecorator( + Execs.multiThreaded(threads, "StandardNestedColumnSupplierTest-%d") + ); + Collection> futures = new ArrayList<>(threads); + final CountDownLatch threadsStartLatch = new CountDownLatch(1); + for (int i = 0; i < threads; ++i) { + futures.add( + executorService.submit(() -> { + try { + threadsStartLatch.await(); + for (int iter = 0; iter < 5000; iter++) { + try (VariantArrayColumn column = (VariantArrayColumn) supplier.get()) { + smokeTest(supplier, column); + } + } + } + catch (Throwable ex) { + failureReason.set(ex.getMessage()); + } + }) + ); + } + threadsStartLatch.countDown(); + Futures.allAsList(futures).get(); + Assert.assertEquals(expectedReason, failureReason.get()); + } + + private void smokeTest(VariantArrayColumnAndIndexSupplier supplier, VariantArrayColumn column) + { + SimpleAscendingOffset offset = new SimpleAscendingOffset(data.size()); + ColumnValueSelector valueSelector = column.makeColumnValueSelector(offset); + + StringValueSetIndex valueSetIndex = supplier.as(StringValueSetIndex.class); + Assert.assertNull(valueSetIndex); + DruidPredicateIndex predicateIndex = supplier.as(DruidPredicateIndex.class); + Assert.assertNull(predicateIndex); + NullValueIndex nullValueIndex = supplier.as(NullValueIndex.class); + Assert.assertNotNull(nullValueIndex); + + SortedMap fields = column.getFieldTypeInfo(); + Assert.assertEquals( + ImmutableMap.of(NestedPathFinder.JSON_PATH_ROOT, new FieldTypeInfo.MutableTypeSet().add(ColumnType.LONG_ARRAY)), + fields + ); + + for (int i = 0; i < data.size(); i++) { + List row = data.get(i); + + // in default value mode, even though the input row had an empty string, the selector spits out null, so we want + // to take the null checking path + + if (row != null) { + Assert.assertArrayEquals(row.toArray(), (Object[]) valueSelector.getObject()); + Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + + } else { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + } + + offset.increment(); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java b/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java index de90337b73d1..df7a10298442 100644 --- a/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java @@ -49,6 +49,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { + private static final String COLUMN_NAME = "missing"; private static final ByteBuffer EMPTY_BUFFER = ByteBuffer.allocate(0); private static final ColumnConfig A_CONFIG = () -> 0; diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionPlannerTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionPlannerTest.java index 37a09a10af6d..0e8cfef90367 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionPlannerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionPlannerTest.java @@ -30,7 +30,6 @@ import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; @@ -830,11 +829,11 @@ public void testNestedColumnExpression() ExpressionType.toColumnType(thePlan.getOutputType()) ); Assert.assertEquals( - NestedDataComplexTypeSerde.TYPE.getType(), + ColumnType.NESTED_DATA.getType(), inferred.getType() ); Assert.assertEquals( - NestedDataExpressions.TYPE.getComplexTypeName(), + ColumnType.NESTED_DATA.getComplexTypeName(), inferred.getComplexTypeName() ); } diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java index 4bbc23068f68..737e86d74144 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java @@ -51,7 +51,6 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.ZeroIndexedInts; -import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; @@ -485,7 +484,7 @@ public ColumnCapabilities getColumnCapabilities(String column) return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG); } if ("n".equals(column)) { - return ColumnCapabilitiesImpl.createDefault().setType(NestedDataComplexTypeSerde.TYPE); + return ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA); } return null; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java index 5a722e342e63..43e3f096541f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java @@ -29,7 +29,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; @@ -352,15 +352,15 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) FireHydrant lastHydrant = hydrants.get(numHydrants - 1); newCount = lastHydrant.getCount() + 1; if (!indexSchema.getDimensionsSpec().hasCustomDimensions()) { - Map oldCapabilities; + Map oldFormat; if (lastHydrant.hasSwapped()) { - oldCapabilities = new HashMap<>(); + oldFormat = new HashMap<>(); ReferenceCountingSegment segment = lastHydrant.getIncrementedSegment(); try { QueryableIndex oldIndex = segment.asQueryableIndex(); for (String dim : oldIndex.getAvailableDimensions()) { dimOrder.add(dim); - oldCapabilities.put(dim, oldIndex.getColumnHolder(dim).getHandlerCapabilities()); + oldFormat.put(dim, oldIndex.getColumnHolder(dim).getColumnFormat()); } } finally { @@ -369,9 +369,9 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) } else { IncrementalIndex oldIndex = lastHydrant.getIndex(); dimOrder.addAll(oldIndex.getDimensionOrder()); - oldCapabilities = oldIndex.getColumnHandlerCapabilities(); + oldFormat = oldIndex.getColumnFormats(); } - newIndex.loadDimensionIterable(dimOrder, oldCapabilities); + newIndex.loadDimensionIterable(dimOrder, oldFormat); } } currHydrant = new FireHydrant(newIndex, newCount, getSegment().getId()); diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 17b02089592b..f78548adc4b6 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -475,9 +475,9 @@ public static void runDumpNestedColumn( } jg.writeEndArray(); - Indexed globalStringDictionary = nestedDataColumn.getStringDictionary(); - FixedIndexed globalLongDictionary = nestedDataColumn.getLongDictionary(); - FixedIndexed globalDoubleDictionary = nestedDataColumn.getDoubleDictionary(); + Indexed globalStringDictionary = nestedDataColumn.getUtf8BytesDictionary(); + Indexed globalLongDictionary = nestedDataColumn.getLongDictionary(); + Indexed globalDoubleDictionary = nestedDataColumn.getDoubleDictionary(); jg.writeFieldName("dictionaries"); jg.writeStartObject(); { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java index 6f2cd8f00154..c2d78b306a5a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java @@ -95,7 +95,7 @@ public DimFilter toDruidFilter( final DruidExpression leftExpr = druidExpressions.get(0); final DruidExpression rightExpr = druidExpressions.get(1); - if (leftExpr.isSimpleExtraction() && !(leftExpr.getDruidType() != null && leftExpr.getDruidType().isArray())) { + if (leftExpr.isSimpleExtraction() && !(leftExpr.isDirectColumnAccess() && leftExpr.getDruidType() != null && leftExpr.getDruidType().isArray())) { Expr expr = Parser.parse(rightExpr.getExpression(), plannerContext.getExprMacroTable()); // To convert this expression filter into an And of Selector filters, we need to extract all array elements. // For now, we can optimize only when rightExpr is a literal because there is no way to extract the array elements diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java index 9651221b5373..cda69526f744 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java @@ -91,18 +91,22 @@ public DimFilter toDruidFilter( } // Converts array_overlaps() function into an OR of Selector filters if possible. - final boolean leftSimpleExtractionExpr = druidExpressions.get(0).isSimpleExtraction(); - final boolean rightSimpleExtractionExpr = druidExpressions.get(1).isSimpleExtraction(); + final DruidExpression leftExpression = druidExpressions.get(0); + final DruidExpression rightExpression = druidExpressions.get(1); + final boolean leftSimpleExtractionExpr = leftExpression.isSimpleExtraction(); + final boolean leftArrayColumn = leftExpression.isDirectColumnAccess() && leftExpression.getDruidType() != null && leftExpression.getDruidType().isArray(); + final boolean rightSimpleExtractionExpr = rightExpression.isSimpleExtraction(); + final boolean rightArrayColumn = rightExpression.isDirectColumnAccess() && rightExpression.getDruidType() != null && rightExpression.getDruidType().isArray(); final DruidExpression simpleExtractionExpr; final DruidExpression complexExpr; - if (leftSimpleExtractionExpr ^ rightSimpleExtractionExpr) { + if (leftSimpleExtractionExpr ^ rightSimpleExtractionExpr && !(leftArrayColumn || rightArrayColumn)) { if (leftSimpleExtractionExpr) { - simpleExtractionExpr = druidExpressions.get(0); - complexExpr = druidExpressions.get(1); + simpleExtractionExpr = leftExpression; + complexExpr = rightExpression; } else { - simpleExtractionExpr = druidExpressions.get(1); - complexExpr = druidExpressions.get(0); + simpleExtractionExpr = rightExpression; + complexExpr = leftExpression; } } else { return toExpressionFilter(plannerContext, getDruidFunctionName(), druidExpressions); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java index b6e0df60e869..872af4bec7fe 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java @@ -47,7 +47,6 @@ import org.apache.druid.math.expr.Parser; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.apache.druid.segment.nested.NestedPathFinder; import org.apache.druid.segment.nested.NestedPathPart; import org.apache.druid.segment.virtual.NestedFieldVirtualColumn; @@ -72,7 +71,7 @@ public class NestedDataOperatorConversions public static final SqlReturnTypeInference NESTED_RETURN_TYPE_INFERENCE = opBinding -> RowSignatures.makeComplexType( opBinding.getTypeFactory(), - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, true ); @@ -213,10 +212,10 @@ public DruidExpression toDruidExpression( if (druidExpressions.get(0).isSimpleExtraction()) { return DruidExpression.ofVirtualColumn( - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, builder, ImmutableList.of( - DruidExpression.ofColumn(NestedDataComplexTypeSerde.TYPE, druidExpressions.get(0).getDirectColumn()) + DruidExpression.ofColumn(ColumnType.NESTED_DATA, druidExpressions.get(0).getDirectColumn()) ), (name, outputType, expression, macroTable) -> new NestedFieldVirtualColumn( druidExpressions.get(0).getDirectColumn(), @@ -229,7 +228,7 @@ public DruidExpression toDruidExpression( ) ); } - return DruidExpression.ofExpression(NestedDataComplexTypeSerde.TYPE, builder, druidExpressions); + return DruidExpression.ofExpression(ColumnType.NESTED_DATA, builder, druidExpressions); } } @@ -408,7 +407,7 @@ public DruidExpression toDruidExpression( druidType, builder, ImmutableList.of( - DruidExpression.ofColumn(NestedDataComplexTypeSerde.TYPE, druidExpressions.get(0).getDirectColumn()) + DruidExpression.ofColumn(ColumnType.NESTED_DATA, druidExpressions.get(0).getDirectColumn()) ), (name, outputType, expression, macroTable) -> new NestedFieldVirtualColumn( druidExpressions.get(0).getDirectColumn(), @@ -538,7 +537,7 @@ public DruidExpression toDruidExpression( druidType, builder, ImmutableList.of( - DruidExpression.ofColumn(NestedDataComplexTypeSerde.TYPE, druidExpressions.get(0).getDirectColumn()) + DruidExpression.ofColumn(ColumnType.NESTED_DATA, druidExpressions.get(0).getDirectColumn()) ), (name, outputType, expression, macroTable) -> new NestedFieldVirtualColumn( druidExpressions.get(0).getDirectColumn(), @@ -709,7 +708,7 @@ public DruidExpression toDruidExpression( columnType, builder, ImmutableList.of( - DruidExpression.ofColumn(NestedDataComplexTypeSerde.TYPE, druidExpressions.get(0).getDirectColumn()) + DruidExpression.ofColumn(ColumnType.NESTED_DATA, druidExpressions.get(0).getDirectColumn()) ), (name, outputType, expression, macroTable) -> new NestedFieldVirtualColumn( druidExpressions.get(0).getDirectColumn(), @@ -761,7 +760,7 @@ public DruidExpression toDruidExpression(PlannerContext plannerContext, RowSigna { final DruidExpression.DruidExpressionCreator expressionFunction = druidExpressions -> DruidExpression.ofExpression( - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, null, DruidExpression.functionCall("json_object"), druidExpressions @@ -815,7 +814,7 @@ public DruidExpression toDruidExpression( rowSignature, rexNode, druidExpressions -> DruidExpression.ofExpression( - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, DruidExpression.functionCall("to_json_string"), druidExpressions ) @@ -853,7 +852,7 @@ public DruidExpression toDruidExpression( rowSignature, rexNode, druidExpressions -> DruidExpression.ofExpression( - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, DruidExpression.functionCall("parse_json"), druidExpressions ) @@ -891,7 +890,7 @@ public DruidExpression toDruidExpression( rowSignature, rexNode, druidExpressions -> DruidExpression.ofExpression( - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, DruidExpression.functionCall("try_parse_json"), druidExpressions ) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java index 7117be8ea267..6d7d2f5264d8 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java @@ -56,7 +56,6 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -816,8 +815,8 @@ DatasourceTable.PhysicalDatasourceMetadata buildDruidTable(final String dataSour return existingType; } // if any are json, are all json - if (NestedDataComplexTypeSerde.TYPE.equals(columnType) || NestedDataComplexTypeSerde.TYPE.equals(existingType)) { - return NestedDataComplexTypeSerde.TYPE; + if (ColumnType.NESTED_DATA.equals(columnType) || ColumnType.NESTED_DATA.equals(existingType)) { + return ColumnType.NESTED_DATA; } // "existing type" is the 'newest' type, since we iterate the segments list by newest start time return existingType; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 4545bc09cddd..6cbdd6015085 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -57,14 +57,13 @@ import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.topn.DimensionTopNMetricSpec; import org.apache.druid.query.topn.TopNQueryBuilder; +import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.NestedDataDimensionSchema; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.JoinableFactoryWrapper; -import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.virtual.NestedFieldVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -144,11 +143,11 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest new TimestampSpec("t", "iso", null), DimensionsSpec.builder().setDimensions( ImmutableList.builder() - .add(new NestedDataDimensionSchema("string")) - .add(new NestedDataDimensionSchema("nest")) - .add(new NestedDataDimensionSchema("nester")) - .add(new NestedDataDimensionSchema("long")) - .add(new NestedDataDimensionSchema("string_sparse")) + .add(new AutoTypeColumnSchema("string")) + .add(new AutoTypeColumnSchema("nest")) + .add(new AutoTypeColumnSchema("nester")) + .add(new AutoTypeColumnSchema("long")) + .add(new AutoTypeColumnSchema("string_sparse")) .build() ).build(), null @@ -159,8 +158,8 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest DimensionsSpec.builder().setDimensions( ImmutableList.builder() .add(new StringDimensionSchema("string")) - .add(new NestedDataDimensionSchema("nest")) - .add(new NestedDataDimensionSchema("nester")) + .add(new AutoTypeColumnSchema("nest")) + .add(new AutoTypeColumnSchema("nester")) .add(new LongDimensionSchema("long")) .add(new StringDimensionSchema("string_sparse")) .build() @@ -921,7 +920,6 @@ public void testJsonValueArrays() @Test public void testUnnestRootSingleTypeArrayLongNulls() { - cannotVectorize(); testBuilder() .sql("SELECT longs FROM druid.arrays, UNNEST(arrayLongNulls) as u(longs)") .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) @@ -1308,6 +1306,59 @@ public void testGroupByRootSingleTypeArrayLongNullsFiltered() .run(); } + + @Test + public void testGroupByRootSingleTypeArrayLongNullsFilteredMore() + { + cannotVectorize(); + testBuilder() + .sql( + "SELECT " + + "arrayLongNulls, " + + "SUM(cnt) " + + "FROM druid.arrays WHERE ARRAY_CONTAINS(arrayLongNulls, 1) OR ARRAY_OVERLAP(arrayLongNulls, ARRAY[2, 3]) GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayLongNulls", "d0", ColumnType.LONG_ARRAY) + ) + ) + .setDimFilter( + or( + expressionFilter("array_contains(\"arrayLongNulls\",1)"), + expressionFilter("array_overlap(\"arrayLongNulls\",array(2,3))") + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{Arrays.asList(null, 2L, 9L), 2L}, + new Object[]{Collections.singletonList(1L), 1L}, + new Object[]{Arrays.asList(1L, null, 3L), 2L}, + new Object[]{Arrays.asList(1L, 2L, 3L), 2L}, + new Object[]{Arrays.asList(2L, 3L), 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayLongNulls", ColumnType.LONG_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .build() + ) + .run(); + } + @Test public void testGroupByRootSingleTypeArrayString() { @@ -1865,7 +1916,6 @@ public void testGroupByRootSingleTypeArrayLongElementDefault() @Test public void testGroupByRootSingleTypeArrayStringElement() { - cannotVectorize(); testBuilder() .sql( "SELECT " @@ -1911,7 +1961,6 @@ public void testGroupByRootSingleTypeArrayStringElement() @Test public void testGroupByRootSingleTypeArrayStringElementFiltered() { - cannotVectorize(); testBuilder() .sql( "SELECT " @@ -1959,7 +2008,6 @@ public void testGroupByRootSingleTypeArrayStringElementFiltered() @Test public void testGroupByRootSingleTypeArrayDoubleElement() { - cannotVectorize(); testBuilder() .sql( "SELECT " @@ -2005,7 +2053,6 @@ public void testGroupByRootSingleTypeArrayDoubleElement() @Test public void testGroupByRootSingleTypeArrayDoubleElementFiltered() { - cannotVectorize(); testBuilder() .sql( "SELECT " @@ -2203,7 +2250,7 @@ public void testJsonAndArrayAgg() ), RowSignature.builder() .add("string", ColumnType.STRING) - .add("EXPR$1", ColumnType.ofArray(NestedDataComplexTypeSerde.TYPE)) + .add("EXPR$1", ColumnType.ofArray(ColumnType.NESTED_DATA)) .add("EXPR$2", ColumnType.LONG) .build() ); @@ -4065,7 +4112,7 @@ public void testJsonQuery() new NestedFieldVirtualColumn( "nester", "v0", - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, null, true, "$.n", @@ -4074,7 +4121,7 @@ public void testJsonQuery() new NestedFieldVirtualColumn( "nester", "v1", - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, null, true, "$.", @@ -4096,8 +4143,8 @@ public void testJsonQuery() new Object[]{null, "2"} ), RowSignature.builder() - .add("EXPR$0", NestedDataComplexTypeSerde.TYPE) - .add("EXPR$1", NestedDataComplexTypeSerde.TYPE) + .add("EXPR$0", ColumnType.NESTED_DATA) + .add("EXPR$1", ColumnType.NESTED_DATA) .build() ); @@ -4117,13 +4164,13 @@ public void testJsonQueryAndJsonObject() new ExpressionVirtualColumn( "v0", "json_object('n',\"v1\",'x',\"v2\")", - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, queryFramework().macroTable() ), new NestedFieldVirtualColumn( "nester", "v1", - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, null, true, "$.n", @@ -4146,7 +4193,7 @@ public void testJsonQueryAndJsonObject() new Object[]{"{\"x\":null,\"n\":null}"} ), RowSignature.builder() - .add("EXPR$0", NestedDataComplexTypeSerde.TYPE) + .add("EXPR$0", ColumnType.NESTED_DATA) .build() ); } @@ -4214,19 +4261,19 @@ public void testToJsonAndParseJson() new ExpressionVirtualColumn( "v0", "try_parse_json(to_json_string(\"string\"))", - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, macroTable ), new ExpressionVirtualColumn( "v1", "parse_json('{\\u0022foo\\u0022:1}')", - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, macroTable ), new ExpressionVirtualColumn( "v2", "parse_json(to_json_string(\"nester\"))", - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, macroTable ) ) @@ -4251,9 +4298,9 @@ public void testToJsonAndParseJson() ), RowSignature.builder() .add("string", ColumnType.STRING) - .add("EXPR$1", NestedDataComplexTypeSerde.TYPE) - .add("EXPR$2", NestedDataComplexTypeSerde.TYPE) - .add("EXPR$3", NestedDataComplexTypeSerde.TYPE) + .add("EXPR$1", ColumnType.NESTED_DATA) + .add("EXPR$2", ColumnType.NESTED_DATA) + .add("EXPR$3", ColumnType.NESTED_DATA) .build() ); } @@ -4318,7 +4365,7 @@ public void testJsonPathNegativeIndex() new NestedFieldVirtualColumn( "nester", "v1", - NestedDataComplexTypeSerde.TYPE, + ColumnType.NESTED_DATA, null, true, "$.array[-1]", @@ -4342,7 +4389,7 @@ public void testJsonPathNegativeIndex() ), RowSignature.builder() .add("EXPR$0", ColumnType.STRING) - .add("EXPR$1", NestedDataComplexTypeSerde.TYPE) + .add("EXPR$1", ColumnType.NESTED_DATA) .add("EXPR$2", ColumnType.STRING_ARRAY) .build() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java index 82384ea32564..83bb87436be5 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java @@ -34,7 +34,6 @@ import org.apache.druid.metadata.DefaultPasswordProvider; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.sql.calcite.external.ExternalDataSource; @@ -337,7 +336,7 @@ public void testHttpJson() RowSignature.builder() .add("x", ColumnType.STRING) .add("y", ColumnType.STRING) - .add("z", NestedDataComplexTypeSerde.TYPE) + .add("z", ColumnType.NESTED_DATA) .build() ); testIngestionQuery()