From 02bdd390b92eaef82fbaa118b549fe27041a953d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 13 Jan 2023 17:34:26 -0800 Subject: [PATCH 01/15] discover nested columns when using nested column indexer for schemaless --- .../druid/data/input/InputRowSchema.java | 14 +- .../druid/data/input/impl/JsonLineReader.java | 6 +- .../druid/data/input/impl/JsonNodeReader.java | 6 +- .../druid/data/input/impl/JsonReader.java | 6 +- .../common/parsers/JSONFlattenerMaker.java | 5 +- .../util/common/parsers/JSONPathParser.java | 6 +- .../util/common/parsers/ObjectFlatteners.java | 9 +- .../parsers/JSONFlattenerMakerTest.java | 29 ++++ .../common/parsers/ObjectFlattenersTest.java | 3 +- .../data/input/avro/AvroFlattenerMaker.java | 5 +- .../druid/data/input/avro/AvroOCFReader.java | 6 +- .../druid/data/input/avro/AvroParsers.java | 6 +- .../data/input/avro/AvroStreamReader.java | 6 +- .../input/avro/AvroFlattenerMakerTest.java | 53 +++++++ .../input/kafkainput/KafkaInputFormat.java | 8 +- .../input/orc/OrcHadoopInputRowParser.java | 6 +- .../druid/data/input/orc/OrcReader.java | 6 +- .../input/orc/OrcStructFlattenerMaker.java | 5 +- .../druid/data/input/orc/OrcReaderTest.java | 107 ++++++++++++++ .../data/input/parquet/ParquetReader.java | 6 +- .../avro/ParquetAvroHadoopInputRowParser.java | 3 +- .../simple/ParquetGroupFlattenerMaker.java | 5 +- .../simple/ParquetHadoopInputRowParser.java | 6 +- .../NestedColumnParquetReaderTest.java | 79 +++++++++++ .../protobuf/ProtobufFlattenerMaker.java | 6 +- .../protobuf/ProtobufInputRowSchema.java | 14 +- .../data/input/protobuf/ProtobufReader.java | 6 +- .../protobuf/ProtobufInputFormatTest.java | 72 ++++++++++ .../common/task/AbstractBatchIndexTask.java | 3 +- .../druid/indexing/common/task/IndexTask.java | 2 + .../common/task/InputSourceProcessor.java | 3 + .../PartialDimensionCardinalityTask.java | 1 + .../PartialDimensionDistributionTask.java | 1 + .../parallel/PartialSegmentGenerateTask.java | 1 + .../batch/parallel/SinglePhaseSubTask.java | 1 + .../druid/indexing/input/InputRowSchemas.java | 7 +- .../sampler/IndexTaskSamplerSpec.java | 6 +- .../overlord/sampler/InputSourceSampler.java | 8 +- .../SeekableStreamIndexTaskRunner.java | 2 +- .../SeekableStreamSamplerSpec.java | 2 +- .../indexing/input/DruidInputSourceTest.java | 6 +- .../indexing/input/InputRowSchemasTest.java | 4 +- .../sampler/CsvInputSourceSamplerTest.java | 1 + .../sampler/IndexTaskSamplerSpecTest.java | 5 +- .../sampler/InputSourceSamplerTest.java | 41 +++--- .../cases/cluster/Common/dependencies.yaml | 2 +- .../indexer/AbstractITBatchIndexTest.java | 41 +++++- ...ractLocalInputSourceParallelIndexTest.java | 35 ++++- ...calInputSourceAllFormatSchemalessTest.java | 133 ++++++++++++++++++ .../ITLocalInputSourceAllInputFormatTest.java | 1 - .../indexer/ITOverwriteBatchIndexTest.java | 1 + .../leadership/ITHighAvailabilityTest.java | 2 +- .../indexer/wikipedia_index_queries.json | 2 +- .../wikipedia_index_schemaless_queries.json | 32 +++++ ...al_input_source_index_task_schemaless.json | 45 ++++++ 55 files changed, 806 insertions(+), 71 deletions(-) create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllFormatSchemalessTest.java create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task_schemaless.json diff --git a/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java b/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java index 3c4263ba9993..8bdb37f38c37 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java +++ b/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java @@ -34,6 +34,9 @@ public class InputRowSchema private final TimestampSpec timestampSpec; private final DimensionsSpec dimensionsSpec; private final ColumnsFilter columnsFilter; + + private final boolean discoverNestedColumns; + /** * Set of metric names for further downstream processing by {@link InputSource}. * Empty set if no metric given. @@ -47,20 +50,22 @@ public InputRowSchema( final ColumnsFilter columnsFilter ) { - this(timestampSpec, dimensionsSpec, columnsFilter, ImmutableSet.of()); + this(timestampSpec, dimensionsSpec, columnsFilter, ImmutableSet.of(), false); } public InputRowSchema( final TimestampSpec timestampSpec, final DimensionsSpec dimensionsSpec, final ColumnsFilter columnsFilter, - final Set metricNames + final Set metricNames, + boolean discoverNestedColumns ) { this.timestampSpec = timestampSpec; this.dimensionsSpec = dimensionsSpec; this.columnsFilter = columnsFilter; this.metricNames = metricNames == null ? ImmutableSet.of() : metricNames; + this.discoverNestedColumns = discoverNestedColumns; } @NotNull @@ -92,4 +97,9 @@ public ColumnsFilter getColumnsFilter() { return columnsFilter; } + + public boolean shouldDiscoverNestedColumns() + { + return discoverNestedColumns; + } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java index 19c0d685a97b..7f34197bc10a 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java @@ -59,7 +59,11 @@ public class JsonLineReader extends TextReader ) { super(inputRowSchema, source); - this.flattener = ObjectFlatteners.create(flattenSpec, new JSONFlattenerMaker(keepNullColumns)); + this.flattener = ObjectFlatteners.create( + flattenSpec, + new JSONFlattenerMaker(keepNullColumns), + inputRowSchema.shouldDiscoverNestedColumns() + ); this.mapper = mapper; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java index 7a7b98c4529a..62efc05d5fdf 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java @@ -80,7 +80,11 @@ public class JsonNodeReader extends IntermediateRowParsingReader { this.inputRowSchema = inputRowSchema; this.source = source; - this.flattener = ObjectFlatteners.create(flattenSpec, new JSONFlattenerMaker(keepNullColumns)); + this.flattener = ObjectFlatteners.create( + flattenSpec, + new JSONFlattenerMaker(keepNullColumns), + inputRowSchema.shouldDiscoverNestedColumns() + ); this.mapper = mapper; this.jsonFactory = new JsonFactory(); } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java index 8dee12dc30f6..09b12870cb4f 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java @@ -77,7 +77,11 @@ public class JsonReader extends IntermediateRowParsingReader { this.inputRowSchema = inputRowSchema; this.source = source; - this.flattener = ObjectFlatteners.create(flattenSpec, new JSONFlattenerMaker(keepNullColumns)); + this.flattener = ObjectFlatteners.create( + flattenSpec, + new JSONFlattenerMaker(keepNullColumns), + inputRowSchema.shouldDiscoverNestedColumns() + ); this.mapper = mapper; this.jsonFactory = new JsonFactory(); } diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java index b54b51f86348..cc1843aab121 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java @@ -64,8 +64,11 @@ public JSONFlattenerMaker(boolean keepNullValues) } @Override - public Iterable discoverRootFields(final JsonNode obj) + public Iterable discoverRootFields(final JsonNode obj, boolean discoverNestedFields) { + if (discoverNestedFields) { + return obj::fieldNames; + } return FluentIterable.from(obj::fields) .filter( entry -> { diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONPathParser.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONPathParser.java index 48777a8311a1..268319ce446b 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONPathParser.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONPathParser.java @@ -42,7 +42,11 @@ public class JSONPathParser implements Parser public JSONPathParser(JSONPathSpec flattenSpec, ObjectMapper mapper, boolean keepNullColumns) { this.mapper = mapper == null ? new ObjectMapper() : mapper; - this.flattener = ObjectFlatteners.create(flattenSpec, new JSONFlattenerMaker(keepNullColumns)); + this.flattener = ObjectFlatteners.create( + flattenSpec, + new JSONFlattenerMaker(keepNullColumns), + false + ); } @Override diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java index 98f087333a55..9887090bf1f2 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java @@ -48,7 +48,8 @@ private ObjectFlatteners() public static ObjectFlattener create( @Nullable final JSONPathSpec flattenSpecInput, - final FlattenerMaker flattenerMaker + final FlattenerMaker flattenerMaker, + boolean discoverNestedColumns ) { final Map> extractors = new LinkedHashMap<>(); @@ -153,7 +154,7 @@ public void clear() public Set keySet() { if (flattenSpec.isUseFieldDiscovery()) { - final Iterable rootFields = flattenerMaker.discoverRootFields(obj); + final Iterable rootFields = flattenerMaker.discoverRootFields(obj, discoverNestedColumns); if (extractors.isEmpty() && rootFields instanceof Set) { return (Set) rootFields; } else { @@ -218,9 +219,9 @@ public interface FlattenerMaker { JsonProvider getJsonProvider(); /** - * List all "root" primitive properties and primitive lists (no nested objects, no lists of objects) + * List all "root" fields, optionally filtering to include only fields that contain primitive and lists of primitive values */ - Iterable discoverRootFields(T obj); + Iterable discoverRootFields(T obj, boolean discoverNestedFields); /** * Get a top level field from a "json" object diff --git a/core/src/test/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMakerTest.java b/core/src/test/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMakerTest.java index 0a1b09578bed..8fa82a61e508 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMakerTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMakerTest.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.node.BinaryNode; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import org.apache.druid.java.util.common.StringUtils; import org.junit.Assert; import org.junit.Test; @@ -169,4 +170,32 @@ public void testNested() throws JsonProcessingException result = FLATTENER_MAKER.finalizeConversionForMap(node); Assert.assertEquals(expectedList, result); } + + @Test + public void testDiscovery() throws JsonProcessingException + { + Map theMap = + ImmutableMap.builder() + .put("bool", true) + .put("int", 1) + .put("long", 1L) + .put("float", 0.11f) + .put("double", 0.33) + .put("binary", new byte[]{0x01, 0x02, 0x03}) + .put("list", ImmutableList.of("foo", "bar", "baz")) + .put("anotherList", ImmutableList.of(1, 2, 3)) + .put("nested", ImmutableMap.of("x", 1L, "y", 2L, "z", 3L)) + .build(); + + JsonNode node = OBJECT_MAPPER.readTree(OBJECT_MAPPER.writeValueAsString(theMap)); + Assert.assertTrue(node.isObject()); + Assert.assertEquals( + ImmutableSet.of("bool", "int", "long", "float", "double", "binary", "list", "anotherList"), + ImmutableSet.copyOf(FLATTENER_MAKER.discoverRootFields(node, false)) + ); + Assert.assertEquals( + ImmutableSet.of("bool", "int", "long", "float", "double", "binary", "list", "anotherList", "nested"), + ImmutableSet.copyOf(FLATTENER_MAKER.discoverRootFields(node, true)) + ); + } } diff --git a/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java b/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java index e0b0fbcc510c..1ab5dc85d707 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java @@ -40,7 +40,8 @@ public class ObjectFlattenersTest true, ImmutableList.of(new JSONPathFieldSpec(JSONPathFieldType.PATH, "extract", "$.bar")) ), - FLATTENER_MAKER + FLATTENER_MAKER, + true ); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java index 4e254c18eef2..ba2d94ccb85f 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java @@ -111,8 +111,11 @@ public AvroFlattenerMaker(final boolean fromPigAvroStorage, final boolean binary } @Override - public Set discoverRootFields(final GenericRecord obj) + public Set discoverRootFields(final GenericRecord obj, boolean discoverNestedFields) { + if (discoverNestedFields) { + return obj.getSchema().getFields().stream().map(Schema.Field::name).collect(Collectors.toSet()); + } return obj.getSchema() .getFields() .stream() diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java index 66552f9eda05..e8ef9c28b4d0 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java @@ -64,7 +64,11 @@ public class AvroOCFReader extends IntermediateRowParsingReader this.source = source; this.temporaryDirectory = temporaryDirectory; this.readerSchema = readerSchema; - this.recordFlattener = ObjectFlatteners.create(flattenSpec, new AvroFlattenerMaker(false, binaryAsString, extractUnionsByType)); + this.recordFlattener = ObjectFlatteners.create( + flattenSpec, + new AvroFlattenerMaker(false, binaryAsString, extractUnionsByType), + inputRowSchema.shouldDiscoverNestedColumns() + ); } @Override diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java index 6d399a8dc679..d83811c4b764 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java @@ -50,7 +50,11 @@ public static ObjectFlattener makeFlattener( flattenSpec = JSONPathSpec.DEFAULT; } - return ObjectFlatteners.create(flattenSpec, new AvroFlattenerMaker(fromPigAvroStorage, binaryAsString, extractUnionsByType)); + return ObjectFlatteners.create( + flattenSpec, + new AvroFlattenerMaker(fromPigAvroStorage, binaryAsString, extractUnionsByType), + false + ); } public static List parseGenericRecord( diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java index 4ed6a8a2ab26..974505393821 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java @@ -60,7 +60,11 @@ public class AvroStreamReader extends IntermediateRowParsingReader settableByteEntitySource = (SettableByteEntity) source; - InputRowSchema newInputRowSchema = new InputRowSchema(dummyTimestampSpec, inputRowSchema.getDimensionsSpec(), inputRowSchema.getColumnsFilter()); + InputRowSchema newInputRowSchema = new InputRowSchema( + dummyTimestampSpec, + inputRowSchema.getDimensionsSpec(), + inputRowSchema.getColumnsFilter(), + inputRowSchema.getMetricNames(), + inputRowSchema.shouldDiscoverNestedColumns() + ); return new KafkaInputReader( inputRowSchema, settableByteEntitySource, diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java index 9fcefcba2a60..66ff658d58a3 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java @@ -55,7 +55,11 @@ public OrcHadoopInputRowParser( } else { flattenSpec = JSONPathSpec.DEFAULT; } - this.orcStructFlattener = ObjectFlatteners.create(flattenSpec, new OrcStructFlattenerMaker(this.binaryAsString)); + this.orcStructFlattener = ObjectFlatteners.create( + flattenSpec, + new OrcStructFlattenerMaker(this.binaryAsString), + false + ); this.parser = new MapInputRowParser(parseSpec); } diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcReader.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcReader.java index 5bcec8c80eb1..3d92223b0aec 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcReader.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcReader.java @@ -70,7 +70,11 @@ public class OrcReader extends IntermediateRowParsingReader this.inputRowSchema = inputRowSchema; this.source = source; this.temporaryDirectory = temporaryDirectory; - this.orcStructFlattener = ObjectFlatteners.create(flattenSpec, new OrcStructFlattenerMaker(binaryAsString)); + this.orcStructFlattener = ObjectFlatteners.create( + flattenSpec, + new OrcStructFlattenerMaker(binaryAsString), + inputRowSchema.shouldDiscoverNestedColumns() + ); } @Override diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java index ee770d3e8e20..00ddae5f7799 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java @@ -55,8 +55,11 @@ public class OrcStructFlattenerMaker implements ObjectFlatteners.FlattenerMaker< } @Override - public Iterable discoverRootFields(OrcStruct obj) + public Iterable discoverRootFields(OrcStruct obj, boolean discoverNestedFields) { + if (discoverNestedFields) { + return obj.getSchema().getFieldNames(); + } List fields = obj.getSchema().getFieldNames(); List children = obj.getSchema().getChildren(); List primitiveFields = new ArrayList<>(); 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 03083f61ebc8..ebdc28516ed8 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 @@ -52,6 +52,7 @@ import java.io.File; import java.io.IOException; import java.util.Collections; +import java.util.List; public class OrcReaderTest extends InitializedNullHandlingTest { @@ -421,6 +422,112 @@ public void testNestedColumn() throws IOException } } + @Test + public void testNestedColumnSchemaless() throws IOException + { + final OrcInputFormat inputFormat = new OrcInputFormat( + new JSONPathSpec(true, ImmutableList.of()), + null, + new Configuration() + ); + final InputRowSchema schema = new InputRowSchema( + new TimestampSpec("ts", "millis", null), + new DimensionsSpec( + ImmutableList.of() + ), + ColumnsFilter.all(), + null, + true + ); + final FileEntity entity = new FileEntity(new File("example/orc-file-11-format.orc")); + + final InputEntityReader reader = inputFormat.createReader(schema, entity, temporaryFolder.newFolder()); + + List dims = ImmutableList.of( + "boolean1", + "byte1", + "short1", + "int1", + "long1", + "float1", + "double1", + "bytes1", + "string1", + "middle", + "list", + "map", + "ts", + "decimal1" + ); + try (CloseableIterator iterator = reader.read()) { + int actualRowCount = 0; + + // Check the first row + Assert.assertTrue(iterator.hasNext()); + InputRow row = iterator.next(); + + Assert.assertEquals(dims, row.getDimensions()); + actualRowCount++; + Assert.assertEquals( + ImmutableMap.of( + "list", + ImmutableList.of( + ImmutableMap.of("int1", 1, "string1", "bye"), + ImmutableMap.of("int1", 2, "string1", "sigh") + ) + ), + row.getRaw("middle") + ); + Assert.assertEquals( + ImmutableList.of( + ImmutableMap.of("int1", 3, "string1", "good"), + ImmutableMap.of("int1", 4, "string1", "bad") + ), + row.getRaw("list") + ); + Assert.assertEquals( + ImmutableMap.of(), + row.getRaw("map") + ); + Assert.assertEquals(DateTimes.of("2000-03-12T15:00:00.0Z"), row.getTimestamp()); + + while (iterator.hasNext()) { + actualRowCount++; + row = iterator.next(); + Assert.assertEquals(dims, row.getDimensions()); + } + + // Check the last row + Assert.assertEquals( + ImmutableMap.of( + "list", + ImmutableList.of( + ImmutableMap.of("int1", 1, "string1", "bye"), + ImmutableMap.of("int1", 2, "string1", "sigh") + ) + ), + row.getRaw("middle") + ); + Assert.assertEquals( + ImmutableList.of( + ImmutableMap.of("int1", 100000000, "string1", "cat"), + ImmutableMap.of("int1", -100000, "string1", "in"), + ImmutableMap.of("int1", 1234, "string1", "hat") + ), + row.getRaw("list") + ); + Assert.assertEquals( + ImmutableMap.of( + "chani", ImmutableMap.of("int1", 5, "string1", "chani"), + "mauddib", ImmutableMap.of("int1", 1, "string1", "mauddib") + ), + row.getRaw("map") + ); + + Assert.assertEquals(7500, actualRowCount); + } + } + @Test public void testListMap() throws IOException { diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java index aced1106eefa..4a2d53544e8e 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java @@ -66,7 +66,11 @@ public class ParquetReader extends IntermediateRowParsingReader this.inputRowSchema = inputRowSchema; this.source = source; this.temporaryDirectory = temporaryDirectory; - this.flattener = ObjectFlatteners.create(flattenSpec, new ParquetGroupFlattenerMaker(binaryAsString)); + this.flattener = ObjectFlatteners.create( + flattenSpec, + new ParquetGroupFlattenerMaker(binaryAsString), + inputRowSchema.shouldDiscoverNestedColumns() + ); } @Override diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java index 796891ea8ef8..6327cc5bd196 100755 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java @@ -78,7 +78,8 @@ public ParquetAvroHadoopInputRowParser( this.recordFlattener = ObjectFlatteners.create( flattenSpec, - new AvroFlattenerMaker(false, this.binaryAsString, this.extractUnionsByType) + new AvroFlattenerMaker(false, this.binaryAsString, this.extractUnionsByType), + false ); } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java index 772505546271..42f14a5d7f2e 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java @@ -53,8 +53,11 @@ public ParquetGroupFlattenerMaker(boolean binaryAsString) } @Override - public Set discoverRootFields(Group obj) + public Set discoverRootFields(Group obj, boolean discoverNestedFields) { + if (discoverNestedFields) { + return obj.getType().getFields().stream().map(Type::getName).collect(Collectors.toSet()); + } return obj.getType() .getFields() .stream() diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java index 762bb709d1c5..78e0f9633ef9 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java @@ -54,7 +54,11 @@ public ParquetHadoopInputRowParser( } else { flattenSpec = JSONPathSpec.DEFAULT; } - this.groupFlattener = ObjectFlatteners.create(flattenSpec, new ParquetGroupFlattenerMaker(this.binaryAsString)); + this.groupFlattener = ObjectFlatteners.create( + flattenSpec, + new ParquetGroupFlattenerMaker(this.binaryAsString), + false + ); this.parser = new MapInputRowParser(parseSpec); } 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 387ff2cf9194..56b24e9bf89a 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 @@ -162,4 +162,83 @@ public void testNestedColumnTransformsNestedNullableListFile() throws IOExceptio Assert.assertEquals(1L, rows.get(0).getRaw("t_a2_1_b1")); Assert.assertEquals(1L, rows.get(0).getRaw("tt_a2_0_b1")); } + + @Test + public void testNestedColumnSchemalessNestedTestFileNoNested() throws IOException + { + final String file = "example/flattening/test_nested_1.parquet"; + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec( + ImmutableList.of() + ), + ColumnsFilter.all(), + null, + false + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, ImmutableList.of()); + InputEntityReader reader = createReader( + file, + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals(ImmutableList.of("dim1", "metric1", "timestamp"), rows.get(0).getDimensions()); + Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); + Assert.assertEquals(ImmutableList.of("d1v1"), rows.get(0).getDimension("dim1")); + Assert.assertEquals("d1v1", rows.get(0).getRaw("dim1")); + Assert.assertEquals(ImmutableList.of("1"), rows.get(0).getDimension("metric1")); + Assert.assertEquals(1, rows.get(0).getRaw("metric1")); + Assert.assertEquals(1, rows.get(0).getMetric("metric1")); + // can still read even if it doesn't get reported as a dimension + Assert.assertEquals( + ImmutableMap.of( + "listDim", ImmutableList.of("listDim1v1", "listDim1v2"), + "dim3", 1, + "dim2", "d2v1", + "metric2", 2 + ), + rows.get(0).getRaw("nestedData") + ); + } + + @Test + public void testNestedColumnSchemalessNestedTestFile() throws IOException + { + final String file = "example/flattening/test_nested_1.parquet"; + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec( + ImmutableList.of() + ), + ColumnsFilter.all(), + null, + true + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, ImmutableList.of()); + InputEntityReader reader = createReader( + file, + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals(ImmutableList.of("nestedData", "dim1", "metric1", "timestamp"), rows.get(0).getDimensions()); + Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); + Assert.assertEquals(ImmutableList.of("d1v1"), rows.get(0).getDimension("dim1")); + Assert.assertEquals("d1v1", rows.get(0).getRaw("dim1")); + Assert.assertEquals(ImmutableList.of("1"), rows.get(0).getDimension("metric1")); + Assert.assertEquals(1, rows.get(0).getRaw("metric1")); + Assert.assertEquals(1, rows.get(0).getMetric("metric1")); + Assert.assertEquals( + ImmutableMap.of( + "listDim", ImmutableList.of("listDim1v1", "listDim1v2"), + "dim3", 1, + "dim2", "d2v1", + "metric2", 2 + ), + rows.get(0).getRaw("nestedData") + ); + } } diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java index a3314e42731d..9ccfe10db4c7 100644 --- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java +++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java @@ -63,9 +63,11 @@ public JsonProvider getJsonProvider() } @Override - public Iterable discoverRootFields(Map obj) + public Iterable discoverRootFields(Map obj, boolean discoverNestedFields) { - // in the future we can just return obj.keySet(), but for now this doesnt expect nested fields... + if (discoverNestedFields) { + return obj.keySet(); + } Set rootFields = Sets.newHashSetWithExpectedSize(obj.keySet().size()); for (Map.Entry entry : obj.entrySet()) { if (entry.getValue() instanceof List || entry.getValue() instanceof Map) { diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java index 9bb50a247b3e..cf604dab3c45 100644 --- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java +++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java @@ -40,7 +40,13 @@ public class ProtobufInputRowSchema extends InputRowSchema { public ProtobufInputRowSchema(InputRowSchema inputRowSchema) { - super(new ProtobufTimestampSpec(inputRowSchema.getTimestampSpec()), inputRowSchema.getDimensionsSpec(), inputRowSchema.getColumnsFilter()); + super( + new ProtobufTimestampSpec(inputRowSchema.getTimestampSpec()), + inputRowSchema.getDimensionsSpec(), + inputRowSchema.getColumnsFilter(), + inputRowSchema.getMetricNames(), + inputRowSchema.shouldDiscoverNestedColumns() + ); } static class ProtobufTimestampSpec extends TimestampSpec @@ -51,9 +57,9 @@ public ProtobufTimestampSpec(TimestampSpec timestampSpec) } /** - * Extracts the timestamp from the record. If the timestamp column is of complex type such as {@link Timestamp}, then the timestamp - * is first serialized to string via {@link JsonFormat}. Directly calling {@code toString()} on {@code Timestamp} - * returns an unparseable string. + * Extracts the timestamp from the record. If the timestamp column is of complex type such as {@link Timestamp}, + * then the timestamp is first serialized to string via {@link JsonFormat}. Directly calling {@code toString()} + * on {@code Timestamp} returns an unparseable string. */ @Override @Nullable diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java index 2dc6aa1f8caa..20c809017fb9 100644 --- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java +++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java @@ -57,7 +57,11 @@ public class ProtobufReader extends IntermediateRowParsingReader ) { this.inputRowSchema = inputRowSchema; - this.recordFlattener = ObjectFlatteners.create(flattenSpec, new ProtobufFlattenerMaker()); + this.recordFlattener = ObjectFlatteners.create( + flattenSpec, + new ProtobufFlattenerMaker(), + inputRowSchema.shouldDiscoverNestedColumns() + ); this.source = source; this.protobufBytesDecoder = protobufBytesDecoder; } 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 0b5d096449c0..7c273031db99 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 @@ -328,6 +328,78 @@ public void testParseNestedData() throws Exception } + @Test + public void testParseNestedDataSchemaless() throws Exception + { + ProtobufInputFormat protobufInputFormat = new ProtobufInputFormat( + JSONPathSpec.DEFAULT, + decoder + ); + + //create binary of proto test event + DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); + ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildNestedData(dateTime); + + final ByteEntity entity = new ByteEntity(ProtobufInputRowParserTest.toByteBuffer(event)); + + InputEntityReader reader = protobufInputFormat.createReader( + new InputRowSchema( + timestampSpec, + new DimensionsSpec(Collections.emptyList()), + null, + null, + true + ), + entity, + null + ); + + TransformSpec transformSpec = new TransformSpec( + null, + Lists.newArrayList( + new ExpressionTransform("foobar", "JSON_VALUE(foo, '$.bar')", TestExprMacroTable.INSTANCE), + new ExpressionTransform("bar0", "JSON_VALUE(bar, '$[0].bar')", TestExprMacroTable.INSTANCE) + ) + ); + TransformingInputEntityReader transformingReader = new TransformingInputEntityReader( + reader, + transformSpec.toTransformer() + ); + + + InputRow row = transformingReader.read().next(); + + Assert.assertEquals( + ImmutableList.of( + "someOtherId", + "bar", + "someIntColumn", + "isValid", + "foo", + "description", + "someLongColumn", + "someFloatColumn", + "eventType", + "id", + "someBytesColumn", + "timestamp" + ), + row.getDimensions() + ); + + Assert.assertEquals(ImmutableMap.of("bar", "baz"), row.getRaw("foo")); + Assert.assertEquals( + ImmutableList.of(ImmutableMap.of("bar", "bar0"), ImmutableMap.of("bar", "bar1")), + row.getRaw("bar") + ); + Assert.assertArrayEquals( + new byte[]{0x01, 0x02, 0x03, 0x04}, + (byte[]) row.getRaw("someBytesColumn") + ); + ProtobufInputRowParserTest.verifyNestedData(row, dateTime); + + } + @Test public void testParseNestedDataTransformsOnly() throws Exception { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 6e6e12f2ad87..d1dd277e571d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -202,6 +202,7 @@ public void stopGracefully(TaskConfig taskConfig) public static FilteringCloseableInputRowIterator inputSourceReader( File tmpDir, DataSchema dataSchema, + TuningConfig tuningConfig, InputSource inputSource, @Nullable InputFormat inputFormat, Predicate rowFilter, @@ -211,7 +212,7 @@ public static FilteringCloseableInputRowIterator inputSourceReader( { final InputSourceReader inputSourceReader = dataSchema.getTransformSpec().decorate( inputSource.reader( - InputRowSchemas.fromDataSchema(dataSchema), + InputRowSchemas.fromDataSchema(dataSchema, tuningConfig), inputFormat, tmpDir ) 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 288107d0b944..b490462da358 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 @@ -767,6 +767,7 @@ private Map> collectIntervalsAndShardSp try (final CloseableIterator inputRowIterator = AbstractBatchIndexTask.inputSourceReader( tmpDir, ingestionSchema.getDataSchema(), + ingestionSchema.getTuningConfig(), inputSource, inputSource.needsFormat() ? getInputFormat(ingestionSchema) : null, rowFilter, @@ -921,6 +922,7 @@ private TaskStatus generateAndPublishSegments( SegmentsAndCommitMetadata pushed = InputSourceProcessor.process( dataSchema, + tuningConfig, driver, partitionsSpec, inputSource, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java index e91f5d5fb0d1..b884487c78ec 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java @@ -34,6 +34,7 @@ import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; @@ -60,6 +61,7 @@ public class InputSourceProcessor */ public static SegmentsAndCommitMetadata process( DataSchema dataSchema, + TuningConfig tuningConfig, BatchAppenderatorDriver driver, PartitionsSpec partitionsSpec, InputSource inputSource, @@ -82,6 +84,7 @@ public static SegmentsAndCommitMetadata process( final CloseableIterator inputRowIterator = AbstractBatchIndexTask.inputSourceReader( tmpDir, dataSchema, + tuningConfig, inputSource, inputFormat, AbstractBatchIndexTask.defaultRowFilter(granularitySpec), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java index 0b4def915ac0..eabf738c9353 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java @@ -175,6 +175,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception final CloseableIterator inputRowIterator = AbstractBatchIndexTask.inputSourceReader( toolbox.getIndexingTmpDir(), dataSchema, + tuningConfig, inputSource, inputFormat, determineIntervals ? Objects::nonNull : AbstractBatchIndexTask.defaultRowFilter(granularitySpec), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java index e491e0244250..d63d00685c6e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java @@ -222,6 +222,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception final CloseableIterator inputRowIterator = AbstractBatchIndexTask.inputSourceReader( toolbox.getIndexingTmpDir(), dataSchema, + tuningConfig, inputSource, inputFormat, determineIntervals ? Objects::nonNull : AbstractBatchIndexTask.defaultRowFilter(granularitySpec), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index 03a66d1fd972..96db7670ebb0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -207,6 +207,7 @@ private List generateSegments( final SegmentsAndCommitMetadata pushed = InputSourceProcessor.process( dataSchema, + tuningConfig, driver, partitionsSpec, inputSource, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 1b83562d7c7c..89c306db68fc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -412,6 +412,7 @@ private Set generateAndPushSegments( final CloseableIterator inputRowIterator = AbstractBatchIndexTask.inputSourceReader( tmpDir, dataSchema, + tuningConfig, inputSource, inputSource.needsFormat() ? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema) : null, inputRow -> { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java index c895eb14b711..86cd880715b0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java @@ -26,9 +26,11 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.transform.Transform; import org.apache.druid.segment.transform.TransformSpec; +import javax.annotation.Nullable; import java.util.Arrays; import java.util.HashSet; import java.util.Set; @@ -47,7 +49,7 @@ private InputRowSchemas() /** * Creates an {@link InputRowSchema} from a given {@link DataSchema}. */ - public static InputRowSchema fromDataSchema(final DataSchema dataSchema) + public static InputRowSchema fromDataSchema(final DataSchema dataSchema, @Nullable final TuningConfig tuningConfig) { return new InputRowSchema( dataSchema.getTimestampSpec(), @@ -60,7 +62,8 @@ public static InputRowSchema fromDataSchema(final DataSchema dataSchema) ), Arrays.stream(dataSchema.getAggregators()) .map(AggregatorFactory::getName) - .collect(Collectors.toSet()) + .collect(Collectors.toSet()), + tuningConfig != null ? tuningConfig.getAppendableIndexSpec().useNestedColumnIndexerForSchemaDiscovery() : false ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java index 4140e014ef7a..6662fc6e9ddd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.TuningConfig; import javax.annotation.Nullable; @@ -50,6 +51,8 @@ public class IndexTaskSamplerSpec implements SamplerSpec private final SamplerConfig samplerConfig; private final InputSourceSampler inputSourceSampler; + private final TuningConfig tuningConfig; + @JsonCreator public IndexTaskSamplerSpec( @JsonProperty("spec") final IndexTask.IndexIngestionSpec ingestionSpec, @@ -88,11 +91,12 @@ public IndexTaskSamplerSpec( this.samplerConfig = samplerConfig; this.inputSourceSampler = inputSourceSampler; + this.tuningConfig = ingestionSpec.getTuningConfig(); } @Override public SamplerResponse sample() { - return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig); + return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig, tuningConfig); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java index c91d8434c21b..2d937141b8b0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java @@ -49,6 +49,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.TuningConfig; import javax.annotation.Nullable; import javax.inject.Inject; @@ -97,7 +98,8 @@ public SamplerResponse sample( // inputFormat can be null only if inputSource.needsFormat() = false or parser is specified. @Nullable final InputFormat inputFormat, @Nullable final DataSchema dataSchema, - @Nullable final SamplerConfig samplerConfig + @Nullable final SamplerConfig samplerConfig, + @Nullable final TuningConfig tuningConfig ) { Preconditions.checkNotNull(inputSource, "inputSource required"); @@ -119,6 +121,7 @@ public SamplerResponse sample( final InputSourceReader reader = buildReader( nonNullSamplerConfig, nonNullDataSchema, + tuningConfig, inputSource, inputFormat, tempDir @@ -250,12 +253,13 @@ public SamplerResponse sample( private InputSourceReader buildReader( SamplerConfig samplerConfig, DataSchema dataSchema, + TuningConfig tuningConfig, InputSource inputSource, @Nullable InputFormat inputFormat, File tempDir ) { - final InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(dataSchema); + final InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(dataSchema, tuningConfig); InputSourceReader reader = inputSource.reader(inputRowSchema, inputFormat, tempDir); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index e509699d5ced..db581388507f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -247,7 +247,7 @@ public SeekableStreamIndexTaskRunner( this.task = task; this.ioConfig = task.getIOConfig(); this.tuningConfig = task.getTuningConfig(); - this.inputRowSchema = InputRowSchemas.fromDataSchema(task.getDataSchema()); + this.inputRowSchema = InputRowSchemas.fromDataSchema(task.getDataSchema(), tuningConfig); this.inputFormat = ioConfig.getInputFormat(); this.parser = parser; this.authorizerMapper = authorizerMapper; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java index f27cdb50fcc6..ed5c87ae8903 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java @@ -115,7 +115,7 @@ public SamplerResponse sample() ); } - return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig); + return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig, tuningConfig != null ? tuningConfig.convertToTaskTuningConfig() : null); } protected abstract RecordSupplier createRecordSupplier(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java index 2989415a5e6e..a5e9a1e81620 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java @@ -244,7 +244,8 @@ public void testReaderColumnsFilterWithMetricGiven() DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")) ), originalColumnsFilter, - ImmutableSet.of(metricName) + ImmutableSet.of(metricName), + true ); DruidInputSource druidInputSource = new DruidInputSource( datasource, @@ -279,7 +280,8 @@ public void testReaderColumnsFilterWithNoMetricGiven() DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")) ), originalColumnsFilter, - ImmutableSet.of() + ImmutableSet.of(), + true ); DruidInputSource druidInputSource = new DruidInputSource( datasource, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java index 991a5950f9aa..8bb582d1e4e9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java @@ -134,7 +134,7 @@ public void testFromDataSchema() null ); - InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema); + InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema, null); Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec()); Assert.assertEquals(dimensionsSpec.getDimensions(), inputRowSchema.getDimensionsSpec().getDimensions()); Assert.assertEquals(dimensionsSpec.getDimensionNames(), inputRowSchema.getDimensionsSpec().getDimensionNames()); @@ -163,7 +163,7 @@ public void testFromDataSchemaWithNoAggregator() null ); - InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema); + InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema, null); Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec()); Assert.assertEquals(dimensionsSpec.getDimensions(), inputRowSchema.getDimensionsSpec().getDimensions()); Assert.assertEquals(dimensionsSpec.getDimensionNames(), inputRowSchema.getDimensionsSpec().getDimensionNames()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java index 464579764d11..d209c6d15e7c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java @@ -67,6 +67,7 @@ public void testCSVColumnAllNull() inputSource, inputFormat, dataSchema, + null, null ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java index b305cd989ede..a4424440e7e8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java @@ -30,6 +30,7 @@ import org.apache.druid.guice.FirehoseModule; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.TuningConfig; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; @@ -96,6 +97,7 @@ public void testSerde() throws IOException Capture capturedInputFormat = EasyMock.newCapture(); Capture capturedDataSchema = EasyMock.newCapture(); Capture capturedSamplerConfig = EasyMock.newCapture(); + Capture capturedTuningConfig = EasyMock.newCapture(); IndexTaskSamplerSpec spec = MAPPER.readValue(json, IndexTaskSamplerSpec.class); @@ -103,7 +105,8 @@ public void testSerde() throws IOException EasyMock.capture(capturedInputSource), EasyMock.capture(capturedInputFormat), EasyMock.capture(capturedDataSchema), - EasyMock.capture(capturedSamplerConfig) + EasyMock.capture(capturedSamplerConfig), + EasyMock.capture(capturedTuningConfig) )).andReturn(new SamplerResponse(0, 0, null)); replayAll(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java index cbd58d1adfbb..5081fdd8bfe9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java @@ -169,14 +169,14 @@ public void testNoParams() expectedException.expect(NullPointerException.class); expectedException.expectMessage("inputSource required"); - inputSourceSampler.sample(null, null, null, null); + inputSourceSampler.sample(null, null, null, null, null); } @Test public void testNoDataSchema() { final InputSource inputSource = createInputSource(getTestRows(), null); - final SamplerResponse response = inputSourceSampler.sample(inputSource, createInputFormat(), null, null); + final SamplerResponse response = inputSourceSampler.sample(inputSource, createInputFormat(), null, null, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(0, response.getNumRowsIndexed()); @@ -248,7 +248,8 @@ public void testNoDataSchemaNumRows() inputSource, createInputFormat(), null, - new SamplerConfig(3, null, null, null) + new SamplerConfig(3, null, null, null), + null ); Assert.assertEquals(3, response.getNumRowsRead()); @@ -295,7 +296,7 @@ public void testMissingValueTimestampSpec() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(6, response.getNumRowsIndexed()); @@ -404,7 +405,7 @@ public void testWithTimestampSpec() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -504,7 +505,7 @@ public void testWithDimensionSpec() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -610,7 +611,7 @@ public void testWithNoRollup() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -721,7 +722,7 @@ public void testWithRollup() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -804,7 +805,7 @@ public void testWithMoreRollup() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -875,7 +876,7 @@ public void testWithTransformsAutoDimensions() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -964,7 +965,7 @@ public void testWithTransformsDimensionsSpec() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -1032,7 +1033,7 @@ public void testWithFilter() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); Assert.assertEquals(5, response.getNumRowsRead()); Assert.assertEquals(4, response.getNumRowsIndexed()); @@ -1119,7 +1120,7 @@ public void testIndexParseException() throws IOException final InputSource inputSource = createInputSource(inputTestRows, dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); Assert.assertEquals(7, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -1233,7 +1234,8 @@ public void testMultipleJsonStringInOneBlock() throws IOException new RecordSupplierInputSource("topicName", new TestRecordSupplier(jsonBlockList), true, 3000), createInputFormat(), dataSchema, - new SamplerConfig(200, 3000/*default timeout is 10s, shorten it to speed up*/, null, null) + new SamplerConfig(200, 3000/*default timeout is 10s, shorten it to speed up*/, null, null), + null ); // @@ -1329,7 +1331,7 @@ public InputSourceReader reader( throw new RuntimeException(); } }; - inputSourceSampler.sample(failingReaderInputSource, null, null, null); + inputSourceSampler.sample(failingReaderInputSource, null, null, null, null); } @Test @@ -1358,7 +1360,8 @@ public void testRowLimiting() throws IOException inputSource, inputFormat, dataSchema, - new SamplerConfig(4, null, null, null) + new SamplerConfig(4, null, null, null), + null ); Assert.assertEquals(4, response.getNumRowsRead()); @@ -1393,7 +1396,8 @@ public void testMaxBytesInMemoryLimiting() throws IOException inputSource, inputFormat, dataSchema, - new SamplerConfig(null, null, HumanReadableBytes.valueOf(256), null) + new SamplerConfig(null, null, HumanReadableBytes.valueOf(256), null), + null ); Assert.assertEquals(4, response.getNumRowsRead()); @@ -1427,7 +1431,8 @@ public void testMaxClientResponseBytesLimiting() throws IOException inputSource, inputFormat, dataSchema, - new SamplerConfig(null, null, null, HumanReadableBytes.valueOf(300)) + new SamplerConfig(null, null, null, HumanReadableBytes.valueOf(300)), + null ); Assert.assertEquals(4, response.getNumRowsRead()); diff --git a/integration-tests-ex/cases/cluster/Common/dependencies.yaml b/integration-tests-ex/cases/cluster/Common/dependencies.yaml index ccfce630a2a5..ab331a4946d4 100644 --- a/integration-tests-ex/cases/cluster/Common/dependencies.yaml +++ b/integration-tests-ex/cases/cluster/Common/dependencies.yaml @@ -67,7 +67,7 @@ services: # See https://hub.docker.com/_/mysql # The image will intialize the user and DB upon first start. metadata: - # platform: linux/x86_64 - Add when running on M1 Macs + platform: linux/x86_64 #- Add when running on M1 Macs image: mysql:$MYSQL_IMAGE_VERSION container_name: metadata command: diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java index 144722eea9b8..7b983b9cf1e4 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java @@ -155,14 +155,23 @@ public String getStringFromFileAndReplaceDatasource(String filePath, String data return fileString; } + protected void doTestQuery(String dataSource, String queryFilePath) + { + doTestQuery(dataSource, queryFilePath, false); + } + /** * Reads native queries from a file and runs against the provided datasource. */ - protected void doTestQuery(String dataSource, String queryFilePath) + protected void doTestQuery(String dataSource, String queryFilePath, boolean isSql) { try { String query = getStringFromFileAndReplaceDatasource(queryFilePath, dataSource); - queryHelper.testQueriesFromString(query); + if (isSql) { + sqlQueryHelper.testQueriesFromString(query); + } else { + queryHelper.testQueriesFromString(query); + } } catch (Exception e) { LOG.error(e, "Error while running test query at path " + queryFilePath); @@ -255,6 +264,31 @@ protected void doIndexTest( queryFilePath, waitForNewVersion, runTestQueries, + false, + waitForSegmentsToLoad, + segmentAvailabilityConfirmationPair + ); + } + + protected void doIndexTest( + String dataSource, + String indexTaskFilePath, + Function taskSpecTransform, + String queryFilePath, + boolean waitForNewVersion, + boolean runTestQueries, + boolean waitForSegmentsToLoad, + Pair segmentAvailabilityConfirmationPair + ) throws IOException + { + doIndexTest( + dataSource, + indexTaskFilePath, + taskSpecTransform, + queryFilePath, + waitForNewVersion, + runTestQueries, + false, waitForSegmentsToLoad, segmentAvailabilityConfirmationPair ); @@ -267,6 +301,7 @@ protected void doIndexTest( String queryFilePath, boolean waitForNewVersion, boolean runTestQueries, + boolean isSqlQueries, boolean waitForSegmentsToLoad, Pair segmentAvailabilityConfirmationPair ) throws IOException @@ -288,7 +323,7 @@ protected void doIndexTest( segmentAvailabilityConfirmationPair ); if (runTestQueries) { - doTestQuery(dataSource, queryFilePath); + doTestQuery(dataSource, queryFilePath, isSqlQueries); } } diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java index ce8a9f5c13cd..868ffdb8e530 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java @@ -26,6 +26,7 @@ import javax.annotation.Nonnull; import java.io.Closeable; +import java.util.Collections; import java.util.Map; import java.util.UUID; import java.util.function.Function; @@ -48,6 +49,28 @@ public void doIndexTest( @Nonnull Map extraInputFormatMap, Pair segmentAvailabilityConfirmationPair ) throws Exception + { + doIndexTest( + inputFormatDetails, + INDEX_TASK, + INDEX_QUERIES_RESOURCE, + false, + Collections.emptyMap(), + extraInputFormatMap, + segmentAvailabilityConfirmationPair + ); + } + + + public void doIndexTest( + InputFormatDetails inputFormatDetails, + String ingestSpecTemplate, + String queries, + boolean useSqlQueries, + @Nonnull Map templateValues, + @Nonnull Map extraInputFormatMap, + Pair segmentAvailabilityConfirmationPair + ) throws Exception { final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); Map inputFormatMap = new ImmutableMap.Builder().putAll(extraInputFormatMap) @@ -58,6 +81,13 @@ public void doIndexTest( ) { final Function sqlInputSourcePropsTransform = spec -> { try { + for (Map.Entry entry : templateValues.entrySet()) { + spec = StringUtils.replace( + spec, + "%%" + entry.getKey() + "%%", + jsonMapper.writeValueAsString(entry.getValue()) + ); + } spec = StringUtils.replace( spec, "%%PARTITIONS_SPEC%%", @@ -102,11 +132,12 @@ public void doIndexTest( doIndexTest( indexDatasource, - INDEX_TASK, + ingestSpecTemplate, sqlInputSourcePropsTransform, - INDEX_QUERIES_RESOURCE, + queries, false, true, + useSqlQueries, true, segmentAvailabilityConfirmationPair ); diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllFormatSchemalessTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllFormatSchemalessTest.java new file mode 100644 index 000000000000..61deca75ed88 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllFormatSchemalessTest.java @@ -0,0 +1,133 @@ +/* + * 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.testsEx.indexer; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.testsEx.categories.InputFormat; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +@RunWith(DruidTestRunner.class) +@Category(InputFormat.class) +public class ITLocalInputSourceAllFormatSchemalessTest extends AbstractLocalInputSourceParallelIndexTest +{ + private static final String INDEX_TASK = "/indexer/wikipedia_local_input_source_index_task_schemaless.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_schemaless_queries.json"; + + @Test + public void testAvroInputFormatIndexDataIngestionSpecWithFileSchemaSchemaless() throws Exception + { + List fieldList = ImmutableList.of( + ImmutableMap.of("name", "timestamp", "type", "string"), + ImmutableMap.of("name", "page", "type", "string"), + ImmutableMap.of("name", "language", "type", "string"), + ImmutableMap.of("name", "user", "type", "string"), + ImmutableMap.of("name", "unpatrolled", "type", "string"), + ImmutableMap.of("name", "newPage", "type", "string"), + ImmutableMap.of("name", "robot", "type", "string"), + ImmutableMap.of("name", "anonymous", "type", "string"), + ImmutableMap.of("name", "namespace", "type", "string"), + ImmutableMap.of("name", "continent", "type", "string"), + ImmutableMap.of("name", "country", "type", "string"), + ImmutableMap.of("name", "region", "type", "string"), + ImmutableMap.of("name", "city", "type", "string"), + ImmutableMap.of("name", "added", "type", "int"), + ImmutableMap.of("name", "deleted", "type", "int"), + ImmutableMap.of("name", "delta", "type", "int") + ); + Map schema = ImmutableMap.of("namespace", "org.apache.druid.data.input", + "type", "record", + "name", "wikipedia", + "fields", fieldList); + doIndexTest( + AbstractITBatchIndexTest.InputFormatDetails.AVRO, + INDEX_TASK, + INDEX_QUERIES_RESOURCE, + true, + ImmutableMap.of("USE_NESTED_COLUMN_INDEXER", true), + ImmutableMap.of("schema", schema), + new Pair<>(false, false) + ); + } + + @Test + public void testAvroInputFormatIndexDataIngestionSpecNoFileSchemaSchemaless() throws Exception + { + doIndexTest( + AbstractITBatchIndexTest.InputFormatDetails.AVRO, + INDEX_TASK, + INDEX_QUERIES_RESOURCE, + true, + ImmutableMap.of("USE_NESTED_COLUMN_INDEXER", true), + Collections.emptyMap(), + new Pair<>(false, false) + ); + } + + @Test + public void testJsonInputFormatIndexDataIngestionSpecSchemaless() throws Exception + { + doIndexTest( + AbstractITBatchIndexTest.InputFormatDetails.JSON, + INDEX_TASK, + INDEX_QUERIES_RESOURCE, + true, + ImmutableMap.of("USE_NESTED_COLUMN_INDEXER", true), + Collections.emptyMap(), + new Pair<>(false, false) + ); + } + + @Test + public void testParquetInputFormatIndexDataIngestionSpecSchemaless() throws Exception + { + doIndexTest( + AbstractITBatchIndexTest.InputFormatDetails.PARQUET, + INDEX_TASK, + INDEX_QUERIES_RESOURCE, + true, + ImmutableMap.of("USE_NESTED_COLUMN_INDEXER", true), + Collections.emptyMap(), + new Pair<>(false, false) + ); + } + + @Test + public void testOrcInputFormatIndexDataIngestionSpecSchemaless() throws Exception + { + doIndexTest( + AbstractITBatchIndexTest.InputFormatDetails.ORC, + INDEX_TASK, + INDEX_QUERIES_RESOURCE, + true, + ImmutableMap.of("USE_NESTED_COLUMN_INDEXER", true), + Collections.emptyMap(), + new Pair<>(false, false) + ); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllInputFormatTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllInputFormatTest.java index 8482952db243..641d385865fa 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllInputFormatTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllInputFormatTest.java @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.tests.indexer.AbstractLocalInputSourceParallelIndexTest; import org.apache.druid.testsEx.categories.InputFormat; import org.apache.druid.testsEx.config.DruidTestRunner; import org.junit.Test; diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java index 73fc73d42d87..4f19a226d9b8 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java @@ -159,6 +159,7 @@ private void submitIngestionTaskAndVerify( null, false, false, + false, true, new Pair<>(false, false) ); diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java index 735bc0a50ff7..7c57af7f764c 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java @@ -32,7 +32,7 @@ import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.guice.TestClient; import org.apache.druid.testing.utils.SqlTestQueryHelper; -import org.apache.druid.tests.indexer.AbstractIndexerTest; +import org.apache.druid.testsEx.indexer.AbstractIndexerTest; import org.apache.druid.testsEx.categories.HighAvailability; import org.apache.druid.testsEx.cluster.DruidClusterClient; import org.apache.druid.testsEx.config.DruidTestRunner; diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json index 928effe65e97..62c3ac637159 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json @@ -1,6 +1,6 @@ [ { - "description": "timeseries, 1 agg, all", + "description": "timeboundary", "query":{ "queryType" : "timeBoundary", "dataSource": "%%DATASOURCE%%" diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json new file mode 100644 index 000000000000..5794744ee02c --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json @@ -0,0 +1,32 @@ +[ + { + "description": "select *", + "query": { + "query": "SELECT * FROM \"%%DATASOURCE%%\"" + }, + "expectedResults": [ + {"__time":"2013-08-31T01:02:33.000Z","continent":"North America","country":"United States","city":"San Francisco","added":57,"unpatrolled":"true","delta":-143,"language":"en","robot":"false","deleted":200,"newPage":"true","namespace":"article","anonymous":"false","page":"Gypsy Danger","region":"Bay Area","user":"nuclear"}, + {"__time":"2013-08-31T03:32:45.000Z","continent":"Australia","country":"Australia","city":"Syndey","added":459,"unpatrolled":"false","delta":330,"language":"en","robot":"true","deleted":129,"newPage":"true","namespace":"wikipedia","anonymous":"false","page":"Striker Eureka","region":"Cantebury","user":"speed"}, + {"__time":"2013-08-31T07:11:21.000Z","continent":"Asia","country":"Russia","city":"Moscow","added":123,"unpatrolled":"false","delta":111,"language":"ru","robot":"true","deleted":12,"newPage":"true","namespace":"article","anonymous":"false","page":"Cherno Alpha","region":"Oblast","user":"masterYi"}, + {"__time":"2013-08-31T11:58:39.000Z","continent":"Asia","country":"China","city":"Taiyuan","added":905,"unpatrolled":"true","delta":900,"language":"zh","robot":"true","deleted":5,"newPage":"false","namespace":"wikipedia","anonymous":"false","page":"Crimson Typhoon","region":"Shanxi","user":"triplets"}, + {"__time":"2013-08-31T12:41:27.000Z","continent":"Asia","country":"Japan","city":"Tokyo","added":1,"unpatrolled":"true","delta":-9,"language":"ja","robot":"true","deleted":10,"newPage":"false","namespace":"wikipedia","anonymous":"false","page":"Coyote Tango","region":"Kanto","user":"stringer"}, + {"__time":"2013-09-01T01:02:33.000Z","continent":"North America","country":"United States","city":"San Francisco","added":57,"unpatrolled":"true","delta":-143,"language":"en","robot":"false","deleted":200,"newPage":"true","namespace":"article","anonymous":"false","page":"Gypsy Danger","region":"Bay Area","user":"nuclear"}, + {"__time":"2013-09-01T03:32:45.000Z","continent":"Australia","country":"Australia","city":"Syndey","added":459,"unpatrolled":"false","delta":330,"language":"en","robot":"true","deleted":129,"newPage":"true","namespace":"wikipedia","anonymous":"false","page":"Striker Eureka","region":"Cantebury","user":"speed"}, + {"__time":"2013-09-01T07:11:21.000Z","continent":"Asia","country":"Russia","city":"Moscow","added":123,"unpatrolled":"false","delta":111,"language":"ru","robot":"true","deleted":12,"newPage":"true","namespace":"article","anonymous":"false","page":"Cherno Alpha","region":"Oblast","user":"masterYi"}, + {"__time":"2013-09-01T11:58:39.000Z","continent":"Asia","country":"China","city":"Taiyuan","added":905,"unpatrolled":"true","delta":900,"language":"zh","robot":"true","deleted":5,"newPage":"false","namespace":"wikipedia","anonymous":"false","page":"Crimson Typhoon","region":"Shanxi","user":"triplets"}, + {"__time":"2013-09-01T12:41:27.000Z","continent":"Asia","country":"Japan","city":"Tokyo","added":1,"unpatrolled":"true","delta":-9,"language":"ja","robot":"true","deleted":10,"newPage":"false","namespace":"wikipedia","anonymous":"false","page":"Coyote Tango","region":"Kanto","user":"stringer"} + ] + }, + { + "description": "simple group by", + "query": { + "query": "SELECT page, SUM(added) as added FROM \"%%DATASOURCE%%\" WHERE continent = 'Asia' GROUP BY 1 ORDER BY 2 DESC" + }, + "expectedResults": [ + {"page":"Crimson Typhoon","added":1810}, + {"page":"Cherno Alpha","added":246}, + {"page":"Coyote Tango","added":2} + ] + } + +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task_schemaless.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task_schemaless.json new file mode 100644 index 000000000000..3e25773759e6 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task_schemaless.json @@ -0,0 +1,45 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [] + }, + "metricsSpec": [], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "local", + "filter" : "%%INPUT_SOURCE_FILTER%%", + "baseDir": "%%INPUT_SOURCE_BASE_DIR%%" + }, + "appendToExisting": %%APPEND_TO_EXISTING%%, + "dropExisting": %%DROP_EXISTING%%, + "inputFormat": %%INPUT_FORMAT%% + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 4, + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + }, + "forceGuaranteedRollup": %%FORCE_GUARANTEED_ROLLUP%%, + "partitionsSpec": %%PARTITIONS_SPEC%%, + "appendableIndexSpec": { + "type": "onheap", + "useNestedColumnIndexerForSchemaDiscovery": %%USE_NESTED_COLUMN_INDEXER%% + } + } + } +} \ No newline at end of file From 3f410a91ccc450809539353a4af56d512af8cfd4 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 13 Jan 2023 21:45:22 -0800 Subject: [PATCH 02/15] fixes --- .../druid/indexing/input/InputRowSchemas.java | 2 +- ...SeekableStreamIndexTaskRunnerAuthTest.java | 37 ++++++++++++++++++- .../testsEx/cluster/MetastoreClient.java | 1 - .../config/IntegrationTestingConfigEx.java | 1 - .../leadership/ITHighAvailabilityTest.java | 2 +- 5 files changed, 38 insertions(+), 5 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java index 86cd880715b0..74d343ef26a9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java @@ -63,7 +63,7 @@ public static InputRowSchema fromDataSchema(final DataSchema dataSchema, @Nullab Arrays.stream(dataSchema.getAggregators()) .map(AggregatorFactory::getName) .collect(Collectors.toSet()), - tuningConfig != null ? tuningConfig.getAppendableIndexSpec().useNestedColumnIndexerForSchemaDiscovery() : false + tuningConfig != null && tuningConfig.getAppendableIndexSpec().useNestedColumnIndexerForSchemaDiscovery() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java index db51a416fc90..79a9357f54c5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java @@ -54,6 +54,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; +import java.io.File; import java.util.Collections; import java.util.List; import java.util.Map; @@ -117,7 +118,41 @@ public Authorizer getAuthorizer(String name) null, null ); - SeekableStreamIndexTaskTuningConfig tuningConfig = mock(SeekableStreamIndexTaskTuningConfig.class); + SeekableStreamIndexTaskTuningConfig tuningConfig = new SeekableStreamIndexTaskTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + { + @Override + public SeekableStreamIndexTaskTuningConfig withBasePersistDirectory(File dir) + { + return null; + } + + @Override + public String toString() + { + return null; + } + }; SeekableStreamIndexTaskIOConfig ioConfig = new TestSeekableStreamIndexTaskIOConfig(); // Initiliaze task and task runner diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/MetastoreClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/MetastoreClient.java index 86e0ece8d525..58e47bbe7eb4 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/MetastoreClient.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/MetastoreClient.java @@ -26,7 +26,6 @@ import org.skife.jdbi.v2.Handle; import javax.inject.Inject; - import java.sql.Connection; import java.sql.SQLException; diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java index c14ea745aa81..e23c996c8844 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java @@ -25,7 +25,6 @@ import org.apache.druid.testing.IntegrationTestingConfigProvider; import javax.inject.Inject; - import java.util.Map; import java.util.Properties; diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java index 7c57af7f764c..60d22f3f9f8f 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java @@ -32,11 +32,11 @@ import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.guice.TestClient; import org.apache.druid.testing.utils.SqlTestQueryHelper; -import org.apache.druid.testsEx.indexer.AbstractIndexerTest; import org.apache.druid.testsEx.categories.HighAvailability; import org.apache.druid.testsEx.cluster.DruidClusterClient; import org.apache.druid.testsEx.config.DruidTestRunner; import org.apache.druid.testsEx.config.Initializer; +import org.apache.druid.testsEx.indexer.AbstractIndexerTest; import org.apache.druid.testsEx.utils.DruidClusterAdminClient; import org.junit.Test; import org.junit.experimental.categories.Category; From b785c3b7981a540aa743a286e6a96f25d50db44e Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sat, 14 Jan 2023 06:01:15 -0800 Subject: [PATCH 03/15] stable test output --- .../indexing/overlord/sampler/SamplerConfig.java | 14 +++++++------- .../wikipedia_index_schemaless_queries.json | 4 ++-- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java index aaa2cd07898b..75ddb341d2c1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java @@ -92,13 +92,13 @@ public int getTimeoutMs() /** * Maximum number of bytes in memory that the {@link org.apache.druid.segment.incremental.IncrementalIndex} used by - * {@link InputSourceSampler#sample(InputSource, InputFormat, DataSchema, SamplerConfig)} will be allowed to - * accumulate before aborting sampling. Particularly useful for limiting footprint of sample operations as well as - * overall response size from sample requests. However, it is not directly correlated to response size since it - * also contains the "raw" input data, so actual responses will likely be at least twice the size of this value, - * depending on factors such as number of transforms, aggregations in the case of rollup, whether all columns - * of the input are present in the dimension spec, and so on. If it is preferred to control client response size, - * use {@link SamplerConfig#getMaxClientResponseBytes()} instead. + * {@link InputSourceSampler#sample(InputSource, InputFormat, DataSchema, SamplerConfig, org.apache.druid.segment.indexing.TuningConfig)} + * will be allowed to accumulate before aborting sampling. Particularly useful for limiting footprint of sample + * operations as well as overall response size from sample requests. However, it is not directly correlated to + * response size since it also contains the "raw" input data, so actual responses will likely be at least twice the + * size of this value, depending on factors such as number of transforms, aggregations in the case of rollup, whether + * all columns of the input are present in the dimension spec, and so on. If it is preferred to control client + * response size, use {@link SamplerConfig#getMaxClientResponseBytes()} instead. */ public long getMaxBytesInMemory() { diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json index 5794744ee02c..574b58f3ee3a 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json @@ -1,8 +1,8 @@ [ { - "description": "select *", + "description": "select all things", "query": { - "query": "SELECT * FROM \"%%DATASOURCE%%\"" + "query": "SELECT \"__time\",\"continent\",\"country\",\"city\",\"added\",\"unpatrolled\",\"delta\",\"language\",\"robot\",\"deleted\",\"newPage\",\"namespace\",\"anonymous\",\"page\",\"region\",\"user\" FROM \"%%DATASOURCE%%\"" }, "expectedResults": [ {"__time":"2013-08-31T01:02:33.000Z","continent":"North America","country":"United States","city":"San Francisco","added":57,"unpatrolled":"true","delta":-143,"language":"en","robot":"false","deleted":200,"newPage":"true","namespace":"article","anonymous":"false","page":"Gypsy Danger","region":"Bay Area","user":"nuclear"}, From 7def946503bc455d2247a47b47978fc5f5dd35e6 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sat, 14 Jan 2023 16:57:46 -0800 Subject: [PATCH 04/15] more stable --- .../resources/indexer/wikipedia_index_schemaless_queries.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json index 574b58f3ee3a..316ec5fd8ae4 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_schemaless_queries.json @@ -2,7 +2,7 @@ { "description": "select all things", "query": { - "query": "SELECT \"__time\",\"continent\",\"country\",\"city\",\"added\",\"unpatrolled\",\"delta\",\"language\",\"robot\",\"deleted\",\"newPage\",\"namespace\",\"anonymous\",\"page\",\"region\",\"user\" FROM \"%%DATASOURCE%%\"" + "query": "SELECT \"__time\",\"continent\",\"country\",\"city\",\"added\",\"unpatrolled\",\"delta\",\"language\",\"robot\",\"deleted\",\"newPage\",\"namespace\",\"anonymous\",\"page\",\"region\",\"user\" FROM \"%%DATASOURCE%%\" ORDER BY __time" }, "expectedResults": [ {"__time":"2013-08-31T01:02:33.000Z","continent":"North America","country":"United States","city":"San Francisco","added":57,"unpatrolled":"true","delta":-143,"language":"en","robot":"false","deleted":200,"newPage":"true","namespace":"article","anonymous":"false","page":"Gypsy Danger","region":"Bay Area","user":"nuclear"}, From eec9bd8dc3727492a3d9eb0e00fbbcf5b7d50de2 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sun, 15 Jan 2023 23:03:28 -0800 Subject: [PATCH 05/15] push flag into FlattenerMaker constructors --- .../druid/data/input/impl/JsonLineReader.java | 3 +-- .../druid/data/input/impl/JsonNodeReader.java | 3 +-- .../apache/druid/data/input/impl/JsonReader.java | 3 +-- .../util/common/parsers/JSONFlattenerMaker.java | 7 +++++-- .../java/util/common/parsers/JSONPathParser.java | 3 +-- .../java/util/common/parsers/ObjectFlatteners.java | 7 +++---- .../util/common/parsers/JSONFlattenerMakerTest.java | 7 ++++--- .../util/common/parsers/ObjectFlattenersTest.java | 5 ++--- .../druid/data/input/avro/AvroFlattenerMaker.java | 12 ++++++++++-- .../apache/druid/data/input/avro/AvroOCFReader.java | 8 ++++++-- .../apache/druid/data/input/avro/AvroParsers.java | 3 +-- .../druid/data/input/avro/AvroStreamReader.java | 8 ++++++-- .../data/input/avro/AvroFlattenerMakerTest.java | 13 +++++++------ .../data/input/orc/OrcHadoopInputRowParser.java | 3 +-- .../org/apache/druid/data/input/orc/OrcReader.java | 3 +-- .../data/input/orc/OrcStructFlattenerMaker.java | 7 +++++-- .../druid/data/input/parquet/ParquetReader.java | 3 +-- .../avro/ParquetAvroHadoopInputRowParser.java | 3 +-- .../parquet/simple/ParquetGroupFlattenerMaker.java | 7 +++++-- .../parquet/simple/ParquetHadoopInputRowParser.java | 3 +-- .../data/input/protobuf/ProtobufFlattenerMaker.java | 9 ++++++++- .../druid/data/input/protobuf/ProtobufReader.java | 3 +-- .../seekablestream/SeekableStreamSamplerSpec.java | 2 +- .../cases/cluster/Common/dependencies.yaml | 3 ++- 24 files changed, 75 insertions(+), 53 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java index 7f34197bc10a..b89559e1c42f 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java @@ -61,8 +61,7 @@ public class JsonLineReader extends TextReader super(inputRowSchema, source); this.flattener = ObjectFlatteners.create( flattenSpec, - new JSONFlattenerMaker(keepNullColumns), - inputRowSchema.shouldDiscoverNestedColumns() + new JSONFlattenerMaker(keepNullColumns, inputRowSchema.shouldDiscoverNestedColumns()) ); this.mapper = mapper; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java index 62efc05d5fdf..1fd58bbabb6f 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java @@ -82,8 +82,7 @@ public class JsonNodeReader extends IntermediateRowParsingReader this.source = source; this.flattener = ObjectFlatteners.create( flattenSpec, - new JSONFlattenerMaker(keepNullColumns), - inputRowSchema.shouldDiscoverNestedColumns() + new JSONFlattenerMaker(keepNullColumns, inputRowSchema.shouldDiscoverNestedColumns()) ); this.mapper = mapper; this.jsonFactory = new JsonFactory(); diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java index 09b12870cb4f..415f1c1396c7 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java @@ -79,8 +79,7 @@ public class JsonReader extends IntermediateRowParsingReader this.source = source; this.flattener = ObjectFlatteners.create( flattenSpec, - new JSONFlattenerMaker(keepNullColumns), - inputRowSchema.shouldDiscoverNestedColumns() + new JSONFlattenerMaker(keepNullColumns, inputRowSchema.shouldDiscoverNestedColumns()) ); this.mapper = mapper; this.jsonFactory = new JsonFactory(); diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java index cc1843aab121..a8d45b167910 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java @@ -57,14 +57,17 @@ public class JSONFlattenerMaker implements ObjectFlatteners.FlattenerMaker discoverRootFields(final JsonNode obj, boolean discoverNestedFields) + public Iterable discoverRootFields(final JsonNode obj) { if (discoverNestedFields) { return obj::fieldNames; diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONPathParser.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONPathParser.java index 268319ce446b..281d8442cc22 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONPathParser.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONPathParser.java @@ -44,8 +44,7 @@ public JSONPathParser(JSONPathSpec flattenSpec, ObjectMapper mapper, boolean kee this.mapper = mapper == null ? new ObjectMapper() : mapper; this.flattener = ObjectFlatteners.create( flattenSpec, - new JSONFlattenerMaker(keepNullColumns), - false + new JSONFlattenerMaker(keepNullColumns, false) ); } diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java index 9887090bf1f2..f370a2ec6b10 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java @@ -48,8 +48,7 @@ private ObjectFlatteners() public static ObjectFlattener create( @Nullable final JSONPathSpec flattenSpecInput, - final FlattenerMaker flattenerMaker, - boolean discoverNestedColumns + final FlattenerMaker flattenerMaker ) { final Map> extractors = new LinkedHashMap<>(); @@ -154,7 +153,7 @@ public void clear() public Set keySet() { if (flattenSpec.isUseFieldDiscovery()) { - final Iterable rootFields = flattenerMaker.discoverRootFields(obj, discoverNestedColumns); + final Iterable rootFields = flattenerMaker.discoverRootFields(obj); if (extractors.isEmpty() && rootFields instanceof Set) { return (Set) rootFields; } else { @@ -221,7 +220,7 @@ public interface FlattenerMaker /** * List all "root" fields, optionally filtering to include only fields that contain primitive and lists of primitive values */ - Iterable discoverRootFields(T obj, boolean discoverNestedFields); + Iterable discoverRootFields(T obj); /** * Get a top level field from a "json" object diff --git a/core/src/test/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMakerTest.java b/core/src/test/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMakerTest.java index 8fa82a61e508..5583081db10e 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMakerTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMakerTest.java @@ -38,7 +38,8 @@ public class JSONFlattenerMakerTest { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final JSONFlattenerMaker FLATTENER_MAKER = new JSONFlattenerMaker(true); + private static final JSONFlattenerMaker FLATTENER_MAKER = new JSONFlattenerMaker(true, false); + private static final JSONFlattenerMaker FLATTENER_MAKER_NESTED = new JSONFlattenerMaker(true, true); @Test public void testStrings() throws JsonProcessingException @@ -191,11 +192,11 @@ public void testDiscovery() throws JsonProcessingException Assert.assertTrue(node.isObject()); Assert.assertEquals( ImmutableSet.of("bool", "int", "long", "float", "double", "binary", "list", "anotherList"), - ImmutableSet.copyOf(FLATTENER_MAKER.discoverRootFields(node, false)) + ImmutableSet.copyOf(FLATTENER_MAKER.discoverRootFields(node)) ); Assert.assertEquals( ImmutableSet.of("bool", "int", "long", "float", "double", "binary", "list", "anotherList", "nested"), - ImmutableSet.copyOf(FLATTENER_MAKER.discoverRootFields(node, true)) + ImmutableSet.copyOf(FLATTENER_MAKER_NESTED.discoverRootFields(node)) ); } } diff --git a/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java b/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java index 1ab5dc85d707..ab6c50ef188d 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java @@ -34,14 +34,13 @@ public class ObjectFlattenersTest { private static final String SOME_JSON = "{\"foo\": null, \"bar\": 1}"; - private static final ObjectFlatteners.FlattenerMaker FLATTENER_MAKER = new JSONFlattenerMaker(true); + private static final ObjectFlatteners.FlattenerMaker FLATTENER_MAKER = new JSONFlattenerMaker(true, false); private static final ObjectFlattener FLATTENER = ObjectFlatteners.create( new JSONPathSpec( true, ImmutableList.of(new JSONPathFieldSpec(JSONPathFieldType.PATH, "extract", "$.bar")) ), - FLATTENER_MAKER, - true + FLATTENER_MAKER ); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java index ba2d94ccb85f..cad558491c7f 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java @@ -92,14 +92,22 @@ private static boolean isFieldPrimitive(Schema.Field field) private final boolean fromPigAvroStorage; private final boolean binaryAsString; + private final boolean discoverNestedFields; + /** * @param fromPigAvroStorage boolean to specify the data file is stored using AvroStorage * @param binaryAsString boolean to encode the byte[] as a string. */ - public AvroFlattenerMaker(final boolean fromPigAvroStorage, final boolean binaryAsString, final boolean extractUnionsByType) + public AvroFlattenerMaker( + final boolean fromPigAvroStorage, + final boolean binaryAsString, + final boolean extractUnionsByType, + final boolean discoverNestedFields + ) { this.fromPigAvroStorage = fromPigAvroStorage; this.binaryAsString = binaryAsString; + this.discoverNestedFields = discoverNestedFields; this.avroJsonProvider = new GenericAvroJsonProvider(extractUnionsByType); this.jsonPathConfiguration = @@ -111,7 +119,7 @@ public AvroFlattenerMaker(final boolean fromPigAvroStorage, final boolean binary } @Override - public Set discoverRootFields(final GenericRecord obj, boolean discoverNestedFields) + public Set discoverRootFields(final GenericRecord obj) { if (discoverNestedFields) { return obj.getSchema().getFields().stream().map(Schema.Field::name).collect(Collectors.toSet()); diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java index e8ef9c28b4d0..b76fd1c636da 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java @@ -66,8 +66,12 @@ public class AvroOCFReader extends IntermediateRowParsingReader this.readerSchema = readerSchema; this.recordFlattener = ObjectFlatteners.create( flattenSpec, - new AvroFlattenerMaker(false, binaryAsString, extractUnionsByType), - inputRowSchema.shouldDiscoverNestedColumns() + new AvroFlattenerMaker( + false, + binaryAsString, + extractUnionsByType, + inputRowSchema.shouldDiscoverNestedColumns() + ) ); } diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java index d83811c4b764..b55fc2e7859a 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java @@ -52,8 +52,7 @@ public static ObjectFlattener makeFlattener( return ObjectFlatteners.create( flattenSpec, - new AvroFlattenerMaker(fromPigAvroStorage, binaryAsString, extractUnionsByType), - false + new AvroFlattenerMaker(fromPigAvroStorage, binaryAsString, extractUnionsByType, false) ); } diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java index 974505393821..9f0907486084 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java @@ -62,8 +62,12 @@ public class AvroStreamReader extends IntermediateRowParsingReader this.temporaryDirectory = temporaryDirectory; this.orcStructFlattener = ObjectFlatteners.create( flattenSpec, - new OrcStructFlattenerMaker(binaryAsString), - inputRowSchema.shouldDiscoverNestedColumns() + new OrcStructFlattenerMaker(binaryAsString, inputRowSchema.shouldDiscoverNestedColumns()) ); } diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java index 00ddae5f7799..4afccbb8d925 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java @@ -43,7 +43,9 @@ public class OrcStructFlattenerMaker implements ObjectFlatteners.FlattenerMaker< private final JsonProvider orcJsonProvider; private final OrcStructConverter converter; - OrcStructFlattenerMaker(boolean binaryAsString) + private final boolean discoverNestedFields; + + OrcStructFlattenerMaker(boolean binaryAsString, boolean disocverNestedFields) { this.converter = new OrcStructConverter(binaryAsString); this.orcJsonProvider = new OrcStructJsonProvider(converter); @@ -52,10 +54,11 @@ public class OrcStructFlattenerMaker implements ObjectFlatteners.FlattenerMaker< .mappingProvider(new NotImplementedMappingProvider()) .options(EnumSet.of(Option.SUPPRESS_EXCEPTIONS)) .build(); + this.discoverNestedFields = disocverNestedFields; } @Override - public Iterable discoverRootFields(OrcStruct obj, boolean discoverNestedFields) + public Iterable discoverRootFields(OrcStruct obj) { if (discoverNestedFields) { return obj.getSchema().getFieldNames(); diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java index 4a2d53544e8e..c532d78a76e0 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java @@ -68,8 +68,7 @@ public class ParquetReader extends IntermediateRowParsingReader this.temporaryDirectory = temporaryDirectory; this.flattener = ObjectFlatteners.create( flattenSpec, - new ParquetGroupFlattenerMaker(binaryAsString), - inputRowSchema.shouldDiscoverNestedColumns() + new ParquetGroupFlattenerMaker(binaryAsString, inputRowSchema.shouldDiscoverNestedColumns()) ); } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java index 6327cc5bd196..8526e43508c0 100755 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java @@ -78,8 +78,7 @@ public ParquetAvroHadoopInputRowParser( this.recordFlattener = ObjectFlatteners.create( flattenSpec, - new AvroFlattenerMaker(false, this.binaryAsString, this.extractUnionsByType), - false + new AvroFlattenerMaker(false, this.binaryAsString, this.extractUnionsByType, false) ); } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java index 42f14a5d7f2e..1916b9631b16 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java @@ -41,7 +41,9 @@ public class ParquetGroupFlattenerMaker implements ObjectFlatteners.FlattenerMak private final ParquetGroupConverter converter; private final JsonProvider parquetJsonProvider; - public ParquetGroupFlattenerMaker(boolean binaryAsString) + private final boolean discoverNestedFields; + + public ParquetGroupFlattenerMaker(boolean binaryAsString, boolean discoverNestedFields) { this.converter = new ParquetGroupConverter(binaryAsString); this.parquetJsonProvider = new ParquetGroupJsonProvider(converter); @@ -50,10 +52,11 @@ public ParquetGroupFlattenerMaker(boolean binaryAsString) .mappingProvider(new NotImplementedMappingProvider()) .options(EnumSet.of(Option.SUPPRESS_EXCEPTIONS)) .build(); + this.discoverNestedFields = discoverNestedFields; } @Override - public Set discoverRootFields(Group obj, boolean discoverNestedFields) + public Set discoverRootFields(Group obj) { if (discoverNestedFields) { return obj.getType().getFields().stream().map(Type::getName).collect(Collectors.toSet()); diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java index 78e0f9633ef9..50f597a92d39 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java @@ -56,8 +56,7 @@ public ParquetHadoopInputRowParser( } this.groupFlattener = ObjectFlatteners.create( flattenSpec, - new ParquetGroupFlattenerMaker(this.binaryAsString), - false + new ParquetGroupFlattenerMaker(this.binaryAsString, false) ); this.parser = new MapInputRowParser(parseSpec); } diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java index 9ccfe10db4c7..4441f86fa5d4 100644 --- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java +++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java @@ -56,6 +56,13 @@ public class ProtobufFlattenerMaker implements ObjectFlatteners.FlattenerMaker discoverRootFields(Map obj, boolean discoverNestedFields) + public Iterable discoverRootFields(Map obj) { if (discoverNestedFields) { return obj.keySet(); diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java index 20c809017fb9..c16977b535e4 100644 --- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java +++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java @@ -59,8 +59,7 @@ public class ProtobufReader extends IntermediateRowParsingReader this.inputRowSchema = inputRowSchema; this.recordFlattener = ObjectFlatteners.create( flattenSpec, - new ProtobufFlattenerMaker(), - inputRowSchema.shouldDiscoverNestedColumns() + new ProtobufFlattenerMaker(inputRowSchema.shouldDiscoverNestedColumns()) ); this.source = source; this.protobufBytesDecoder = protobufBytesDecoder; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java index ed5c87ae8903..5dde26df2e9f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java @@ -115,7 +115,7 @@ public SamplerResponse sample() ); } - return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig, tuningConfig != null ? tuningConfig.convertToTaskTuningConfig() : null); + return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig, tuningConfig == null ? null : tuningConfig.convertToTaskTuningConfig()); } protected abstract RecordSupplier createRecordSupplier(); diff --git a/integration-tests-ex/cases/cluster/Common/dependencies.yaml b/integration-tests-ex/cases/cluster/Common/dependencies.yaml index ab331a4946d4..4eeb85a49cd6 100644 --- a/integration-tests-ex/cases/cluster/Common/dependencies.yaml +++ b/integration-tests-ex/cases/cluster/Common/dependencies.yaml @@ -67,7 +67,8 @@ services: # See https://hub.docker.com/_/mysql # The image will intialize the user and DB upon first start. metadata: - platform: linux/x86_64 #- Add when running on M1 Macs + # Uncomment the following when running on M1 Macs: + # platform: linux/x86_64 image: mysql:$MYSQL_IMAGE_VERSION container_name: metadata command: From 185dd2ca7740276f8e9ce63b7d08ed3305cd58cd Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 17 Jan 2023 15:41:13 -0800 Subject: [PATCH 06/15] comments and javadoc --- .../java/util/common/parsers/JSONFlattenerMaker.java | 2 ++ .../druid/data/input/avro/AvroFlattenerMaker.java | 11 +++++++++-- .../druid/data/input/orc/OrcStructFlattenerMaker.java | 2 ++ .../parquet/simple/ParquetGroupFlattenerMaker.java | 2 ++ .../data/input/protobuf/ProtobufFlattenerMaker.java | 2 ++ 5 files changed, 17 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java index a8d45b167910..0b8244e29b75 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java @@ -69,6 +69,8 @@ public JSONFlattenerMaker(boolean keepNullValues, boolean discoverNestedFields) @Override public Iterable discoverRootFields(final JsonNode obj) { + // if discovering nested fields, just return all root fields since we want everything + // else, we filter for literals and arrays of literals if (discoverNestedFields) { return obj::fieldNames; } diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java index cad558491c7f..ba9d895b1f9b 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java @@ -95,8 +95,13 @@ private static boolean isFieldPrimitive(Schema.Field field) private final boolean discoverNestedFields; /** - * @param fromPigAvroStorage boolean to specify the data file is stored using AvroStorage - * @param binaryAsString boolean to encode the byte[] as a string. + * @param fromPigAvroStorage boolean to specify the data file is stored using AvroStorage + * @param binaryAsString if true, treat byte[] as utf8 encoded values and coerce to strings, else leave as byte[] + * @param extractUnionsByType if true, unions will be extracted to separate nested fields for each type. See + * {@link GenericAvroJsonProvider#extractUnionTypes(Object)} for more details + * @param discoverNestedFields if true, {@link #discoverRootFields(GenericRecord)} will return the full set of + * fields, else this list will be filtered to contain only simple literals and arrays + * of simple literals */ public AvroFlattenerMaker( final boolean fromPigAvroStorage, @@ -121,6 +126,8 @@ public AvroFlattenerMaker( @Override public Set discoverRootFields(final GenericRecord obj) { + // if discovering nested fields, just return all root fields since we want everything + // else, we filter for literals and arrays of literals if (discoverNestedFields) { return obj.getSchema().getFields().stream().map(Schema.Field::name).collect(Collectors.toSet()); } diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java index 4afccbb8d925..016ddb3f90c4 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java @@ -60,6 +60,8 @@ public class OrcStructFlattenerMaker implements ObjectFlatteners.FlattenerMaker< @Override public Iterable discoverRootFields(OrcStruct obj) { + // if discovering nested fields, just return all root fields since we want everything + // else, we filter for literals and arrays of literals if (discoverNestedFields) { return obj.getSchema().getFieldNames(); } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java index 1916b9631b16..a243107cc238 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java @@ -58,6 +58,8 @@ public ParquetGroupFlattenerMaker(boolean binaryAsString, boolean discoverNested @Override public Set discoverRootFields(Group obj) { + // if discovering nested fields, just return all root fields since we want everything + // else, we filter for literals and arrays of literals if (discoverNestedFields) { return obj.getType().getFields().stream().map(Type::getName).collect(Collectors.toSet()); } diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java index 4441f86fa5d4..a3d69226fd48 100644 --- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java +++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufFlattenerMaker.java @@ -72,6 +72,8 @@ public JsonProvider getJsonProvider() @Override public Iterable discoverRootFields(Map obj) { + // if discovering nested fields, just return all root fields since we want everything + // else, we filter for literals and arrays of literals if (discoverNestedFields) { return obj.keySet(); } From 4ccd6a684fb5a0933f734904d15f1bde52710d5d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 17 Jan 2023 20:56:42 -0800 Subject: [PATCH 07/15] move useNestedColumnIndexerForSchemaDiscovery from AppendableIndexSpec to DimensionsSpec --- .../druid/data/input/InputRowSchema.java | 13 +--- .../druid/data/input/impl/DimensionsSpec.java | 61 ++++++++++++++++--- .../druid/data/input/impl/JsonLineReader.java | 5 +- .../druid/data/input/impl/JsonNodeReader.java | 5 +- .../druid/data/input/impl/JsonReader.java | 5 +- .../druid/data/input/avro/AvroOCFReader.java | 2 +- .../druid/data/input/avro/AvroParsers.java | 10 ++- .../data/input/avro/AvroStreamReader.java | 2 +- .../input/kafkainput/KafkaInputFormat.java | 3 +- .../input/orc/OrcHadoopInputRowParser.java | 7 ++- .../druid/data/input/orc/OrcReader.java | 5 +- .../druid/data/input/orc/OrcReaderTest.java | 7 +-- .../data/input/parquet/ParquetReader.java | 5 +- .../avro/ParquetAvroHadoopInputRowParser.java | 10 ++- .../simple/ParquetHadoopInputRowParser.java | 7 ++- .../NestedColumnParquetReaderTest.java | 14 ++--- .../protobuf/ProtobufInputRowSchema.java | 3 +- .../data/input/protobuf/ProtobufReader.java | 2 +- .../protobuf/ProtobufInputFormatTest.java | 5 +- .../common/task/AbstractBatchIndexTask.java | 3 +- .../druid/indexing/common/task/IndexTask.java | 2 - .../common/task/InputSourceProcessor.java | 3 - .../PartialDimensionCardinalityTask.java | 1 - .../PartialDimensionDistributionTask.java | 1 - .../parallel/PartialSegmentGenerateTask.java | 1 - .../batch/parallel/SinglePhaseSubTask.java | 1 - .../druid/indexing/input/InputRowSchemas.java | 7 +-- .../overlord/sampler/InputSourceSampler.java | 4 +- .../SeekableStreamIndexTaskRunner.java | 2 +- .../ClientCompactionTaskQuerySerdeTest.java | 4 +- .../indexing/input/DruidInputSourceTest.java | 6 +- .../indexing/input/InputRowSchemasTest.java | 4 +- ...al_input_source_index_task_schemaless.json | 9 +-- .../incremental/AppendableIndexBuilder.java | 10 --- .../incremental/AppendableIndexSpec.java | 3 - .../segment/incremental/IncrementalIndex.java | 6 +- .../incremental/OnheapIncrementalIndex.java | 39 ++++-------- .../segment/NestedDataColumnIndexerTest.java | 4 +- .../OnheapIncrementalIndexBenchmark.java | 6 +- ...ClientCompactionTaskQueryTuningConfig.java | 4 +- .../DataSourceCompactionConfigTest.java | 2 +- ...erCompactionTaskQueryTuningConfigTest.java | 2 +- .../duty/NewestSegmentFirstPolicyTest.java | 4 +- 43 files changed, 157 insertions(+), 142 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java b/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java index 8bdb37f38c37..dc7d50afd4d4 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java +++ b/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java @@ -35,8 +35,6 @@ public class InputRowSchema private final DimensionsSpec dimensionsSpec; private final ColumnsFilter columnsFilter; - private final boolean discoverNestedColumns; - /** * Set of metric names for further downstream processing by {@link InputSource}. * Empty set if no metric given. @@ -50,22 +48,20 @@ public InputRowSchema( final ColumnsFilter columnsFilter ) { - this(timestampSpec, dimensionsSpec, columnsFilter, ImmutableSet.of(), false); + this(timestampSpec, dimensionsSpec, columnsFilter, ImmutableSet.of()); } public InputRowSchema( final TimestampSpec timestampSpec, final DimensionsSpec dimensionsSpec, final ColumnsFilter columnsFilter, - final Set metricNames, - boolean discoverNestedColumns + final Set metricNames ) { this.timestampSpec = timestampSpec; this.dimensionsSpec = dimensionsSpec; this.columnsFilter = columnsFilter; this.metricNames = metricNames == null ? ImmutableSet.of() : metricNames; - this.discoverNestedColumns = discoverNestedColumns; } @NotNull @@ -97,9 +93,4 @@ public ColumnsFilter getColumnsFilter() { return columnsFilter; } - - public boolean shouldDiscoverNestedColumns() - { - return discoverNestedColumns; - } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java index 6e412c30ad8a..d04914591d99 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java @@ -49,7 +49,9 @@ public class DimensionsSpec private final Map dimensionSchemaMap; private final boolean includeAllDimensions; - public static final DimensionsSpec EMPTY = new DimensionsSpec(null, null, null, false); + private final boolean useNestedColumnIndexerForSchemaDiscovery; + + public static final DimensionsSpec EMPTY = new DimensionsSpec(null, null, null, false, false); public static List getDefaultSchemas(List dimNames) { @@ -78,7 +80,7 @@ public static Builder builder() public DimensionsSpec(List dimensions) { - this(dimensions, null, null, false); + this(dimensions, null, null, false, null); } @JsonCreator @@ -86,7 +88,8 @@ private DimensionsSpec( @JsonProperty("dimensions") List dimensions, @JsonProperty("dimensionExclusions") List dimensionExclusions, @Deprecated @JsonProperty("spatialDimensions") List spatialDimensions, - @JsonProperty("includeAllDimensions") boolean includeAllDimensions + @JsonProperty("includeAllDimensions") boolean includeAllDimensions, + @JsonProperty("useNestedColumnIndexerForSchemaDiscovery") Boolean useNestedColumnIndexerForSchemaDiscovery ) { this.dimensions = dimensions == null @@ -115,6 +118,8 @@ private DimensionsSpec( dimensionSchemaMap.put(newSchema.getName(), newSchema); } this.includeAllDimensions = includeAllDimensions; + this.useNestedColumnIndexerForSchemaDiscovery = + useNestedColumnIndexerForSchemaDiscovery != null && useNestedColumnIndexerForSchemaDiscovery; } @JsonProperty @@ -135,6 +140,12 @@ public boolean isIncludeAllDimensions() return includeAllDimensions; } + @JsonProperty + public boolean useNestedColumnIndexerForSchemaDiscovery() + { + return useNestedColumnIndexerForSchemaDiscovery; + } + @Deprecated @JsonIgnore public List getSpatialDimensions() @@ -188,7 +199,13 @@ public boolean hasCustomDimensions() @PublicApi public DimensionsSpec withDimensions(List dims) { - return new DimensionsSpec(dims, ImmutableList.copyOf(dimensionExclusions), null, includeAllDimensions); + return new DimensionsSpec( + dims, + ImmutableList.copyOf(dimensionExclusions), + null, + includeAllDimensions, + useNestedColumnIndexerForSchemaDiscovery + ); } public DimensionsSpec withDimensionExclusions(Set dimExs) @@ -197,14 +214,21 @@ public DimensionsSpec withDimensionExclusions(Set dimExs) dimensions, ImmutableList.copyOf(Sets.union(dimensionExclusions, dimExs)), null, - includeAllDimensions + includeAllDimensions, + useNestedColumnIndexerForSchemaDiscovery ); } @Deprecated public DimensionsSpec withSpatialDimensions(List spatials) { - return new DimensionsSpec(dimensions, ImmutableList.copyOf(dimensionExclusions), spatials, includeAllDimensions); + return new DimensionsSpec( + dimensions, + ImmutableList.copyOf(dimensionExclusions), + spatials, + includeAllDimensions, + useNestedColumnIndexerForSchemaDiscovery + ); } private void verify(List spatialDimensions) @@ -243,6 +267,7 @@ public boolean equals(Object o) } DimensionsSpec that = (DimensionsSpec) o; return includeAllDimensions == that.includeAllDimensions + && useNestedColumnIndexerForSchemaDiscovery == that.useNestedColumnIndexerForSchemaDiscovery && Objects.equals(dimensions, that.dimensions) && Objects.equals(dimensionExclusions, that.dimensionExclusions); } @@ -250,7 +275,12 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(dimensions, dimensionExclusions, includeAllDimensions); + return Objects.hash( + dimensions, + dimensionExclusions, + includeAllDimensions, + useNestedColumnIndexerForSchemaDiscovery + ); } @Override @@ -260,6 +290,7 @@ public String toString() "dimensions=" + dimensions + ", dimensionExclusions=" + dimensionExclusions + ", includeAllDimensions=" + includeAllDimensions + + ", useNestedColumnIndexerForSchemaDiscovery=" + useNestedColumnIndexerForSchemaDiscovery + '}'; } @@ -270,6 +301,8 @@ public static final class Builder private List spatialDimensions; private boolean includeAllDimensions; + private boolean useNestedColumnIndexerForSchemaDiscovery; + public Builder setDimensions(List dimensions) { this.dimensions = dimensions; @@ -301,9 +334,21 @@ public Builder setIncludeAllDimensions(boolean includeAllDimensions) return this; } + public Builder setUseNestedColumnIndexerForSchemaDiscovery(boolean useNestedColumnIndexerForSchemaDiscovery) + { + this.useNestedColumnIndexerForSchemaDiscovery = useNestedColumnIndexerForSchemaDiscovery; + return this; + } + public DimensionsSpec build() { - return new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions, includeAllDimensions); + return new DimensionsSpec( + dimensions, + dimensionExclusions, + spatialDimensions, + includeAllDimensions, + useNestedColumnIndexerForSchemaDiscovery + ); } } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java index b89559e1c42f..aa5ecbb08674 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonLineReader.java @@ -61,7 +61,10 @@ public class JsonLineReader extends TextReader super(inputRowSchema, source); this.flattener = ObjectFlatteners.create( flattenSpec, - new JSONFlattenerMaker(keepNullColumns, inputRowSchema.shouldDiscoverNestedColumns()) + new JSONFlattenerMaker( + keepNullColumns, + inputRowSchema.getDimensionsSpec().useNestedColumnIndexerForSchemaDiscovery() + ) ); this.mapper = mapper; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java index 1fd58bbabb6f..eaf42244cb32 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java @@ -82,7 +82,10 @@ public class JsonNodeReader extends IntermediateRowParsingReader this.source = source; this.flattener = ObjectFlatteners.create( flattenSpec, - new JSONFlattenerMaker(keepNullColumns, inputRowSchema.shouldDiscoverNestedColumns()) + new JSONFlattenerMaker( + keepNullColumns, + inputRowSchema.getDimensionsSpec().useNestedColumnIndexerForSchemaDiscovery() + ) ); this.mapper = mapper; this.jsonFactory = new JsonFactory(); diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java index 415f1c1396c7..1b1402f22a12 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java @@ -79,7 +79,10 @@ public class JsonReader extends IntermediateRowParsingReader this.source = source; this.flattener = ObjectFlatteners.create( flattenSpec, - new JSONFlattenerMaker(keepNullColumns, inputRowSchema.shouldDiscoverNestedColumns()) + new JSONFlattenerMaker( + keepNullColumns, + inputRowSchema.getDimensionsSpec().useNestedColumnIndexerForSchemaDiscovery() + ) ); this.mapper = mapper; this.jsonFactory = new JsonFactory(); diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java index b76fd1c636da..3039fbb7c843 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java @@ -70,7 +70,7 @@ public class AvroOCFReader extends IntermediateRowParsingReader false, binaryAsString, extractUnionsByType, - inputRowSchema.shouldDiscoverNestedColumns() + inputRowSchema.getDimensionsSpec().useNestedColumnIndexerForSchemaDiscovery() ) ); } diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java index b55fc2e7859a..12f8a20bab7b 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParsers.java @@ -21,6 +21,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.java.util.common.parsers.JSONPathSpec; @@ -50,9 +51,16 @@ public static ObjectFlattener makeFlattener( flattenSpec = JSONPathSpec.DEFAULT; } + final DimensionsSpec dimensionsSpec = parseSpec.getDimensionsSpec(); + return ObjectFlatteners.create( flattenSpec, - new AvroFlattenerMaker(fromPigAvroStorage, binaryAsString, extractUnionsByType, false) + new AvroFlattenerMaker( + fromPigAvroStorage, + binaryAsString, + extractUnionsByType, + dimensionsSpec != null && dimensionsSpec.useNestedColumnIndexerForSchemaDiscovery() + ) ); } diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java index 9f0907486084..51bc733d4305 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java @@ -66,7 +66,7 @@ public class AvroStreamReader extends IntermediateRowParsingReader this.temporaryDirectory = temporaryDirectory; this.orcStructFlattener = ObjectFlatteners.create( flattenSpec, - new OrcStructFlattenerMaker(binaryAsString, inputRowSchema.shouldDiscoverNestedColumns()) + new OrcStructFlattenerMaker( + binaryAsString, + inputRowSchema.getDimensionsSpec().useNestedColumnIndexerForSchemaDiscovery() + ) ); } 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 ebdc28516ed8..cf8ba19d9b74 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 @@ -432,12 +432,9 @@ public void testNestedColumnSchemaless() throws IOException ); final InputRowSchema schema = new InputRowSchema( new TimestampSpec("ts", "millis", null), - new DimensionsSpec( - ImmutableList.of() - ), + DimensionsSpec.builder().setUseNestedColumnIndexerForSchemaDiscovery(true).build(), ColumnsFilter.all(), - null, - true + null ); final FileEntity entity = new FileEntity(new File("example/orc-file-11-format.orc")); diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java index c532d78a76e0..494f66251633 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java @@ -68,7 +68,10 @@ public class ParquetReader extends IntermediateRowParsingReader this.temporaryDirectory = temporaryDirectory; this.flattener = ObjectFlatteners.create( flattenSpec, - new ParquetGroupFlattenerMaker(binaryAsString, inputRowSchema.shouldDiscoverNestedColumns()) + new ParquetGroupFlattenerMaker( + binaryAsString, + inputRowSchema.getDimensionsSpec().useNestedColumnIndexerForSchemaDiscovery() + ) ); } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java index 8526e43508c0..5c2dd2844f19 100755 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java @@ -32,6 +32,7 @@ import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.avro.AvroFlattenerMaker; import org.apache.druid.data.input.avro.AvroParseSpec; +import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -76,9 +77,16 @@ public ParquetAvroHadoopInputRowParser( flattenSpec = JSONPathSpec.DEFAULT; } + final DimensionsSpec dimensionsSpec = parseSpec.getDimensionsSpec(); + this.recordFlattener = ObjectFlatteners.create( flattenSpec, - new AvroFlattenerMaker(false, this.binaryAsString, this.extractUnionsByType, false) + new AvroFlattenerMaker( + false, + this.binaryAsString, + this.extractUnionsByType, + dimensionsSpec != null && dimensionsSpec.useNestedColumnIndexerForSchemaDiscovery() + ) ); } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java index 50f597a92d39..f3e853968511 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetHadoopInputRowParser.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.ParseSpec; @@ -54,9 +55,13 @@ public ParquetHadoopInputRowParser( } else { flattenSpec = JSONPathSpec.DEFAULT; } + final DimensionsSpec dimensionsSpec = parseSpec.getDimensionsSpec(); this.groupFlattener = ObjectFlatteners.create( flattenSpec, - new ParquetGroupFlattenerMaker(this.binaryAsString, false) + new ParquetGroupFlattenerMaker( + this.binaryAsString, + dimensionsSpec != null && dimensionsSpec.useNestedColumnIndexerForSchemaDiscovery() + ) ); this.parser = new MapInputRowParser(parseSpec); } 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 56b24e9bf89a..950039d06956 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 @@ -169,12 +169,9 @@ public void testNestedColumnSchemalessNestedTestFileNoNested() throws IOExceptio final String file = "example/flattening/test_nested_1.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec( - ImmutableList.of() - ), + DimensionsSpec.builder().setUseNestedColumnIndexerForSchemaDiscovery(false).build(), ColumnsFilter.all(), - null, - false + null ); JSONPathSpec flattenSpec = new JSONPathSpec(true, ImmutableList.of()); InputEntityReader reader = createReader( @@ -209,12 +206,9 @@ public void testNestedColumnSchemalessNestedTestFile() throws IOException final String file = "example/flattening/test_nested_1.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec( - ImmutableList.of() - ), + DimensionsSpec.builder().setUseNestedColumnIndexerForSchemaDiscovery(true).build(), ColumnsFilter.all(), - null, - true + null ); JSONPathSpec flattenSpec = new JSONPathSpec(true, ImmutableList.of()); InputEntityReader reader = createReader( diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java index cf604dab3c45..7c0881b69c82 100644 --- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java +++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java @@ -44,8 +44,7 @@ public ProtobufInputRowSchema(InputRowSchema inputRowSchema) new ProtobufTimestampSpec(inputRowSchema.getTimestampSpec()), inputRowSchema.getDimensionsSpec(), inputRowSchema.getColumnsFilter(), - inputRowSchema.getMetricNames(), - inputRowSchema.shouldDiscoverNestedColumns() + inputRowSchema.getMetricNames() ); } diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java index c16977b535e4..ac77a545dd7b 100644 --- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java +++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java @@ -59,7 +59,7 @@ public class ProtobufReader extends IntermediateRowParsingReader this.inputRowSchema = inputRowSchema; this.recordFlattener = ObjectFlatteners.create( flattenSpec, - new ProtobufFlattenerMaker(inputRowSchema.shouldDiscoverNestedColumns()) + new ProtobufFlattenerMaker(inputRowSchema.getDimensionsSpec().useNestedColumnIndexerForSchemaDiscovery()) ); this.source = source; this.protobufBytesDecoder = protobufBytesDecoder; 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 7c273031db99..9255e1e7ebd2 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 @@ -345,10 +345,9 @@ public void testParseNestedDataSchemaless() throws Exception InputEntityReader reader = protobufInputFormat.createReader( new InputRowSchema( timestampSpec, - new DimensionsSpec(Collections.emptyList()), + DimensionsSpec.builder().setUseNestedColumnIndexerForSchemaDiscovery(true).build(), null, - null, - true + null ), entity, null diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index d1dd277e571d..6e6e12f2ad87 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -202,7 +202,6 @@ public void stopGracefully(TaskConfig taskConfig) public static FilteringCloseableInputRowIterator inputSourceReader( File tmpDir, DataSchema dataSchema, - TuningConfig tuningConfig, InputSource inputSource, @Nullable InputFormat inputFormat, Predicate rowFilter, @@ -212,7 +211,7 @@ public static FilteringCloseableInputRowIterator inputSourceReader( { final InputSourceReader inputSourceReader = dataSchema.getTransformSpec().decorate( inputSource.reader( - InputRowSchemas.fromDataSchema(dataSchema, tuningConfig), + InputRowSchemas.fromDataSchema(dataSchema), inputFormat, tmpDir ) 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 b490462da358..288107d0b944 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 @@ -767,7 +767,6 @@ private Map> collectIntervalsAndShardSp try (final CloseableIterator inputRowIterator = AbstractBatchIndexTask.inputSourceReader( tmpDir, ingestionSchema.getDataSchema(), - ingestionSchema.getTuningConfig(), inputSource, inputSource.needsFormat() ? getInputFormat(ingestionSchema) : null, rowFilter, @@ -922,7 +921,6 @@ private TaskStatus generateAndPublishSegments( SegmentsAndCommitMetadata pushed = InputSourceProcessor.process( dataSchema, - tuningConfig, driver, partitionsSpec, inputSource, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java index b884487c78ec..e91f5d5fb0d1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java @@ -34,7 +34,6 @@ import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; @@ -61,7 +60,6 @@ public class InputSourceProcessor */ public static SegmentsAndCommitMetadata process( DataSchema dataSchema, - TuningConfig tuningConfig, BatchAppenderatorDriver driver, PartitionsSpec partitionsSpec, InputSource inputSource, @@ -84,7 +82,6 @@ public static SegmentsAndCommitMetadata process( final CloseableIterator inputRowIterator = AbstractBatchIndexTask.inputSourceReader( tmpDir, dataSchema, - tuningConfig, inputSource, inputFormat, AbstractBatchIndexTask.defaultRowFilter(granularitySpec), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java index eabf738c9353..0b4def915ac0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java @@ -175,7 +175,6 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception final CloseableIterator inputRowIterator = AbstractBatchIndexTask.inputSourceReader( toolbox.getIndexingTmpDir(), dataSchema, - tuningConfig, inputSource, inputFormat, determineIntervals ? Objects::nonNull : AbstractBatchIndexTask.defaultRowFilter(granularitySpec), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java index d63d00685c6e..e491e0244250 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java @@ -222,7 +222,6 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception final CloseableIterator inputRowIterator = AbstractBatchIndexTask.inputSourceReader( toolbox.getIndexingTmpDir(), dataSchema, - tuningConfig, inputSource, inputFormat, determineIntervals ? Objects::nonNull : AbstractBatchIndexTask.defaultRowFilter(granularitySpec), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index 96db7670ebb0..03a66d1fd972 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -207,7 +207,6 @@ private List generateSegments( final SegmentsAndCommitMetadata pushed = InputSourceProcessor.process( dataSchema, - tuningConfig, driver, partitionsSpec, inputSource, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 89c306db68fc..1b83562d7c7c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -412,7 +412,6 @@ private Set generateAndPushSegments( final CloseableIterator inputRowIterator = AbstractBatchIndexTask.inputSourceReader( tmpDir, dataSchema, - tuningConfig, inputSource, inputSource.needsFormat() ? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema) : null, inputRow -> { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java index 74d343ef26a9..c895eb14b711 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java @@ -26,11 +26,9 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.transform.Transform; import org.apache.druid.segment.transform.TransformSpec; -import javax.annotation.Nullable; import java.util.Arrays; import java.util.HashSet; import java.util.Set; @@ -49,7 +47,7 @@ private InputRowSchemas() /** * Creates an {@link InputRowSchema} from a given {@link DataSchema}. */ - public static InputRowSchema fromDataSchema(final DataSchema dataSchema, @Nullable final TuningConfig tuningConfig) + public static InputRowSchema fromDataSchema(final DataSchema dataSchema) { return new InputRowSchema( dataSchema.getTimestampSpec(), @@ -62,8 +60,7 @@ public static InputRowSchema fromDataSchema(final DataSchema dataSchema, @Nullab ), Arrays.stream(dataSchema.getAggregators()) .map(AggregatorFactory::getName) - .collect(Collectors.toSet()), - tuningConfig != null && tuningConfig.getAppendableIndexSpec().useNestedColumnIndexerForSchemaDiscovery() + .collect(Collectors.toSet()) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java index 2d937141b8b0..33594e481476 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java @@ -121,7 +121,6 @@ public SamplerResponse sample( final InputSourceReader reader = buildReader( nonNullSamplerConfig, nonNullDataSchema, - tuningConfig, inputSource, inputFormat, tempDir @@ -253,13 +252,12 @@ public SamplerResponse sample( private InputSourceReader buildReader( SamplerConfig samplerConfig, DataSchema dataSchema, - TuningConfig tuningConfig, InputSource inputSource, @Nullable InputFormat inputFormat, File tempDir ) { - final InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(dataSchema, tuningConfig); + final InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(dataSchema); InputSourceReader reader = inputSource.reader(inputRowSchema, inputFormat, tempDir); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index db581388507f..e509699d5ced 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -247,7 +247,7 @@ public SeekableStreamIndexTaskRunner( this.task = task; this.ioConfig = task.getIOConfig(); this.tuningConfig = task.getTuningConfig(); - this.inputRowSchema = InputRowSchemas.fromDataSchema(task.getDataSchema(), tuningConfig); + this.inputRowSchema = InputRowSchemas.fromDataSchema(task.getDataSchema()); this.inputFormat = ioConfig.getInputFormat(); this.parser = parser; this.authorizerMapper = authorizerMapper; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index b3db2b71d1b9..c6b6d6fdf0a4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -252,7 +252,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException new ParallelIndexTuningConfig( null, null, - new OnheapIncrementalIndex.Spec(true, false), + new OnheapIncrementalIndex.Spec(true), 40000, 2000L, null, @@ -316,7 +316,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException ), new ClientCompactionTaskQueryTuningConfig( 100, - new OnheapIncrementalIndex.Spec(true, false), + new OnheapIncrementalIndex.Spec(true), 40000, 2000L, 30000L, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java index a5e9a1e81620..2989415a5e6e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java @@ -244,8 +244,7 @@ public void testReaderColumnsFilterWithMetricGiven() DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")) ), originalColumnsFilter, - ImmutableSet.of(metricName), - true + ImmutableSet.of(metricName) ); DruidInputSource druidInputSource = new DruidInputSource( datasource, @@ -280,8 +279,7 @@ public void testReaderColumnsFilterWithNoMetricGiven() DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")) ), originalColumnsFilter, - ImmutableSet.of(), - true + ImmutableSet.of() ); DruidInputSource druidInputSource = new DruidInputSource( datasource, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java index 8bb582d1e4e9..991a5950f9aa 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java @@ -134,7 +134,7 @@ public void testFromDataSchema() null ); - InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema, null); + InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema); Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec()); Assert.assertEquals(dimensionsSpec.getDimensions(), inputRowSchema.getDimensionsSpec().getDimensions()); Assert.assertEquals(dimensionsSpec.getDimensionNames(), inputRowSchema.getDimensionsSpec().getDimensionNames()); @@ -163,7 +163,7 @@ public void testFromDataSchemaWithNoAggregator() null ); - InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema, null); + InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema); Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec()); Assert.assertEquals(dimensionsSpec.getDimensions(), inputRowSchema.getDimensionsSpec().getDimensions()); Assert.assertEquals(dimensionsSpec.getDimensionNames(), inputRowSchema.getDimensionsSpec().getDimensionNames()); diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task_schemaless.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task_schemaless.json index 3e25773759e6..bc96c730ee77 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task_schemaless.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task_schemaless.json @@ -7,7 +7,8 @@ "column": "timestamp" }, "dimensionsSpec": { - "dimensions": [] + "dimensions": [], + "useNestedColumnIndexerForSchemaDiscovery": %%USE_NESTED_COLUMN_INDEXER%% }, "metricsSpec": [], "granularitySpec": { @@ -35,11 +36,7 @@ "maxNumFiles": 1 }, "forceGuaranteedRollup": %%FORCE_GUARANTEED_ROLLUP%%, - "partitionsSpec": %%PARTITIONS_SPEC%%, - "appendableIndexSpec": { - "type": "onheap", - "useNestedColumnIndexerForSchemaDiscovery": %%USE_NESTED_COLUMN_INDEXER%% - } + "partitionsSpec": %%PARTITIONS_SPEC%% } } } \ No newline at end of file diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java index 887c931204e8..1269fe1e6b3c 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java @@ -42,8 +42,6 @@ public abstract class AppendableIndexBuilder protected boolean preserveExistingMetrics = false; protected boolean useMaxMemoryEstimates = true; - protected boolean useNestedColumnIndexerForSchemaDiscovery = false; - protected final Logger log = new Logger(this.getClass()); public AppendableIndexBuilder setIndexSchema(final IncrementalIndexSchema incrementalIndexSchema) @@ -132,14 +130,6 @@ public AppendableIndexBuilder setUseMaxMemoryEstimates(final boolean useMaxMemor return this; } - public AppendableIndexBuilder setUseNestedColumnIndexerForSchemaDiscovery( - boolean useNestedColumnIndexerForSchemaDiscovery - ) - { - this.useNestedColumnIndexerForSchemaDiscovery = useNestedColumnIndexerForSchemaDiscovery; - return this; - } - public void validate() { if (maxRowCount <= 0) { diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java index 3af6bfa4cbe8..67cdabdf5673 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java @@ -32,7 +32,4 @@ public interface AppendableIndexSpec // Returns the default max bytes in memory for this index. long getDefaultMaxBytesInMemory(); - - @SuppressWarnings("unused") - boolean useNestedColumnIndexerForSchemaDiscovery(); } 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 0085b58859e2..ddb52c309cb8 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 @@ -276,8 +276,7 @@ protected IncrementalIndex( final boolean deserializeComplexMetrics, final boolean concurrentEventAdd, final boolean preserveExistingMetrics, - final boolean useMaxMemoryEstimates, - final boolean useNestedColumnIndexerForSchemaDiscovery + final boolean useMaxMemoryEstimates ) { this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); @@ -289,7 +288,8 @@ protected IncrementalIndex( this.deserializeComplexMetrics = deserializeComplexMetrics; this.preserveExistingMetrics = preserveExistingMetrics; this.useMaxMemoryEstimates = useMaxMemoryEstimates; - this.useNestedColumnIndexerForSchemaDiscovery = useNestedColumnIndexerForSchemaDiscovery; + this.useNestedColumnIndexerForSchemaDiscovery = incrementalIndexSchema.getDimensionsSpec() + .useNestedColumnIndexerForSchemaDiscovery(); this.timeAndMetricsColumnCapabilities = new HashMap<>(); this.metricDescs = Maps.newLinkedHashMap(); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java index 1912136a0734..35ccd8961da5 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java @@ -126,11 +126,16 @@ public class OnheapIncrementalIndex extends IncrementalIndex // preserveExistingMetrics should only be set true for DruidInputSource since that is the only case where we can have existing metrics // This is currently only use by auto compaction and should not be use for anything else. boolean preserveExistingMetrics, - boolean useMaxMemoryEstimates, - boolean useNestedColumnIndexerSchemaDiscovery + boolean useMaxMemoryEstimates ) { - super(incrementalIndexSchema, deserializeComplexMetrics, concurrentEventAdd, preserveExistingMetrics, useMaxMemoryEstimates, useNestedColumnIndexerSchemaDiscovery); + super( + incrementalIndexSchema, + deserializeComplexMetrics, + concurrentEventAdd, + preserveExistingMetrics, + useMaxMemoryEstimates + ); this.maxRowCount = maxRowCount; this.maxBytesInMemory = maxBytesInMemory == 0 ? Long.MAX_VALUE : maxBytesInMemory; this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) @@ -657,8 +662,7 @@ protected OnheapIncrementalIndex buildInner() maxRowCount, maxBytesInMemory, preserveExistingMetrics, - useMaxMemoryEstimates, - useNestedColumnIndexerForSchemaDiscovery + useMaxMemoryEstimates ); } } @@ -666,7 +670,6 @@ protected OnheapIncrementalIndex buildInner() public static class Spec implements AppendableIndexSpec { private static final boolean DEFAULT_PRESERVE_EXISTING_METRICS = false; - private static final boolean DEFAULT_USE_NESTED_COLUMN_INDEXER_SCHEMA_DISCOVERY = false; public static final String TYPE = "onheap"; // When set to true, for any row that already has metric (with the same name defined in metricSpec), @@ -676,26 +679,19 @@ public static class Spec implements AppendableIndexSpec // This is currently only use by auto compaction and should not be use for anything else. final boolean preserveExistingMetrics; - final boolean useNestedColumnIndexerForSchemaDiscovery; - public Spec() { this.preserveExistingMetrics = DEFAULT_PRESERVE_EXISTING_METRICS; - this.useNestedColumnIndexerForSchemaDiscovery = DEFAULT_USE_NESTED_COLUMN_INDEXER_SCHEMA_DISCOVERY; } @JsonCreator public Spec( - final @JsonProperty("preserveExistingMetrics") @Nullable Boolean preserveExistingMetrics, - final @JsonProperty("useNestedColumnIndexerForSchemaDiscovery") @Nullable Boolean useNestedColumnIndexerForSchemaDiscovery + final @JsonProperty("preserveExistingMetrics") @Nullable Boolean preserveExistingMetrics ) { this.preserveExistingMetrics = preserveExistingMetrics != null ? preserveExistingMetrics : DEFAULT_PRESERVE_EXISTING_METRICS; - this.useNestedColumnIndexerForSchemaDiscovery = useNestedColumnIndexerForSchemaDiscovery != null - ? useNestedColumnIndexerForSchemaDiscovery - : DEFAULT_USE_NESTED_COLUMN_INDEXER_SCHEMA_DISCOVERY; } @JsonProperty @@ -707,8 +703,7 @@ public boolean isPreserveExistingMetrics() @Override public AppendableIndexBuilder builder() { - return new Builder().setPreserveExistingMetrics(preserveExistingMetrics) - .setUseNestedColumnIndexerForSchemaDiscovery(useNestedColumnIndexerForSchemaDiscovery); + return new Builder().setPreserveExistingMetrics(preserveExistingMetrics); } @Override @@ -720,13 +715,6 @@ public long getDefaultMaxBytesInMemory() return JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes() / 6; } - @JsonProperty - @Override - public boolean useNestedColumnIndexerForSchemaDiscovery() - { - return useNestedColumnIndexerForSchemaDiscovery; - } - @Override public boolean equals(Object o) { @@ -737,14 +725,13 @@ public boolean equals(Object o) return false; } Spec spec = (Spec) o; - return preserveExistingMetrics == spec.preserveExistingMetrics && - useNestedColumnIndexerForSchemaDiscovery == spec.useNestedColumnIndexerForSchemaDiscovery; + return preserveExistingMetrics == spec.preserveExistingMetrics; } @Override public int hashCode() { - return Objects.hash(preserveExistingMetrics, useNestedColumnIndexerForSchemaDiscovery); + return Objects.hash(preserveExistingMetrics); } } } diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java index 54bfe1dcb59f..e5a1c41bc422 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java @@ -46,7 +46,6 @@ import org.junit.Test; import javax.annotation.Nonnull; -import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -548,13 +547,12 @@ private static IncrementalIndex makeIncrementalIndex(long minTimestamp) new TimestampSpec(TIME_COL, "millis", null), Granularities.NONE, VirtualColumns.EMPTY, - new DimensionsSpec(Collections.emptyList()), + DimensionsSpec.builder().setUseNestedColumnIndexerForSchemaDiscovery(true).build(), new AggregatorFactory[0], false ) ) .setMaxRowCount(1000) - .setUseNestedColumnIndexerForSchemaDiscovery(true) .build(); return index; } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index dad2a4c2134d..02d43e275061 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -125,8 +125,7 @@ public MapIncrementalIndex( maxRowCount, maxBytesInMemory, false, - true, - false + true ); } @@ -150,8 +149,7 @@ public MapIncrementalIndex( maxRowCount, maxBytesInMemory, false, - true, - false + true ); } diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryTuningConfig.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryTuningConfig.java index ad8827f2a95b..7b1a7c54682b 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryTuningConfig.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryTuningConfig.java @@ -88,7 +88,7 @@ public static ClientCompactionTaskQueryTuningConfig from( if (userCompactionTaskQueryTuningConfig == null) { return new ClientCompactionTaskQueryTuningConfig( maxRowsPerSegment, - new OnheapIncrementalIndex.Spec(preserveExistingMetrics, false), + new OnheapIncrementalIndex.Spec(preserveExistingMetrics), null, null, null, @@ -111,7 +111,7 @@ public static ClientCompactionTaskQueryTuningConfig from( } else { AppendableIndexSpec appendableIndexSpecToUse = userCompactionTaskQueryTuningConfig.getAppendableIndexSpec() != null ? userCompactionTaskQueryTuningConfig.getAppendableIndexSpec() - : new OnheapIncrementalIndex.Spec(preserveExistingMetrics, false); + : new OnheapIncrementalIndex.Spec(preserveExistingMetrics); return new ClientCompactionTaskQueryTuningConfig( maxRowsPerSegment, appendableIndexSpecToUse, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java index 4a48f3280c7c..a0f4cf9a6101 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java @@ -259,7 +259,7 @@ public void testSerdeUserCompactionTuningConfigWithAppendableIndexSpec() throws { final UserCompactionTaskQueryTuningConfig tuningConfig = new UserCompactionTaskQueryTuningConfig( 40000, - new OnheapIncrementalIndex.Spec(true, false), + new OnheapIncrementalIndex.Spec(true), 2000L, null, new SegmentsSplitHintSpec(new HumanReadableBytes(100000L), null), diff --git a/server/src/test/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfigTest.java index 947328722964..01c889ad2c3c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfigTest.java @@ -78,7 +78,7 @@ public void testSerde() throws IOException { final UserCompactionTaskQueryTuningConfig tuningConfig = new UserCompactionTaskQueryTuningConfig( 40000, - new OnheapIncrementalIndex.Spec(true, false), + new OnheapIncrementalIndex.Spec(true), 2000L, null, new SegmentsSplitHintSpec(new HumanReadableBytes(42L), null), diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java index b1f3ad99527c..73abcb8d5eac 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java @@ -1523,7 +1523,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() null, new UserCompactionTaskQueryTuningConfig( null, - new OnheapIncrementalIndex.Spec(true, false), + new OnheapIncrementalIndex.Spec(true), null, 1000L, null, @@ -1558,7 +1558,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() null, new UserCompactionTaskQueryTuningConfig( null, - new OnheapIncrementalIndex.Spec(false, false), + new OnheapIncrementalIndex.Spec(false), null, 1000L, null, From 38a15ca996e1e3cc13936886b9f15501efd3534d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 17 Jan 2023 20:59:41 -0800 Subject: [PATCH 08/15] revert --- .../overlord/sampler/IndexTaskSamplerSpec.java | 6 +----- .../overlord/sampler/InputSourceSampler.java | 4 +--- .../indexing/overlord/sampler/SamplerConfig.java | 14 +++++++------- .../seekablestream/SeekableStreamSamplerSpec.java | 2 +- 4 files changed, 10 insertions(+), 16 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java index 6662fc6e9ddd..4140e014ef7a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java @@ -33,7 +33,6 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.TuningConfig; import javax.annotation.Nullable; @@ -51,8 +50,6 @@ public class IndexTaskSamplerSpec implements SamplerSpec private final SamplerConfig samplerConfig; private final InputSourceSampler inputSourceSampler; - private final TuningConfig tuningConfig; - @JsonCreator public IndexTaskSamplerSpec( @JsonProperty("spec") final IndexTask.IndexIngestionSpec ingestionSpec, @@ -91,12 +88,11 @@ public IndexTaskSamplerSpec( this.samplerConfig = samplerConfig; this.inputSourceSampler = inputSourceSampler; - this.tuningConfig = ingestionSpec.getTuningConfig(); } @Override public SamplerResponse sample() { - return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig, tuningConfig); + return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java index 33594e481476..c91d8434c21b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java @@ -49,7 +49,6 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.TuningConfig; import javax.annotation.Nullable; import javax.inject.Inject; @@ -98,8 +97,7 @@ public SamplerResponse sample( // inputFormat can be null only if inputSource.needsFormat() = false or parser is specified. @Nullable final InputFormat inputFormat, @Nullable final DataSchema dataSchema, - @Nullable final SamplerConfig samplerConfig, - @Nullable final TuningConfig tuningConfig + @Nullable final SamplerConfig samplerConfig ) { Preconditions.checkNotNull(inputSource, "inputSource required"); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java index 75ddb341d2c1..739fcafb466d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java @@ -92,13 +92,13 @@ public int getTimeoutMs() /** * Maximum number of bytes in memory that the {@link org.apache.druid.segment.incremental.IncrementalIndex} used by - * {@link InputSourceSampler#sample(InputSource, InputFormat, DataSchema, SamplerConfig, org.apache.druid.segment.indexing.TuningConfig)} - * will be allowed to accumulate before aborting sampling. Particularly useful for limiting footprint of sample - * operations as well as overall response size from sample requests. However, it is not directly correlated to - * response size since it also contains the "raw" input data, so actual responses will likely be at least twice the - * size of this value, depending on factors such as number of transforms, aggregations in the case of rollup, whether - * all columns of the input are present in the dimension spec, and so on. If it is preferred to control client - * response size, use {@link SamplerConfig#getMaxClientResponseBytes()} instead. + * {@link InputSourceSampler#sample(InputSource, InputFormat, DataSchema, SamplerConfig}) will be allowed to + * accumulate before aborting sampling. Particularly useful for limiting footprint of sample operations as well as + * overall response size from sample requests. However, it is not directly correlated to response size since it also + * contains the "raw" input data, so actual responses will likely be at least twice the size of this value, depending + * on factors such as number of transforms, aggregations in the case of rollup, whether all columns of the input are + * present in the dimension spec, and so on. If it is preferred to control client response size, use + * {@link SamplerConfig#getMaxClientResponseBytes()} instead. */ public long getMaxBytesInMemory() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java index 5dde26df2e9f..f27cdb50fcc6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java @@ -115,7 +115,7 @@ public SamplerResponse sample() ); } - return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig, tuningConfig == null ? null : tuningConfig.convertToTaskTuningConfig()); + return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig); } protected abstract RecordSupplier createRecordSupplier(); From 55e27f00326f0d63de3e45af7168446d4f9e034e Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 17 Jan 2023 21:01:38 -0800 Subject: [PATCH 09/15] fix build --- .../sampler/IndexTaskSamplerSpecTest.java | 5 +-- .../sampler/InputSourceSamplerTest.java | 39 ++++++++----------- 2 files changed, 18 insertions(+), 26 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java index a4424440e7e8..b305cd989ede 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java @@ -30,7 +30,6 @@ import org.apache.druid.guice.FirehoseModule; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.TuningConfig; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; @@ -97,7 +96,6 @@ public void testSerde() throws IOException Capture capturedInputFormat = EasyMock.newCapture(); Capture capturedDataSchema = EasyMock.newCapture(); Capture capturedSamplerConfig = EasyMock.newCapture(); - Capture capturedTuningConfig = EasyMock.newCapture(); IndexTaskSamplerSpec spec = MAPPER.readValue(json, IndexTaskSamplerSpec.class); @@ -105,8 +103,7 @@ public void testSerde() throws IOException EasyMock.capture(capturedInputSource), EasyMock.capture(capturedInputFormat), EasyMock.capture(capturedDataSchema), - EasyMock.capture(capturedSamplerConfig), - EasyMock.capture(capturedTuningConfig) + EasyMock.capture(capturedSamplerConfig) )).andReturn(new SamplerResponse(0, 0, null)); replayAll(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java index 5081fdd8bfe9..b1e039a4defb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java @@ -169,7 +169,7 @@ public void testNoParams() expectedException.expect(NullPointerException.class); expectedException.expectMessage("inputSource required"); - inputSourceSampler.sample(null, null, null, null, null); + inputSourceSampler.sample(null, null, null, null); } @Test @@ -248,8 +248,7 @@ public void testNoDataSchemaNumRows() inputSource, createInputFormat(), null, - new SamplerConfig(3, null, null, null), - null + new SamplerConfig(3, null, null, null) ); Assert.assertEquals(3, response.getNumRowsRead()); @@ -296,7 +295,7 @@ public void testMissingValueTimestampSpec() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(6, response.getNumRowsIndexed()); @@ -405,7 +404,7 @@ public void testWithTimestampSpec() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -505,7 +504,7 @@ public void testWithDimensionSpec() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -611,7 +610,7 @@ public void testWithNoRollup() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -722,7 +721,7 @@ public void testWithRollup() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -805,7 +804,7 @@ public void testWithMoreRollup() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -876,7 +875,7 @@ public void testWithTransformsAutoDimensions() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -965,7 +964,7 @@ public void testWithTransformsDimensionsSpec() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -1033,7 +1032,7 @@ public void testWithFilter() throws IOException final InputSource inputSource = createInputSource(getTestRows(), dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); Assert.assertEquals(5, response.getNumRowsRead()); Assert.assertEquals(4, response.getNumRowsIndexed()); @@ -1120,7 +1119,7 @@ public void testIndexParseException() throws IOException final InputSource inputSource = createInputSource(inputTestRows, dataSchema); final InputFormat inputFormat = createInputFormat(); - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null, null); + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); Assert.assertEquals(7, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); @@ -1234,8 +1233,7 @@ public void testMultipleJsonStringInOneBlock() throws IOException new RecordSupplierInputSource("topicName", new TestRecordSupplier(jsonBlockList), true, 3000), createInputFormat(), dataSchema, - new SamplerConfig(200, 3000/*default timeout is 10s, shorten it to speed up*/, null, null), - null + new SamplerConfig(200, 3000/*default timeout is 10s, shorten it to speed up*/, null, null) ); // @@ -1331,7 +1329,7 @@ public InputSourceReader reader( throw new RuntimeException(); } }; - inputSourceSampler.sample(failingReaderInputSource, null, null, null, null); + inputSourceSampler.sample(failingReaderInputSource, null, null, null); } @Test @@ -1360,8 +1358,7 @@ public void testRowLimiting() throws IOException inputSource, inputFormat, dataSchema, - new SamplerConfig(4, null, null, null), - null + new SamplerConfig(4, null, null, null) ); Assert.assertEquals(4, response.getNumRowsRead()); @@ -1396,8 +1393,7 @@ public void testMaxBytesInMemoryLimiting() throws IOException inputSource, inputFormat, dataSchema, - new SamplerConfig(null, null, HumanReadableBytes.valueOf(256), null), - null + new SamplerConfig(null, null, HumanReadableBytes.valueOf(256), null) ); Assert.assertEquals(4, response.getNumRowsRead()); @@ -1431,8 +1427,7 @@ public void testMaxClientResponseBytesLimiting() throws IOException inputSource, inputFormat, dataSchema, - new SamplerConfig(null, null, null, HumanReadableBytes.valueOf(300)), - null + new SamplerConfig(null, null, null, HumanReadableBytes.valueOf(300)) ); Assert.assertEquals(4, response.getNumRowsRead()); From 922a3b18062f282b117b24b3b0f867abc5d6db4b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 17 Jan 2023 21:04:29 -0800 Subject: [PATCH 10/15] actually fix build this time --- .../indexing/overlord/sampler/CsvInputSourceSamplerTest.java | 1 - .../druid/indexing/overlord/sampler/InputSourceSamplerTest.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java index d209c6d15e7c..464579764d11 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java @@ -67,7 +67,6 @@ public void testCSVColumnAllNull() inputSource, inputFormat, dataSchema, - null, null ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java index b1e039a4defb..cbd58d1adfbb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java @@ -176,7 +176,7 @@ public void testNoParams() public void testNoDataSchema() { final InputSource inputSource = createInputSource(getTestRows(), null); - final SamplerResponse response = inputSourceSampler.sample(inputSource, createInputFormat(), null, null, null); + final SamplerResponse response = inputSourceSampler.sample(inputSource, createInputFormat(), null, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(0, response.getNumRowsIndexed()); From a830c1002b847fb32b93096e5a77c2a18ca868ef Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 17 Jan 2023 21:18:13 -0800 Subject: [PATCH 11/15] adjust --- .../java/org/apache/druid/data/input/impl/DimensionsSpec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java index d04914591d99..6dfaa9ad3d0a 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java @@ -51,7 +51,7 @@ public class DimensionsSpec private final boolean useNestedColumnIndexerForSchemaDiscovery; - public static final DimensionsSpec EMPTY = new DimensionsSpec(null, null, null, false, false); + public static final DimensionsSpec EMPTY = new DimensionsSpec(null, null, null, false, null); public static List getDefaultSchemas(List dimNames) { From 514bd3b4f46ade28aeb79e06674049152ae74ed3 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 17 Jan 2023 22:20:03 -0800 Subject: [PATCH 12/15] javadoc imports are not real imports i guess --- .../overlord/sampler/SamplerConfig.java | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java index 739fcafb466d..4e3534996ad9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java @@ -22,10 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.data.input.InputFormat; -import org.apache.druid.data.input.InputSource; import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.segment.indexing.DataSchema; import javax.annotation.Nullable; @@ -92,13 +89,13 @@ public int getTimeoutMs() /** * Maximum number of bytes in memory that the {@link org.apache.druid.segment.incremental.IncrementalIndex} used by - * {@link InputSourceSampler#sample(InputSource, InputFormat, DataSchema, SamplerConfig}) will be allowed to - * accumulate before aborting sampling. Particularly useful for limiting footprint of sample operations as well as - * overall response size from sample requests. However, it is not directly correlated to response size since it also - * contains the "raw" input data, so actual responses will likely be at least twice the size of this value, depending - * on factors such as number of transforms, aggregations in the case of rollup, whether all columns of the input are - * present in the dimension spec, and so on. If it is preferred to control client response size, use - * {@link SamplerConfig#getMaxClientResponseBytes()} instead. + * {@link InputSourceSampler#sample(org.apache.druid.data.input.InputSource, org.apache.druid.data.input.InputFormat, org.apache.druid.segment.indexing.DataSchema, SamplerConfig}) + * will be allowed to accumulate before aborting sampling. Particularly useful for limiting footprint of sample + * operations as well as overall response size from sample requests. However, it is not directly correlated to + * response size since it also contains the "raw" input data, so actual responses will likely be at least twice the + * size of this value, depending on factors such as number of transforms, aggregations in the case of rollup, whether + * all columns of the input are present in the dimension spec, and so on. If it is preferred to control client + * response size, use {@link SamplerConfig#getMaxClientResponseBytes()} instead. */ public long getMaxBytesInMemory() { From 430c66ea26b96e61409d1f74264746dea06f9692 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 17 Jan 2023 22:35:13 -0800 Subject: [PATCH 13/15] fix javadoc --- .../druid/java/util/common/parsers/ObjectFlatteners.java | 4 +++- .../apache/druid/indexing/overlord/sampler/SamplerConfig.java | 4 ---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java index f370a2ec6b10..b2040d73db21 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java @@ -218,7 +218,9 @@ public interface FlattenerMaker { JsonProvider getJsonProvider(); /** - * List all "root" fields, optionally filtering to include only fields that contain primitive and lists of primitive values + * List all "root" fields. If + * {@link org.apache.druid.data.input.impl.DimensionsSpec#useNestedColumnIndexerForSchemaDiscovery} is false, this + * method should filter fields to include only fields that contain primitive and lists of primitive values */ Iterable discoverRootFields(T obj); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java index 4e3534996ad9..8824decc05fb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java @@ -32,13 +32,9 @@ public class SamplerConfig private static final int MAX_NUM_ROWS = 5000; private static final int DEFAULT_TIMEOUT_MS = 10000; - - private final int numRows; private final int timeoutMs; - private final long maxBytesInMemory; - private final long maxClientResponseBytes; @JsonCreator From 2b32878c6ac61bd5a518fd066c32b7df839ee874 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 17 Jan 2023 22:48:12 -0800 Subject: [PATCH 14/15] revert --- ...SeekableStreamIndexTaskRunnerAuthTest.java | 37 +------------------ 1 file changed, 1 insertion(+), 36 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java index 79a9357f54c5..db51a416fc90 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java @@ -54,7 +54,6 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; -import java.io.File; import java.util.Collections; import java.util.List; import java.util.Map; @@ -118,41 +117,7 @@ public Authorizer getAuthorizer(String name) null, null ); - SeekableStreamIndexTaskTuningConfig tuningConfig = new SeekableStreamIndexTaskTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) - { - @Override - public SeekableStreamIndexTaskTuningConfig withBasePersistDirectory(File dir) - { - return null; - } - - @Override - public String toString() - { - return null; - } - }; + SeekableStreamIndexTaskTuningConfig tuningConfig = mock(SeekableStreamIndexTaskTuningConfig.class); SeekableStreamIndexTaskIOConfig ioConfig = new TestSeekableStreamIndexTaskIOConfig(); // Initiliaze task and task runner From 04d5d66f68bf5ab16cb2aae08936e74a03c87559 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 18 Jan 2023 02:39:58 -0800 Subject: [PATCH 15/15] add test --- .../OnheapIncrementalIndexTest.java | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) create mode 100644 processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java new file mode 100644 index 000000000000..07ed6f7502fa --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java @@ -0,0 +1,46 @@ +/* + * 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.incremental; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class OnheapIncrementalIndexTest +{ + private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); + + @Test + public void testSerde() throws JsonProcessingException + { + OnheapIncrementalIndex.Spec spec = new OnheapIncrementalIndex.Spec(true); + Assert.assertEquals(spec, MAPPER.readValue(MAPPER.writeValueAsString(spec), OnheapIncrementalIndex.Spec.class)); + } + @Test + public void testSpecEqualsAndHashCode() + { + EqualsVerifier.forClass(OnheapIncrementalIndex.Spec.class) + .usingGetClass() + .verify(); + } +}