From 3c8b88565ef6700627b10e34602623b907b76966 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 16 Mar 2026 11:57:17 -0700 Subject: [PATCH 1/3] remove deprecated InputRowParser from SeekableStreamIndexTaskRunner (kafka, kinesis, etc) --- .../benchmark/ProtobufParserBenchmark.java | 157 ------ docs/development/extensions-core/avro.md | 2 - docs/ingestion/data-formats.md | 207 +------ docs/ingestion/ingestion-spec.md | 4 +- docs/ingestion/kafka-ingestion.md | 7 +- docs/ingestion/kinesis-ingestion.md | 5 +- .../indexing/KafkaIndexDataFormatsTest.java | 22 - .../StreamIndexDataFormatsTestBase.java | 278 +--------- .../kinesis/KinesisDataFormatsTest.java | 18 - .../rabbitstream/RabbitStreamIndexTask.java | 6 +- .../RabbitStreamIndexTaskRunner.java | 9 +- .../data/input/AvroStreamInputRowParser.java | 135 ----- .../data/input/avro/AvroExtensionsModule.java | 2 - .../input/AvroHadoopInputRowParserTest.java | 30 +- .../data/input/AvroStreamInputFormatTest.java | 235 +++++++- .../input/AvroStreamInputRowParserTest.java | 514 ------------------ .../input/avro/AvroFlattenerMakerTest.java | 37 +- .../data/input/avro/AvroOCFReaderTest.java | 4 +- .../InlineSchemaAvroBytesDecoderTest.java | 8 +- .../InlineSchemasAvroBytesDecoderTest.java | 10 +- ...hemaRegistryBasedAvroBytesDecoderTest.java | 10 +- .../druid/indexing/kafka/KafkaIndexTask.java | 6 +- .../indexing/kafka/KafkaIndexTaskRunner.java | 9 +- .../indexing/kafka/KafkaSamplerSpecTest.java | 54 -- .../indexing/kinesis/KinesisIndexTask.java | 6 +- .../kinesis/KinesisIndexTaskRunner.java | 5 +- .../kinesis/KinesisSamplerSpecTest.java | 88 --- .../protobuf/ProtobufExtensionsModule.java | 1 - .../protobuf/ProtobufInputRowParser.java | 141 ----- .../protobuf/ProtobufInputFormatTest.java | 175 +++++- .../protobuf/ProtobufInputRowParserTest.java | 331 ----------- .../input/protobuf/ProtobufReaderTest.java | 24 +- ...RegistryBasedProtobufBytesDecoderTest.java | 2 +- .../SeekableStreamIndexTaskRunner.java | 14 +- .../SeekableStreamSamplerSpec.java | 211 +------ ...hunkParser.java => StreamChunkReader.java} | 119 +--- .../sampler/InputSourceSamplerTest.java | 104 +--- ...SeekableStreamIndexTaskRunnerAuthTest.java | 2 +- .../SeekableStreamIndexTaskRunnerTest.java | 28 +- ...erTest.java => StreamChunkReaderTest.java} | 153 +----- 40 files changed, 610 insertions(+), 2563 deletions(-) delete mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/ProtobufParserBenchmark.java delete mode 100644 extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroStreamInputRowParser.java delete mode 100644 extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java delete mode 100644 extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParser.java delete mode 100644 extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/{StreamChunkParser.java => StreamChunkReader.java} (52%) rename indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/{StreamChunkParserTest.java => StreamChunkReaderTest.java} (71%) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ProtobufParserBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ProtobufParserBenchmark.java deleted file mode 100644 index 4cbfaa2dbba3..000000000000 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ProtobufParserBenchmark.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * 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.benchmark; - -import com.google.common.collect.Lists; -import com.google.common.io.Files; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.data.input.protobuf.FileBasedProtobufBytesDecoder; -import org.apache.druid.data.input.protobuf.ProtobufInputRowParser; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; -import org.apache.druid.java.util.common.parsers.JSONPathFieldType; -import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Warmup; -import org.openjdk.jmh.infra.Blackhole; - -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.concurrent.TimeUnit; - -@State(Scope.Benchmark) -@Fork(value = 1) -@Warmup(iterations = 10) -@Measurement(iterations = 25) -public class ProtobufParserBenchmark -{ - @Param({"75000"}) - private int rowsPerSegment; - - private static final Logger log = new Logger(ProtobufParserBenchmark.class); - - private ParseSpec nestedParseSpec; - private ProtobufInputRowParser nestedParser; - private ParseSpec flatParseSpec; - private ProtobufInputRowParser flatParser; - private byte[] protoInputs; - private String protoFilePath; - private FileBasedProtobufBytesDecoder decoder; - - @Setup - public void setup() - { - nestedParseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(Lists.newArrayList( - new StringDimensionSchema("event"), - new StringDimensionSchema("id"), - new StringDimensionSchema("someOtherId"), - new StringDimensionSchema("isValid") - )), - new JSONPathSpec( - true, - Lists.newArrayList( - new JSONPathFieldSpec(JSONPathFieldType.ROOT, "eventType", "eventType"), - new JSONPathFieldSpec(JSONPathFieldType.PATH, "foobar", "$.foo.bar"), - new JSONPathFieldSpec(JSONPathFieldType.PATH, "bar0", "$.bar[0].bar") - ) - ), - null, - null - ); - - flatParseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(Lists.newArrayList( - new StringDimensionSchema("event"), - new StringDimensionSchema("id"), - new StringDimensionSchema("someOtherId"), - new StringDimensionSchema("isValid") - )), - null, - null, - null - ); - decoder = new FileBasedProtobufBytesDecoder("prototest.desc", "ProtoTestEvent"); - - protoFilePath = "ProtoFile"; - protoInputs = getProtoInputs(protoFilePath); - nestedParser = new ProtobufInputRowParser(nestedParseSpec, decoder, null, null); - flatParser = new ProtobufInputRowParser(flatParseSpec, decoder, null, null); - } - - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MICROSECONDS) - public void consumeFlatData(Blackhole blackhole) - { - for (int i = 0; i < rowsPerSegment; i++) { - InputRow row = flatParser.parseBatch(ByteBuffer.wrap(protoInputs)).get(0); - blackhole.consume(row); - } - } - - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MICROSECONDS) - public void consumeNestedData(Blackhole blackhole) - { - for (int i = 0; i < rowsPerSegment; i++) { - InputRow row = nestedParser.parseBatch(ByteBuffer.wrap(protoInputs)).get(0); - blackhole.consume(row); - } - - } - private byte[] getProtoInputs(String fileName) - { - String filePath = this.getClass().getClassLoader().getResource(fileName).getPath(); - byte[] bytes = null; - try { - File file = new File(filePath); - bytes = new byte[(int) file.length()]; - bytes = Files.toByteArray(file); - } - catch (FileNotFoundException e) { - log.error("Cannot find the file: " + filePath); - e.printStackTrace(); - } - catch (IOException e) { - e.printStackTrace(); - } - return bytes; - } -} diff --git a/docs/development/extensions-core/avro.md b/docs/development/extensions-core/avro.md index 7db7530b07de..2ce6451a4222 100644 --- a/docs/development/extensions-core/avro.md +++ b/docs/development/extensions-core/avro.md @@ -27,8 +27,6 @@ This Apache Druid extension enables Druid to ingest and parse the Apache Avro da - [Avro OCF input format](../../ingestion/data-formats.md#avro-ocf) for native batch ingestion. - [Avro Hadoop Parser](../../ingestion/data-formats.md#avro-hadoop-parser). -The [Avro Stream Parser](../../ingestion/data-formats.md#avro-stream-parser) is deprecated. - ## Load the Avro extension To use the Avro extension, add the `druid-avro-extensions` to the list of loaded extensions. See [Loading extensions](../../configuration/extensions.md#loading-extensions) for more information. diff --git a/docs/ingestion/data-formats.md b/docs/ingestion/data-formats.md index 4bdf99a4ea22..d6c588e2184f 100644 --- a/docs/ingestion/data-formats.md +++ b/docs/ingestion/data-formats.md @@ -385,7 +385,7 @@ Note that it is essentially a map of integer schema ID to avro schema object. Th ##### SchemaRepo Based Avro Bytes Decoder -This Avro bytes decoder first extracts `subject` and `id` from the input message bytes, and then uses them to look up the Avro schema used to decode the Avro record from bytes. For details, see the [schema repo](https://github.com/schema-repo/schema-repo). You need an HTTP service like schema repo to hold the Avro schema. For information on registering a schema on the message producer side, see `org.apache.druid.data.input.AvroStreamInputRowParserTest#testParse()`. +This Avro bytes decoder first extracts `subject` and `id` from the input message bytes, and then uses them to look up the Avro schema used to decode the Avro record from bytes. For details, see the [schema repo](https://github.com/schema-repo/schema-repo). You need an HTTP service like schema repo to hold the Avro schema. For information on registering a schema on the message producer side, see `org.apache.druid.data.input.avro.AvroStreamInputFormatTest#testParse()`. | Field | Type | Description | Required | |-------|------|-------------|----------| @@ -964,9 +964,7 @@ Each entry in the `fields` list can have the following components: ## Parser :::info - The Parser is deprecated for [native batch tasks](./native-batch.md), [Kafka indexing service](../ingestion/kafka-ingestion.md), -and [Kinesis indexing service](../ingestion/kinesis-ingestion.md). -Consider using the [input format](#input-format) instead for these types of ingestion. + The Parser is used for [Apache Hadoop batch tasks](./hadoop.md), which is deprecated. ::: This section lists all default and core extension parsers. @@ -1552,207 +1550,6 @@ an explicitly defined [format](http://www.joda.org/joda-time/apidocs/org/joda/ti } ``` -### Avro Stream Parser - -:::info - You need to include the [`druid-avro-extensions`](../development/extensions-core/avro.md) as an extension to use the Avro Stream Parser. -::: - -:::info - See the [Avro Types](../development/extensions-core/avro.md#avro-types) section for how Avro types are handled in Druid -::: - -This parser is for [stream ingestion](./index.md#streaming) and reads Avro data from a stream directly. - -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| type | String | Set value to `avro_stream`. | no | -| avroBytesDecoder | JSON Object | Specifies [`avroBytesDecoder`](#Avro Bytes Decoder) to decode bytes to Avro record. | yes | -| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be an "avro" parseSpec. | yes | - -An Avro parseSpec can contain a [`flattenSpec`](#flattenspec) using either the "root" or "path" -field types, which can be used to read nested Avro records. The "jq" and "tree" field type is not currently supported for Avro. - -For example, using Avro stream parser with schema repo Avro bytes decoder: - -```json -"parser" : { - "type" : "avro_stream", - "avroBytesDecoder" : { - "type" : "schema_repo", - "subjectAndIdConverter" : { - "type" : "avro_1124", - "topic" : "${YOUR_TOPIC}" - }, - "schemaRepository" : { - "type" : "avro_1124_rest_client", - "url" : "${YOUR_SCHEMA_REPO_END_POINT}", - } - }, - "parseSpec" : { - "format": "avro", - "timestampSpec": , - "dimensionsSpec": , - "flattenSpec": - } -} -``` - -### Protobuf Parser - -:::info - You need to include the [`druid-protobuf-extensions`](../development/extensions-core/protobuf.md) as an extension to use the Protobuf Parser. -::: - -This parser is for [stream ingestion](./index.md#streaming) and reads Protocol buffer data from a stream directly. - -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| type | String | Set value to `protobuf`. | yes | -| `protoBytesDecoder` | JSON Object | Specifies how to decode bytes to Protobuf record. | yes | -| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. The format must be JSON. See [JSON ParseSpec](#json-parsespec) for more configuration options. Note that `timeAndDims` `parseSpec` is no longer supported. | yes | - -Sample spec: - -```json -"parser": { - "type": "protobuf", - "protoBytesDecoder": { - "type": "file", - "descriptor": "file:///tmp/metrics.desc", - "protoMessageType": "Metrics" - }, - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [ - "unit", - "http_method", - "http_code", - "page", - "metricType", - "server" - ], - "dimensionExclusions": [ - "timestamp", - "value" - ] - } - } -} -``` - -See the [extension description](../development/extensions-core/protobuf.md) for -more details and examples. - -#### Protobuf Bytes Decoder - -If `type` is not included, the `protoBytesDecoder` defaults to `schema_registry`. - -##### File-based Protobuf Bytes Decoder - -This Protobuf bytes decoder first read a descriptor file, and then parse it to get schema used to decode the Protobuf record from bytes. - -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| type | String | Set value to `file`. | yes | -| descriptor | String | Protobuf descriptor file name in the classpath or URL. | yes | -| protoMessageType | String | Protobuf message type in the descriptor. Both short name and fully qualified name are accepted. The parser uses the first message type found in the descriptor if not specified. | no | - -Sample spec: - -```json -"protoBytesDecoder": { - "type": "file", - "descriptor": "file:///tmp/metrics.desc", - "protoMessageType": "Metrics" -} -``` - -#### Inline Descriptor Protobuf Bytes Decoder - -This Protobuf bytes decoder allows the user to provide the contents of a Protobuf descriptor file inline, encoded as a Base64 string, and then parse it to get schema used to decode the Protobuf record from bytes. - -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| type | String | Set value to `inline`. | yes | -| descriptorString | String | A compiled Protobuf descriptor, encoded as a Base64 string. | yes | -| protoMessageType | String | Protobuf message type in the descriptor. Both short name and fully qualified name are accepted. The parser uses the first message type found in the descriptor if not specified. | no | - -Sample spec: - -```json -"protoBytesDecoder": { - "type": "inline", - "descriptorString": , - "protoMessageType": "Metrics" -} -``` - -##### Confluent Schema Registry-based Protobuf Bytes Decoder - -This Protobuf bytes decoder first extracts a unique `id` from input message bytes, and then uses it to look up the schema in the Schema Registry used to decode the Avro record from bytes. -For details, see the Schema Registry [documentation](http://docs.confluent.io/current/schema-registry/docs/) and [repository](https://github.com/confluentinc/schema-registry). - -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| type | String | Set value to `schema_registry`. | yes | -| url | String | Specifies the URL endpoint of the Schema Registry. | yes | -| capacity | Integer | Specifies the max size of the cache (default = Integer.MAX_VALUE). | no | -| urls | ARRAY\ | Specifies the URL endpoints of the multiple Schema Registry instances. | yes (if `url` is not provided) | -| config | Json | To send additional configurations, configured for Schema Registry. This can be supplied via a [DynamicConfigProvider](../operations/dynamic-config-provider.md). | no | -| headers | Json | To send headers to the Schema Registry. This can be supplied via a [DynamicConfigProvider](../operations/dynamic-config-provider.md) | no | - -For a single schema registry instance, use Field `url` or `urls` for multi instances. - -Single Instance: - -```json -... -"protoBytesDecoder": { - "url": , - "type": "schema_registry" -} -... -``` - -Multiple Instances: - -```json -... -"protoBytesDecoder": { - "urls": [, , ...], - "type": "schema_registry", - "capacity": 100, - "config" : { - "basic.auth.credentials.source": "USER_INFO", - "basic.auth.user.info": "fred:letmein", - "schema.registry.ssl.truststore.location": "/some/secrets/kafka.client.truststore.jks", - "schema.registry.ssl.truststore.password": "", - "schema.registry.ssl.keystore.location": "/some/secrets/kafka.client.keystore.jks", - "schema.registry.ssl.keystore.password": "", - "schema.registry.ssl.key.password": "", - ... - }, - "headers": { - "traceID" : "b29c5de2-0db4-490b-b421", - "timeStamp" : "1577191871865", - "druid.dynamic.config.provider":{ - "type":"mapString", - "config":{ - "registry.header.prop.1":"value.1", - "registry.header.prop.2":"value.2" - } - } - ... - } -} -... -``` ## ParseSpec diff --git a/docs/ingestion/ingestion-spec.md b/docs/ingestion/ingestion-spec.md index 496c687bec13..4d607ada7544 100644 --- a/docs/ingestion/ingestion-spec.md +++ b/docs/ingestion/ingestion-spec.md @@ -454,8 +454,8 @@ The legacy `dataSchema` spec has below two more components in addition to the on #### `parser` (Deprecated) In legacy `dataSchema`, the `parser` is located in the `dataSchema` → `parser` and is responsible for configuring a wide variety of -items related to parsing input records. The `parser` is deprecated and it is highly recommended to use `inputFormat` instead. -For details about `inputFormat` and supported `parser` types, see the ["Data formats" page](data-formats.md). +items related to parsing input records. The `parser` is only supported by Hadoop ingestion, and is deprecated. +For details about supported `parser` types, see the ["Data formats" page](data-formats.md). For details about major components of the `parseSpec`, refer to their subsections: diff --git a/docs/ingestion/kafka-ingestion.md b/docs/ingestion/kafka-ingestion.md index 67b1a96a3d79..54c75b7e76ea 100644 --- a/docs/ingestion/kafka-ingestion.md +++ b/docs/ingestion/kafka-ingestion.md @@ -266,9 +266,9 @@ The following example shows a supervisor spec with idle configuration enabled: #### Data format -The Kafka indexing service supports both [`inputFormat`](data-formats.md#input-format) and [`parser`](data-formats.md#parser) to specify the data format. Use the `inputFormat` to specify the data format for the Kafka indexing service unless you need a format only supported by the legacy `parser`. For more information, see [Source input formats](data-formats.md). +The Kafka indexing service supports [`inputFormat`](data-formats.md#input-format). For more information, see [Source input formats](data-formats.md). -The Kinesis indexing service supports the following values for `inputFormat`: +The Kafka indexing service supports the following values for `inputFormat`: * `csv` * `tvs` @@ -276,8 +276,7 @@ The Kinesis indexing service supports the following values for `inputFormat`: * `kafka` * `avro_stream` * `protobuf` - -You can use `parser` to read [`thrift`](../development/extensions-contrib/thrift.md) formats. +* `thrift` ##### Kafka input format supervisor spec example diff --git a/docs/ingestion/kinesis-ingestion.md b/docs/ingestion/kinesis-ingestion.md index c0eced948926..487a74f2a1d5 100644 --- a/docs/ingestion/kinesis-ingestion.md +++ b/docs/ingestion/kinesis-ingestion.md @@ -136,7 +136,7 @@ For configuration properties shared across all streaming ingestion methods, refe #### Data format -The Kinesis indexing service supports both [`inputFormat`](data-formats.md#input-format) and [`parser`](data-formats.md#parser) to specify the data format. Use the `inputFormat` to specify the data format for the Kinesis indexing service unless you need a format only supported by the legacy `parser`. For more information, see [Source input formats](data-formats.md). +The Kinesis indexing service supports [`inputFormat`](data-formats.md#input-format). For more information, see [Source input formats](data-formats.md). The Kinesis indexing service supports the following values for `inputFormat`: @@ -146,8 +146,7 @@ The Kinesis indexing service supports the following values for `inputFormat`: * `json` * `avro_stream` * `protobuf` - -You can use `parser` to read [`thrift`](../development/extensions-contrib/thrift.md) formats. +* `thrift` ### Tuning configuration diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaIndexDataFormatsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaIndexDataFormatsTest.java index d3cfcb4029a8..eb63b791329c 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaIndexDataFormatsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaIndexDataFormatsTest.java @@ -26,8 +26,6 @@ import org.apache.druid.testing.embedded.StreamIngestResource; import org.joda.time.Period; -import java.util.Map; - public class KafkaIndexDataFormatsTest extends StreamIndexDataFormatsTestBase { private final KafkaResource kafka = new KafkaResource(); @@ -38,26 +36,6 @@ protected StreamIngestResource getStreamResource() return kafka; } - @Override - public SupervisorSpec createSupervisorWithParser(String dataSource, String topic, Map parserMap) - { - return MoreResources.Supervisor.KAFKA_JSON - .get() - .withDataSchema( - schema -> schema - .withTimestamp(new TimestampSpec("timestamp", null, null)) - .withParserMap(parserMap) - ) - .withIoConfig( - ioConfig -> ioConfig - .withInputFormat(null) - .withConsumerProperties(kafka.consumerProperties()) - .withSupervisorRunPeriod(Period.millis(10)) - ) - .withTuningConfig(tuningConfig -> tuningConfig.withMaxRowsPerSegment(1)) - .build(dataSource, topic); - } - @Override public SupervisorSpec createSupervisor(String dataSource, String topic, InputFormat inputFormat) { diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/StreamIndexDataFormatsTestBase.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/StreamIndexDataFormatsTestBase.java index 03f12337a627..f25e22a5aa56 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/StreamIndexDataFormatsTestBase.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/StreamIndexDataFormatsTestBase.java @@ -19,28 +19,18 @@ package org.apache.druid.testing.embedded.indexing; -import com.fasterxml.jackson.core.type.TypeReference; -import org.apache.druid.data.input.AvroStreamInputRowParser; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.avro.AvroExtensionsModule; -import org.apache.druid.data.input.avro.AvroParseSpec; import org.apache.druid.data.input.avro.AvroStreamInputFormat; import org.apache.druid.data.input.avro.InlineSchemaAvroBytesDecoder; import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; -import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DelimitedInputFormat; -import org.apache.druid.data.input.impl.DelimitedParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JsonInputFormat; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.protobuf.FileBasedProtobufBytesDecoder; import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; import org.apache.druid.data.input.protobuf.ProtobufInputFormat; -import org.apache.druid.data.input.protobuf.ProtobufInputRowParser; import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; import org.apache.druid.data.input.thrift.ThriftExtensionsModule; import org.apache.druid.data.input.thrift.ThriftInputFormat; @@ -88,17 +78,29 @@ *
  • Thrift
  • *
  • TSV
  • * - * This tests both InputFormat and Parser. Parser is deprecated for Streaming Ingestion, - * and those tests will be removed in the future. */ public abstract class StreamIndexDataFormatsTestBase extends EmbeddedClusterTestBase { private static final long CYCLE_PADDING_MS = 100; private static final int EVENTS_PER_SECOND = 6; - private static final List WIKI_DIM_LIST = List.of("timestamp", "page", "language", "user", - "unpatrolled", "newPage", "robot", "anonymous", "namespace", - "continent", "country", "region", "city", "added", - "deleted", "delta"); + private static final List WIKI_DIM_LIST = List.of( + "timestamp", + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "added", + "deleted", + "delta" + ); private final EmbeddedBroker broker = new EmbeddedBroker(); private final EmbeddedIndexer indexer = new EmbeddedIndexer(); @@ -117,17 +119,6 @@ protected abstract SupervisorSpec createSupervisor( InputFormat inputFormat ); - /** - * Creates a {@link SupervisorSpec} that uses an {@link InputRowParser} instead - * of an {@link InputFormat}. Tests using the parser will be removed in the - * future. - */ - protected abstract SupervisorSpec createSupervisorWithParser( - String dataSource, - String topic, - Map parserMap - ); - @Override public EmbeddedDruidCluster createCluster() { @@ -157,43 +148,6 @@ public EmbeddedDruidCluster createCluster() return cluster; } - @Test - @Timeout(30) - public void test_avroDataFormat_withParser() throws Exception - { - streamResource.createTopicWithPartitions(dataSource, 3); - EventSerializer serializer = new AvroEventSerializer(); - int recordCount = generateStreamAndPublish(dataSource, serializer, false); - - Map avroSchema = createWikipediaAvroSchemaMap(); - - InlineSchemaAvroBytesDecoder avroBytesDecoder = new InlineSchemaAvroBytesDecoder( - overlord.bindings().jsonMapper(), - avroSchema - ); - - AvroParseSpec parseSpec = new AvroParseSpec( - new TimestampSpec("timestamp", "auto", null), - createWikipediaDimensionsSpec(), - null - ); - - AvroStreamInputRowParser parser = new AvroStreamInputRowParser( - parseSpec, - avroBytesDecoder, - false, - null - ); - - SupervisorSpec supervisorSpec = createSupervisorWithParser(dataSource, dataSource, parser); - - final String supervisorId = cluster.callApi().postSupervisor(supervisorSpec); - Assertions.assertEquals(dataSource, supervisorId); - - waitForDataAndVerifyIngestedEvents(dataSource, recordCount); - stopSupervisor(supervisorSpec); - } - @Test @Timeout(30) public void test_avroDataFormat() throws Exception @@ -226,45 +180,6 @@ public void test_avroDataFormat() throws Exception stopSupervisor(supervisorSpec); } - @Test - @Timeout(30) - public void test_avroDataFormatWithSchemaRegistry_withParser() - { - streamResource.createTopicWithPartitions(dataSource, 3); - EventSerializer serializer = new AvroSchemaRegistryEventSerializer(schemaRegistry.getHostandPort()); - serializer.initialize(dataSource); - int recordCount = generateStreamAndPublish(dataSource, serializer, true); - SchemaRegistryBasedAvroBytesDecoder avroBytesDecoder = new SchemaRegistryBasedAvroBytesDecoder( - schemaRegistry.getConnectURI(), - null, - null, - null, - null, - overlord.bindings().jsonMapper() - ); - - AvroParseSpec parseSpec = new AvroParseSpec( - new TimestampSpec("timestamp", "auto", null), - createWikipediaDimensionsSpec(), - null - ); - - AvroStreamInputRowParser parser = new AvroStreamInputRowParser( - parseSpec, - avroBytesDecoder, - false, - null - ); - - SupervisorSpec supervisorSpec = createSupervisorWithParser(dataSource, dataSource, parser); - - final String supervisorId = cluster.callApi().postSupervisor(supervisorSpec); - Assertions.assertEquals(dataSource, supervisorId); - - waitForDataAndVerifyIngestedEvents(dataSource, recordCount); - stopSupervisor(supervisorSpec); - } - @Test @Timeout(30) public void test_avroDataFormatWithSchemaRegistry() @@ -309,59 +224,6 @@ public void test_csvDataFormat() stopSupervisor(supervisorSpec); } - @Test - @Timeout(30) - public void test_csvDataFormat_withParser() - { - streamResource.createTopicWithPartitions(dataSource, 3); - EventSerializer serializer = new CsvEventSerializer(); - int recordCount = generateStreamAndPublish(dataSource, serializer, false); - CSVParseSpec parseSpec = new CSVParseSpec( - new TimestampSpec("timestamp", "auto", null), - createWikipediaDimensionsSpec(), - null, - WIKI_DIM_LIST, - false, - 0 - ); - - StringInputRowParser parser = new StringInputRowParser(parseSpec, null); - - SupervisorSpec supervisorSpec = createSupervisorWithParser(dataSource, dataSource, parser); - - final String supervisorId = cluster.callApi().postSupervisor(supervisorSpec); - Assertions.assertEquals(dataSource, supervisorId); - - waitForDataAndVerifyIngestedEvents(dataSource, recordCount); - stopSupervisor(supervisorSpec); - } - - @Test - @Timeout(30) - public void test_jsonDataFormat_withParser() - { - streamResource.createTopicWithPartitions(dataSource, 3); - EventSerializer serializer = new JsonEventSerializer(overlord.bindings().jsonMapper()); - int recordCount = generateStreamAndPublish(dataSource, serializer, false); - JSONParseSpec parseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "auto", null), - createWikipediaDimensionsSpec(), - new JSONPathSpec(true, null), - null, - false - ); - - StringInputRowParser parser = new StringInputRowParser(parseSpec, null); - - SupervisorSpec supervisorSpec = createSupervisorWithParser(dataSource, dataSource, parser); - - final String supervisorId = cluster.callApi().postSupervisor(supervisorSpec); - Assertions.assertEquals(dataSource, supervisorId); - - waitForDataAndVerifyIngestedEvents(dataSource, recordCount); - stopSupervisor(supervisorSpec); - } - @Test @Timeout(30) public void test_jsonDataFormat() @@ -380,37 +242,6 @@ public void test_jsonDataFormat() stopSupervisor(supervisorSpec); } - @Test - @Timeout(30) - public void test_protobufDataFormat_withParser() - { - streamResource.createTopicWithPartitions(dataSource, 3); - EventSerializer serializer = new ProtobufEventSerializer(); - int recordCount = generateStreamAndPublish(dataSource, serializer, false); - - FileBasedProtobufBytesDecoder protobufBytesDecoder = new FileBasedProtobufBytesDecoder( - MoreResources.ProbufData.WIKI_PROTOBUF_BYTES_DECODER_RESOURCE, - MoreResources.ProbufData.WIKI_PROTO_MESSAGE_TYPE - ); - - JSONParseSpec parseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "auto", null), - createWikipediaDimensionsSpec(), - new JSONPathSpec(true, null), - null, - false - ); - - ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, protobufBytesDecoder, null, null); - SupervisorSpec supervisorSpec = createSupervisorWithParser(dataSource, dataSource, parser); - - final String supervisorId = cluster.callApi().postSupervisor(supervisorSpec); - Assertions.assertEquals(dataSource, supervisorId); - - waitForDataAndVerifyIngestedEvents(dataSource, recordCount); - stopSupervisor(supervisorSpec); - } - @Test public void test_protobufDataFormat() { @@ -435,41 +266,6 @@ public void test_protobufDataFormat() stopSupervisor(supervisorSpec); } - @Test - @Timeout(30) - public void test_protobufDataFormatWithSchemaRegistry_withParser() - { - streamResource.createTopicWithPartitions(dataSource, 3); - EventSerializer serializer = new ProtobufSchemaRegistryEventSerializer(schemaRegistry.getHostandPort()); - serializer.initialize(dataSource); - int recordCount = generateStreamAndPublish(dataSource, serializer, true); - SchemaRegistryBasedProtobufBytesDecoder protobufBytesDecoder = new SchemaRegistryBasedProtobufBytesDecoder( - schemaRegistry.getConnectURI(), - null, - null, - null, - null, - overlord.bindings().jsonMapper() - ); - - JSONParseSpec parseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "auto", null), - createWikipediaDimensionsSpec(), - new JSONPathSpec(true, null), - null, - false - ); - - ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, protobufBytesDecoder, null, null); - SupervisorSpec supervisorSpec = createSupervisorWithParser(dataSource, dataSource, parser); - - final String supervisorId = cluster.callApi().postSupervisor(supervisorSpec); - Assertions.assertEquals(dataSource, supervisorId); - - waitForDataAndVerifyIngestedEvents(dataSource, recordCount); - stopSupervisor(supervisorSpec); - } - @Test @Timeout(30) public void test_protobufDataFormatWithSchemaRegistry() @@ -495,35 +291,6 @@ public void test_protobufDataFormatWithSchemaRegistry() stopSupervisor(supervisorSpec); } - @Test - @Timeout(30) - public void test_tsvDataFormat_withParser() - { - streamResource.createTopicWithPartitions(dataSource, 3); - EventSerializer serializer = new DelimitedEventSerializer(); - int recordCount = generateStreamAndPublish(dataSource, serializer, false); - // Build DelimitedParseSpec with proper object construction for TSV - DelimitedParseSpec parseSpec = new DelimitedParseSpec( - new TimestampSpec("timestamp", "auto", null), - createWikipediaDimensionsSpec(), - "\t", - null, - WIKI_DIM_LIST, - false, - 0 - ); - - StringInputRowParser parser = new StringInputRowParser(parseSpec, null); - - SupervisorSpec supervisorSpec = createSupervisorWithParser(dataSource, dataSource, parser); - - final String supervisorId = cluster.callApi().postSupervisor(supervisorSpec); - Assertions.assertEquals(dataSource, supervisorId); - - waitForDataAndVerifyIngestedEvents(dataSource, recordCount); - stopSupervisor(supervisorSpec); - } - @Test @Timeout(30) public void test_tsvDataFormat() @@ -610,15 +377,6 @@ private int generateStreamAndPublish(String topic, EventSerializer serializer, b return records.size(); } - private SupervisorSpec createSupervisorWithParser(String supervisorId, String topic, InputRowParser parser) - { - Map parserMap = overlord.bindings().jsonMapper().convertValue( - parser, - new TypeReference<>() {} - ); - return createSupervisorWithParser(supervisorId, topic, parserMap); - } - private Map createWikipediaAvroSchemaMap() { Map schema = new HashMap<>(); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/kinesis/KinesisDataFormatsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/kinesis/KinesisDataFormatsTest.java index 1a76594ec87f..c60149272242 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/kinesis/KinesisDataFormatsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/kinesis/KinesisDataFormatsTest.java @@ -61,24 +61,6 @@ protected SupervisorSpec createSupervisor(String dataSource, String topic, Input return createKinesisSupervisorSpec(dataSource, topic, inputFormat); } - @Override - protected SupervisorSpec createSupervisorWithParser(String dataSource, String topic, Map parserMap) - { - final KinesisSupervisorSpec baseSpec = createKinesisSupervisorSpec(dataSource, topic, null); - return new KinesisSupervisorSpec( - dataSource, - null, - DataSchema.builder(baseSpec.getSpec().getDataSchema()) - .withParserMap(parserMap) - .build(), - baseSpec.getSpec().getTuningConfig(), - baseSpec.getSpec().getIOConfig(), - Map.of(), - false, - null, null, null, null, null, null, null, null, null, null - ); - } - private KinesisSupervisorSpec createKinesisSupervisorSpec(String dataSource, String topic, InputFormat inputFormat) { return new KinesisSupervisorSpec( diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java index 452ccf5e0186..dc9dee8ad434 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java @@ -94,11 +94,7 @@ public TaskStatus runTask(TaskToolbox toolbox) protected SeekableStreamIndexTaskRunner createTaskRunner() { // noinspection unchecked - return new RabbitStreamIndexTaskRunner( - this, - dataSchema.getParser(), - lockGranularityToUse - ); + return new RabbitStreamIndexTaskRunner(this, lockGranularityToUse); } @Override diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskRunner.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskRunner.java index bfe93dedbfb9..e34f0d9e6d15 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskRunner.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskRunner.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.data.input.impl.ByteEntity; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; @@ -40,7 +39,6 @@ import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.List; import java.util.Map; import java.util.Set; @@ -57,15 +55,10 @@ public class RabbitStreamIndexTaskRunner RabbitStreamIndexTaskRunner( RabbitStreamIndexTask task, - @Nullable InputRowParser parser, LockGranularity lockGranularityToUse ) { - super( - task, - parser, - lockGranularityToUse - ); + super(task, lockGranularityToUse); this.task = task; } diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroStreamInputRowParser.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroStreamInputRowParser.java deleted file mode 100644 index 969bf259b446..000000000000 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroStreamInputRowParser.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.data.input; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import org.apache.avro.generic.GenericRecord; -import org.apache.druid.data.input.avro.AvroBytesDecoder; -import org.apache.druid.data.input.avro.AvroParsers; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.java.util.common.parsers.ObjectFlattener; - -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Objects; - -public class AvroStreamInputRowParser implements ByteBufferInputRowParser -{ - private final ParseSpec parseSpec; - private final Boolean binaryAsString; - private final Boolean extractUnionsByType; - private final AvroBytesDecoder avroBytesDecoder; - private final ObjectFlattener avroFlattener; - private final MapInputRowParser mapParser; - - @JsonCreator - public AvroStreamInputRowParser( - @JsonProperty("parseSpec") ParseSpec parseSpec, - @JsonProperty("avroBytesDecoder") AvroBytesDecoder avroBytesDecoder, - @JsonProperty("binaryAsString") Boolean binaryAsString, - @JsonProperty("extractUnionsByType") Boolean extractUnionsByType - ) - { - this.parseSpec = Preconditions.checkNotNull(parseSpec, "parseSpec"); - this.avroBytesDecoder = Preconditions.checkNotNull(avroBytesDecoder, "avroBytesDecoder"); - this.binaryAsString = binaryAsString != null && binaryAsString; - this.extractUnionsByType = extractUnionsByType != null && extractUnionsByType; - this.avroFlattener = AvroParsers.makeFlattener(parseSpec, false, this.binaryAsString, this.extractUnionsByType); - this.mapParser = new MapInputRowParser(parseSpec); - } - - @Override - public List parseBatch(ByteBuffer input) - { - return AvroParsers.parseGenericRecord(avroBytesDecoder.parse(input), mapParser, avroFlattener); - } - - @JsonProperty - @Override - public ParseSpec getParseSpec() - { - return parseSpec; - } - - @JsonProperty - public AvroBytesDecoder getAvroBytesDecoder() - { - return avroBytesDecoder; - } - - @JsonProperty - public Boolean getBinaryAsString() - { - return binaryAsString; - } - - @JsonProperty - public Boolean isExtractUnionsByType() - { - return extractUnionsByType; - } - - @Override - public ByteBufferInputRowParser withParseSpec(ParseSpec parseSpec) - { - return new AvroStreamInputRowParser( - parseSpec, - avroBytesDecoder, - binaryAsString, - extractUnionsByType - ); - } - - @Override - public boolean equals(final Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - final AvroStreamInputRowParser that = (AvroStreamInputRowParser) o; - return Objects.equals(parseSpec, that.parseSpec) && - Objects.equals(avroBytesDecoder, that.avroBytesDecoder) && - Objects.equals(binaryAsString, that.binaryAsString) && - Objects.equals(extractUnionsByType, that.extractUnionsByType); - } - - @Override - public int hashCode() - { - return Objects.hash(parseSpec, avroBytesDecoder, binaryAsString, extractUnionsByType); - } - - @Override - public String toString() - { - return "AvroStreamInputRowParser{" + - "parseSpec=" + parseSpec + - ", binaryAsString=" + binaryAsString + - ", extractUnionsByType=" + extractUnionsByType + - ", avroBytesDecoder=" + avroBytesDecoder + - '}'; - } -} diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroExtensionsModule.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroExtensionsModule.java index 296c6dab44b6..eb4def9573a8 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroExtensionsModule.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroExtensionsModule.java @@ -28,7 +28,6 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; import org.apache.druid.data.input.AvroHadoopInputRowParser; -import org.apache.druid.data.input.AvroStreamInputRowParser; import org.apache.druid.data.input.schemarepo.Avro1124RESTRepositoryClientWrapper; import org.apache.druid.initialization.DruidModule; import org.schemarepo.InMemoryRepository; @@ -52,7 +51,6 @@ public List getJacksonModules() return Collections.singletonList( new SimpleModule("AvroInputRowParserModule") .registerSubtypes( - new NamedType(AvroStreamInputRowParser.class, "avro_stream"), new NamedType(AvroHadoopInputRowParser.class, "avro_hadoop"), new NamedType(AvroParseSpec.class, "avro"), new NamedType(AvroOCFInputFormat.class, "avro_ocf"), diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java index 12791b548a68..379eae575204 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import org.apache.avro.file.DataFileReader; import org.apache.avro.file.DataFileWriter; import org.apache.avro.file.FileReader; @@ -28,7 +29,13 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.specific.SpecificDatumWriter; import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroParseSpec; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; +import org.apache.druid.java.util.common.parsers.JSONPathFieldType; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -39,6 +46,17 @@ public class AvroHadoopInputRowParserTest { + static final AvroParseSpec PARSE_SPEC = new AvroParseSpec( + new TimestampSpec("nested", "millis", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(AvroStreamInputFormatTest.DIMENSIONS)), + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.PATH, "nested", "someRecord.subLong"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "nestedArrayVal", "someRecordArray[?(@.nestedString=='string in record')].nestedString") + ) + ) + ); private final ObjectMapper jsonMapper = new ObjectMapper(); @Before @@ -52,7 +70,7 @@ public void setUp() @Test public void testSerde() throws IOException { - AvroHadoopInputRowParser parser = new AvroHadoopInputRowParser(AvroStreamInputRowParserTest.PARSE_SPEC, false, false, false); + AvroHadoopInputRowParser parser = new AvroHadoopInputRowParser(PARSE_SPEC, false, false, false); AvroHadoopInputRowParser parser2 = jsonMapper.readValue( jsonMapper.writeValueAsBytes(parser), AvroHadoopInputRowParser.class @@ -63,7 +81,7 @@ public void testSerde() throws IOException @Test public void testSerdeNonDefaults() throws IOException { - AvroHadoopInputRowParser parser = new AvroHadoopInputRowParser(AvroStreamInputRowParserTest.PARSE_SPEC, true, true, true); + AvroHadoopInputRowParser parser = new AvroHadoopInputRowParser(PARSE_SPEC, true, true, true); AvroHadoopInputRowParser parser2 = jsonMapper.readValue( jsonMapper.writeValueAsBytes(parser), AvroHadoopInputRowParser.class @@ -74,7 +92,7 @@ public void testSerdeNonDefaults() throws IOException @Test public void testParseNotFromPigAvroStorage() throws IOException { - testParse(AvroStreamInputRowParserTest.buildSomeAvroDatum(), false); + testParse(AvroStreamInputFormatTest.buildSomeAvroDatum(), false); } @Test @@ -85,20 +103,20 @@ public void testParseFromPigAvroStorage() throws IOException private void testParse(GenericRecord record, boolean fromPigAvroStorage) throws IOException { - AvroHadoopInputRowParser parser = new AvroHadoopInputRowParser(AvroStreamInputRowParserTest.PARSE_SPEC, fromPigAvroStorage, false, false); + AvroHadoopInputRowParser parser = new AvroHadoopInputRowParser(PARSE_SPEC, fromPigAvroStorage, false, false); AvroHadoopInputRowParser parser2 = jsonMapper.readValue( jsonMapper.writeValueAsBytes(parser), AvroHadoopInputRowParser.class ); Assert.assertEquals(parser, parser2); InputRow inputRow = parser2.parseBatch(record).get(0); - AvroStreamInputRowParserTest.assertInputRowCorrect(inputRow, AvroStreamInputRowParserTest.DIMENSIONS, fromPigAvroStorage); + AvroStreamInputFormatTest.assertInputRowCorrect(inputRow, AvroStreamInputFormatTest.DIMENSIONS, fromPigAvroStorage); } private static GenericRecord buildAvroFromFile() throws IOException { return buildAvroFromFile( - AvroStreamInputRowParserTest.buildSomeAvroDatum() + AvroStreamInputFormatTest.buildSomeAvroDatum() ); } diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java index 323db4fc5247..0aab7ccf8b5e 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java @@ -22,8 +22,12 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.DatumWriter; @@ -46,6 +50,7 @@ import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; import org.apache.druid.java.util.common.parsers.JSONPathFieldType; import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.math.expr.Evals; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.nested.StructuredData; @@ -64,14 +69,22 @@ import org.schemarepo.api.converter.IdentityConverter; import org.schemarepo.api.converter.IntegerConverter; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; - -import static org.apache.druid.data.input.AvroStreamInputRowParserTest.assertInputRowCorrect; -import static org.apache.druid.data.input.AvroStreamInputRowParserTest.buildSomeAvroDatum; +import java.util.Map; +import java.util.regex.Pattern; +import java.util.stream.Collectors; /** * test data row: @@ -106,6 +119,55 @@ public class AvroStreamInputFormatTest extends InitializedNullHandlingTest private static final String NESTED_ARRAY_VAL = "nestedArrayVal"; private static final String IS_VALID = "isValid"; private static final String TOPIC = "aTopic"; + private static final String EVENT_TYPE_VALUE = "type-a"; + private static final long ID_VALUE = 1976491L; + private static final long SOME_OTHER_ID_VALUE = 6568719896L; + private static final float SOME_FLOAT_VALUE = 0.23555f; + private static final int SOME_INT_VALUE = 1; + private static final long SOME_LONG_VALUE = 679865987569912369L; + private static final ZonedDateTime DATE_TIME = ZonedDateTime.of(2015, 10, 25, 19, 30, 0, 0, ZoneOffset.UTC); + private static final MyFixed SOME_FIXED_VALUE = new MyFixed(ByteBuffer.allocate(16).array()); + private static final long SUB_LONG_VALUE = 1543698L; + private static final int SUB_INT_VALUE = 4892; + private static final MySubRecord SOME_RECORD_VALUE = MySubRecord.newBuilder() + .setSubInt(SUB_INT_VALUE) + .setSubLong(SUB_LONG_VALUE) + .build(); + private static final List SOME_STRING_ARRAY_VALUE = Arrays.asList("8", "4", "2", "1", null); + private static final List SOME_INT_ARRAY_VALUE = Arrays.asList(1, 2, 4, 8); + static final Map SOME_INT_VALUE_MAP_VALUE = Maps.asMap( + new HashSet<>(Arrays.asList("8", "2", "4", "1")), new Function<>() + { + @Nonnull + @Override + public Integer apply(@Nullable CharSequence input) + { + return Integer.parseInt(input.toString()); + } + } + ); + static final Map SOME_STRING_VALUE_MAP_VALUE = Maps.asMap( + new HashSet<>(Arrays.asList("8", "2", "4", "1")), new Function<>() + { + @Nonnull + @Override + public CharSequence apply(@Nullable CharSequence input) + { + return input.toString(); + } + } + ); + private static final String SOME_UNION_VALUE = "string as union"; + private static final Integer SOME_UNION_MEMBER_VALUE = 1; + private static final ByteBuffer SOME_BYTES_VALUE = ByteBuffer.allocate(8); + private static final String SOME_RECORD_STRING_VALUE = "string in record"; + private static final List SOME_RECORD_ARRAY_VALUE = Collections.singletonList(MyNestedRecord.newBuilder() + .setNestedString( + SOME_RECORD_STRING_VALUE) + .build()); + private static final Pattern BRACES_AND_SPACE = Pattern.compile("[{} ]"); + + static final List DIMENSIONS = Arrays.asList(EVENT_TYPE, ID, SOME_OTHER_ID, IS_VALID, NESTED_ARRAY_VAL); private static final List DIMENSIONS_SCHEMALESS = Arrays.asList( NESTED_ARRAY_VAL, @@ -363,11 +425,11 @@ public void testParseTransformNested() throws SchemaValidationException, IOExcep Assert.assertEquals(1543698L, inputRow.getTimestampFromEpoch()); Assert.assertEquals( - AvroStreamInputRowParserTest.SOME_INT_VALUE_MAP_VALUE, + SOME_INT_VALUE_MAP_VALUE, StructuredData.unwrap(inputRow.getRaw("someIntValueMap")) ); Assert.assertEquals( - AvroStreamInputRowParserTest.SOME_STRING_VALUE_MAP_VALUE, + SOME_STRING_VALUE_MAP_VALUE, StructuredData.unwrap(inputRow.getRaw("someStringValueMap")) ); Assert.assertEquals( @@ -434,4 +496,167 @@ public void testParseSchemaless() throws SchemaValidationException, IOException assertInputRowCorrect(inputRow, DIMENSIONS_SCHEMALESS, false); } } + + static void assertInputRowCorrect(InputRow inputRow, List expectedDimensions, boolean isFromPigAvro) + { + Assert.assertEquals(expectedDimensions, inputRow.getDimensions()); + Assert.assertEquals(1543698L, inputRow.getTimestampFromEpoch()); + + // test dimensions + Assert.assertEquals(Collections.singletonList(EVENT_TYPE_VALUE), inputRow.getDimension(EVENT_TYPE)); + Assert.assertEquals(Collections.singletonList(String.valueOf(ID_VALUE)), inputRow.getDimension(ID)); + Assert.assertEquals( + Collections.singletonList(String.valueOf(SOME_OTHER_ID_VALUE)), + inputRow.getDimension(SOME_OTHER_ID) + ); + Assert.assertEquals(Collections.singletonList(String.valueOf(true)), inputRow.getDimension(IS_VALID)); + + // someRecordArray represents a record generated from Pig using AvroStorage + // as it implicitly converts array elements to a record + if (isFromPigAvro) { + Assert.assertEquals( + Collections.singletonList(SOME_RECORD_ARRAY_VALUE.get(0).getNestedString()), + inputRow.getDimension("someRecordArray") + ); + } else { + Assert.assertEquals( + Lists.transform(SOME_INT_ARRAY_VALUE, String::valueOf), + inputRow.getDimension("someIntArray") + ); + // For string array, nulls are preserved so use ArrayList (ImmutableList doesn't support nulls) + Assert.assertEquals( + SOME_STRING_ARRAY_VALUE.stream().map(Evals::asString).collect(Collectors.toList()), + inputRow.getDimension("someStringArray") + ); + + final Object someRecordArrayObj = inputRow.getRaw("someRecordArray"); + Assert.assertNotNull(someRecordArrayObj); + Assert.assertTrue(someRecordArrayObj instanceof List); + Assert.assertEquals(1, ((List) someRecordArrayObj).size()); + final Object recordArrayElementObj = ((List) someRecordArrayObj).get(0); + Assert.assertNotNull(recordArrayElementObj); + Assert.assertTrue(recordArrayElementObj instanceof LinkedHashMap); + LinkedHashMap recordArrayElement = (LinkedHashMap) recordArrayElementObj; + Assert.assertEquals("string in record", recordArrayElement.get("nestedString")); + } + + final Object someIntValueMapObj = inputRow.getRaw("someIntValueMap"); + Assert.assertNotNull(someIntValueMapObj); + Assert.assertTrue(someIntValueMapObj instanceof LinkedHashMap); + LinkedHashMap someIntValueMap = (LinkedHashMap) someIntValueMapObj; + Assert.assertEquals(4, someIntValueMap.size()); + Assert.assertEquals(1, someIntValueMap.get("1")); + Assert.assertEquals(2, someIntValueMap.get("2")); + Assert.assertEquals(4, someIntValueMap.get("4")); + Assert.assertEquals(8, someIntValueMap.get("8")); + + + final Object someStringValueMapObj = inputRow.getRaw("someStringValueMap"); + Assert.assertNotNull(someStringValueMapObj); + Assert.assertTrue(someStringValueMapObj instanceof LinkedHashMap); + LinkedHashMap someStringValueMap = (LinkedHashMap) someStringValueMapObj; + Assert.assertEquals(4, someStringValueMap.size()); + Assert.assertEquals("1", someStringValueMap.get("1")); + Assert.assertEquals("2", someStringValueMap.get("2")); + Assert.assertEquals("4", someStringValueMap.get("4")); + Assert.assertEquals("8", someStringValueMap.get("8")); + + + final Object someRecordObj = inputRow.getRaw("someRecord"); + Assert.assertNotNull(someRecordObj); + Assert.assertTrue(someRecordObj instanceof LinkedHashMap); + LinkedHashMap someRecord = (LinkedHashMap) someRecordObj; + Assert.assertEquals(4892, someRecord.get("subInt")); + Assert.assertEquals(1543698L, someRecord.get("subLong")); + + final Object someList = inputRow.getDimension("nestedArrayVal"); + Assert.assertNotNull(someList); + Assert.assertTrue(someList instanceof List); + List someRecordObj3List = (List) someList; + Assert.assertEquals(1, someRecordObj3List.size()); + Assert.assertEquals("string in record", someRecordObj3List.get(0)); + + + // towards Map avro field as druid dimension, need to convert its toString() back to HashMap to check equality + Assert.assertEquals(1, inputRow.getDimension("someIntValueMap").size()); + Assert.assertEquals( + SOME_INT_VALUE_MAP_VALUE, + new HashMap( + Maps.transformValues( + Splitter + .on(",") + .withKeyValueSeparator("=") + .split(BRACES_AND_SPACE.matcher(inputRow.getDimension("someIntValueMap").get(0)).replaceAll("")), + new Function<>() + { + @Nullable + @Override + public Integer apply(@Nullable String input) + { + return Integer.valueOf(input); + } + } + ) + ) + ); + Assert.assertEquals( + SOME_STRING_VALUE_MAP_VALUE, + new HashMap( + Splitter + .on(",") + .withKeyValueSeparator("=") + .split(BRACES_AND_SPACE.matcher(inputRow.getDimension("someIntValueMap").get(0)).replaceAll("")) + ) + ); + Assert.assertEquals(Collections.singletonList(SOME_UNION_VALUE), inputRow.getDimension("someUnion")); + Assert.assertEquals(Collections.emptyList(), inputRow.getDimension("someNull")); + Assert.assertEquals( + Arrays.toString(SOME_FIXED_VALUE.bytes()), + Arrays.toString((byte[]) (inputRow.getRaw("someFixed"))) + ); + Assert.assertEquals( + Arrays.toString(SOME_BYTES_VALUE.array()), + Arrays.toString((byte[]) (inputRow.getRaw("someBytes"))) + ); + Assert.assertEquals(Collections.singletonList(String.valueOf(MyEnum.ENUM1)), inputRow.getDimension("someEnum")); + Assert.assertEquals( + Collections.singletonList(ImmutableMap.of("subInt", 4892, "subLong", 1543698L).toString()), + inputRow.getDimension("someRecord") + ); + + // test metrics + Assert.assertEquals(SOME_FLOAT_VALUE, inputRow.getMetric("someFloat").floatValue(), 0); + Assert.assertEquals(SOME_LONG_VALUE, inputRow.getMetric("someLong")); + Assert.assertEquals(SOME_INT_VALUE, inputRow.getMetric("someInt")); + } + + public static SomeAvroDatum buildSomeAvroDatum() + { + return createSomeAvroDatumBuilderDefaults().build(); + } + + public static SomeAvroDatum.Builder createSomeAvroDatumBuilderDefaults() + { + return SomeAvroDatum.newBuilder() + .setTimestamp(DATE_TIME.toInstant().toEpochMilli()) + .setEventType(EVENT_TYPE_VALUE) + .setId(ID_VALUE) + .setSomeOtherId(SOME_OTHER_ID_VALUE) + .setIsValid(true) + .setSomeFloat(SOME_FLOAT_VALUE) + .setSomeInt(SOME_INT_VALUE) + .setSomeLong(SOME_LONG_VALUE) + .setSomeIntArray(SOME_INT_ARRAY_VALUE) + .setSomeStringArray(SOME_STRING_ARRAY_VALUE) + .setSomeIntValueMap(SOME_INT_VALUE_MAP_VALUE) + .setSomeStringValueMap(SOME_STRING_VALUE_MAP_VALUE) + .setSomeUnion(SOME_UNION_VALUE) + .setSomeMultiMemberUnion(SOME_UNION_MEMBER_VALUE) + .setSomeFixed(SOME_FIXED_VALUE) + .setSomeBytes(SOME_BYTES_VALUE) + .setSomeNull(null) + .setSomeEnum(MyEnum.ENUM1) + .setSomeRecord(SOME_RECORD_VALUE) + .setSomeRecordArray(SOME_RECORD_ARRAY_VALUE); + } } diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java deleted file mode 100644 index 2fd9dd55b91a..000000000000 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java +++ /dev/null @@ -1,514 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.data.input; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.base.Splitter; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import org.apache.avro.AvroRuntimeException; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.io.DatumWriter; -import org.apache.avro.io.EncoderFactory; -import org.apache.avro.specific.SpecificDatumWriter; -import org.apache.druid.data.input.avro.AvroExtensionsModule; -import org.apache.druid.data.input.avro.AvroParseSpec; -import org.apache.druid.data.input.avro.SchemaRepoBasedAvroBytesDecoder; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.data.input.schemarepo.Avro1124RESTRepositoryClientWrapper; -import org.apache.druid.data.input.schemarepo.Avro1124SubjectAndIdConverter; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; -import org.apache.druid.java.util.common.parsers.JSONPathFieldType; -import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.math.expr.Evals; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.schemarepo.InMemoryRepository; -import org.schemarepo.Repository; -import org.schemarepo.SchemaValidationException; -import org.schemarepo.api.TypedSchemaRepository; -import org.schemarepo.api.converter.AvroSchemaConverter; -import org.schemarepo.api.converter.IdentityConverter; -import org.schemarepo.api.converter.IntegerConverter; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.time.ZoneOffset; -import java.time.ZonedDateTime; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - -public class AvroStreamInputRowParserTest -{ - private static final String EVENT_TYPE = "eventType"; - private static final String ID = "id"; - private static final String SOME_OTHER_ID = "someOtherId"; - private static final String IS_VALID = "isValid"; - private static final String NESTED_ARRAY_VAL = "nestedArrayVal"; - private static final String TOPIC = "aTopic"; - private static final String EVENT_TYPE_VALUE = "type-a"; - private static final long ID_VALUE = 1976491L; - private static final long SOME_OTHER_ID_VALUE = 6568719896L; - private static final float SOME_FLOAT_VALUE = 0.23555f; - private static final int SOME_INT_VALUE = 1; - private static final long SOME_LONG_VALUE = 679865987569912369L; - private static final ZonedDateTime DATE_TIME = ZonedDateTime.of(2015, 10, 25, 19, 30, 0, 0, ZoneOffset.UTC); - static final List DIMENSIONS = Arrays.asList(EVENT_TYPE, ID, SOME_OTHER_ID, IS_VALID, NESTED_ARRAY_VAL); - private static final List DIMENSIONS_SCHEMALESS = Arrays.asList( - NESTED_ARRAY_VAL, - SOME_OTHER_ID, - "someIntArray", - "someFloat", - "someUnion", - EVENT_TYPE, - ID, - "someFixed", - "someBytes", - "someEnum", - "someLong", - "someInt", - "timestamp" - ); - static final AvroParseSpec PARSE_SPEC = new AvroParseSpec( - new TimestampSpec("nested", "millis", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(DIMENSIONS)), - new JSONPathSpec( - true, - ImmutableList.of( - new JSONPathFieldSpec(JSONPathFieldType.PATH, "nested", "someRecord.subLong"), - new JSONPathFieldSpec(JSONPathFieldType.PATH, "nestedArrayVal", "someRecordArray[?(@.nestedString=='string in record')].nestedString") - ) - ) - ); - private static final AvroParseSpec PARSE_SPEC_SCHEMALESS = new AvroParseSpec( - new TimestampSpec("nested", "millis", null), - DimensionsSpec.EMPTY, - new JSONPathSpec( - true, - ImmutableList.of( - new JSONPathFieldSpec(JSONPathFieldType.PATH, "nested", "someRecord.subLong"), - new JSONPathFieldSpec(JSONPathFieldType.PATH, "nestedArrayVal", "someRecordArray[?(@.nestedString=='string in record')].nestedString") - - ) - ) - ); - private static final MyFixed SOME_FIXED_VALUE = new MyFixed(ByteBuffer.allocate(16).array()); - private static final long SUB_LONG_VALUE = 1543698L; - private static final int SUB_INT_VALUE = 4892; - private static final MySubRecord SOME_RECORD_VALUE = MySubRecord.newBuilder() - .setSubInt(SUB_INT_VALUE) - .setSubLong(SUB_LONG_VALUE) - .build(); - private static final List SOME_STRING_ARRAY_VALUE = Arrays.asList("8", "4", "2", "1", null); - private static final List SOME_INT_ARRAY_VALUE = Arrays.asList(1, 2, 4, 8); - static final Map SOME_INT_VALUE_MAP_VALUE = Maps.asMap( - new HashSet<>(Arrays.asList("8", "2", "4", "1")), new Function<>() - { - @Nonnull - @Override - public Integer apply(@Nullable CharSequence input) - { - return Integer.parseInt(input.toString()); - } - } - ); - static final Map SOME_STRING_VALUE_MAP_VALUE = Maps.asMap( - new HashSet<>(Arrays.asList("8", "2", "4", "1")), new Function<>() - { - @Nonnull - @Override - public CharSequence apply(@Nullable CharSequence input) - { - return input.toString(); - } - } - ); - private static final String SOME_UNION_VALUE = "string as union"; - private static final Integer SOME_UNION_MEMBER_VALUE = 1; - private static final ByteBuffer SOME_BYTES_VALUE = ByteBuffer.allocate(8); - private static final String SOME_RECORD_STRING_VALUE = "string in record"; - private static final List SOME_RECORD_ARRAY_VALUE = Collections.singletonList(MyNestedRecord.newBuilder() - .setNestedString( - SOME_RECORD_STRING_VALUE) - .build()); - private static final Pattern BRACES_AND_SPACE = Pattern.compile("[{} ]"); - - private final ObjectMapper jsonMapper = new DefaultObjectMapper(); - - - @Before - public void before() - { - for (Module jacksonModule : new AvroExtensionsModule().getJacksonModules()) { - jsonMapper.registerModule(jacksonModule); - } - } - - @Test - public void testSerde() throws IOException - { - Repository repository = new Avro1124RESTRepositoryClientWrapper("http://github.io"); - AvroStreamInputRowParser parser = new AvroStreamInputRowParser( - PARSE_SPEC, - new SchemaRepoBasedAvroBytesDecoder<>(new Avro1124SubjectAndIdConverter(TOPIC), repository), - false, - false - ); - ByteBufferInputRowParser parser2 = jsonMapper.readValue( - jsonMapper.writeValueAsString(parser), - ByteBufferInputRowParser.class - ); - - Assert.assertEquals(parser, parser2); - } - - @Test - public void testSerdeNonDefault() throws IOException - { - Repository repository = new Avro1124RESTRepositoryClientWrapper("http://github.io"); - AvroStreamInputRowParser parser = new AvroStreamInputRowParser( - PARSE_SPEC, - new SchemaRepoBasedAvroBytesDecoder<>(new Avro1124SubjectAndIdConverter(TOPIC), repository), - true, - true - ); - ByteBufferInputRowParser parser2 = jsonMapper.readValue( - jsonMapper.writeValueAsString(parser), - ByteBufferInputRowParser.class - ); - - Assert.assertEquals(parser, parser2); - } - - @Test - public void testParse() throws SchemaValidationException, IOException - { - // serde test - Repository repository = new InMemoryRepository(null); - AvroStreamInputRowParser parser = new AvroStreamInputRowParser( - PARSE_SPEC, - new SchemaRepoBasedAvroBytesDecoder<>(new Avro1124SubjectAndIdConverter(TOPIC), repository), - false, - false - ); - ByteBufferInputRowParser parser2 = jsonMapper.readValue( - jsonMapper.writeValueAsString(parser), - ByteBufferInputRowParser.class - ); - repository = ((SchemaRepoBasedAvroBytesDecoder) ((AvroStreamInputRowParser) parser2).getAvroBytesDecoder()).getSchemaRepository(); - - // prepare data - GenericRecord someAvroDatum = buildSomeAvroDatum(); - - // encode schema id - Avro1124SubjectAndIdConverter converter = new Avro1124SubjectAndIdConverter(TOPIC); - TypedSchemaRepository repositoryClient = new TypedSchemaRepository<>( - repository, - new IntegerConverter(), - new AvroSchemaConverter(), - new IdentityConverter() - ); - Integer id = repositoryClient.registerSchema(TOPIC, SomeAvroDatum.getClassSchema()); - ByteBuffer byteBuffer = ByteBuffer.allocate(4); - converter.putSubjectAndId(id, byteBuffer); - ByteArrayOutputStream out = new ByteArrayOutputStream(); - out.write(byteBuffer.array()); - // encode data - DatumWriter writer = new SpecificDatumWriter<>(someAvroDatum.getSchema()); - // write avro datum to bytes - writer.write(someAvroDatum, EncoderFactory.get().directBinaryEncoder(out, null)); - - InputRow inputRow = parser2.parseBatch(ByteBuffer.wrap(out.toByteArray())).get(0); - - assertInputRowCorrect(inputRow, DIMENSIONS, false); - } - - @Test - public void testParseSchemaless() throws SchemaValidationException, IOException - { - // serde test - Repository repository = new InMemoryRepository(null); - AvroStreamInputRowParser parser = new AvroStreamInputRowParser( - PARSE_SPEC_SCHEMALESS, - new SchemaRepoBasedAvroBytesDecoder<>(new Avro1124SubjectAndIdConverter(TOPIC), repository), - false, - false - ); - ByteBufferInputRowParser parser2 = jsonMapper.readValue( - jsonMapper.writeValueAsString(parser), - ByteBufferInputRowParser.class - ); - repository = ((SchemaRepoBasedAvroBytesDecoder) ((AvroStreamInputRowParser) parser2).getAvroBytesDecoder()).getSchemaRepository(); - - // prepare data - GenericRecord someAvroDatum = buildSomeAvroDatum(); - - // encode schema id - Avro1124SubjectAndIdConverter converter = new Avro1124SubjectAndIdConverter(TOPIC); - TypedSchemaRepository repositoryClient = new TypedSchemaRepository<>( - repository, - new IntegerConverter(), - new AvroSchemaConverter(), - new IdentityConverter() - ); - Integer id = repositoryClient.registerSchema(TOPIC, SomeAvroDatum.getClassSchema()); - ByteBuffer byteBuffer = ByteBuffer.allocate(4); - converter.putSubjectAndId(id, byteBuffer); - try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { - out.write(byteBuffer.array()); - // encode data - DatumWriter writer = new SpecificDatumWriter<>(someAvroDatum.getSchema()); - // write avro datum to bytes - writer.write(someAvroDatum, EncoderFactory.get().directBinaryEncoder(out, null)); - - InputRow inputRow = parser2.parseBatch(ByteBuffer.wrap(out.toByteArray())).get(0); - - assertInputRowCorrect(inputRow, DIMENSIONS_SCHEMALESS, false); - } - } - - @Test - public void testParseInvalidData() throws IOException, SchemaValidationException - { - Repository repository = new InMemoryRepository(null); - SchemaRepoBasedAvroBytesDecoder decoder = new SchemaRepoBasedAvroBytesDecoder<>( - new Avro1124SubjectAndIdConverter(TOPIC), - repository - ); - - // prepare data - GenericRecord someAvroDatum = buildSomeAvroDatum(); - - // encode schema id - Avro1124SubjectAndIdConverter converter = new Avro1124SubjectAndIdConverter(TOPIC); - TypedSchemaRepository repositoryClient = new TypedSchemaRepository<>( - repository, - new IntegerConverter(), - new AvroSchemaConverter(), - new IdentityConverter() - ); - Integer id = repositoryClient.registerSchema(TOPIC, SomeAvroDatum.getClassSchema()); - ByteBuffer byteBuffer = ByteBuffer.allocate(20); - converter.putSubjectAndId(id, byteBuffer); - ByteArrayOutputStream out = new ByteArrayOutputStream(); - out.write(new byte[0]); - out.write(byteBuffer.array()); - - DatumWriter writer = new SpecificDatumWriter<>(someAvroDatum.getSchema()); - // write avro datum to bytes - writer.write(someAvroDatum, EncoderFactory.get().directBinaryEncoder(out, null)); - - ParseException parseException = Assert.assertThrows( - ParseException.class, - () -> decoder.parse(ByteBuffer.wrap(out.toByteArray())) - ); - Assert.assertTrue(parseException.getCause() instanceof AvroRuntimeException); - Assert.assertTrue(parseException.getMessage().contains("Failed to read Avro message")); - } - - static void assertInputRowCorrect(InputRow inputRow, List expectedDimensions, boolean isFromPigAvro) - { - Assert.assertEquals(expectedDimensions, inputRow.getDimensions()); - Assert.assertEquals(1543698L, inputRow.getTimestampFromEpoch()); - - // test dimensions - Assert.assertEquals(Collections.singletonList(EVENT_TYPE_VALUE), inputRow.getDimension(EVENT_TYPE)); - Assert.assertEquals(Collections.singletonList(String.valueOf(ID_VALUE)), inputRow.getDimension(ID)); - Assert.assertEquals( - Collections.singletonList(String.valueOf(SOME_OTHER_ID_VALUE)), - inputRow.getDimension(SOME_OTHER_ID) - ); - Assert.assertEquals(Collections.singletonList(String.valueOf(true)), inputRow.getDimension(IS_VALID)); - - // someRecordArray represents a record generated from Pig using AvroStorage - // as it implicitly converts array elements to a record - if (isFromPigAvro) { - Assert.assertEquals( - Collections.singletonList(SOME_RECORD_ARRAY_VALUE.get(0).getNestedString()), - inputRow.getDimension("someRecordArray") - ); - } else { - Assert.assertEquals( - Lists.transform(SOME_INT_ARRAY_VALUE, String::valueOf), - inputRow.getDimension("someIntArray") - ); - // For string array, nulls are preserved so use ArrayList (ImmutableList doesn't support nulls) - Assert.assertEquals( - SOME_STRING_ARRAY_VALUE.stream().map(Evals::asString).collect(Collectors.toList()), - inputRow.getDimension("someStringArray") - ); - - final Object someRecordArrayObj = inputRow.getRaw("someRecordArray"); - Assert.assertNotNull(someRecordArrayObj); - Assert.assertTrue(someRecordArrayObj instanceof List); - Assert.assertEquals(1, ((List) someRecordArrayObj).size()); - final Object recordArrayElementObj = ((List) someRecordArrayObj).get(0); - Assert.assertNotNull(recordArrayElementObj); - Assert.assertTrue(recordArrayElementObj instanceof LinkedHashMap); - LinkedHashMap recordArrayElement = (LinkedHashMap) recordArrayElementObj; - Assert.assertEquals("string in record", recordArrayElement.get("nestedString")); - } - - final Object someIntValueMapObj = inputRow.getRaw("someIntValueMap"); - Assert.assertNotNull(someIntValueMapObj); - Assert.assertTrue(someIntValueMapObj instanceof LinkedHashMap); - LinkedHashMap someIntValueMap = (LinkedHashMap) someIntValueMapObj; - Assert.assertEquals(4, someIntValueMap.size()); - Assert.assertEquals(1, someIntValueMap.get("1")); - Assert.assertEquals(2, someIntValueMap.get("2")); - Assert.assertEquals(4, someIntValueMap.get("4")); - Assert.assertEquals(8, someIntValueMap.get("8")); - - - final Object someStringValueMapObj = inputRow.getRaw("someStringValueMap"); - Assert.assertNotNull(someStringValueMapObj); - Assert.assertTrue(someStringValueMapObj instanceof LinkedHashMap); - LinkedHashMap someStringValueMap = (LinkedHashMap) someStringValueMapObj; - Assert.assertEquals(4, someStringValueMap.size()); - Assert.assertEquals("1", someStringValueMap.get("1")); - Assert.assertEquals("2", someStringValueMap.get("2")); - Assert.assertEquals("4", someStringValueMap.get("4")); - Assert.assertEquals("8", someStringValueMap.get("8")); - - - final Object someRecordObj = inputRow.getRaw("someRecord"); - Assert.assertNotNull(someRecordObj); - Assert.assertTrue(someRecordObj instanceof LinkedHashMap); - LinkedHashMap someRecord = (LinkedHashMap) someRecordObj; - Assert.assertEquals(4892, someRecord.get("subInt")); - Assert.assertEquals(1543698L, someRecord.get("subLong")); - - final Object someList = inputRow.getDimension("nestedArrayVal"); - Assert.assertNotNull(someList); - Assert.assertTrue(someList instanceof List); - List someRecordObj3List = (List) someList; - Assert.assertEquals(1, someRecordObj3List.size()); - Assert.assertEquals("string in record", someRecordObj3List.get(0)); - - - // towards Map avro field as druid dimension, need to convert its toString() back to HashMap to check equality - Assert.assertEquals(1, inputRow.getDimension("someIntValueMap").size()); - Assert.assertEquals( - SOME_INT_VALUE_MAP_VALUE, - new HashMap( - Maps.transformValues( - Splitter - .on(",") - .withKeyValueSeparator("=") - .split(BRACES_AND_SPACE.matcher(inputRow.getDimension("someIntValueMap").get(0)).replaceAll("")), - new Function<>() - { - @Nullable - @Override - public Integer apply(@Nullable String input) - { - return Integer.valueOf(input); - } - } - ) - ) - ); - Assert.assertEquals( - SOME_STRING_VALUE_MAP_VALUE, - new HashMap( - Splitter - .on(",") - .withKeyValueSeparator("=") - .split(BRACES_AND_SPACE.matcher(inputRow.getDimension("someIntValueMap").get(0)).replaceAll("")) - ) - ); - Assert.assertEquals(Collections.singletonList(SOME_UNION_VALUE), inputRow.getDimension("someUnion")); - Assert.assertEquals(Collections.emptyList(), inputRow.getDimension("someNull")); - Assert.assertEquals( - Arrays.toString(SOME_FIXED_VALUE.bytes()), - Arrays.toString((byte[]) (inputRow.getRaw("someFixed"))) - ); - Assert.assertEquals( - Arrays.toString(SOME_BYTES_VALUE.array()), - Arrays.toString((byte[]) (inputRow.getRaw("someBytes"))) - ); - Assert.assertEquals(Collections.singletonList(String.valueOf(MyEnum.ENUM1)), inputRow.getDimension("someEnum")); - Assert.assertEquals( - Collections.singletonList(ImmutableMap.of("subInt", 4892, "subLong", 1543698L).toString()), - inputRow.getDimension("someRecord") - ); - - // test metrics - Assert.assertEquals(SOME_FLOAT_VALUE, inputRow.getMetric("someFloat").floatValue(), 0); - Assert.assertEquals(SOME_LONG_VALUE, inputRow.getMetric("someLong")); - Assert.assertEquals(SOME_INT_VALUE, inputRow.getMetric("someInt")); - } - - private static SomeAvroDatum.Builder createSomeAvroDatumBuilderDefaults() - { - return SomeAvroDatum.newBuilder() - .setTimestamp(DATE_TIME.toInstant().toEpochMilli()) - .setEventType(EVENT_TYPE_VALUE) - .setId(ID_VALUE) - .setSomeOtherId(SOME_OTHER_ID_VALUE) - .setIsValid(true) - .setSomeFloat(SOME_FLOAT_VALUE) - .setSomeInt(SOME_INT_VALUE) - .setSomeLong(SOME_LONG_VALUE) - .setSomeIntArray(SOME_INT_ARRAY_VALUE) - .setSomeStringArray(SOME_STRING_ARRAY_VALUE) - .setSomeIntValueMap(SOME_INT_VALUE_MAP_VALUE) - .setSomeStringValueMap(SOME_STRING_VALUE_MAP_VALUE) - .setSomeUnion(SOME_UNION_VALUE) - .setSomeMultiMemberUnion(SOME_UNION_MEMBER_VALUE) - .setSomeFixed(SOME_FIXED_VALUE) - .setSomeBytes(SOME_BYTES_VALUE) - .setSomeNull(null) - .setSomeEnum(MyEnum.ENUM1) - .setSomeRecord(SOME_RECORD_VALUE) - .setSomeRecordArray(SOME_RECORD_ARRAY_VALUE); - } - - public static SomeAvroDatum buildSomeAvroDatum() - { - return createSomeAvroDatumBuilderDefaults().build(); - } - - public static SomeAvroDatum buildSomeAvroDatumWithUnionValue(Object unionValue) - { - return createSomeAvroDatumBuilderDefaults() - .setSomeMultiMemberUnion(unionValue) - .build(); - } -} diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/AvroFlattenerMakerTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/AvroFlattenerMakerTest.java index c30b5106707a..a52cc1c38a60 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/AvroFlattenerMakerTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/AvroFlattenerMakerTest.java @@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.avro.generic.GenericRecord; import org.apache.avro.util.Utf8; -import org.apache.druid.data.input.AvroStreamInputRowParserTest; +import org.apache.druid.data.input.AvroStreamInputFormatTest; import org.apache.druid.data.input.SomeAvroDatum; import org.apache.druid.data.input.UnionSubEnum; import org.apache.druid.data.input.UnionSubFixed; @@ -46,7 +46,14 @@ public class AvroFlattenerMakerTest private static final AvroFlattenerMaker FLATTENER_WITH_EXTRACT_UNION_BY_TYPE = new AvroFlattenerMaker(false, false, true, false); - private static final SomeAvroDatum RECORD = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + private static final SomeAvroDatum RECORD = AvroStreamInputFormatTest.buildSomeAvroDatum(); + + public static SomeAvroDatum buildSomeAvroDatumWithUnionValue(Object unionValue) + { + return AvroStreamInputFormatTest.createSomeAvroDatumBuilderDefaults() + .setSomeMultiMemberUnion(unionValue) + .build(); + } @Test public void getRootField_flattenerWithoutExtractUnionsByType() @@ -85,31 +92,31 @@ public void jsonPathExtractorExtractUnionsByType() // int Assert.assertEquals(1, flattener.makeJsonPathExtractor("$.someMultiMemberUnion.int").apply( - AvroStreamInputRowParserTest.buildSomeAvroDatumWithUnionValue(1))); + buildSomeAvroDatumWithUnionValue(1))); // long Assert.assertEquals(1L, flattener.makeJsonPathExtractor("$.someMultiMemberUnion.long").apply( - AvroStreamInputRowParserTest.buildSomeAvroDatumWithUnionValue(1L))); + buildSomeAvroDatumWithUnionValue(1L))); // float Assert.assertEquals((float) 1.0, flattener.makeJsonPathExtractor("$.someMultiMemberUnion.float").apply( - AvroStreamInputRowParserTest.buildSomeAvroDatumWithUnionValue((float) 1.0))); + buildSomeAvroDatumWithUnionValue((float) 1.0))); // double Assert.assertEquals(1.0, flattener.makeJsonPathExtractor("$.someMultiMemberUnion.double").apply( - AvroStreamInputRowParserTest.buildSomeAvroDatumWithUnionValue(1.0))); + buildSomeAvroDatumWithUnionValue(1.0))); // string Assert.assertEquals("string", flattener.makeJsonPathExtractor("$.someMultiMemberUnion.string").apply( - AvroStreamInputRowParserTest.buildSomeAvroDatumWithUnionValue(new Utf8("string")))); + buildSomeAvroDatumWithUnionValue(new Utf8("string")))); // bytes Assert.assertArrayEquals(new byte[] {1}, (byte[]) flattener.makeJsonPathExtractor("$.someMultiMemberUnion.bytes").apply( - AvroStreamInputRowParserTest.buildSomeAvroDatumWithUnionValue(ByteBuffer.wrap(new byte[] {1})))); + buildSomeAvroDatumWithUnionValue(ByteBuffer.wrap(new byte[] {1})))); // map Assert.assertEquals(2, flattener.makeJsonPathExtractor("$.someMultiMemberUnion.map.two").apply( - AvroStreamInputRowParserTest.buildSomeAvroDatumWithUnionValue( + buildSomeAvroDatumWithUnionValue( ImmutableMap.builder() .put("one", 1) .put("two", 2) @@ -119,13 +126,13 @@ public void jsonPathExtractorExtractUnionsByType() // array Assert.assertEquals(3, flattener.makeJsonPathExtractor("$.someMultiMemberUnion.array[2]").apply( - AvroStreamInputRowParserTest.buildSomeAvroDatumWithUnionValue(Arrays.asList(1, 2, 3)))); + buildSomeAvroDatumWithUnionValue(Arrays.asList(1, 2, 3)))); // Named types are accessed by name // record Assert.assertEquals("subRecordString", flattener.makeJsonPathExtractor("$.someMultiMemberUnion.UnionSubRecord.subString").apply( - AvroStreamInputRowParserTest.buildSomeAvroDatumWithUnionValue( + buildSomeAvroDatumWithUnionValue( UnionSubRecord.newBuilder() .setSubString("subRecordString") .build()))); @@ -133,11 +140,11 @@ public void jsonPathExtractorExtractUnionsByType() // fixed final byte[] fixedBytes = new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; Assert.assertEquals(fixedBytes, flattener.makeJsonPathExtractor("$.someMultiMemberUnion.UnionSubFixed").apply( - AvroStreamInputRowParserTest.buildSomeAvroDatumWithUnionValue(new UnionSubFixed(fixedBytes)))); + buildSomeAvroDatumWithUnionValue(new UnionSubFixed(fixedBytes)))); // enum Assert.assertEquals(String.valueOf(UnionSubEnum.ENUM1), flattener.makeJsonPathExtractor("$.someMultiMemberUnion.UnionSubEnum").apply( - AvroStreamInputRowParserTest.buildSomeAvroDatumWithUnionValue(UnionSubEnum.ENUM1))); + buildSomeAvroDatumWithUnionValue(UnionSubEnum.ENUM1))); } @Test(expected = UnsupportedOperationException.class) @@ -164,7 +171,7 @@ public void testDiscovery() final AvroFlattenerMaker flattener = new AvroFlattenerMaker(false, false, true, false); final AvroFlattenerMaker flattenerNested = new AvroFlattenerMaker(false, false, true, true); - SomeAvroDatum input = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + SomeAvroDatum input = AvroStreamInputFormatTest.buildSomeAvroDatum(); // isFieldPrimitive on someStringArray is false // as it contains items as nulls and strings // so flattenerNested should only be able to discover it @@ -218,7 +225,7 @@ public void testNullsInStringArray() { final AvroFlattenerMaker flattenerNested = new AvroFlattenerMaker(false, false, true, true); - SomeAvroDatum input = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + SomeAvroDatum input = AvroStreamInputFormatTest.buildSomeAvroDatum(); Assert.assertEquals( ImmutableSet.of( diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/AvroOCFReaderTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/AvroOCFReaderTest.java index 9ee52c2ce43d..668142a1cfea 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/AvroOCFReaderTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/AvroOCFReaderTest.java @@ -26,7 +26,7 @@ import com.google.common.collect.Iterables; import org.apache.avro.generic.GenericRecord; import org.apache.druid.data.input.AvroHadoopInputRowParserTest; -import org.apache.druid.data.input.AvroStreamInputRowParserTest; +import org.apache.druid.data.input.AvroStreamInputFormatTest; import org.apache.druid.data.input.ColumnsFilter; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; @@ -191,7 +191,7 @@ private InputEntityReader createReader( Map readerSchema ) throws Exception { - final GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + final GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); final File someAvroFile = AvroHadoopInputRowParserTest.createAvroFile(someAvroDatum); final TimestampSpec timestampSpec = new TimestampSpec("timestamp", "auto", null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of( diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/InlineSchemaAvroBytesDecoderTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/InlineSchemaAvroBytesDecoderTest.java index 19e88937750b..bb3a1b027692 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/InlineSchemaAvroBytesDecoderTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/InlineSchemaAvroBytesDecoderTest.java @@ -27,7 +27,7 @@ import org.apache.avro.io.DatumWriter; import org.apache.avro.io.EncoderFactory; import org.apache.avro.specific.SpecificDatumWriter; -import org.apache.druid.data.input.AvroStreamInputRowParserTest; +import org.apache.druid.data.input.AvroStreamInputFormatTest; import org.apache.druid.data.input.SomeAvroDatum; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.parsers.ParseException; @@ -79,7 +79,7 @@ public void testSerde() throws Exception @Test public void testParse() throws Exception { - GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); Schema schema = SomeAvroDatum.getClassSchema(); ByteArrayOutputStream out = new ByteArrayOutputStream(); @@ -94,7 +94,7 @@ public void testParse() throws Exception @Test public void testParseInvalidEncodedData() throws Exception { - GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); Schema schema = SomeAvroDatum.getClassSchema(); // Encode data incorrectly @@ -119,7 +119,7 @@ public void testParseInvalidEncodedData() throws Exception @Test public void testParseSmallInvalidChunk() throws Exception { - GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); Schema schema = SomeAvroDatum.getClassSchema(); // Write a small chunk of data to trigger an AvroRuntimeException diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/InlineSchemasAvroBytesDecoderTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/InlineSchemasAvroBytesDecoderTest.java index 873f51236607..335ee3e3a0b8 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/InlineSchemasAvroBytesDecoderTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/InlineSchemasAvroBytesDecoderTest.java @@ -27,7 +27,7 @@ import org.apache.avro.io.DatumWriter; import org.apache.avro.io.EncoderFactory; import org.apache.avro.specific.SpecificDatumWriter; -import org.apache.druid.data.input.AvroStreamInputRowParserTest; +import org.apache.druid.data.input.AvroStreamInputFormatTest; import org.apache.druid.data.input.SomeAvroDatum; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.parsers.ParseException; @@ -89,7 +89,7 @@ public void testSerde() throws Exception @Test public void testParse() throws Exception { - GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); Schema schema = SomeAvroDatum.getClassSchema(); ByteArrayOutputStream out = new ByteArrayOutputStream(); @@ -111,7 +111,7 @@ public void testParse() throws Exception @Test public void testParseInvalidVersion() throws Exception { - GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); Schema schema = SomeAvroDatum.getClassSchema(); ByteArrayOutputStream out = new ByteArrayOutputStream(); @@ -134,7 +134,7 @@ public void testParseInvalidVersion() throws Exception @Test public void testParseInvalidSchemaId() throws Exception { - GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); Schema schema = SomeAvroDatum.getClassSchema(); ByteArrayOutputStream out = new ByteArrayOutputStream(); @@ -158,7 +158,7 @@ public void testParseInvalidSchemaId() throws Exception @Test public void testParseInvalidData() throws Exception { - GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); Schema schema = SomeAvroDatum.getClassSchema(); ByteArrayOutputStream out = new ByteArrayOutputStream(); diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java index 2aad88e8ff58..cf1135cc85b9 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java @@ -31,7 +31,7 @@ import org.apache.avro.io.DatumWriter; import org.apache.avro.io.EncoderFactory; import org.apache.avro.specific.SpecificDatumWriter; -import org.apache.druid.data.input.AvroStreamInputRowParserTest; +import org.apache.druid.data.input.AvroStreamInputFormatTest; import org.apache.druid.data.input.SomeAvroDatum; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.parsers.ParseException; @@ -116,7 +116,7 @@ public void testParse() throws Exception // Given Mockito.when(registry.getSchemaById(ArgumentMatchers.eq(1234))) .thenReturn(new AvroSchema(SomeAvroDatum.getClassSchema())); - GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); Schema schema = SomeAvroDatum.getClassSchema(); byte[] bytes = getAvroDatum(schema, someAvroDatum); ByteBuffer bb = ByteBuffer.allocate(bytes.length + 5).put((byte) 0).putInt(1234).put(bytes); @@ -153,7 +153,7 @@ public void testParseCorruptedPartial() throws Exception // Given Mockito.when(registry.getSchemaById(ArgumentMatchers.eq(1234))) .thenReturn(new AvroSchema(SomeAvroDatum.getClassSchema())); - GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); Schema schema = SomeAvroDatum.getClassSchema(); byte[] bytes = getAvroDatum(schema, someAvroDatum); ByteBuffer bb = ByteBuffer.allocate(4 + 5).put((byte) 0).putInt(1234).put(bytes, 5, 4); @@ -265,7 +265,7 @@ public void testParseWhenUnauthenticatedException() throws IOException, RestClie // Given Mockito.when(registry.getSchemaById(ArgumentMatchers.eq(1234))) .thenThrow(new RestClientException("unauthenticated", 401, 401)); - GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); Schema schema = SomeAvroDatum.getClassSchema(); byte[] bytes = getAvroDatum(schema, someAvroDatum); ByteBuffer bb = ByteBuffer.allocate(4 + 5).put((byte) 0).putInt(1234).put(bytes, 5, 4); @@ -292,7 +292,7 @@ public void testParseWhenResourceNotFoundException() throws IOException, RestCli // Given Mockito.when(registry.getSchemaById(ArgumentMatchers.eq(1234))) .thenThrow(new RestClientException("resource doesn't exist", 404, 404)); - GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum(); + GenericRecord someAvroDatum = AvroStreamInputFormatTest.buildSomeAvroDatum(); Schema schema = SomeAvroDatum.getClassSchema(); byte[] bytes = getAvroDatum(schema, someAvroDatum); ByteBuffer bb = ByteBuffer.allocate(4 + 5).put((byte) 0).putInt(1234).put(bytes, 5, 4); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 402c6b9144b7..936bd32718eb 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -101,11 +101,7 @@ long getPollRetryMs() protected SeekableStreamIndexTaskRunner createTaskRunner() { //noinspection unchecked - return new KafkaIndexTaskRunner( - this, - dataSchema.getParser(), - lockGranularityToUse - ); + return new KafkaIndexTaskRunner(this, lockGranularityToUse); } @Override diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java index 8dbcbdcf92cf..3da19165771b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.kafka.KafkaRecordEntity; import org.apache.druid.data.input.kafka.KafkaTopicPartition; import org.apache.druid.indexing.common.LockGranularity; @@ -45,7 +44,6 @@ import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -64,15 +62,10 @@ public class KafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner parser, LockGranularity lockGranularityToUse ) { - super( - task, - parser, - lockGranularityToUse - ); + super(task, lockGranularityToUse); this.task = task; } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java index 872752b7927c..e0e46cf41b22 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java @@ -25,14 +25,10 @@ import org.apache.curator.test.TestingCluster; import org.apache.druid.client.indexing.SamplerResponse; import org.apache.druid.client.indexing.SamplerSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; @@ -42,10 +38,8 @@ import org.apache.druid.indexing.overlord.sampler.SamplerConfig; import org.apache.druid.indexing.overlord.sampler.SamplerException; import org.apache.druid.indexing.overlord.sampler.SamplerTestUtils; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.TestHelper; @@ -64,7 +58,6 @@ import org.junit.Test; import org.junit.rules.ExpectedException; -import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -237,53 +230,6 @@ public void testSampleKafkaInputFormat() Assert.assertFalse(it.hasNext()); } - @Test - public void testWithInputRowParser() - { - insertData(generateRecords(TOPIC)); - - ObjectMapper objectMapper = new DefaultObjectMapper(); - TimestampSpec timestampSpec = new TimestampSpec("timestamp", "iso", null); - DimensionsSpec dimensionsSpec = new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ); - InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8"); - - Consumer dataSchema = - builder -> builder.withDataSource("test_ds") - .withParserMap(objectMapper.convertValue(parser, Map.class)) - .withAggregators( - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - ) - .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) - .withObjectMapper(objectMapper); - - KafkaSupervisorSpec supervisorSpec = new KafkaSupervisorSpecBuilder() - .withDataSchema(dataSchema) - .withIoConfig( - ioConfig -> ioConfig - .withConsumerProperties(kafkaServer.consumerProperties()) - .withUseEarliestSequenceNumber(true) - ) - .build(DATASOURCE, TOPIC); - - KafkaSamplerSpec samplerSpec = new KafkaSamplerSpec( - supervisorSpec, - new SamplerConfig(5, 5_000, null, null), - new InputSourceSampler(new DefaultObjectMapper()), - OBJECT_MAPPER - ); - - runSamplerAndCompareResponse(samplerSpec, false); - } - private static void runSamplerAndCompareResponse(SamplerSpec samplerSpec, boolean useInputFormat) { SamplerResponse response = samplerSpec.sample(); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 626586c23ac5..b6c47d05bb39 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -110,11 +110,7 @@ public TaskStatus runTask(TaskToolbox toolbox) protected SeekableStreamIndexTaskRunner createTaskRunner() { //noinspection unchecked - return new KinesisIndexTaskRunner( - this, - dataSchema.getParser(), - lockGranularityToUse - ); + return new KinesisIndexTaskRunner(this, lockGranularityToUse); } @Override diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index 1c6bdabe0c46..3e2f656329f9 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; @@ -40,7 +39,6 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -57,11 +55,10 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner parser, LockGranularity lockGranularityToUse ) { - super(task, parser, lockGranularityToUse); + super(task, lockGranularityToUse); this.task = task; } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index c967f8ae5e89..4c9f4a1c8e2c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -26,14 +26,10 @@ import org.apache.druid.client.indexing.SamplerResponse; import org.apache.druid.client.indexing.SamplerSpec; import org.apache.druid.common.aws.AWSCredentialsConfig; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexer.granularity.UniformGranularitySpec; @@ -61,14 +57,11 @@ import org.junit.Test; import software.amazon.kinesis.retrieval.KinesisClientRecord; -import java.io.IOException; import java.nio.ByteBuffer; import java.time.Instant; -import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Map; public class KinesisSamplerSpecTest extends EasyMockSupport { @@ -173,87 +166,6 @@ public void testSample() throws InterruptedException runSamplerAndCompareResponse(samplerSpec, true); } - @Test - public void testSampleWithInputRowParser() throws IOException, InterruptedException - { - ObjectMapper objectMapper = new DefaultObjectMapper(); - TimestampSpec timestampSpec = new TimestampSpec("timestamp", "iso", null); - DimensionsSpec dimensionsSpec = new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ); - InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8"); - - DataSchema dataSchema = DataSchema.builder() - .withDataSource("test_ds") - .withParserMap( - objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class) - ) - .withAggregators( - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - ) - .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) - .withObjectMapper(objectMapper) - .build(); - - KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec( - null, - null, - dataSchema, - null, - new KinesisSupervisorIOConfig( - STREAM, - null, - null, - null, - null, - null, - null, - null, - null, - true, - null, - null, - null, - null, - null, - null, - null, - null, - null, - false, - null - ), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); - - KinesisSamplerSpec samplerSpec = new TestableKinesisSamplerSpec( - supervisorSpec, - new SamplerConfig(5, null, null, null), - new InputSourceSampler(new DefaultObjectMapper()), - null - ); - - runSamplerAndCompareResponse(samplerSpec, false); - } - @Test public void testGetInputSourceResources() { diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufExtensionsModule.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufExtensionsModule.java index a2293ec794cc..5a74f003c9b7 100644 --- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufExtensionsModule.java +++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufExtensionsModule.java @@ -37,7 +37,6 @@ public List getJacksonModules() return Collections.singletonList( new SimpleModule("ProtobufInputRowParserModule") .registerSubtypes( - new NamedType(ProtobufInputRowParser.class, "protobuf"), new NamedType(ProtobufInputFormat.class, "protobuf") ) ); diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParser.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParser.java deleted file mode 100644 index 9b63edebdd33..000000000000 --- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParser.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.data.input.protobuf; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.google.protobuf.DynamicMessage; -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.util.JsonFormat; -import org.apache.druid.data.input.ByteBufferInputRowParser; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.java.util.common.parsers.Parser; -import org.apache.druid.utils.CollectionUtils; -import org.joda.time.DateTime; - -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Map; - -public class ProtobufInputRowParser implements ByteBufferInputRowParser -{ - private final ParseSpec parseSpec; - // timestamp spec to be used for parsing timestamp - private final TimestampSpec timestampSpec; - // whether the spec has any fields to flat - private final boolean isFlatSpec; - private final ProtobufBytesDecoder protobufBytesDecoder; - private Parser parser; - private final List dimensions; - - @JsonCreator - public ProtobufInputRowParser( - @JsonProperty("parseSpec") ParseSpec parseSpec, - @JsonProperty("protoBytesDecoder") ProtobufBytesDecoder protobufBytesDecoder, - @Deprecated - @JsonProperty("descriptor") String descriptorFilePath, - @Deprecated - @JsonProperty("protoMessageType") String protoMessageType - ) - { - this.parseSpec = parseSpec; - this.dimensions = parseSpec.getDimensionsSpec().getDimensionNames(); - this.isFlatSpec = parseSpec instanceof JSONParseSpec && ((JSONParseSpec) parseSpec).getFlattenSpec().getFields().isEmpty(); - if (descriptorFilePath != null || protoMessageType != null) { - this.protobufBytesDecoder = new FileBasedProtobufBytesDecoder(descriptorFilePath, protoMessageType); - } else { - this.protobufBytesDecoder = protobufBytesDecoder; - } - if (isFlatSpec) { - this.timestampSpec = new ProtobufInputRowSchema.ProtobufTimestampSpec(parseSpec.getTimestampSpec()); - } else { - this.timestampSpec = parseSpec.getTimestampSpec(); - } - - } - - @Override - @JsonProperty("parseSpec") - public ParseSpec getParseSpec() - { - return parseSpec; - } - - @JsonProperty("protoBytesDecoder") - public ProtobufBytesDecoder getProtobufBytesDecoder() - { - return protobufBytesDecoder; - } - - @Override - public ProtobufInputRowParser withParseSpec(ParseSpec parseSpec) - { - return new ProtobufInputRowParser(parseSpec, protobufBytesDecoder, null, null); - } - - @Override - public List parseBatch(ByteBuffer input) - { - if (parser == null) { - parser = parseSpec.makeParser(); - } - Map record; - DateTime timestamp; - - if (isFlatSpec) { - try { - DynamicMessage message = protobufBytesDecoder.parse(input); - record = CollectionUtils.mapKeys(message.getAllFields(), k -> k.getJsonName()); - timestamp = this.timestampSpec.extractTimestamp(record); - } - catch (Exception ex) { - throw new ParseException(null, ex, "Protobuf message could not be parsed"); - } - } else { - try { - DynamicMessage message = protobufBytesDecoder.parse(input); - String json = JsonFormat.printer().print(message); - record = parser.parseToMap(json); - timestamp = this.timestampSpec.extractTimestamp(record); - } - catch (InvalidProtocolBufferException e) { - throw new ParseException(null, e, "Protobuf message could not be parsed"); - } - } - - final List dimensions; - if (!this.dimensions.isEmpty()) { - dimensions = this.dimensions; - } else { - dimensions = Lists.newArrayList( - Sets.difference(record.keySet(), parseSpec.getDimensionsSpec().getDimensionExclusions()) - ); - } - return ImmutableList.of(new MapBasedInputRow(timestamp, dimensions, record)); - } -} 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 36664418b68c..34e0a183390b 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 @@ -27,6 +27,8 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.io.Files; +import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; @@ -51,10 +53,14 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashSet; +import java.util.List; +import java.util.Objects; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -144,9 +150,9 @@ public void testParseFlattenData() throws Exception //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildNestedData(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = buildNestedData(dateTime); - final ByteEntity entity = new ByteEntity(ProtobufInputRowParserTest.toByteBuffer(event)); + final ByteEntity entity = new ByteEntity(toByteBuffer(event)); InputRow row = protobufInputFormat.createReader( new InputRowSchema(timestampSpec, dimensionsSpec, null), @@ -165,7 +171,7 @@ public void testParseFlattenData() throws Exception row.getDimensions() ); - ProtobufInputRowParserTest.verifyNestedData(row, dateTime); + verifyNestedData(row, dateTime); } @Test @@ -186,9 +192,9 @@ public void testParseFlattenDataJq() throws Exception //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildNestedData(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = buildNestedData(dateTime); - final ByteEntity entity = new ByteEntity(ProtobufInputRowParserTest.toByteBuffer(event)); + final ByteEntity entity = new ByteEntity(toByteBuffer(event)); InputRow row = protobufInputFormat.createReader( new InputRowSchema(timestampSpec, dimensionsSpec, null), @@ -207,7 +213,7 @@ public void testParseFlattenDataJq() throws Exception row.getDimensions() ); - ProtobufInputRowParserTest.verifyNestedData(row, dateTime); + verifyNestedData(row, dateTime); } @Test @@ -218,9 +224,9 @@ public void testParseFlattenDataDiscover() throws Exception //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildNestedData(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = buildNestedData(dateTime); - final ByteEntity entity = new ByteEntity(ProtobufInputRowParserTest.toByteBuffer(event)); + final ByteEntity entity = new ByteEntity(toByteBuffer(event)); InputRow row = protobufInputFormat.createReader( new InputRowSchema(timestampSpec, new DimensionsSpec(Collections.emptyList()), null), @@ -245,7 +251,7 @@ public void testParseFlattenDataDiscover() throws Exception new HashSet<>(row.getDimensions()) ); - ProtobufInputRowParserTest.verifyNestedData(row, dateTime); + verifyNestedData(row, dateTime); } @Test @@ -258,9 +264,9 @@ public void testParseNestedData() throws Exception //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildNestedData(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = buildNestedData(dateTime); - final ByteEntity entity = new ByteEntity(ProtobufInputRowParserTest.toByteBuffer(event)); + final ByteEntity entity = new ByteEntity(toByteBuffer(event)); InputEntityReader reader = protobufInputFormat.createReader( new InputRowSchema( @@ -321,7 +327,7 @@ public void testParseNestedData() throws Exception new byte[]{0x01, 0x02, 0x03, 0x04}, (byte[]) row.getRaw("someBytesColumn") ); - ProtobufInputRowParserTest.verifyNestedData(row, dateTime); + verifyNestedData(row, dateTime); } @@ -335,9 +341,9 @@ public void testParseNestedDataSchemaless() throws Exception //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildNestedData(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = buildNestedData(dateTime); - final ByteEntity entity = new ByteEntity(ProtobufInputRowParserTest.toByteBuffer(event)); + final ByteEntity entity = new ByteEntity(toByteBuffer(event)); InputEntityReader reader = protobufInputFormat.createReader( new InputRowSchema( @@ -391,7 +397,7 @@ public void testParseNestedDataSchemaless() throws Exception new byte[]{0x01, 0x02, 0x03, 0x04}, (byte[]) row.getRaw("someBytesColumn") ); - ProtobufInputRowParserTest.verifyNestedData(row, dateTime); + verifyNestedData(row, dateTime); } @@ -405,9 +411,9 @@ public void testParseNestedDataTransformsOnly() throws Exception //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildNestedData(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = buildNestedData(dateTime); - final ByteEntity entity = new ByteEntity(ProtobufInputRowParserTest.toByteBuffer(event)); + final ByteEntity entity = new ByteEntity(toByteBuffer(event)); InputEntityReader reader = protobufInputFormat.createReader( new InputRowSchema( @@ -441,7 +447,7 @@ public void testParseNestedDataTransformsOnly() throws Exception InputRow row = transformingReader.read().next(); - ProtobufInputRowParserTest.verifyNestedData(row, dateTime); + verifyNestedData(row, dateTime); } @@ -453,9 +459,9 @@ public void testParseFlatData() throws Exception //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildFlatData(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = buildFlatData(dateTime); - final ByteEntity entity = new ByteEntity(ProtobufInputRowParserTest.toByteBuffer(event)); + final ByteEntity entity = new ByteEntity(toByteBuffer(event)); InputRow row = protobufInputFormat.createReader( new InputRowSchema(timestampSpec, dimensionsSpec, null), @@ -463,7 +469,7 @@ public void testParseFlatData() throws Exception null ).read().next(); - ProtobufInputRowParserTest.verifyFlatData(row, dateTime, false); + verifyFlatData(row, dateTime, false); } @Test @@ -474,9 +480,9 @@ public void testParseNestedDataWithInlineSchema() throws Exception //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildNestedData(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = buildNestedData(dateTime); - final ByteEntity entity = new ByteEntity(ProtobufInputRowParserTest.toByteBuffer(event)); + final ByteEntity entity = new ByteEntity(toByteBuffer(event)); InputRow row = protobufInputFormat.createReader( new InputRowSchema(timestampSpec, dimensionsSpec, null), @@ -484,6 +490,127 @@ public void testParseNestedDataWithInlineSchema() throws Exception null ).read().next(); - ProtobufInputRowParserTest.verifyNestedData(row, dateTime); + verifyNestedData(row, dateTime); + } + + + + private static void assertDimensionEquals(InputRow row, String dimension, Object expected) + { + List values = row.getDimension(dimension); + assertEquals(1, values.size()); + assertEquals(expected, values.get(0)); + } + + static ProtoTestEventWrapper.ProtoTestEvent buildFlatData(DateTime dateTime) + { + return ProtoTestEventWrapper.ProtoTestEvent.newBuilder() + .setDescription("description") + .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE) + .setId(4711L) + .setIsValid(true) + .setSomeOtherId(4712) + .setTimestamp(dateTime.toString()) + .setSomeFloatColumn(47.11F) + .setSomeIntColumn(815) + .setSomeLongColumn(816L) + .setSomeBytesColumn(ByteString.copyFrom(new byte[]{0x01, 0x02, 0x03, 0x04})) + .build(); + } + + static void verifyFlatData(InputRow row, DateTime dateTime, boolean badBytesConversion) + { + assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch()); + + assertDimensionEquals(row, "id", "4711"); + assertDimensionEquals(row, "isValid", "true"); + assertDimensionEquals(row, "someOtherId", "4712"); + assertDimensionEquals(row, "description", "description"); + if (badBytesConversion) { + // legacy flattener used by parser doesn't convert bytes, instead calls tostring + // this can be removed if we update the parser to use the protobuf flattener used by the input format/reader + assertDimensionEquals(row, "someBytesColumn", Objects.requireNonNull(row.getRaw("someBytesColumn")).toString()); + } else { + assertDimensionEquals(row, "someBytesColumn", StringUtils.encodeBase64String(new byte[]{0x01, 0x02, 0x03, 0x04})); + } + + assertEquals(47.11F, row.getMetric("someFloatColumn").floatValue(), 0.0); + assertEquals(815.0F, row.getMetric("someIntColumn").floatValue(), 0.0); + assertEquals(816.0F, row.getMetric("someLongColumn").floatValue(), 0.0); + } + + static ProtoTestEventWrapper.ProtoTestEvent buildNestedData(DateTime dateTime) + { + return ProtoTestEventWrapper.ProtoTestEvent.newBuilder() + .setDescription("description") + .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE) + .setId(4711L) + .setIsValid(true) + .setSomeOtherId(4712) + .setTimestamp(dateTime.toString()) + .setSomeFloatColumn(47.11F) + .setSomeIntColumn(815) + .setSomeLongColumn(816L) + .setSomeBytesColumn(ByteString.copyFrom(new byte[]{0x01, 0x02, 0x03, 0x04})) + .setFoo(ProtoTestEventWrapper.ProtoTestEvent.Foo + .newBuilder() + .setBar("baz")) + .addBar(ProtoTestEventWrapper.ProtoTestEvent.Foo + .newBuilder() + .setBar("bar0")) + .addBar(ProtoTestEventWrapper.ProtoTestEvent.Foo + .newBuilder() + .setBar("bar1")) + .build(); + } + + static void verifyNestedData(InputRow row, DateTime dateTime) + { + assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch()); + + assertDimensionEquals(row, "id", "4711"); + assertDimensionEquals(row, "isValid", "true"); + assertDimensionEquals(row, "someOtherId", "4712"); + assertDimensionEquals(row, "description", "description"); + + assertDimensionEquals(row, "eventType", ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE.name()); + assertDimensionEquals(row, "foobar", "baz"); + assertDimensionEquals(row, "bar0", "bar0"); + assertDimensionEquals(row, "someBytesColumn", StringUtils.encodeBase64String(new byte[]{0x01, 0x02, 0x03, 0x04})); + + assertEquals(47.11F, row.getMetric("someFloatColumn").floatValue(), 0.0); + assertEquals(815.0F, row.getMetric("someIntColumn").floatValue(), 0.0); + assertEquals(816.0F, row.getMetric("someLongColumn").floatValue(), 0.0); + } + + static ProtoTestEventWrapper.ProtoTestEvent buildFlatDataWithComplexTimestamp(DateTime dateTime) + { + Timestamp timestamp = Timestamp.newBuilder().setSeconds(dateTime.getMillis() / 1000).setNanos((int) ((dateTime.getMillis() % 1000) * 1000 * 1000)).build(); + return ProtoTestEventWrapper.ProtoTestEvent.newBuilder() + .setDescription("description") + .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE) + .setId(4711L) + .setIsValid(true) + .setSomeOtherId(4712) + .setOtherTimestamp(timestamp) + .setTimestamp("unused") + .setSomeFloatColumn(47.11F) + .setSomeIntColumn(815) + .setSomeLongColumn(816L) + .setSomeBytesColumn(ByteString.copyFrom(new byte[]{0x01, 0x02, 0x03, 0x04})) + .build(); + } + + static void verifyFlatDataWithComplexTimestamp(InputRow row, DateTime dateTime, boolean badBytesConversion) + { + verifyFlatData(row, dateTime, badBytesConversion); + } + + static ByteBuffer toByteBuffer(ProtoTestEventWrapper.ProtoTestEvent event) throws IOException + { + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + event.writeTo(out); + return ByteBuffer.wrap(out.toByteArray()); + } } } diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java deleted file mode 100644 index 09f41b7470de..000000000000 --- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java +++ /dev/null @@ -1,331 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.data.input.protobuf; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.protobuf.ByteString; -import com.google.protobuf.Timestamp; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.JavaScriptParseSpec; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; -import org.apache.druid.java.util.common.parsers.JSONPathFieldType; -import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.js.JavaScriptConfig; -import org.apache.druid.segment.TestHelper; -import org.joda.time.DateTime; -import org.joda.time.chrono.ISOChronology; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Objects; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - -public class ProtobufInputRowParserTest -{ - private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); - - private ParseSpec parseSpec; - private ParseSpec flatParseSpec; - private ParseSpec flatParseSpecWithComplexTimestamp; - private FileBasedProtobufBytesDecoder decoder; - - @BeforeEach - public void setUp() - { - parseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(Lists.newArrayList( - new StringDimensionSchema("event"), - new StringDimensionSchema("id"), - new StringDimensionSchema("someOtherId"), - new StringDimensionSchema("isValid") - )), - new JSONPathSpec( - true, - Lists.newArrayList( - new JSONPathFieldSpec(JSONPathFieldType.ROOT, "eventType", "eventType"), - new JSONPathFieldSpec(JSONPathFieldType.PATH, "foobar", "$.foo.bar"), - new JSONPathFieldSpec(JSONPathFieldType.PATH, "bar0", "$.bar[0].bar") - ) - ), - null, - null - ); - - flatParseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(Lists.newArrayList( - new StringDimensionSchema("event"), - new StringDimensionSchema("id"), - new StringDimensionSchema("someOtherId"), - new StringDimensionSchema("isValid") - )), - - null, - null, - null - ); - - flatParseSpecWithComplexTimestamp = new JSONParseSpec( - new TimestampSpec("otherTimestamp", "iso", null), - new DimensionsSpec(Lists.newArrayList( - new StringDimensionSchema("event"), - new StringDimensionSchema("id"), - new StringDimensionSchema("someOtherId"), - new StringDimensionSchema("isValid") - )), - - null, - null, - null - ); - decoder = new FileBasedProtobufBytesDecoder("proto_test_event.desc", "ProtoTestEvent"); - } - - @Test - public void testSerde() throws Exception - { - ProtobufInputRowParser originalParser = new ProtobufInputRowParser(parseSpec, decoder, null, null); - - assertEquals(parseSpec, originalParser.getParseSpec()); - assertEquals(decoder, originalParser.getProtobufBytesDecoder()); - - String json = OBJECT_MAPPER.writeValueAsString(originalParser); - - ProtobufInputRowParser deserializedParser = OBJECT_MAPPER.readValue(json, ProtobufInputRowParser.class); - - assertEquals(originalParser.getParseSpec(), deserializedParser.getParseSpec()); - assertEquals(originalParser.getProtobufBytesDecoder(), deserializedParser.getProtobufBytesDecoder()); - } - - @Test - public void testParseNestedData() throws Exception - { - //configure parser with desc file - ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, decoder, null, null); - - //create binary of proto test event - DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - - ProtoTestEventWrapper.ProtoTestEvent event = buildNestedData(dateTime); - - InputRow row = parser.parseBatch(toByteBuffer(event)).get(0); - verifyNestedData(row, dateTime); - } - - @Test - public void testParseFlatData() throws Exception - { - //configure parser with desc file - ProtobufInputRowParser parser = new ProtobufInputRowParser(flatParseSpec, decoder, null, null); - - //create binary of proto test event - DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = buildFlatData(dateTime); - - InputRow row = parser.parseBatch(toByteBuffer(event)).get(0); - verifyFlatData(row, dateTime, true); - } - - @Test - public void testParseFlatDataWithComplexTimestamp() throws Exception - { - ProtobufInputRowParser parser = new ProtobufInputRowParser(flatParseSpecWithComplexTimestamp, decoder, null, null); - - //create binary of proto test event - DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = buildFlatDataWithComplexTimestamp(dateTime); - - InputRow row = parser.parseBatch(toByteBuffer(event)).get(0); - - verifyFlatDataWithComplexTimestamp(row, dateTime, true); - } - - @Test - public void testDisableJavaScript() - { - final JavaScriptParseSpec parseSpec = new JavaScriptParseSpec( - new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas( - ImmutableList.of( - "dim1", - "dim2" - ) - ) - ), - "func", - new JavaScriptConfig(false) - ); - final ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, decoder, null, null); - - assertThrows(IllegalStateException.class, () -> { - parser.parseBatch(ByteBuffer.allocate(1)).get(0); - }, "JavaScript is disabled"); - } - - @Test - public void testOldParserConfig() throws Exception - { - //configure parser with desc file - ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, null, "proto_test_event.desc", "ProtoTestEvent"); - - //create binary of proto test event - DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = buildNestedData(dateTime); - - InputRow row = parser.parseBatch(toByteBuffer(event)).get(0); - - verifyNestedData(row, dateTime); - } - - private static void assertDimensionEquals(InputRow row, String dimension, Object expected) - { - List values = row.getDimension(dimension); - assertEquals(1, values.size()); - assertEquals(expected, values.get(0)); - } - - static ProtoTestEventWrapper.ProtoTestEvent buildFlatData(DateTime dateTime) - { - return ProtoTestEventWrapper.ProtoTestEvent.newBuilder() - .setDescription("description") - .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE) - .setId(4711L) - .setIsValid(true) - .setSomeOtherId(4712) - .setTimestamp(dateTime.toString()) - .setSomeFloatColumn(47.11F) - .setSomeIntColumn(815) - .setSomeLongColumn(816L) - .setSomeBytesColumn(ByteString.copyFrom(new byte[]{0x01, 0x02, 0x03, 0x04})) - .build(); - } - - static void verifyFlatData(InputRow row, DateTime dateTime, boolean badBytesConversion) - { - assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch()); - - assertDimensionEquals(row, "id", "4711"); - assertDimensionEquals(row, "isValid", "true"); - assertDimensionEquals(row, "someOtherId", "4712"); - assertDimensionEquals(row, "description", "description"); - if (badBytesConversion) { - // legacy flattener used by parser doesn't convert bytes, instead calls tostring - // this can be removed if we update the parser to use the protobuf flattener used by the input format/reader - assertDimensionEquals(row, "someBytesColumn", Objects.requireNonNull(row.getRaw("someBytesColumn")).toString()); - } else { - assertDimensionEquals(row, "someBytesColumn", StringUtils.encodeBase64String(new byte[]{0x01, 0x02, 0x03, 0x04})); - } - - assertEquals(47.11F, row.getMetric("someFloatColumn").floatValue(), 0.0); - assertEquals(815.0F, row.getMetric("someIntColumn").floatValue(), 0.0); - assertEquals(816.0F, row.getMetric("someLongColumn").floatValue(), 0.0); - } - - static ProtoTestEventWrapper.ProtoTestEvent buildNestedData(DateTime dateTime) - { - return ProtoTestEventWrapper.ProtoTestEvent.newBuilder() - .setDescription("description") - .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE) - .setId(4711L) - .setIsValid(true) - .setSomeOtherId(4712) - .setTimestamp(dateTime.toString()) - .setSomeFloatColumn(47.11F) - .setSomeIntColumn(815) - .setSomeLongColumn(816L) - .setSomeBytesColumn(ByteString.copyFrom(new byte[]{0x01, 0x02, 0x03, 0x04})) - .setFoo(ProtoTestEventWrapper.ProtoTestEvent.Foo - .newBuilder() - .setBar("baz")) - .addBar(ProtoTestEventWrapper.ProtoTestEvent.Foo - .newBuilder() - .setBar("bar0")) - .addBar(ProtoTestEventWrapper.ProtoTestEvent.Foo - .newBuilder() - .setBar("bar1")) - .build(); - } - - static void verifyNestedData(InputRow row, DateTime dateTime) - { - assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch()); - - assertDimensionEquals(row, "id", "4711"); - assertDimensionEquals(row, "isValid", "true"); - assertDimensionEquals(row, "someOtherId", "4712"); - assertDimensionEquals(row, "description", "description"); - - assertDimensionEquals(row, "eventType", ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE.name()); - assertDimensionEquals(row, "foobar", "baz"); - assertDimensionEquals(row, "bar0", "bar0"); - assertDimensionEquals(row, "someBytesColumn", StringUtils.encodeBase64String(new byte[]{0x01, 0x02, 0x03, 0x04})); - - assertEquals(47.11F, row.getMetric("someFloatColumn").floatValue(), 0.0); - assertEquals(815.0F, row.getMetric("someIntColumn").floatValue(), 0.0); - assertEquals(816.0F, row.getMetric("someLongColumn").floatValue(), 0.0); - } - - static ProtoTestEventWrapper.ProtoTestEvent buildFlatDataWithComplexTimestamp(DateTime dateTime) - { - Timestamp timestamp = Timestamp.newBuilder().setSeconds(dateTime.getMillis() / 1000).setNanos((int) ((dateTime.getMillis() % 1000) * 1000 * 1000)).build(); - return ProtoTestEventWrapper.ProtoTestEvent.newBuilder() - .setDescription("description") - .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE) - .setId(4711L) - .setIsValid(true) - .setSomeOtherId(4712) - .setOtherTimestamp(timestamp) - .setTimestamp("unused") - .setSomeFloatColumn(47.11F) - .setSomeIntColumn(815) - .setSomeLongColumn(816L) - .setSomeBytesColumn(ByteString.copyFrom(new byte[]{0x01, 0x02, 0x03, 0x04})) - .build(); - } - - static void verifyFlatDataWithComplexTimestamp(InputRow row, DateTime dateTime, boolean badBytesConversion) - { - verifyFlatData(row, dateTime, badBytesConversion); - } - - static ByteBuffer toByteBuffer(ProtoTestEventWrapper.ProtoTestEvent event) throws IOException - { - try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { - event.writeTo(out); - return ByteBuffer.wrap(out.toByteArray()); - } - } -} diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufReaderTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufReaderTest.java index f630f550afc8..b8ef64d8b742 100644 --- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufReaderTest.java +++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufReaderTest.java @@ -79,13 +79,13 @@ public void testParseNestedData() throws Exception //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildNestedData(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputFormatTest.buildNestedData(dateTime); - ByteBuffer buffer = ProtobufInputRowParserTest.toByteBuffer(event); + ByteBuffer buffer = ProtobufInputFormatTest.toByteBuffer(event); InputRow row = reader.parseInputRows(decoder.parse(buffer)).get(0); - ProtobufInputRowParserTest.verifyNestedData(row, dateTime); + ProtobufInputFormatTest.verifyNestedData(row, dateTime); } @Test @@ -95,13 +95,13 @@ public void testParseFlatData() throws Exception //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildFlatData(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputFormatTest.buildFlatData(dateTime); - ByteBuffer buffer = ProtobufInputRowParserTest.toByteBuffer(event); + ByteBuffer buffer = ProtobufInputFormatTest.toByteBuffer(event); InputRow row = reader.parseInputRows(decoder.parse(buffer)).get(0); - ProtobufInputRowParserTest.verifyFlatData(row, dateTime, false); + ProtobufInputFormatTest.verifyFlatData(row, dateTime, false); } @Test @@ -111,13 +111,13 @@ public void testParseFlatDataWithComplexTimestamp() throws Exception //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildFlatDataWithComplexTimestamp(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputFormatTest.buildFlatDataWithComplexTimestamp(dateTime); - ByteBuffer buffer = ProtobufInputRowParserTest.toByteBuffer(event); + ByteBuffer buffer = ProtobufInputFormatTest.toByteBuffer(event); InputRow row = reader.parseInputRows(decoder.parse(buffer)).get(0); - ProtobufInputRowParserTest.verifyFlatDataWithComplexTimestamp(row, dateTime, false); + ProtobufInputFormatTest.verifyFlatDataWithComplexTimestamp(row, dateTime, false); } @Test @@ -132,12 +132,12 @@ public void testParseFlatDataWithComplexTimestampWithDefaultFlattenSpec() throws //create binary of proto test event DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildFlatDataWithComplexTimestamp(dateTime); + ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputFormatTest.buildFlatDataWithComplexTimestamp(dateTime); - ByteBuffer buffer = ProtobufInputRowParserTest.toByteBuffer(event); + ByteBuffer buffer = ProtobufInputFormatTest.toByteBuffer(event); InputRow row = reader.parseInputRows(decoder.parse(buffer)).get(0); - ProtobufInputRowParserTest.verifyFlatDataWithComplexTimestamp(row, dateTime, false); + ProtobufInputFormatTest.verifyFlatDataWithComplexTimestamp(row, dateTime, false); } } diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/SchemaRegistryBasedProtobufBytesDecoderTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/SchemaRegistryBasedProtobufBytesDecoderTest.java index 29d598a2be20..1a5a27e1f04f 100644 --- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/SchemaRegistryBasedProtobufBytesDecoderTest.java +++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/SchemaRegistryBasedProtobufBytesDecoderTest.java @@ -119,7 +119,7 @@ public void testGivenCapacity() private ProtoTestEventWrapper.ProtoTestEvent getTestEvent() { DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC()); - return ProtobufInputRowParserTest.buildFlatData(dateTime); + return ProtobufInputFormatTest.buildFlatData(dateTime); } 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 a448e01f9189..0ab9f613cbe3 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 @@ -43,7 +43,6 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.ByteEntity; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.discovery.NodeRole; @@ -111,7 +110,6 @@ import javax.ws.rs.core.Response; import java.io.File; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -212,10 +210,7 @@ public enum Status private final SeekableStreamIndexTaskIOConfig ioConfig; private final SeekableStreamIndexTaskTuningConfig tuningConfig; private final InputRowSchema inputRowSchema; - @Nullable private final InputFormat inputFormat; - @Nullable - private final InputRowParser parser; private final String stream; private final Set publishingSequences = Sets.newConcurrentHashSet(); @@ -258,7 +253,6 @@ public enum Status public SeekableStreamIndexTaskRunner( final SeekableStreamIndexTask task, - @Nullable final InputRowParser parser, final LockGranularity lockGranularityToUse ) { @@ -268,7 +262,6 @@ public SeekableStreamIndexTaskRunner( this.tuningConfig = task.getTuningConfig(); this.inputRowSchema = InputRowSchemas.fromDataSchema(task.getDataSchema()); this.inputFormat = ioConfig.getInputFormat(); - this.parser = parser; this.stream = ioConfig.getStartSequenceNumbers().getStream(); this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap()); this.sequences = new CopyOnWriteArrayList<>(); @@ -414,8 +407,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception ); // Now we can initialize StreamChunkReader with the given toolbox. - final StreamChunkParser parser = new StreamChunkParser( - this.parser, + final StreamChunkReader reader = new StreamChunkReader( inputFormat, inputRowSchema, task.getDataSchema().getTransformSpec(), @@ -673,7 +665,7 @@ public void run() ); if (shouldProcess) { - final List rows = parser.parse(record.getData(), isEndOfShard(record.getSequenceNumber())); + final List rows = reader.parse(record.getData(), isEndOfShard(record.getSequenceNumber())); boolean isPersistRequired = false; final SequenceMetadata sequenceToUse = sequences @@ -714,7 +706,7 @@ public void run() pushTriggeringAddResult = addResult; sequenceToCheckpoint = sequenceToUse; } - isPersistRequired |= addResult.isPersistRequired(); + isPersistRequired = isPersistRequired || addResult.isPersistRequired(); partitionsThroughput.merge(record.getPartitionId(), 1L, Long::sum); } else { // Failure to allocate segment puts determinism at risk, bail out to be safe. 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 dddb46012891..177be6ade8c6 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 @@ -23,21 +23,9 @@ import com.google.common.base.Throwables; import org.apache.druid.client.indexing.SamplerResponse; import org.apache.druid.client.indexing.SamplerSpec; -import org.apache.druid.data.input.AbstractInputSource; -import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputFormat; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusRawValues; -import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; -import org.apache.druid.data.input.InputSourceReader; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.InputStats; -import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.ByteEntity; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.SplittableInputSource; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.indexing.overlord.sampler.SamplerConfig; import org.apache.druid.indexing.overlord.sampler.SamplerException; @@ -45,17 +33,9 @@ import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; -import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.indexing.DataSchema; import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Map; -import java.util.stream.Stream; public abstract class SeekableStreamSamplerSpec implements SamplerSpec { @@ -88,185 +68,28 @@ public SamplerResponse sample() { final InputSource inputSource; final InputFormat inputFormat; - if (dataSchema.getParser() != null) { - inputSource = new SeekableStreamSamplerInputSource(dataSchema.getParser()); - inputFormat = null; - } else { - RecordSupplier recordSupplier; + RecordSupplier recordSupplier; - try { - recordSupplier = createRecordSupplier(); - } - catch (Exception e) { - throw new SamplerException(e, "Unable to create RecordSupplier: %s", Throwables.getRootCause(e).getMessage()); - } - - inputSource = new RecordSupplierInputSource<>( - ioConfig.getStream(), - recordSupplier, - ioConfig.isUseEarliestSequenceNumber(), - samplerConfig.getTimeoutMs() <= 0 ? null : samplerConfig.getTimeoutMs() - ); - inputFormat = Preconditions.checkNotNull( - ioConfig.getInputFormat(), - "[spec.ioConfig.inputFormat] is required" - ); + try { + recordSupplier = createRecordSupplier(); } - - return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig); - } - - protected abstract RecordSupplier createRecordSupplier(); - - private class SeekableStreamSamplerInputSource extends AbstractInputSource implements SplittableInputSource - { - private final InputRowParser parser; - - public SeekableStreamSamplerInputSource(InputRowParser parser) - { - this.parser = parser; + catch (Exception e) { + throw new SamplerException(e, "Unable to create RecordSupplier: %s", Throwables.getRootCause(e).getMessage()); } - public InputRowParser getParser() - { - return parser; - } + inputSource = new RecordSupplierInputSource<>( + ioConfig.getStream(), + recordSupplier, + ioConfig.isUseEarliestSequenceNumber(), + samplerConfig.getTimeoutMs() <= 0 ? null : samplerConfig.getTimeoutMs() + ); + inputFormat = Preconditions.checkNotNull( + ioConfig.getInputFormat(), + "[spec.ioConfig.inputFormat] is required" + ); - @Override - public boolean isSplittable() - { - return false; - } - - @Override - public Stream createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) - { - throw new UnsupportedOperationException(); - } - - @Override - public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) - { - throw new UnsupportedOperationException(); - } - - @Override - public SplittableInputSource withSplit(InputSplit split) - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean needsFormat() - { - return false; - } - - @Override - protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) - { - return new SeekableStreamSamplerInputSourceReader(parser); - } + return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig); } - private class SeekableStreamSamplerInputSourceReader implements InputSourceReader - { - private final InputRowParser parser; - private final CloseableIterator entityIterator; - - public SeekableStreamSamplerInputSourceReader(InputRowParser parser) - { - this.parser = parser; - if (parser instanceof StringInputRowParser) { - ((StringInputRowParser) parser).startFileFromBeginning(); - } - - RecordSupplierInputSource inputSource = new RecordSupplierInputSource<>( - ioConfig.getStream(), - createRecordSupplier(), - ioConfig.isUseEarliestSequenceNumber(), - samplerConfig.getTimeoutMs() <= 0 ? null : samplerConfig.getTimeoutMs() - ); - this.entityIterator = inputSource.createEntityIterator(); - } - - @Override - public CloseableIterator read() - { - return new CloseableIterator<>() - { - - @Override - public boolean hasNext() - { - return entityIterator.hasNext(); - } - - @Override - public InputRow next() - { - throw new UnsupportedOperationException(); - } - - @Override - public void close() throws IOException - { - entityIterator.close(); - } - }; - } - - @Override - public CloseableIterator read(InputStats inputStats) - { - return null; - } - - @Override - public CloseableIterator sample() - { - return new CloseableIterator<>() - { - @Override - public boolean hasNext() - { - return entityIterator.hasNext(); - } - - @Override - public InputRowListPlusRawValues next() - { - // We need to modify the position of the buffer, so duplicate it. - final ByteBuffer bb = ((ByteEntity) entityIterator.next()).getBuffer().duplicate(); - - final Map rawColumns; - try { - if (parser instanceof StringInputRowParser) { - rawColumns = ((StringInputRowParser) parser).buildStringKeyMap(bb); - } else { - rawColumns = null; - } - } - catch (ParseException e) { - return InputRowListPlusRawValues.of(null, e); - } - - try { - bb.position(0); - final List rows = parser.parseBatch(bb); - return InputRowListPlusRawValues.of(rows.isEmpty() ? null : rows, rawColumns); - } - catch (ParseException e) { - return InputRowListPlusRawValues.of(rawColumns, e); - } - } - - @Override - public void close() throws IOException - { - entityIterator.close(); - } - }; - } - } + protected abstract RecordSupplier createRecordSupplier(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkReader.java similarity index 52% rename from indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkReader.java index ad9886e424af..fb24323baad8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkReader.java @@ -20,17 +20,14 @@ package org.apache.druid.indexing.seekablestream; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.Lists; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.error.InvalidInput; import org.apache.druid.indexing.common.task.FilteringCloseableInputRowIterator; import org.apache.druid.indexing.common.task.InputRowFilter; -import org.apache.druid.java.util.common.CloseableIterators; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.ParseExceptionHandler; @@ -40,7 +37,6 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -49,11 +45,8 @@ * Abstraction for parsing stream data which internally uses {@link org.apache.druid.data.input.InputEntityReader} * or {@link InputRowParser}. This class will be useful until we remove the deprecated {@link InputRowParser}. */ -class StreamChunkParser +class StreamChunkReader { - @Nullable - private final InputRowParser parser; - @Nullable private final SettableByteEntityReader byteEntityReader; private final InputRowFilter rowFilter; private final RowIngestionMeters rowIngestionMeters; @@ -62,9 +55,8 @@ class StreamChunkParser /** * Either parser or inputFormat shouldn't be null. */ - StreamChunkParser( - @Nullable InputRowParser parser, - @Nullable InputFormat inputFormat, + StreamChunkReader( + InputFormat inputFormat, InputRowSchema inputRowSchema, TransformSpec transformSpec, File indexingTmpDir, @@ -73,39 +65,27 @@ class StreamChunkParser ParseExceptionHandler parseExceptionHandler ) { - if (parser == null && inputFormat == null) { - throw new IAE("Either parser or inputFormat should be set"); - } - // parser is already decorated with transformSpec in DataSchema - this.parser = parser; - if (inputFormat != null) { - this.byteEntityReader = new SettableByteEntityReader<>( - inputFormat, - inputRowSchema, - transformSpec, - indexingTmpDir - ); - } else { - this.byteEntityReader = null; - } + InvalidInput.conditionalException(inputFormat != null, "inputFormat must not be null"); + this.byteEntityReader = new SettableByteEntityReader<>( + inputFormat, + inputRowSchema, + transformSpec, + indexingTmpDir + ); this.rowFilter = rowFilter; this.rowIngestionMeters = rowIngestionMeters; this.parseExceptionHandler = parseExceptionHandler; } @VisibleForTesting - StreamChunkParser( - @Nullable InputRowParser parser, - @Nullable SettableByteEntityReader byteEntityReader, + StreamChunkReader( + SettableByteEntityReader byteEntityReader, InputRowFilter rowFilter, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler ) { - if (parser == null && byteEntityReader == null) { - throw new IAE("Either parser or byteEntityReader should be set"); - } - this.parser = parser; + InvalidInput.conditionalException(byteEntityReader != null, "byteEntityReader must not be null"); this.byteEntityReader = byteEntityReader; this.rowFilter = rowFilter; this.rowIngestionMeters = rowIngestionMeters; @@ -122,62 +102,23 @@ List parse(@Nullable List streamChunk, boolean isEndOfShar } return Collections.emptyList(); } else { - if (byteEntityReader != null) { - return parseWithInputFormat(byteEntityReader, streamChunk); - } else { - return parseWithParser(parser, streamChunk); - } - } - } - - private List parseWithParser(InputRowParser parser, List valueBytes) - { - final FluentIterable iterable = FluentIterable - .from(valueBytes) - .transform(entity -> entity.getBuffer().duplicate() /* Parsing may need to modify buffer position */) - .transform(this::incrementProcessedBytes) - .transformAndConcat(parser::parseBatch); - - final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( - CloseableIterators.withEmptyBaggage(iterable.iterator()), - rowFilter, - rowIngestionMeters, - parseExceptionHandler - ); - return Lists.newArrayList(rowIterator); - } - - /** - * Increments the processed bytes with the number of bytes remaining in the - * given buffer. This method must be called before reading the buffer. - */ - private ByteBuffer incrementProcessedBytes(final ByteBuffer recordByteBuffer) - { - rowIngestionMeters.incrementProcessedBytes(recordByteBuffer.remaining()); - return recordByteBuffer; - } - - private List parseWithInputFormat( - SettableByteEntityReader byteEntityReader, - List valueBytess - ) throws IOException - { - final List rows = new ArrayList<>(); - for (ByteEntity valueBytes : valueBytess) { - rowIngestionMeters.incrementProcessedBytes(valueBytes.getBuffer().remaining()); - byteEntityReader.setEntity(valueBytes); - try (FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( - byteEntityReader.read(), - rowFilter, - rowIngestionMeters, - parseExceptionHandler - )) { - rowIterator.forEachRemaining(rows::add); - } - catch (ParseException pe) { - parseExceptionHandler.handle(pe); + final List rows = new ArrayList<>(); + for (RecordType valueBytes : streamChunk) { + rowIngestionMeters.incrementProcessedBytes(valueBytes.getBuffer().remaining()); + byteEntityReader.setEntity(valueBytes); + try (FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( + byteEntityReader.read(), + rowFilter, + rowIngestionMeters, + parseExceptionHandler + )) { + rowIterator.forEachRemaining(rows::add); + } + catch (ParseException pe) { + parseExceptionHandler.handle(pe); + } } + return rows; } - return rows; } } 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 83339f1a3572..77aab1da8485 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 @@ -32,14 +32,10 @@ import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.CsvInputFormat; -import org.apache.druid.data.input.impl.DelimitedParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InlineInputSource; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.granularity.UniformGranularitySpec; @@ -117,27 +113,23 @@ private enum ParserType private List> mapOfRows; private InputSourceSampler inputSourceSampler; private ParserType parserType; - private boolean useInputFormatApi; @Rule public ExpectedException expectedException = ExpectedException.none(); - @Parameterized.Parameters(name = "parserType = {0}, useInputFormatApi={1}") + @Parameterized.Parameters(name = "parserType = {0}") public static Iterable constructorFeeder() { OBJECT_MAPPER.registerModules(new SamplerModule().getJacksonModules()); return ImmutableList.of( - new Object[]{ParserType.STR_JSON, false}, - new Object[]{ParserType.STR_JSON, true}, - new Object[]{ParserType.STR_CSV, false}, - new Object[]{ParserType.STR_CSV, true} + new Object[]{ParserType.STR_JSON}, + new Object[]{ParserType.STR_CSV} ); } - public InputSourceSamplerTest(ParserType parserType, boolean useInputFormatApi) + public InputSourceSamplerTest(ParserType parserType) { this.parserType = parserType; - this.useInputFormatApi = useInputFormatApi; } @Before @@ -285,7 +277,7 @@ public void testNoDataSchemaNumRows() } @Test - public void testMissingValueTimestampSpec() throws IOException + public void testMissingValueTimestampSpec() { final TimestampSpec timestampSpec = new TimestampSpec(null, null, DateTimes.of("1970")); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); @@ -394,7 +386,7 @@ public void testMissingValueTimestampSpec() throws IOException } @Test - public void testWithTimestampSpec() throws IOException + public void testWithTimestampSpec() { final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); @@ -492,7 +484,7 @@ public void testWithTimestampSpec() throws IOException } @Test - public void testWithDimensionSpec() throws IOException + public void testWithDimensionSpec() { final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec( @@ -587,7 +579,7 @@ public void testWithDimensionSpec() throws IOException } @Test - public void testWithNoRollup() throws IOException + public void testWithNoRollup() { final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); @@ -698,7 +690,7 @@ public void testWithNoRollup() throws IOException } @Test - public void testWithRollup() throws IOException + public void testWithRollup() { final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); @@ -781,7 +773,7 @@ public void testWithRollup() throws IOException } @Test - public void testWithMoreRollup() throws IOException + public void testWithMoreRollup() { final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(ImmutableList.of(StringDimensionSchema.create("dim1"))); @@ -848,7 +840,7 @@ public void testWithMoreRollup() throws IOException } @Test - public void testWithTransformsAutoDimensions() throws IOException + public void testWithTransformsAutoDimensions() { final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); @@ -935,7 +927,7 @@ public void testWithTransformsAutoDimensions() throws IOException } @Test - public void testWithTransformsDimensionsSpec() throws IOException + public void testWithTransformsDimensionsSpec() { final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec( @@ -1008,7 +1000,7 @@ public void testWithTransformsDimensionsSpec() throws IOException } @Test - public void testWithFilter() throws IOException + public void testWithFilter() { final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); @@ -1190,9 +1182,9 @@ public void testIndexParseException() throws IOException * */ @Test - public void testMultipleJsonStringInOneBlock() throws IOException + public void testMultipleJsonStringInOneBlock() { - if (!ParserType.STR_JSON.equals(parserType) || !useInputFormatApi) { + if (!ParserType.STR_JSON.equals(parserType)) { return; } @@ -1328,7 +1320,7 @@ public InputSourceReader reader( } @Test - public void testRowLimiting() throws IOException + public void testRowLimiting() { final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); @@ -1363,7 +1355,7 @@ public void testRowLimiting() throws IOException } @Test - public void testMaxBytesInMemoryLimiting() throws IOException + public void testMaxBytesInMemoryLimiting() { final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); @@ -1397,7 +1389,7 @@ public void testMaxBytesInMemoryLimiting() throws IOException } @Test - public void testMaxClientResponseBytesLimiting() throws IOException + public void testMaxClientResponseBytesLimiting() { final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); @@ -1466,64 +1458,22 @@ private InputFormat createInputFormat() } } - private InputRowParser createInputRowParser(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) - { - switch (parserType) { - case STR_JSON: - return new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, null, null, null)); - case STR_CSV: - return new StringInputRowParser( - new DelimitedParseSpec( - timestampSpec, - dimensionsSpec, - ",", - null, - ImmutableList.of("t", "dim1", "dim2", "met1"), - false, - 0 - ) - ); - default: - throw new IAE("Unknown parser type: %s", parserType); - } - } - private DataSchema createDataSchema( @Nullable TimestampSpec timestampSpec, @Nullable DimensionsSpec dimensionsSpec, @Nullable AggregatorFactory[] aggregators, @Nullable GranularitySpec granularitySpec, @Nullable TransformSpec transformSpec - ) throws IOException + ) { - if (useInputFormatApi) { - return DataSchema.builder() - .withDataSource("sampler") - .withTimestamp(timestampSpec) - .withDimensions(dimensionsSpec) - .withAggregators(aggregators) - .withGranularity(granularitySpec) - .withTransform(transformSpec) - .build(); - } else { - final Map parserMap = getParserMap(createInputRowParser(timestampSpec, dimensionsSpec)); - return DataSchema.builder() - .withDataSource("sampler") - .withParserMap(parserMap) - .withAggregators(aggregators) - .withGranularity(granularitySpec) - .withTransform(transformSpec) - .withObjectMapper(OBJECT_MAPPER) - .build(); - } - } - - private Map getParserMap(InputRowParser parser) throws IOException - { - if (useInputFormatApi) { - throw new RuntimeException("Don't call this if useInputFormatApi = true"); - } - return OBJECT_MAPPER.readValue(OBJECT_MAPPER.writeValueAsBytes(parser), Map.class); + return DataSchema.builder() + .withDataSource("sampler") + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .withAggregators(aggregators) + .withGranularity(granularitySpec) + .withTransform(transformSpec) + .build(); } private InputSource createInputSource(List rows) 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 63387ccffbdf..e63d3dbd6e5c 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 @@ -281,7 +281,7 @@ private TestSeekableStreamIndexTaskRunner( SeekableStreamIndexTask task ) { - super(task, null, LockGranularity.SEGMENT); + super(task, LockGranularity.SEGMENT); } @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java index eb08944d46bd..7ceae8451703 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap; 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.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -102,8 +101,10 @@ public void testWithinMinMaxTime() Mockito.when(task.getIOConfig()).thenReturn(ioConfig); Mockito.when(task.getTuningConfig()).thenReturn(tuningConfig); - TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner(task, null, - LockGranularity.TIME_CHUNK); + TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner( + task, + LockGranularity.TIME_CHUNK + ); Mockito.when(row.getTimestamp()).thenReturn(now); Assert.assertEquals(InputRowFilterResult.ACCEPTED, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); @@ -155,8 +156,10 @@ public void testWithinMinMaxTimeNotPopulated() Mockito.when(task.getDataSchema()).thenReturn(schema); Mockito.when(task.getIOConfig()).thenReturn(ioConfig); Mockito.when(task.getTuningConfig()).thenReturn(tuningConfig); - TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner(task, null, - LockGranularity.TIME_CHUNK); + TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner( + task, + LockGranularity.TIME_CHUNK + ); Mockito.when(row.getTimestamp()).thenReturn(now); Assert.assertEquals(InputRowFilterResult.ACCEPTED, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); @@ -206,8 +209,10 @@ public void testEnsureRowRejectionReasonForNullRow() Mockito.when(task.getIOConfig()).thenReturn(ioConfig); Mockito.when(task.getTuningConfig()).thenReturn(tuningConfig); - TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner(task, null, - LockGranularity.TIME_CHUNK); + TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner( + task, + LockGranularity.TIME_CHUNK + ); Assert.assertEquals(InputRowFilterResult.NULL_OR_EMPTY_RECORD, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(null)); } @@ -249,8 +254,10 @@ public void testGetSupervisorId() Mockito.when(task.getTuningConfig()).thenReturn(tuningConfig); Mockito.when(task.getSupervisorId()).thenReturn("supervisorId"); - TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner(task, null, - LockGranularity.TIME_CHUNK); + TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner( + task, + LockGranularity.TIME_CHUNK + ); Assert.assertEquals("supervisorId", runner.getSupervisorId()); } @@ -258,11 +265,10 @@ static class TestasbleSeekableStreamIndexTaskRunner extends SeekableStreamIndexT { public TestasbleSeekableStreamIndexTaskRunner( SeekableStreamIndexTask task, - @Nullable InputRowParser parser, LockGranularity lockGranularityToUse ) { - super(task, parser, lockGranularityToUse); + super(task, lockGranularityToUse); } @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkReaderTest.java similarity index 71% rename from indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkReaderTest.java index 92ed5785abb3..f3bc46a75247 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkReaderTest.java @@ -28,14 +28,11 @@ import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JsonInputFormat; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.indexing.common.task.InputRowFilter; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.JSONPathSpec; @@ -57,14 +54,13 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @RunWith(MockitoJUnitRunner.class) -public class StreamChunkParserTest +public class StreamChunkReaderTest { private static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec(null, null, null); @@ -86,38 +82,10 @@ public class StreamChunkParserTest private SettableByteEntityReader mockedByteEntityReader; @Test - public void testWithParserAndNullInputformatParseProperly() throws IOException - { - final InputRowParser parser = new StringInputRowParser( - new JSONParseSpec( - TIMESTAMP_SPEC, - DimensionsSpec.EMPTY, - JSONPathSpec.DEFAULT, - Collections.emptyMap(), - false - ), - StringUtils.UTF8_STRING - ); - final StreamChunkParser chunkParser = new StreamChunkParser<>( - parser, - // Set nulls for all parameters below since inputFormat will never be used. - null, - null, - null, - null, - InputRowFilter.allowAll(), - rowIngestionMeters, - parseExceptionHandler - ); - parseAndAssertResult(chunkParser); - } - - @Test - public void testWithNullParserAndInputformatParseProperly() throws IOException + public void testInputformatParseProperly() throws IOException { final JsonInputFormat inputFormat = new JsonInputFormat(JSONPathSpec.DEFAULT, Collections.emptyMap(), null, null, null); - final StreamChunkParser chunkParser = new StreamChunkParser<>( - null, + final StreamChunkReader chunkParser = new StreamChunkReader<>( inputFormat, new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), TransformSpec.NONE, @@ -132,51 +100,21 @@ public void testWithNullParserAndInputformatParseProperly() throws IOException @Test public void testWithNullParserAndNullInputformatFailToCreateParser() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Either parser or inputFormat should be set"); - final StreamChunkParser chunkParser = new StreamChunkParser<>( - null, - null, - null, - null, - null, - InputRowFilter.allowAll(), - rowIngestionMeters, - parseExceptionHandler + Throwable t = Assert.assertThrows( + DruidException.class, + () -> new StreamChunkReader<>( + null, + null, + null, + null, + InputRowFilter.allowAll(), + rowIngestionMeters, + parseExceptionHandler + ) ); + Assert.assertEquals("inputFormat must not be null", t.getMessage()); } - @Test - public void testBothParserAndInputFormatParseProperlyUsingInputFormat() throws IOException - { - final InputRowParser parser = new StringInputRowParser( - new JSONParseSpec( - TIMESTAMP_SPEC, - DimensionsSpec.EMPTY, - JSONPathSpec.DEFAULT, - Collections.emptyMap(), - false - ), - StringUtils.UTF8_STRING - ); - - final TrackingJsonInputFormat inputFormat = new TrackingJsonInputFormat( - JSONPathSpec.DEFAULT, - Collections.emptyMap() - ); - final StreamChunkParser chunkParser = new StreamChunkParser<>( - parser, - inputFormat, - new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), - TransformSpec.NONE, - temporaryFolder.newFolder(), - InputRowFilter.allowAll(), - rowIngestionMeters, - parseExceptionHandler - ); - parseAndAssertResult(chunkParser); - Assert.assertTrue(inputFormat.props.used); - } @Test public void parseEmptyNotEndOfShard() throws IOException @@ -185,8 +123,7 @@ public void parseEmptyNotEndOfShard() throws IOException JSONPathSpec.DEFAULT, Collections.emptyMap() ); - final StreamChunkParser chunkParser = new StreamChunkParser<>( - null, + final StreamChunkReader chunkParser = new StreamChunkReader<>( inputFormat, new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), TransformSpec.NONE, @@ -208,8 +145,7 @@ public void parseEmptyEndOfShard() throws IOException JSONPathSpec.DEFAULT, Collections.emptyMap() ); - final StreamChunkParser chunkParser = new StreamChunkParser<>( - null, + final StreamChunkReader chunkParser = new StreamChunkReader<>( inputFormat, new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), TransformSpec.NONE, @@ -227,20 +163,8 @@ public void parseEmptyEndOfShard() throws IOException @Test public void testParseMalformedDataWithAllowedParseExceptions_thenNoException() throws IOException { - final InputRowParser parser = new StringInputRowParser( - new JSONParseSpec( - TIMESTAMP_SPEC, - DimensionsSpec.EMPTY, - JSONPathSpec.DEFAULT, - Collections.emptyMap(), - false - ), - StringUtils.UTF8_STRING - ); - final int maxAllowedParseExceptions = 1; - final StreamChunkParser chunkParser = new StreamChunkParser<>( - parser, + final StreamChunkReader chunkParser = new StreamChunkReader<>( mockedByteEntityReader, InputRowFilter.allowAll(), rowIngestionMeters, @@ -266,19 +190,7 @@ public void testParseMalformedDataWithAllowedParseExceptions_thenNoException() t @Test public void testParseMalformedDataException() throws IOException { - final InputRowParser parser = new StringInputRowParser( - new JSONParseSpec( - TIMESTAMP_SPEC, - DimensionsSpec.EMPTY, - JSONPathSpec.DEFAULT, - Collections.emptyMap(), - false - ), - StringUtils.UTF8_STRING - ); - - final StreamChunkParser chunkParser = new StreamChunkParser<>( - parser, + final StreamChunkReader chunkParser = new StreamChunkReader<>( mockedByteEntityReader, InputRowFilter.allowAll(), rowIngestionMeters, @@ -305,19 +217,7 @@ public void testParseMalformedDataException() throws IOException @Test public void testParseMalformedDataWithUnlimitedAllowedParseExceptions_thenNoException() throws IOException { - final InputRowParser parser = new StringInputRowParser( - new JSONParseSpec( - TIMESTAMP_SPEC, - DimensionsSpec.EMPTY, - JSONPathSpec.DEFAULT, - Collections.emptyMap(), - false - ), - StringUtils.UTF8_STRING - ); - - final StreamChunkParser chunkParser = new StreamChunkParser<>( - parser, + final StreamChunkReader chunkParser = new StreamChunkReader<>( mockedByteEntityReader, InputRowFilter.allowAll(), rowIngestionMeters, @@ -349,20 +249,19 @@ public void testParseMalformedDataWithUnlimitedAllowedParseExceptions_thenNoExce @Test public void testWithNullParserAndNullByteEntityReaderFailToInstantiate() { - Assert.assertThrows( - "Either parser or byteEntityReader should be set", - IAE.class, - () -> new StreamChunkParser<>( - null, + Throwable t = Assert.assertThrows( + DruidException.class, + () -> new StreamChunkReader<>( null, InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ) ); + Assert.assertEquals("byteEntityReader must not be null", t.getMessage()); } - private void parseAndAssertResult(StreamChunkParser chunkParser) throws IOException + private void parseAndAssertResult(StreamChunkReader chunkParser) throws IOException { final String json = "{\"timestamp\": \"2020-01-01\", \"dim\": \"val\", \"met\": \"val2\"}"; List parsedRows = chunkParser.parse(Collections.singletonList(new ByteEntity(json.getBytes(StringUtils.UTF8_STRING))), false); From 40ed493328eff64f8098a62b8bcd411d7a496f66 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 16 Mar 2026 12:03:48 -0700 Subject: [PATCH 2/3] oops, forgot to add replacement benchmark --- .../ProtobufInputFormatBenchmark.java | 120 ++++++++++++++++++ 1 file changed, 120 insertions(+) create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/ProtobufInputFormatBenchmark.java diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ProtobufInputFormatBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ProtobufInputFormatBenchmark.java new file mode 100644 index 000000000000..5f21db36584d --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ProtobufInputFormatBenchmark.java @@ -0,0 +1,120 @@ +/* + * 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.benchmark; + +import com.google.common.collect.Lists; +import com.google.common.io.Files; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; +import org.apache.druid.java.util.common.parsers.JSONPathFieldType; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 10) +@Measurement(iterations = 25) +public class ProtobufInputFormatBenchmark +{ + @Param({"75000"}) + private int rowsPerSegment; + + private static final Logger log = new Logger(ProtobufInputFormatBenchmark.class); + + private ProtobufInputFormat nestedInputFormat; + private ProtobufInputFormat flatInputFormat; + private InputRowSchema inputRowSchema; + private byte[] protoInputs; + + @Setup + public void setup() throws IOException + { + final TimestampSpec timestampSpec = new TimestampSpec("timestamp", "iso", null); + final DimensionsSpec dimensionsSpec = new DimensionsSpec(Lists.newArrayList( + new StringDimensionSchema("event"), + new StringDimensionSchema("id"), + new StringDimensionSchema("someOtherId"), + new StringDimensionSchema("isValid") + )); + final FileBasedProtobufBytesDecoder decoder = new FileBasedProtobufBytesDecoder("prototest.desc", "ProtoTestEvent"); + + inputRowSchema = new InputRowSchema(timestampSpec, dimensionsSpec, null); + + final JSONPathSpec flattenSpec = new JSONPathSpec( + true, + Lists.newArrayList( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "eventType", "eventType"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "foobar", "$.foo.bar"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "bar0", "$.bar[0].bar") + ) + ); + + nestedInputFormat = new ProtobufInputFormat(flattenSpec, decoder); + flatInputFormat = new ProtobufInputFormat(null, decoder); + + final String filePath = this.getClass().getClassLoader().getResource("ProtoFile").getPath(); + protoInputs = Files.toByteArray(new File(filePath)); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void consumeFlatData(Blackhole blackhole) throws IOException + { + for (int i = 0; i < rowsPerSegment; i++) { + InputRow row = flatInputFormat.createReader(inputRowSchema, new ByteEntity(protoInputs), null).read().next(); + blackhole.consume(row); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void consumeNestedData(Blackhole blackhole) throws IOException + { + for (int i = 0; i < rowsPerSegment; i++) { + InputRow row = nestedInputFormat.createReader(inputRowSchema, new ByteEntity(protoInputs), null).read().next(); + blackhole.consume(row); + } + } +} From 4fc2f3d62828211149ae1714121545be086deddb Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 16 Mar 2026 14:21:17 -0700 Subject: [PATCH 3/3] fixes --- docs/development/extensions-core/protobuf.md | 195 ++++++++---------- .../indexing/kafka/KafkaIndexTaskTest.java | 46 ----- .../SeekableStreamSamplerSpecTest.java | 49 ++--- 3 files changed, 107 insertions(+), 183 deletions(-) diff --git a/docs/development/extensions-core/protobuf.md b/docs/development/extensions-core/protobuf.md index 5e6fefbdd6ce..ff976224dd47 100644 --- a/docs/development/extensions-core/protobuf.md +++ b/docs/development/extensions-core/protobuf.md @@ -25,7 +25,7 @@ title: "Protobuf" This Apache Druid extension enables Druid to ingest and understand the Protobuf data format. Make sure to [include](../../configuration/extensions.md#loading-extensions) `druid-protobuf-extensions` in the extensions load list. -The `druid-protobuf-extensions` provides the [Protobuf Parser](../../ingestion/data-formats.md#protobuf-parser) +The `druid-protobuf-extensions` provides the [Protobuf InputFormat](../../ingestion/data-formats.md#protobuf) for [stream ingestion](../../ingestion/index.md#streaming). See corresponding docs for details. ## Example: Load Protobuf messages from Kafka @@ -116,95 +116,83 @@ Important supervisor properties ```json { -"type": "kafka", -"spec": { - "dataSchema": { - "dataSource": "metrics-protobuf", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [ - "unit", - "http_method", - "http_code", - "page", - "metricType", - "server" - ], - "dimensionExclusions": [ - "timestamp", - "value" - ] - }, - "metricsSpec": [ - { - "name": "count", - "type": "count" + "type": "kafka", + "spec": { + "dataSchema": { + "dataSource": "metrics-protobuf", + "timestampSpec": { + "column": "timestamp", + "format": "auto" }, - { - "name": "value_sum", - "fieldName": "value", - "type": "doubleSum" + "dimensionsSpec": { + "dimensions": [ + "unit", + "http_method", + "http_code", + "page", + "metricType", + "server" + ], + "dimensionExclusions": [ + "timestamp", + "value" + ] }, - { - "name": "value_min", - "fieldName": "value", - "type": "doubleMin" - }, - { - "name": "value_max", - "fieldName": "value", - "type": "doubleMax" + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "value_sum", + "fieldName": "value", + "type": "doubleSum" + }, + { + "name": "value_min", + "fieldName": "value", + "type": "doubleMin" + }, + { + "name": "value_max", + "fieldName": "value", + "type": "doubleMax" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "HOUR", + "queryGranularity": "NONE" } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "HOUR", - "queryGranularity": "NONE" - } - }, - "tuningConfig": { - "type": "kafka", - "maxRowsPerSegment": 5000000 - }, - "ioConfig": { - "topic": "metrics_pb", - "consumerProperties": { - "bootstrap.servers": "localhost:9092" }, - "inputFormat": { - "type": "protobuf", - "protoBytesDecoder": { - "type": "file", - "descriptor": "file:///tmp/metrics.desc", - "protoMessageType": "Metrics" + "tuningConfig": { + "type": "kafka", + "maxRowsPerSegment": 5000000 + }, + "ioConfig": { + "topic": "metrics_pb", + "consumerProperties": { + "bootstrap.servers": "localhost:9092" }, - "flattenSpec": { - "useFieldDiscovery": true + "inputFormat": { + "type": "protobuf", + "protoBytesDecoder": { + "type": "file", + "descriptor": "file:///tmp/metrics.desc", + "protoMessageType": "Metrics" + }, + "flattenSpec": { + "useFieldDiscovery": true + }, + "binaryAsString": false }, - "binaryAsString": false - }, - "taskCount": 1, - "replicas": 1, - "taskDuration": "PT1H", - "type": "kafka" + "taskCount": 1, + "replicas": 1, + "taskDuration": "PT1H", + "type": "kafka" + } } } -} -``` - -To adopt to old version. You can use old parser style, which also works. - -```json -{ - "parser": { - "type": "protobuf", - "descriptor": "file:///tmp/metrics.desc", - "protoMessageType": "Metrics" - } -} ``` ### When using Schema Registry @@ -215,33 +203,28 @@ Important supervisor properties - `protoBytesDecoder.capacity` capacity for schema registry cached schemas. - `protoBytesDecoder.config` to send additional configurations, configured for Schema Registry. - `protoBytesDecoder.headers` to send headers to the Schema Registry. -- `protoBytesDecoder.type` set to `schema_registry`, indicate use schema registry to decode Protobuf file. -- `parser` should have `type` set to `protobuf`, but note that the `format` of the `parseSpec` must be `json`. +- `protoBytesDecoder.type` set to `schema_registry`, indicate use schema registry to decode Protobuf file.. ```json + { - "parser": { - "type": "protobuf", - "protoBytesDecoder": { - "urls": ["http://schemaregistry.example1.com:8081","http://schemaregistry.example2.com:8081"], - "type": "schema_registry", - "capacity": 100, - "config" : { - "basic.auth.credentials.source": "USER_INFO", - "basic.auth.user.info": "fred:letmein", - "schema.registry.ssl.truststore.location": "/some/secrets/kafka.client.truststore.jks", - "schema.registry.ssl.truststore.password": "", - "schema.registry.ssl.keystore.location": "/some/secrets/kafka.client.keystore.jks", - "schema.registry.ssl.keystore.password": "", - "schema.registry.ssl.key.password": "", - ... - }, - "headers": { - "traceID" : "b29c5de2-0db4-490b-b421", - "timeStamp" : "1577191871865", - ... - } - } + "urls": ["http://schemaregistry.example1.com:8081","http://schemaregistry.example2.com:8081"], + "type": "schema_registry", + "capacity": 100, + "config" : { + "basic.auth.credentials.source": "USER_INFO", + "basic.auth.user.info": "fred:letmein", + "schema.registry.ssl.truststore.location": "/some/secrets/kafka.client.truststore.jks", + "schema.registry.ssl.truststore.password": "", + "schema.registry.ssl.keystore.location": "/some/secrets/kafka.client.keystore.jks", + "schema.registry.ssl.keystore.password": "", + "schema.registry.ssl.key.password": "", + ... + }, + "headers": { + "traceID" : "b29c5de2-0db4-490b-b421", + "timeStamp" : "1577191871865", + ... } } ``` diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index a5a3243b5b88..1edeca1bf215 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -502,52 +502,6 @@ public void testIngestNullColumnAfterDataInserted_storeEmptyColumnsOff_shouldNot } } - @Test(timeout = 60_000L) - public void testRunAfterDataInsertedWithLegacyParser() throws Exception - { - // Insert data - insertData(); - - final KafkaIndexTask task = createTask( - null, - OLD_DATA_SCHEMA, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null, - null, - null, - Duration.standardHours(2).getStandardMinutes() - ) - ); - - final ListenableFuture future = runTask(task); - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); - Assert.assertTrue(task.getRunner().getSegmentGenerationMetrics().isProcessingDone()); - - // Check published metadata and segments in deep storage - assertEqualsExceptVersion( - ImmutableList.of( - sdd("2010/P1D", 0, ImmutableList.of("c")), - sdd("2011/P1D", 0, ImmutableList.of("d", "e")) - ), - publishedDescriptors() - ); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))), - newDataSchemaMetadata() - ); - } - @Test(timeout = 60_000L) public void testRunBeforeDataInserted() throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java index 97456165f361..5af21ca7b6aa 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java @@ -26,12 +26,10 @@ import org.apache.druid.client.indexing.SamplerResponse; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.ByteEntity; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; @@ -48,7 +46,6 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; @@ -60,15 +57,12 @@ import org.junit.Test; import javax.annotation.Nullable; -import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Map; public class SeekableStreamSamplerSpecTest extends EasyMockSupport { - private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); private static final String STREAM = "sampling"; private static final String SHARD_ID = "1"; @@ -103,39 +97,32 @@ public void testSampleWithInputRowParser() throws Exception { DataSchema dataSchema = DataSchema.builder() .withDataSource("test_ds") - .withParserMap( - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of(), - false - ) - ), - Map.class + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions( + List.of( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") ) ) .withAggregators( new DoubleSumAggregatorFactory("met1sum", "met1"), new CountAggregatorFactory("rows") ) - .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) - .withObjectMapper(OBJECT_MAPPER) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + null + ) + ) .build(); final SeekableStreamSupervisorIOConfig supervisorIOConfig = new TestableSeekableStreamSupervisorIOConfig( STREAM, - null, + new JsonInputFormat(null, null, null, null, null), null, null, null, @@ -267,7 +254,7 @@ public void testSampleWithInputRowParser() throws Exception null, null, true, - "Unable to parse row [unparseable]" + "Unable to parse row [unparseable] into JSON" ), it.next()); Assert.assertFalse(it.hasNext());