From f8909b253a75edcc70272fbab829a15357b864b7 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 27 Feb 2025 03:53:28 -0800 Subject: [PATCH 01/24] adds NestedCommonFormatColumnFormatSpec to customize json and auto column physical storage details on per column basis changes: * NestedDataColumnSchema and AutoTypeColumnSchema now accept a NestedCommonFormatColumn as a new 'columnFormatSpec' property, which allows customizing how json and auto columns are compressed, how strings are encoded in dictionaries, etc, on a per column level. The nested column format spec has several properties: objectKeysEncoding, objectStorageEncoding, objectStorageCompression, stringDictionaryEncoding, dictionaryEncodedColumnCompression, longColumnEncoding, longColumnCompression, doubleColumnCompression NestedCommonFormatColumn blends in segment level config from IndexSpec to populate any values not explicitly specified with segment level. * JSON field names can now be stored with front-coding with columnFormatSpec property objectKeysEncoding * Adds 'druid.indexing.formats.nestedColumnFormatSpec' to define a default NestedCommonFormatColumn --- .../benchmark/query/InPlanningBenchmark.java | 2 +- .../benchmark/query/SqlBenchmarkDatasets.java | 6 +- .../delta/input/ComplexTypesDeltaTable.java | 10 +- .../druid/delta/input/SnapshotDeltaTable.java | 4 +- .../data/input/AvroStreamInputFormatTest.java | 10 +- .../segment/DatasketchesProjectionTest.java | 4 +- .../druid/msq/util/DimensionSchemaUtils.java | 6 +- .../msq/indexing/MSQCompactionRunnerTest.java | 2 +- .../msq/util/DimensionSchemaUtilsTest.java | 26 +- .../druid/data/input/orc/OrcReaderTest.java | 20 +- .../NestedColumnParquetReaderTest.java | 12 +- .../protobuf/ProtobufInputFormatTest.java | 16 +- .../common/task/CompactionTaskRunTest.java | 32 +- .../input/DruidSegmentReaderTest.java | 8 +- .../InputSourceSamplerDiscoveryTest.java | 24 +- .../overlord/sampler/SamplerResponseTest.java | 2 +- .../duty/ITAutoCompactionTest.java | 2 +- .../data/input/impl/DimensionSchema.java | 2 +- .../druid/guice/BuiltInTypesModule.java | 30 +- .../druid/segment/AutoTypeColumnIndexer.java | 21 +- .../druid/segment/AutoTypeColumnMerger.java | 39 +- .../druid/segment/AutoTypeColumnSchema.java | 31 +- .../segment/DefaultColumnFormatConfig.java | 47 ++- .../NestedCommonFormatColumnHandler.java | 24 +- .../druid/segment/NestedDataColumnSchema.java | 22 +- .../column/StringEncodingStrategy.java | 4 +- .../segment/incremental/IncrementalIndex.java | 3 +- .../CompressedNestedDataComplexColumn.java | 72 ++-- ...balDictionaryEncodedFieldColumnWriter.java | 21 +- .../nested/NestedCommonFormatColumn.java | 20 +- .../NestedCommonFormatColumnFormatSpec.java | 392 ++++++++++++++++++ .../nested/NestedDataColumnSerializer.java | 29 +- .../nested/NestedDataColumnSupplier.java | 17 +- .../nested/NestedDataColumnSupplierV4.java | 15 +- .../segment/nested/NestedDataColumnV3.java | 6 +- .../segment/nested/NestedDataColumnV4.java | 6 +- .../segment/nested/NestedDataColumnV5.java | 7 +- .../nested/NestedDataComplexTypeSerde.java | 3 +- .../segment/nested/ObjectStorageEncoding.java | 45 ++ .../nested/ScalarDoubleColumnSerializer.java | 7 +- .../nested/ScalarDoubleFieldColumnWriter.java | 7 +- .../nested/ScalarLongColumnSerializer.java | 9 +- .../nested/ScalarLongFieldColumnWriter.java | 9 +- ...larNestedCommonFormatColumnSerializer.java | 15 +- .../nested/ScalarStringColumnSerializer.java | 7 +- .../nested/ScalarStringFieldColumnWriter.java | 5 +- .../nested/VariantArrayFieldColumnWriter.java | 7 +- .../nested/VariantColumnSerializer.java | 23 +- .../nested/VariantFieldColumnWriter.java | 5 +- .../NestedCommonFormatColumnPartSerde.java | 61 ++- .../virtual/NestedFieldVirtualColumn.java | 2 +- .../druid/guice/BuiltInTypesModuleTest.java | 9 + .../druid/query/NestedDataTestUtils.java | 12 +- .../query/scan/NestedDataScanQueryTest.java | 6 +- .../segment/AutoTypeColumnIndexerTest.java | 16 +- .../segment/CursorFactoryProjectionTest.java | 4 +- .../DefaultColumnFormatsConfigTest.java | 4 +- .../segment/NestedDataColumnSchemaTest.java | 40 +- .../druid/segment/filter/BaseFilterTest.java | 16 +- .../incremental/IncrementalIndexTest.java | 10 +- ...estedCommonFormatColumnFormatSpecTest.java | 163 ++++++++ .../nested/NestedDataColumnSupplierTest.java | 111 +++-- .../ScalarDoubleColumnSupplierTest.java | 4 +- .../nested/ScalarLongColumnSupplierTest.java | 4 +- .../ScalarStringColumnSupplierTest.java | 4 +- .../nested/VariantColumnSupplierTest.java | 58 +-- .../quidem/KttmNestedComponentSupplier.java | 4 +- .../org/apache/druid/cli/DumpSegment.java | 4 +- .../org/apache/druid/cli/DumpSegmentTest.java | 12 +- .../calcite/CalciteNestedDataQueryTest.java | 14 +- .../sql/calcite/util/SqlTestFramework.java | 2 +- .../sql/calcite/util/TestDataBuilder.java | 2 +- 72 files changed, 1275 insertions(+), 423 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java create mode 100644 processing/src/main/java/org/apache/druid/segment/nested/ObjectStorageEncoding.java create mode 100644 processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java index f691d2cebecb..c3e7ef42eacf 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java @@ -161,7 +161,7 @@ public void setup() throws JsonProcessingException List columnSchemas = schemaInfo.getDimensionsSpec() .getDimensions() .stream() - .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .map(x -> AutoTypeColumnSchema.of(x.getName())) .collect(Collectors.toList()); index = segmentGenerator.generate( dataSegment, diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmarkDatasets.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmarkDatasets.java index 7b6b48802f34..f8ee8ba117ee 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmarkDatasets.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmarkDatasets.java @@ -117,7 +117,7 @@ public class SqlBenchmarkDatasets ImmutableList.copyOf( Iterables.concat( expressionsSchema.getDimensionsSpecExcludeAggs().getDimensions(), - Collections.singletonList(new AutoTypeColumnSchema("nested", null)) + Collections.singletonList(AutoTypeColumnSchema.of("nested")) ) ) ).build(), @@ -402,7 +402,7 @@ public BenchmarkSchema asAutoDimensions() dimensionsSpec.withDimensions( dimensionsSpec.getDimensions() .stream() - .map(dim -> new AutoTypeColumnSchema(dim.getName(), null)) + .map(dim -> AutoTypeColumnSchema.of(dim.getName())) .collect(Collectors.toList()) ), aggregators, @@ -412,7 +412,7 @@ public BenchmarkSchema asAutoDimensions() projection.getVirtualColumns(), projection.getGroupingColumns() .stream() - .map(dim -> new AutoTypeColumnSchema(dim.getName(), null)) + .map(dim -> AutoTypeColumnSchema.of(dim.getName())) .collect(Collectors.toList()), projection.getAggregators() )).collect(Collectors.toList()), diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/ComplexTypesDeltaTable.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/ComplexTypesDeltaTable.java index 81edba927770..4793266f6472 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/ComplexTypesDeltaTable.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/ComplexTypesDeltaTable.java @@ -124,11 +124,11 @@ public class ComplexTypesDeltaTable new TimestampSpec("na", "posix", DateTimes.of("2024-01-01")), new DimensionsSpec( ImmutableList.of( - new AutoTypeColumnSchema("id", null), - new AutoTypeColumnSchema("array_info", null), - new AutoTypeColumnSchema("struct_info", null), - new AutoTypeColumnSchema("nested_struct_info", null), - new AutoTypeColumnSchema("map_info", null) + AutoTypeColumnSchema.of("id"), + AutoTypeColumnSchema.of("array_info"), + AutoTypeColumnSchema.of("struct_info"), + AutoTypeColumnSchema.of("nested_struct_info"), + AutoTypeColumnSchema.of("map_info") ) ), ColumnsFilter.all() diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java index 781fa087a159..fd57b841d1d7 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java @@ -120,8 +120,8 @@ public class SnapshotDeltaTable new TimestampSpec("na", "posix", DateTimes.of("2024-01-01")), new DimensionsSpec( ImmutableList.of( - new AutoTypeColumnSchema("id", null), - new AutoTypeColumnSchema("map_info", null) + AutoTypeColumnSchema.of("id"), + AutoTypeColumnSchema.of("map_info") ) ), ColumnsFilter.all() diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java index a13eb9b56454..323db4fc5247 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java @@ -303,15 +303,15 @@ public void testParseTransformNested() throws SchemaValidationException, IOExcep DimensionsSpec dimensionsSpec = new DimensionsSpec( ImmutableList.of( - new AutoTypeColumnSchema("someIntValueMap", null), - new AutoTypeColumnSchema("someStringValueMap", null), - new AutoTypeColumnSchema("someRecord", null), - new AutoTypeColumnSchema("someRecordArray", null), + AutoTypeColumnSchema.of("someIntValueMap"), + AutoTypeColumnSchema.of("someStringValueMap"), + AutoTypeColumnSchema.of("someRecord"), + AutoTypeColumnSchema.of("someRecordArray"), new LongDimensionSchema("tSomeIntValueMap8"), new LongDimensionSchema("tSomeIntValueMap8_2"), new StringDimensionSchema("tSomeStringValueMap8"), new LongDimensionSchema("tSomeRecordSubLong"), - new AutoTypeColumnSchema("tSomeRecordArray0", null), + AutoTypeColumnSchema.of("tSomeRecordArray0"), new StringDimensionSchema("tSomeRecordArray0nestedString") ) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/segment/DatasketchesProjectionTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/segment/DatasketchesProjectionTest.java index e0548babab3e..1f295c273eba 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/segment/DatasketchesProjectionTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/segment/DatasketchesProjectionTest.java @@ -123,7 +123,7 @@ public class DatasketchesProjectionTest extends InitializedNullHandlingTest projection.getVirtualColumns(), projection.getGroupingColumns() .stream() - .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .map(x -> AutoTypeColumnSchema.of(x.getName())) .collect(Collectors.toList()), projection.getAggregators() ); @@ -161,7 +161,7 @@ public static Collection constructorFeeder() List autoDims = dimsOrdered.getDimensions() .stream() - .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .map(x -> AutoTypeColumnSchema.of(x.getName())) .collect(Collectors.toList()); for (boolean incremental : new boolean[]{true, false}) { for (boolean sortByDim : new boolean[]{true, false}) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java index 3ad1eb484487..424700c30f01 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java @@ -87,9 +87,9 @@ public static DimensionSchema createDimensionSchema( } if (queryType != null && (queryType.isPrimitive() || queryType.isPrimitiveArray())) { - return new AutoTypeColumnSchema(column, queryType); + return new AutoTypeColumnSchema(column, queryType, null); } - return new AutoTypeColumnSchema(column, null); + return AutoTypeColumnSchema.of(column); } else { // dimensionType may not be identical to queryType, depending on arrayIngestMode. final ColumnType dimensionType = getDimensionType(column, queryType, arrayIngestMode); @@ -109,7 +109,7 @@ public static DimensionSchema createDimensionSchema( } else if (dimensionType.getType() == ValueType.DOUBLE) { return new DoubleDimensionSchema(column); } else if (dimensionType.getType() == ValueType.ARRAY) { - return new AutoTypeColumnSchema(column, dimensionType); + return new AutoTypeColumnSchema(column, dimensionType, null); } else { final ColumnCapabilities capabilities = ColumnCapabilitiesImpl.createDefault().setType(dimensionType); return DimensionHandlerUtils.getHandlerFromCapabilities(column, capabilities, null) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 1bf0da2a1488..ab9022fd028f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -107,7 +107,7 @@ public class MSQCompactionRunnerTest ); private static final LongDimensionSchema LONG_DIMENSION = new LongDimensionSchema("long_dim"); private static final NestedDataColumnSchema NESTED_DIMENSION = new NestedDataColumnSchema("nested_dim", 5); - private static final AutoTypeColumnSchema AUTO_DIMENSION = new AutoTypeColumnSchema("auto_dim", null); + private static final AutoTypeColumnSchema AUTO_DIMENSION = AutoTypeColumnSchema.of("auto_dim"); private static final List DIMENSIONS = ImmutableList.of( STRING_DIMENSION, LONG_DIMENSION, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/DimensionSchemaUtilsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/DimensionSchemaUtilsTest.java index 0a4e3ddbd814..e4f3bb51f1cc 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/DimensionSchemaUtilsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/DimensionSchemaUtilsTest.java @@ -85,7 +85,7 @@ public void testSchemaForceAuto() true, mode ); - DimensionSchema expected = new AutoTypeColumnSchema("x", ColumnType.LONG); + DimensionSchema expected = new AutoTypeColumnSchema("x", ColumnType.LONG, null); Assert.assertEquals(expected, dimensionSchema); dimensionSchema = DimensionSchemaUtils.createDimensionSchema( @@ -94,7 +94,7 @@ public void testSchemaForceAuto() true, mode ); - expected = new AutoTypeColumnSchema("x", ColumnType.DOUBLE); + expected = new AutoTypeColumnSchema("x", ColumnType.DOUBLE, null); Assert.assertEquals(expected, dimensionSchema); dimensionSchema = DimensionSchemaUtils.createDimensionSchema( @@ -103,7 +103,7 @@ public void testSchemaForceAuto() true, mode ); - expected = new AutoTypeColumnSchema("x", ColumnType.FLOAT); + expected = new AutoTypeColumnSchema("x", ColumnType.FLOAT, null); Assert.assertEquals(expected, dimensionSchema); dimensionSchema = DimensionSchemaUtils.createDimensionSchema( @@ -112,7 +112,7 @@ public void testSchemaForceAuto() true, mode ); - expected = new AutoTypeColumnSchema("x", ColumnType.STRING); + expected = new AutoTypeColumnSchema("x", ColumnType.STRING, null); Assert.assertEquals(expected, dimensionSchema); @@ -122,7 +122,7 @@ public void testSchemaForceAuto() true, mode ); - expected = new AutoTypeColumnSchema("x", ColumnType.LONG_ARRAY); + expected = new AutoTypeColumnSchema("x", ColumnType.LONG_ARRAY, null); Assert.assertEquals(expected, dimensionSchema); dimensionSchema = DimensionSchemaUtils.createDimensionSchema( @@ -131,7 +131,7 @@ public void testSchemaForceAuto() true, mode ); - expected = new AutoTypeColumnSchema("x", ColumnType.DOUBLE_ARRAY); + expected = new AutoTypeColumnSchema("x", ColumnType.DOUBLE_ARRAY, null); Assert.assertEquals(expected, dimensionSchema); dimensionSchema = DimensionSchemaUtils.createDimensionSchema( @@ -140,7 +140,7 @@ public void testSchemaForceAuto() true, mode ); - expected = new AutoTypeColumnSchema("x", ColumnType.FLOAT_ARRAY); + expected = new AutoTypeColumnSchema("x", ColumnType.FLOAT_ARRAY, null); Assert.assertEquals(expected, dimensionSchema); dimensionSchema = DimensionSchemaUtils.createDimensionSchema( @@ -149,7 +149,7 @@ public void testSchemaForceAuto() true, mode ); - expected = new AutoTypeColumnSchema("x", ColumnType.STRING_ARRAY); + expected = new AutoTypeColumnSchema("x", ColumnType.STRING_ARRAY, null); Assert.assertEquals(expected, dimensionSchema); dimensionSchema = DimensionSchemaUtils.createDimensionSchema( @@ -158,7 +158,7 @@ public void testSchemaForceAuto() true, mode ); - expected = new AutoTypeColumnSchema("x", null); + expected = AutoTypeColumnSchema.of("x"); Assert.assertEquals(expected, dimensionSchema); } } @@ -203,7 +203,7 @@ public void testSchemaArrayMode() false, ArrayIngestMode.ARRAY ); - DimensionSchema expected = new AutoTypeColumnSchema("x", ColumnType.STRING_ARRAY); + DimensionSchema expected = new AutoTypeColumnSchema("x", ColumnType.STRING_ARRAY, null); Assert.assertEquals(expected, dimensionSchema); dimensionSchema = DimensionSchemaUtils.createDimensionSchema( @@ -212,7 +212,7 @@ public void testSchemaArrayMode() false, ArrayIngestMode.ARRAY ); - expected = new AutoTypeColumnSchema("x", ColumnType.LONG_ARRAY); + expected = new AutoTypeColumnSchema("x", ColumnType.LONG_ARRAY, null); Assert.assertEquals(expected, dimensionSchema); dimensionSchema = DimensionSchemaUtils.createDimensionSchema( @@ -221,7 +221,7 @@ public void testSchemaArrayMode() false, ArrayIngestMode.ARRAY ); - expected = new AutoTypeColumnSchema("x", ColumnType.DOUBLE_ARRAY); + expected = new AutoTypeColumnSchema("x", ColumnType.DOUBLE_ARRAY, null); Assert.assertEquals(expected, dimensionSchema); dimensionSchema = DimensionSchemaUtils.createDimensionSchema( @@ -230,7 +230,7 @@ public void testSchemaArrayMode() false, ArrayIngestMode.ARRAY ); - expected = new AutoTypeColumnSchema("x", ColumnType.FLOAT_ARRAY); + expected = new AutoTypeColumnSchema("x", ColumnType.FLOAT_ARRAY, null); Assert.assertEquals(expected, dimensionSchema); } } diff --git a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java index 45f92ae0a5ea..7a3457b3aa5c 100644 --- a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java +++ b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java @@ -336,9 +336,9 @@ public void testNestedColumn() throws IOException new TimestampSpec("ts", "millis", null), new DimensionsSpec( ImmutableList.of( - new AutoTypeColumnSchema("middle", null), - new AutoTypeColumnSchema("list", null), - new AutoTypeColumnSchema("map", null) + AutoTypeColumnSchema.of("middle"), + AutoTypeColumnSchema.of("list"), + AutoTypeColumnSchema.of("map") ) ), inputFormat, @@ -545,8 +545,8 @@ public void testListMap() throws IOException new TimestampSpec("timestamp", "auto", null), new DimensionsSpec( ImmutableList.of( - new AutoTypeColumnSchema("a", null), - new AutoTypeColumnSchema("b", null) + AutoTypeColumnSchema.of("a"), + AutoTypeColumnSchema.of("b") ) ), inputFormat, @@ -611,11 +611,11 @@ public void testNestedArray() throws IOException new TimestampSpec("timestamp", "auto", null), new DimensionsSpec( ImmutableList.of( - new AutoTypeColumnSchema("a", null), - new AutoTypeColumnSchema("b", null), - new AutoTypeColumnSchema("c", null), - new AutoTypeColumnSchema("d", null), - new AutoTypeColumnSchema("t_d_0", null) + AutoTypeColumnSchema.of("a"), + AutoTypeColumnSchema.of("b"), + AutoTypeColumnSchema.of("c"), + AutoTypeColumnSchema.of("d"), + AutoTypeColumnSchema.of("t_d_0") ) ), inputFormat, diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java index 24205993eb6a..1f453f75ecf3 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java @@ -51,8 +51,8 @@ public void testNestedColumnTransformsNestedTestFile() throws IOException new TimestampSpec("timestamp", "auto", null), new DimensionsSpec( ImmutableList.of( - new AutoTypeColumnSchema("nestedData", null), - new AutoTypeColumnSchema("t_nestedData_listDim", null), + AutoTypeColumnSchema.of("nestedData"), + AutoTypeColumnSchema.of("t_nestedData_listDim"), new StringDimensionSchema("t_nestedData_listDim_string"), new StringDimensionSchema("t_nestedData_dim2"), new LongDimensionSchema("t_nestedData_dim3"), @@ -105,10 +105,10 @@ public void testNestedColumnTransformsNestedNullableListFile() throws IOExceptio new TimestampSpec("timestamp", "auto", null), new DimensionsSpec( ImmutableList.of( - new AutoTypeColumnSchema("a1", null), - new AutoTypeColumnSchema("a2", null), - new AutoTypeColumnSchema("t_a2", null), - new AutoTypeColumnSchema("t_a1_b1", null), + AutoTypeColumnSchema.of("a1"), + AutoTypeColumnSchema.of("a2"), + AutoTypeColumnSchema.of("t_a2"), + AutoTypeColumnSchema.of("t_a1_b1"), new LongDimensionSchema("t_a1_b1_c1"), new LongDimensionSchema("t_e2_0_b1"), new LongDimensionSchema("tt_a2_0_b1") 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 2069f23893b7..68fa78864727 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 @@ -269,14 +269,14 @@ public void testParseNestedData() throws Exception timestampSpec, new DimensionsSpec( Lists.newArrayList( - new AutoTypeColumnSchema("event", null), - new AutoTypeColumnSchema("id", null), - new AutoTypeColumnSchema("someOtherId", null), - new AutoTypeColumnSchema("isValid", null), - new AutoTypeColumnSchema("eventType", null), - new AutoTypeColumnSchema("foo", null), - new AutoTypeColumnSchema("bar", null), - new AutoTypeColumnSchema("someBytesColumn", null) + AutoTypeColumnSchema.of("event"), + AutoTypeColumnSchema.of("id"), + AutoTypeColumnSchema.of("someOtherId"), + AutoTypeColumnSchema.of("isValid"), + AutoTypeColumnSchema.of("eventType"), + AutoTypeColumnSchema.of("foo"), + AutoTypeColumnSchema.of("bar"), + AutoTypeColumnSchema.of("someBytesColumn") ) ), null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 613a8277530c..bd0894b88f14 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -1745,10 +1745,10 @@ public void testRunWithAutoCastDimensions() throws Exception new TimestampSpec("ts", "auto", null), DimensionsSpec.builder() .setDimensions(Arrays.asList( - new AutoTypeColumnSchema("ts", ColumnType.STRING), - new AutoTypeColumnSchema("dim", null), - new AutoTypeColumnSchema("x", ColumnType.LONG), - new AutoTypeColumnSchema("y", ColumnType.LONG) + new AutoTypeColumnSchema("ts", ColumnType.STRING, null), + AutoTypeColumnSchema.of("dim"), + new AutoTypeColumnSchema("x", ColumnType.LONG, null), + new AutoTypeColumnSchema("y", ColumnType.LONG, null) )) .build(), "|", @@ -1791,10 +1791,10 @@ public void testRunWithAutoCastDimensions() throws Exception DimensionsSpec.builder() .setDimensions(Arrays.asList( // check explicitly specified types are preserved - new AutoTypeColumnSchema("ts", ColumnType.STRING), - new AutoTypeColumnSchema("dim", null), - new AutoTypeColumnSchema("x", ColumnType.LONG), - new AutoTypeColumnSchema("y", ColumnType.LONG) + new AutoTypeColumnSchema("ts", ColumnType.STRING, null), + AutoTypeColumnSchema.of("dim"), + new AutoTypeColumnSchema("x", ColumnType.LONG, null), + new AutoTypeColumnSchema("y", ColumnType.LONG, null) )) .build(), expectedLongSumMetric @@ -1873,11 +1873,11 @@ public void testRunWithAutoCastDimensionsSortByDimension() throws Exception new TimestampSpec("ts", "auto", null), DimensionsSpec.builder() .setDimensions(Arrays.asList( - new AutoTypeColumnSchema("x", ColumnType.LONG), + new AutoTypeColumnSchema("x", ColumnType.LONG, null), new LongDimensionSchema("__time"), - new AutoTypeColumnSchema("ts", ColumnType.STRING), - new AutoTypeColumnSchema("dim", null), - new AutoTypeColumnSchema("y", ColumnType.LONG) + new AutoTypeColumnSchema("ts", ColumnType.STRING, null), + AutoTypeColumnSchema.of("dim"), + new AutoTypeColumnSchema("y", ColumnType.LONG, null) )) .setForceSegmentSortByTime(false) .build(), @@ -1921,11 +1921,11 @@ public void testRunWithAutoCastDimensionsSortByDimension() throws Exception DimensionsSpec.builder() .setDimensions(Arrays.asList( // check explicitly that time ordering is preserved - new AutoTypeColumnSchema("x", ColumnType.LONG), + new AutoTypeColumnSchema("x", ColumnType.LONG, null), new LongDimensionSchema("__time"), - new AutoTypeColumnSchema("ts", ColumnType.STRING), - new AutoTypeColumnSchema("dim", null), - new AutoTypeColumnSchema("y", ColumnType.LONG) + new AutoTypeColumnSchema("ts", ColumnType.STRING, null), + AutoTypeColumnSchema.of("dim"), + new AutoTypeColumnSchema("y", ColumnType.LONG, null) )) .setForceSegmentSortByTime(false) .build(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java index 0875cdde8677..5c6a9bc71999 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java @@ -694,7 +694,7 @@ public void testArrayColumns() throws IOException ImmutableList.of( StringDimensionSchema.create("strCol"), new DoubleDimensionSchema("dblCol"), - new AutoTypeColumnSchema("arrayCol", null) + AutoTypeColumnSchema.of("arrayCol") ) ); List metrics = ImmutableList.of( @@ -768,7 +768,7 @@ public void testArrayColumns() throws IOException ImmutableList.of( StringDimensionSchema.create("strCol"), new DoubleDimensionSchema("dblCol"), - new AutoTypeColumnSchema("arrayCol", null) + AutoTypeColumnSchema.of("arrayCol") ) ), ColumnsFilter.all(), @@ -806,7 +806,7 @@ public void testArrayColumnsCast() throws IOException ImmutableList.of( StringDimensionSchema.create("strCol"), new DoubleDimensionSchema("dblCol"), - new AutoTypeColumnSchema("arrayCol", ColumnType.STRING_ARRAY) + new AutoTypeColumnSchema("arrayCol", ColumnType.STRING_ARRAY, null) ) ); List metrics = ImmutableList.of( @@ -880,7 +880,7 @@ public void testArrayColumnsCast() throws IOException ImmutableList.of( StringDimensionSchema.create("strCol"), new DoubleDimensionSchema("dblCol"), - new AutoTypeColumnSchema("arrayCol", ColumnType.STRING_ARRAY) + new AutoTypeColumnSchema("arrayCol", ColumnType.STRING_ARRAY, null) ) ), ColumnsFilter.all(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java index 463416fc33e6..ffc20ee20520 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java @@ -80,21 +80,21 @@ public void testDiscoveredTypesStrictBooleans() new DoubleDimensionSchema("double"), new LongDimensionSchema("bool"), new StringDimensionSchema("variant"), - new AutoTypeColumnSchema("array", null), - new AutoTypeColumnSchema("nested", null) + AutoTypeColumnSchema.of("array"), + AutoTypeColumnSchema.of("nested") ), response.getLogicalDimensions() ); Assert.assertEquals( ImmutableList.of( - new AutoTypeColumnSchema("string", null), - new AutoTypeColumnSchema("long", null), - new AutoTypeColumnSchema("double", null), - new AutoTypeColumnSchema("bool", null), - new AutoTypeColumnSchema("variant", null), - new AutoTypeColumnSchema("array", null), - new AutoTypeColumnSchema("nested", null) + AutoTypeColumnSchema.of("string"), + AutoTypeColumnSchema.of("long"), + AutoTypeColumnSchema.of("double"), + AutoTypeColumnSchema.of("bool"), + AutoTypeColumnSchema.of("variant"), + AutoTypeColumnSchema.of("array"), + AutoTypeColumnSchema.of("nested") ), response.getPhysicalDimensions() ); @@ -183,9 +183,9 @@ public void testTypesNoDiscoveryExplicitSchema() new LongDimensionSchema("long"), new DoubleDimensionSchema("double"), new StringDimensionSchema("bool"), - new AutoTypeColumnSchema("variant", null), - new AutoTypeColumnSchema("array", null), - new AutoTypeColumnSchema("nested", null) + AutoTypeColumnSchema.of("variant"), + AutoTypeColumnSchema.of("array"), + AutoTypeColumnSchema.of("nested") ) .build(); final SamplerResponse response = inputSourceSampler.sample( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerResponseTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerResponseTest.java index f552ed076f45..f3329a9c5238 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerResponseTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerResponseTest.java @@ -65,7 +65,7 @@ public void testSerde() throws IOException new StringDimensionSchema("dim1") ), ImmutableList.of( - new AutoTypeColumnSchema("dim1", null) + AutoTypeColumnSchema.of("dim1") ), RowSignature.builder().addTimeColumn().add("dim1", ColumnType.STRING).add("met1", ColumnType.LONG).build(), data diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index baf7177b373d..df9cb1bebd1f 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -535,7 +535,7 @@ public void testAutoCompactionPreservesCreateBitmapIndexInDimensionSchema(Compac List dimensionSchemas = ImmutableList.of( new StringDimensionSchema("language", DimensionSchema.MultiValueHandling.SORTED_ARRAY, false), - new AutoTypeColumnSchema("deleted", ColumnType.DOUBLE) + new AutoTypeColumnSchema("deleted", ColumnType.DOUBLE, null) ); submitCompactionConfig( 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 b5933e722d8c..ea6ae9f0b758 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 @@ -70,7 +70,7 @@ public static DimensionSchema getDefaultSchemaForBuiltInType(String name, TypeSi return new DoubleDimensionSchema(name); default: // the auto column indexer can handle any type - return new AutoTypeColumnSchema(name, null); + return AutoTypeColumnSchema.of(name); } } diff --git a/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java b/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java index 96866d44a2a0..08cdf60244c1 100644 --- a/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java +++ b/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java @@ -32,6 +32,7 @@ import org.apache.druid.segment.DimensionHandlerProvider; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.NestedCommonFormatColumnHandler; +import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.apache.druid.segment.nested.StructuredData; import org.apache.druid.segment.nested.StructuredDataJsonSerializer; @@ -50,6 +51,8 @@ public class BuiltInTypesModule implements DruidModule * {@link #initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig)}. */ private static DimensionSchema.MultiValueHandling STRING_MV_MODE = DimensionSchema.MultiValueHandling.SORTED_ARRAY; + @Nullable + private static NestedCommonFormatColumnFormatSpec DEFAULT_NESTED_COMMON_FORMAT_SPEC = null; @Override public List getJacksonModules() @@ -71,14 +74,9 @@ public void configure(Binder binder) @LazySingleton public SideEffectRegisterer initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig formatsConfig) { - if (formatsConfig.getNestedColumnFormatVersion() == null || formatsConfig.getNestedColumnFormatVersion() == 5) { - DimensionHandlerUtils.registerDimensionHandlerProvider( - NestedDataComplexTypeSerde.TYPE_NAME, - new NestedCommonFormatHandlerProvider() - ); - } - setStringMultiValueHandlingModeIfConfigured(formatsConfig.getStringMultiValueHandlingMode()); + setNestedColumnDefaults(formatsConfig); + return new SideEffectRegisterer(); } @@ -89,6 +87,17 @@ private static void setStringMultiValueHandlingModeIfConfigured(@Nullable String } } + private static void setNestedColumnDefaults(DefaultColumnFormatConfig formatsConfig) + { + if (formatsConfig.getNestedColumnFormatVersion() == null || formatsConfig.getNestedColumnFormatVersion() == 5) { + DimensionHandlerUtils.registerDimensionHandlerProvider( + NestedDataComplexTypeSerde.TYPE_NAME, + new NestedCommonFormatHandlerProvider() + ); + } + DEFAULT_NESTED_COMMON_FORMAT_SPEC = formatsConfig.getNestedColumnFormatSpec(); + } + /** * @return the configured string multi value handling mode from the system config if set; otherwise, returns * the default. @@ -98,6 +107,11 @@ public static DimensionSchema.MultiValueHandling getStringMultiValueHandlingMode return STRING_MV_MODE; } + public static NestedCommonFormatColumnFormatSpec getDefaultNestedCommonFormatSpec() + { + return DEFAULT_NESTED_COMMON_FORMAT_SPEC; + } + public static List getJacksonModulesList() { return Collections.singletonList( @@ -134,7 +148,7 @@ public static class NestedCommonFormatHandlerProvider @Override public DimensionHandler get(String dimensionName) { - return new NestedCommonFormatColumnHandler(dimensionName, null); + return new NestedCommonFormatColumnHandler(dimensionName, null, DEFAULT_NESTED_COMMON_FORMAT_SPEC); } } diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java index 28b679d49e28..6c4ebee10aa2 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java @@ -43,6 +43,7 @@ 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.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.nested.NestedPathFinder; import org.apache.druid.segment.nested.NestedPathPart; import org.apache.druid.segment.nested.SortedValueDictionary; @@ -86,6 +87,8 @@ public class AutoTypeColumnIndexer implements DimensionIndexer processArrayField( } }; - public AutoTypeColumnIndexer(String name, @Nullable ColumnType castToType) + public AutoTypeColumnIndexer(String name, @Nullable ColumnType castToType, NestedCommonFormatColumnFormatSpec columnFormatSpec) { this.columnName = name; if (castToType != null && (castToType.isPrimitive() || castToType.isPrimitiveArray())) { @@ -139,6 +142,7 @@ public AutoTypeColumnIndexer(String name, @Nullable ColumnType castToType) this.castToType = null; this.castToExpressionType = null; } + this.columnFormatSpec = columnFormatSpec; } @Override @@ -477,7 +481,7 @@ public Object getConstantValue() @Override public ColumnFormat getFormat() { - return new Format(getLogicalType(), hasNulls, castToType != null); + return new Format(getLogicalType(), hasNulls, castToType != null, columnFormatSpec); } @Override @@ -762,12 +766,14 @@ static class Format implements ColumnFormat private final ColumnType logicalType; private final boolean hasNulls; private final boolean enforceLogicalType; + private final NestedCommonFormatColumnFormatSpec columnFormatSpec; - Format(ColumnType logicalType, boolean hasNulls, boolean enforceLogicalType) + Format(ColumnType logicalType, boolean hasNulls, boolean enforceLogicalType, NestedCommonFormatColumnFormatSpec columnFormatSpec) { this.logicalType = logicalType; this.hasNulls = hasNulls; this.enforceLogicalType = enforceLogicalType; + this.columnFormatSpec = columnFormatSpec; } @Override @@ -779,13 +785,13 @@ public ColumnType getLogicalType() @Override public DimensionHandler getColumnHandler(String columnName) { - return new NestedCommonFormatColumnHandler(columnName, enforceLogicalType ? logicalType : null); + return new NestedCommonFormatColumnHandler(columnName, enforceLogicalType ? logicalType : null, columnFormatSpec); } @Override public DimensionSchema getColumnSchema(String columnName) { - return new AutoTypeColumnSchema(columnName, enforceLogicalType ? logicalType : null); + return new AutoTypeColumnSchema(columnName, enforceLogicalType ? logicalType : null, columnFormatSpec); } @Override @@ -796,10 +802,11 @@ public ColumnFormat merge(@Nullable ColumnFormat otherFormat) } if (otherFormat instanceof Format) { final Format other = (Format) otherFormat; + // todo (clint): actually merge column format spec, maybe? if (!getLogicalType().equals(other.getLogicalType())) { - return new Format(ColumnType.NESTED_DATA, hasNulls || other.hasNulls, false); + return new Format(ColumnType.NESTED_DATA, hasNulls || other.hasNulls, false, columnFormatSpec); } - return new Format(logicalType, hasNulls || other.hasNulls, enforceLogicalType || other.enforceLogicalType); + return new Format(logicalType, hasNulls || other.hasNulls, enforceLogicalType || other.enforceLogicalType, columnFormatSpec); } throw new ISE( "Cannot merge columns of type[%s] and format[%s] and with [%s] and [%s]", diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java index 0f8ea4df1450..61796fdcaa9f 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java @@ -34,6 +34,7 @@ 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.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.nested.NestedCommonFormatColumnSerializer; import org.apache.druid.segment.nested.NestedDataColumnSerializer; import org.apache.druid.segment.nested.NestedPathFinder; @@ -77,7 +78,6 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 private final String name; private final String outputName; - private final IndexSpec indexSpec; private final SegmentWriteOutMedium segmentWriteOutMedium; private final Closer closer; private NestedCommonFormatColumnSerializer serializer; @@ -87,6 +87,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 private final ColumnType castToType; private boolean isVariantType = false; private byte variantTypeByte = 0x00; + private final NestedCommonFormatColumnFormatSpec columnFormatSpec; private final File segmentBaseDir; @@ -97,7 +98,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 * projection name so that multiple projections can store the same column name at * different smoosh file "paths" * @param castToType optional mechanism to enforce that all values are a specific type - * @param indexSpec segment level storage options such as compression format and bitmap type + * @param columnFormatSpec column level storage options such as compression format and bitmap type * @param segmentWriteOutMedium temporary storage location to stage segment outputs before finalizing into the segment * @param closer resource closer if this merger needs to attach any closables that should be cleaned up * when the segment is finished writing @@ -106,7 +107,7 @@ public AutoTypeColumnMerger( String name, String outputName, @Nullable ColumnType castToType, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, SegmentWriteOutMedium segmentWriteOutMedium, File segmentBaseDir, Closer closer @@ -116,7 +117,7 @@ public AutoTypeColumnMerger( this.name = name; this.outputName = outputName; this.castToType = castToType; - this.indexSpec = indexSpec; + this.columnFormatSpec = columnFormatSpec; this.segmentWriteOutMedium = segmentWriteOutMedium; this.segmentBaseDir = segmentBaseDir; this.closer = closer; @@ -187,7 +188,7 @@ public void writeMergedValueDictionary(List adapters) throws I logicalType = ColumnType.STRING; serializer = new ScalarStringColumnSerializer( outputName, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); @@ -201,7 +202,7 @@ public void writeMergedValueDictionary(List adapters) throws I case LONG: serializer = new ScalarLongColumnSerializer( outputName, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); @@ -209,7 +210,7 @@ public void writeMergedValueDictionary(List adapters) throws I case DOUBLE: serializer = new ScalarDoubleColumnSerializer( outputName, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); @@ -217,7 +218,7 @@ public void writeMergedValueDictionary(List adapters) throws I case STRING: serializer = new ScalarStringColumnSerializer( outputName, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); @@ -227,7 +228,7 @@ public void writeMergedValueDictionary(List adapters) throws I outputName, logicalType, null, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); @@ -255,7 +256,7 @@ public void writeMergedValueDictionary(List adapters) throws I outputName, null, variantTypeByte, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); @@ -264,7 +265,7 @@ public void writeMergedValueDictionary(List adapters) throws I logicalType = ColumnType.NESTED_DATA; serializer = new NestedDataColumnSerializer( outputName, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); @@ -380,7 +381,9 @@ public ColumnDescriptor makeColumnDescriptor() .isVariantType(isVariantType) .withEnforceLogicalType(castToType != null) .withByteOrder(ByteOrder.nativeOrder()) - .withBitmapSerdeFactory(indexSpec.getBitmapSerdeFactory()) + .withColumnFormatSpec(columnFormatSpec) + // for backwards compatibility keep writing this for now + .withBitmapSerdeFactory(columnFormatSpec.getBitmapEncoding()) .withSerializer(serializer) .build(); descriptorBuilder.setValueType(ValueType.COMPLEX) // this doesn't really matter... you could say.. its complicated.. @@ -408,21 +411,21 @@ public void attachParent(DimensionMergerV9 parent, List projec if (autoParent.serializer instanceof ScalarStringColumnSerializer) { serializer = new ScalarStringColumnSerializer( outputName, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); } else if (autoParent.serializer instanceof ScalarLongColumnSerializer) { serializer = new ScalarLongColumnSerializer( outputName, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); } else if (autoParent.serializer instanceof ScalarDoubleColumnSerializer) { serializer = new ScalarDoubleColumnSerializer( outputName, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); @@ -432,7 +435,7 @@ public void attachParent(DimensionMergerV9 parent, List projec outputName, null, variantTypeByte, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); @@ -441,7 +444,7 @@ public void attachParent(DimensionMergerV9 parent, List projec outputName, logicalType, null, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); @@ -449,7 +452,7 @@ public void attachParent(DimensionMergerV9 parent, List projec } else { serializer = new NestedDataColumnSerializer( outputName, - indexSpec, + columnFormatSpec, segmentWriteOutMedium, closer ); diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java index 9959c9ef2493..8d385dd9bb86 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java @@ -23,8 +23,10 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.nested.NestedCommonFormatColumn; +import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.nested.NestedCommonFormatColumnSerializer; import org.apache.druid.segment.nested.NestedDataColumnSerializer; import org.apache.druid.segment.nested.ScalarDoubleColumnSerializer; @@ -64,15 +66,24 @@ */ public class AutoTypeColumnSchema extends DimensionSchema { + public static AutoTypeColumnSchema of(String name) + { + return new AutoTypeColumnSchema(name, null, null); + } + public static final String TYPE = "auto"; @Nullable private final ColumnType castToType; + @Nullable + private final NestedCommonFormatColumnFormatSpec columnFormatSpec; + @JsonCreator public AutoTypeColumnSchema( @JsonProperty("name") String name, - @JsonProperty("castToType") @Nullable ColumnType castToType + @JsonProperty("castToType") @Nullable ColumnType castToType, + @JsonProperty("columnFormatSpec") @Nullable NestedCommonFormatColumnFormatSpec columnFormatSpec ) { super(name, null, true); @@ -84,6 +95,9 @@ public AutoTypeColumnSchema( } else { this.castToType = castToType; } + this.columnFormatSpec = columnFormatSpec == null + ? BuiltInTypesModule.getDefaultNestedCommonFormatSpec() + : columnFormatSpec; } @Override @@ -106,10 +120,18 @@ public ColumnType getCastToType() return castToType; } + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public NestedCommonFormatColumnFormatSpec getColumnFormatSpec() + { + return columnFormatSpec; + } + @Override public DimensionHandler getDimensionHandler() { - return new NestedCommonFormatColumnHandler(getName(), castToType); + return new NestedCommonFormatColumnHandler(getName(), castToType, columnFormatSpec); } @Override @@ -125,13 +147,13 @@ public boolean equals(Object o) return false; } AutoTypeColumnSchema that = (AutoTypeColumnSchema) o; - return Objects.equals(castToType, that.castToType); + return Objects.equals(castToType, that.castToType) && Objects.equals(columnFormatSpec, that.columnFormatSpec); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), castToType); + return Objects.hash(super.hashCode(), castToType, columnFormatSpec); } @Override @@ -144,6 +166,7 @@ public String toString() ", multiValueHandling=" + getMultiValueHandling() + ", createBitmapIndex=" + hasBitmapIndex() + ", castToType=" + castToType + + ", columnFormatSpec=" + columnFormatSpec + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java b/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java index 0149deba1538..28e9705f8470 100644 --- a/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java +++ b/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java @@ -24,6 +24,7 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import javax.annotation.Nullable; import java.util.Arrays; @@ -33,16 +34,22 @@ public class DefaultColumnFormatConfig { private static final Logger LOG = new Logger(DefaultColumnFormatConfig.class); - public static void validateNestedFormatVersion(@Nullable Integer formatVersion) + @Nullable + public static Integer validateNestedFormatVersion(@Nullable Integer formatVersion) { if (formatVersion != null) { if (formatVersion != 5) { LOG.warn("Unsupported nested column format version[%s], using default version instead", formatVersion); + return null; } } + return formatVersion; } - private static void validateMultiValueHandlingMode(@Nullable String stringMultiValueHandlingMode) + @Nullable + private static String validateMultiValueHandlingMode( + @Nullable String stringMultiValueHandlingMode + ) { if (stringMultiValueHandlingMode != null) { try { @@ -59,6 +66,7 @@ private static void validateMultiValueHandlingMode(@Nullable String stringMultiV ); } } + return stringMultiValueHandlingMode; } @Nullable @@ -69,17 +77,30 @@ private static void validateMultiValueHandlingMode(@Nullable String stringMultiV @JsonProperty("stringMultiValueHandlingMode") private final String stringMultiValueHandlingMode; + @Nullable + @JsonProperty + private final NestedCommonFormatColumnFormatSpec nestedColumnFormatSpec; + @JsonCreator public DefaultColumnFormatConfig( + @JsonProperty("stringMultiValueHandlingMode") @Nullable String stringMultiValueHandlingMode, @JsonProperty("nestedColumnFormatVersion") @Nullable Integer nestedColumnFormatVersion, - @JsonProperty("stringMultiValueHandlingMode") @Nullable String stringMultiValueHandlingMode + @JsonProperty("nestedColumnFormatSpec") @Nullable NestedCommonFormatColumnFormatSpec nestedColumnFormatSpec ) { - validateNestedFormatVersion(nestedColumnFormatVersion); validateMultiValueHandlingMode(stringMultiValueHandlingMode); + validateNestedFormatVersion(nestedColumnFormatVersion); + this.stringMultiValueHandlingMode = validateMultiValueHandlingMode(stringMultiValueHandlingMode); this.nestedColumnFormatVersion = nestedColumnFormatVersion; - this.stringMultiValueHandlingMode = stringMultiValueHandlingMode; + this.nestedColumnFormatSpec = nestedColumnFormatSpec; + } + + @Nullable + @JsonProperty("stringMultiValueHandlingMode") + public String getStringMultiValueHandlingMode() + { + return stringMultiValueHandlingMode; } @Nullable @@ -90,10 +111,10 @@ public Integer getNestedColumnFormatVersion() } @Nullable - @JsonProperty("stringMultiValueHandlingMode") - public String getStringMultiValueHandlingMode() + @JsonProperty("nestedColumnFormatSpec") + public NestedCommonFormatColumnFormatSpec getNestedColumnFormatSpec() { - return stringMultiValueHandlingMode; + return nestedColumnFormatSpec; } @Override @@ -107,21 +128,23 @@ public boolean equals(Object o) } DefaultColumnFormatConfig that = (DefaultColumnFormatConfig) o; return Objects.equals(nestedColumnFormatVersion, that.nestedColumnFormatVersion) - && Objects.equals(stringMultiValueHandlingMode, that.stringMultiValueHandlingMode); + && Objects.equals(stringMultiValueHandlingMode, that.stringMultiValueHandlingMode) + && Objects.equals(nestedColumnFormatSpec, that.nestedColumnFormatSpec); } @Override public int hashCode() { - return Objects.hash(nestedColumnFormatVersion, stringMultiValueHandlingMode); + return Objects.hash(nestedColumnFormatVersion, stringMultiValueHandlingMode, nestedColumnFormatSpec); } @Override public String toString() { return "DefaultColumnFormatConfig{" + - "nestedColumnFormatVersion=" + nestedColumnFormatVersion + - ", stringMultiValueHandlingMode=" + stringMultiValueHandlingMode + + "stringMultiValueHandlingMode=" + stringMultiValueHandlingMode + + ", nestedColumnFormatVersion=" + nestedColumnFormatVersion + + ", nestedColumnFormatSpec=" + nestedColumnFormatSpec + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java b/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java index ab3f9053bf19..b3e7e5d43fd0 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java @@ -25,6 +25,7 @@ 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.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.nested.StructuredData; import org.apache.druid.segment.selector.settable.SettableColumnValueSelector; import org.apache.druid.segment.selector.settable.SettableObjectColumnValueSelector; @@ -45,11 +46,18 @@ public class NestedCommonFormatColumnHandler implements DimensionHandler makeIndexer(boolean useMaxMemoryEstimates) { - return new AutoTypeColumnIndexer(name, castTo); + return new AutoTypeColumnIndexer(name, castTo, columnFormatSpec); } @Override @@ -87,7 +95,15 @@ public DimensionMergerV9 makeMerger( Closer closer ) { - return new AutoTypeColumnMerger(name, outputName, castTo, indexSpec, segmentWriteOutMedium, segmentBaseDir, closer); + return new AutoTypeColumnMerger( + name, + outputName, + castTo, + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(columnFormatSpec, indexSpec), + segmentWriteOutMedium, + segmentBaseDir, + closer + ); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java index bae8435ecb72..7427db0d30be 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java @@ -24,6 +24,7 @@ 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.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import javax.annotation.Nullable; @@ -36,11 +37,14 @@ public class NestedDataColumnSchema extends DimensionSchema { final int formatVersion; + @Nullable + final NestedCommonFormatColumnFormatSpec columnFormatSpec; @JsonCreator public NestedDataColumnSchema( @JsonProperty("name") String name, @JsonProperty("formatVersion") @Nullable Integer version, + @JsonProperty("columnFormatSpec") @Nullable NestedCommonFormatColumnFormatSpec columnFormatSpec, @JacksonInject DefaultColumnFormatConfig defaultFormatConfig ) { @@ -55,6 +59,9 @@ public NestedDataColumnSchema( formatVersion = 5; } DefaultColumnFormatConfig.validateNestedFormatVersion(this.formatVersion); + this.columnFormatSpec = columnFormatSpec == null + ? defaultFormatConfig.getNestedColumnFormatSpec() + : columnFormatSpec; } public NestedDataColumnSchema( @@ -65,6 +72,7 @@ public NestedDataColumnSchema( super(name, null, true); this.formatVersion = version; DefaultColumnFormatConfig.validateNestedFormatVersion(this.formatVersion); + this.columnFormatSpec = null; } @JsonProperty("formatVersion") @@ -73,6 +81,12 @@ public int getFormatVersion() return formatVersion; } + @JsonProperty("columnFormatSpec") + public NestedCommonFormatColumnFormatSpec getColumnFormatSpec() + { + return columnFormatSpec; + } + @Override public String getTypeName() { @@ -88,7 +102,7 @@ public ColumnType getColumnType() @Override public DimensionHandler getDimensionHandler() { - return new NestedCommonFormatColumnHandler(getName(), null); + return new NestedCommonFormatColumnHandler(getName(), null, columnFormatSpec); } @Override @@ -104,13 +118,14 @@ public boolean equals(Object o) return false; } NestedDataColumnSchema that = (NestedDataColumnSchema) o; - return Objects.equals(formatVersion, that.formatVersion); + return Objects.equals(formatVersion, that.formatVersion) && + Objects.equals(columnFormatSpec, that.columnFormatSpec); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), formatVersion); + return Objects.hash(super.hashCode(), formatVersion, columnFormatSpec); } @Override @@ -118,6 +133,7 @@ public String toString() { return "NestedDataColumnSchema{" + "formatVersion=" + formatVersion + + "columnFormatSpec=" + columnFormatSpec + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/StringEncodingStrategy.java b/processing/src/main/java/org/apache/druid/segment/column/StringEncodingStrategy.java index a8246c9d75d1..ff76aa9e6bce 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/StringEncodingStrategy.java +++ b/processing/src/main/java/org/apache/druid/segment/column/StringEncodingStrategy.java @@ -36,7 +36,9 @@ }) public interface StringEncodingStrategy { - Utf8 DEFAULT = new Utf8(); + Utf8 UTF8_STRATEGY = new Utf8(); + Utf8 DEFAULT = UTF8_STRATEGY; + String UTF8 = "utf8"; String FRONT_CODED = "frontCoded"; 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 a07299154544..37a62ddcffab 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 @@ -37,6 +37,7 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.SpatialDimensionSchema; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -542,7 +543,7 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) wasNewDim = true; final DimensionHandler handler; if (useSchemaDiscovery) { - handler = new NestedCommonFormatColumnHandler(dimension, null); + handler = new NestedCommonFormatColumnHandler(dimension, null, BuiltInTypesModule.getDefaultNestedCommonFormatSpec()); } else { // legacy behavior: for schemaless type discovery, everything is a String handler = DimensionHandlerUtils.getHandlerFromCapabilities( 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 5913425cc1a4..aace29356f16 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 @@ -97,14 +97,14 @@ * 'raw' {@link StructuredData} values and provides selectors for nested field columns specified by ordered lists of * {@link NestedPathPart}. *

- * The list of available nested paths is stored in {@link #fields}, and their associated types stored in - * {@link #fieldInfo} which can be accessed by the index of the field in {@link #fields}. + * The list of available nested paths is stored in {@link #fieldsSupplier}, and their associated types stored in + * {@link #fieldInfo} which can be accessed by the index of the field in {@link #fieldsSupplier}. *

* In the case that the nested column has only a single field, and that field is the 'root' path, specified by * {@link #rootFieldPath}, the selectors created for the complex column itself will use the 'root' path selectors * instead. */ -public abstract class CompressedNestedDataComplexColumn> +public abstract class CompressedNestedDataComplexColumn, TStringDictionary extends Indexed> extends NestedDataComplexColumn implements NestedCommonFormatColumn { private static final Map, Function> AS_MAP = @@ -116,7 +116,7 @@ public abstract class CompressedNestedDataComplexColumn fields; + private final Supplier fieldsSupplier; private final FieldTypeInfo fieldInfo; private final Supplier stringDictionarySupplier; private final Supplier> longDictionarySupplier; @@ -138,7 +138,7 @@ public CompressedNestedDataComplexColumn( @SuppressWarnings("unused") ColumnConfig columnConfig, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, - GenericIndexed fields, + Supplier fieldsSupplier, FieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, @@ -153,7 +153,7 @@ public CompressedNestedDataComplexColumn( this.columnName = columnName; this.logicalType = logicalType; this.nullValues = nullValues; - this.fields = fields; + this.fieldsSupplier = fieldsSupplier; this.fieldInfo = fieldInfo; this.stringDictionarySupplier = stringDictionary; this.longDictionarySupplier = longDictionarySupplier; @@ -177,9 +177,10 @@ public CompressedNestedDataComplexColumn( @Override public SortedMap getFieldTypeInfo() { - SortedMap fieldMap = new TreeMap<>(); + final TKeyDictionary fields = fieldsSupplier.get(); + final SortedMap fieldMap = new TreeMap<>(); for (int i = 0; i < fields.size(); i++) { - String fieldPath = fields.get(i); + String fieldPath = StringUtils.fromUtf8(fields.get(i)); FieldTypeInfo.TypeSet types = fieldInfo.getTypes(i); fieldMap.put(fieldPath, new FieldTypeInfo.MutableTypeSet(types.getByteValue())); } @@ -195,9 +196,10 @@ public ColumnType getLogicalType() @Override public List> getNestedFields() { - List> fieldParts = new ArrayList<>(fields.size()); + final TKeyDictionary fields = fieldsSupplier.get(); + final List> fieldParts = new ArrayList<>(fields.size()); for (int i = 0; i < fields.size(); i++) { - fieldParts.add(parsePath(fields.get(i))); + fieldParts.add(parsePath(StringUtils.fromUtf8(fields.get(i)))); } return fieldParts; } @@ -335,7 +337,8 @@ public Object getRowValue(int rowNum) @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - if (!logicalType.equals(ColumnType.NESTED_DATA) && fields.size() == 1 && rootFieldPath.equals(fields.get(0))) { + final TKeyDictionary fields = fieldsSupplier.get(); + if (!logicalType.equals(ColumnType.NESTED_DATA) && fields.size() == 1 && rootFieldPath.equals(StringUtils.fromUtf8(fields.get(0)))) { return makeColumnValueSelector( ImmutableList.of(), offset @@ -375,7 +378,8 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset) { - if (!logicalType.equals(ColumnType.NESTED_DATA) && fields.size() == 1 && rootFieldPath.equals(fields.get(0))) { + final TKeyDictionary fields = fieldsSupplier.get(); + if (!logicalType.equals(ColumnType.NESTED_DATA) && fields.size() == 1 && rootFieldPath.equals(StringUtils.fromUtf8(fields.get(0)))) { return makeVectorObjectSelector( Collections.emptyList(), offset @@ -446,7 +450,8 @@ public int getMaxVectorSize() @Override public VectorValueSelector makeVectorValueSelector(ReadableVectorOffset offset) { - if (!logicalType.equals(ColumnType.NESTED_DATA) && fields.size() == 1 && rootFieldPath.equals(fields.get(0))) { + final TKeyDictionary fields = fieldsSupplier.get(); + if (!logicalType.equals(ColumnType.NESTED_DATA) && fields.size() == 1 && rootFieldPath.equals(StringUtils.fromUtf8(fields.get(0)))) { return makeVectorValueSelector( Collections.emptyList(), offset @@ -477,9 +482,10 @@ public DimensionSelector makeDimensionSelector( ExtractionFn fn ) { + final TKeyDictionary fields = fieldsSupplier.get(); final String field = getField(path); Preconditions.checkNotNull(field, "Null field"); - final int fieldIndex = fields.indexOf(field); + final int fieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(field)); if (fieldIndex >= 0) { DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(field, fieldIndex).getColumn(); return col.makeDimensionSelector(readableOffset, fn); @@ -487,7 +493,7 @@ public DimensionSelector makeDimensionSelector( if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { final NestedPathPart lastPath = path.get(path.size() - 1); final String arrayField = getField(path.subList(0, path.size() - 1)); - final int arrayFieldIndex = fields.indexOf(arrayField); + final int arrayFieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(arrayField)); if (arrayFieldIndex >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); if (elementNumber < 0) { @@ -529,10 +535,11 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public ColumnValueSelector makeColumnValueSelector(List path, ReadableOffset readableOffset) { + final TKeyDictionary fields = fieldsSupplier.get(); final String field = getField(path); - Preconditions.checkNotNull(field, "Null field"); - final int fieldIndex = fields.indexOf(field); + Preconditions.checkNotNull(field, "Null field"); + final int fieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(field)); if (fieldIndex >= 0) { BaseColumn col = getColumnHolder(field, fieldIndex).getColumn(); return col.makeColumnValueSelector(readableOffset); @@ -540,7 +547,7 @@ public ColumnValueSelector makeColumnValueSelector(List path, if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { final NestedPathPart lastPath = path.get(path.size() - 1); final String arrayField = getField(path.subList(0, path.size() - 1)); - final int arrayFieldIndex = fields.indexOf(arrayField); + final int arrayFieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(arrayField)); if (arrayFieldIndex >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); if (elementNumber < 0) { @@ -620,9 +627,10 @@ public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector ReadableVectorOffset readableOffset ) { + final TKeyDictionary fields = fieldsSupplier.get(); final String field = getField(path); Preconditions.checkNotNull(field, "Null field"); - final int fieldIndex = fields.indexOf(field); + final int fieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(field)); if (fieldIndex >= 0) { DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(field, fieldIndex).getColumn(); return col.makeSingleValueDimensionVectorSelector(readableOffset); @@ -634,9 +642,10 @@ public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector @Override public VectorObjectSelector makeVectorObjectSelector(List path, ReadableVectorOffset readableOffset) { + final TKeyDictionary fields = fieldsSupplier.get(); final String field = getField(path); Preconditions.checkNotNull(field, "Null field"); - final int fieldIndex = fields.indexOf(field); + final int fieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(field)); if (fieldIndex >= 0) { BaseColumn col = getColumnHolder(field, fieldIndex).getColumn(); return col.makeVectorObjectSelector(readableOffset); @@ -644,7 +653,7 @@ public VectorObjectSelector makeVectorObjectSelector(List path, if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { final NestedPathPart lastPath = path.get(path.size() - 1); final String arrayField = getField(path.subList(0, path.size() - 1)); - final int arrayFieldIndex = fields.indexOf(arrayField); + final int arrayFieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(arrayField)); if (arrayFieldIndex >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); if (elementNumber < 0) { @@ -707,9 +716,10 @@ public int getCurrentVectorSize() @Override public VectorValueSelector makeVectorValueSelector(List path, ReadableVectorOffset readableOffset) { + final TKeyDictionary fields = fieldsSupplier.get(); final String field = getField(path); Preconditions.checkNotNull(field, "Null field"); - final int fieldIndex = fields.indexOf(field); + final int fieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(field)); if (fieldIndex >= 0) { BaseColumn col = getColumnHolder(field, fieldIndex).getColumn(); return col.makeVectorValueSelector(readableOffset); @@ -717,7 +727,7 @@ public VectorValueSelector makeVectorValueSelector(List path, Re if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { final NestedPathPart lastPath = path.get(path.size() - 1); final String arrayField = getField(path.subList(0, path.size() - 1)); - final int arrayFieldIndex = fields.indexOf(arrayField); + final int arrayFieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(arrayField)); if (arrayFieldIndex >= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); if (elementNumber < 0) { @@ -846,12 +856,13 @@ public int getCurrentVectorSize() @Override public Set getColumnTypes(List path) { + final TKeyDictionary fields = fieldsSupplier.get(); String field = getField(path); - int index = fields.indexOf(field); + int index = fields.indexOf(StringUtils.toUtf8ByteBuffer(field)); if (index < 0) { if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { final String arrayField = getField(path.subList(0, path.size() - 1)); - index = fields.indexOf(arrayField); + index = fields.indexOf(StringUtils.toUtf8ByteBuffer(arrayField)); } if (index < 0) { return null; @@ -874,8 +885,9 @@ public Set getColumnTypes(List path) @Override public ColumnHolder getColumnHolder(List path) { + final TKeyDictionary fields = fieldsSupplier.get(); final String field = getField(path); - final int fieldIndex = fields.indexOf(field); + final int fieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(field)); return getColumnHolder(field, fieldIndex); } @@ -883,14 +895,15 @@ public ColumnHolder getColumnHolder(List path) @Override public ColumnIndexSupplier getColumnIndexSupplier(List path) { + final TKeyDictionary fields = fieldsSupplier.get(); final String field = getField(path); - int fieldIndex = fields.indexOf(field); + int fieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(field)); if (fieldIndex >= 0) { return getColumnHolder(field, fieldIndex).getIndexSupplier(); } if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) { final String arrayField = getField(path.subList(0, path.size() - 1)); - final int arrayFieldIndex = fields.indexOf(arrayField); + final int arrayFieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(arrayField)); if (arrayFieldIndex >= 0) { return NoIndexesColumnIndexSupplier.getInstance(); } @@ -901,8 +914,9 @@ public ColumnIndexSupplier getColumnIndexSupplier(List path) @Override public boolean isNumeric(List path) { + final TKeyDictionary fields = fieldsSupplier.get(); final String field = getField(path); - final int fieldIndex = fields.indexOf(field); + final int fieldIndex = fields.indexOf(StringUtils.toUtf8ByteBuffer(field)); if (fieldIndex < 0) { return true; } 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 99f68dbffeef..c20541c87891 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 @@ -33,7 +33,6 @@ 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.segment.IndexSpec; import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSerializer; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.data.FixedIndexedIntWriter; @@ -77,7 +76,7 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter protected final SegmentWriteOutMedium segmentWriteOutMedium; protected final String columnName; protected final String fieldName; - protected final IndexSpec indexSpec; + protected final NestedCommonFormatColumnFormatSpec columnFormatSpec; protected final DictionaryIdLookup globalDictionaryIdLookup; protected final LocalDimensionDictionary localDictionary = new LocalDimensionDictionary(); @@ -97,14 +96,14 @@ protected GlobalDictionaryEncodedFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, DictionaryIdLookup globalDictionaryIdLookup ) { this.columnName = columnName; this.fieldName = fieldName; + this.columnFormatSpec = columnFormatSpec; this.segmentWriteOutMedium = segmentWriteOutMedium; - this.indexSpec = indexSpec; this.globalDictionaryIdLookup = globalDictionaryIdLookup; } @@ -205,14 +204,14 @@ public void writeTo(int finalRowCount, FileSmoosher smoosher) throws IOException GenericIndexedWriter bitmapIndexWriter = new GenericIndexedWriter<>( tmpWriteoutMedium, columnName, - indexSpec.getBitmapSerdeFactory().getObjectStrategy() + columnFormatSpec.getBitmapEncoding().getObjectStrategy() ); bitmapIndexWriter.open(); bitmapIndexWriter.setObjectsNotSorted(); GenericIndexedWriter arrayElementIndexWriter = new GenericIndexedWriter<>( tmpWriteoutMedium, columnName, - indexSpec.getBitmapSerdeFactory().getObjectStrategy() + columnFormatSpec.getBitmapEncoding().getObjectStrategy() ); arrayElementIndexWriter.open(); arrayElementIndexWriter.setObjectsNotSorted(); @@ -233,13 +232,13 @@ public void writeTo(int finalRowCount, FileSmoosher smoosher) throws IOException sortedDictionaryWriter.write(globalId); final int unsortedId = globalToUnsorted.get(globalId); unsortedToSorted[unsortedId] = index; - bitmaps[index] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); + bitmaps[index] = columnFormatSpec.getBitmapEncoding().getBitmapFactory().makeEmptyMutableBitmap(); } for (Int2ObjectMap.Entry arrayElement : arrayElements.int2ObjectEntrySet()) { arrayElementDictionaryWriter.write(arrayElement.getIntKey()); arrayElementIndexWriter.write( - indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(arrayElement.getValue()) + columnFormatSpec.getBitmapEncoding().getBitmapFactory().makeImmutableBitmap(arrayElement.getValue()) ); } @@ -257,7 +256,7 @@ public void writeTo(int finalRowCount, FileSmoosher smoosher) throws IOException for (int i = 0; i < bitmaps.length; i++) { final MutableBitmap bitmap = bitmaps[i]; bitmapIndexWriter.write( - indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(bitmap) + columnFormatSpec.getBitmapEncoding().getBitmapFactory().makeImmutableBitmap(bitmap) ); bitmaps[i] = null; // Reclaim memory } @@ -308,14 +307,14 @@ public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws I public void openColumnSerializer(SegmentWriteOutMedium medium, int maxId) throws IOException { - if (indexSpec.getDimensionCompression() != CompressionStrategy.UNCOMPRESSED) { + if (columnFormatSpec.getDictionaryEncodedColumnCompression() != CompressionStrategy.UNCOMPRESSED) { this.version = DictionaryEncodedColumnPartSerde.VERSION.COMPRESSED; encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create( fieldName, medium, columnName, maxId, - indexSpec.getDimensionCompression(), + columnFormatSpec.getDictionaryEncodedColumnCompression(), fieldResourceCloser ); } else { 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 index fe65fc904447..0bf0395fa0e3 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java @@ -90,12 +90,19 @@ class Format implements ColumnFormat private final ColumnType logicalType; private final boolean hasNulls; private final boolean enforceLogicalType; - - public Format(ColumnType logicalType, boolean hasNulls, boolean enforceLogicalType) + private final NestedCommonFormatColumnFormatSpec columnFormatSpec; + + public Format( + ColumnType logicalType, + boolean hasNulls, + boolean enforceLogicalType, + NestedCommonFormatColumnFormatSpec columnFormatSpec + ) { this.logicalType = logicalType; this.hasNulls = hasNulls; this.enforceLogicalType = enforceLogicalType; + this.columnFormatSpec = columnFormatSpec; } @Override @@ -107,13 +114,13 @@ public ColumnType getLogicalType() @Override public DimensionHandler getColumnHandler(String columnName) { - return new NestedCommonFormatColumnHandler(columnName, enforceLogicalType ? logicalType : null); + return new NestedCommonFormatColumnHandler(columnName, enforceLogicalType ? logicalType : null, columnFormatSpec); } @Override public DimensionSchema getColumnSchema(String columnName) { - return new AutoTypeColumnSchema(columnName, enforceLogicalType ? logicalType : null); + return new AutoTypeColumnSchema(columnName, enforceLogicalType ? logicalType : null, columnFormatSpec); } @Override @@ -125,10 +132,11 @@ public ColumnFormat merge(@Nullable ColumnFormat otherFormat) if (otherFormat instanceof Format) { final Format other = (Format) otherFormat; + // todo (clint): actually merge columnFormatSpec, maybe if (!getLogicalType().equals(other.getLogicalType())) { - return new Format(ColumnType.NESTED_DATA, hasNulls || other.hasNulls, false); + return new Format(ColumnType.NESTED_DATA, hasNulls || other.hasNulls, false, columnFormatSpec); } - return new Format(logicalType, hasNulls || other.hasNulls, enforceLogicalType || other.enforceLogicalType); + return new Format(logicalType, hasNulls || other.hasNulls, enforceLogicalType || other.enforceLogicalType, columnFormatSpec); } throw new ISE( "Cannot merge columns of type[%s] and format[%s] and with [%s] and [%s]", diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java new file mode 100644 index 000000000000..bd33793f0d6e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java @@ -0,0 +1,392 @@ +/* + * 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.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.column.StringEncodingStrategy; +import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.CompressionFactory; +import org.apache.druid.segment.data.CompressionStrategy; + +import javax.annotation.Nullable; +import java.util.Objects; + +public class NestedCommonFormatColumnFormatSpec +{ + public static Builder builder() + { + return new Builder(); + } + + public static NestedCommonFormatColumnFormatSpec getEffectiveFormatSpec( + @Nullable NestedCommonFormatColumnFormatSpec columnFormatSpec, + IndexSpec indexSpec + ) + { + return Objects.requireNonNullElseGet( + columnFormatSpec, + () -> NestedCommonFormatColumnFormatSpec.builder() + .setObjectKeysEncoding(StringEncodingStrategy.UTF8_STRATEGY) + .setObjectStorageEncoding(ObjectStorageEncoding.SMILE) + .build() + ).getEffectiveSpec(indexSpec); + } + + @Nullable + @JsonProperty + private final StringEncodingStrategy objectKeysEncoding; + @Nullable + @JsonProperty + private final ObjectStorageEncoding objectStorageEncoding; + @Nullable + @JsonProperty + private final CompressionStrategy objectStorageCompression; + @Nullable + @JsonProperty + private final StringEncodingStrategy stringDictionaryEncoding; + @Nullable + @JsonProperty + private final CompressionStrategy dictionaryEncodedColumnCompression; + @Nullable + @JsonProperty + private final CompressionFactory.LongEncodingStrategy longColumnEncoding; + @Nullable + @JsonProperty + private final CompressionStrategy longColumnCompression; + @Nullable + @JsonProperty + private final CompressionStrategy doubleColumnCompression; + @Nullable + @JsonProperty + private final BitmapSerdeFactory bitmapEncoding; + + @JsonCreator + public NestedCommonFormatColumnFormatSpec( + @JsonProperty("objectKeysEncoding") @Nullable StringEncodingStrategy objectKeysEncoding, + @JsonProperty("objectStorageEncoding") @Nullable ObjectStorageEncoding objectStorageEncoding, + @JsonProperty("objectStorageCompression") @Nullable CompressionStrategy objectStorageCompression, + @JsonProperty("stringDictionaryEncoding") @Nullable StringEncodingStrategy stringDictionaryEncoding, + @JsonProperty("dictionaryEncodedColumnCompression") @Nullable CompressionStrategy dictionaryEncodedColumnCompression, + @JsonProperty("longColumnEncoding") @Nullable CompressionFactory.LongEncodingStrategy longColumnEncoding, + @JsonProperty("longColumnCompression") @Nullable CompressionStrategy longColumnCompression, + @JsonProperty("doubleColumnCompression") @Nullable CompressionStrategy doubleColumnCompression, + @JsonProperty("bitmapEncoding") @Nullable BitmapSerdeFactory bitmapEncoding + ) + { + this.objectKeysEncoding = objectKeysEncoding; + this.objectStorageEncoding = objectStorageEncoding; + this.objectStorageCompression = objectStorageCompression; + this.stringDictionaryEncoding = stringDictionaryEncoding; + this.dictionaryEncodedColumnCompression = dictionaryEncodedColumnCompression; + this.longColumnEncoding = longColumnEncoding; + this.longColumnCompression = longColumnCompression; + this.doubleColumnCompression = doubleColumnCompression; + this.bitmapEncoding = bitmapEncoding; + } + + public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) + { + if (bitmapEncoding != null && !bitmapEncoding.equals(indexSpec.getBitmapSerdeFactory())) { + throw new ISE( + "bitmapEncoding[%s] does not match indexSpec.bitmap[%s]", + bitmapEncoding, + indexSpec.getBitmapSerdeFactory() + ); + } + return new NestedCommonFormatColumnFormatSpec( + objectKeysEncoding != null ? objectKeysEncoding : StringEncodingStrategy.DEFAULT, + objectStorageEncoding != null ? objectStorageEncoding : ObjectStorageEncoding.SMILE, + objectStorageCompression != null + ? objectStorageCompression + : indexSpec.getJsonCompression() != null ? indexSpec.getJsonCompression() : CompressionStrategy.LZ4, + stringDictionaryEncoding != null ? stringDictionaryEncoding : indexSpec.getStringDictionaryEncoding(), + dictionaryEncodedColumnCompression != null + ? dictionaryEncodedColumnCompression + : indexSpec.getDimensionCompression(), + longColumnEncoding != null ? longColumnEncoding : indexSpec.getLongEncoding(), + longColumnCompression != null ? longColumnCompression : indexSpec.getMetricCompression(), + doubleColumnCompression != null ? doubleColumnCompression : indexSpec.getMetricCompression(), + bitmapEncoding != null ? bitmapEncoding : indexSpec.getBitmapSerdeFactory() + ); + } + + @Nullable + @JsonProperty + public StringEncodingStrategy getObjectKeysEncoding() + { + return objectKeysEncoding; + } + + @Nullable + @JsonProperty + public ObjectStorageEncoding getObjectStorageEncoding() + { + return objectStorageEncoding; + } + + @Nullable + @JsonProperty + public CompressionStrategy getObjectStorageCompression() + { + return objectStorageCompression; + } + + @Nullable + @JsonProperty + public StringEncodingStrategy getStringDictionaryEncoding() + { + return stringDictionaryEncoding; + } + + @Nullable + @JsonProperty + public CompressionStrategy getDictionaryEncodedColumnCompression() + { + return dictionaryEncodedColumnCompression; + } + + @Nullable + @JsonProperty + public CompressionFactory.LongEncodingStrategy getLongColumnEncoding() + { + return longColumnEncoding; + } + + @Nullable + @JsonProperty + public CompressionStrategy getLongColumnCompression() + { + return longColumnCompression; + } + + @Nullable + @JsonProperty + public CompressionStrategy getDoubleColumnCompression() + { + return doubleColumnCompression; + } + + @Nullable + @JsonProperty + public BitmapSerdeFactory getBitmapEncoding() + { + return bitmapEncoding; + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) { + return false; + } + NestedCommonFormatColumnFormatSpec that = (NestedCommonFormatColumnFormatSpec) o; + return Objects.equals(objectKeysEncoding, that.objectKeysEncoding) + && objectStorageEncoding == that.objectStorageEncoding + && objectStorageCompression == that.objectStorageCompression + && Objects.equals(stringDictionaryEncoding, that.stringDictionaryEncoding) + && dictionaryEncodedColumnCompression == that.dictionaryEncodedColumnCompression + && longColumnEncoding == that.longColumnEncoding + && longColumnCompression == that.longColumnCompression + && doubleColumnCompression == that.doubleColumnCompression + && Objects.equals(bitmapEncoding, that.bitmapEncoding); + } + + @Override + public int hashCode() + { + return Objects.hash( + objectKeysEncoding, + objectStorageEncoding, + objectStorageCompression, + stringDictionaryEncoding, + dictionaryEncodedColumnCompression, + longColumnEncoding, + longColumnCompression, + doubleColumnCompression, + bitmapEncoding + ); + } + + @Override + public String toString() + { + return "NestedCommonFormatColumnFormatSpec{" + + "objectKeysEncoding=" + objectKeysEncoding + + ", objectStorageEncoding=" + objectStorageEncoding + + ", objectStorageCompression=" + objectStorageCompression + + ", stringDictionaryEncoding=" + stringDictionaryEncoding + + ", dictionaryEncodedColumnCompression=" + dictionaryEncodedColumnCompression + + ", longColumnEncoding=" + longColumnEncoding + + ", longColumnCompression=" + longColumnCompression + + ", doubleColumnCompression=" + doubleColumnCompression + + ", bitmapEncoding=" + bitmapEncoding + + '}'; + } + + public static class Builder + { + @Nullable + private StringEncodingStrategy objectKeysEncoding; + @Nullable + private ObjectStorageEncoding objectStorageEncoding; + @Nullable + private CompressionStrategy objectStorageCompression; + @Nullable + private StringEncodingStrategy stringDictionaryEncoding; + @Nullable + private CompressionStrategy dictionaryEncodedColumnCompression; + @Nullable + private CompressionFactory.LongEncodingStrategy longColumnEncoding; + @Nullable + private CompressionStrategy longColumnCompression; + @Nullable + private CompressionStrategy doubleColumnCompression; + @Nullable + private BitmapSerdeFactory bitmapEncoding; + + @Nullable + public StringEncodingStrategy getObjectKeysEncoding() + { + return objectKeysEncoding; + } + + public Builder setObjectKeysEncoding(@Nullable StringEncodingStrategy objectKeysEncoding) + { + this.objectKeysEncoding = objectKeysEncoding; + return this; + } + + @Nullable + public ObjectStorageEncoding getObjectStorageEncoding() + { + return objectStorageEncoding; + } + + public Builder setObjectStorageEncoding(@Nullable ObjectStorageEncoding objectStorageEncoding) + { + this.objectStorageEncoding = objectStorageEncoding; + return this; + } + + @Nullable + public CompressionStrategy getObjectStorageCompression() + { + return objectStorageCompression; + } + + public Builder setObjectStorageCompression(@Nullable CompressionStrategy objectStorageCompression) + { + this.objectStorageCompression = objectStorageCompression; + return this; + } + + @Nullable + public StringEncodingStrategy getStringDictionaryEncoding() + { + return stringDictionaryEncoding; + } + + public Builder setStringDictionaryEncoding(@Nullable StringEncodingStrategy stringDictionaryEncoding) + { + this.stringDictionaryEncoding = stringDictionaryEncoding; + return this; + } + + @Nullable + public CompressionStrategy getDictionaryEncodedColumnCompression() + { + return dictionaryEncodedColumnCompression; + } + + public Builder setDictionaryEncodedColumnCompression( + @Nullable CompressionStrategy dictionaryEncodedColumnCompression + ) + { + this.dictionaryEncodedColumnCompression = dictionaryEncodedColumnCompression; + return this; + } + + @Nullable + public CompressionFactory.LongEncodingStrategy getLongColumnEncoding() + { + return longColumnEncoding; + } + + public Builder setLongColumnEncoding(@Nullable CompressionFactory.LongEncodingStrategy longColumnEncoding) + { + this.longColumnEncoding = longColumnEncoding; + return this; + } + + @Nullable + public CompressionStrategy getLongColumnCompression() + { + return longColumnCompression; + } + + public Builder setLongColumnCompression(@Nullable CompressionStrategy longColumnCompression) + { + this.longColumnCompression = longColumnCompression; + return this; + } + + @Nullable + public CompressionStrategy getDoubleColumnCompression() + { + return doubleColumnCompression; + } + + public Builder setDoubleColumnCompression(@Nullable CompressionStrategy doubleColumnCompression) + { + this.doubleColumnCompression = doubleColumnCompression; + return this; + } + + @Nullable + public BitmapSerdeFactory getBitmapEncoding() + { + return bitmapEncoding; + } + + public Builder setBitmapEncoding(@Nullable BitmapSerdeFactory bitmapEncoding) + { + this.bitmapEncoding = bitmapEncoding; + return this; + } + + public NestedCommonFormatColumnFormatSpec build() + { + return new NestedCommonFormatColumnFormatSpec( + objectKeysEncoding, + objectStorageEncoding, + objectStorageCompression, + stringDictionaryEncoding, + dictionaryEncodedColumnCompression, + longColumnEncoding, + longColumnCompression, + doubleColumnCompression, + bitmapEncoding + ); + } + } +} 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 3dcef1d2d85b..e6a62e57d930 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 @@ -30,14 +30,12 @@ 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.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.FrontCodedIntArrayIndexedWriter; @@ -81,8 +79,7 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ private final String name; private final SegmentWriteOutMedium segmentWriteOutMedium; - private final IndexSpec indexSpec; - @SuppressWarnings("unused") + private final NestedCommonFormatColumnFormatSpec columnFormatSpec; private final Closer closer; private final StructuredDataProcessor fieldProcessor = new StructuredDataProcessor() @@ -141,7 +138,7 @@ public ProcessedValue processArrayField( private DictionaryIdLookup globalDictionaryIdLookup; private SortedMap fields; - private GenericIndexedWriter fieldsWriter; + private DictionaryWriter fieldsWriter; private FieldTypeInfo.Writer fieldsInfoWriter; private DictionaryWriter dictionaryWriter; private FixedIndexedWriter longDictionaryWriter; @@ -160,14 +157,14 @@ public ProcessedValue processArrayField( public NestedDataColumnSerializer( String name, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, SegmentWriteOutMedium segmentWriteOutMedium, Closer closer ) { this.name = name; this.segmentWriteOutMedium = segmentWriteOutMedium; - this.indexSpec = indexSpec; + this.columnFormatSpec = columnFormatSpec; this.closer = closer; } @@ -207,7 +204,7 @@ public void openDictionaryWriter(File segmentBaseDir) throws IOException fieldsInfoWriter.open(); dictionaryWriter = StringEncodingStrategies.getStringDictionaryWriter( - indexSpec.getStringDictionaryEncoding(), + columnFormatSpec.getStringDictionaryEncoding(), segmentWriteOutMedium, name ); @@ -255,17 +252,17 @@ public void open() throws IOException rawWriter = new CompressedVariableSizedBlobColumnSerializer( ColumnSerializerUtils.getInternalFileName(name, RAW_FILE_NAME), segmentWriteOutMedium, - indexSpec.getJsonCompression() != null ? indexSpec.getJsonCompression() : CompressionStrategy.LZ4 + columnFormatSpec.getObjectStorageCompression() ); rawWriter.open(); nullBitmapWriter = new ByteBufferWriter<>( segmentWriteOutMedium, - indexSpec.getBitmapSerdeFactory().getObjectStrategy() + columnFormatSpec.getBitmapEncoding().getObjectStrategy() ); nullBitmapWriter.open(); - nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); + nullRowsBitmap = columnFormatSpec.getBitmapEncoding().getBitmapFactory().makeEmptyMutableBitmap(); } @Override @@ -287,7 +284,7 @@ public void serializeFields(SortedMap fiel name, fieldFileName, segmentWriteOutMedium, - indexSpec, + columnFormatSpec, globalDictionaryIdLookup ); } else if (Types.is(type, ValueType.LONG)) { @@ -295,7 +292,7 @@ public void serializeFields(SortedMap fiel name, fieldFileName, segmentWriteOutMedium, - indexSpec, + columnFormatSpec, globalDictionaryIdLookup ); } else if (Types.is(type, ValueType.DOUBLE)) { @@ -303,7 +300,7 @@ public void serializeFields(SortedMap fiel name, fieldFileName, segmentWriteOutMedium, - indexSpec, + columnFormatSpec, globalDictionaryIdLookup ); } else if (Types.is(type, ValueType.ARRAY)) { @@ -311,7 +308,7 @@ public void serializeFields(SortedMap fiel name, fieldFileName, segmentWriteOutMedium, - indexSpec, + columnFormatSpec, globalDictionaryIdLookup ); } else { @@ -322,7 +319,7 @@ public void serializeFields(SortedMap fiel name, fieldFileName, segmentWriteOutMedium, - indexSpec, + columnFormatSpec, globalDictionaryIdLookup ); } 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 71072dfad99a..f447595ec805 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 @@ -33,7 +33,6 @@ import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.FrontCodedIntArrayIndexed; -import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.VByte; import org.apache.druid.segment.index.SimpleImmutableBitmapIndex; @@ -66,7 +65,7 @@ public static NestedDataColumnSupplier read( if (version == NestedCommonFormatColumnSerializer.V0) { try { final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); - final GenericIndexed fields; + final Supplier> fieldsSupplier; final FieldTypeInfo fieldInfo; final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier; final ImmutableBitmap nullValues; @@ -75,8 +74,8 @@ public static NestedDataColumnSupplier read( final Supplier> doubleDictionarySupplier; final Supplier arrayDictionarySupplier; - fields = GenericIndexed.read(bb, GenericIndexed.STRING_STRATEGY, mapper); - fieldInfo = FieldTypeInfo.read(bb, fields.size()); + fieldsSupplier = StringEncodingStrategies.getStringDictionarySupplier(mapper, bb, byteOrder); + fieldInfo = FieldTypeInfo.read(bb, fieldsSupplier.get().size()); if (parent != null) { stringDictionarySupplier = parent.stringDictionarySupplier; @@ -158,7 +157,7 @@ public static NestedDataColumnSupplier read( return new NestedDataColumnSupplier( columnName, - fields, + fieldsSupplier, fieldInfo, compressedRawColumnSupplier, nullValues, @@ -182,7 +181,7 @@ public static NestedDataColumnSupplier read( } private final String columnName; - private final GenericIndexed fields; + private final Supplier> fieldSupplier; private final FieldTypeInfo fieldInfo; private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier; private final ImmutableBitmap nullValues; @@ -200,7 +199,7 @@ public static NestedDataColumnSupplier read( private NestedDataColumnSupplier( String columnName, - GenericIndexed fields, + Supplier> fieldSupplier, FieldTypeInfo fieldInfo, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, @@ -216,7 +215,7 @@ private NestedDataColumnSupplier( ) { this.columnName = columnName; - this.fields = fields; + this.fieldSupplier = fieldSupplier; this.fieldInfo = fieldInfo; this.compressedRawColumnSupplier = compressedRawColumnSupplier; this.nullValues = nullValues; @@ -240,7 +239,7 @@ public NestedCommonFormatColumn get() columnConfig, compressedRawColumnSupplier, nullValues, - fields, + fieldSupplier, fieldInfo, stringDictionarySupplier, longDictionarySupplier, 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 index 4b118e334399..9698170018df 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4.java @@ -23,6 +23,7 @@ import com.google.common.base.Supplier; 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.segment.IndexMerger; import org.apache.druid.segment.column.ColumnBuilder; @@ -80,7 +81,7 @@ public static NestedDataColumnSupplierV4 read( try { final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); final ComplexColumnMetadata metadata; - final GenericIndexed fields; + final GenericIndexed fields; final FieldTypeInfo fieldInfo; final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier; final ImmutableBitmap nullValues; @@ -95,7 +96,7 @@ public static NestedDataColumnSupplierV4 read( IndexMerger.SERIALIZER_UTILS.readString(bb), ComplexColumnMetadata.class ); - fields = GenericIndexed.read(bb, GenericIndexed.STRING_STRATEGY, mapper); + fields = GenericIndexed.read(bb, GenericIndexed.UTF8_STRATEGY, mapper); fieldInfo = FieldTypeInfo.read(bb, fields.size()); if (fields.size() == 0) { @@ -103,8 +104,8 @@ public static NestedDataColumnSupplierV4 read( // 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)))) + ((version == 0x03 && NestedPathFinder.JQ_PATH_ROOT.equals(StringUtils.fromUtf8(fields.get(0)))) || + ((version == 0x04 || version == 0x05) && NestedPathFinder.JSON_PATH_ROOT.equals(StringUtils.fromUtf8(fields.get(0))))) ) { simpleType = fieldInfo.getTypes(0).getSingleType(); } else { @@ -208,7 +209,7 @@ public static NestedDataColumnSupplierV4 read( private final byte version; private final String columnName; private final ColumnConfig columnConfig; - private final GenericIndexed fields; + private final GenericIndexed fields; private final FieldTypeInfo fieldInfo; private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier; private final ImmutableBitmap nullValues; @@ -228,7 +229,7 @@ private NestedDataColumnSupplierV4( byte version, String columnName, ColumnConfig columnConfig, - GenericIndexed fields, + GenericIndexed fields, FieldTypeInfo fieldInfo, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, @@ -324,7 +325,7 @@ private NestedDataColumnV5 makeV5() columnConfig, compressedRawColumnSupplier, nullValues, - fields, + fields::singleThreaded, fieldInfo, stringDictionarySupplier, longDictionarySupplier, 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 906ffe821e26..796eb6ea556b 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 @@ -36,7 +36,7 @@ import java.util.List; public final class NestedDataColumnV3> - extends CompressedNestedDataComplexColumn + extends CompressedNestedDataComplexColumn, TStringDictionary> { public NestedDataColumnV3( String columnName, @@ -44,7 +44,7 @@ public NestedDataColumnV3( ColumnConfig columnConfig, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, - GenericIndexed fields, + GenericIndexed fields, FieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, @@ -60,7 +60,7 @@ public NestedDataColumnV3( columnConfig, compressedRawColumnSupplier, nullValues, - fields, + fields::singleThreaded, fieldInfo, stringDictionary, longDictionarySupplier, 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 124c3fae1a0d..83487f1f5b2c 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 @@ -36,7 +36,7 @@ import java.util.List; public final class NestedDataColumnV4> - extends CompressedNestedDataComplexColumn + extends CompressedNestedDataComplexColumn, TStringDictionary> { public NestedDataColumnV4( String columnName, @@ -44,7 +44,7 @@ public NestedDataColumnV4( ColumnConfig columnConfig, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, - GenericIndexed fields, + GenericIndexed fields, FieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, @@ -60,7 +60,7 @@ public NestedDataColumnV4( columnConfig, compressedRawColumnSupplier, nullValues, - fields, + fields::singleThreaded, fieldInfo, stringDictionary, longDictionarySupplier, 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 9962f8b46330..bdc90b9d9b4a 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 @@ -28,7 +28,6 @@ import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.FrontCodedIntArrayIndexed; -import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.serde.ColumnSerializerUtils; @@ -45,8 +44,8 @@ * 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 class NestedDataColumnV5, TStringDictionary extends Indexed> + extends CompressedNestedDataComplexColumn { public NestedDataColumnV5( String columnName, @@ -54,7 +53,7 @@ public NestedDataColumnV5( ColumnConfig columnConfig, CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier, ImmutableBitmap nullValues, - GenericIndexed fields, + Supplier fields, FieldTypeInfo fieldInfo, Supplier stringDictionary, Supplier> longDictionarySupplier, 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 c18a23037565..63d52ebdad99 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,6 +23,7 @@ import it.unimi.dsi.fastutil.Hash; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.error.DruidException; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.NestedCommonFormatColumnHandler; @@ -234,7 +235,7 @@ public ColumnType getLogicalType() @Override public DimensionHandler getColumnHandler(String columnName) { - return new NestedCommonFormatColumnHandler(columnName, null); + return new NestedCommonFormatColumnHandler(columnName, null, BuiltInTypesModule.getDefaultNestedCommonFormatSpec()); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ObjectStorageEncoding.java b/processing/src/main/java/org/apache/druid/segment/nested/ObjectStorageEncoding.java new file mode 100644 index 000000000000..3b1c823dbe44 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/nested/ObjectStorageEncoding.java @@ -0,0 +1,45 @@ +/* + * 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.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import org.apache.druid.java.util.common.StringUtils; + +public enum ObjectStorageEncoding +{ + SMILE; + + @JsonValue + @Override + public String toString() + { + return StringUtils.toLowerCase(this.name()); + } + + @JsonCreator + public static ObjectStorageEncoding fromString(String name) + { + if (name == null) { + return SMILE; + } + return valueOf(StringUtils.toUpperCase(name)); + } +} 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 index 52d798149fd2..c315ca2cf462 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExpressionType; -import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.ColumnarDoublesSerializer; import org.apache.druid.segment.data.CompressionFactory; @@ -47,12 +46,12 @@ public class ScalarDoubleColumnSerializer extends ScalarNestedCommonFormatColumn public ScalarDoubleColumnSerializer( String name, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, SegmentWriteOutMedium segmentWriteOutMedium, Closer closer ) { - super(name, indexSpec, segmentWriteOutMedium, closer); + super(name, columnFormatSpec, segmentWriteOutMedium, closer); } @Override @@ -96,7 +95,7 @@ protected void openValueColumnSerializer() throws IOException segmentWriteOutMedium, StringUtils.format("%s.double_column", name), ByteOrder.nativeOrder(), - indexSpec.getDimensionCompression(), + columnFormatSpec.getDoubleColumnCompression(), segmentWriteOutMedium.getCloser() ); doublesSerializer.open(); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleFieldColumnWriter.java index 09e8dc121c8e..b343645222d8 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleFieldColumnWriter.java @@ -22,7 +22,6 @@ import com.google.common.primitives.Ints; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; -import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.ColumnarDoublesSerializer; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; @@ -45,11 +44,11 @@ public ScalarDoubleFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, DictionaryIdLookup globalDictionaryIdLookup ) { - super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); + super(columnName, fieldName, segmentWriteOutMedium, columnFormatSpec, globalDictionaryIdLookup); } @Override @@ -77,7 +76,7 @@ public void openColumnSerializer(SegmentWriteOutMedium medium, int maxId) throws medium, StringUtils.format("%s.double_column", fieldName), ByteOrder.nativeOrder(), - indexSpec.getDimensionCompression(), + columnFormatSpec.getDoubleColumnCompression(), fieldResourceCloser ); doublesSerializer.open(); 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 index e62c1ebe8aa2..5320156c7cad 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExpressionType; -import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.ColumnarLongsSerializer; import org.apache.druid.segment.data.CompressionFactory; @@ -47,12 +46,12 @@ public class ScalarLongColumnSerializer extends ScalarNestedCommonFormatColumnSe public ScalarLongColumnSerializer( String name, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, SegmentWriteOutMedium segmentWriteOutMedium, Closer closer ) { - super(name, indexSpec, segmentWriteOutMedium, closer); + super(name, columnFormatSpec, segmentWriteOutMedium, closer); } @Override @@ -97,8 +96,8 @@ protected void openValueColumnSerializer() throws IOException segmentWriteOutMedium, StringUtils.format("%s.long_column", name), ByteOrder.nativeOrder(), - indexSpec.getLongEncoding(), - indexSpec.getDimensionCompression(), + columnFormatSpec.getLongColumnEncoding(), + columnFormatSpec.getLongColumnCompression(), segmentWriteOutMedium.getCloser() ); longsSerializer.open(); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongFieldColumnWriter.java index d9191c4e8050..0fcef9f31880 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongFieldColumnWriter.java @@ -22,7 +22,6 @@ import com.google.common.primitives.Ints; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; -import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.ColumnarLongsSerializer; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; @@ -45,11 +44,11 @@ public ScalarLongFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, DictionaryIdLookup globalDictionaryIdLookup ) { - super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); + super(columnName, fieldName, segmentWriteOutMedium, columnFormatSpec, globalDictionaryIdLookup); } @Override @@ -77,8 +76,8 @@ public void openColumnSerializer(SegmentWriteOutMedium medium, int maxId) throws medium, StringUtils.format("%s.long_column", fieldName), ByteOrder.nativeOrder(), - indexSpec.getLongEncoding(), - indexSpec.getDimensionCompression(), + columnFormatSpec.getLongColumnEncoding(), + columnFormatSpec.getLongColumnCompression(), fieldResourceCloser ); longsSerializer.open(); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java index dc18970ad799..8f971efd7333 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java @@ -29,7 +29,6 @@ import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSerializer; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.data.DictionaryWriter; @@ -50,7 +49,7 @@ public abstract class ScalarNestedCommonFormatColumnSerializer extends Nested protected final String name; protected final SegmentWriteOutMedium segmentWriteOutMedium; - protected final IndexSpec indexSpec; + protected final NestedCommonFormatColumnFormatSpec columnFormatSpec; @SuppressWarnings("unused") protected final Closer closer; @@ -67,14 +66,14 @@ public abstract class ScalarNestedCommonFormatColumnSerializer extends Nested public ScalarNestedCommonFormatColumnSerializer( String name, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, SegmentWriteOutMedium segmentWriteOutMedium, Closer closer ) { this.name = name; this.segmentWriteOutMedium = segmentWriteOutMedium; - this.indexSpec = indexSpec; + this.columnFormatSpec = columnFormatSpec; this.closer = closer; } @@ -186,7 +185,7 @@ public void writeTo( // easy iteration of the values it writes out, so that we could just build the bitmap indexes here instead of // doing both things String filenameBase = StringUtils.format("%s.forward_dim", name); - final CompressionStrategy compression = indexSpec.getDimensionCompression(); + final CompressionStrategy compression = columnFormatSpec.getDictionaryEncodedColumnCompression(); final CompressionStrategy compressionToUse; if (compression != CompressionStrategy.UNCOMPRESSED && compression != CompressionStrategy.NONE) { compressionToUse = compression; @@ -206,14 +205,14 @@ public void writeTo( final GenericIndexedWriter bitmapIndexWriter = new GenericIndexedWriter<>( segmentWriteOutMedium, name, - indexSpec.getBitmapSerdeFactory().getObjectStrategy() + columnFormatSpec.getBitmapEncoding().getObjectStrategy() ); bitmapIndexWriter.open(); bitmapIndexWriter.setObjectsNotSorted(); final MutableBitmap[] bitmaps; bitmaps = new MutableBitmap[getCardinality()]; for (int i = 0; i < bitmaps.length; i++) { - bitmaps[i] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); + bitmaps[i] = columnFormatSpec.getBitmapEncoding().getBitmapFactory().makeEmptyMutableBitmap(); } final IntIterator rows = intermediateValueWriter.getIterator(); @@ -227,7 +226,7 @@ public void writeTo( for (int i = 0; i < bitmaps.length; i++) { final MutableBitmap bitmap = bitmaps[i]; bitmapIndexWriter.write( - indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(bitmap) + columnFormatSpec.getBitmapEncoding().getBitmapFactory().makeImmutableBitmap(bitmap) ); bitmaps[i] = null; // Reclaim memory } 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 index 55a62512389b..f100f4d2e70f 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExpressionType; -import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.StringEncodingStrategies; import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.serde.ColumnSerializerUtils; @@ -43,12 +42,12 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn { public ScalarStringColumnSerializer( String name, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, SegmentWriteOutMedium segmentWriteOutMedium, Closer closer ) { - super(name, indexSpec, segmentWriteOutMedium, closer); + super(name, columnFormatSpec, segmentWriteOutMedium, closer); } @Override @@ -64,7 +63,7 @@ protected int processValue(@Nullable Object rawValue) public void openDictionaryWriter(File segmentBaseDir) throws IOException { dictionaryWriter = StringEncodingStrategies.getStringDictionaryWriter( - indexSpec.getStringDictionaryEncoding(), + columnFormatSpec.getStringDictionaryEncoding(), segmentWriteOutMedium, name ); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringFieldColumnWriter.java index b6ac6aba1698..6c04b371c450 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringFieldColumnWriter.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.nested; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; -import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; @@ -35,11 +34,11 @@ public ScalarStringFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, DictionaryIdLookup globalDictionaryIdLookup ) { - super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); + super(columnName, fieldName, segmentWriteOutMedium, columnFormatSpec, globalDictionaryIdLookup); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayFieldColumnWriter.java index 3166548abc14..fbf713cb8db1 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantArrayFieldColumnWriter.java @@ -22,7 +22,6 @@ import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; -import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; @@ -39,11 +38,11 @@ public VariantArrayFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, DictionaryIdLookup globalDictionaryIdLookup ) { - super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); + super(columnName, fieldName, segmentWriteOutMedium, columnFormatSpec, globalDictionaryIdLookup); } @Override @@ -68,7 +67,7 @@ int[] processValue(int row, Object value) 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() + (id) -> columnFormatSpec.getBitmapEncoding().getBitmapFactory().makeEmptyMutableBitmap() ).add(row); } return globalIds; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnSerializer.java index 0073e595dd73..3acd44e16b2d 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnSerializer.java @@ -33,7 +33,6 @@ 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.column.ColumnType; import org.apache.druid.segment.column.StringEncodingStrategies; import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSerializer; @@ -67,7 +66,7 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer private final String name; private final SegmentWriteOutMedium segmentWriteOutMedium; - private final IndexSpec indexSpec; + private final NestedCommonFormatColumnFormatSpec columnFormatSpec; @SuppressWarnings("unused") private final Closer closer; private DictionaryIdLookup dictionaryIdLookup; @@ -93,7 +92,7 @@ public VariantColumnSerializer( String name, @Nullable ColumnType logicalType, @Nullable Byte variantTypeSetByte, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, SegmentWriteOutMedium segmentWriteOutMedium, Closer closer ) @@ -102,7 +101,7 @@ public VariantColumnSerializer( this.expectedExpressionType = logicalType != null ? ExpressionType.fromColumnTypeStrict(logicalType) : null; this.variantTypeSetByte = variantTypeSetByte; this.segmentWriteOutMedium = segmentWriteOutMedium; - this.indexSpec = indexSpec; + this.columnFormatSpec = columnFormatSpec; this.closer = closer; } @@ -136,7 +135,7 @@ public boolean hasNulls() public void openDictionaryWriter(File segmentBaseDir) throws IOException { dictionaryWriter = StringEncodingStrategies.getStringDictionaryWriter( - indexSpec.getStringDictionaryEncoding(), + columnFormatSpec.getStringDictionaryEncoding(), segmentWriteOutMedium, name ); @@ -311,7 +310,7 @@ private void closeForWrite() throws IOException + dictionaryIdLookup.getLongCardinality() + dictionaryIdLookup.getDoubleCardinality(); final int cardinality = scalarCardinality + dictionaryIdLookup.getArrayCardinality(); - final CompressionStrategy compression = indexSpec.getDimensionCompression(); + final CompressionStrategy compression = columnFormatSpec.getDictionaryEncodedColumnCompression(); final CompressionStrategy compressionToUse; if (compression != CompressionStrategy.UNCOMPRESSED && compression != CompressionStrategy.NONE) { compressionToUse = compression; @@ -332,19 +331,19 @@ private void closeForWrite() throws IOException final GenericIndexedWriter bitmapIndexWriter = new GenericIndexedWriter<>( segmentWriteOutMedium, name, - indexSpec.getBitmapSerdeFactory().getObjectStrategy() + columnFormatSpec.getBitmapEncoding().getObjectStrategy() ); bitmapIndexWriter.open(); bitmapIndexWriter.setObjectsNotSorted(); final MutableBitmap[] bitmaps = new MutableBitmap[cardinality]; final MutableBitmap[] arrayElements = new MutableBitmap[scalarCardinality]; for (int i = 0; i < bitmaps.length; i++) { - bitmaps[i] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); + bitmaps[i] = columnFormatSpec.getBitmapEncoding().getBitmapFactory().makeEmptyMutableBitmap(); } final GenericIndexedWriter arrayElementIndexWriter = new GenericIndexedWriter<>( segmentWriteOutMedium, name + "_arrays", - indexSpec.getBitmapSerdeFactory().getObjectStrategy() + columnFormatSpec.getBitmapEncoding().getObjectStrategy() ); arrayElementIndexWriter.open(); arrayElementIndexWriter.setObjectsNotSorted(); @@ -363,7 +362,7 @@ private void closeForWrite() throws IOException for (int elementId : array) { MutableBitmap bitmap = arrayElements[elementId]; if (bitmap == null) { - bitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); + bitmap = columnFormatSpec.getBitmapEncoding().getBitmapFactory().makeEmptyMutableBitmap(); arrayElements[elementId] = bitmap; } bitmap.add(rowCount); @@ -374,9 +373,7 @@ private void closeForWrite() throws IOException for (int i = 0; i < bitmaps.length; i++) { final MutableBitmap bitmap = bitmaps[i]; - bitmapIndexWriter.write( - indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(bitmap) - ); + bitmapIndexWriter.write(columnFormatSpec.getBitmapEncoding().getBitmapFactory().makeImmutableBitmap(bitmap)); bitmaps[i] = null; // Reclaim memory } if (writeDictionary) { 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 2f3cb8d3fcb7..295d60152700 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 @@ -21,7 +21,6 @@ import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; -import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; @@ -36,11 +35,11 @@ public VariantFieldColumnWriter( String columnName, String fieldName, SegmentWriteOutMedium segmentWriteOutMedium, - IndexSpec indexSpec, + NestedCommonFormatColumnFormatSpec columnFormatSpec, DictionaryIdLookup globalDictionaryIdLookup ) { - super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup); + super(columnName, fieldName, segmentWriteOutMedium, columnFormatSpec, globalDictionaryIdLookup); } 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 index ae14d4c16f85..d6c9ae38e3d7 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java @@ -32,7 +32,9 @@ import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.nested.NestedCommonFormatColumn; +import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.nested.NestedDataColumnSupplier; +import org.apache.druid.segment.nested.ObjectStorageEncoding; import org.apache.druid.segment.nested.ScalarDoubleColumnAndIndexSupplier; import org.apache.druid.segment.nested.ScalarLongColumnAndIndexSupplier; import org.apache.druid.segment.nested.ScalarStringColumnAndIndexSupplier; @@ -80,7 +82,8 @@ public static NestedCommonFormatColumnPartSerde createDeserializer( @JsonProperty("isVariantType") boolean isVariantType, @JsonProperty("enforceLogicalType") boolean enforceLogicalType, @JsonProperty("byteOrder") ByteOrder byteOrder, - @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory + @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory, + @JsonProperty("columnFormatSpec") @Nullable NestedCommonFormatColumnFormatSpec columnFormatSpec ) { return new NestedCommonFormatColumnPartSerde( @@ -90,6 +93,7 @@ public static NestedCommonFormatColumnPartSerde createDeserializer( enforceLogicalType, byteOrder, bitmapSerdeFactory, + columnFormatSpec, null ); } @@ -100,6 +104,8 @@ public static NestedCommonFormatColumnPartSerde createDeserializer( private final boolean enforceLogicalType; private final ByteOrder byteOrder; private final BitmapSerdeFactory bitmapSerdeFactory; + @Nullable + private final NestedCommonFormatColumnFormatSpec columnFormatSpec; @Nullable private final Serializer serializer; @@ -112,6 +118,7 @@ private NestedCommonFormatColumnPartSerde( boolean enforceLogicalType, ByteOrder byteOrder, BitmapSerdeFactory bitmapSerdeFactory, + @Nullable NestedCommonFormatColumnFormatSpec columnFormatSpec, @Nullable Serializer serializer ) { @@ -122,6 +129,7 @@ private NestedCommonFormatColumnPartSerde( this.byteOrder = byteOrder; this.bitmapSerdeFactory = bitmapSerdeFactory; this.serializer = serializer; + this.columnFormatSpec = columnFormatSpec; } @JsonIgnore @@ -186,6 +194,13 @@ public BitmapSerdeFactory getBitmapSerdeFactory() return bitmapSerdeFactory; } + @Nullable + @JsonProperty + public NestedCommonFormatColumnFormatSpec getColumnFormatSpec() + { + return columnFormatSpec; + } + private class StringColumnDeserializer implements Deserializer { @Override @@ -193,12 +208,12 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo { ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read( byteOrder, - bitmapSerdeFactory, + columnFormatSpec != null ? columnFormatSpec.getBitmapEncoding() : bitmapSerdeFactory, buffer, builder, parent == null ? null : (ScalarStringColumnAndIndexSupplier) parent.getColumnSupplier() ); - ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + final ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); capabilitiesBuilder.setDictionaryEncoded(true); capabilitiesBuilder.setDictionaryValuesSorted(true); capabilitiesBuilder.setDictionaryValuesUnique(true); @@ -209,7 +224,8 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setColumnFormat(new NestedCommonFormatColumn.Format( logicalType, capabilitiesBuilder.hasNulls().isTrue(), - enforceLogicalType + enforceLogicalType, + columnFormatSpec )); } } @@ -221,7 +237,7 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo { ScalarLongColumnAndIndexSupplier supplier = ScalarLongColumnAndIndexSupplier.read( byteOrder, - bitmapSerdeFactory, + columnFormatSpec != null ? columnFormatSpec.getBitmapEncoding() : bitmapSerdeFactory, buffer, builder, columnConfig, @@ -237,7 +253,8 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setColumnFormat(new NestedCommonFormatColumn.Format( logicalType, capabilitiesBuilder.hasNulls().isTrue(), - enforceLogicalType + enforceLogicalType, + columnFormatSpec )); } } @@ -249,7 +266,7 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo { ScalarDoubleColumnAndIndexSupplier supplier = ScalarDoubleColumnAndIndexSupplier.read( byteOrder, - bitmapSerdeFactory, + columnFormatSpec != null ? columnFormatSpec.getBitmapEncoding() : bitmapSerdeFactory, buffer, builder, columnConfig, @@ -265,7 +282,8 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setColumnFormat(new NestedCommonFormatColumn.Format( logicalType, capabilitiesBuilder.hasNulls().isTrue(), - enforceLogicalType + enforceLogicalType, + columnFormatSpec )); } } @@ -278,7 +296,7 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo VariantColumnAndIndexSupplier supplier = VariantColumnAndIndexSupplier.read( logicalType, byteOrder, - bitmapSerdeFactory, + columnFormatSpec != null ? columnFormatSpec.getBitmapEncoding() : bitmapSerdeFactory, buffer, builder.getFileMapper(), parent == null ? null : (VariantColumnAndIndexSupplier) parent.getColumnSupplier() @@ -299,7 +317,8 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setColumnFormat(new NestedCommonFormatColumn.Format( logicalType, capabilitiesBuilder.hasNulls().isTrue(), - enforceLogicalType + enforceLogicalType, + columnFormatSpec )); } } @@ -309,13 +328,22 @@ private class NestedColumnDeserializer implements Deserializer @Override public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig, @Nullable ColumnHolder parent) { + final NestedCommonFormatColumnFormatSpec formatSpec; + if (columnFormatSpec == null) { + formatSpec = NestedCommonFormatColumnFormatSpec.builder() + .setObjectStorageEncoding(ObjectStorageEncoding.SMILE) + .setBitmapEncoding(bitmapSerdeFactory) + .build(); + } else { + formatSpec = columnFormatSpec; + } NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read( logicalType, hasNulls, buffer, builder, columnConfig, - bitmapSerdeFactory, + formatSpec != null ? formatSpec.getBitmapEncoding() : bitmapSerdeFactory, byteOrder, parent == null ? null : (NestedDataColumnSupplier) parent.getColumnSupplier() ); @@ -329,7 +357,7 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo if (hasNulls) { builder.setIndexSupplier(supplier, false, false); } - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, hasNulls, enforceLogicalType)); + builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, hasNulls, enforceLogicalType, formatSpec)); } } @@ -344,6 +372,8 @@ public static class SerializerBuilder BitmapSerdeFactory bitmapSerdeFactory = RoaringBitmapSerdeFactory.getInstance(); @Nullable private Serializer serializer = null; + @Nullable + private NestedCommonFormatColumnFormatSpec columnFormatSpec = null; public SerializerBuilder withLogicalType(ColumnType logicalType) { @@ -387,6 +417,12 @@ public SerializerBuilder withEnforceLogicalType(boolean enforceLogicalType) return this; } + public SerializerBuilder withColumnFormatSpec(NestedCommonFormatColumnFormatSpec columnFormatSpec) + { + this.columnFormatSpec = columnFormatSpec; + return this; + } + public NestedCommonFormatColumnPartSerde build() { return new NestedCommonFormatColumnPartSerde( @@ -396,6 +432,7 @@ public NestedCommonFormatColumnPartSerde build() enforceLogicalType, byteOrder, bitmapSerdeFactory, + columnFormatSpec, serializer ); } 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 663b272917b9..f455e6cbf2f3 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 @@ -1167,7 +1167,7 @@ public ColumnIndexSupplier getIndexSupplier( } BaseColumn theColumn = holder.getColumn(); if (theColumn instanceof CompressedNestedDataComplexColumn) { - final CompressedNestedDataComplexColumn nestedColumn = (CompressedNestedDataComplexColumn) theColumn; + final CompressedNestedDataComplexColumn nestedColumn = (CompressedNestedDataComplexColumn) theColumn; final ColumnIndexSupplier nestedColumnPathIndexSupplier = nestedColumn.getColumnIndexSupplier(fieldSpec.parts); if (nestedColumnPathIndexSupplier == null && fieldSpec.processFromRaw) { // if processing from raw, a non-exstent path from parts doesn't mean the path doesn't really exist diff --git a/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java b/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java index 373f8a458c2e..59a5c942a355 100644 --- a/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java +++ b/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java @@ -26,6 +26,8 @@ import org.apache.druid.segment.DimensionHandlerProvider; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.NestedCommonFormatColumnHandler; +import org.apache.druid.segment.column.StringEncodingStrategy; +import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.junit.AfterClass; import org.junit.Assert; @@ -88,6 +90,9 @@ public void testOverride() DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.remove(NestedDataComplexTypeSerde.TYPE_NAME); Properties props = new Properties(); props.setProperty("druid.indexing.formats.stringMultiValueHandlingMode", "sorted_array"); + props.setProperty("druid.indexing.formats.nestedColumnFormatSpec.stringDictionaryEncoding.type", StringEncodingStrategy.FRONT_CODED); + props.setProperty("druid.indexing.formats.nestedColumnFormatSpec.stringDictionaryEncoding.bucketSize", "16"); + props.setProperty("druid.indexing.formats.nestedColumnFormatSpec.stringDictionaryEncoding.formatVersion", "1"); Injector gadget = makeInjector(props); // side effects @@ -101,6 +106,10 @@ public void testOverride() DimensionSchema.MultiValueHandling.SORTED_ARRAY, BuiltInTypesModule.getStringMultiValueHandlingMode() ); + Assert.assertEquals( + new NestedCommonFormatColumnFormatSpec(null, null, null, new StringEncodingStrategy.FrontCoded(16, (byte) 1), null, null, null, null, null), + BuiltInTypesModule.getDefaultNestedCommonFormatSpec() + ); } @Test 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 0aaaf77c66c9..1c6411c80368 100644 --- a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java +++ b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java @@ -96,12 +96,12 @@ public class NestedDataTestUtils DimensionsSpec.builder() .setDimensions( Arrays.asList( - new AutoTypeColumnSchema("dim", null), - new AutoTypeColumnSchema("nest_json", null), - new AutoTypeColumnSchema("nester_json", null), - new AutoTypeColumnSchema("variant_json", null), - new AutoTypeColumnSchema("list_json", null), - new AutoTypeColumnSchema("nonexistent", null) + AutoTypeColumnSchema.of("dim"), + AutoTypeColumnSchema.of("nest_json"), + AutoTypeColumnSchema.of("nester_json"), + AutoTypeColumnSchema.of("variant_json"), + AutoTypeColumnSchema.of("list_json"), + AutoTypeColumnSchema.of("nonexistent") ) ) .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 baee61a86fab..2f0e81a83e82 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 @@ -551,9 +551,9 @@ public void testIngestAndScanSegmentsRealtimeAutoExplicit() throws Exception DimensionsSpec spec = DimensionsSpec.builder() .setDimensions( ImmutableList.of( - new AutoTypeColumnSchema("str", ColumnType.STRING), - new AutoTypeColumnSchema("long", ColumnType.LONG), - new AutoTypeColumnSchema("double", ColumnType.FLOAT) + new AutoTypeColumnSchema("str", ColumnType.STRING, null), + new AutoTypeColumnSchema("long", ColumnType.LONG, null), + new AutoTypeColumnSchema("double", ColumnType.FLOAT, null) ) ) .build(); diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java index 0d78b03a7867..6ce4dd58ac72 100644 --- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -62,7 +62,7 @@ public static void setup() @Test public void testKeySizeEstimation() { - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, null); Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, indexer.getCardinality()); int baseCardinality = 0; Assert.assertEquals(baseCardinality, indexer.globalDictionary.getCardinality()); @@ -443,7 +443,7 @@ public void testNestedColumnIndexerSchemaDiscoveryTypeCoercion() throws IndexSiz .withTimestampSpec(new TimestampSpec(TIME_COL, "millis", null)) .withDimensionsSpec( DimensionsSpec.builder() - .setDimensions(ImmutableList.of(new AutoTypeColumnSchema(NESTED_COL, ColumnType.STRING))) + .setDimensions(ImmutableList.of(new AutoTypeColumnSchema(NESTED_COL, ColumnType.STRING, null))) .useSchemaDiscovery(true) .build() ) @@ -496,7 +496,7 @@ public void testNestedColumnIndexerSchemaDiscoveryTypeCoercion() throws IndexSiz public void testConstantNull() { int baseCardinality = 0; - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, null); EncodedKeyComponent key; key = indexer.processRowValsToUnsortedEncodedKeyComponent(null, true); @@ -521,7 +521,7 @@ public void testConstantNull() public void testConstantString() { int baseCardinality = 0; - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, null); EncodedKeyComponent key; key = indexer.processRowValsToUnsortedEncodedKeyComponent("abcd", true); @@ -545,7 +545,7 @@ public void testConstantString() public void testConstantLong() { int baseCardinality = 0; - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, null); EncodedKeyComponent key; key = indexer.processRowValsToUnsortedEncodedKeyComponent(1234L, true); @@ -569,7 +569,7 @@ public void testConstantLong() public void testConstantEmptyArray() { int baseCardinality = 0; - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, null); EncodedKeyComponent key; key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(), true); @@ -593,7 +593,7 @@ public void testConstantEmptyArray() public void testConstantArray() { int baseCardinality = 0; - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, null); EncodedKeyComponent key; key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 3L), true); @@ -617,7 +617,7 @@ public void testConstantArray() public void testConstantEmptyObject() { int baseCardinality = 0; - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, null); EncodedKeyComponent key; key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of(), true); diff --git a/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java b/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java index 403ae7058ba9..839e4e6c50ab 100644 --- a/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java @@ -252,7 +252,7 @@ public class CursorFactoryProjectionTest extends InitializedNullHandlingTest projection.getVirtualColumns(), projection.getGroupingColumns() .stream() - .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .map(x -> AutoTypeColumnSchema.of(x.getName())) .collect(Collectors.toList()), projection.getAggregators() ); @@ -279,7 +279,7 @@ public static Collection constructorFeeder() List autoDims = dimsOrdered.getDimensions() .stream() - .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .map(x -> AutoTypeColumnSchema.of(x.getName())) .collect(Collectors.toList()); for (boolean incremental : new boolean[]{true, false}) { for (boolean sortByDim : new boolean[]{true, false}) { diff --git a/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java b/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java index 42a819037f14..d9e9e9a95251 100644 --- a/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java +++ b/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java @@ -34,7 +34,7 @@ public class DefaultColumnFormatsConfigTest @Test public void testDefaultsSerde() throws JsonProcessingException { - DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig(null, null); + DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig(null, null, null); String there = MAPPER.writeValueAsString(defaultColumnFormatConfig); DefaultColumnFormatConfig andBack = MAPPER.readValue(there, DefaultColumnFormatConfig.class); Assert.assertEquals(defaultColumnFormatConfig, andBack); @@ -45,7 +45,7 @@ public void testDefaultsSerde() throws JsonProcessingException @Test public void testDefaultsSerdeOverride() throws JsonProcessingException { - DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig(5, "ARRAY"); + DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig("ARRAY", 5, null); String there = MAPPER.writeValueAsString(defaultColumnFormatConfig); DefaultColumnFormatConfig andBack = MAPPER.readValue(there, DefaultColumnFormatConfig.class); Assert.assertEquals(defaultColumnFormatConfig, andBack); diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java index 2dd5455ceb5a..b6e12f7b3d49 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java @@ -23,13 +23,31 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.column.StringEncodingStrategy; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.FrontCodedIndexed; +import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.junit.Assert; import org.junit.Test; public class NestedDataColumnSchemaTest { - private static final DefaultColumnFormatConfig DEFAULT_CONFIG = new DefaultColumnFormatConfig(null, null); + private static final DefaultColumnFormatConfig DEFAULT_CONFIG = new DefaultColumnFormatConfig(null, null, null); + private static final NestedCommonFormatColumnFormatSpec DEFAULT_NESTED_SPEC = + NestedCommonFormatColumnFormatSpec.builder() + .setObjectKeysEncoding( + new StringEncodingStrategy.FrontCoded(8, FrontCodedIndexed.V1) + ) + .setObjectStorageCompression(CompressionStrategy.ZSTD) + .build(); + private static final DefaultColumnFormatConfig DEFAULT_NESTED_SPEC_CONFIG = new DefaultColumnFormatConfig( + null, + null, + DEFAULT_NESTED_SPEC + ); + private static final ObjectMapper MAPPER; + private static final ObjectMapper DEFAULT_NESTED_SPEC_MAPPER; static { MAPPER = new DefaultObjectMapper(); @@ -39,12 +57,19 @@ public class NestedDataColumnSchemaTest DEFAULT_CONFIG ) ); + DEFAULT_NESTED_SPEC_MAPPER = new DefaultObjectMapper(); + DEFAULT_NESTED_SPEC_MAPPER.setInjectableValues( + new InjectableValues.Std().addValue( + DefaultColumnFormatConfig.class, + DEFAULT_NESTED_SPEC_CONFIG + ) + ); } @Test public void testSerdeRoundTrip() throws JsonProcessingException { - final NestedDataColumnSchema v5 = new NestedDataColumnSchema("test", 5); + final NestedDataColumnSchema v5 = new NestedDataColumnSchema("test", 5, DEFAULT_NESTED_SPEC, DEFAULT_CONFIG); Assert.assertEquals(v5, MAPPER.readValue(MAPPER.writeValueAsString(v5), NestedDataColumnSchema.class)); } @@ -55,4 +80,15 @@ public void testSerdeDefault() throws JsonProcessingException NestedDataColumnSchema andBack = MAPPER.readValue(there, NestedDataColumnSchema.class); Assert.assertEquals(new NestedDataColumnSchema("test", 5), andBack); } + + @Test + public void testSerdeDefaultNestedSpec() throws JsonProcessingException + { + final String there = "{\"type\":\"json\", \"name\":\"test\"}"; + NestedDataColumnSchema andBack = DEFAULT_NESTED_SPEC_MAPPER.readValue(there, NestedDataColumnSchema.class); + Assert.assertEquals( + new NestedDataColumnSchema("test", 5, DEFAULT_NESTED_SPEC, DEFAULT_NESTED_SPEC_CONFIG), + andBack + ); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index bddbf7f453fc..9060e603be05 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -186,11 +186,11 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .add(new DoubleDimensionSchema("d0")) .add(new FloatDimensionSchema("f0")) .add(new LongDimensionSchema("l0")) - .add(new AutoTypeColumnSchema("arrayString", ColumnType.STRING_ARRAY)) - .add(new AutoTypeColumnSchema("arrayLong", ColumnType.LONG_ARRAY)) - .add(new AutoTypeColumnSchema("arrayDouble", ColumnType.DOUBLE_ARRAY)) - .add(new AutoTypeColumnSchema("variant", null)) - .add(new AutoTypeColumnSchema("nested", null)) + .add(new AutoTypeColumnSchema("arrayString", ColumnType.STRING_ARRAY, null)) + .add(new AutoTypeColumnSchema("arrayLong", ColumnType.LONG_ARRAY, null)) + .add(new AutoTypeColumnSchema("arrayDouble", ColumnType.DOUBLE_ARRAY, null)) + .add(AutoTypeColumnSchema.of("variant")) + .add(AutoTypeColumnSchema.of("nested")) .build() ); @@ -527,7 +527,7 @@ public static Collection makeConstructors() .getDimensions() .stream() .map( - dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName(), null) + dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName(), null, null) ) .collect(Collectors.toList()) ), @@ -556,7 +556,7 @@ public static Collection makeConstructors() .getDimensions() .stream() .map( - dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName(), null) + dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName(), null, null) ) .collect(Collectors.toList()) ), @@ -586,7 +586,7 @@ public static Collection makeConstructors() .getDimensions() .stream() .map( - dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName(), null) + dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName(), null, null) ) .collect(Collectors.toList()) ), diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java index 7a99951c6bd7..f437ae7ac9f3 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java @@ -79,11 +79,11 @@ public IncrementalIndexTest( new DoubleDimensionSchema("double"), new StringDimensionSchema("bool_string"), new LongDimensionSchema("bool_long"), - new AutoTypeColumnSchema("bool_auto", null), - new AutoTypeColumnSchema("array_string", ColumnType.STRING_ARRAY), - new AutoTypeColumnSchema("array_double", ColumnType.DOUBLE_ARRAY), - new AutoTypeColumnSchema("array_long", ColumnType.LONG_ARRAY), - new AutoTypeColumnSchema("nested", null) + AutoTypeColumnSchema.of("bool_auto"), + new AutoTypeColumnSchema("array_string", ColumnType.STRING_ARRAY, null), + new AutoTypeColumnSchema("array_double", ColumnType.DOUBLE_ARRAY, null), + new AutoTypeColumnSchema("array_long", ColumnType.LONG_ARRAY, null), + AutoTypeColumnSchema.of("nested") ) ); AggregatorFactory[] metrics = { diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java new file mode 100644 index 000000000000..b2d4e7d701aa --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java @@ -0,0 +1,163 @@ +/* + * 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.core.JsonProcessingException; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.StringEncodingStrategy; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; +import org.apache.druid.segment.data.FrontCodedIndexed; +import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.junit.Assert; +import org.junit.jupiter.api.Test; + +public class NestedCommonFormatColumnFormatSpecTest +{ + @Test + public void testSerde() throws JsonProcessingException + { + NestedCommonFormatColumnFormatSpec spec = + NestedCommonFormatColumnFormatSpec.builder() + .setObjectKeysEncoding( + new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1) + ) + .setObjectStorageCompression(CompressionStrategy.ZSTD) + .setStringDictionaryEncoding( + new StringEncodingStrategy.FrontCoded(16, FrontCodedIndexed.V1) + ) + .build(); + Assert.assertEquals( + spec, + TestHelper.JSON_MAPPER.readValue( + TestHelper.JSON_MAPPER.writeValueAsString(spec), + NestedCommonFormatColumnFormatSpec.class + ) + ); + } + + @Test + public void testGetEffectiveSpecDefaults() + { + NestedCommonFormatColumnFormatSpec defaults = NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( + null, + IndexSpec.DEFAULT + ); + + Assert.assertEquals( + StringEncodingStrategy.UTF8_STRATEGY, + defaults.getObjectKeysEncoding() + ); + Assert.assertEquals( + ObjectStorageEncoding.SMILE, + defaults.getObjectStorageEncoding() + ); + Assert.assertEquals( + CompressionStrategy.LZ4, + defaults.getObjectStorageCompression() + ); + Assert.assertEquals( + IndexSpec.DEFAULT.getDimensionCompression(), + defaults.getDictionaryEncodedColumnCompression() + ); + Assert.assertEquals( + IndexSpec.DEFAULT.getStringDictionaryEncoding(), + defaults.getStringDictionaryEncoding() + ); + Assert.assertEquals( + IndexSpec.DEFAULT.getMetricCompression(), + defaults.getLongColumnCompression() + ); + Assert.assertEquals( + IndexSpec.DEFAULT.getMetricCompression(), + defaults.getDoubleColumnCompression() + ); + } + + @Test + public void testGetEffectiveSpecMerge() + { + NestedCommonFormatColumnFormatSpec merged = NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( + NestedCommonFormatColumnFormatSpec.builder() + .setObjectKeysEncoding( + new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1) + ) + .setObjectStorageCompression(CompressionStrategy.ZSTD) + .setStringDictionaryEncoding( + new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1) + ) + .setDoubleColumnCompression(CompressionStrategy.ZSTD) + .build(), + IndexSpec.DEFAULT + ); + + Assert.assertEquals( + new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1), + merged.getObjectKeysEncoding() + ); + Assert.assertEquals( + new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1), + merged.getStringDictionaryEncoding() + ); + Assert.assertEquals( + ObjectStorageEncoding.SMILE, + merged.getObjectStorageEncoding() + ); + Assert.assertEquals( + IndexSpec.DEFAULT.getDimensionCompression(), + merged.getDictionaryEncodedColumnCompression() + ); + Assert.assertEquals( + CompressionStrategy.ZSTD, + merged.getObjectStorageCompression() + ); + Assert.assertEquals( + IndexSpec.DEFAULT.getMetricCompression(), + merged.getLongColumnCompression() + ); + Assert.assertEquals( + CompressionStrategy.ZSTD, + merged.getDoubleColumnCompression() + ); + } + + @Test + public void testGetEffectiveSpecInvalid() + { + Throwable t = Assert.assertThrows( + ISE.class, + () -> NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( + NestedCommonFormatColumnFormatSpec.builder().setBitmapEncoding(new ConciseBitmapSerdeFactory()).build(), + IndexSpec.builder().withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()).build() + ) + ); + + Assert.assertEquals("bitmapEncoding[ConciseBitmapSerdeFactory{}] does not match indexSpec.bitmap[RoaringBitmapSerdeFactory{}]", t.getMessage()); + } + + @Test + public void testEqualsAndHashcode() + { + EqualsVerifier.forClass(NestedCommonFormatColumnFormatSpec.class).usingGetClass().verify(); + } +} 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 991456aa9c56..409c7e2e3df9 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 @@ -53,7 +53,10 @@ 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.StringEncodingStrategy; import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.FrontCodedIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; import org.apache.druid.segment.index.semantic.NullValueIndex; @@ -74,6 +77,8 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import javax.annotation.Nullable; import java.io.File; @@ -91,19 +96,14 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicReference; +@RunWith(Parameterized.class) public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest { private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); 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 = ImmutableList.of( + private static final 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", ""), @@ -112,7 +112,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest TestHelper.makeMap("x", 4L, "y", 2.0, "z", "e", "v", 11111L, "nullish", null) ); - List> arrayTestData = ImmutableList.of( + private static final List> ARRAY_TEST_DATA = 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", @@ -135,6 +135,51 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest TestHelper.makeMap("l", new Object[]{1L, 2L, 3L}, "d", new Object[]{3.1, 2.2, 1.9}) ); + @BeforeClass + public static void staticSetup() + { + BuiltInTypesModule.registerHandlersAndSerde(); + } + + @Parameterized.Parameters(name = "data = {0}") + public static Collection constructorFeeder() + { + + NestedCommonFormatColumnFormatSpec defaultSpec = NestedCommonFormatColumnFormatSpec.builder().build(); + + NestedCommonFormatColumnFormatSpec frontCodedKeysAndDicts = + NestedCommonFormatColumnFormatSpec.builder() + .setObjectKeysEncoding( + new StringEncodingStrategy.FrontCoded(16, FrontCodedIndexed.V1) + ) + .setStringDictionaryEncoding( + new StringEncodingStrategy.FrontCoded(16, FrontCodedIndexed.V1) + ) + .build(); + + NestedCommonFormatColumnFormatSpec zstdRaw = + NestedCommonFormatColumnFormatSpec.builder() + .setObjectStorageCompression(CompressionStrategy.ZSTD) + .build(); + NestedCommonFormatColumnFormatSpec lzf = + NestedCommonFormatColumnFormatSpec.builder() + .setDictionaryEncodedColumnCompression(CompressionStrategy.LZF) + .setLongColumnCompression(CompressionStrategy.LZF) + .setDoubleColumnCompression(CompressionStrategy.LZF) + .build(); + final List constructors = ImmutableList.of( + new Object[]{defaultSpec}, + new Object[]{frontCodedKeysAndDicts}, + new Object[]{zstdRaw}, + new Object[]{lzf} + ); + + return constructors; + } + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + Closer closer = Closer.create(); SmooshedFileMapper fileMapper; @@ -145,10 +190,20 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest ByteBuffer arrayBaseBuffer; - @BeforeClass - public static void staticSetup() + private final NestedCommonFormatColumnFormatSpec columnFormatSpec; + private final BitmapSerdeFactory bitmapSerdeFactory; + private final DefaultBitmapResultFactory resultFactory; + + public NestedDataColumnSupplierTest( + NestedCommonFormatColumnFormatSpec columnFormatSpec + ) { - BuiltInTypesModule.registerHandlersAndSerde(); + this.columnFormatSpec = NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( + columnFormatSpec, + IndexSpec.DEFAULT + ); + this.bitmapSerdeFactory = this.columnFormatSpec.getBitmapEncoding(); + this.resultFactory = new DefaultBitmapResultFactory(bitmapSerdeFactory.getBitmapFactory()); } @Before @@ -156,9 +211,9 @@ public void setup() throws IOException { final String fileNameBase = "test/column"; final String arrayFileNameBase = "array"; - fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), data); + fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), DATA); baseBuffer = fileMapper.mapFile(fileNameBase); - arrayFileMapper = smooshify(arrayFileNameBase, tempFolder.newFolder(), arrayTestData); + arrayFileMapper = smooshify(arrayFileNameBase, tempFolder.newFolder(), ARRAY_TEST_DATA); arrayBaseBuffer = arrayFileMapper.mapFile(arrayFileNameBase); } @@ -173,12 +228,12 @@ private SmooshedFileMapper smooshify( try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { NestedDataColumnSerializer serializer = new NestedDataColumnSerializer( fileNameBase, - IndexSpec.DEFAULT, + columnFormatSpec, writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, null); for (Object o : data) { indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); } @@ -239,7 +294,8 @@ public void testBasicFunctionality() throws IOException false, false, ByteOrder.nativeOrder(), - RoaringBitmapSerdeFactory.getInstance() + RoaringBitmapSerdeFactory.getInstance(), + columnFormatSpec ); bob.setFileMapper(fileMapper); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); @@ -263,7 +319,8 @@ public void testArrayFunctionality() throws IOException false, false, ByteOrder.nativeOrder(), - RoaringBitmapSerdeFactory.getInstance() + RoaringBitmapSerdeFactory.getInstance(), + columnFormatSpec ); bob.setFileMapper(arrayFileMapper); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); @@ -331,7 +388,7 @@ public void testConcurrency() throws ExecutionException, InterruptedException private void smokeTest(NestedDataComplexColumn column) throws IOException { - SimpleAscendingOffset offset = new SimpleAscendingOffset(data.size()); + SimpleAscendingOffset offset = new SimpleAscendingOffset(DATA.size()); ColumnValueSelector rawSelector = column.makeColumnValueSelector(offset); final List xPath = NestedPathFinder.parseJsonPath("$.x"); @@ -394,8 +451,8 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException Assert.assertEquals(ImmutableList.of(nullishPath, vPath, xPath, yPath, zPath), column.getNestedFields()); - for (int i = 0; i < data.size(); i++) { - Map row = data.get(i); + 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())) @@ -423,10 +480,10 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException private void smokeTestArrays(NestedDataComplexColumn column) throws IOException { - SimpleAscendingOffset offset = new SimpleAscendingOffset(arrayTestData.size()); - NoFilterVectorOffset vectorOffset = new NoFilterVectorOffset(4, 0, arrayTestData.size()); + SimpleAscendingOffset offset = new SimpleAscendingOffset(ARRAY_TEST_DATA.size()); + NoFilterVectorOffset vectorOffset = new NoFilterVectorOffset(4, 0, ARRAY_TEST_DATA.size()); WrappedRoaringBitmap bitmap = new WrappedRoaringBitmap(); - for (int i = 0; i < arrayTestData.size(); i++) { + for (int i = 0; i < ARRAY_TEST_DATA.size(); i++) { if (i % 2 == 0) { bitmap.add(i); } @@ -435,7 +492,7 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException 4, bitmap.toImmutableBitmap(), 0, - arrayTestData.size() + ARRAY_TEST_DATA.size() ); ColumnValueSelector rawSelector = column.makeColumnValueSelector(offset); @@ -526,7 +583,7 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException int rowCounter = 0; while (offset.withinBounds()) { - Map row = arrayTestData.get(rowCounter); + Map row = ARRAY_TEST_DATA.get(rowCounter); Assert.assertEquals( JSON_MAPPER.writeValueAsString(row), JSON_MAPPER.writeValueAsString(StructuredData.unwrap(rawSelector.getObject())) @@ -582,7 +639,7 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException for (int i = 0; i < vectorOffset.getCurrentVectorSize(); i++, rowCounter++) { - Map row = arrayTestData.get(rowCounter); + Map row = ARRAY_TEST_DATA.get(rowCounter); Assert.assertEquals( JSON_MAPPER.writeValueAsString(row), JSON_MAPPER.writeValueAsString(StructuredData.unwrap(rawVector[i])) @@ -631,7 +688,7 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException final boolean[] dElementNulls = dElementFilteredVectorSelector.getNullVector(); for (int i = 0; i < bitmapVectorOffset.getCurrentVectorSize(); i++, rowCounter += 2) { - Map row = arrayTestData.get(rowCounter); + Map row = ARRAY_TEST_DATA.get(rowCounter); Assert.assertEquals( JSON_MAPPER.writeValueAsString(row), JSON_MAPPER.writeValueAsString(StructuredData.unwrap(rawVector[i])) 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 index f615cb3c050b..ef1265ceffba 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java @@ -126,12 +126,12 @@ private SmooshedFileMapper smooshify( try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { ScalarDoubleColumnSerializer serializer = new ScalarDoubleColumnSerializer( fileNameBase, - IndexSpec.DEFAULT, + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.DEFAULT), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, null); for (Object o : data) { indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); } 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 index 3a851c47ebfb..84d66c0a5769 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java @@ -126,12 +126,12 @@ private SmooshedFileMapper smooshify( try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { ScalarLongColumnSerializer serializer = new ScalarLongColumnSerializer( fileNameBase, - IndexSpec.DEFAULT, + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.DEFAULT), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, null); for (Object o : data) { indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); } 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 index d71508203bc7..51e84c1726ce 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -126,12 +126,12 @@ private SmooshedFileMapper smooshify( try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { ScalarStringColumnSerializer serializer = new ScalarStringColumnSerializer( fileNameBase, - IndexSpec.DEFAULT, + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.DEFAULT), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, null); for (Object o : data) { indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java index e6b3f073bc08..e9ab4395566e 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java @@ -44,7 +44,6 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.StringEncodingStrategy; import org.apache.druid.segment.data.BitmapSerdeFactory; -import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.FrontCodedIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.index.semantic.ArrayElementIndexes; @@ -173,29 +172,32 @@ public static void staticSetup() @Parameterized.Parameters(name = "data = {0}") public static Collection constructorFeeder() { - IndexSpec fancy = IndexSpec.builder() - .withLongEncoding(CompressionFactory.LongEncodingStrategy.AUTO) - .withStringDictionaryEncoding( - new StringEncodingStrategy.FrontCoded(16, FrontCodedIndexed.V1) - ) - .build(); + + NestedCommonFormatColumnFormatSpec defaultSpec = NestedCommonFormatColumnFormatSpec.builder().build(); + + NestedCommonFormatColumnFormatSpec frontCodedDict = + NestedCommonFormatColumnFormatSpec.builder() + .setStringDictionaryEncoding( + new StringEncodingStrategy.FrontCoded(16, FrontCodedIndexed.V1) + ) + .build(); final List constructors = ImmutableList.of( - new Object[]{"ARRAY", LONG_ARRAY, IndexSpec.DEFAULT}, - new Object[]{"ARRAY", LONG_ARRAY, fancy}, - new Object[]{"ARRAY", DOUBLE_ARRAY, IndexSpec.DEFAULT}, - new Object[]{"ARRAY", DOUBLE_ARRAY, fancy}, - new Object[]{"ARRAY", STRING_ARRAY, IndexSpec.DEFAULT}, - new Object[]{"ARRAY", STRING_ARRAY, fancy}, - new Object[]{"DOUBLE,LONG", VARIANT_NUMERIC, IndexSpec.DEFAULT}, - new Object[]{"DOUBLE,LONG", VARIANT_NUMERIC, fancy}, - new Object[]{"DOUBLE,LONG,STRING", VARIANT_SCALAR, IndexSpec.DEFAULT}, - new Object[]{"DOUBLE,LONG,STRING", VARIANT_SCALAR, fancy}, - new Object[]{"ARRAY,ARRAY,DOUBLE,LONG,STRING", VARIANT_SCALAR_AND_ARRAY, IndexSpec.DEFAULT}, - new Object[]{"ARRAY,ARRAY,DOUBLE,LONG,STRING", VARIANT_SCALAR_AND_ARRAY, fancy}, - new Object[]{"ARRAY,ARRAY,ARRAY", VARIANT_ARRAY, IndexSpec.DEFAULT}, - new Object[]{"ARRAY,ARRAY,ARRAY", VARIANT_ARRAY, fancy}, - new Object[]{"ARRAY", NO_TYPE_ARRAY, IndexSpec.DEFAULT}, - new Object[]{"ARRAY", NO_TYPE_ARRAY, fancy} + new Object[]{"ARRAY", LONG_ARRAY, defaultSpec}, + new Object[]{"ARRAY", LONG_ARRAY, frontCodedDict}, + new Object[]{"ARRAY", DOUBLE_ARRAY, defaultSpec}, + new Object[]{"ARRAY", DOUBLE_ARRAY, frontCodedDict}, + new Object[]{"ARRAY", STRING_ARRAY, defaultSpec}, + new Object[]{"ARRAY", STRING_ARRAY, frontCodedDict}, + new Object[]{"DOUBLE,LONG", VARIANT_NUMERIC, defaultSpec}, + new Object[]{"DOUBLE,LONG", VARIANT_NUMERIC, frontCodedDict}, + new Object[]{"DOUBLE,LONG,STRING", VARIANT_SCALAR, defaultSpec}, + new Object[]{"DOUBLE,LONG,STRING", VARIANT_SCALAR, frontCodedDict}, + new Object[]{"ARRAY,ARRAY,DOUBLE,LONG,STRING", VARIANT_SCALAR_AND_ARRAY, defaultSpec}, + new Object[]{"ARRAY,ARRAY,DOUBLE,LONG,STRING", VARIANT_SCALAR_AND_ARRAY, frontCodedDict}, + new Object[]{"ARRAY,ARRAY,ARRAY", VARIANT_ARRAY, defaultSpec}, + new Object[]{"ARRAY,ARRAY,ARRAY", VARIANT_ARRAY, frontCodedDict}, + new Object[]{"ARRAY", NO_TYPE_ARRAY, defaultSpec}, + new Object[]{"ARRAY", NO_TYPE_ARRAY, frontCodedDict} ); return constructors; @@ -212,16 +214,16 @@ public static Collection constructorFeeder() ColumnType expectedLogicalType = null; private final List data; - private final IndexSpec indexSpec; + private final NestedCommonFormatColumnFormatSpec columnFormatSpec; public VariantColumnSupplierTest( @SuppressWarnings("unused") String name, List data, - IndexSpec indexSpec + NestedCommonFormatColumnFormatSpec columnFormatSpec ) { this.data = data; - this.indexSpec = indexSpec; + this.columnFormatSpec = columnFormatSpec; } @Before @@ -241,7 +243,7 @@ private SmooshedFileMapper smooshify( SegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance(); try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { - AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null); + AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null, columnFormatSpec); for (Object o : data) { indexer.processRowValsToUnsortedEncodedKeyComponent(o, false); } @@ -270,7 +272,7 @@ private SmooshedFileMapper smooshify( fileNameBase, expectedTypes.getSingleType() == null ? null : expectedLogicalType, expectedTypes.getSingleType() == null ? expectedTypes.getByteValue() : null, - indexSpec, + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(columnFormatSpec, IndexSpec.DEFAULT), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); diff --git a/quidem-ut/src/main/java/org/apache/druid/quidem/KttmNestedComponentSupplier.java b/quidem-ut/src/main/java/org/apache/druid/quidem/KttmNestedComponentSupplier.java index 8c17c22d2bc8..88ee3a78c37d 100644 --- a/quidem-ut/src/main/java/org/apache/druid/quidem/KttmNestedComponentSupplier.java +++ b/quidem-ut/src/main/java/org/apache/druid/quidem/KttmNestedComponentSupplier.java @@ -93,8 +93,8 @@ public IncrementalIndex makeKttmNestedIndex() final List dimensions = Arrays.asList( new StringDimensionSchema("session"), new StringDimensionSchema("number"), - new AutoTypeColumnSchema("event", null), - new AutoTypeColumnSchema("agent", null), + AutoTypeColumnSchema.of("event"), + AutoTypeColumnSchema.of("agent"), new StringDimensionSchema("client_ip"), new StringDimensionSchema("geo_ip"), new StringDimensionSchema("language"), 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 a98b9aafdbf6..e31fbd03c1d6 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -459,8 +459,8 @@ public static void runDumpNestedColumn( final ColumnHolder columnHolder = index.getColumnHolder(columnName); final BaseColumn baseColumn = columnHolder.getColumn(); Preconditions.checkArgument(baseColumn instanceof CompressedNestedDataComplexColumn); - final CompressedNestedDataComplexColumn nestedDataColumn = - (CompressedNestedDataComplexColumn) baseColumn; + final CompressedNestedDataComplexColumn nestedDataColumn = + (CompressedNestedDataComplexColumn) baseColumn; jg.writeFieldName("fields"); jg.writeStartArray(); diff --git a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java index 3e1170bc6927..c5e00513ace3 100644 --- a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java +++ b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java @@ -127,10 +127,10 @@ public void testDumpRows() throws Exception new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) .addValue(ObjectMapper.class.getName(), mapper) - .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null, null)) + .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null, null, null)) ); Mockito.when(injector.getInstance(Key.get(ObjectMapper.class, Json.class))).thenReturn(mapper); - Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); + Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null, null)); List segments = createSegments(tempFolder, closer); QueryableIndex queryableIndex = segments.get(0).as(QueryableIndex.class); @@ -201,10 +201,10 @@ public void testDumpNestedColumn() throws Exception new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) .addValue(ObjectMapper.class.getName(), mapper) - .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null, null)) + .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null, null, null)) ); Mockito.when(injector.getInstance(Key.get(ObjectMapper.class, Json.class))).thenReturn(mapper); - Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); + Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null, null)); List segments = createSegments(tempFolder, closer); QueryableIndex queryableIndex = segments.get(0).as(QueryableIndex.class); @@ -234,10 +234,10 @@ public void testDumpNestedColumnPath() throws Exception new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) .addValue(ObjectMapper.class.getName(), mapper) - .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null, null)) + .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null, null, null)) ); Mockito.when(injector.getInstance(Key.get(ObjectMapper.class, Json.class))).thenReturn(mapper); - Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); + Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null, null)); List segments = createSegments(tempFolder, closer); QueryableIndex queryableIndex = segments.get(0).as(QueryableIndex.class); 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 c892c4732691..8dc179858736 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 @@ -152,11 +152,11 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest new TimestampSpec("t", "iso", null), DimensionsSpec.builder().setDimensions( ImmutableList.builder() - .add(new AutoTypeColumnSchema("string", null)) - .add(new AutoTypeColumnSchema("nest", null)) - .add(new AutoTypeColumnSchema("nester", null)) - .add(new AutoTypeColumnSchema("long", null)) - .add(new AutoTypeColumnSchema("string_sparse", null)) + .add(AutoTypeColumnSchema.of("string")) + .add(AutoTypeColumnSchema.of("nest")) + .add(AutoTypeColumnSchema.of("nester")) + .add(AutoTypeColumnSchema.of("long")) + .add(AutoTypeColumnSchema.of("string_sparse")) .build() ).build(), null @@ -167,8 +167,8 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest DimensionsSpec.builder().setDimensions( ImmutableList.builder() .add(new StringDimensionSchema("string")) - .add(new AutoTypeColumnSchema("nest", null)) - .add(new AutoTypeColumnSchema("nester", null)) + .add(AutoTypeColumnSchema.of("nest")) + .add(AutoTypeColumnSchema.of("nester")) .add(new LongDimensionSchema("long")) .add(new StringDimensionSchema("string_sparse")) .build() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java index 90cc53d1b9a0..0fb88330c29e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java @@ -765,7 +765,7 @@ public void configure(Binder binder) { binder.bind(DruidOperatorTable.class).in(LazySingleton.class); binder.bind(DataSegment.PruneSpecsHolder.class).toInstance(DataSegment.PruneSpecsHolder.DEFAULT); - binder.bind(DefaultColumnFormatConfig.class).toInstance(new DefaultColumnFormatConfig(null, null)); + binder.bind(DefaultColumnFormatConfig.class).toInstance(new DefaultColumnFormatConfig(null, null, null)); binder.bind(new TypeLiteral>(){}) .annotatedWith(Global.class) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java index 4fd90a95c32b..8c8f8cb897c4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java @@ -1016,7 +1016,7 @@ public static void makeQueryableIndexForBenchmarkDatasource(Closer closer, int r List columnSchemas = schemaInfo.getDimensionsSpec() .getDimensions() .stream() - .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .map(x -> AutoTypeColumnSchema.of(x.getName())) .collect(Collectors.toList()); QUERYABLE_INDEX_FOR_BENCHMARK_DATASOURCE = segmentGenerator.generate( dataSegment, From 721ff100969f6df652cf023c32e24403c6014fc2 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 27 Feb 2025 04:36:40 -0800 Subject: [PATCH 02/24] fix test --- .../common/task/CompactionTaskRunTest.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index bd0894b88f14..5fa755334020 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -96,6 +96,7 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.loading.TombstoneLoadSpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.segment.transform.CompactionTransformSpec; @@ -155,6 +156,9 @@ public class CompactionTaskRunTest extends IngestionTestBase 0 ); + private static final NestedCommonFormatColumnFormatSpec DEFAULT_NESTED_SPEC = + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.DEFAULT); + private static final List TEST_ROWS = ImmutableList.of( "2014-01-01T00:00:10Z,a,1\n", "2014-01-01T00:00:10Z,b,2\n", @@ -1791,10 +1795,10 @@ public void testRunWithAutoCastDimensions() throws Exception DimensionsSpec.builder() .setDimensions(Arrays.asList( // check explicitly specified types are preserved - new AutoTypeColumnSchema("ts", ColumnType.STRING, null), - AutoTypeColumnSchema.of("dim"), - new AutoTypeColumnSchema("x", ColumnType.LONG, null), - new AutoTypeColumnSchema("y", ColumnType.LONG, null) + new AutoTypeColumnSchema("ts", ColumnType.STRING, DEFAULT_NESTED_SPEC), + new AutoTypeColumnSchema("dim", null, DEFAULT_NESTED_SPEC), + new AutoTypeColumnSchema("x", ColumnType.LONG, DEFAULT_NESTED_SPEC), + new AutoTypeColumnSchema("y", ColumnType.LONG, DEFAULT_NESTED_SPEC) )) .build(), expectedLongSumMetric @@ -1921,11 +1925,11 @@ public void testRunWithAutoCastDimensionsSortByDimension() throws Exception DimensionsSpec.builder() .setDimensions(Arrays.asList( // check explicitly that time ordering is preserved - new AutoTypeColumnSchema("x", ColumnType.LONG, null), + new AutoTypeColumnSchema("x", ColumnType.LONG, DEFAULT_NESTED_SPEC), new LongDimensionSchema("__time"), - new AutoTypeColumnSchema("ts", ColumnType.STRING, null), - AutoTypeColumnSchema.of("dim"), - new AutoTypeColumnSchema("y", ColumnType.LONG, null) + new AutoTypeColumnSchema("ts", ColumnType.STRING, DEFAULT_NESTED_SPEC), + new AutoTypeColumnSchema("dim", null, DEFAULT_NESTED_SPEC), + new AutoTypeColumnSchema("y", ColumnType.LONG, DEFAULT_NESTED_SPEC) )) .setForceSegmentSortByTime(false) .build(), From a1b7ac7e823f611f34997e49c123cdb6b780d148 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 29 Jul 2025 03:54:06 -0700 Subject: [PATCH 03/24] compaction stuff --- .../data/input/impl/DimensionSchema.java | 6 + .../druid/data/input/impl/DimensionsSpec.java | 19 +++ .../druid/segment/AutoTypeColumnSchema.java | 10 ++ .../druid/timeline/CompactionState.java | 10 +- .../server/compaction/CompactionStatus.java | 37 +++--- .../compaction/CompactionStatusTest.java | 112 ++++++++++++++++++ 6 files changed, 178 insertions(+), 16 deletions(-) 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 ea6ae9f0b758..66e84f35c986 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 @@ -33,6 +33,7 @@ import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.NestedDataColumnSchema; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.TypeSignature; @@ -166,6 +167,11 @@ public DimensionHandler getDimensionHandler() ); } + public DimensionSchema getEffectiveSchema(IndexSpec indexSpec) + { + return this; + } + @Override public boolean equals(final Object o) { diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java index c13b69105ea2..75dc2d771051 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java @@ -103,6 +103,11 @@ public static Builder builder() return new Builder(); } + public static Builder builder(DimensionsSpec dimensionsSpec) + { + return new Builder(dimensionsSpec); + } + public DimensionsSpec(List dimensions) { this(dimensions, null, null, false, null, null); @@ -364,6 +369,20 @@ public static final class Builder private boolean useSchemaDiscovery; private Boolean forceSegmentSortByTime; + private Builder() + { + + } + + private Builder(DimensionsSpec dimensionSpec) + { + this.dimensions = dimensionSpec.dimensions; + this.dimensionExclusions = List.copyOf(dimensionSpec.dimensionExclusions); + this.includeAllDimensions = dimensionSpec.includeAllDimensions; + this.useSchemaDiscovery = dimensionSpec.useSchemaDiscovery; + this.forceSegmentSortByTime = dimensionSpec.forceSegmentSortByTime; + } + public Builder setDimensions(List dimensions) { this.dimensions = dimensions; diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java index 8d385dd9bb86..0c6416950f56 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java @@ -134,6 +134,16 @@ public DimensionHandler getDimen return new NestedCommonFormatColumnHandler(getName(), castToType, columnFormatSpec); } + @Override + public DimensionSchema getEffectiveSchema(IndexSpec indexSpec) + { + return new AutoTypeColumnSchema( + getName(), + castToType, + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(columnFormatSpec, indexSpec) + ); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index 303f8af3c952..1330e39f1eac 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -180,9 +180,17 @@ public static Function, Set> addCompactionStateToS @Nullable List projections ) { + DimensionsSpec effectiveDimensions = DimensionsSpec.builder(dimensionsSpec) + .setDimensions( + dimensionsSpec.getDimensions() + .stream() + .map(dim -> dim.getEffectiveSchema(indexSpec)) + .collect(Collectors.toList()) + ) + .build(); CompactionState compactionState = new CompactionState( partitionsSpec, - dimensionsSpec, + effectiveDimensions, metricsSpec, transformSpec, indexSpec, diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index 98194eefdbd5..d298552d6ec7 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -209,7 +209,8 @@ static CompactionStatus compute( ) { final Evaluator evaluator = new Evaluator(candidateSegments, config, objectMapper); - return CHECKS.stream().map(f -> f.apply(evaluator)) + return CHECKS.stream() + .map(f -> f.apply(evaluator)) .filter(status -> !status.isComplete()) .findFirst().orElse(COMPLETE); } @@ -234,17 +235,21 @@ static PartitionsSpec findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuni private static List getNonPartitioningDimensions( @Nullable final List dimensionSchemas, - @Nullable final PartitionsSpec partitionsSpec + @Nullable final PartitionsSpec partitionsSpec, + @Nullable final IndexSpec indexSpec ) { if (dimensionSchemas == null || !(partitionsSpec instanceof DimensionRangePartitionsSpec)) { - return dimensionSchemas; + return dimensionSchemas.stream() + .map(dim -> dim.getEffectiveSchema(indexSpec)) + .collect(Collectors.toList()); } final List partitionsDimensions = ((DimensionRangePartitionsSpec) partitionsSpec).getPartitionDimensions(); return dimensionSchemas.stream() - .filter(dim -> !partitionsDimensions.contains(dim.getName())) - .collect(Collectors.toList()); + .filter(dim -> !partitionsDimensions.contains(dim.getName())) + .map(dim -> dim.getEffectiveSchema(indexSpec)) + .collect(Collectors.toList()); } /** @@ -430,20 +435,22 @@ private CompactionStatus dimensionsSpecIsUpToDate() lastCompactionState.getDimensionsSpec() == null ? null : lastCompactionState.getDimensionsSpec().getDimensions(), - lastCompactionState.getPartitionsSpec() + lastCompactionState.getPartitionsSpec(), + lastCompactionState.getIndexSpec() ); List configuredDimensions = getNonPartitioningDimensions( compactionConfig.getDimensionsSpec().getDimensions(), - compactionConfig.getTuningConfig() == null ? null : compactionConfig.getTuningConfig().getPartitionsSpec() + compactionConfig.getTuningConfig() == null ? null : compactionConfig.getTuningConfig().getPartitionsSpec(), + compactionConfig.getTuningConfig() == null + ? IndexSpec.DEFAULT + : compactionConfig.getTuningConfig().getIndexSpec() + ); + return CompactionStatus.completeIfEqual( + "dimensionsSpec", + configuredDimensions, + existingDimensions, + String::valueOf ); - { - return CompactionStatus.completeIfEqual( - "dimensionsSpec", - configuredDimensions, - existingDimensions, - String::valueOf - ); - } } } diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index 41d9a8d3d5fb..d0196d222bb3 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.indexer.granularity.GranularitySpec; @@ -35,12 +36,15 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.apache.druid.timeline.CompactionState; @@ -431,6 +435,114 @@ public void testStatusWhenProjectionsMismatch() Assert.assertFalse(status.isComplete()); } + @Test + public void testStatusWhenAutoSchemaMatch() + { + final GranularitySpec currentGranularitySpec + = new UniformGranularitySpec(Granularities.HOUR, null, null); + final PartitionsSpec currentPartitionsSpec = new DynamicPartitionsSpec(100, 0L); + + final CompactionState lastCompactionState = new CompactionState( + currentPartitionsSpec, + DimensionsSpec.builder() + .setDimensions( + List.of( + AutoTypeColumnSchema.of("x").getEffectiveSchema(IndexSpec.DEFAULT), + AutoTypeColumnSchema.of("y").getEffectiveSchema(IndexSpec.DEFAULT) + ) + ) + .build(), + null, + null, + IndexSpec.DEFAULT, + currentGranularitySpec, + Collections.emptyList() + ); + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withDimensionsSpec( + new UserCompactionTaskDimensionsConfig( + List.of( + new AutoTypeColumnSchema( + "x", + null, + NestedCommonFormatColumnFormatSpec.builder() + .setDoubleColumnCompression(CompressionStrategy.LZ4) + .build() + ), + AutoTypeColumnSchema.of("y") + ) + ) + ) + .withTuningConfig(createTuningConfig(currentPartitionsSpec, IndexSpec.DEFAULT)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) + .withProjections(Collections.emptyList()) + .build(); + + final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); + final CompactionStatus status = CompactionStatus.compute( + CompactionCandidate.from(Collections.singletonList(segment)), + compactionConfig, + OBJECT_MAPPER + ); + Assert.assertTrue(status.isComplete()); + } + + @Test + public void testStatusWhenAutoSchemaMismatch() + { + final GranularitySpec currentGranularitySpec + = new UniformGranularitySpec(Granularities.HOUR, null, null); + final PartitionsSpec currentPartitionsSpec = new DynamicPartitionsSpec(100, 0L); + + final CompactionState lastCompactionState = new CompactionState( + currentPartitionsSpec, + DimensionsSpec.builder() + .setDimensions( + List.of( + AutoTypeColumnSchema.of("x").getEffectiveSchema(IndexSpec.DEFAULT), + AutoTypeColumnSchema.of("y").getEffectiveSchema(IndexSpec.DEFAULT) + ) + ) + .build(), + null, + null, + IndexSpec.DEFAULT, + currentGranularitySpec, + Collections.emptyList() + ); + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withDimensionsSpec( + new UserCompactionTaskDimensionsConfig( + List.of( + new AutoTypeColumnSchema( + "x", + null, + NestedCommonFormatColumnFormatSpec.builder() + .setDoubleColumnCompression(CompressionStrategy.ZSTD) + .build() + ), + AutoTypeColumnSchema.of("y") + ) + ) + ) + .withTuningConfig(createTuningConfig(currentPartitionsSpec, IndexSpec.DEFAULT)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) + .withProjections(Collections.emptyList()) + .build(); + + final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); + final CompactionStatus status = CompactionStatus.compute( + CompactionCandidate.from(Collections.singletonList(segment)), + compactionConfig, + OBJECT_MAPPER + ); + Assert.assertFalse(status.isComplete()); + } + private void verifyCompactionStatusIsPendingBecause( CompactionState lastCompactionState, DataSourceCompactionConfig compactionConfig, From 97aa0da7c4980491b025f0fb65496f6c9fa869e8 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 29 Jul 2025 10:09:53 -0700 Subject: [PATCH 04/24] fix compile --- .../java/org/apache/druid/msq/exec/MSQReplaceTest.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 0afc2a45d324..df99956a3968 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -2779,9 +2779,10 @@ private CompactionState expectedCompactionState( return expectedCompactionState( context, partitionDimensions, - new DimensionsSpec.Builder().setDimensions(dimensions) - .setDimensionExclusions(Collections.singletonList("__time")) - .build(), + DimensionsSpec.builder() + .setDimensions(dimensions) + .setDimensionExclusions(Collections.singletonList("__time")) + .build(), segmentGranularity, interval ); From 1138259474e81d473e3102e5e132921736c90938 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 29 Jul 2025 10:52:20 -0700 Subject: [PATCH 05/24] fixes --- .../NestedCommonFormatColumnFormatSpec.java | 76 +++---------------- .../nested/NestedDataColumnSerializer.java | 8 +- .../segment/NestedDataColumnSchemaTest.java | 2 +- ...estedCommonFormatColumnFormatSpecTest.java | 8 +- .../nested/NestedDataColumnSupplierTest.java | 4 +- .../sql/calcite/util/SqlTestFramework.java | 2 - 6 files changed, 24 insertions(+), 76 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java index bd33793f0d6e..ac277d6da795 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java @@ -46,7 +46,7 @@ public static NestedCommonFormatColumnFormatSpec getEffectiveFormatSpec( return Objects.requireNonNullElseGet( columnFormatSpec, () -> NestedCommonFormatColumnFormatSpec.builder() - .setObjectKeysEncoding(StringEncodingStrategy.UTF8_STRATEGY) + .setObjectFieldsEncoding(StringEncodingStrategy.UTF8_STRATEGY) .setObjectStorageEncoding(ObjectStorageEncoding.SMILE) .build() ).getEffectiveSpec(indexSpec); @@ -54,7 +54,7 @@ public static NestedCommonFormatColumnFormatSpec getEffectiveFormatSpec( @Nullable @JsonProperty - private final StringEncodingStrategy objectKeysEncoding; + private final StringEncodingStrategy objectFieldsEncoding; @Nullable @JsonProperty private final ObjectStorageEncoding objectStorageEncoding; @@ -82,7 +82,7 @@ public static NestedCommonFormatColumnFormatSpec getEffectiveFormatSpec( @JsonCreator public NestedCommonFormatColumnFormatSpec( - @JsonProperty("objectKeysEncoding") @Nullable StringEncodingStrategy objectKeysEncoding, + @JsonProperty("objectFieldsEncoding") @Nullable StringEncodingStrategy objectFieldsEncoding, @JsonProperty("objectStorageEncoding") @Nullable ObjectStorageEncoding objectStorageEncoding, @JsonProperty("objectStorageCompression") @Nullable CompressionStrategy objectStorageCompression, @JsonProperty("stringDictionaryEncoding") @Nullable StringEncodingStrategy stringDictionaryEncoding, @@ -93,7 +93,7 @@ public NestedCommonFormatColumnFormatSpec( @JsonProperty("bitmapEncoding") @Nullable BitmapSerdeFactory bitmapEncoding ) { - this.objectKeysEncoding = objectKeysEncoding; + this.objectFieldsEncoding = objectFieldsEncoding; this.objectStorageEncoding = objectStorageEncoding; this.objectStorageCompression = objectStorageCompression; this.stringDictionaryEncoding = stringDictionaryEncoding; @@ -114,7 +114,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) ); } return new NestedCommonFormatColumnFormatSpec( - objectKeysEncoding != null ? objectKeysEncoding : StringEncodingStrategy.DEFAULT, + objectFieldsEncoding != null ? objectFieldsEncoding : StringEncodingStrategy.DEFAULT, objectStorageEncoding != null ? objectStorageEncoding : ObjectStorageEncoding.SMILE, objectStorageCompression != null ? objectStorageCompression @@ -132,9 +132,9 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) @Nullable @JsonProperty - public StringEncodingStrategy getObjectKeysEncoding() + public StringEncodingStrategy getObjectFieldsEncoding() { - return objectKeysEncoding; + return objectFieldsEncoding; } @Nullable @@ -200,7 +200,7 @@ public boolean equals(Object o) return false; } NestedCommonFormatColumnFormatSpec that = (NestedCommonFormatColumnFormatSpec) o; - return Objects.equals(objectKeysEncoding, that.objectKeysEncoding) + return Objects.equals(objectFieldsEncoding, that.objectFieldsEncoding) && objectStorageEncoding == that.objectStorageEncoding && objectStorageCompression == that.objectStorageCompression && Objects.equals(stringDictionaryEncoding, that.stringDictionaryEncoding) @@ -215,7 +215,7 @@ public boolean equals(Object o) public int hashCode() { return Objects.hash( - objectKeysEncoding, + objectFieldsEncoding, objectStorageEncoding, objectStorageCompression, stringDictionaryEncoding, @@ -231,7 +231,7 @@ public int hashCode() public String toString() { return "NestedCommonFormatColumnFormatSpec{" + - "objectKeysEncoding=" + objectKeysEncoding + + "objectKeysEncoding=" + objectFieldsEncoding + ", objectStorageEncoding=" + objectStorageEncoding + ", objectStorageCompression=" + objectStorageCompression + ", stringDictionaryEncoding=" + stringDictionaryEncoding + @@ -264,60 +264,30 @@ public static class Builder @Nullable private BitmapSerdeFactory bitmapEncoding; - @Nullable - public StringEncodingStrategy getObjectKeysEncoding() - { - return objectKeysEncoding; - } - - public Builder setObjectKeysEncoding(@Nullable StringEncodingStrategy objectKeysEncoding) + public Builder setObjectFieldsEncoding(@Nullable StringEncodingStrategy objectKeysEncoding) { this.objectKeysEncoding = objectKeysEncoding; return this; } - @Nullable - public ObjectStorageEncoding getObjectStorageEncoding() - { - return objectStorageEncoding; - } - public Builder setObjectStorageEncoding(@Nullable ObjectStorageEncoding objectStorageEncoding) { this.objectStorageEncoding = objectStorageEncoding; return this; } - @Nullable - public CompressionStrategy getObjectStorageCompression() - { - return objectStorageCompression; - } - public Builder setObjectStorageCompression(@Nullable CompressionStrategy objectStorageCompression) { this.objectStorageCompression = objectStorageCompression; return this; } - @Nullable - public StringEncodingStrategy getStringDictionaryEncoding() - { - return stringDictionaryEncoding; - } - public Builder setStringDictionaryEncoding(@Nullable StringEncodingStrategy stringDictionaryEncoding) { this.stringDictionaryEncoding = stringDictionaryEncoding; return this; } - @Nullable - public CompressionStrategy getDictionaryEncodedColumnCompression() - { - return dictionaryEncodedColumnCompression; - } - public Builder setDictionaryEncodedColumnCompression( @Nullable CompressionStrategy dictionaryEncodedColumnCompression ) @@ -326,48 +296,24 @@ public Builder setDictionaryEncodedColumnCompression( return this; } - @Nullable - public CompressionFactory.LongEncodingStrategy getLongColumnEncoding() - { - return longColumnEncoding; - } - public Builder setLongColumnEncoding(@Nullable CompressionFactory.LongEncodingStrategy longColumnEncoding) { this.longColumnEncoding = longColumnEncoding; return this; } - @Nullable - public CompressionStrategy getLongColumnCompression() - { - return longColumnCompression; - } - public Builder setLongColumnCompression(@Nullable CompressionStrategy longColumnCompression) { this.longColumnCompression = longColumnCompression; return this; } - @Nullable - public CompressionStrategy getDoubleColumnCompression() - { - return doubleColumnCompression; - } - public Builder setDoubleColumnCompression(@Nullable CompressionStrategy doubleColumnCompression) { this.doubleColumnCompression = doubleColumnCompression; return this; } - @Nullable - public BitmapSerdeFactory getBitmapEncoding() - { - return bitmapEncoding; - } - public Builder setBitmapEncoding(@Nullable BitmapSerdeFactory bitmapEncoding) { this.bitmapEncoding = bitmapEncoding; 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 e6a62e57d930..85c8e1c8c471 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 @@ -39,8 +39,6 @@ import org.apache.druid.segment.data.DictionaryWriter; import org.apache.druid.segment.data.FixedIndexedWriter; import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter; -import org.apache.druid.segment.data.GenericIndexed; -import org.apache.druid.segment.data.GenericIndexedWriter; import org.apache.druid.segment.serde.ColumnSerializerUtils; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; @@ -197,7 +195,11 @@ public boolean hasNulls() @Override public void openDictionaryWriter(File segmentBaseDir) throws IOException { - fieldsWriter = new GenericIndexedWriter<>(segmentWriteOutMedium, name, GenericIndexed.STRING_STRATEGY); + fieldsWriter = StringEncodingStrategies.getStringDictionaryWriter( + columnFormatSpec.getObjectFieldsEncoding(), + segmentWriteOutMedium, + name + ); fieldsWriter.open(); fieldsInfoWriter = new FieldTypeInfo.Writer(segmentWriteOutMedium); diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java index b6e12f7b3d49..9403bc371bca 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java @@ -35,7 +35,7 @@ public class NestedDataColumnSchemaTest private static final DefaultColumnFormatConfig DEFAULT_CONFIG = new DefaultColumnFormatConfig(null, null, null); private static final NestedCommonFormatColumnFormatSpec DEFAULT_NESTED_SPEC = NestedCommonFormatColumnFormatSpec.builder() - .setObjectKeysEncoding( + .setObjectFieldsEncoding( new StringEncodingStrategy.FrontCoded(8, FrontCodedIndexed.V1) ) .setObjectStorageCompression(CompressionStrategy.ZSTD) diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java index b2d4e7d701aa..ade9503cd1a0 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java @@ -39,7 +39,7 @@ public void testSerde() throws JsonProcessingException { NestedCommonFormatColumnFormatSpec spec = NestedCommonFormatColumnFormatSpec.builder() - .setObjectKeysEncoding( + .setObjectFieldsEncoding( new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1) ) .setObjectStorageCompression(CompressionStrategy.ZSTD) @@ -66,7 +66,7 @@ public void testGetEffectiveSpecDefaults() Assert.assertEquals( StringEncodingStrategy.UTF8_STRATEGY, - defaults.getObjectKeysEncoding() + defaults.getObjectFieldsEncoding() ); Assert.assertEquals( ObjectStorageEncoding.SMILE, @@ -99,7 +99,7 @@ public void testGetEffectiveSpecMerge() { NestedCommonFormatColumnFormatSpec merged = NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( NestedCommonFormatColumnFormatSpec.builder() - .setObjectKeysEncoding( + .setObjectFieldsEncoding( new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1) ) .setObjectStorageCompression(CompressionStrategy.ZSTD) @@ -113,7 +113,7 @@ public void testGetEffectiveSpecMerge() Assert.assertEquals( new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1), - merged.getObjectKeysEncoding() + merged.getObjectFieldsEncoding() ); Assert.assertEquals( new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1), 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 c97953d4c097..bad70b083ad3 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 @@ -55,6 +55,7 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.StringEncodingStrategy; import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.data.FrontCodedIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; @@ -150,7 +151,7 @@ public static Collection constructorFeeder() NestedCommonFormatColumnFormatSpec frontCodedKeysAndDicts = NestedCommonFormatColumnFormatSpec.builder() - .setObjectKeysEncoding( + .setObjectFieldsEncoding( new StringEncodingStrategy.FrontCoded(16, FrontCodedIndexed.V1) ) .setStringDictionaryEncoding( @@ -165,6 +166,7 @@ public static Collection constructorFeeder() NestedCommonFormatColumnFormatSpec lzf = NestedCommonFormatColumnFormatSpec.builder() .setDictionaryEncodedColumnCompression(CompressionStrategy.LZF) + .setLongColumnEncoding(CompressionFactory.LongEncodingStrategy.LONGS) .setLongColumnCompression(CompressionStrategy.LZF) .setDoubleColumnCompression(CompressionStrategy.LZF) .build(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java index 407669250f54..4cd08014bb33 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java @@ -78,7 +78,6 @@ import org.apache.druid.query.topn.TopNQueryConfig; import org.apache.druid.quidem.ProjectPathUtils; import org.apache.druid.quidem.TestSqlModule; -import org.apache.druid.segment.DefaultColumnFormatConfig; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; @@ -923,7 +922,6 @@ public void configure(Binder binder) { binder.bind(DruidOperatorTable.class).in(LazySingleton.class); binder.bind(DataSegment.PruneSpecsHolder.class).toInstance(DataSegment.PruneSpecsHolder.DEFAULT); - binder.bind(DefaultColumnFormatConfig.class).toInstance(new DefaultColumnFormatConfig(null, null, null)); } @Provides From 7c700f21f82c4bc49c0b5cd68b7bb58b6536fea4 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 29 Jul 2025 10:54:39 -0700 Subject: [PATCH 06/24] better name --- .../NestedCommonFormatColumnFormatSpec.java | 28 +++++++++---------- .../nested/NestedDataColumnSerializer.java | 2 +- .../segment/NestedDataColumnSchemaTest.java | 2 +- ...estedCommonFormatColumnFormatSpecTest.java | 8 +++--- .../nested/NestedDataColumnSupplierTest.java | 2 +- 5 files changed, 21 insertions(+), 21 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java index ac277d6da795..f7567a43045e 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java @@ -46,7 +46,7 @@ public static NestedCommonFormatColumnFormatSpec getEffectiveFormatSpec( return Objects.requireNonNullElseGet( columnFormatSpec, () -> NestedCommonFormatColumnFormatSpec.builder() - .setObjectFieldsEncoding(StringEncodingStrategy.UTF8_STRATEGY) + .setObjectFieldsDictionaryEncoding(StringEncodingStrategy.UTF8_STRATEGY) .setObjectStorageEncoding(ObjectStorageEncoding.SMILE) .build() ).getEffectiveSpec(indexSpec); @@ -54,7 +54,7 @@ public static NestedCommonFormatColumnFormatSpec getEffectiveFormatSpec( @Nullable @JsonProperty - private final StringEncodingStrategy objectFieldsEncoding; + private final StringEncodingStrategy objectFieldsDictionaryEncoding; @Nullable @JsonProperty private final ObjectStorageEncoding objectStorageEncoding; @@ -82,7 +82,7 @@ public static NestedCommonFormatColumnFormatSpec getEffectiveFormatSpec( @JsonCreator public NestedCommonFormatColumnFormatSpec( - @JsonProperty("objectFieldsEncoding") @Nullable StringEncodingStrategy objectFieldsEncoding, + @JsonProperty("objectFieldsDictionaryEncoding") @Nullable StringEncodingStrategy objectFieldsDictionaryEncoding, @JsonProperty("objectStorageEncoding") @Nullable ObjectStorageEncoding objectStorageEncoding, @JsonProperty("objectStorageCompression") @Nullable CompressionStrategy objectStorageCompression, @JsonProperty("stringDictionaryEncoding") @Nullable StringEncodingStrategy stringDictionaryEncoding, @@ -93,7 +93,7 @@ public NestedCommonFormatColumnFormatSpec( @JsonProperty("bitmapEncoding") @Nullable BitmapSerdeFactory bitmapEncoding ) { - this.objectFieldsEncoding = objectFieldsEncoding; + this.objectFieldsDictionaryEncoding = objectFieldsDictionaryEncoding; this.objectStorageEncoding = objectStorageEncoding; this.objectStorageCompression = objectStorageCompression; this.stringDictionaryEncoding = stringDictionaryEncoding; @@ -114,7 +114,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) ); } return new NestedCommonFormatColumnFormatSpec( - objectFieldsEncoding != null ? objectFieldsEncoding : StringEncodingStrategy.DEFAULT, + objectFieldsDictionaryEncoding != null ? objectFieldsDictionaryEncoding : StringEncodingStrategy.DEFAULT, objectStorageEncoding != null ? objectStorageEncoding : ObjectStorageEncoding.SMILE, objectStorageCompression != null ? objectStorageCompression @@ -132,9 +132,9 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) @Nullable @JsonProperty - public StringEncodingStrategy getObjectFieldsEncoding() + public StringEncodingStrategy getObjectFieldsDictionaryEncoding() { - return objectFieldsEncoding; + return objectFieldsDictionaryEncoding; } @Nullable @@ -200,7 +200,7 @@ public boolean equals(Object o) return false; } NestedCommonFormatColumnFormatSpec that = (NestedCommonFormatColumnFormatSpec) o; - return Objects.equals(objectFieldsEncoding, that.objectFieldsEncoding) + return Objects.equals(objectFieldsDictionaryEncoding, that.objectFieldsDictionaryEncoding) && objectStorageEncoding == that.objectStorageEncoding && objectStorageCompression == that.objectStorageCompression && Objects.equals(stringDictionaryEncoding, that.stringDictionaryEncoding) @@ -215,7 +215,7 @@ public boolean equals(Object o) public int hashCode() { return Objects.hash( - objectFieldsEncoding, + objectFieldsDictionaryEncoding, objectStorageEncoding, objectStorageCompression, stringDictionaryEncoding, @@ -231,7 +231,7 @@ public int hashCode() public String toString() { return "NestedCommonFormatColumnFormatSpec{" + - "objectKeysEncoding=" + objectFieldsEncoding + + "objectFieldsDictionaryEncoding=" + objectFieldsDictionaryEncoding + ", objectStorageEncoding=" + objectStorageEncoding + ", objectStorageCompression=" + objectStorageCompression + ", stringDictionaryEncoding=" + stringDictionaryEncoding + @@ -246,7 +246,7 @@ public String toString() public static class Builder { @Nullable - private StringEncodingStrategy objectKeysEncoding; + private StringEncodingStrategy objectFieldsDictionaryEncoding; @Nullable private ObjectStorageEncoding objectStorageEncoding; @Nullable @@ -264,9 +264,9 @@ public static class Builder @Nullable private BitmapSerdeFactory bitmapEncoding; - public Builder setObjectFieldsEncoding(@Nullable StringEncodingStrategy objectKeysEncoding) + public Builder setObjectFieldsDictionaryEncoding(@Nullable StringEncodingStrategy objectFieldsDictionaryEncoding) { - this.objectKeysEncoding = objectKeysEncoding; + this.objectFieldsDictionaryEncoding = objectFieldsDictionaryEncoding; return this; } @@ -323,7 +323,7 @@ public Builder setBitmapEncoding(@Nullable BitmapSerdeFactory bitmapEncoding) public NestedCommonFormatColumnFormatSpec build() { return new NestedCommonFormatColumnFormatSpec( - objectKeysEncoding, + objectFieldsDictionaryEncoding, objectStorageEncoding, objectStorageCompression, stringDictionaryEncoding, 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 85c8e1c8c471..821cf0e6d328 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 @@ -196,7 +196,7 @@ public boolean hasNulls() public void openDictionaryWriter(File segmentBaseDir) throws IOException { fieldsWriter = StringEncodingStrategies.getStringDictionaryWriter( - columnFormatSpec.getObjectFieldsEncoding(), + columnFormatSpec.getObjectFieldsDictionaryEncoding(), segmentWriteOutMedium, name ); diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java index 9403bc371bca..23c0e0302d8d 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java @@ -35,7 +35,7 @@ public class NestedDataColumnSchemaTest private static final DefaultColumnFormatConfig DEFAULT_CONFIG = new DefaultColumnFormatConfig(null, null, null); private static final NestedCommonFormatColumnFormatSpec DEFAULT_NESTED_SPEC = NestedCommonFormatColumnFormatSpec.builder() - .setObjectFieldsEncoding( + .setObjectFieldsDictionaryEncoding( new StringEncodingStrategy.FrontCoded(8, FrontCodedIndexed.V1) ) .setObjectStorageCompression(CompressionStrategy.ZSTD) diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java index ade9503cd1a0..284270eb1e90 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java @@ -39,7 +39,7 @@ public void testSerde() throws JsonProcessingException { NestedCommonFormatColumnFormatSpec spec = NestedCommonFormatColumnFormatSpec.builder() - .setObjectFieldsEncoding( + .setObjectFieldsDictionaryEncoding( new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1) ) .setObjectStorageCompression(CompressionStrategy.ZSTD) @@ -66,7 +66,7 @@ public void testGetEffectiveSpecDefaults() Assert.assertEquals( StringEncodingStrategy.UTF8_STRATEGY, - defaults.getObjectFieldsEncoding() + defaults.getObjectFieldsDictionaryEncoding() ); Assert.assertEquals( ObjectStorageEncoding.SMILE, @@ -99,7 +99,7 @@ public void testGetEffectiveSpecMerge() { NestedCommonFormatColumnFormatSpec merged = NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( NestedCommonFormatColumnFormatSpec.builder() - .setObjectFieldsEncoding( + .setObjectFieldsDictionaryEncoding( new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1) ) .setObjectStorageCompression(CompressionStrategy.ZSTD) @@ -113,7 +113,7 @@ public void testGetEffectiveSpecMerge() Assert.assertEquals( new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1), - merged.getObjectFieldsEncoding() + merged.getObjectFieldsDictionaryEncoding() ); Assert.assertEquals( new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1), 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 bad70b083ad3..23ecd7e31275 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 @@ -151,7 +151,7 @@ public static Collection constructorFeeder() NestedCommonFormatColumnFormatSpec frontCodedKeysAndDicts = NestedCommonFormatColumnFormatSpec.builder() - .setObjectFieldsEncoding( + .setObjectFieldsDictionaryEncoding( new StringEncodingStrategy.FrontCoded(16, FrontCodedIndexed.V1) ) .setStringDictionaryEncoding( From 29a15ee5644b5d3fb37e47ea3f2497a8d686f72a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 29 Jul 2025 10:56:04 -0700 Subject: [PATCH 07/24] fix --- .../druid/server/compaction/CompactionStatus.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index d298552d6ec7..3af7c5c08efa 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -233,6 +233,7 @@ static PartitionsSpec findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuni } } + @Nullable private static List getNonPartitioningDimensions( @Nullable final List dimensionSchemas, @Nullable final PartitionsSpec partitionsSpec, @@ -240,9 +241,12 @@ private static List getNonPartitioningDimensions( ) { if (dimensionSchemas == null || !(partitionsSpec instanceof DimensionRangePartitionsSpec)) { - return dimensionSchemas.stream() - .map(dim -> dim.getEffectiveSchema(indexSpec)) - .collect(Collectors.toList()); + if (dimensionSchemas != null) { + return dimensionSchemas.stream() + .map(dim -> dim.getEffectiveSchema(indexSpec)) + .collect(Collectors.toList()); + } + return null; } final List partitionsDimensions = ((DimensionRangePartitionsSpec) partitionsSpec).getPartitionDimensions(); From 2419dd9d7f2123c1a35b96c79bbd3db3855599ea Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 29 Jul 2025 11:21:03 -0700 Subject: [PATCH 08/24] fix order --- .../org/apache/druid/segment/incremental/IncrementalIndex.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 41cb339458dc..655ef1bdf1fe 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 @@ -37,8 +37,8 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.SpatialDimensionSchema; -import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.error.DruidException; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; From 0c47d5090a2af38cc80411772eb5abcd5334b9a1 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 29 Jul 2025 17:33:57 -0700 Subject: [PATCH 09/24] fix test --- .../druid/testing/embedded/compact/AutoCompactionTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java index 18bf06d5724f..81e929dac3a6 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java @@ -62,6 +62,7 @@ import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.AutoTypeColumnSchema; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.FixedIntervalOrderPolicy; @@ -499,7 +500,7 @@ public void testAutoCompactionPreservesCreateBitmapIndexInDimensionSchema(Compac List dimensionSchemas = ImmutableList.of( new StringDimensionSchema("language", DimensionSchema.MultiValueHandling.SORTED_ARRAY, false), - new AutoTypeColumnSchema("deleted", ColumnType.DOUBLE, null) + new AutoTypeColumnSchema("deleted", ColumnType.DOUBLE, null).getEffectiveSchema(IndexSpec.DEFAULT) ); submitCompactionConfig( From 026b958ae9d45a42eed57fe44ad6dfa65e08c532 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 22 Sep 2025 09:19:10 -0700 Subject: [PATCH 10/24] changes: * IndexSpec can now be defined as system default druid.indexing.formats.indexSpec * IndexSpec now has a NestedCommonFormatColumnFormatSpec autoColumnFormatSpec which can define default nested and auto column format in IndexSpec (either system default or job default) --- .../druid/indexer/HadoopTuningConfig.java | 7 +- .../druid/indexing/common/task/IndexTask.java | 3 +- .../SeekableStreamIndexTaskTuningConfig.java | 3 +- .../druid/guice/BuiltInTypesModule.java | 20 +-- .../druid/segment/AutoTypeColumnIndexer.java | 68 +------- .../druid/segment/AutoTypeColumnSchema.java | 3 +- .../segment/DefaultColumnFormatConfig.java | 25 ++- .../org/apache/druid/segment/IndexSpec.java | 39 ++++- .../druid/segment/NestedDataColumnSchema.java | 16 +- .../segment/incremental/IncrementalIndex.java | 4 +- .../nested/NestedCommonFormatColumn.java | 27 ++- .../NestedCommonFormatColumnFormatSpec.java | 158 +++++++++++++++--- .../nested/NestedDataComplexTypeSerde.java | 4 +- .../NestedCommonFormatColumnPartSerde.java | 76 ++++++++- .../druid/guice/BuiltInTypesModuleTest.java | 52 ++++-- .../druid/segment/CompleteSegmentTest.java | 6 + .../DefaultColumnFormatsConfigTest.java | 10 +- .../segment/NestedDataColumnSchemaTest.java | 3 +- .../nested/NestedDataColumnSupplierTest.java | 4 +- 19 files changed, 362 insertions(+), 166 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java index 32040fd37656..1aeeb8c117f4 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java @@ -46,7 +46,6 @@ public class HadoopTuningConfig implements TuningConfig private static final DimensionBasedPartitionsSpec DEFAULT_PARTITIONS_SPEC = HashedPartitionsSpec.defaultSpec(); private static final Map> DEFAULT_SHARD_SPECS = ImmutableMap.of(); - private static final IndexSpec DEFAULT_INDEX_SPEC = IndexSpec.DEFAULT; private static final boolean DEFAULT_USE_COMBINER = false; private static final int DEFAULT_NUM_BACKGROUND_PERSIST_THREADS = 0; @@ -57,8 +56,8 @@ public static HadoopTuningConfig makeDefaultTuningConfig() DateTimes.nowUtc().toString(), DEFAULT_PARTITIONS_SPEC, DEFAULT_SHARD_SPECS, - DEFAULT_INDEX_SPEC, - DEFAULT_INDEX_SPEC, + IndexSpec.DEFAULT, + IndexSpec.DEFAULT, DEFAULT_APPENDABLE_INDEX, DEFAULT_MAX_ROWS_IN_MEMORY_BATCH, 0L, @@ -151,7 +150,7 @@ public HadoopTuningConfig( this.version = Configs.valueOrDefault(version, DateTimes.nowUtc().toString()); this.partitionsSpec = Configs.valueOrDefault(partitionsSpec, DEFAULT_PARTITIONS_SPEC); this.shardSpecs = Configs.valueOrDefault(shardSpecs, DEFAULT_SHARD_SPECS); - this.indexSpec = Configs.valueOrDefault(indexSpec, DEFAULT_INDEX_SPEC); + this.indexSpec = Configs.valueOrDefault(indexSpec, IndexSpec.DEFAULT); this.indexSpecForIntermediatePersists = Configs.valueOrDefault( indexSpecForIntermediatePersists, this.indexSpec diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index d9eb9152b54f..51dd1aa0528e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -1182,7 +1182,6 @@ public boolean isDropExisting() public static class IndexTuningConfig implements AppenderatorConfig { - private static final IndexSpec DEFAULT_INDEX_SPEC = IndexSpec.DEFAULT; private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; private static final boolean DEFAULT_GUARANTEE_ROLLUP = false; private static final boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = false; @@ -1364,7 +1363,7 @@ private IndexTuningConfig( ? IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE : maxColumnsToMerge; this.partitionsSpec = partitionsSpec; - this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; + this.indexSpec = indexSpec == null ? IndexSpec.DEFAULT : indexSpec; this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists == null ? this.indexSpec : indexSpecForIntermediatePersists; this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index ea531d77f277..fd63ba1132e6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -40,7 +40,6 @@ public abstract class SeekableStreamIndexTaskTuningConfig implements TuningConfi private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; private static final boolean DEFAULT_SKIP_SEQUENCE_NUMBER_AVAILABILITY_CHECK = false; private static final Period DEFAULT_INTERMEDIATE_PERSIST_PERIOD = new Period("PT10M"); - private static final IndexSpec DEFAULT_INDEX_SPEC = IndexSpec.DEFAULT; private static final Boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = Boolean.FALSE; private static final boolean DEFAULT_RELEASE_LOCKS_ON_HANDOFF = false; private static final long DEFAULT_HANDOFF_CONDITION_TIMEOUT = Duration.ofMinutes(15).toMillis(); @@ -110,7 +109,7 @@ public SeekableStreamIndexTaskTuningConfig( intermediatePersistPeriod == null ? DEFAULT_INTERMEDIATE_PERSIST_PERIOD : intermediatePersistPeriod; this.basePersistDirectory = basePersistDirectory; this.maxPendingPersists = maxPendingPersists == null ? 0 : maxPendingPersists; - this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; + this.indexSpec = indexSpec == null ? IndexSpec.DEFAULT : indexSpec; this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists == null ? this.indexSpec : indexSpecForIntermediatePersists; this.reportParseExceptions = reportParseExceptions == null diff --git a/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java b/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java index 15e757ffa3bd..f08bd693739e 100644 --- a/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java +++ b/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java @@ -30,8 +30,8 @@ import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.DimensionHandlerProvider; import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.NestedCommonFormatColumnHandler; -import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.apache.druid.segment.nested.StructuredData; import org.apache.druid.segment.nested.StructuredDataJsonSerializer; @@ -52,8 +52,6 @@ public class BuiltInTypesModule implements DruidModule * {@link #initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig)}. */ private static DimensionSchema.MultiValueHandling STRING_MV_MODE = DimensionSchema.MultiValueHandling.SORTED_ARRAY; - @Nullable - private static NestedCommonFormatColumnFormatSpec DEFAULT_NESTED_COMMON_FORMAT_SPEC = null; @Override public List getJacksonModules() @@ -76,6 +74,7 @@ public void configure(Binder binder) public SideEffectRegisterer initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig formatsConfig) { setStringMultiValueHandlingModeIfConfigured(formatsConfig.getStringMultiValueHandlingMode()); + setIndexSpecDefaults(formatsConfig); setNestedColumnDefaults(formatsConfig); return new SideEffectRegisterer(); @@ -88,6 +87,13 @@ private static void setStringMultiValueHandlingModeIfConfigured(@Nullable String } } + private static void setIndexSpecDefaults(DefaultColumnFormatConfig defaultColumnFormatConfig) + { + if (defaultColumnFormatConfig.getIndexSpec() != null) { + IndexSpec.DEFAULT = defaultColumnFormatConfig.getIndexSpec(); + } + } + private static void setNestedColumnDefaults(DefaultColumnFormatConfig formatsConfig) { if (formatsConfig.getNestedColumnFormatVersion() == null || formatsConfig.getNestedColumnFormatVersion() == 5) { @@ -96,7 +102,6 @@ private static void setNestedColumnDefaults(DefaultColumnFormatConfig formatsCon new NestedCommonFormatHandlerProvider() ); } - DEFAULT_NESTED_COMMON_FORMAT_SPEC = formatsConfig.getNestedColumnFormatSpec(); } /** @@ -108,11 +113,6 @@ public static DimensionSchema.MultiValueHandling getStringMultiValueHandlingMode return STRING_MV_MODE; } - public static NestedCommonFormatColumnFormatSpec getDefaultNestedCommonFormatSpec() - { - return DEFAULT_NESTED_COMMON_FORMAT_SPEC; - } - public static List getJacksonModulesList() { return Collections.singletonList( @@ -153,7 +153,7 @@ public static class NestedCommonFormatHandlerProvider @Override public DimensionHandler get(String dimensionName) { - return new NestedCommonFormatColumnHandler(dimensionName, null, DEFAULT_NESTED_COMMON_FORMAT_SPEC); + return new NestedCommonFormatColumnHandler(dimensionName, null, IndexSpec.DEFAULT.getAutoColumnFormatSpec()); } } diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java index 724c4d6ff284..0f1ca2ba5273 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java @@ -23,9 +23,7 @@ import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.common.guava.GuavaUtils; -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.Numbers; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.parsers.ParseException; @@ -43,6 +41,7 @@ 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.NestedCommonFormatColumn; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.nested.NestedPathFinder; import org.apache.druid.segment.nested.NestedPathPart; @@ -758,60 +757,17 @@ public boolean isSingleType() } } - static class Format implements ColumnFormat + static class Format extends NestedCommonFormatColumn.Format { - private final ColumnType logicalType; - private final boolean hasNulls; - private final boolean enforceLogicalType; - private final NestedCommonFormatColumnFormatSpec columnFormatSpec; - Format(ColumnType logicalType, boolean hasNulls, boolean enforceLogicalType, NestedCommonFormatColumnFormatSpec columnFormatSpec) - { - this.logicalType = logicalType; - this.hasNulls = hasNulls; - this.enforceLogicalType = enforceLogicalType; - this.columnFormatSpec = columnFormatSpec; - } - - @Override - public ColumnType getLogicalType() - { - return logicalType; - } - - @Override - public DimensionHandler getColumnHandler(String columnName) - { - return new NestedCommonFormatColumnHandler(columnName, enforceLogicalType ? logicalType : null, columnFormatSpec); - } - - @Override - public DimensionSchema getColumnSchema(String columnName) - { - return new AutoTypeColumnSchema(columnName, enforceLogicalType ? logicalType : null, columnFormatSpec); - } - - @Override - public ColumnFormat merge(@Nullable ColumnFormat otherFormat) + public Format( + ColumnType logicalType, + boolean hasNulls, + boolean enforceLogicalType, + NestedCommonFormatColumnFormatSpec columnFormatSpec + ) { - if (otherFormat == null) { - return this; - } - if (otherFormat instanceof Format) { - final Format other = (Format) otherFormat; - // todo (clint): actually merge column format spec, maybe? - if (!getLogicalType().equals(other.getLogicalType())) { - return new Format(ColumnType.NESTED_DATA, hasNulls || other.hasNulls, false, columnFormatSpec); - } - return new Format(logicalType, hasNulls || other.hasNulls, enforceLogicalType || other.enforceLogicalType, columnFormatSpec); - } - 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() - ); + super(logicalType, hasNulls, enforceLogicalType, columnFormatSpec); } @Override @@ -822,10 +778,4 @@ public ColumnCapabilities toColumnCapabilities() .setHasNulls(hasNulls); } } - - @Nullable - private static Object getDefaultValueForType(@Nullable ColumnType columnType) - { - return null; - } } diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java index 0c6416950f56..7769cdfd1c7e 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.nested.NestedCommonFormatColumn; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; @@ -96,7 +95,7 @@ public AutoTypeColumnSchema( this.castToType = castToType; } this.columnFormatSpec = columnFormatSpec == null - ? BuiltInTypesModule.getDefaultNestedCommonFormatSpec() + ? IndexSpec.DEFAULT.getAutoColumnFormatSpec() : columnFormatSpec; } diff --git a/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java b/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java index 28e9705f8470..210ec5c686b0 100644 --- a/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java +++ b/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java @@ -24,7 +24,6 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import javax.annotation.Nullable; import java.util.Arrays; @@ -69,23 +68,23 @@ private static String validateMultiValueHandlingMode( return stringMultiValueHandlingMode; } + @JsonProperty("stringMultiValueHandlingMode") @Nullable - @JsonProperty("nestedColumnFormatVersion") private final Integer nestedColumnFormatVersion; + @JsonProperty("nestedColumnFormatVersion") @Nullable - @JsonProperty("stringMultiValueHandlingMode") private final String stringMultiValueHandlingMode; + @JsonProperty("indexSpec") @Nullable - @JsonProperty - private final NestedCommonFormatColumnFormatSpec nestedColumnFormatSpec; + private final IndexSpec indexSpec; @JsonCreator public DefaultColumnFormatConfig( @JsonProperty("stringMultiValueHandlingMode") @Nullable String stringMultiValueHandlingMode, @JsonProperty("nestedColumnFormatVersion") @Nullable Integer nestedColumnFormatVersion, - @JsonProperty("nestedColumnFormatSpec") @Nullable NestedCommonFormatColumnFormatSpec nestedColumnFormatSpec + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec ) { validateMultiValueHandlingMode(stringMultiValueHandlingMode); @@ -93,7 +92,7 @@ public DefaultColumnFormatConfig( this.stringMultiValueHandlingMode = validateMultiValueHandlingMode(stringMultiValueHandlingMode); this.nestedColumnFormatVersion = nestedColumnFormatVersion; - this.nestedColumnFormatSpec = nestedColumnFormatSpec; + this.indexSpec = indexSpec; } @Nullable @@ -111,10 +110,10 @@ public Integer getNestedColumnFormatVersion() } @Nullable - @JsonProperty("nestedColumnFormatSpec") - public NestedCommonFormatColumnFormatSpec getNestedColumnFormatSpec() + @JsonProperty("indexSpec") + public IndexSpec getIndexSpec() { - return nestedColumnFormatSpec; + return indexSpec; } @Override @@ -129,13 +128,13 @@ public boolean equals(Object o) DefaultColumnFormatConfig that = (DefaultColumnFormatConfig) o; return Objects.equals(nestedColumnFormatVersion, that.nestedColumnFormatVersion) && Objects.equals(stringMultiValueHandlingMode, that.stringMultiValueHandlingMode) - && Objects.equals(nestedColumnFormatSpec, that.nestedColumnFormatSpec); + && Objects.equals(indexSpec, that.indexSpec); } @Override public int hashCode() { - return Objects.hash(nestedColumnFormatVersion, stringMultiValueHandlingMode, nestedColumnFormatSpec); + return Objects.hash(nestedColumnFormatVersion, stringMultiValueHandlingMode, indexSpec); } @Override @@ -144,7 +143,7 @@ public String toString() return "DefaultColumnFormatConfig{" + "stringMultiValueHandlingMode=" + stringMultiValueHandlingMode + ", nestedColumnFormatVersion=" + nestedColumnFormatVersion + - ", nestedColumnFormatSpec=" + nestedColumnFormatSpec + + ", indexSpec=" + indexSpec + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java index b111975b3945..c4903b31a894 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java @@ -28,6 +28,7 @@ import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.loading.SegmentizerFactory; +import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import javax.annotation.Nullable; import java.util.Objects; @@ -40,7 +41,7 @@ */ public class IndexSpec { - public static final IndexSpec DEFAULT = IndexSpec.builder().build(); + public static IndexSpec DEFAULT = IndexSpec.builder().build(); public static Builder builder() { @@ -58,6 +59,7 @@ public static Builder builder() private final CompressionStrategy jsonCompression; @Nullable private final SegmentizerFactory segmentLoader; + private final NestedCommonFormatColumnFormatSpec autoColumnFormatSpec; /** * Creates an IndexSpec with the given storage format settings. @@ -83,8 +85,9 @@ public IndexSpec( @JsonProperty("metricCompression") @Nullable CompressionStrategy metricCompression, @JsonProperty("longEncoding") @Nullable CompressionFactory.LongEncodingStrategy longEncoding, @JsonProperty("complexMetricCompression") @Nullable CompressionStrategy complexMetricCompression, - @JsonProperty("jsonCompression") @Nullable CompressionStrategy jsonCompression, - @JsonProperty("segmentLoader") @Nullable SegmentizerFactory segmentLoader + @Deprecated @JsonProperty("jsonCompression") @Nullable CompressionStrategy jsonCompression, + @JsonProperty("segmentLoader") @Nullable SegmentizerFactory segmentLoader, + @JsonProperty("autoColumnFormatSpec") @Nullable NestedCommonFormatColumnFormatSpec autoColumnFormatSpec ) { this.bitmapSerdeFactory = bitmapSerdeFactory != null @@ -106,6 +109,11 @@ public IndexSpec( : longEncoding; this.jsonCompression = jsonCompression; this.segmentLoader = segmentLoader; + this.autoColumnFormatSpec = autoColumnFormatSpec == null + ? NestedCommonFormatColumnFormatSpec.builder() + .setObjectStorageCompression(jsonCompression) + .build() + : autoColumnFormatSpec; } @JsonProperty("bitmap") @@ -154,6 +162,7 @@ public SegmentizerFactory getSegmentLoader() return segmentLoader; } + @Deprecated @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable @@ -162,6 +171,13 @@ public CompressionStrategy getJsonCompression() return jsonCompression; } + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public NestedCommonFormatColumnFormatSpec getAutoColumnFormatSpec() + { + return autoColumnFormatSpec; + } + @Override public boolean equals(Object o) { @@ -179,7 +195,8 @@ public boolean equals(Object o) longEncoding == indexSpec.longEncoding && Objects.equals(complexMetricCompression, indexSpec.complexMetricCompression) && Objects.equals(jsonCompression, indexSpec.jsonCompression) && - Objects.equals(segmentLoader, indexSpec.segmentLoader); + Objects.equals(segmentLoader, indexSpec.segmentLoader) && + Objects.equals(autoColumnFormatSpec, indexSpec.autoColumnFormatSpec); } @Override @@ -193,7 +210,8 @@ public int hashCode() longEncoding, complexMetricCompression, jsonCompression, - segmentLoader + segmentLoader, + autoColumnFormatSpec ); } @@ -230,6 +248,8 @@ public static class Builder private CompressionStrategy jsonCompression; @Nullable private SegmentizerFactory segmentLoader; + @Nullable + private NestedCommonFormatColumnFormatSpec autoColumnFormatSpec; public Builder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory) { @@ -279,6 +299,12 @@ public Builder withSegmentLoader(SegmentizerFactory segmentLoader) return this; } + public Builder withAutoColumnFormatSpec(NestedCommonFormatColumnFormatSpec autoColumnFormatSpec) + { + this.autoColumnFormatSpec = autoColumnFormatSpec; + return this; + } + public IndexSpec build() { return new IndexSpec( @@ -289,7 +315,8 @@ public IndexSpec build() longEncoding, complexMetricCompression, jsonCompression, - segmentLoader + segmentLoader, + autoColumnFormatSpec ); } } diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java index 7427db0d30be..64ec1a2b768f 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java @@ -58,10 +58,16 @@ public NestedDataColumnSchema( // but as far as this is concerned it is v5 formatVersion = 5; } - DefaultColumnFormatConfig.validateNestedFormatVersion(this.formatVersion); - this.columnFormatSpec = columnFormatSpec == null - ? defaultFormatConfig.getNestedColumnFormatSpec() - : columnFormatSpec; + DefaultColumnFormatConfig.validateNestedFormatVersion(formatVersion); + if (columnFormatSpec == null) { + if (defaultFormatConfig.getIndexSpec() != null) { + this.columnFormatSpec = defaultFormatConfig.getIndexSpec().getAutoColumnFormatSpec(); + } else { + this.columnFormatSpec = IndexSpec.DEFAULT.getAutoColumnFormatSpec(); + } + } else { + this.columnFormatSpec = columnFormatSpec; + } } public NestedDataColumnSchema( @@ -72,7 +78,7 @@ public NestedDataColumnSchema( super(name, null, true); this.formatVersion = version; DefaultColumnFormatConfig.validateNestedFormatVersion(this.formatVersion); - this.columnFormatSpec = null; + this.columnFormatSpec = IndexSpec.DEFAULT.getAutoColumnFormatSpec(); } @JsonProperty("formatVersion") 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 655ef1bdf1fe..0da4898df2a5 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 @@ -38,7 +38,6 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.SpatialDimensionSchema; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -61,6 +60,7 @@ import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.EncodedKeyComponent; import org.apache.druid.segment.FloatColumnSelector; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.NestedCommonFormatColumnHandler; @@ -521,7 +521,7 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) wasNewDim = true; final DimensionHandler handler; if (useSchemaDiscovery) { - handler = new NestedCommonFormatColumnHandler(dimension, null, BuiltInTypesModule.getDefaultNestedCommonFormatSpec()); + handler = new NestedCommonFormatColumnHandler(dimension, null, IndexSpec.DEFAULT.getAutoColumnFormatSpec()); } else { // legacy behavior: for schemaless type discovery, everything is a String handler = DimensionHandlerUtils.getHandlerFromCapabilities( 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 index 0bf0395fa0e3..5db56a30a72a 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java @@ -87,10 +87,10 @@ default SortedMap getFieldTypeInfo() class Format implements ColumnFormat { - private final ColumnType logicalType; - private final boolean hasNulls; - private final boolean enforceLogicalType; - private final NestedCommonFormatColumnFormatSpec columnFormatSpec; + protected final ColumnType logicalType; + protected final boolean hasNulls; + protected final boolean enforceLogicalType; + protected final NestedCommonFormatColumnFormatSpec columnFormatSpec; public Format( ColumnType logicalType, @@ -132,11 +132,22 @@ public ColumnFormat merge(@Nullable ColumnFormat otherFormat) if (otherFormat instanceof Format) { final Format other = (Format) otherFormat; - // todo (clint): actually merge columnFormatSpec, maybe - if (!getLogicalType().equals(other.getLogicalType())) { - return new Format(ColumnType.NESTED_DATA, hasNulls || other.hasNulls, false, columnFormatSpec); + // when merging formats in the same ingestion job, all segments should have the exact same columnFormatSpec, so + // no need to merge that + if (!logicalType.equals(other.logicalType)) { + return new Format( + ColumnType.leastRestrictiveType(logicalType, other.logicalType), + hasNulls || other.hasNulls, + false, + columnFormatSpec + ); } - return new Format(logicalType, hasNulls || other.hasNulls, enforceLogicalType || other.enforceLogicalType, columnFormatSpec); + return new Format( + logicalType, + hasNulls || other.hasNulls, + enforceLogicalType || other.enforceLogicalType, + columnFormatSpec + ); } throw new ISE( "Cannot merge columns of type[%s] and format[%s] and with [%s] and [%s]", diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java index f7567a43045e..8380e618c943 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.nested; 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.ISE; import org.apache.druid.segment.IndexSpec; @@ -53,31 +54,22 @@ public static NestedCommonFormatColumnFormatSpec getEffectiveFormatSpec( } @Nullable - @JsonProperty private final StringEncodingStrategy objectFieldsDictionaryEncoding; @Nullable - @JsonProperty private final ObjectStorageEncoding objectStorageEncoding; @Nullable - @JsonProperty private final CompressionStrategy objectStorageCompression; @Nullable - @JsonProperty private final StringEncodingStrategy stringDictionaryEncoding; @Nullable - @JsonProperty private final CompressionStrategy dictionaryEncodedColumnCompression; @Nullable - @JsonProperty private final CompressionFactory.LongEncodingStrategy longColumnEncoding; @Nullable - @JsonProperty private final CompressionStrategy longColumnCompression; @Nullable - @JsonProperty private final CompressionStrategy doubleColumnCompression; @Nullable - @JsonProperty private final BitmapSerdeFactory bitmapEncoding; @JsonCreator @@ -89,8 +81,37 @@ public NestedCommonFormatColumnFormatSpec( @JsonProperty("dictionaryEncodedColumnCompression") @Nullable CompressionStrategy dictionaryEncodedColumnCompression, @JsonProperty("longColumnEncoding") @Nullable CompressionFactory.LongEncodingStrategy longColumnEncoding, @JsonProperty("longColumnCompression") @Nullable CompressionStrategy longColumnCompression, - @JsonProperty("doubleColumnCompression") @Nullable CompressionStrategy doubleColumnCompression, - @JsonProperty("bitmapEncoding") @Nullable BitmapSerdeFactory bitmapEncoding + @JsonProperty("doubleColumnCompression") @Nullable CompressionStrategy doubleColumnCompression + ) + { + this( + objectFieldsDictionaryEncoding, + objectStorageEncoding, + objectStorageCompression, + stringDictionaryEncoding, + dictionaryEncodedColumnCompression, + longColumnEncoding, + longColumnCompression, + doubleColumnCompression, + null + ); + } + + /** + * Internal constructor used by {@link Builder} to set {@link #bitmapEncoding} during the process of resolving values + * for {@link #getEffectiveSpec(IndexSpec)}. {@link #bitmapEncoding} cannot vary per column, and is always set from + * {@link IndexSpec#getBitmapSerdeFactory()}. + */ + protected NestedCommonFormatColumnFormatSpec( + @Nullable StringEncodingStrategy objectFieldsDictionaryEncoding, + @Nullable ObjectStorageEncoding objectStorageEncoding, + @Nullable CompressionStrategy objectStorageCompression, + @Nullable StringEncodingStrategy stringDictionaryEncoding, + @Nullable CompressionStrategy dictionaryEncodedColumnCompression, + @Nullable CompressionFactory.LongEncodingStrategy longColumnEncoding, + @Nullable CompressionStrategy longColumnCompression, + @Nullable CompressionStrategy doubleColumnCompression, + @Nullable BitmapSerdeFactory bitmapEncoding ) { this.objectFieldsDictionaryEncoding = objectFieldsDictionaryEncoding; @@ -104,8 +125,14 @@ public NestedCommonFormatColumnFormatSpec( this.bitmapEncoding = bitmapEncoding; } + /** + * Fully populate all fields of {@link NestedCommonFormatColumnFormatSpec}. Null values are populated first checking + * {@link IndexSpec#getAutoColumnFormatSpec()}, then falling back to fields on {@link IndexSpec} itself if applicable, + * and finally resorting to hard coded defaults. + */ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) { + // this is a defensive check, the json spec can't set this, only the builder can if (bitmapEncoding != null && !bitmapEncoding.equals(indexSpec.getBitmapSerdeFactory())) { throw new ISE( "bitmapEncoding[%s] does not match indexSpec.bitmap[%s]", @@ -113,21 +140,80 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) indexSpec.getBitmapSerdeFactory() ); } - return new NestedCommonFormatColumnFormatSpec( - objectFieldsDictionaryEncoding != null ? objectFieldsDictionaryEncoding : StringEncodingStrategy.DEFAULT, - objectStorageEncoding != null ? objectStorageEncoding : ObjectStorageEncoding.SMILE, - objectStorageCompression != null - ? objectStorageCompression - : indexSpec.getJsonCompression() != null ? indexSpec.getJsonCompression() : CompressionStrategy.LZ4, - stringDictionaryEncoding != null ? stringDictionaryEncoding : indexSpec.getStringDictionaryEncoding(), - dictionaryEncodedColumnCompression != null - ? dictionaryEncodedColumnCompression - : indexSpec.getDimensionCompression(), - longColumnEncoding != null ? longColumnEncoding : indexSpec.getLongEncoding(), - longColumnCompression != null ? longColumnCompression : indexSpec.getMetricCompression(), - doubleColumnCompression != null ? doubleColumnCompression : indexSpec.getMetricCompression(), - bitmapEncoding != null ? bitmapEncoding : indexSpec.getBitmapSerdeFactory() - ); + Builder builder = new Builder(this); + builder.setBitmapEncoding(indexSpec.getBitmapSerdeFactory()); + + if (objectFieldsDictionaryEncoding == null) { + if (indexSpec.getAutoColumnFormatSpec().getObjectFieldsDictionaryEncoding() != null) { + builder.setObjectFieldsDictionaryEncoding( + indexSpec.getAutoColumnFormatSpec().getObjectFieldsDictionaryEncoding() + ); + } else { + builder.setObjectFieldsDictionaryEncoding(StringEncodingStrategy.DEFAULT); + } + } + + if (objectStorageEncoding == null) { + if (indexSpec.getAutoColumnFormatSpec().getObjectStorageEncoding() != null) { + builder.setObjectStorageEncoding(indexSpec.getAutoColumnFormatSpec().getObjectStorageEncoding()); + } else { + builder.setObjectStorageEncoding(ObjectStorageEncoding.SMILE); + } + } + + if (objectStorageCompression == null) { + if (indexSpec.getAutoColumnFormatSpec().getObjectStorageCompression() != null) { + builder.setObjectStorageCompression(indexSpec.getAutoColumnFormatSpec().getObjectStorageCompression()); + } else if (indexSpec.getJsonCompression() != null) { + builder.setObjectStorageCompression(indexSpec.getJsonCompression()); + } else { + builder.setObjectStorageCompression(CompressionStrategy.LZ4); + } + } + + if (stringDictionaryEncoding == null) { + if (indexSpec.getAutoColumnFormatSpec().getStringDictionaryEncoding() != null) { + builder.setStringDictionaryEncoding(indexSpec.getAutoColumnFormatSpec().getStringDictionaryEncoding()); + } else { + builder.setStringDictionaryEncoding(indexSpec.getStringDictionaryEncoding()); + } + } + + if (dictionaryEncodedColumnCompression == null) { + if (indexSpec.getAutoColumnFormatSpec().getDictionaryEncodedColumnCompression() != null) { + builder.setDictionaryEncodedColumnCompression( + indexSpec.getAutoColumnFormatSpec().getDictionaryEncodedColumnCompression() + ); + } else { + builder.setDictionaryEncodedColumnCompression(indexSpec.getDimensionCompression()); + } + } + + if (longColumnEncoding == null) { + if (indexSpec.getAutoColumnFormatSpec().getLongColumnEncoding() != null) { + builder.setLongColumnEncoding(indexSpec.getAutoColumnFormatSpec().getLongColumnEncoding()); + } else { + builder.setLongColumnEncoding(indexSpec.getLongEncoding()); + } + } + + if (longColumnCompression == null) { + if (indexSpec.getAutoColumnFormatSpec().getLongColumnCompression() != null) { + builder.setLongColumnCompression(indexSpec.getAutoColumnFormatSpec().getLongColumnCompression()); + } else { + builder.setLongColumnCompression(indexSpec.getMetricCompression()); + } + } + + if (doubleColumnCompression == null) { + if (indexSpec.getAutoColumnFormatSpec().getDoubleColumnCompression() != null) { + builder.setDoubleColumnCompression(indexSpec.getAutoColumnFormatSpec().getDoubleColumnCompression()); + } else { + builder.setDoubleColumnCompression(indexSpec.getMetricCompression()); + } + } + + return builder.build(); } @Nullable @@ -187,7 +273,7 @@ public CompressionStrategy getDoubleColumnCompression() } @Nullable - @JsonProperty + @JsonIgnore public BitmapSerdeFactory getBitmapEncoding() { return bitmapEncoding; @@ -264,6 +350,24 @@ public static class Builder @Nullable private BitmapSerdeFactory bitmapEncoding; + public Builder() + { + + } + + public Builder(NestedCommonFormatColumnFormatSpec spec) + { + this.objectFieldsDictionaryEncoding = spec.objectFieldsDictionaryEncoding; + this.objectStorageEncoding = spec.objectStorageEncoding; + this.objectStorageCompression = spec.objectStorageCompression; + this.stringDictionaryEncoding = spec.stringDictionaryEncoding; + this.dictionaryEncodedColumnCompression = spec.dictionaryEncodedColumnCompression; + this.longColumnEncoding = spec.longColumnEncoding; + this.longColumnCompression = spec.longColumnCompression; + this.doubleColumnCompression = spec.doubleColumnCompression; + this.bitmapEncoding = spec.bitmapEncoding; + } + public Builder setObjectFieldsDictionaryEncoding(@Nullable StringEncodingStrategy objectFieldsDictionaryEncoding) { this.objectFieldsDictionaryEncoding = objectFieldsDictionaryEncoding; 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 63d52ebdad99..1a0e3a6120b4 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,9 +23,9 @@ import it.unimi.dsi.fastutil.Hash; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.NestedCommonFormatColumnHandler; import org.apache.druid.segment.NestedDataColumnSchema; import org.apache.druid.segment.column.ColumnBuilder; @@ -235,7 +235,7 @@ public ColumnType getLogicalType() @Override public DimensionHandler getColumnHandler(String columnName) { - return new NestedCommonFormatColumnHandler(columnName, null, BuiltInTypesModule.getDefaultNestedCommonFormatSpec()); + return new NestedCommonFormatColumnHandler(columnName, null, IndexSpec.DEFAULT.getAutoColumnFormatSpec()); } @Override 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 index d6c9ae38e3d7..fa74a2d98c32 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java @@ -28,8 +28,11 @@ import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.StringEncodingStrategy; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.CompressionFactory; +import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.nested.NestedCommonFormatColumn; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; @@ -83,7 +86,7 @@ public static NestedCommonFormatColumnPartSerde createDeserializer( @JsonProperty("enforceLogicalType") boolean enforceLogicalType, @JsonProperty("byteOrder") ByteOrder byteOrder, @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory, - @JsonProperty("columnFormatSpec") @Nullable NestedCommonFormatColumnFormatSpec columnFormatSpec + @JsonProperty("columnFormatSpec") @Nullable FormatSpec columnFormatSpec ) { return new NestedCommonFormatColumnPartSerde( @@ -105,7 +108,7 @@ public static NestedCommonFormatColumnPartSerde createDeserializer( private final ByteOrder byteOrder; private final BitmapSerdeFactory bitmapSerdeFactory; @Nullable - private final NestedCommonFormatColumnFormatSpec columnFormatSpec; + private final FormatSpec columnFormatSpec; @Nullable private final Serializer serializer; @@ -118,7 +121,7 @@ private NestedCommonFormatColumnPartSerde( boolean enforceLogicalType, ByteOrder byteOrder, BitmapSerdeFactory bitmapSerdeFactory, - @Nullable NestedCommonFormatColumnFormatSpec columnFormatSpec, + @Nullable FormatSpec columnFormatSpec, @Nullable Serializer serializer ) { @@ -196,7 +199,7 @@ public BitmapSerdeFactory getBitmapSerdeFactory() @Nullable @JsonProperty - public NestedCommonFormatColumnFormatSpec getColumnFormatSpec() + public FormatSpec getColumnFormatSpec() { return columnFormatSpec; } @@ -432,9 +435,72 @@ public NestedCommonFormatColumnPartSerde build() enforceLogicalType, byteOrder, bitmapSerdeFactory, - columnFormatSpec, + FormatSpec.forSerde(columnFormatSpec), serializer ); } } + + /** + * Overrides {@link NestedCommonFormatColumnFormatSpec} so that {@link #getBitmapEncoding()} participates in serde + * so that it can store the complete object in the column metadata + */ + public static class FormatSpec extends NestedCommonFormatColumnFormatSpec + { + @Nullable + public static FormatSpec forSerde( + @Nullable NestedCommonFormatColumnFormatSpec spec + ) + { + if (spec == null) { + return null; + } + return new FormatSpec( + spec.getObjectFieldsDictionaryEncoding(), + spec.getObjectStorageEncoding(), + spec.getObjectStorageCompression(), + spec.getStringDictionaryEncoding(), + spec.getDictionaryEncodedColumnCompression(), + spec.getLongColumnEncoding(), + spec.getLongColumnCompression(), + spec.getDoubleColumnCompression(), + spec.getBitmapEncoding() + ); + } + + @JsonCreator + public FormatSpec( + @JsonProperty("objectFieldsDictionaryEncoding")@Nullable StringEncodingStrategy objectFieldsDictionaryEncoding, + @JsonProperty("objectStorageEncoding")@Nullable ObjectStorageEncoding objectStorageEncoding, + @JsonProperty("objectStorageCompression")@Nullable CompressionStrategy objectStorageCompression, + @JsonProperty("stringDictionaryEncoding")@Nullable StringEncodingStrategy stringDictionaryEncoding, + @JsonProperty("dictionaryEncodedColumnCompression")@Nullable CompressionStrategy dictionaryEncodedColumnCompression, + @JsonProperty("longColumnEncoding")@Nullable CompressionFactory.LongEncodingStrategy longColumnEncoding, + @JsonProperty("longColumnCompression")@Nullable CompressionStrategy longColumnCompression, + @JsonProperty("doubleColumnCompression")@Nullable CompressionStrategy doubleColumnCompression, + @JsonProperty("bitmapEncoding") @Nullable BitmapSerdeFactory bitmapEncoding + ) + { + super( + objectFieldsDictionaryEncoding, + objectStorageEncoding, + objectStorageCompression, + stringDictionaryEncoding, + dictionaryEncodedColumnCompression, + longColumnEncoding, + longColumnCompression, + doubleColumnCompression, + bitmapEncoding + ); + } + + @JsonProperty("bitmapEncoding") + @JsonIgnore(false) + @Nullable + @Override + public BitmapSerdeFactory getBitmapEncoding() + { + return super.getBitmapEncoding(); + } + } } diff --git a/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java b/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java index 59a5c942a355..c3ceffe200c5 100644 --- a/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java +++ b/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java @@ -25,14 +25,18 @@ import org.apache.druid.segment.DefaultColumnFormatConfig; import org.apache.druid.segment.DimensionHandlerProvider; import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.NestedCommonFormatColumnHandler; import org.apache.druid.segment.column.StringEncodingStrategy; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; import javax.annotation.Nullable; import java.util.Properties; @@ -42,7 +46,7 @@ public class BuiltInTypesModuleTest @Nullable private static DimensionHandlerProvider DEFAULT_HANDLER_PROVIDER; - @BeforeClass + @BeforeAll public static void setup() { DEFAULT_HANDLER_PROVIDER = DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.get( @@ -50,6 +54,13 @@ public static void setup() ); DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.remove(NestedDataComplexTypeSerde.TYPE_NAME); } + + @AfterEach + public void beforeEach() + { + IndexSpec.DEFAULT = IndexSpec.builder().build(); + } + @AfterClass public static void teardown() { @@ -61,6 +72,7 @@ public static void teardown() DEFAULT_HANDLER_PROVIDER ); } + IndexSpec.DEFAULT = IndexSpec.builder().build(); } @Test @@ -76,9 +88,9 @@ public void testDefaults() DimensionHandlerProvider provider = DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.get( NestedDataComplexTypeSerde.TYPE_NAME ); - Assert.assertTrue(provider.get("test") instanceof NestedCommonFormatColumnHandler); + Assertions.assertInstanceOf(NestedCommonFormatColumnHandler.class, provider.get("test")); - Assert.assertEquals( + Assertions.assertEquals( DimensionSchema.MultiValueHandling.SORTED_ARRAY, BuiltInTypesModule.getStringMultiValueHandlingMode() ); @@ -90,9 +102,14 @@ public void testOverride() DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.remove(NestedDataComplexTypeSerde.TYPE_NAME); Properties props = new Properties(); props.setProperty("druid.indexing.formats.stringMultiValueHandlingMode", "sorted_array"); - props.setProperty("druid.indexing.formats.nestedColumnFormatSpec.stringDictionaryEncoding.type", StringEncodingStrategy.FRONT_CODED); - props.setProperty("druid.indexing.formats.nestedColumnFormatSpec.stringDictionaryEncoding.bucketSize", "16"); - props.setProperty("druid.indexing.formats.nestedColumnFormatSpec.stringDictionaryEncoding.formatVersion", "1"); + props.setProperty("druid.indexing.formats.indexSpec.complexMetricCompression", CompressionStrategy.LZ4.toString()); + props.setProperty("druid.indexing.formats.indexSpec.autoColumnFormatSpec.stringDictionaryEncoding.type", StringEncodingStrategy.FRONT_CODED); + props.setProperty("druid.indexing.formats.indexSpec.autoColumnFormatSpec.stringDictionaryEncoding.bucketSize", "16"); + props.setProperty("druid.indexing.formats.indexSpec.autoColumnFormatSpec.stringDictionaryEncoding.formatVersion", "1"); + // ensure that this cannot be set + props.setProperty("druid.indexing.formats.indexSpec.autoColumnFormatSpec.bitmapEncoding", "roaring"); + props.setProperty("druid.indexing.formats.indexSpec.metricCompression", CompressionStrategy.ZSTD.toString()); + props.setProperty("druid.indexing.formats.indexSpec.bitmap", "{\"type\":\"concise\"}"); Injector gadget = makeInjector(props); // side effects @@ -102,14 +119,19 @@ public void testOverride() NestedDataComplexTypeSerde.TYPE_NAME ); - Assert.assertEquals( + Assertions.assertEquals( DimensionSchema.MultiValueHandling.SORTED_ARRAY, BuiltInTypesModule.getStringMultiValueHandlingMode() ); - Assert.assertEquals( - new NestedCommonFormatColumnFormatSpec(null, null, null, new StringEncodingStrategy.FrontCoded(16, (byte) 1), null, null, null, null, null), - BuiltInTypesModule.getDefaultNestedCommonFormatSpec() + Assertions.assertEquals(CompressionStrategy.LZ4, IndexSpec.DEFAULT.getComplexMetricCompression()); + Assertions.assertEquals( + NestedCommonFormatColumnFormatSpec.builder().setStringDictionaryEncoding(new StringEncodingStrategy.FrontCoded(16, (byte) 1)).build(), + IndexSpec.DEFAULT.getAutoColumnFormatSpec() ); + + Assertions.assertNull(IndexSpec.DEFAULT.getAutoColumnFormatSpec().getBitmapEncoding()); + Assertions.assertEquals(CompressionStrategy.ZSTD, IndexSpec.DEFAULT.getMetricCompression()); + Assertions.assertEquals(new ConciseBitmapSerdeFactory(), IndexSpec.DEFAULT.getBitmapSerdeFactory()); } @Test @@ -121,7 +143,7 @@ public void testOverrideMultiValueHandlingModeCaseInsensitive() gadget.getInstance(BuiltInTypesModule.SideEffectRegisterer.class); - Assert.assertEquals( + Assertions.assertEquals( DimensionSchema.MultiValueHandling.ARRAY, BuiltInTypesModule.getStringMultiValueHandlingMode() ); @@ -134,11 +156,11 @@ public void testInvalidMultiValueHandlingMode() props.setProperty("druid.indexing.formats.stringMultiValueHandlingMode", "boo"); final Injector gadget = makeInjector(props); - final Exception exception = Assert.assertThrows( + final Exception exception = Assertions.assertThrows( Exception.class, () -> gadget.getInstance(BuiltInTypesModule.SideEffectRegisterer.class) ); - Assert.assertTrue(exception.getMessage().contains( + Assertions.assertTrue(exception.getMessage().contains( "Invalid value[boo] specified for 'druid.indexing.formats.stringMultiValueHandlingMode'." + " Supported values are [[SORTED_ARRAY, SORTED_SET, ARRAY]]." )); diff --git a/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java index 2c256c1d2adf..1aa737cd704a 100644 --- a/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java @@ -20,6 +20,7 @@ package org.apache.druid.segment; import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.segment.data.CompressionStrategy; import org.junit.Test; import java.io.IOException; @@ -43,6 +44,11 @@ public void testEquals() { EqualsVerifier.forClass(CompleteSegment.class) .withNonnullFields("segment", "dataSegment") + .withPrefabValues( + IndexSpec.class, + IndexSpec.builder().build(), + IndexSpec.builder().withComplexMetricCompression(CompressionStrategy.ZSTD).build() + ) .usingGetClass() .verify(); } diff --git a/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java b/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java index d9e9e9a95251..1e2b69239d39 100644 --- a/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java +++ b/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java @@ -24,6 +24,7 @@ import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.data.CompressionStrategy; import org.junit.Assert; import org.junit.Test; @@ -56,6 +57,13 @@ public void testDefaultsSerdeOverride() throws JsonProcessingException @Test public void testEqualsAndHashcode() { - EqualsVerifier.forClass(DefaultColumnFormatConfig.class).usingGetClass().verify(); + EqualsVerifier.forClass(DefaultColumnFormatConfig.class) + .usingGetClass() + .withPrefabValues( + IndexSpec.class, + IndexSpec.builder().build(), + IndexSpec.builder().withComplexMetricCompression(CompressionStrategy.ZSTD).build() + ) + .verify(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java index 23c0e0302d8d..c2b508f5bc2e 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java @@ -40,10 +40,11 @@ public class NestedDataColumnSchemaTest ) .setObjectStorageCompression(CompressionStrategy.ZSTD) .build(); + private static final DefaultColumnFormatConfig DEFAULT_NESTED_SPEC_CONFIG = new DefaultColumnFormatConfig( null, null, - DEFAULT_NESTED_SPEC + IndexSpec.builder().withAutoColumnFormatSpec(DEFAULT_NESTED_SPEC).build() ); private static final ObjectMapper MAPPER; 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 fa03543c6d44..4100e883daf5 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 @@ -298,7 +298,7 @@ public void testBasicFunctionality() throws IOException false, ByteOrder.nativeOrder(), RoaringBitmapSerdeFactory.getInstance(), - columnFormatSpec + NestedCommonFormatColumnPartSerde.FormatSpec.forSerde(columnFormatSpec) ); bob.setFileMapper(fileMapper); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); @@ -323,7 +323,7 @@ public void testArrayFunctionality() throws IOException false, ByteOrder.nativeOrder(), RoaringBitmapSerdeFactory.getInstance(), - columnFormatSpec + NestedCommonFormatColumnPartSerde.FormatSpec.forSerde(columnFormatSpec) ); bob.setFileMapper(arrayFileMapper); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); From 1f82393117c169b85c98dae3db5d75b599256eab Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 22 Sep 2025 11:08:29 -0700 Subject: [PATCH 11/24] static method --- .../benchmark/FilterPartitionBenchmark.java | 2 +- .../FilteredAggregatorBenchmark.java | 2 +- .../GroupByTypeInterfaceBenchmark.java | 2 +- .../benchmark/TopNTypeInterfaceBenchmark.java | 2 +- .../indexing/IndexMergeBenchmark.java | 4 +- .../indexing/IndexPersistBenchmark.java | 2 +- .../lookup/SqlReverseLookupBenchmark.java | 2 +- .../benchmark/query/GroupByBenchmark.java | 2 +- .../benchmark/query/InPlanningBenchmark.java | 2 +- .../druid/benchmark/query/ScanBenchmark.java | 2 +- .../benchmark/query/SearchBenchmark.java | 2 +- .../benchmark/query/TimeseriesBenchmark.java | 2 +- .../druid/benchmark/query/TopNBenchmark.java | 2 +- .../timecompare/TimeCompareBenchmark.java | 2 +- .../embedded/compact/AutoCompactionTest.java | 2 +- ...RabbitStreamIndexTaskTuningConfigTest.java | 2 +- ...abbitStreamSupervisorTuningConfigTest.java | 2 +- .../kafka/KafkaIndexTaskTuningConfigTest.java | 20 ++--- .../KafkaSupervisorTuningConfigTest.java | 4 +- .../KinesisIndexTaskTuningConfigTest.java | 18 ++--- .../KinesisSupervisorTuningConfigTest.java | 2 +- .../k8s/overlord/common/K8sTestUtils.java | 2 +- .../druid/indexer/HadoopTuningConfig.java | 6 +- .../druid/indexer/HadoopTaskSerdeTest.java | 2 +- .../druid/indexer/HadoopTuningConfigTest.java | 4 +- ...tasourceRecordReaderSegmentReaderTest.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 2 +- .../SeekableStreamIndexTaskTuningConfig.java | 2 +- .../druid/indexing/common/TestIndexTask.java | 2 +- .../common/task/CompactionTaskRunTest.java | 8 +- .../task/CompactionTuningConfigTest.java | 2 +- .../indexing/common/task/IndexTaskTest.java | 4 +- .../indexing/common/task/TaskSerdeTest.java | 4 +- .../ParallelIndexSupervisorTaskTest.java | 6 +- .../ParallelIndexTuningConfigTest.java | 2 +- .../input/DruidSegmentReaderTest.java | 6 +- .../indexing/overlord/TaskLifecycleTest.java | 2 +- .../druid/msq/indexing/MSQTuningConfig.java | 2 +- .../apache/druid/msq/exec/MSQReplaceTest.java | 2 +- .../msq/exec/TaskDataSegmentProviderTest.java | 2 +- .../druid/msq/indexing/MSQSpecTest.java | 6 +- .../msq/indexing/MSQTuningConfigTest.java | 4 +- .../msq/util/MultiStageQueryContextTest.java | 4 +- .../druid/guice/BuiltInTypesModule.java | 77 ++++++++++--------- .../druid/segment/AutoTypeColumnSchema.java | 2 +- .../org/apache/druid/segment/IndexSpec.java | 6 +- .../druid/segment/NestedDataColumnSchema.java | 4 +- .../segment/incremental/IncrementalIndex.java | 2 +- .../nested/NestedDataComplexTypeSerde.java | 2 +- .../druid/guice/BuiltInTypesModuleTest.java | 14 ++-- .../apache/druid/query/DoubleStorageTest.java | 2 +- .../druid/query/MultiValuedDimensionTest.java | 4 +- .../druid/query/NestedDataTestUtils.java | 12 +-- .../aggregation/AggregationTestHelper.java | 10 +-- ...ePairLongDoubleComplexMetricSerdeTest.java | 2 +- ...lePairLongFloatComplexMetricSerdeTest.java | 2 +- ...blePairLongLongComplexMetricSerdeTest.java | 2 +- ...ePairLongStringComplexMetricSerdeTest.java | 4 +- ...ByLimitPushDownInsufficientBufferTest.java | 4 +- ...roupByLimitPushDownMultiNodeMergeTest.java | 12 +-- .../groupby/GroupByMultiSegmentTest.java | 4 +- .../groupby/NestedQueryPushDownTest.java | 4 +- .../query/scan/NestedDataScanQueryTest.java | 20 ++--- .../druid/segment/CompleteSegmentTest.java | 6 -- .../segment/CustomSegmentizerFactoryTest.java | 2 +- .../DefaultColumnFormatsConfigTest.java | 6 -- .../apache/druid/segment/EmptyIndexTest.java | 2 +- .../apache/druid/segment/IndexBuilder.java | 2 +- .../IndexIONullColumnsCompatibilityTest.java | 2 +- .../segment/IndexMergerNullHandlingTest.java | 2 +- .../druid/segment/IndexMergerRollupTest.java | 2 +- .../druid/segment/IndexMergerTestBase.java | 2 +- .../IndexMergerV9WithSpatialIndexTest.java | 2 +- .../apache/druid/segment/IndexSpecTest.java | 4 +- .../druid/segment/SchemalessIndexTest.java | 2 +- .../org/apache/druid/segment/TestIndex.java | 2 +- .../segment/UnnestCursorFactoryTest.java | 2 +- .../filter/SpatialFilterBonusTest.java | 2 +- .../segment/filter/SpatialFilterTest.java | 2 +- .../segment/generator/SegmentGenerator.java | 2 +- .../BroadcastSegmentIndexedTableTest.java | 2 +- ...estedCommonFormatColumnFormatSpecTest.java | 16 ++-- .../nested/NestedDataColumnSupplierTest.java | 2 +- .../NestedFieldColumnSelectorsTest.java | 4 +- .../ScalarDoubleColumnSupplierTest.java | 2 +- .../nested/ScalarLongColumnSupplierTest.java | 2 +- .../ScalarStringColumnSupplierTest.java | 2 +- .../nested/VariantColumnSupplierTest.java | 2 +- .../quidem/KttmNestedComponentSupplier.java | 2 +- .../server/compaction/CompactionStatus.java | 4 +- .../ClientCompactQueryTuningConfigTest.java | 2 +- .../loading/SegmentLocalCacheManagerTest.java | 2 +- .../appenderator/BatchAppenderatorTester.java | 2 +- .../StreamAppenderatorTester.java | 2 +- .../compaction/CompactionStatusTest.java | 16 ++-- .../NewestSegmentFirstPolicyTest.java | 32 ++++---- .../org/apache/druid/cli/DumpSegmentTest.java | 2 +- .../cli/validate/DruidJsonValidatorTest.java | 2 +- .../sql/calcite/util/TestDataBuilder.java | 2 +- 99 files changed, 244 insertions(+), 245 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java index dcf4edf74a01..5340a48dfeb9 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java @@ -166,7 +166,7 @@ public void setup() throws IOException indexFile = INDEX_MERGER_V9.persist( incIndex, tmpDir, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); qIndex = INDEX_IO.loadIndex(indexFile); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java index b524aaad958d..3c4c24d8d730 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java @@ -272,7 +272,7 @@ public void setup(FilteredAggregatorBenchmark global) throws IOException File indexFile = INDEX_MERGER_V9.persist( incIndex, qIndexesDir, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); incIndex.close(); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java index 6e085e162870..300841fab5d7 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -310,7 +310,7 @@ public void setup() throws IOException final File file = INDEX_MERGER_V9.persist( index, new File(tmpDir, String.valueOf(i)), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java index f4db1495aa65..5683d51a56f1 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java @@ -277,7 +277,7 @@ public void setup() throws IOException File indexFile = INDEX_MERGER_V9.persist( incIndexes.get(i), tmpFile, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java index 09b265e60e7c..e18867684482 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java @@ -134,7 +134,7 @@ public void setup() throws IOException File indexFile = indexMergerV9.persist( incIndex, tmpDir, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); @@ -159,7 +159,7 @@ public void mergeV9(Blackhole blackhole) throws Exception rollup, schemaInfo.getAggsArray(), tmpFile, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null, -1 ); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java index 9dbc0e31a14e..0f22ae266bb8 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java @@ -185,7 +185,7 @@ public void persistV9(Blackhole blackhole) throws Exception File indexFile = INDEX_MERGER_V9.persist( incIndex, tmpDir, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/SqlReverseLookupBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/SqlReverseLookupBenchmark.java index bd97113ba80a..c527b7b32a58 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/SqlReverseLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/SqlReverseLookupBenchmark.java @@ -105,7 +105,7 @@ public void setup() final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator()); final QueryableIndex index = - segmentGenerator.generate(dataSegment, schemaInfo, IndexSpec.DEFAULT, Granularities.NONE, 1); + segmentGenerator.generate(dataSegment, schemaInfo, IndexSpec.getDefault(), Granularities.NONE, 1); final Pair sqlSystem = SqlBaseBenchmark.createSqlSystem( ImmutableMap.of(dataSegment, index), diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index 44ef5c60e2ed..2056ffbea0d3 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -578,7 +578,7 @@ public void setup(GroupByBenchmark global) throws IOException File indexFile = INDEX_MERGER_V9.persist( incIndex, new File(qIndexesDir, String.valueOf(i)), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); incIndex.close(); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java index 89785ee87572..565796375845 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java @@ -169,7 +169,7 @@ public void setup() throws JsonProcessingException schemaInfo, DimensionsSpec.builder().setDimensions(columnSchemas).build(), TransformSpec.NONE, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), Granularities.NONE, rowsPerSegment ); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java index a4e8162861aa..e9ee9fa58997 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -330,7 +330,7 @@ public void setup(ScanBenchmark global) throws IOException File indexFile = INDEX_MERGER_V9.persist( incIndex, new File(qIndexesDir, String.valueOf(i)), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); incIndex.close(); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java index 7c3a62da0410..ad92288c1e12 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java @@ -397,7 +397,7 @@ public void setup(SearchBenchmark global) throws IOException File indexFile = INDEX_MERGER_V9.persist( incIndex, new File(qIndexesDir, String.valueOf(i)), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); incIndex.close(); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java index e1657127e5b6..0ddb941ea0b7 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java @@ -325,7 +325,7 @@ public void setup(TimeseriesBenchmark global) throws IOException File indexFile = INDEX_MERGER_V9.persist( incIndex, new File(qIndexesDir, String.valueOf(i)), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); incIndex.close(); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java index 4d3c9309dbee..438143e311ca 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java @@ -301,7 +301,7 @@ public void setup(TopNBenchmark global) throws IOException File indexFile = INDEX_MERGER_V9.persist( incIndex, new File(qIndexesDir, String.valueOf(i)), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); incIndex.close(); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java index 99d4a02a712f..f61b25fd06f8 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java @@ -329,7 +329,7 @@ public void setup() throws IOException File indexFile = INDEX_MERGER_V9.persist( incIndexes.get(i), tmpDir, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java index c18e73df5050..f207d4d356be 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java @@ -504,7 +504,7 @@ public void testAutoCompactionPreservesCreateBitmapIndexInDimensionSchema(Compac List dimensionSchemas = ImmutableList.of( new StringDimensionSchema("language", DimensionSchema.MultiValueHandling.SORTED_ARRAY, false), - new AutoTypeColumnSchema("deleted", ColumnType.DOUBLE, null).getEffectiveSchema(IndexSpec.DEFAULT) + new AutoTypeColumnSchema("deleted", ColumnType.DOUBLE, null).getEffectiveSchema(IndexSpec.getDefault()) ); submitCompactionConfig( diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java index d507199495a3..e7237c706747 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java @@ -65,7 +65,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment().intValue()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); - // Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpec()); + // Assert.assertEquals(IndexSpec.getDefault(), config.getIndexSpec()); Assert.assertEquals(false, config.isReportParseExceptions()); Assert.assertEquals(Duration.ofMinutes(15).toMillis(), config.getHandoffConditionTimeout()); diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfigTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfigTest.java index 63900207829e..697306b175f3 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfigTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfigTest.java @@ -63,7 +63,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment().intValue()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); - // Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpec()); + // Assert.assertEquals(IndexSpec.getDefault(), config.getIndexSpec()); Assert.assertEquals(false, config.isReportParseExceptions()); Assert.assertEquals(java.time.Duration.ofMinutes(15).toMillis(), config.getHandoffConditionTimeout()); Assert.assertNull(config.getWorkerThreads()); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 54ae10080a7b..0083e76ea75c 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -69,8 +69,8 @@ public void testSerdeWithDefaults() throws Exception Assert.assertNull(config.getMaxTotalRows()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); - Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpec()); - Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpecForIntermediatePersists()); + Assert.assertEquals(IndexSpec.getDefault(), config.getIndexSpec()); + Assert.assertEquals(IndexSpec.getDefault(), config.getIndexSpecForIntermediatePersists()); Assert.assertFalse(config.isReportParseExceptions()); Assert.assertEquals(Duration.ofMinutes(15).toMillis(), config.getHandoffConditionTimeout()); Assert.assertEquals(1, config.getNumPersistThreads()); @@ -139,8 +139,8 @@ public void testConvert() .withMaxRowsPerSegment(2) .withMaxTotalRows(10L) .withMaxPendingPersists(4) - .withIndexSpec(IndexSpec.DEFAULT) - .withIndexSpecForIntermediatePersists(IndexSpec.DEFAULT) + .withIndexSpec(IndexSpec.getDefault()) + .withIndexSpecForIntermediatePersists(IndexSpec.getDefault()) .withReportParseExceptions(true) .withMaxColumnsToMerge(5) .build(); @@ -154,7 +154,7 @@ public void testConvert() Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod()); Assert.assertNull(copy.getBasePersistDirectory()); Assert.assertEquals(4, copy.getMaxPendingPersists()); - Assert.assertEquals(IndexSpec.DEFAULT, copy.getIndexSpec()); + Assert.assertEquals(IndexSpec.getDefault(), copy.getIndexSpec()); Assert.assertTrue(copy.isReportParseExceptions()); Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); Assert.assertEquals(2, copy.getNumPersistThreads()); @@ -174,8 +174,8 @@ public void testSerdeWithModifiedTuningConfigAddedField() throws IOException new Period("PT3S"), new File("/tmp/xxx"), 4, - IndexSpec.DEFAULT, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), + IndexSpec.getDefault(), true, 5L, null, @@ -227,8 +227,8 @@ public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException 10L, new Period("PT3S"), 4, - IndexSpec.DEFAULT, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), + IndexSpec.getDefault(), true, 5L, null, @@ -273,7 +273,7 @@ public void testEqualsAndHashCode() EqualsVerifier.forClass(KafkaIndexTaskTuningConfig.class) .withPrefabValues( IndexSpec.class, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build() ) .usingGetClass() diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java index 376ca256303c..7c13dcbfb4a3 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java @@ -62,8 +62,8 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment().intValue()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); - Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpec()); - Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpecForIntermediatePersists()); + Assert.assertEquals(IndexSpec.getDefault(), config.getIndexSpec()); + Assert.assertEquals(IndexSpec.getDefault(), config.getIndexSpecForIntermediatePersists()); Assert.assertFalse(config.isReportParseExceptions()); Assert.assertEquals(java.time.Duration.ofMinutes(15).toMillis(), config.getHandoffConditionTimeout()); Assert.assertNull(config.getWorkerThreads()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 375e26e2ed28..5afbb4e78233 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -74,7 +74,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment().intValue()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); - Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpec()); + Assert.assertEquals(IndexSpec.getDefault(), config.getIndexSpec()); Assert.assertFalse(config.isReportParseExceptions()); Assert.assertEquals(Duration.ofMinutes(15).toMillis(), config.getHandoffConditionTimeout()); Assert.assertNull(config.getRecordBufferSizeBytesConfigured()); @@ -151,8 +151,8 @@ public void testSerdeWithModifiedTuningConfigAddedField() throws IOException new Period("PT3S"), new File("/tmp/xxx"), 4, - IndexSpec.DEFAULT, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), + IndexSpec.getDefault(), true, 5L, true, @@ -216,8 +216,8 @@ public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException new Period("PT3S"), new File("/tmp/xxx"), 4, - IndexSpec.DEFAULT, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), + IndexSpec.getDefault(), true, 5L, true, @@ -304,8 +304,8 @@ public void testConvert() 100L, new Period("PT3S"), 4, - IndexSpec.DEFAULT, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), + IndexSpec.getDefault(), true, 5L, true, @@ -341,7 +341,7 @@ public void testConvert() Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod()); Assert.assertNull(copy.getBasePersistDirectory()); Assert.assertEquals(4, copy.getMaxPendingPersists()); - Assert.assertEquals(IndexSpec.DEFAULT, copy.getIndexSpec()); + Assert.assertEquals(IndexSpec.getDefault(), copy.getIndexSpec()); Assert.assertTrue(copy.isReportParseExceptions()); Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); Assert.assertEquals(1000, (int) copy.getRecordBufferSizeBytesConfigured()); @@ -361,7 +361,7 @@ public void testEqualsAndHashCode() EqualsVerifier.forClass(KinesisIndexTaskTuningConfig.class) .withPrefabValues( IndexSpec.class, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build() ) .usingGetClass() diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java index 7fd1bbb9c582..350f32003b2c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java @@ -61,7 +61,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment().intValue()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); - Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpec()); + Assert.assertEquals(IndexSpec.getDefault(), config.getIndexSpec()); Assert.assertEquals(false, config.isReportParseExceptions()); Assert.assertEquals(java.time.Duration.ofMinutes(15).toMillis(), config.getHandoffConditionTimeout()); Assert.assertNull(config.getWorkerThreads()); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java index a86099047eb4..8a76341a237e 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java @@ -46,7 +46,7 @@ public class K8sTestUtils { - private static final IndexSpec INDEX_SPEC = IndexSpec.DEFAULT; + private static final IndexSpec INDEX_SPEC = IndexSpec.getDefault(); public static PodSpec getDummyPodSpec() { diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java index 1aeeb8c117f4..d57134e0b600 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java @@ -56,8 +56,8 @@ public static HadoopTuningConfig makeDefaultTuningConfig() DateTimes.nowUtc().toString(), DEFAULT_PARTITIONS_SPEC, DEFAULT_SHARD_SPECS, - IndexSpec.DEFAULT, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), + IndexSpec.getDefault(), DEFAULT_APPENDABLE_INDEX, DEFAULT_MAX_ROWS_IN_MEMORY_BATCH, 0L, @@ -150,7 +150,7 @@ public HadoopTuningConfig( this.version = Configs.valueOrDefault(version, DateTimes.nowUtc().toString()); this.partitionsSpec = Configs.valueOrDefault(partitionsSpec, DEFAULT_PARTITIONS_SPEC); this.shardSpecs = Configs.valueOrDefault(shardSpecs, DEFAULT_SHARD_SPECS); - this.indexSpec = Configs.valueOrDefault(indexSpec, IndexSpec.DEFAULT); + this.indexSpec = Configs.valueOrDefault(indexSpec, IndexSpec.getDefault()); this.indexSpecForIntermediatePersists = Configs.valueOrDefault( indexSpecForIntermediatePersists, this.indexSpec diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTaskSerdeTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTaskSerdeTest.java index 8519f05e7e7f..87176b3a3cfe 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTaskSerdeTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTaskSerdeTest.java @@ -55,7 +55,7 @@ public class HadoopTaskSerdeTest { private final ObjectMapper jsonMapper; - private final IndexSpec indexSpec = IndexSpec.DEFAULT; + private final IndexSpec indexSpec = IndexSpec.getDefault(); @Rule public ExpectedException thrown = ExpectedException.none(); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java index 97c764dd10b1..3fe155fc96fd 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java @@ -74,8 +74,8 @@ public void testSerde() throws Exception Assert.assertEquals(new OnheapIncrementalIndex.Spec(), actual.getAppendableIndexSpec()); Assert.assertNotNull(actual.getPartitionsSpec()); Assert.assertEquals(ImmutableMap.>of(), actual.getShardSpecs()); - Assert.assertEquals(IndexSpec.DEFAULT, actual.getIndexSpec()); - Assert.assertEquals(IndexSpec.DEFAULT, actual.getIndexSpecForIntermediatePersists()); + Assert.assertEquals(IndexSpec.getDefault(), actual.getIndexSpec()); + Assert.assertEquals(IndexSpec.getDefault(), actual.getIndexSpecForIntermediatePersists()); Assert.assertEquals(100, actual.getMaxRowsInMemory()); Assert.assertEquals(true, actual.isLeaveIntermediate()); Assert.assertEquals(true, actual.isCleanupOnFailure()); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java index 047f9884498a..0d7e2eea9039 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java @@ -236,7 +236,7 @@ private void createTestIndex(File segmentDir) throws Exception for (String line : rows) { index.add(parser.parse(line)); } - indexMerger.persist(index, segmentDir, IndexSpec.DEFAULT, null); + indexMerger.persist(index, segmentDir, IndexSpec.getDefault(), null); } } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 51dd1aa0528e..6ac259aa4c6d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -1363,7 +1363,7 @@ private IndexTuningConfig( ? IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE : maxColumnsToMerge; this.partitionsSpec = partitionsSpec; - this.indexSpec = indexSpec == null ? IndexSpec.DEFAULT : indexSpec; + this.indexSpec = indexSpec == null ? IndexSpec.getDefault() : indexSpec; this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists == null ? this.indexSpec : indexSpecForIntermediatePersists; this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index fd63ba1132e6..6dda59421198 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -109,7 +109,7 @@ public SeekableStreamIndexTaskTuningConfig( intermediatePersistPeriod == null ? DEFAULT_INTERMEDIATE_PERSIST_PERIOD : intermediatePersistPeriod; this.basePersistDirectory = basePersistDirectory; this.maxPendingPersists = maxPendingPersists == null ? 0 : maxPendingPersists; - this.indexSpec = indexSpec == null ? IndexSpec.DEFAULT : indexSpec; + this.indexSpec = indexSpec == null ? IndexSpec.getDefault() : indexSpec; this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists == null ? this.indexSpec : indexSpecForIntermediatePersists; this.reportParseExceptions = reportParseExceptions == null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java index ff0aa674ef3f..69537c8627cb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java @@ -70,7 +70,7 @@ public TestIndexTask( ), TuningConfigBuilder.forIndexTask() .withMaxRowsInMemory(10) - .withIndexSpec(IndexSpec.DEFAULT) + .withIndexSpec(IndexSpec.getDefault()) .withPartitionsSpec(new DynamicPartitionsSpec(10000, null)) .withForceGuaranteedRollup(false) .withMaxPendingPersists(3) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index f3aab693ae76..4eb9af6cd3fc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -158,7 +158,7 @@ public class CompactionTaskRunTest extends IngestionTestBase ); private static final NestedCommonFormatColumnFormatSpec DEFAULT_NESTED_SPEC = - NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.DEFAULT); + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault()); private static final List TEST_ROWS = ImmutableList.of( "2014-01-01T00:00:10Z,a,1\n", @@ -258,7 +258,7 @@ public static CompactionState getDefaultCompactionState( expectedDims, ImmutableList.of(expectedMetric), null, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), new UniformGranularitySpec( segmentGranularity, queryGranularity, @@ -774,7 +774,7 @@ public void testCompactionWithFilterInTransformSpec() throws Exception ), ImmutableList.of(expectedLongSumMetric), compactionTask.getTransformSpec(), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), new UniformGranularitySpec( Granularities.DAY, Granularities.MINUTE, @@ -829,7 +829,7 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception ), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), compactionTask.getTransformSpec(), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), new UniformGranularitySpec( Granularities.DAY, Granularities.MINUTE, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java index a342dab82300..d4805e7bc09c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java @@ -93,7 +93,7 @@ public void testEqualsAndHashCode() EqualsVerifier.forClass(CompactionTask.CompactionTuningConfig.class) .withPrefabValues( IndexSpec.class, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build() ) .usingGetClass() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index a27ba1327b1a..2d9991102882 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -196,7 +196,7 @@ public static Iterable constructorFeeder() ); } - private static final IndexSpec INDEX_SPEC = IndexSpec.DEFAULT; + private static final IndexSpec INDEX_SPEC = IndexSpec.getDefault(); private final ObjectMapper jsonMapper; private final IndexIO indexIO; private final LockGranularity lockGranularity; @@ -2725,7 +2725,7 @@ public void testEqualsAndHashCode() EqualsVerifier.forClass(IndexTuningConfig.class) .withPrefabValues( IndexSpec.class, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build() ) .usingGetClass() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index 5e4b8197a35c..5064d915c5a9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -52,7 +52,7 @@ public class TaskSerdeTest { private final ObjectMapper jsonMapper; - private final IndexSpec indexSpec = IndexSpec.DEFAULT; + private final IndexSpec indexSpec = IndexSpec.getDefault(); @Rule public ExpectedException thrown = ExpectedException.none(); @@ -88,7 +88,7 @@ public void testIndexTaskTuningConfigDefaults() throws Exception ); Assert.assertFalse(tuningConfig.isReportParseExceptions()); - Assert.assertEquals(IndexSpec.DEFAULT, tuningConfig.getIndexSpec()); + Assert.assertEquals(IndexSpec.getDefault(), tuningConfig.getIndexSpec()); Assert.assertEquals(new Period(Integer.MAX_VALUE), tuningConfig.getIntermediatePersistPeriod()); Assert.assertEquals(0, tuningConfig.getMaxPendingPersists()); Assert.assertEquals(1000000, tuningConfig.getMaxRowsInMemory()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index b908d274e6c3..6d2b7eded2a5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -249,7 +249,7 @@ public void testFailToConstructWhenBothAppendToExistingAndForceGuaranteedRollupA .withLongEncoding(LongEncodingStrategy.LONGS) .build() ) - .withIndexSpecForIntermediatePersists(IndexSpec.DEFAULT) + .withIndexSpecForIntermediatePersists(IndexSpec.getDefault()) .withMaxPendingPersists(1) .withForceGuaranteedRollup(forceGuaranteedRollup) .withReportParseExceptions(true) @@ -305,7 +305,7 @@ public void testFailToConstructWhenBothInputSourceAndParserAreSet() .withLongEncoding(LongEncodingStrategy.LONGS) .build() ) - .withIndexSpecForIntermediatePersists(IndexSpec.DEFAULT) + .withIndexSpecForIntermediatePersists(IndexSpec.getDefault()) .withMaxPendingPersists(1) .withForceGuaranteedRollup(true) .withReportParseExceptions(true) @@ -540,7 +540,7 @@ public void testCompactionTaskDoesntCleanup() throws Exception .withLongEncoding(LongEncodingStrategy.LONGS) .build() ) - .withIndexSpecForIntermediatePersists(IndexSpec.DEFAULT) + .withIndexSpecForIntermediatePersists(IndexSpec.getDefault()) .withMaxPendingPersists(1) .withForceGuaranteedRollup(forceGuaranteedRollup) .withReportParseExceptions(true) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java index 33d6a7aaa76f..b7c79e9a53a2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java @@ -155,7 +155,7 @@ public void testEqualsAndHashCode() .usingGetClass() .withPrefabValues( IndexSpec.class, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build() ) .verify(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java index e8fe6825566b..54a6c2d84343 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java @@ -743,7 +743,7 @@ public void testArrayColumns() throws IOException ).persist( incrementalIndex, segmentDirectory, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); segmentSize = FileUtils.getFileSize(segmentDirectory); @@ -855,7 +855,7 @@ public void testArrayColumnsCast() throws IOException ).persist( incrementalIndex, segmentDirectory, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); segmentSize = FileUtils.getFileSize(segmentDirectory); @@ -1030,7 +1030,7 @@ private void persistSegment(List rows) throws IOException ).persist( incrementalIndex, segmentDirectory, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); segmentSize = FileUtils.getFileSize(segmentDirectory); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index ff6b5329c692..fd4673669e0f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -376,7 +376,7 @@ public void setUp() throws Exception // initialize variables announcedSinks = 0; pushedSegments = 0; - indexSpec = IndexSpec.DEFAULT; + indexSpec = IndexSpec.getDefault(); emitter = newMockEmitter(); EmittingLogger.registerEmitter(emitter); mapper = TEST_UTILS.getTestObjectMapper(); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTuningConfig.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTuningConfig.java index b6958b2df5ed..582d4fbcc9fd 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTuningConfig.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTuningConfig.java @@ -140,7 +140,7 @@ public Integer getMaxNumSegments() public IndexSpec getIndexSpec() { - return indexSpec != null ? indexSpec : IndexSpec.DEFAULT; + return indexSpec != null ? indexSpec : IndexSpec.getDefault(); } @Override diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index df99956a3968..0c7cfbea2ba3 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -2815,7 +2815,7 @@ private CompactionState expectedCompactionState( ); } - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); GranularitySpec granularitySpec = new UniformGranularitySpec( segmentGranularity.getDefaultGranularity(), GranularityType.NONE.getDefaultGranularity(), diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java index ca881efe99e7..f22e7cb4fe07 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java @@ -98,7 +98,7 @@ class TaskDataSegmentProviderTest extends InitializedNullHandlingTest public static void setupStatic(@TempDir Path tempDir) throws IOException { File segDir = tempDir.resolve("segment").toFile(); - File segmentFile = TestIndex.persist(TestIndex.getIncrementalTestIndex(), IndexSpec.DEFAULT, segDir); + File segmentFile = TestIndex.persist(TestIndex.getIncrementalTestIndex(), IndexSpec.getDefault(), segDir); File zipPath = tempDir.resolve("zip").toFile(); FileUtils.mkdirp(zipPath); SEGMENT_ZIP_FILE = new File(zipPath, "index.zip"); diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQSpecTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQSpecTest.java index fab8e9964c6a..e000e64834fb 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQSpecTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQSpecTest.java @@ -34,7 +34,7 @@ public void testLegacyEquals() .withNonnullFields("destination", "tuningConfig") .withPrefabValues( IndexSpec.class, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build() ) .usingGetClass() @@ -48,7 +48,7 @@ public void testQueryDefEquals() .withNonnullFields("destination", "tuningConfig") .withPrefabValues( IndexSpec.class, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build() ) .usingGetClass() @@ -62,7 +62,7 @@ public void testEquals() .withNonnullFields("destination", "tuningConfig") .withPrefabValues( IndexSpec.class, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build() ) .usingGetClass() diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTuningConfigTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTuningConfigTest.java index 773b70b2b0bf..9dfc7e59b78f 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTuningConfigTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTuningConfigTest.java @@ -71,7 +71,7 @@ public void testEquals() EqualsVerifier.forClass(MSQTuningConfig.class) .withPrefabValues( IndexSpec.class, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build() ) .usingGetClass() @@ -86,7 +86,7 @@ public void testDefaultValuesForElements() Assert.assertEquals(100000, msqTuningConfig.getMaxRowsInMemory()); Assert.assertEquals(PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT, msqTuningConfig.getRowsPerSegment()); Assert.assertEquals(null, msqTuningConfig.getMaxNumSegments()); - Assert.assertEquals(IndexSpec.DEFAULT, msqTuningConfig.getIndexSpec()); + Assert.assertEquals(IndexSpec.getDefault(), msqTuningConfig.getIndexSpec()); } @Test diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java index 95bfe2c7fff1..5838c03dd819 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java @@ -285,8 +285,8 @@ public void testGetIndexSpec() { Assert.assertNull(decodeIndexSpec(null)); - Assert.assertEquals(IndexSpec.DEFAULT, decodeIndexSpec("{}")); - Assert.assertEquals(IndexSpec.DEFAULT, decodeIndexSpec(Collections.emptyMap())); + Assert.assertEquals(IndexSpec.getDefault(), decodeIndexSpec("{}")); + Assert.assertEquals(IndexSpec.getDefault(), decodeIndexSpec(Collections.emptyMap())); Assert.assertEquals( IndexSpec.builder() diff --git a/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java b/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java index f08bd693739e..71433b5cce48 100644 --- a/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java +++ b/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java @@ -52,6 +52,21 @@ public class BuiltInTypesModule implements DruidModule * {@link #initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig)}. */ private static DimensionSchema.MultiValueHandling STRING_MV_MODE = DimensionSchema.MultiValueHandling.SORTED_ARRAY; + private static IndexSpec DEFAULT_INDEX_SPEC = IndexSpec.builder().build(); + + /** + * @return the configured string multi value handling mode from the system config if set; otherwise, returns + * the default. + */ + public static DimensionSchema.MultiValueHandling getStringMultiValueHandlingMode() + { + return STRING_MV_MODE; + } + + public static IndexSpec getDefaultIndexSpec() + { + return DEFAULT_INDEX_SPEC; + } @Override public List getJacksonModules() @@ -74,45 +89,12 @@ public void configure(Binder binder) public SideEffectRegisterer initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig formatsConfig) { setStringMultiValueHandlingModeIfConfigured(formatsConfig.getStringMultiValueHandlingMode()); - setIndexSpecDefaults(formatsConfig); + setIndexSpecDefaults(formatsConfig.getIndexSpec()); setNestedColumnDefaults(formatsConfig); return new SideEffectRegisterer(); } - private static void setStringMultiValueHandlingModeIfConfigured(@Nullable String stringMultiValueHandlingMode) - { - if (stringMultiValueHandlingMode != null) { - STRING_MV_MODE = DimensionSchema.MultiValueHandling.fromString(stringMultiValueHandlingMode); - } - } - - private static void setIndexSpecDefaults(DefaultColumnFormatConfig defaultColumnFormatConfig) - { - if (defaultColumnFormatConfig.getIndexSpec() != null) { - IndexSpec.DEFAULT = defaultColumnFormatConfig.getIndexSpec(); - } - } - - private static void setNestedColumnDefaults(DefaultColumnFormatConfig formatsConfig) - { - if (formatsConfig.getNestedColumnFormatVersion() == null || formatsConfig.getNestedColumnFormatVersion() == 5) { - DimensionHandlerUtils.registerDimensionHandlerProvider( - NestedDataComplexTypeSerde.TYPE_NAME, - new NestedCommonFormatHandlerProvider() - ); - } - } - - /** - * @return the configured string multi value handling mode from the system config if set; otherwise, returns - * the default. - */ - public static DimensionSchema.MultiValueHandling getStringMultiValueHandlingMode() - { - return STRING_MV_MODE; - } - public static List getJacksonModulesList() { return Collections.singletonList( @@ -146,6 +128,31 @@ private static void registerSerde() } } + private static void setStringMultiValueHandlingModeIfConfigured(@Nullable String stringMultiValueHandlingMode) + { + if (stringMultiValueHandlingMode != null) { + STRING_MV_MODE = DimensionSchema.MultiValueHandling.fromString(stringMultiValueHandlingMode); + } + } + + @VisibleForTesting + static void setIndexSpecDefaults(@Nullable IndexSpec indexSpec) + { + if (indexSpec != null) { + DEFAULT_INDEX_SPEC = indexSpec; + } + } + + private static void setNestedColumnDefaults(DefaultColumnFormatConfig formatsConfig) + { + if (formatsConfig.getNestedColumnFormatVersion() == null || formatsConfig.getNestedColumnFormatVersion() == 5) { + DimensionHandlerUtils.registerDimensionHandlerProvider( + NestedDataComplexTypeSerde.TYPE_NAME, + new NestedCommonFormatHandlerProvider() + ); + } + } + public static class NestedCommonFormatHandlerProvider implements DimensionHandlerProvider { @@ -153,7 +160,7 @@ public static class NestedCommonFormatHandlerProvider @Override public DimensionHandler get(String dimensionName) { - return new NestedCommonFormatColumnHandler(dimensionName, null, IndexSpec.DEFAULT.getAutoColumnFormatSpec()); + return new NestedCommonFormatColumnHandler(dimensionName, null, IndexSpec.getDefault().getAutoColumnFormatSpec()); } } diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java index 7769cdfd1c7e..2d63386dbb00 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java @@ -95,7 +95,7 @@ public AutoTypeColumnSchema( this.castToType = castToType; } this.columnFormatSpec = columnFormatSpec == null - ? IndexSpec.DEFAULT.getAutoColumnFormatSpec() + ? IndexSpec.getDefault().getAutoColumnFormatSpec() : columnFormatSpec; } diff --git a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java index c4903b31a894..9d9ef761a85e 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.segment.column.StringEncodingStrategy; import org.apache.druid.segment.data.BitmapSerde; import org.apache.druid.segment.data.BitmapSerdeFactory; @@ -41,7 +42,10 @@ */ public class IndexSpec { - public static IndexSpec DEFAULT = IndexSpec.builder().build(); + public static IndexSpec getDefault() + { + return BuiltInTypesModule.getDefaultIndexSpec(); + } public static Builder builder() { diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java index 64ec1a2b768f..4d7a2790a80d 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnSchema.java @@ -63,7 +63,7 @@ public NestedDataColumnSchema( if (defaultFormatConfig.getIndexSpec() != null) { this.columnFormatSpec = defaultFormatConfig.getIndexSpec().getAutoColumnFormatSpec(); } else { - this.columnFormatSpec = IndexSpec.DEFAULT.getAutoColumnFormatSpec(); + this.columnFormatSpec = IndexSpec.getDefault().getAutoColumnFormatSpec(); } } else { this.columnFormatSpec = columnFormatSpec; @@ -78,7 +78,7 @@ public NestedDataColumnSchema( super(name, null, true); this.formatVersion = version; DefaultColumnFormatConfig.validateNestedFormatVersion(this.formatVersion); - this.columnFormatSpec = IndexSpec.DEFAULT.getAutoColumnFormatSpec(); + this.columnFormatSpec = IndexSpec.getDefault().getAutoColumnFormatSpec(); } @JsonProperty("formatVersion") 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 0da4898df2a5..81118250234b 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 @@ -521,7 +521,7 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) wasNewDim = true; final DimensionHandler handler; if (useSchemaDiscovery) { - handler = new NestedCommonFormatColumnHandler(dimension, null, IndexSpec.DEFAULT.getAutoColumnFormatSpec()); + handler = new NestedCommonFormatColumnHandler(dimension, null, IndexSpec.getDefault().getAutoColumnFormatSpec()); } else { // legacy behavior: for schemaless type discovery, everything is a String handler = DimensionHandlerUtils.getHandlerFromCapabilities( 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 1a0e3a6120b4..e3abfff80140 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 @@ -235,7 +235,7 @@ public ColumnType getLogicalType() @Override public DimensionHandler getColumnHandler(String columnName) { - return new NestedCommonFormatColumnHandler(columnName, null, IndexSpec.DEFAULT.getAutoColumnFormatSpec()); + return new NestedCommonFormatColumnHandler(columnName, null, IndexSpec.getDefault().getAutoColumnFormatSpec()); } @Override diff --git a/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java b/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java index c3ceffe200c5..d81960044aa6 100644 --- a/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java +++ b/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java @@ -58,7 +58,7 @@ public static void setup() @AfterEach public void beforeEach() { - IndexSpec.DEFAULT = IndexSpec.builder().build(); + BuiltInTypesModule.setIndexSpecDefaults(IndexSpec.builder().build()); } @AfterClass @@ -72,7 +72,7 @@ public static void teardown() DEFAULT_HANDLER_PROVIDER ); } - IndexSpec.DEFAULT = IndexSpec.builder().build(); + BuiltInTypesModule.setIndexSpecDefaults(IndexSpec.builder().build()); } @Test @@ -123,15 +123,15 @@ public void testOverride() DimensionSchema.MultiValueHandling.SORTED_ARRAY, BuiltInTypesModule.getStringMultiValueHandlingMode() ); - Assertions.assertEquals(CompressionStrategy.LZ4, IndexSpec.DEFAULT.getComplexMetricCompression()); + Assertions.assertEquals(CompressionStrategy.LZ4, IndexSpec.getDefault().getComplexMetricCompression()); Assertions.assertEquals( NestedCommonFormatColumnFormatSpec.builder().setStringDictionaryEncoding(new StringEncodingStrategy.FrontCoded(16, (byte) 1)).build(), - IndexSpec.DEFAULT.getAutoColumnFormatSpec() + IndexSpec.getDefault().getAutoColumnFormatSpec() ); - Assertions.assertNull(IndexSpec.DEFAULT.getAutoColumnFormatSpec().getBitmapEncoding()); - Assertions.assertEquals(CompressionStrategy.ZSTD, IndexSpec.DEFAULT.getMetricCompression()); - Assertions.assertEquals(new ConciseBitmapSerdeFactory(), IndexSpec.DEFAULT.getBitmapSerdeFactory()); + Assertions.assertNull(IndexSpec.getDefault().getAutoColumnFormatSpec().getBitmapEncoding()); + Assertions.assertEquals(CompressionStrategy.ZSTD, IndexSpec.getDefault().getMetricCompression()); + Assertions.assertEquals(new ConciseBitmapSerdeFactory(), IndexSpec.getDefault().getBitmapSerdeFactory()); } @Test diff --git a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java index ba4d3af6b1a4..f80e75e8eb1c 100644 --- a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java +++ b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java @@ -353,7 +353,7 @@ private static QueryableIndex buildIndex(String storeDoubleAsFloat) throws IOExc File someTmpFile = File.createTempFile("billy", "yay"); someTmpFile.delete(); FileUtils.mkdirp(someTmpFile); - INDEX_MERGER_V9.persist(index, someTmpFile, IndexSpec.DEFAULT, null); + INDEX_MERGER_V9.persist(index, someTmpFile, IndexSpec.getDefault(), null); someTmpFile.delete(); return INDEX_IO.loadIndex(someTmpFile); } diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java index 37de8dca8dae..99d167c00ec5 100644 --- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java @@ -170,7 +170,7 @@ public void setup() throws Exception persistedSegmentDir = FileUtils.createTempDir(); TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory) - .persist(incrementalIndex, persistedSegmentDir, IndexSpec.DEFAULT, null); + .persist(incrementalIndex, persistedSegmentDir, IndexSpec.getDefault(), null); queryableIndex = TestHelper.getTestIndexIO().loadIndex(persistedSegmentDir); @@ -203,7 +203,7 @@ public void setup() throws Exception } persistedSegmentDirNullSampler = FileUtils.createTempDir(); TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory) - .persist(incrementalIndexNullSampler, persistedSegmentDirNullSampler, IndexSpec.DEFAULT, null); + .persist(incrementalIndexNullSampler, persistedSegmentDirNullSampler, IndexSpec.getDefault(), null); queryableIndexNullSampler = TestHelper.getTestIndexIO().loadIndex(persistedSegmentDirNullSampler); } 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 1c6411c80368..d41a51f2ea8d 100644 --- a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java +++ b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java @@ -210,7 +210,7 @@ public static List createSimpleNestedTestDataTsvSegments( COUNT, granularity, rollup, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); } @@ -236,7 +236,7 @@ public static List createSimpleNestedTestDataSegments( SIMPLE_DATA_FILE, Granularities.NONE, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); } @@ -322,7 +322,7 @@ public static List createSegmentsWithConcatenatedJsonInput( COUNT, granularity, rollup, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); } @@ -505,7 +505,7 @@ public List apply(TemporaryFolder tempFolder, Closer closer) tempFolder, closer, jsonInputFile, - IndexSpec.DEFAULT + IndexSpec.getDefault() ) ) .add(NestedDataTestUtils.createIncrementalIndexForJsonInput(tempFolder, jsonInputFile)) @@ -556,7 +556,7 @@ public List apply(TemporaryFolder tempFolder, Closer closer) tempFolder, closer, jsonInputFile, - IndexSpec.DEFAULT + IndexSpec.getDefault() ) ) .addAll( @@ -564,7 +564,7 @@ public List apply(TemporaryFolder tempFolder, Closer closer) tempFolder, closer, jsonInputFile, - IndexSpec.DEFAULT + IndexSpec.getDefault() ) ) .build(); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java index a8b122f4374c..be9a2c8b6ae3 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java @@ -524,7 +524,7 @@ public void createIndex( if (!index.canAppendRow()) { File tmp = tempFolder.newFolder(); toMerge.add(tmp); - indexMerger.persist(index, tmp, IndexSpec.DEFAULT, null); + indexMerger.persist(index, tmp, IndexSpec.getDefault(), null); index.close(); index = new OnheapIncrementalIndex.Builder() .setIndexSchema( @@ -552,19 +552,19 @@ public void createIndex( if (toMerge.size() > 0) { File tmp = tempFolder.newFolder(); toMerge.add(tmp); - indexMerger.persist(index, tmp, IndexSpec.DEFAULT, null); + indexMerger.persist(index, tmp, IndexSpec.getDefault(), null); List indexes = new ArrayList<>(toMerge.size()); for (File file : toMerge) { indexes.add(indexIO.loadIndex(file)); } - indexMerger.mergeQueryableIndex(indexes, rollup, metrics, outDir, IndexSpec.DEFAULT, null, -1); + indexMerger.mergeQueryableIndex(indexes, rollup, metrics, outDir, IndexSpec.getDefault(), null, -1); for (QueryableIndex qi : indexes) { qi.close(); } } else { - indexMerger.persist(index, outDir, IndexSpec.DEFAULT, null); + indexMerger.persist(index, outDir, IndexSpec.getDefault(), null); } } finally { @@ -655,7 +655,7 @@ public Segment persistIncrementalIndex( if (outDir == null) { outDir = tempFolder.newFolder(); } - indexMerger.persist(index, outDir, IndexSpec.DEFAULT, null); + indexMerger.persist(index, outDir, IndexSpec.getDefault(), null); return new QueryableIndexSegment(indexIO.loadIndex(outDir), SegmentId.dummy("")); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongDoubleComplexMetricSerdeTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongDoubleComplexMetricSerdeTest.java index e2ce60dd2752..f74dc318db73 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongDoubleComplexMetricSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongDoubleComplexMetricSerdeTest.java @@ -143,7 +143,7 @@ private ByteBuffer assertExpected( SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium(); ByteBuffer compressedBuffer = serializeAllValuesToByteBuffer( expected, - COMPRESSED_SERDE.getSerializer(writeOutMedium, "not-used", IndexSpec.DEFAULT), + COMPRESSED_SERDE.getSerializer(writeOutMedium, "not-used", IndexSpec.getDefault()), expectedCompressedSize ).asReadOnlyBuffer(); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongFloatComplexMetricSerdeTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongFloatComplexMetricSerdeTest.java index 6872b531e342..0a00a9e82a9d 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongFloatComplexMetricSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongFloatComplexMetricSerdeTest.java @@ -143,7 +143,7 @@ private ByteBuffer assertExpected( SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium(); ByteBuffer compressedBuffer = serializeAllValuesToByteBuffer( expected, - COMPRESSED_SERDE.getSerializer(writeOutMedium, "not-used", IndexSpec.DEFAULT), + COMPRESSED_SERDE.getSerializer(writeOutMedium, "not-used", IndexSpec.getDefault()), expectedCompressedSize ).asReadOnlyBuffer(); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongLongComplexMetricSerdeTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongLongComplexMetricSerdeTest.java index 9df47dff3647..de8b76766318 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongLongComplexMetricSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongLongComplexMetricSerdeTest.java @@ -143,7 +143,7 @@ private ByteBuffer assertExpected( SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium(); ByteBuffer compressedBuffer = serializeAllValuesToByteBuffer( expected, - COMPRESSED_SERDE.getSerializer(writeOutMedium, "not-used", IndexSpec.DEFAULT), + COMPRESSED_SERDE.getSerializer(writeOutMedium, "not-used", IndexSpec.getDefault()), expectedCompressedSize ).asReadOnlyBuffer(); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexMetricSerdeTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexMetricSerdeTest.java index 94f8bf8b6664..e9ce680caf55 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexMetricSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexMetricSerdeTest.java @@ -148,12 +148,12 @@ private ByteBuffer assertExpected( SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium(); ByteBuffer legacyBuffer = serializeAllValuesToByteBuffer( expected, - LEGACY_SERDE.getSerializer(writeOutMedium, "not-used", IndexSpec.DEFAULT), + LEGACY_SERDE.getSerializer(writeOutMedium, "not-used", IndexSpec.getDefault()), expectedLegacySize ).asReadOnlyBuffer(); ByteBuffer compressedBuffer = serializeAllValuesToByteBuffer( expected, - COMPRESSED_SERDE.getSerializer(writeOutMedium, "not-used", IndexSpec.DEFAULT), + COMPRESSED_SERDE.getSerializer(writeOutMedium, "not-used", IndexSpec.getDefault()), expectedCompressedSize ).asReadOnlyBuffer(); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java index b2c723a0fe27..9e4706d44f58 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java @@ -201,7 +201,7 @@ public void setup() throws Exception final File fileA = INDEX_MERGER_V9.persist( indexA, new File(tmpDir, "A"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), OffHeapMemorySegmentWriteOutMediumFactory.instance() ); QueryableIndex qindexA = INDEX_IO.loadIndex(fileA); @@ -243,7 +243,7 @@ public void setup() throws Exception final File fileB = INDEX_MERGER_V9.persist( indexB, new File(tmpDir, "B"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), OffHeapMemorySegmentWriteOutMediumFactory.instance() ); QueryableIndex qindexB = INDEX_IO.loadIndex(fileB); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java index 116e3b811748..df03020a8acf 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java @@ -200,7 +200,7 @@ public void setup() throws Exception final File fileA = INDEX_MERGER_V9.persist( indexA, new File(tmpDir, "A"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); QueryableIndex qindexA = INDEX_IO.loadIndex(fileA); @@ -236,7 +236,7 @@ public void setup() throws Exception final File fileB = INDEX_MERGER_V9.persist( indexB, new File(tmpDir, "B"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); QueryableIndex qindexB = INDEX_IO.loadIndex(fileB); @@ -271,7 +271,7 @@ public void setup() throws Exception final File fileC = INDEX_MERGER_V9.persist( indexC, new File(tmpDir, "C"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); QueryableIndex qindexC = INDEX_IO.loadIndex(fileC); @@ -307,7 +307,7 @@ public void setup() throws Exception final File fileD = INDEX_MERGER_V9.persist( indexD, new File(tmpDir, "D"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); QueryableIndex qindexD = INDEX_IO.loadIndex(fileD); @@ -422,7 +422,7 @@ public void setup() throws Exception final File fileE = INDEX_MERGER_V9.persist( indexE, new File(tmpDir, "E"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); QueryableIndex qindexE = INDEX_IO.loadIndex(fileE); @@ -517,7 +517,7 @@ public void setup() throws Exception final File fileF = INDEX_MERGER_V9.persist( indexF, new File(tmpDir, "F"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); QueryableIndex qindexF = INDEX_IO.loadIndex(fileF); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java index e5de62ee7c07..bb0cb89d69dd 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java @@ -160,7 +160,7 @@ public void setup() throws Exception final File fileA = INDEX_MERGER_V9.persist( indexA, new File(tmpDir, "A"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); QueryableIndex qindexA = INDEX_IO.loadIndex(fileA); @@ -182,7 +182,7 @@ public void setup() throws Exception final File fileB = INDEX_MERGER_V9.persist( indexB, new File(tmpDir, "B"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); QueryableIndex qindexB = INDEX_IO.loadIndex(fileB); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java index bf3484af4e08..5b89ca1c1415 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java @@ -185,7 +185,7 @@ public void setup() throws Exception final File fileA = INDEX_MERGER_V9.persist( indexA, new File(tmpDir, "A"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); QueryableIndex qindexA = INDEX_IO.loadIndex(fileA); @@ -229,7 +229,7 @@ public void setup() throws Exception final File fileB = INDEX_MERGER_V9.persist( indexB, new File(tmpDir, "B"), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); QueryableIndex qindexB = INDEX_IO.loadIndex(fileB); 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 bd98287407d0..e297cedc425b 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 @@ -141,7 +141,7 @@ public void testIngestAndScanSegmentsRollup() throws Exception NestedDataTestUtils.COUNT, Granularities.YEAR, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ) ).build(); @@ -345,7 +345,7 @@ public void testIngestWithMergesAndScanSegments() throws Exception NestedDataTestUtils.SIMPLE_DATA_FILE, Granularities.HOUR, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); final Sequence seq = helper.runQueryOnSegmentsObjs(segs, scanQuery); @@ -529,7 +529,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscovery() throws Exception NestedDataTestUtils.COUNT, Granularities.DAY, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); @@ -592,7 +592,7 @@ public void testIngestAndScanSegmentsRealtimeAutoExplicit() throws Exception NestedDataTestUtils.COUNT, Granularities.DAY, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); @@ -648,7 +648,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryArrayTypes() throws aggs, Granularities.NONE, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); @@ -703,7 +703,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryMoreArrayTypes() thr aggs, Granularities.NONE, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); @@ -766,7 +766,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryTypeGauntlet() throw aggs, Granularities.NONE, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); @@ -820,7 +820,7 @@ public void testIngestAndScanSegmentsAndFilterPartialPathArrayIndex() throws Exc NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE, Granularities.HOUR, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); List realtimeSegs = ImmutableList.of( @@ -878,7 +878,7 @@ public void testIngestAndScanSegmentsAndFilterPartialPath() throws Exception NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE, Granularities.HOUR, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); List realtimeSegs = ImmutableList.of( @@ -943,7 +943,7 @@ public void testIngestAndScanSegmentsNestedColumnNotNullFilter() throws Exceptio aggs, Granularities.NONE, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); diff --git a/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java index 1aa737cd704a..2c256c1d2adf 100644 --- a/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java @@ -20,7 +20,6 @@ package org.apache.druid.segment; import nl.jqno.equalsverifier.EqualsVerifier; -import org.apache.druid.segment.data.CompressionStrategy; import org.junit.Test; import java.io.IOException; @@ -44,11 +43,6 @@ public void testEquals() { EqualsVerifier.forClass(CompleteSegment.class) .withNonnullFields("segment", "dataSegment") - .withPrefabValues( - IndexSpec.class, - IndexSpec.builder().build(), - IndexSpec.builder().withComplexMetricCompression(CompressionStrategy.ZSTD).build() - ) .usingGetClass() .verify(); } diff --git a/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java index 11bef46dc686..c57ae35fa5ca 100644 --- a/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java @@ -83,7 +83,7 @@ public void testDefaultSegmentizerPersist() throws IOException data, Intervals.of("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z"), segment, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); diff --git a/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java b/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java index 1e2b69239d39..41af9742bfa9 100644 --- a/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java +++ b/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java @@ -24,7 +24,6 @@ import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.segment.data.CompressionStrategy; import org.junit.Assert; import org.junit.Test; @@ -59,11 +58,6 @@ public void testEqualsAndHashcode() { EqualsVerifier.forClass(DefaultColumnFormatConfig.class) .usingGetClass() - .withPrefabValues( - IndexSpec.class, - IndexSpec.builder().build(), - IndexSpec.builder().withComplexMetricCompression(CompressionStrategy.ZSTD).build() - ) .verify(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java b/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java index c9026a177c2c..9fab20408ff3 100644 --- a/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java @@ -90,7 +90,7 @@ public void testEmptyIndex() throws Exception new AggregatorFactory[0], tmpDir, DimensionsSpec.EMPTY, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), -1 ); diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java index 7e52760a1768..7d4dbda00d87 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -75,7 +75,7 @@ public class IndexBuilder private SegmentWriteOutMediumFactory segmentWriteOutMediumFactory = OffHeapMemorySegmentWriteOutMediumFactory.instance(); private IndexMerger indexMerger; private File tmpDir; - private IndexSpec indexSpec = IndexSpec.DEFAULT; + private IndexSpec indexSpec = IndexSpec.getDefault(); private int maxRows = DEFAULT_MAX_ROWS; private int intermediatePersistSize = ROWS_PER_INDEX_FOR_MERGING; private IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() diff --git a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java index b2c9f4e7fb44..ebbf23832de3 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java @@ -102,7 +102,7 @@ public void setup() throws IOException segmentDir = indexMerger.persist( incrementalIndex, temporaryFolder.newFolder(), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), OffHeapMemorySegmentWriteOutMediumFactory.instance() ); } diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java index 02fb30d417f2..7b6fda4a18b4 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java @@ -70,7 +70,7 @@ public void setUp() { indexMerger = TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()); indexIO = TestHelper.getTestIndexIO(); - indexSpec = IndexSpec.DEFAULT; + indexSpec = IndexSpec.getDefault(); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerRollupTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerRollupTest.java index c0b8cceb24b6..0651eca96ab2 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerRollupTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerRollupTest.java @@ -65,7 +65,7 @@ public void setUp() indexMerger = TestHelper .getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()); indexIO = TestHelper.getTestIndexIO(); - indexSpec = IndexSpec.DEFAULT; + indexSpec = IndexSpec.getDefault(); } private void testFirstLastRollup( diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java index 98eb6cb79061..686f4f571b81 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -3061,7 +3061,7 @@ public void testMergeProjections() throws IOException new AggregatorFactory[0], temporaryFolder.newFolder(), dimensionsBuilder.build(), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), -1 ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java index 8388adf7d895..ac2fa4d33835 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java @@ -89,7 +89,7 @@ public static Collection constructorFeeder() throws IOException IndexMergerV9 indexMergerV9 = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); IndexIO indexIO = TestHelper.getTestIndexIO(); - final IndexSpec indexSpec = IndexSpec.DEFAULT; + final IndexSpec indexSpec = IndexSpec.getDefault(); final IncrementalIndex rtIndex = makeIncrementalIndex(); final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec, indexMergerV9, indexIO); final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec, indexMergerV9, indexIO); diff --git a/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java b/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java index 960f7fb96d54..c4e83f58dbef 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java @@ -63,7 +63,7 @@ public void testSerdeUncompressed() throws Exception @Test public void testDefaults() { - final IndexSpec spec = IndexSpec.DEFAULT; + final IndexSpec spec = IndexSpec.getDefault(); Assert.assertEquals(CompressionStrategy.LZ4, spec.getDimensionCompression()); Assert.assertEquals(CompressionStrategy.LZ4, spec.getMetricCompression()); Assert.assertEquals(LongEncodingStrategy.LONGS, spec.getLongEncoding()); @@ -75,7 +75,7 @@ public void testEquals() EqualsVerifier.forClass(IndexSpec.class) .withPrefabValues( IndexSpec.class, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), IndexSpec.builder() .withJsonCompression(CompressionStrategy.ZSTD) .build() diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java index 0fef93bb83ea..3b6b8ec7bced 100644 --- a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java @@ -75,7 +75,7 @@ public class SchemalessIndexTest new CountAggregatorFactory("count") }; - private static final IndexSpec INDEX_SPEC = IndexSpec.DEFAULT; + private static final IndexSpec INDEX_SPEC = IndexSpec.getDefault(); private static final List> EVENTS = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/segment/TestIndex.java b/processing/src/test/java/org/apache/druid/segment/TestIndex.java index fb3f864c037b..c4d363b8ae8b 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestIndex.java +++ b/processing/src/test/java/org/apache/druid/segment/TestIndex.java @@ -204,7 +204,7 @@ public class TestIndex ) .build() ); - public static final IndexSpec INDEX_SPEC = IndexSpec.DEFAULT; + public static final IndexSpec INDEX_SPEC = IndexSpec.getDefault(); public static final JsonInputFormat DEFAULT_JSON_INPUT_FORMAT = new JsonInputFormat( JSONPathSpec.DEFAULT, diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestCursorFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestCursorFactoryTest.java index 5d16e579bdd6..9c98be20ee35 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestCursorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestCursorFactoryTest.java @@ -142,7 +142,7 @@ public static void setup() throws IOException .withMinTimestamp(0) .build() ) - .indexSpec(IndexSpec.DEFAULT) + .indexSpec(IndexSpec.getDefault()) .inputSource(inputSource) .inputFormat(TestIndex.DEFAULT_JSON_INPUT_FORMAT) .transform(TransformSpec.NONE) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java index 57939f73aed2..407625894ad2 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java @@ -103,7 +103,7 @@ public static Collection constructorFeeder() throws IOException for (SegmentWriteOutMediumFactory segmentWriteOutMediumFactory : SegmentWriteOutMediumFactory.builtInFactories()) { IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); IndexIO indexIO = TestHelper.getTestIndexIO(); - final IndexSpec indexSpec = IndexSpec.DEFAULT; + final IndexSpec indexSpec = IndexSpec.getDefault(); final IncrementalIndex rtIndex = makeIncrementalIndex(); final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec, indexMerger, indexIO); final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec, indexMerger, indexIO); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java index 3b407dc8e947..3ffb5a83e4c8 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java @@ -114,7 +114,7 @@ public SpatialFilterTest(Segment segment) @Parameterized.Parameters public static Collection constructorFeeder() throws IOException { - final IndexSpec indexSpec = IndexSpec.DEFAULT; + final IndexSpec indexSpec = IndexSpec.getDefault(); final IndexSpec frontCodedIndexSpec = IndexSpec.builder() .withStringDictionaryEncoding(new StringEncodingStrategy.FrontCoded(4, FrontCodedIndexed.V1)) diff --git a/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java b/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java index dec956b281d5..b8b4221eb6be 100644 --- a/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java +++ b/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java @@ -117,7 +117,7 @@ public QueryableIndex generate( schemaInfo, schemaInfo.getDimensionsSpec(), TransformSpec.NONE, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), granularity, Collections.emptyList(), numRows diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java index 80374df12954..9f5f82720aab 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java @@ -123,7 +123,7 @@ public void setup() throws IOException, SegmentLoadingException data, testInterval, segment, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null ); File factoryJson = new File(persisted, "factory.json"); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java index 284270eb1e90..6d6f886f3e68 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java @@ -61,7 +61,7 @@ public void testGetEffectiveSpecDefaults() { NestedCommonFormatColumnFormatSpec defaults = NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( null, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); Assert.assertEquals( @@ -77,19 +77,19 @@ public void testGetEffectiveSpecDefaults() defaults.getObjectStorageCompression() ); Assert.assertEquals( - IndexSpec.DEFAULT.getDimensionCompression(), + IndexSpec.getDefault().getDimensionCompression(), defaults.getDictionaryEncodedColumnCompression() ); Assert.assertEquals( - IndexSpec.DEFAULT.getStringDictionaryEncoding(), + IndexSpec.getDefault().getStringDictionaryEncoding(), defaults.getStringDictionaryEncoding() ); Assert.assertEquals( - IndexSpec.DEFAULT.getMetricCompression(), + IndexSpec.getDefault().getMetricCompression(), defaults.getLongColumnCompression() ); Assert.assertEquals( - IndexSpec.DEFAULT.getMetricCompression(), + IndexSpec.getDefault().getMetricCompression(), defaults.getDoubleColumnCompression() ); } @@ -108,7 +108,7 @@ public void testGetEffectiveSpecMerge() ) .setDoubleColumnCompression(CompressionStrategy.ZSTD) .build(), - IndexSpec.DEFAULT + IndexSpec.getDefault() ); Assert.assertEquals( @@ -124,7 +124,7 @@ public void testGetEffectiveSpecMerge() merged.getObjectStorageEncoding() ); Assert.assertEquals( - IndexSpec.DEFAULT.getDimensionCompression(), + IndexSpec.getDefault().getDimensionCompression(), merged.getDictionaryEncodedColumnCompression() ); Assert.assertEquals( @@ -132,7 +132,7 @@ public void testGetEffectiveSpecMerge() merged.getObjectStorageCompression() ); Assert.assertEquals( - IndexSpec.DEFAULT.getMetricCompression(), + IndexSpec.getDefault().getMetricCompression(), merged.getLongColumnCompression() ); Assert.assertEquals( 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 4100e883daf5..0e19122905fe 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 @@ -203,7 +203,7 @@ public NestedDataColumnSupplierTest( { this.columnFormatSpec = NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( columnFormatSpec, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); this.bitmapSerdeFactory = this.columnFormatSpec.getBitmapEncoding(); this.resultFactory = new DefaultBitmapResultFactory(bitmapSerdeFactory.getBitmapFactory()); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java index 6e087d1883a4..7db1445f60f4 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java @@ -343,7 +343,7 @@ private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns vir NestedDataTestUtils.COUNT, Granularities.NONE, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); Assert.assertEquals(1, segments.size()); Segment segment = segments.get(0); @@ -370,7 +370,7 @@ private VectorColumnSelectorFactory getVectorColumnSelectorFactory(VirtualColumn NestedDataTestUtils.COUNT, Granularities.NONE, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); Assert.assertEquals(1, segments.size()); Segment segment = segments.get(0); 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 index ef1265ceffba..87a2abfc77e9 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java @@ -126,7 +126,7 @@ private SmooshedFileMapper smooshify( try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { ScalarDoubleColumnSerializer serializer = new ScalarDoubleColumnSerializer( fileNameBase, - NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.DEFAULT), + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault()), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); 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 index 84d66c0a5769..676c47d9f456 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java @@ -126,7 +126,7 @@ private SmooshedFileMapper smooshify( try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { ScalarLongColumnSerializer serializer = new ScalarLongColumnSerializer( fileNameBase, - NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.DEFAULT), + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault()), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); 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 index 51e84c1726ce..780a2c1b062a 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -126,7 +126,7 @@ private SmooshedFileMapper smooshify( try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { ScalarStringColumnSerializer serializer = new ScalarStringColumnSerializer( fileNameBase, - NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.DEFAULT), + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault()), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java index e9ab4395566e..5f59eff5f5cf 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java @@ -272,7 +272,7 @@ private SmooshedFileMapper smooshify( fileNameBase, expectedTypes.getSingleType() == null ? null : expectedLogicalType, expectedTypes.getSingleType() == null ? expectedTypes.getByteValue() : null, - NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(columnFormatSpec, IndexSpec.DEFAULT), + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(columnFormatSpec, IndexSpec.getDefault()), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); diff --git a/quidem-ut/src/main/java/org/apache/druid/quidem/KttmNestedComponentSupplier.java b/quidem-ut/src/main/java/org/apache/druid/quidem/KttmNestedComponentSupplier.java index 88ee3a78c37d..26ea7c726be0 100644 --- a/quidem-ut/src/main/java/org/apache/druid/quidem/KttmNestedComponentSupplier.java +++ b/quidem-ut/src/main/java/org/apache/druid/quidem/KttmNestedComponentSupplier.java @@ -80,7 +80,7 @@ public QueryableIndex makeKttmIndex(File tmpDir) try { final File directory = new File(tmpDir, StringUtils.format("kttm-index-%s", UUID.randomUUID())); final IncrementalIndex index = makeKttmNestedIndex(); - TestIndex.INDEX_MERGER.persist(index, directory, IndexSpec.DEFAULT, null); + TestIndex.INDEX_MERGER.persist(index, directory, IndexSpec.getDefault(), null); return TestIndex.INDEX_IO.loadIndex(directory); } catch (IOException e) { diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index 3af7c5c08efa..dad1c13cdeb5 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -344,7 +344,7 @@ private CompactionStatus indexSpecIsUpToDate() { return CompactionStatus.completeIfEqual( "indexSpec", - Configs.valueOrDefault(tuningConfig.getIndexSpec(), IndexSpec.DEFAULT), + Configs.valueOrDefault(tuningConfig.getIndexSpec(), IndexSpec.getDefault()), objectMapper.convertValue(lastCompactionState.getIndexSpec(), IndexSpec.class), String::valueOf ); @@ -446,7 +446,7 @@ private CompactionStatus dimensionsSpecIsUpToDate() compactionConfig.getDimensionsSpec().getDimensions(), compactionConfig.getTuningConfig() == null ? null : compactionConfig.getTuningConfig().getPartitionsSpec(), compactionConfig.getTuningConfig() == null - ? IndexSpec.DEFAULT + ? IndexSpec.getDefault() : compactionConfig.getTuningConfig().getIndexSpec() ); return CompactionStatus.completeIfEqual( diff --git a/server/src/test/java/org/apache/druid/indexing/ClientCompactQueryTuningConfigTest.java b/server/src/test/java/org/apache/druid/indexing/ClientCompactQueryTuningConfigTest.java index a8ae549df3fd..7a43e63685e6 100644 --- a/server/src/test/java/org/apache/druid/indexing/ClientCompactQueryTuningConfigTest.java +++ b/server/src/test/java/org/apache/druid/indexing/ClientCompactQueryTuningConfigTest.java @@ -34,7 +34,7 @@ public void testEqualsContract() EqualsVerifier.forClass(ClientCompactionTaskQueryTuningConfig.class) .withPrefabValues( IndexSpec.class, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build() ).usingGetClass() .verify(); diff --git a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java index 958566862a9d..cf64244b86c9 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java @@ -1171,7 +1171,7 @@ private void writeSegmentFile(final DataSegment segment) throws IOException static File makeSegmentZip(File segmentFiles, File zipOutFile) throws IOException { - TestIndex.persist(TestIndex.getIncrementalTestIndex(), IndexSpec.DEFAULT, segmentFiles); + TestIndex.persist(TestIndex.getIncrementalTestIndex(), IndexSpec.getDefault(), segmentFiles); FileUtils.mkdirp(zipOutFile.getParentFile()); CompressionUtils.zip(segmentFiles, zipOutFile); return zipOutFile; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java index 6ac64dc05708..08424ebe327e 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -168,7 +168,7 @@ public BatchAppenderatorTester( maxRowsInMemory, maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, skipBytesInMemoryOverheadCheck, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), 0, false, 0L, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java index d4d0c97535a3..06745e98a040 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java @@ -155,7 +155,7 @@ public StreamAppenderatorTester( maxRowsInMemory, maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, skipBytesInMemoryOverheadCheck, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), 0, false, 0L, diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index c87f2a810cba..99d1e9ac2d63 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -447,14 +447,14 @@ public void testStatusWhenAutoSchemaMatch() DimensionsSpec.builder() .setDimensions( List.of( - AutoTypeColumnSchema.of("x").getEffectiveSchema(IndexSpec.DEFAULT), - AutoTypeColumnSchema.of("y").getEffectiveSchema(IndexSpec.DEFAULT) + AutoTypeColumnSchema.of("x").getEffectiveSchema(IndexSpec.getDefault()), + AutoTypeColumnSchema.of("y").getEffectiveSchema(IndexSpec.getDefault()) ) ) .build(), null, null, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), currentGranularitySpec, Collections.emptyList() ); @@ -475,7 +475,7 @@ public void testStatusWhenAutoSchemaMatch() ) ) ) - .withTuningConfig(createTuningConfig(currentPartitionsSpec, IndexSpec.DEFAULT)) + .withTuningConfig(createTuningConfig(currentPartitionsSpec, IndexSpec.getDefault())) .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) .withProjections(Collections.emptyList()) .build(); @@ -501,14 +501,14 @@ public void testStatusWhenAutoSchemaMismatch() DimensionsSpec.builder() .setDimensions( List.of( - AutoTypeColumnSchema.of("x").getEffectiveSchema(IndexSpec.DEFAULT), - AutoTypeColumnSchema.of("y").getEffectiveSchema(IndexSpec.DEFAULT) + AutoTypeColumnSchema.of("x").getEffectiveSchema(IndexSpec.getDefault()), + AutoTypeColumnSchema.of("y").getEffectiveSchema(IndexSpec.getDefault()) ) ) .build(), null, null, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), currentGranularitySpec, Collections.emptyList() ); @@ -529,7 +529,7 @@ public void testStatusWhenAutoSchemaMismatch() ) ) ) - .withTuningConfig(createTuningConfig(currentPartitionsSpec, IndexSpec.DEFAULT)) + .withTuningConfig(createTuningConfig(currentPartitionsSpec, IndexSpec.getDefault())) .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) .withProjections(Collections.emptyList()) .build(); diff --git a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java index 5cba8cc17c22..c655dc8ae479 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java @@ -742,7 +742,7 @@ public void testIteratorReturnsAllMixedVersionSegmentsInConfiguredSegmentGranula public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGranularityAndSameTimezone() { // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Same partitionsSpec as what is set in the auto compaction config PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( null)); @@ -771,7 +771,7 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGranularityInLastCompactionState() { // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Same partitionsSpec as what is set in the auto compaction config PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( null)); @@ -807,7 +807,7 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSegmentGranularity() { // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Same partitionsSpec as what is set in the auto compaction config PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( null)); @@ -855,7 +855,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSegmentGranularityInLastCompactionState() { // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Same partitionsSpec as what is set in the auto compaction config PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( null)); @@ -904,7 +904,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTimezone() { // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Same partitionsSpec as what is set in the auto compaction config PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( null)); @@ -954,7 +954,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTim public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOrigin() { // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Same partitionsSpec as what is set in the auto compaction config PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( null)); @@ -1003,7 +1003,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOri public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRollup() { // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Same partitionsSpec as what is set in the auto compaction config PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( null)); @@ -1092,7 +1092,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQueryGranularity() { // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Same partitionsSpec as what is set in the auto compaction config PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( null)); @@ -1187,7 +1187,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDimensions() { // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Same partitionsSpec as what is set in the auto compaction config PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( null)); @@ -1306,7 +1306,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim public void testIteratorDoesNotReturnsSegmentsWhenPartitionDimensionsPrefixed() { // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Set range partitions spec with dimensions ["dim2", "dim4"] -- the same as what is set in the auto compaction config PartitionsSpec partitionsSpec = new DimensionRangePartitionsSpec( null, @@ -1377,7 +1377,7 @@ public void testIteratorDoesNotReturnsSegmentsWhenPartitionDimensionsPrefixed() 1000L, null, partitionsSpec, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null, null, null, @@ -1416,7 +1416,7 @@ public void testIteratorDoesNotReturnsSegmentsWhenPartitionDimensionsPrefixed() public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFilter() throws Exception { // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Same partitionsSpec as what is set in the auto compaction config PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( null)); @@ -1538,7 +1538,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) ); // Same indexSpec as what is set in the auto compaction config - IndexSpec indexSpec = IndexSpec.DEFAULT; + IndexSpec indexSpec = IndexSpec.getDefault(); // Same partitionsSpec as what is set in the auto compaction config PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( null)); @@ -1746,7 +1746,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() .startingAt("2017-10-01") .withNumPartitions(4) .withCompactionState( - new CompactionState(partitionsSpec, null, null, null, IndexSpec.DEFAULT, null, null) + new CompactionState(partitionsSpec, null, null, null, IndexSpec.getDefault(), null, null) ) ); @@ -1759,7 +1759,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() 1000L, null, partitionsSpec, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null, null, null, @@ -1787,7 +1787,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() 1000L, null, partitionsSpec, - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null, null, null, diff --git a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java index 2a63fc92f0f0..47b0ae9534f1 100644 --- a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java +++ b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java @@ -292,7 +292,7 @@ public static List createSegments( }, Granularities.HOUR, true, - IndexSpec.DEFAULT + IndexSpec.getDefault() ); } } diff --git a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java index 5ced6c377579..52f1bf4f6263 100644 --- a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java @@ -156,7 +156,7 @@ public void testTaskValidator() throws Exception null, null, new DynamicPartitionsSpec(10000, null), - IndexSpec.DEFAULT, + IndexSpec.getDefault(), null, 3, false, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java index acb669849ded..65d511c55d6a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java @@ -592,7 +592,7 @@ public static QueryableIndex makeWikipediaIndex(File tmpDir) try { final File directory = new File(tmpDir, StringUtils.format("wikipedia-index-%s", UUID.randomUUID())); final IncrementalIndex index = TestIndex.makeWikipediaIncrementalIndex(); - TestIndex.INDEX_MERGER.persist(index, directory, IndexSpec.DEFAULT, null); + TestIndex.INDEX_MERGER.persist(index, directory, IndexSpec.getDefault(), null); return TestIndex.INDEX_IO.loadIndex(directory); } catch (IOException e) { From 054d9da15d8a6b0cd6406af8f2a3fc8edf660d44 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 22 Sep 2025 12:07:30 -0700 Subject: [PATCH 12/24] nullable --- .../main/java/org/apache/druid/segment/IndexSpec.java | 10 ++++------ .../druid/segment/NestedCommonFormatColumnHandler.java | 2 +- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java index 9d9ef761a85e..97dc38324d60 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java @@ -63,6 +63,7 @@ public static Builder builder() private final CompressionStrategy jsonCompression; @Nullable private final SegmentizerFactory segmentLoader; + @Nullable private final NestedCommonFormatColumnFormatSpec autoColumnFormatSpec; /** @@ -113,11 +114,7 @@ public IndexSpec( : longEncoding; this.jsonCompression = jsonCompression; this.segmentLoader = segmentLoader; - this.autoColumnFormatSpec = autoColumnFormatSpec == null - ? NestedCommonFormatColumnFormatSpec.builder() - .setObjectStorageCompression(jsonCompression) - .build() - : autoColumnFormatSpec; + this.autoColumnFormatSpec = autoColumnFormatSpec; } @JsonProperty("bitmap") @@ -176,7 +173,8 @@ public CompressionStrategy getJsonCompression() } @JsonProperty - @JsonInclude(JsonInclude.Include.NON_DEFAULT) + @JsonInclude(JsonInclude.Include.NON_NULL) + @Nullable public NestedCommonFormatColumnFormatSpec getAutoColumnFormatSpec() { return autoColumnFormatSpec; diff --git a/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java b/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java index ad6532a2c2f4..2047301a6f59 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java @@ -52,7 +52,7 @@ public class NestedCommonFormatColumnHandler implements DimensionHandler Date: Mon, 22 Sep 2025 14:09:47 -0700 Subject: [PATCH 13/24] fix npe --- .../NestedCommonFormatColumnFormatSpec.java | 37 +++++++++++-------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java index 8380e618c943..c914dc0c1b98 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java @@ -34,6 +34,12 @@ public class NestedCommonFormatColumnFormatSpec { + private static final NestedCommonFormatColumnFormatSpec DEFAULT = + NestedCommonFormatColumnFormatSpec.builder() + .setObjectFieldsDictionaryEncoding(StringEncodingStrategy.UTF8_STRATEGY) + .setObjectStorageEncoding(ObjectStorageEncoding.SMILE) + .build(); + public static Builder builder() { return new Builder(); @@ -44,13 +50,7 @@ public static NestedCommonFormatColumnFormatSpec getEffectiveFormatSpec( IndexSpec indexSpec ) { - return Objects.requireNonNullElseGet( - columnFormatSpec, - () -> NestedCommonFormatColumnFormatSpec.builder() - .setObjectFieldsDictionaryEncoding(StringEncodingStrategy.UTF8_STRATEGY) - .setObjectStorageEncoding(ObjectStorageEncoding.SMILE) - .build() - ).getEffectiveSpec(indexSpec); + return Objects.requireNonNullElse(columnFormatSpec, DEFAULT).getEffectiveSpec(indexSpec); } @Nullable @@ -143,8 +143,15 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) Builder builder = new Builder(this); builder.setBitmapEncoding(indexSpec.getBitmapSerdeFactory()); + final NestedCommonFormatColumnFormatSpec defaultSpec; + if (indexSpec.getAutoColumnFormatSpec() != null) { + defaultSpec = indexSpec.getAutoColumnFormatSpec(); + } else { + defaultSpec = DEFAULT; + } + if (objectFieldsDictionaryEncoding == null) { - if (indexSpec.getAutoColumnFormatSpec().getObjectFieldsDictionaryEncoding() != null) { + if (defaultSpec.getObjectFieldsDictionaryEncoding() != null) { builder.setObjectFieldsDictionaryEncoding( indexSpec.getAutoColumnFormatSpec().getObjectFieldsDictionaryEncoding() ); @@ -154,7 +161,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) } if (objectStorageEncoding == null) { - if (indexSpec.getAutoColumnFormatSpec().getObjectStorageEncoding() != null) { + if (defaultSpec.getObjectStorageEncoding() != null) { builder.setObjectStorageEncoding(indexSpec.getAutoColumnFormatSpec().getObjectStorageEncoding()); } else { builder.setObjectStorageEncoding(ObjectStorageEncoding.SMILE); @@ -162,7 +169,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) } if (objectStorageCompression == null) { - if (indexSpec.getAutoColumnFormatSpec().getObjectStorageCompression() != null) { + if (defaultSpec.getObjectStorageCompression() != null) { builder.setObjectStorageCompression(indexSpec.getAutoColumnFormatSpec().getObjectStorageCompression()); } else if (indexSpec.getJsonCompression() != null) { builder.setObjectStorageCompression(indexSpec.getJsonCompression()); @@ -172,7 +179,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) } if (stringDictionaryEncoding == null) { - if (indexSpec.getAutoColumnFormatSpec().getStringDictionaryEncoding() != null) { + if (defaultSpec.getStringDictionaryEncoding() != null) { builder.setStringDictionaryEncoding(indexSpec.getAutoColumnFormatSpec().getStringDictionaryEncoding()); } else { builder.setStringDictionaryEncoding(indexSpec.getStringDictionaryEncoding()); @@ -180,7 +187,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) } if (dictionaryEncodedColumnCompression == null) { - if (indexSpec.getAutoColumnFormatSpec().getDictionaryEncodedColumnCompression() != null) { + if (defaultSpec.getDictionaryEncodedColumnCompression() != null) { builder.setDictionaryEncodedColumnCompression( indexSpec.getAutoColumnFormatSpec().getDictionaryEncodedColumnCompression() ); @@ -190,7 +197,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) } if (longColumnEncoding == null) { - if (indexSpec.getAutoColumnFormatSpec().getLongColumnEncoding() != null) { + if (defaultSpec.getLongColumnEncoding() != null) { builder.setLongColumnEncoding(indexSpec.getAutoColumnFormatSpec().getLongColumnEncoding()); } else { builder.setLongColumnEncoding(indexSpec.getLongEncoding()); @@ -198,7 +205,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) } if (longColumnCompression == null) { - if (indexSpec.getAutoColumnFormatSpec().getLongColumnCompression() != null) { + if (defaultSpec.getLongColumnCompression() != null) { builder.setLongColumnCompression(indexSpec.getAutoColumnFormatSpec().getLongColumnCompression()); } else { builder.setLongColumnCompression(indexSpec.getMetricCompression()); @@ -206,7 +213,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) } if (doubleColumnCompression == null) { - if (indexSpec.getAutoColumnFormatSpec().getDoubleColumnCompression() != null) { + if (defaultSpec.getDoubleColumnCompression() != null) { builder.setDoubleColumnCompression(indexSpec.getAutoColumnFormatSpec().getDoubleColumnCompression()); } else { builder.setDoubleColumnCompression(indexSpec.getMetricCompression()); From e453fdf81980843328be5c0709a70f411622c25b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 22 Sep 2025 14:22:36 -0700 Subject: [PATCH 14/24] oops --- .../NestedCommonFormatColumnFormatSpec.java | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java index c914dc0c1b98..781a7ee98d4a 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java @@ -152,9 +152,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) if (objectFieldsDictionaryEncoding == null) { if (defaultSpec.getObjectFieldsDictionaryEncoding() != null) { - builder.setObjectFieldsDictionaryEncoding( - indexSpec.getAutoColumnFormatSpec().getObjectFieldsDictionaryEncoding() - ); + builder.setObjectFieldsDictionaryEncoding(defaultSpec.getObjectFieldsDictionaryEncoding()); } else { builder.setObjectFieldsDictionaryEncoding(StringEncodingStrategy.DEFAULT); } @@ -162,7 +160,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) if (objectStorageEncoding == null) { if (defaultSpec.getObjectStorageEncoding() != null) { - builder.setObjectStorageEncoding(indexSpec.getAutoColumnFormatSpec().getObjectStorageEncoding()); + builder.setObjectStorageEncoding(defaultSpec.getObjectStorageEncoding()); } else { builder.setObjectStorageEncoding(ObjectStorageEncoding.SMILE); } @@ -170,7 +168,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) if (objectStorageCompression == null) { if (defaultSpec.getObjectStorageCompression() != null) { - builder.setObjectStorageCompression(indexSpec.getAutoColumnFormatSpec().getObjectStorageCompression()); + builder.setObjectStorageCompression(defaultSpec.getObjectStorageCompression()); } else if (indexSpec.getJsonCompression() != null) { builder.setObjectStorageCompression(indexSpec.getJsonCompression()); } else { @@ -180,7 +178,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) if (stringDictionaryEncoding == null) { if (defaultSpec.getStringDictionaryEncoding() != null) { - builder.setStringDictionaryEncoding(indexSpec.getAutoColumnFormatSpec().getStringDictionaryEncoding()); + builder.setStringDictionaryEncoding(defaultSpec.getStringDictionaryEncoding()); } else { builder.setStringDictionaryEncoding(indexSpec.getStringDictionaryEncoding()); } @@ -188,9 +186,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) if (dictionaryEncodedColumnCompression == null) { if (defaultSpec.getDictionaryEncodedColumnCompression() != null) { - builder.setDictionaryEncodedColumnCompression( - indexSpec.getAutoColumnFormatSpec().getDictionaryEncodedColumnCompression() - ); + builder.setDictionaryEncodedColumnCompression(defaultSpec.getDictionaryEncodedColumnCompression()); } else { builder.setDictionaryEncodedColumnCompression(indexSpec.getDimensionCompression()); } @@ -198,7 +194,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) if (longColumnEncoding == null) { if (defaultSpec.getLongColumnEncoding() != null) { - builder.setLongColumnEncoding(indexSpec.getAutoColumnFormatSpec().getLongColumnEncoding()); + builder.setLongColumnEncoding(defaultSpec.getLongColumnEncoding()); } else { builder.setLongColumnEncoding(indexSpec.getLongEncoding()); } @@ -206,7 +202,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) if (longColumnCompression == null) { if (defaultSpec.getLongColumnCompression() != null) { - builder.setLongColumnCompression(indexSpec.getAutoColumnFormatSpec().getLongColumnCompression()); + builder.setLongColumnCompression(defaultSpec.getLongColumnCompression()); } else { builder.setLongColumnCompression(indexSpec.getMetricCompression()); } @@ -214,7 +210,7 @@ public NestedCommonFormatColumnFormatSpec getEffectiveSpec(IndexSpec indexSpec) if (doubleColumnCompression == null) { if (defaultSpec.getDoubleColumnCompression() != null) { - builder.setDoubleColumnCompression(indexSpec.getAutoColumnFormatSpec().getDoubleColumnCompression()); + builder.setDoubleColumnCompression(defaultSpec.getDoubleColumnCompression()); } else { builder.setDoubleColumnCompression(indexSpec.getMetricCompression()); } From 60994e16b0f0a9417efa8db14980807c3ee39971 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 24 Sep 2025 11:04:30 -0700 Subject: [PATCH 15/24] IndexSpec fields are now all nullable and not populated by default; properly blend job IndexSpec, system default IndexSpec, and hard-coded defaults with new IndexSpec.getEffectiveSpec method --- .../benchmark/query/SqlBaseBenchmark.java | 3 +- .../embedded/compact/AutoCompactionTest.java | 19 ++- .../task/CompactionTaskParallelRunTest.java | 22 +-- .../common/task/CompactionTaskRunTest.java | 10 +- .../druid/segment/AutoTypeColumnSchema.java | 4 +- .../apache/druid/segment/IndexMergerV9.java | 13 +- .../org/apache/druid/segment/IndexSpec.java | 125 ++++++++++++++---- .../NestedCommonFormatColumnFormatSpec.java | 5 + .../druid/timeline/CompactionState.java | 21 +-- .../segment/IndexMergerNullHandlingTest.java | 2 +- .../apache/druid/segment/IndexSpecTest.java | 9 +- .../druid/segment/filter/BaseFilterTest.java | 8 +- ...estedCommonFormatColumnFormatSpecTest.java | 18 +-- .../nested/NestedDataColumnSupplierTest.java | 2 +- .../ScalarDoubleColumnSupplierTest.java | 2 +- .../nested/ScalarLongColumnSupplierTest.java | 2 +- .../ScalarStringColumnSupplierTest.java | 2 +- .../nested/VariantColumnSupplierTest.java | 2 +- .../server/compaction/CompactionStatus.java | 4 +- 19 files changed, 184 insertions(+), 89 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBaseBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBaseBenchmark.java index fc60fa6bb3c1..e5b932a8bbd8 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBaseBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBaseBenchmark.java @@ -226,7 +226,8 @@ protected IndexSpec getIndexSpec() .withComplexMetricCompression( CompressionStrategy.valueOf(StringUtils.toUpperCase(complexCompression)) ) - .build(); + .build() + .getEffectiveSpec(); } @Setup(Level.Trial) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java index f207d4d356be..f77667b311b8 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java @@ -64,6 +64,7 @@ import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.FixedIntervalOrderPolicy; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; @@ -504,7 +505,7 @@ public void testAutoCompactionPreservesCreateBitmapIndexInDimensionSchema(Compac List dimensionSchemas = ImmutableList.of( new StringDimensionSchema("language", DimensionSchema.MultiValueHandling.SORTED_ARRAY, false), - new AutoTypeColumnSchema("deleted", ColumnType.DOUBLE, null).getEffectiveSchema(IndexSpec.getDefault()) + new AutoTypeColumnSchema("deleted", ColumnType.DOUBLE, null) ); submitCompactionConfig( @@ -520,7 +521,21 @@ public void testAutoCompactionPreservesCreateBitmapIndexInDimensionSchema(Compac // Compacted into 1 segment for the entire year. forceTriggerAutoCompaction(1); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); - verifySegmentsCompactedDimensionSchema(dimensionSchemas); + List expectedDimensionSchemas = List.of( + dimensionSchemas.get(0), + new AutoTypeColumnSchema( + "deleted", + ColumnType.DOUBLE, + // json serialization doesn't store bitmap in compaction state, so need to set to null + NestedCommonFormatColumnFormatSpec.builder( + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( + null, + IndexSpec.getDefault().getEffectiveSpec() + ) + ).setBitmapEncoding(null).build() + ) + ); + verifySegmentsCompactedDimensionSchema(expectedDimensionSchemas); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index e78085b0b104..53a7259f5bc6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -203,7 +203,7 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -251,7 +251,7 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -314,7 +314,7 @@ public void testRunParallelWithRangePartitioning() throws Exception ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -367,7 +367,7 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -420,7 +420,7 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -468,7 +468,7 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -519,7 +519,7 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -600,7 +600,7 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception ), ImmutableList.of(expectedLongSumMetric), compactionTask.getTransformSpec(), - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -655,7 +655,7 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception ), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), compactionTask.getTransformSpec(), - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -931,7 +931,7 @@ public void testRunParallelWithProjections() ), Collections.emptyList(), null, - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -990,7 +990,7 @@ public void testRunParallelAddProjections() ), Collections.emptyList(), null, - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 4eb9af6cd3fc..883abb85e5af 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -158,7 +158,7 @@ public class CompactionTaskRunTest extends IngestionTestBase ); private static final NestedCommonFormatColumnFormatSpec DEFAULT_NESTED_SPEC = - NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault()); + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault().getEffectiveSpec()); private static final List TEST_ROWS = ImmutableList.of( "2014-01-01T00:00:10Z,a,1\n", @@ -258,7 +258,7 @@ public static CompactionState getDefaultCompactionState( expectedDims, ImmutableList.of(expectedMetric), null, - IndexSpec.getDefault(), + IndexSpec.getDefault().getEffectiveSpec(), new UniformGranularitySpec( segmentGranularity, queryGranularity, @@ -389,7 +389,7 @@ public void testRunWithHashPartitioning() throws Exception ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec(), + compactionTask.getTuningConfig().getIndexSpec().getEffectiveSpec(), new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -774,7 +774,7 @@ public void testCompactionWithFilterInTransformSpec() throws Exception ), ImmutableList.of(expectedLongSumMetric), compactionTask.getTransformSpec(), - IndexSpec.getDefault(), + IndexSpec.getDefault().getEffectiveSpec(), new UniformGranularitySpec( Granularities.DAY, Granularities.MINUTE, @@ -829,7 +829,7 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception ), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), compactionTask.getTransformSpec(), - IndexSpec.getDefault(), + IndexSpec.getDefault().getEffectiveSpec(), new UniformGranularitySpec( Granularities.DAY, Granularities.MINUTE, diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java index 2d63386dbb00..a079ac5084dd 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnSchema.java @@ -94,9 +94,7 @@ public AutoTypeColumnSchema( } else { this.castToType = castToType; } - this.columnFormatSpec = columnFormatSpec == null - ? IndexSpec.getDefault().getAutoColumnFormatSpec() - : columnFormatSpec; + this.columnFormatSpec = columnFormatSpec; } @Override 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 d337dd0d0aa1..f4741e6059ad 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java @@ -1052,6 +1052,8 @@ public File persist( throw new IAE("Trying to persist an empty index!"); } + indexSpec = indexSpec.getEffectiveSpec(); + final DateTime firstTimestamp = index.getMinTime(); final DateTime lastTimestamp = index.getMaxTime(); if (!(dataInterval.contains(firstTimestamp) && dataInterval.contains(lastTimestamp))) { @@ -1110,8 +1112,8 @@ public File mergeQueryableIndex( metricAggs, dimensionsSpec, outDir, - indexSpec, - indexSpecForIntermediatePersists, + indexSpec.getEffectiveSpec(), + indexSpecForIntermediatePersists.getEffectiveSpec(), progress, segmentWriteOutMediumFactory, maxColumnsToMerge @@ -1138,8 +1140,8 @@ public File merge( metricAggs, dimensionsSpec, outDir, - indexSpec, - indexSpec, + indexSpec.getEffectiveSpec(), + indexSpec.getEffectiveSpec(), new BaseProgressIndicator(), null, maxColumnsToMerge @@ -1164,6 +1166,9 @@ private File multiphaseMerge( List tempDirs = new ArrayList<>(); + indexSpec = indexSpec.getEffectiveSpec(); + indexSpecForIntermediatePersists = indexSpecForIntermediatePersists.getEffectiveSpec(); + if (maxColumnsToMerge == IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE) { return merge( indexes, diff --git a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java index 97dc38324d60..659e78c768be 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java @@ -20,6 +20,7 @@ package org.apache.druid.segment; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.guice.BuiltInTypesModule; @@ -52,10 +53,15 @@ public static Builder builder() return new Builder(); } + @Nullable private final BitmapSerdeFactory bitmapSerdeFactory; + @Nullable private final CompressionStrategy dimensionCompression; + @Nullable private final StringEncodingStrategy stringDictionaryEncoding; + @Nullable private final CompressionStrategy metricCompression; + @Nullable private final CompressionFactory.LongEncodingStrategy longEncoding; @Nullable private final CompressionStrategy complexMetricCompression; @@ -95,53 +101,47 @@ public IndexSpec( @JsonProperty("autoColumnFormatSpec") @Nullable NestedCommonFormatColumnFormatSpec autoColumnFormatSpec ) { - this.bitmapSerdeFactory = bitmapSerdeFactory != null - ? bitmapSerdeFactory - : new BitmapSerde.DefaultBitmapSerdeFactory(); - this.dimensionCompression = dimensionCompression == null - ? CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY - : dimensionCompression; - this.stringDictionaryEncoding = stringDictionaryEncoding == null - ? StringEncodingStrategy.DEFAULT - : stringDictionaryEncoding; - - this.metricCompression = metricCompression == null - ? CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY - : metricCompression; + this.bitmapSerdeFactory = bitmapSerdeFactory; + this.dimensionCompression = dimensionCompression; + this.stringDictionaryEncoding = stringDictionaryEncoding; + this.metricCompression = metricCompression; this.complexMetricCompression = complexMetricCompression; - this.longEncoding = longEncoding == null - ? CompressionFactory.DEFAULT_LONG_ENCODING_STRATEGY - : longEncoding; + this.longEncoding = longEncoding; this.jsonCompression = jsonCompression; this.segmentLoader = segmentLoader; this.autoColumnFormatSpec = autoColumnFormatSpec; } @JsonProperty("bitmap") + @Nullable public BitmapSerdeFactory getBitmapSerdeFactory() { return bitmapSerdeFactory; } @JsonProperty + @Nullable public CompressionStrategy getDimensionCompression() { return dimensionCompression; } @JsonProperty + @Nullable public StringEncodingStrategy getStringDictionaryEncoding() { return stringDictionaryEncoding; } @JsonProperty + @Nullable public CompressionStrategy getMetricCompression() { return metricCompression; } @JsonProperty + @Nullable public CompressionFactory.LongEncodingStrategy getLongEncoding() { return longEncoding; @@ -180,6 +180,80 @@ public NestedCommonFormatColumnFormatSpec getAutoColumnFormatSpec() return autoColumnFormatSpec; } + @JsonIgnore + public IndexSpec getEffectiveSpec() + { + Builder bob = IndexSpec.builder(); + final IndexSpec defaultSpec = getDefault(); + + if (bitmapSerdeFactory != null) { + bob.withBitmapSerdeFactory(bitmapSerdeFactory); + } else if (defaultSpec.bitmapSerdeFactory != null) { + bob.withBitmapSerdeFactory(defaultSpec.bitmapSerdeFactory); + } else { + bob.withBitmapSerdeFactory(new BitmapSerde.DefaultBitmapSerdeFactory()); + } + + if (dimensionCompression != null) { + bob.withDimensionCompression(dimensionCompression); + } else if (defaultSpec.dimensionCompression != null) { + bob.withDimensionCompression(defaultSpec.dimensionCompression); + } else { + bob.withDimensionCompression(CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY); + } + + if (stringDictionaryEncoding != null) { + bob.withStringDictionaryEncoding(stringDictionaryEncoding); + } else if (defaultSpec.stringDictionaryEncoding != null) { + bob.withStringDictionaryEncoding(defaultSpec.stringDictionaryEncoding); + } else { + bob.withStringDictionaryEncoding(StringEncodingStrategy.DEFAULT); + } + + if (metricCompression != null) { + bob.withMetricCompression(metricCompression); + } else if (defaultSpec.metricCompression != null) { + bob.withMetricCompression(defaultSpec.metricCompression); + } else { + bob.withMetricCompression(CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY); + } + + if (longEncoding != null) { + bob.withLongEncoding(longEncoding); + } else if (defaultSpec.longEncoding != null) { + bob.withLongEncoding(defaultSpec.longEncoding); + } else { + bob.withLongEncoding(CompressionFactory.DEFAULT_LONG_ENCODING_STRATEGY); + } + + if (complexMetricCompression != null) { + bob.withComplexMetricCompression(complexMetricCompression); + } else if (defaultSpec.complexMetricCompression != null) { + bob.withComplexMetricCompression(defaultSpec.complexMetricCompression); + } + + if (jsonCompression != null) { + bob.withJsonCompression(jsonCompression); + } else if (defaultSpec.jsonCompression != null) { + bob.withJsonCompression(defaultSpec.jsonCompression); + } + + if (segmentLoader != null) { + bob.withSegmentLoader(segmentLoader); + } else if (defaultSpec.segmentLoader != null) { + bob.withSegmentLoader(defaultSpec.segmentLoader); + } + + if (autoColumnFormatSpec != null) { + bob.withAutoColumnFormatSpec(autoColumnFormatSpec.getEffectiveSpec(this)); + } else if (defaultSpec.autoColumnFormatSpec != null) { + bob.withAutoColumnFormatSpec(defaultSpec.autoColumnFormatSpec.getEffectiveSpec(this)); + } + + + return bob.build(); + } + @Override public boolean equals(Object o) { @@ -253,55 +327,56 @@ public static class Builder @Nullable private NestedCommonFormatColumnFormatSpec autoColumnFormatSpec; - public Builder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory) + public Builder withBitmapSerdeFactory(@Nullable BitmapSerdeFactory bitmapSerdeFactory) { this.bitmapSerdeFactory = bitmapSerdeFactory; return this; } - public Builder withDimensionCompression(CompressionStrategy dimensionCompression) + public Builder withDimensionCompression(@Nullable CompressionStrategy dimensionCompression) { this.dimensionCompression = dimensionCompression; return this; } - public Builder withStringDictionaryEncoding(StringEncodingStrategy stringDictionaryEncoding) + public Builder withStringDictionaryEncoding(@Nullable StringEncodingStrategy stringDictionaryEncoding) { this.stringDictionaryEncoding = stringDictionaryEncoding; return this; } - public Builder withMetricCompression(CompressionStrategy metricCompression) + public Builder withMetricCompression(@Nullable CompressionStrategy metricCompression) { this.metricCompression = metricCompression; return this; } - public Builder withComplexMetricCompression(CompressionStrategy complexMetricCompression) + public Builder withComplexMetricCompression(@Nullable CompressionStrategy complexMetricCompression) { this.complexMetricCompression = complexMetricCompression; return this; } - public Builder withLongEncoding(CompressionFactory.LongEncodingStrategy longEncoding) + public Builder withLongEncoding(@Nullable CompressionFactory.LongEncodingStrategy longEncoding) { this.longEncoding = longEncoding; return this; } - public Builder withJsonCompression(CompressionStrategy jsonCompression) + @Deprecated + public Builder withJsonCompression(@Nullable CompressionStrategy jsonCompression) { this.jsonCompression = jsonCompression; return this; } - public Builder withSegmentLoader(SegmentizerFactory segmentLoader) + public Builder withSegmentLoader(@Nullable SegmentizerFactory segmentLoader) { this.segmentLoader = segmentLoader; return this; } - public Builder withAutoColumnFormatSpec(NestedCommonFormatColumnFormatSpec autoColumnFormatSpec) + public Builder withAutoColumnFormatSpec(@Nullable NestedCommonFormatColumnFormatSpec autoColumnFormatSpec) { this.autoColumnFormatSpec = autoColumnFormatSpec; return this; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java index 781a7ee98d4a..826a157af040 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java @@ -45,6 +45,11 @@ public static Builder builder() return new Builder(); } + public static Builder builder(NestedCommonFormatColumnFormatSpec spec) + { + return new Builder(spec); + } + public static NestedCommonFormatColumnFormatSpec getEffectiveFormatSpec( @Nullable NestedCommonFormatColumnFormatSpec columnFormatSpec, IndexSpec indexSpec diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index 1330e39f1eac..f99d996a6b73 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -180,20 +180,23 @@ public static Function, Set> addCompactionStateToS @Nullable List projections ) { - DimensionsSpec effectiveDimensions = DimensionsSpec.builder(dimensionsSpec) - .setDimensions( - dimensionsSpec.getDimensions() - .stream() - .map(dim -> dim.getEffectiveSchema(indexSpec)) - .collect(Collectors.toList()) - ) - .build(); + IndexSpec effectiveIndexSpec = indexSpec.getEffectiveSpec(); + DimensionsSpec effectiveDimensions = + DimensionsSpec.builder(dimensionsSpec) + .setDimensions( + dimensionsSpec.getDimensions() + .stream() + .map(dim -> dim.getEffectiveSchema(effectiveIndexSpec)) + .collect(Collectors.toList()) + ) + .build(); + CompactionState compactionState = new CompactionState( partitionsSpec, effectiveDimensions, metricsSpec, transformSpec, - indexSpec, + effectiveIndexSpec, granularitySpec, projections ); diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java index 7b6fda4a18b4..eb48ac8616c4 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java @@ -70,7 +70,7 @@ public void setUp() { indexMerger = TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()); indexIO = TestHelper.getTestIndexIO(); - indexSpec = IndexSpec.getDefault(); + indexSpec = IndexSpec.getDefault().getEffectiveSpec(); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java b/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java index c4e83f58dbef..663f1b80a245 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java @@ -63,7 +63,7 @@ public void testSerdeUncompressed() throws Exception @Test public void testDefaults() { - final IndexSpec spec = IndexSpec.getDefault(); + final IndexSpec spec = IndexSpec.getDefault().getEffectiveSpec(); Assert.assertEquals(CompressionStrategy.LZ4, spec.getDimensionCompression()); Assert.assertEquals(CompressionStrategy.LZ4, spec.getMetricCompression()); Assert.assertEquals(LongEncodingStrategy.LONGS, spec.getLongEncoding()); @@ -73,13 +73,6 @@ public void testDefaults() public void testEquals() { EqualsVerifier.forClass(IndexSpec.class) - .withPrefabValues( - IndexSpec.class, - IndexSpec.getDefault(), - IndexSpec.builder() - .withJsonCompression(CompressionStrategy.ZSTD) - .build() - ) .usingGetClass() .verify(); } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index 81357e0cee1b..86ed8bd5b917 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -514,7 +514,6 @@ public static Collection makeConstructors() .put( "incrementalAutoTypes", input -> { - input.indexSpec(IndexSpec.builder().build()); input.mapSchema( schema -> new IncrementalIndexSchema( @@ -527,7 +526,8 @@ public static Collection makeConstructors() .getDimensions() .stream() .map( - dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName(), null, null) + dimensionSchema -> + AutoTypeColumnSchema.of(dimensionSchema.getName()) ) .collect(Collectors.toList()) ), @@ -543,7 +543,6 @@ public static Collection makeConstructors() .put( "mmappedAutoTypes", input -> { - input.indexSpec(IndexSpec.builder().build()); input.mapSchema( schema -> new IncrementalIndexSchema( @@ -556,7 +555,8 @@ public static Collection makeConstructors() .getDimensions() .stream() .map( - dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName(), null, null) + dimensionSchema -> + AutoTypeColumnSchema.of(dimensionSchema.getName()) ) .collect(Collectors.toList()) ), diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java index 6d6f886f3e68..f489ab11f00e 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpecTest.java @@ -61,7 +61,7 @@ public void testGetEffectiveSpecDefaults() { NestedCommonFormatColumnFormatSpec defaults = NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( null, - IndexSpec.getDefault() + IndexSpec.getDefault().getEffectiveSpec() ); Assert.assertEquals( @@ -77,19 +77,19 @@ public void testGetEffectiveSpecDefaults() defaults.getObjectStorageCompression() ); Assert.assertEquals( - IndexSpec.getDefault().getDimensionCompression(), + IndexSpec.getDefault().getEffectiveSpec().getDimensionCompression(), defaults.getDictionaryEncodedColumnCompression() ); Assert.assertEquals( - IndexSpec.getDefault().getStringDictionaryEncoding(), + IndexSpec.getDefault().getEffectiveSpec().getStringDictionaryEncoding(), defaults.getStringDictionaryEncoding() ); Assert.assertEquals( - IndexSpec.getDefault().getMetricCompression(), + IndexSpec.getDefault().getEffectiveSpec().getMetricCompression(), defaults.getLongColumnCompression() ); Assert.assertEquals( - IndexSpec.getDefault().getMetricCompression(), + IndexSpec.getDefault().getEffectiveSpec().getMetricCompression(), defaults.getDoubleColumnCompression() ); } @@ -108,7 +108,7 @@ public void testGetEffectiveSpecMerge() ) .setDoubleColumnCompression(CompressionStrategy.ZSTD) .build(), - IndexSpec.getDefault() + IndexSpec.getDefault().getEffectiveSpec() ); Assert.assertEquals( @@ -124,7 +124,7 @@ public void testGetEffectiveSpecMerge() merged.getObjectStorageEncoding() ); Assert.assertEquals( - IndexSpec.getDefault().getDimensionCompression(), + IndexSpec.getDefault().getEffectiveSpec().getDimensionCompression(), merged.getDictionaryEncodedColumnCompression() ); Assert.assertEquals( @@ -132,7 +132,7 @@ public void testGetEffectiveSpecMerge() merged.getObjectStorageCompression() ); Assert.assertEquals( - IndexSpec.getDefault().getMetricCompression(), + IndexSpec.getDefault().getEffectiveSpec().getMetricCompression(), merged.getLongColumnCompression() ); Assert.assertEquals( @@ -148,7 +148,7 @@ public void testGetEffectiveSpecInvalid() ISE.class, () -> NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( NestedCommonFormatColumnFormatSpec.builder().setBitmapEncoding(new ConciseBitmapSerdeFactory()).build(), - IndexSpec.builder().withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()).build() + IndexSpec.builder().withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()).build().getEffectiveSpec() ) ); 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 0e19122905fe..39109f5ed049 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 @@ -203,7 +203,7 @@ public NestedDataColumnSupplierTest( { this.columnFormatSpec = NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec( columnFormatSpec, - IndexSpec.getDefault() + IndexSpec.getDefault().getEffectiveSpec() ); this.bitmapSerdeFactory = this.columnFormatSpec.getBitmapEncoding(); this.resultFactory = new DefaultBitmapResultFactory(bitmapSerdeFactory.getBitmapFactory()); 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 index 87a2abfc77e9..756a27eae881 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java @@ -126,7 +126,7 @@ private SmooshedFileMapper smooshify( try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { ScalarDoubleColumnSerializer serializer = new ScalarDoubleColumnSerializer( fileNameBase, - NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault()), + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault().getEffectiveSpec()), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); 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 index 676c47d9f456..93d9bf99a65f 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java @@ -126,7 +126,7 @@ private SmooshedFileMapper smooshify( try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { ScalarLongColumnSerializer serializer = new ScalarLongColumnSerializer( fileNameBase, - NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault()), + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault().getEffectiveSpec()), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); 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 index 780a2c1b062a..b7886a98eeb5 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -126,7 +126,7 @@ private SmooshedFileMapper smooshify( try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) { ScalarStringColumnSerializer serializer = new ScalarStringColumnSerializer( fileNameBase, - NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault()), + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(null, IndexSpec.getDefault().getEffectiveSpec()), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java index 5f59eff5f5cf..d7316ae6d556 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java @@ -272,7 +272,7 @@ private SmooshedFileMapper smooshify( fileNameBase, expectedTypes.getSingleType() == null ? null : expectedLogicalType, expectedTypes.getSingleType() == null ? expectedTypes.getByteValue() : null, - NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(columnFormatSpec, IndexSpec.getDefault()), + NestedCommonFormatColumnFormatSpec.getEffectiveFormatSpec(columnFormatSpec, IndexSpec.getDefault().getEffectiveSpec()), writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()), closer ); diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index dad1c13cdeb5..1d71711831ab 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -344,8 +344,8 @@ private CompactionStatus indexSpecIsUpToDate() { return CompactionStatus.completeIfEqual( "indexSpec", - Configs.valueOrDefault(tuningConfig.getIndexSpec(), IndexSpec.getDefault()), - objectMapper.convertValue(lastCompactionState.getIndexSpec(), IndexSpec.class), + Configs.valueOrDefault(tuningConfig.getIndexSpec(), IndexSpec.getDefault()).getEffectiveSpec(), + objectMapper.convertValue(lastCompactionState.getIndexSpec(), IndexSpec.class).getEffectiveSpec(), String::valueOf ); } From 1e20017a9faa1f45439f69b0ac61b82bffc9b6c1 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 24 Sep 2025 11:15:38 -0700 Subject: [PATCH 16/24] javadoc --- .../data/input/impl/DimensionSchema.java | 6 ++++ .../org/apache/druid/segment/IndexSpec.java | 34 ++++++++++++++----- 2 files changed, 31 insertions(+), 9 deletions(-) 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 66e84f35c986..55a7872a2ce7 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 @@ -167,6 +167,12 @@ public DimensionHandler getDimensionHandler() ); } + /** + * Computes the 'effective' {@link DimensionSchema}, allowing columns which provide mechanisms for customizing storage + * format to fill in values from the segment level {@link IndexSpec} defaults. This is useful for comparising the + * operator explicitly defined schema with the 'effective' schema that was written to the segments for things like + * comparing compaction state. + */ public DimensionSchema getEffectiveSchema(IndexSpec indexSpec) { return this; diff --git a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java index 659e78c768be..428e08cd97f0 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java @@ -39,7 +39,10 @@ * IndexSpec defines segment storage format options to be used at indexing time, * such as bitmap type, and column compression formats. *

- * IndexSpec is specified as part of the TuningConfig for the corresponding index task. + * IndexSpec is specified as part of the TuningConfig for the corresponding index task. Fields which are not explicitly + * defined by the operator default to null values, callers must call {@link #getEffectiveSpec()} to fill in these null + * values, which will replace nulls first with any system defaults defined in {@link #getDefault()} falling back to + * hard coded defaults. */ public class IndexSpec { @@ -76,17 +79,26 @@ public static Builder builder() * Creates an IndexSpec with the given storage format settings. * * @param bitmapSerdeFactory type of bitmap to use (e.g. roaring or concise), null to use the default. - * Defaults to the bitmap type specified by the (deprecated) "druid.processing.bitmap.type" - * setting, or, if none was set, uses the default defined in {@link BitmapSerde} + * Defaults to the bitmap type specified by the (deprecated) + * "druid.processing.bitmap.type" setting, or, if none was set, uses the default + * defined in {@link BitmapSerde} upon calling {@link #getEffectiveSpec()} * @param dimensionCompression compression format for dimension columns, null to use the default. - * Defaults to {@link CompressionStrategy#DEFAULT_COMPRESSION_STRATEGY} + * Defaults to {@link CompressionStrategy#DEFAULT_COMPRESSION_STRATEGY} upon calling + * {@link #getEffectiveSpec()} * @param stringDictionaryEncoding encoding strategy for string dictionaries of dictionary encoded string columns * @param metricCompression compression format for primitive type metric columns, null to use the default. - * Defaults to {@link CompressionStrategy#DEFAULT_COMPRESSION_STRATEGY} - * @param longEncoding encoding strategy for metric and dimension columns with type long, null to use the default. - * Defaults to {@link CompressionFactory#DEFAULT_LONG_ENCODING_STRATEGY} - * @param segmentLoader specify a {@link SegmentizerFactory} which will be written to 'factory.json' and used to load - * the written segment + * Defaults to {@link CompressionStrategy#DEFAULT_COMPRESSION_STRATEGY} upon calling + * {@link #getEffectiveSpec()}. + * @param longEncoding encoding strategy for metric and dimension columns with type long, null to use the + * default. Defaults to {@link CompressionFactory#DEFAULT_LONG_ENCODING_STRATEGY} upon + * calling {@link #getEffectiveSpec()} + * @param complexMetricCompression default {@link CompressionStrategy} to use for complex type columns which use + * generic serializers. Defaults to null which means no compression upon calling + * {@link #getEffectiveSpec()}. + * @param segmentLoader specify a {@link SegmentizerFactory} which will be written to 'factory.json' and + * used to load the written segment + * @param autoColumnFormatSpec specify the default {@link NestedCommonFormatColumnFormatSpec} to use for json and + * auto columns. Defaults to null upon calling {@link #getEffectiveSpec()}. */ @JsonCreator public IndexSpec( @@ -180,6 +192,10 @@ public NestedCommonFormatColumnFormatSpec getAutoColumnFormatSpec() return autoColumnFormatSpec; } + /** + * Populate all null fields of {@link IndexSpec}, first from {@link #getDefault()} and finally falling back to hard + * coded defaults if no overrides are defined. + */ @JsonIgnore public IndexSpec getEffectiveSpec() { From 5eb46a11c64935be54b8e1697fa44e51695aad77 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 24 Sep 2025 11:18:39 -0700 Subject: [PATCH 17/24] missed one --- .../src/main/java/org/apache/druid/segment/IndexSpec.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java index 428e08cd97f0..2f97392d2f3a 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java @@ -95,6 +95,12 @@ public static Builder builder() * @param complexMetricCompression default {@link CompressionStrategy} to use for complex type columns which use * generic serializers. Defaults to null which means no compression upon calling * {@link #getEffectiveSpec()}. + * @param jsonCompression deprecated mechanism to define default json column 'raw' storage compression. This + * has been replaced by {@link #autoColumnFormatSpec} which has + * {@link NestedCommonFormatColumnFormatSpec#getObjectStorageCompression()}. If this + * is set and {@link #autoColumnFormatSpec} is null, this value will be used to set + * the object storage compression of the computed {@link #autoColumnFormatSpec} upon + * calling {@link #getEffectiveSpec()} * @param segmentLoader specify a {@link SegmentizerFactory} which will be written to 'factory.json' and * used to load the written segment * @param autoColumnFormatSpec specify the default {@link NestedCommonFormatColumnFormatSpec} to use for json and From 1940a7784c161b1d03c528cdafbfaff4d37eaaa5 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 24 Sep 2025 12:06:22 -0700 Subject: [PATCH 18/24] fix test --- .../apache/druid/server/compaction/CompactionStatus.java | 5 +++-- .../druid/server/compaction/CompactionStatusTest.java | 6 +++--- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index 1d71711831ab..e34a59185f91 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -240,10 +240,11 @@ private static List getNonPartitioningDimensions( @Nullable final IndexSpec indexSpec ) { + final IndexSpec effectiveIndexSpec = (indexSpec == null ? IndexSpec.getDefault() : indexSpec).getEffectiveSpec(); if (dimensionSchemas == null || !(partitionsSpec instanceof DimensionRangePartitionsSpec)) { if (dimensionSchemas != null) { return dimensionSchemas.stream() - .map(dim -> dim.getEffectiveSchema(indexSpec)) + .map(dim -> dim.getEffectiveSchema(effectiveIndexSpec)) .collect(Collectors.toList()); } return null; @@ -252,7 +253,7 @@ private static List getNonPartitioningDimensions( final List partitionsDimensions = ((DimensionRangePartitionsSpec) partitionsSpec).getPartitionDimensions(); return dimensionSchemas.stream() .filter(dim -> !partitionsDimensions.contains(dim.getName())) - .map(dim -> dim.getEffectiveSchema(indexSpec)) + .map(dim -> dim.getEffectiveSchema(effectiveIndexSpec)) .collect(Collectors.toList()); } diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index 99d1e9ac2d63..3a301d879a0a 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -447,14 +447,14 @@ public void testStatusWhenAutoSchemaMatch() DimensionsSpec.builder() .setDimensions( List.of( - AutoTypeColumnSchema.of("x").getEffectiveSchema(IndexSpec.getDefault()), - AutoTypeColumnSchema.of("y").getEffectiveSchema(IndexSpec.getDefault()) + AutoTypeColumnSchema.of("x").getEffectiveSchema(IndexSpec.getDefault().getEffectiveSpec()), + AutoTypeColumnSchema.of("y").getEffectiveSchema(IndexSpec.getDefault().getEffectiveSpec()) ) ) .build(), null, null, - IndexSpec.getDefault(), + IndexSpec.getDefault().getEffectiveSpec(), currentGranularitySpec, Collections.emptyList() ); From 28e582fb12b9e5db4d97fb0c4653ca491bb0c0ca Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 24 Sep 2025 12:08:40 -0700 Subject: [PATCH 19/24] fix more test --- .../RabbitStreamIndexTaskTuningConfigTest.java | 10 +++++----- .../java/org/apache/druid/msq/exec/MSQReplaceTest.java | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java index e7237c706747..6a70f7155237 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java @@ -155,11 +155,11 @@ public void testtoString() throws Exception "intermediatePersistPeriod=PT1H, " + "maxPendingPersists=100, " + "indexSpec=IndexSpec{" + - "bitmapSerdeFactory=RoaringBitmapSerdeFactory{}, " + - "dimensionCompression=lz4, " + - "stringDictionaryEncoding=Utf8{}, " + - "metricCompression=lz4, " + - "longEncoding=longs, " + + "bitmapSerdeFactory=null, " + + "dimensionCompression=null, " + + "stringDictionaryEncoding=null, " + + "metricCompression=null, " + + "longEncoding=null, " + "complexMetricCompression=null, " + "jsonCompression=null, " + "segmentLoader=null" + diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 0c7cfbea2ba3..8c88c88e18df 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -2815,7 +2815,7 @@ private CompactionState expectedCompactionState( ); } - IndexSpec indexSpec = IndexSpec.getDefault(); + IndexSpec indexSpec = IndexSpec.getDefault().getEffectiveSpec(); GranularitySpec granularitySpec = new UniformGranularitySpec( segmentGranularity.getDefaultGranularity(), GranularityType.NONE.getDefaultGranularity(), From 795bbc0a7660c84a908ff2073b56dccd3dd9fd38 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 24 Sep 2025 17:00:44 -0700 Subject: [PATCH 20/24] more javadoc --- .../segment/nested/NestedCommonFormatColumnFormatSpec.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java index 826a157af040..592e759b72ed 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnFormatSpec.java @@ -32,6 +32,12 @@ import javax.annotation.Nullable; import java.util.Objects; +/** + * Defines storage format for 'auto' and json columns. This can be convered into the 'effective' format spec by calling + * {@link #getEffectiveSpec(IndexSpec)}, which will fill in any values which were not specified from + * {@link IndexSpec#getAutoColumnFormatSpec()}, falling back to hard-coded defaults, useful when generating segments or + * comparing compaction state. + */ public class NestedCommonFormatColumnFormatSpec { private static final NestedCommonFormatColumnFormatSpec DEFAULT = From 1c739c602e053fc55892f1d3973e77349d6a1b44 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 24 Sep 2025 21:51:28 -0700 Subject: [PATCH 21/24] rename class --- .../org/apache/druid/segment/AutoTypeColumnIndexer.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java index 0f1ca2ba5273..7633e96c8b5c 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java @@ -477,7 +477,7 @@ public Object getConstantValue() @Override public ColumnFormat getFormat() { - return new Format(getLogicalType(), hasNulls, castToType != null, columnFormatSpec); + return new IndexerFormat(getLogicalType(), hasNulls, castToType != null, columnFormatSpec); } @Override @@ -757,10 +757,9 @@ public boolean isSingleType() } } - static class Format extends NestedCommonFormatColumn.Format + static class IndexerFormat extends NestedCommonFormatColumn.Format { - - public Format( + public IndexerFormat( ColumnType logicalType, boolean hasNulls, boolean enforceLogicalType, From 56e428c93ebd81e900bd63832ce3168dbbe9b651 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 29 Sep 2025 22:07:39 -0700 Subject: [PATCH 22/24] add log --- .../java/org/apache/druid/segment/IndexMergerV9.java | 11 +++++++---- .../main/java/org/apache/druid/segment/IndexSpec.java | 2 +- 2 files changed, 8 insertions(+), 5 deletions(-) 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 f4741e6059ad..768418d519f6 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java @@ -1052,6 +1052,8 @@ public File persist( throw new IAE("Trying to persist an empty index!"); } + // need to call this here even though multiphaseMerge also calls this because we need to ensure the bitmap factory + // is populated when creating the IncrementalIndexAdapter indexSpec = indexSpec.getEffectiveSpec(); final DateTime firstTimestamp = index.getMinTime(); @@ -1112,8 +1114,8 @@ public File mergeQueryableIndex( metricAggs, dimensionsSpec, outDir, - indexSpec.getEffectiveSpec(), - indexSpecForIntermediatePersists.getEffectiveSpec(), + indexSpec, + indexSpecForIntermediatePersists, progress, segmentWriteOutMediumFactory, maxColumnsToMerge @@ -1140,8 +1142,8 @@ public File merge( metricAggs, dimensionsSpec, outDir, - indexSpec.getEffectiveSpec(), - indexSpec.getEffectiveSpec(), + indexSpec, + indexSpec, new BaseProgressIndicator(), null, maxColumnsToMerge @@ -1168,6 +1170,7 @@ private File multiphaseMerge( indexSpec = indexSpec.getEffectiveSpec(); indexSpecForIntermediatePersists = indexSpecForIntermediatePersists.getEffectiveSpec(); + log.info("Building segment with IndexSpec[%s]", indexSpec); if (maxColumnsToMerge == IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE) { return merge( diff --git a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java index 2f97392d2f3a..9726a8137288 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java @@ -272,7 +272,6 @@ public IndexSpec getEffectiveSpec() bob.withAutoColumnFormatSpec(defaultSpec.autoColumnFormatSpec.getEffectiveSpec(this)); } - return bob.build(); } @@ -323,6 +322,7 @@ public String toString() ", metricCompression=" + metricCompression + ", longEncoding=" + longEncoding + ", complexMetricCompression=" + complexMetricCompression + + ", autoColumnFormatSpec=" + autoColumnFormatSpec + ", jsonCompression=" + jsonCompression + ", segmentLoader=" + segmentLoader + '}'; From 083beb2317dfa96d333a2eaac44bfa68d71d6132 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 30 Sep 2025 10:41:43 -0700 Subject: [PATCH 23/24] fix test --- .../apache/druid/server/compaction/CompactionStatusTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index 3a301d879a0a..e9d70f442581 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -257,11 +257,11 @@ public void testStatusOnIndexSpecMismatch() + "required[IndexSpec{bitmapSerdeFactory=RoaringBitmapSerdeFactory{}," + " dimensionCompression=lz4, stringDictionaryEncoding=Utf8{}," + " metricCompression=lz4, longEncoding=longs, complexMetricCompression=null," - + " jsonCompression=null, segmentLoader=null}], " + + " autoColumnFormatSpec=null, jsonCompression=null, segmentLoader=null}], " + "current[IndexSpec{bitmapSerdeFactory=RoaringBitmapSerdeFactory{}," + " dimensionCompression=zstd, stringDictionaryEncoding=Utf8{}," + " metricCompression=lz4, longEncoding=longs, complexMetricCompression=null," - + " jsonCompression=null, segmentLoader=null}]" + + " autoColumnFormatSpec=null, jsonCompression=null, segmentLoader=null}]" ); } From 2732e1c9716c7d0b3efbed6a8902a99437feb8a4 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 30 Sep 2025 11:59:11 -0700 Subject: [PATCH 24/24] missed another one --- .../rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java index 6a70f7155237..d513dee78d67 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java @@ -161,6 +161,7 @@ public void testtoString() throws Exception "metricCompression=null, " + "longEncoding=null, " + "complexMetricCompression=null, " + + "autoColumnFormatSpec=null, " + "jsonCompression=null, " + "segmentLoader=null" + "}, " +