This is an automated email from the ASF dual-hosted git repository. xvrl pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/druid.git
The following commit(s) were added to refs/heads/master by this push: new ad6609a Kafka Input Format for headers, key and payload parsing (#11630) ad6609a is described below commit ad6609a606c0ca945c6aa57f1b7c40593ba29f80 Author: lokesh-lingarajan <llingara...@confluent.io> AuthorDate: Thu Oct 7 08:56:27 2021 -0700 Kafka Input Format for headers, key and payload parsing (#11630) ### Description Today we ingest a number of high cardinality metrics into Druid across dimensions. These metrics are rolled up on a per minute basis, and are very useful when looking at metrics on a partition or client basis. Events is another class of data that provides useful information about a particular incident/scenario inside a Kafka cluster. Events themselves are carried inside kafka payload, but nonetheless there are some very useful metadata that is carried in kafka headers that can serve a [...] PR(https://github.com/apache/druid/pull/10730) introduced support of Kafka headers in InputFormats. We still need an input format to parse out the headers and translate those into relevant columns in Druid. Until that’s implemented, none of the information available in the Kafka message headers would be exposed. So first there is a need to write an input format that can parse headers in any given format(provided we support the format) like we parse payloads today. Apart from headers there is also some useful information present in the key portion of the kafka record. We also need a [...] This PR is designed to solve the above by providing wrapper around any existing input formats and merging the data into a single unified Druid row. Lets look at a sample input format from the above discussion "inputFormat": { "type": "kafka", // New input format type "headerLabelPrefix": "kafka.header.", // Label prefix for header columns, this will avoid collusions while merging columns "recordTimestampLabelPrefix": "kafka.", // Kafka record's timestamp is made available in case payload does not carry timestamp "headerFormat": // Header parser specifying that values are of type string { "type": "string" }, "valueFormat": // Value parser from json parsing { "type": "json", "flattenSpec": { "useFieldDiscovery": true, "fields": [...] } }, "keyFormat": // Key parser also from json parsing { "type": "json" } } Since we have independent sections for header, key and payload, it will enable parsing each section with its own parser, eg., headers coming in as string and payload as json. KafkaInputFormat will be the uber class extending inputFormat interface and will be responsible for creating individual parsers for header, key and payload, blend the data resolving conflicts in columns and generating a single unified InputRow for Druid ingestion. "headerFormat" will allow users to plug parser type for the header values and will add default header prefix as "kafka.header."(can be overridden) for attributes to avoid collision while merging attributes with payload. Kafka payload parser will be responsible for parsing the Value portion of the Kafka record. This is where most of the data will come from and we should be able to plugin existing parser. One thing to note here is that if batching is performed, then the code is augmenting header and key values to every record in the batch. Kafka key parser will handle parsing Key portion of the Kafka record and will ingest the Key with dimension name as "kafka.key". ## KafkaInputFormat Class: This is the class that orchestrates sending the consumerRecord to each parser, retrieve rows, merge the columns into one final row for Druid consumption. KafkaInputformat should make sure to release the resources that gets allocated as a part of reader in CloseableIterator<InputRow> during normal and exception cases. During conflicts in dimension/metrics names, the code will prefer dimension names from payload and ignore the dimension either from headers/key. This is done so that existing input formats can be easily migrated to this new format without worrying about losing information. --- .../development/extensions-core/kafka-ingestion.md | 1 + docs/ingestion/data-formats.md | 69 ++++ .../data/input/kafkainput/KafkaHeaderFormat.java | 37 ++ .../data/input/kafkainput/KafkaHeaderReader.java | 29 ++ .../data/input/kafkainput/KafkaInputFormat.java | 175 ++++++++ .../data/input/kafkainput/KafkaInputReader.java | 184 +++++++++ .../input/kafkainput/KafkaStringHeaderFormat.java | 80 ++++ .../input/kafkainput/KafkaStringHeaderReader.java | 58 +++ .../druid/indexing/kafka/KafkaIndexTaskModule.java | 4 +- .../input/kafkainput/KafkaInputFormatTest.java | 446 +++++++++++++++++++++ .../kafkainput/KafkaStringHeaderFormatTest.java | 219 ++++++++++ .../druid/indexing/kafka/KafkaIndexTaskTest.java | 87 ++++ website/.spelling | 11 + 13 files changed, 1399 insertions(+), 1 deletion(-) diff --git a/docs/development/extensions-core/kafka-ingestion.md b/docs/development/extensions-core/kafka-ingestion.md index d30bf6a..0355389 100644 --- a/docs/development/extensions-core/kafka-ingestion.md +++ b/docs/development/extensions-core/kafka-ingestion.md @@ -240,6 +240,7 @@ Supported `inputFormat`s include: - `csv` - `delimited` - `json` +- `kafka` - `avro_stream` - `protobuf` diff --git a/docs/ingestion/data-formats.md b/docs/ingestion/data-formats.md index 2f07d5c..a88e5fc 100644 --- a/docs/ingestion/data-formats.md +++ b/docs/ingestion/data-formats.md @@ -151,6 +151,75 @@ Be sure to change the `delimiter` to the appropriate delimiter for your data. Li } ``` +### KAFKA + +The `inputFormat` to load complete kafka record including header, key and value. An example is: + +```json +"ioConfig": { + "inputFormat": { + "type": "kafka", + "headerLabelPrefix": "kafka.header.", + "timestampColumnName": "kafka.timestamp", + "keyColumnName": "kafka.key", + "headerFormat": + { + "type": "string" + }, + "keyFormat": + { + "type": "json" + }, + "valueFormat": + { + "type": "json" + } + }, + ... +} +``` + +The KAFKA `inputFormat` has the following components: + +> Note that KAFKA inputFormat is currently designated as experimental. + +| Field | Type | Description | Required | +|-------|------|-------------|----------| +| type | String | This should say `kafka`. | yes | +| headerLabelPrefix | String | A custom label prefix for all the header columns. | no (default = "kafka.header.") | +| timestampColumnName | String | Specifies the name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") | +| keyColumnName | String | Specifies the name of the column for the kafka record's key.| no (default = "kafka.key") | +| headerFormat | Object | headerFormat specifies how to parse the kafka headers. Current supported type is "string". Since header values are bytes, the current parser by defaults reads it as UTF-8 encoded strings. There is flexibility to change this behavior by implementing your very own parser based on the encoding style. The 'encoding' type in KafkaStringHeaderFormat class needs to change with the custom implementation. | no | +| keyFormat | [InputFormat](#input-format) | keyFormat can be any existing inputFormat to parse the kafka key. The current behavior is to only process the first entry of the input format. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | no | +| valueFormat | [InputFormat](#input-format) | valueFormat can be any existing inputFormat to parse the kafka value payload. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | yes | + +``` +> For any conflicts in dimension/metric names, this inputFormat will prefer kafka value's column names. +> This will enable seemless porting of existing kafka ingestion inputFormat to this new format, with additional columns from kafka header and key. + +> Kafka input format fundamentally blends information from header, key and value portions of a kafka record to create a druid row. It does this by +> exploding individual records from the value and augmenting each of these values with the selected key/header columns. + +> Kafka input format also by default exposes kafka timestamp (timestampColumnName), which can be used as the primary timestamp column. +> One can also choose timestamp column from either key or value payload, if there is no timestamp available then the default kafka timestamp is our savior. +> eg., + + // Below timestampSpec chooses kafka's default timestamp that is available in kafka record + "timestampSpec": + { + "column": "kafka.timestamp", + "format": "millis" + } + + // Assuming there is a timestamp field in the header and we have "kafka.header." as a desired prefix for header columns, + // below example chooses header's timestamp as a primary timestamp column + "timestampSpec": + { + "column": "kafka.header.timestamp", + "format": "millis" + } +``` + ### ORC To use the ORC input format, load the Druid Orc extension ( [`druid-orc-extensions`](../development/extensions-core/orc.md)). diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaHeaderFormat.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaHeaderFormat.java new file mode 100644 index 0000000..b94b34c --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaHeaderFormat.java @@ -0,0 +1,37 @@ +/* + * 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.kafkainput; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.kafka.common.header.Headers; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "string", value = KafkaStringHeaderFormat.class) +}) + +public interface KafkaHeaderFormat +{ + KafkaHeaderReader createReader( + Headers headers, + String headerLabelPrefix + ); +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaHeaderReader.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaHeaderReader.java new file mode 100644 index 0000000..3e32034 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaHeaderReader.java @@ -0,0 +1,29 @@ +/* + * 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.kafkainput; + +import org.apache.druid.java.util.common.Pair; + +import java.util.List; + +public interface KafkaHeaderReader +{ + List<Pair<String, Object>> read(); +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputFormat.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputFormat.java new file mode 100644 index 0000000..b1143da --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputFormat.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.kafkainput; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.data.input.kafka.KafkaRecordEntity; +import org.apache.druid.java.util.common.DateTimes; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Objects; + +public class KafkaInputFormat implements InputFormat +{ + private static final String DEFAULT_HEADER_COLUMN_PREFIX = "kafka.header."; + private static final String DEFAULT_TIMESTAMP_COLUMN_NAME = "kafka.timestamp"; + private static final String DEFAULT_KEY_COLUMN_NAME = "kafka.key"; + public static final String DEFAULT_AUTO_TIMESTAMP_STRING = "__kif_auto_timestamp"; + + // Since KafkaInputFormat blends data from header, key and payload, timestamp spec can be pointing to an attribute within one of these + // 3 sections. To handle scenarios where there is no timestamp value either in key or payload, we induce an artifical timestamp value + // to avoid unnecessary parser barf out. Users in such situations can use the inputFormat's kafka record timestamp as its primary timestamp. + private final TimestampSpec dummyTimestampSpec = new TimestampSpec(DEFAULT_AUTO_TIMESTAMP_STRING, "auto", DateTimes.EPOCH); + + private final KafkaHeaderFormat headerFormat; + private final InputFormat valueFormat; + private final InputFormat keyFormat; + private final String headerColumnPrefix; + private final String keyColumnName; + private final String timestampColumnName; + + public KafkaInputFormat( + @JsonProperty("headerFormat") @Nullable KafkaHeaderFormat headerFormat, + @JsonProperty("keyFormat") @Nullable InputFormat keyFormat, + @JsonProperty("valueFormat") InputFormat valueFormat, + @JsonProperty("headerColumnPrefix") @Nullable String headerColumnPrefix, + @JsonProperty("keyColumnName") @Nullable String keyColumnName, + @JsonProperty("timestampColumnName") @Nullable String timestampColumnName + ) + { + this.headerFormat = headerFormat; + this.keyFormat = keyFormat; + this.valueFormat = Preconditions.checkNotNull(valueFormat, "valueFormat must not be null"); + this.headerColumnPrefix = headerColumnPrefix != null ? headerColumnPrefix : DEFAULT_HEADER_COLUMN_PREFIX; + this.keyColumnName = keyColumnName != null ? keyColumnName : DEFAULT_KEY_COLUMN_NAME; + this.timestampColumnName = timestampColumnName != null ? timestampColumnName : DEFAULT_TIMESTAMP_COLUMN_NAME; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory) + { + KafkaRecordEntity record = (KafkaRecordEntity) source; + InputRowSchema newInputRowSchema = new InputRowSchema(dummyTimestampSpec, inputRowSchema.getDimensionsSpec(), inputRowSchema.getColumnsFilter()); + return new KafkaInputReader( + inputRowSchema, + record, + (headerFormat == null) ? + null : + headerFormat.createReader(record.getRecord().headers(), headerColumnPrefix), + (keyFormat == null || record.getRecord().key() == null) ? + null : + keyFormat.createReader( + newInputRowSchema, + new ByteEntity(record.getRecord().key()), + temporaryDirectory + ), + (record.getRecord().value() == null) ? + null : + valueFormat.createReader( + newInputRowSchema, + source, + temporaryDirectory + ), + keyColumnName, + timestampColumnName + ); + } + + @Nullable + @JsonProperty + public KafkaHeaderFormat getHeaderFormat() + { + return headerFormat; + } + + @JsonProperty + public InputFormat getValueFormat() + { + return valueFormat; + } + + @Nullable + @JsonProperty + public InputFormat getKeyFormat() + { + return keyFormat; + } + + @Nullable + @JsonProperty + public String getHeaderColumnPrefix() + { + return headerColumnPrefix; + } + + @Nullable + @JsonProperty + public String getKeyColumnName() + { + return keyColumnName; + } + + @Nullable + @JsonProperty + public String getTimestampColumnName() + { + return timestampColumnName; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KafkaInputFormat that = (KafkaInputFormat) o; + return Objects.equals(headerFormat, that.headerFormat) + && Objects.equals(valueFormat, that.valueFormat) + && Objects.equals(keyFormat, that.keyFormat) + && Objects.equals(headerColumnPrefix, that.headerColumnPrefix) + && Objects.equals(keyColumnName, that.keyColumnName) + && Objects.equals(timestampColumnName, that.timestampColumnName); + } + + @Override + public int hashCode() + { + return Objects.hash(headerFormat, valueFormat, keyFormat, + headerColumnPrefix, keyColumnName, timestampColumnName + ); + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java new file mode 100644 index 0000000..14b70b0 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java @@ -0,0 +1,184 @@ +/* + * 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.kafkainput; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.apache.druid.data.input.InputEntityReader; +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.MapBasedInputRow; +import org.apache.druid.data.input.kafka.KafkaRecordEntity; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.ParseException; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +public class KafkaInputReader implements InputEntityReader +{ + private static final Logger log = new Logger(KafkaInputReader.class); + + private final InputRowSchema inputRowSchema; + private final KafkaRecordEntity record; + private final KafkaHeaderReader headerParser; + private final InputEntityReader keyParser; + private final InputEntityReader valueParser; + private final String keyColumnName; + private final String timestampColumnName; + + /** + * + * @param inputRowSchema Actual schema from the ingestion spec + * @param record kafka record containing header, key & value + * @param headerParser Header parser for parsing the header section, kafkaInputFormat allows users to skip header parsing section and hence an be null + * @param keyParser Key parser for key section, can be null as well + * @param valueParser Value parser is a required section in kafkaInputFormat, but because of tombstone records we can have a null parser here. + * @param keyColumnName Default key column name + * @param timestampColumnName Default kafka record's timestamp column name + */ + public KafkaInputReader( + InputRowSchema inputRowSchema, + KafkaRecordEntity record, + KafkaHeaderReader headerParser, + InputEntityReader keyParser, + InputEntityReader valueParser, + String keyColumnName, + String timestampColumnName + ) + { + this.inputRowSchema = inputRowSchema; + this.record = record; + this.headerParser = headerParser; + this.keyParser = keyParser; + this.valueParser = valueParser; + this.keyColumnName = keyColumnName; + this.timestampColumnName = timestampColumnName; + } + + private List<String> getFinalDimensionList(HashSet<String> newDimensions) + { + final List<String> schemaDimensions = inputRowSchema.getDimensionsSpec().getDimensionNames(); + if (!schemaDimensions.isEmpty()) { + return schemaDimensions; + } else { + return Lists.newArrayList( + Sets.difference(newDimensions, inputRowSchema.getDimensionsSpec().getDimensionExclusions()) + ); + } + } + + private CloseableIterator<InputRow> buildBlendedRows(InputEntityReader valueParser, Map<String, Object> headerKeyList) throws IOException + { + return valueParser.read().map( + r -> { + MapBasedInputRow valueRow; + try { + // Return type for the value parser should be of type MapBasedInputRow + // Parsers returning other types are not compatible currently. + valueRow = (MapBasedInputRow) r; + } + catch (ClassCastException e) { + throw new ParseException("Unsupported input format in valueFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows"); + } + Map<String, Object> event = new HashMap<>(headerKeyList); + /* Currently we prefer payload attributes if there is a collision in names. + We can change this beahvior in later changes with a config knob. This default + behavior lets easy porting of existing inputFormats to the new one without any changes. + */ + event.putAll(valueRow.getEvent()); + + HashSet<String> newDimensions = new HashSet<String>(valueRow.getDimensions()); + newDimensions.addAll(headerKeyList.keySet()); + // Remove the dummy timestamp added in KafkaInputFormat + newDimensions.remove(KafkaInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING); + + return new MapBasedInputRow( + inputRowSchema.getTimestampSpec().extractTimestamp(event), + getFinalDimensionList(newDimensions), + event + ); + } + ); + } + + private CloseableIterator<InputRow> buildRowsWithoutValuePayload(Map<String, Object> headerKeyList) + { + HashSet<String> newDimensions = new HashSet<String>(headerKeyList.keySet()); + InputRow row = new MapBasedInputRow( + inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList), + getFinalDimensionList(newDimensions), + headerKeyList + ); + List<InputRow> rows = Collections.singletonList(row); + return CloseableIterators.withEmptyBaggage(rows.iterator()); + } + + @Override + public CloseableIterator<InputRow> read() throws IOException + { + Map<String, Object> mergeMap = new HashMap<>(); + if (headerParser != null) { + List<Pair<String, Object>> headerList = headerParser.read(); + for (Pair<String, Object> ele : headerList) { + mergeMap.put(ele.lhs, ele.rhs); + } + } + + // Add kafka record timestamp to the mergelist, we will skip record timestamp if the same key exists already in the header list + mergeMap.putIfAbsent(timestampColumnName, record.getRecord().timestamp()); + + if (keyParser != null) { + try (CloseableIterator<InputRow> keyIterator = keyParser.read()) { + // Key currently only takes the first row and ignores the rest. + if (keyIterator.hasNext()) { + // Return type for the key parser should be of type MapBasedInputRow + // Parsers returning other types are not compatible currently. + MapBasedInputRow keyRow = (MapBasedInputRow) keyIterator.next(); + // Add the key to the mergeList only if the key string is not already present + mergeMap.putIfAbsent(keyColumnName, keyRow.getEvent().entrySet().stream().findFirst().get().getValue()); + } + } + catch (ClassCastException e) { + throw new IOException("Unsupported input format in keyFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows"); + } + } + + if (valueParser != null) { + return buildBlendedRows(valueParser, mergeMap); + } else { + return buildRowsWithoutValuePayload(mergeMap); + } + } + + @Override + public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException + { + return read().map(row -> InputRowListPlusRawValues.of(row, ((MapBasedInputRow) row).getEvent())); + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaStringHeaderFormat.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaStringHeaderFormat.java new file mode 100644 index 0000000..4a85600 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaStringHeaderFormat.java @@ -0,0 +1,80 @@ +/* + * 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.kafkainput; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.kafka.common.header.Headers; + +import javax.annotation.Nullable; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; +import java.util.Objects; + +public class KafkaStringHeaderFormat implements KafkaHeaderFormat +{ + private final Charset encoding; + + public KafkaStringHeaderFormat( + @JsonProperty("encoding") @Nullable String encoding + ) + { + this.encoding = (encoding != null) ? Charset.forName(encoding) : StandardCharsets.UTF_8; + } + + @Nullable + @JsonProperty + public String getEncoding() + { + return encoding.name(); + } + + @Override + public KafkaHeaderReader createReader( + Headers headers, + String headerLabelPrefix + ) + { + return new KafkaStringHeaderReader( + headers, + headerLabelPrefix, + encoding + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KafkaStringHeaderFormat that = (KafkaStringHeaderFormat) o; + return Objects.equals(encoding, that.encoding); + } + + @Override + public int hashCode() + { + return Objects.hash(encoding); + } + +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaStringHeaderReader.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaStringHeaderReader.java new file mode 100644 index 0000000..4ddbf52 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaStringHeaderReader.java @@ -0,0 +1,58 @@ +/* + * 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.kafkainput; + +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; + +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.List; + +public class KafkaStringHeaderReader implements KafkaHeaderReader +{ + private static final Logger log = new Logger(KafkaStringHeaderReader.class); + private final Headers headers; + private final String headerLabelPrefix; + private final Charset encoding; + + public KafkaStringHeaderReader(Headers headers, + String headerLabelPrefix, + Charset encoding) + { + this.headers = headers; + this.headerLabelPrefix = headerLabelPrefix; + this.encoding = encoding; + } + + @Override + public List<Pair<String, Object>> read() + { + List<Pair<String, Object>> events = new ArrayList<>(); + for (Header hdr : headers) { + String s = new String(hdr.value(), this.encoding); + String newKey = this.headerLabelPrefix + hdr.key(); + events.add(Pair.of(newKey, s)); + } + return events; + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java index cf780ad..5c7b9e3 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.TypeLiteral; +import org.apache.druid.data.input.kafkainput.KafkaInputFormat; import org.apache.druid.guice.LazySingleton; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; @@ -49,7 +50,8 @@ public class KafkaIndexTaskModule implements DruidModule new NamedType(KafkaIndexTaskTuningConfig.class, "KafkaTuningConfig"), new NamedType(KafkaSupervisorTuningConfig.class, "kafka"), new NamedType(KafkaSupervisorSpec.class, "kafka"), - new NamedType(KafkaSamplerSpec.class, "kafka") + new NamedType(KafkaSamplerSpec.class, "kafka"), + new NamedType(KafkaInputFormat.class, "kafka") ) ); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java new file mode 100644 index 0000000..aa18100 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java @@ -0,0 +1,446 @@ +/* + * 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.kafkainput; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import org.apache.druid.data.input.ColumnsFilter; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.data.input.kafka.KafkaRecordEntity; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; +import org.apache.druid.java.util.common.parsers.JSONPathFieldType; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; + +public class KafkaInputFormatTest +{ + private KafkaRecordEntity inputEntity; + private long timestamp = DateTimes.of("2021-06-24").getMillis(); + private static final Iterable<Header> SAMPLE_HEADERS = ImmutableList.of(new Header() { + @Override + public String key() + { + return "encoding"; + } + @Override + public byte[] value() + { + return "application/json".getBytes(StandardCharsets.UTF_8); + } + }, + new Header() { + @Override + public String key() + { + return "kafkapkc"; + } + @Override + public byte[] value() + { + return "pkc-bar".getBytes(StandardCharsets.UTF_8); + } + }); + private KafkaInputFormat format; + + @Before + public void setUp() + { + format = new KafkaInputFormat( + new KafkaStringHeaderFormat(null), + // Key Format + new JsonInputFormat( + new JSONPathSpec(true, ImmutableList.of()), + null, null, false //make sure JsonReader is used + ), + // Value Format + new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null, null, false //make sure JsonReader is used + ), + "kafka.newheader.", "kafka.newkey.key", "kafka.newts.timestamp" + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = new ObjectMapper(); + KafkaInputFormat kif = new KafkaInputFormat( + new KafkaStringHeaderFormat(null), + // Key Format + new JsonInputFormat( + new JSONPathSpec(true, ImmutableList.of()), + null, null, false //make sure JsonReader is used + ), + // Value Format + new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null, null, false //make sure JsonReader is used + ), + "kafka.newheader.", "kafka.newkey.key", "kafka.newts.timestamp" + ); + Assert.assertEquals(format, kif); + + final byte[] formatBytes = mapper.writeValueAsBytes(format); + final byte[] kifBytes = mapper.writeValueAsBytes(kif); + Assert.assertTrue(Arrays.equals(formatBytes, kifBytes)); + } + + @Test + public void testWithHeaderKeyAndValue() throws IOException + { + final byte[] key = StringUtils.toUtf8( + "{\n" + + " \"key\": \"sampleKey\"\n" + + "}"); + + final byte[] payload = StringUtils.toUtf8( + "{\n" + + " \"timestamp\": \"2021-06-25\",\n" + + " \"bar\": null,\n" + + " \"foo\": \"x\",\n" + + " \"baz\": 4,\n" + + " \"o\": {\n" + + " \"mg\": 1\n" + + " }\n" + + "}"); + + Headers headers = new RecordHeaders(SAMPLE_HEADERS); + inputEntity = new KafkaRecordEntity(new ConsumerRecord<byte[], byte[]>( + "sample", 0, 0, timestamp, + null, null, 0, 0, + key, payload, headers)); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "bar", "foo", + "kafka.newheader.encoding", + "kafka.newheader.kafkapkc", + "kafka.newts.timestamp" + ))), + ColumnsFilter.all() + ), + inputEntity, + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator<InputRow> iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + + // Payload verifications + Assert.assertEquals(DateTimes.of("2021-06-25"), row.getTimestamp()); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + + // Header verification + Assert.assertEquals("application/json", Iterables.getOnlyElement(row.getDimension("kafka.newheader.encoding"))); + Assert.assertEquals("pkc-bar", Iterables.getOnlyElement(row.getDimension("kafka.newheader.kafkapkc"))); + Assert.assertEquals(String.valueOf(DateTimes.of("2021-06-24").getMillis()), + Iterables.getOnlyElement(row.getDimension("kafka.newts.timestamp"))); + Assert.assertEquals("2021-06-25", + Iterables.getOnlyElement(row.getDimension("timestamp"))); + + // Key verification + Assert.assertEquals("sampleKey", Iterables.getOnlyElement(row.getDimension("kafka.newkey.key"))); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + //Headers cannot be null, so testing only no key use case! + public void testWithOutKey() throws IOException + { + final byte[] payload = StringUtils.toUtf8( + "{\n" + + " \"timestamp\": \"2021-06-24\",\n" + + " \"bar\": null,\n" + + " \"foo\": \"x\",\n" + + " \"baz\": 4,\n" + + " \"o\": {\n" + + " \"mg\": 1\n" + + " }\n" + + "}"); + + Headers headers = new RecordHeaders(SAMPLE_HEADERS); + inputEntity = new KafkaRecordEntity(new ConsumerRecord<byte[], byte[]>( + "sample", 0, 0, timestamp, + null, null, 0, 0, + null, payload, headers)); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "bar", "foo", + "kafka.newheader.encoding", + "kafka.newheader.kafkapkc", + "kafka.newts.timestamp" + ))), + ColumnsFilter.all() + ), + inputEntity, + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator<InputRow> iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + + // Key verification + Assert.assertTrue(row.getDimension("kafka.newkey.key").isEmpty()); + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + + } + + @Test + public void testTimestampFromHeader() throws IOException + { + Iterable<Header> sample_header_with_ts = Iterables.unmodifiableIterable( + Iterables.concat( + SAMPLE_HEADERS, + ImmutableList.of(new Header() { + @Override + public String key() + { + return "headerTs"; + } + @Override + public byte[] value() + { + return "2021-06-24".getBytes(StandardCharsets.UTF_8); + } + } + ))); + final byte[] key = StringUtils.toUtf8( + "{\n" + + " \"key\": \"sampleKey\"\n" + + "}"); + + final byte[] payload = StringUtils.toUtf8( + "{\n" + + " \"timestamp\": \"2021-06-24\",\n" + + " \"bar\": null,\n" + + " \"foo\": \"x\",\n" + + " \"baz\": 4,\n" + + " \"o\": {\n" + + " \"mg\": 1\n" + + " }\n" + + "}"); + + Headers headers = new RecordHeaders(sample_header_with_ts); + inputEntity = new KafkaRecordEntity(new ConsumerRecord<byte[], byte[]>( + "sample", 0, 0, timestamp, + null, null, 0, 0, + key, payload, headers)); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("kafka.newheader.headerTs", "iso", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "bar", "foo", + "kafka.newheader.encoding", + "kafka.newheader.kafkapkc" + ))), + ColumnsFilter.all() + ), + inputEntity, + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator<InputRow> iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + final MapBasedInputRow mrow = (MapBasedInputRow) row; + // Payload verifications + Assert.assertEquals(DateTimes.of("2021-06-24"), row.getTimestamp()); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + + // Header verification + Assert.assertEquals("application/json", Iterables.getOnlyElement(row.getDimension("kafka.newheader.encoding"))); + Assert.assertEquals("pkc-bar", Iterables.getOnlyElement(row.getDimension("kafka.newheader.kafkapkc"))); + Assert.assertEquals(String.valueOf(DateTimes.of("2021-06-24").getMillis()), + Iterables.getOnlyElement(row.getDimension("kafka.newts.timestamp"))); + Assert.assertEquals("2021-06-24", + Iterables.getOnlyElement(row.getDimension("kafka.newheader.headerTs"))); + Assert.assertEquals("2021-06-24", + Iterables.getOnlyElement(row.getDimension("timestamp"))); + + // Key verification + Assert.assertEquals("sampleKey", Iterables.getOnlyElement(row.getDimension("kafka.newkey.key"))); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testWithOutKeyAndHeaderSpecs() throws IOException + { + final byte[] payload = StringUtils.toUtf8( + "{\n" + + " \"timestamp\": \"2021-06-24\",\n" + + " \"bar\": null,\n" + + " \"foo\": \"x\",\n" + + " \"baz\": 4,\n" + + " \"o\": {\n" + + " \"mg\": 1\n" + + " }\n" + + "}"); + + Headers headers = new RecordHeaders(SAMPLE_HEADERS); + inputEntity = new KafkaRecordEntity(new ConsumerRecord<byte[], byte[]>( + "sample", 0, 0, timestamp, + null, null, 0, 0, + null, payload, headers)); + + KafkaInputFormat localFormat = new KafkaInputFormat( + null, + null, + // Value Format + new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null, null, false //make sure JsonReader is used + ), + "kafka.newheader.", "kafka.newkey.", "kafka.newts." + ); + + final InputEntityReader reader = localFormat.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "bar", "foo", + "kafka.newts.timestamp" + ))), + ColumnsFilter.all() + ), + inputEntity, + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator<InputRow> iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + + // Key verification + Assert.assertTrue(row.getDimension("kafka.newkey.key").isEmpty()); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + + } +} diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaStringHeaderFormatTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaStringHeaderFormatTest.java new file mode 100644 index 0000000..4b3bce9 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaStringHeaderFormatTest.java @@ -0,0 +1,219 @@ +/* + * 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.kafkainput; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.kafka.KafkaRecordEntity; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Pair; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; + + +public class KafkaStringHeaderFormatTest +{ + private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final KafkaStringHeaderFormat KAFKAHEADERNOENCODE = new KafkaStringHeaderFormat(null); + private static final Iterable<Header> SAMPLE_HEADERS = ImmutableList.of( + new Header() + { + @Override + public String key() + { + return "encoding"; + } + + @Override + public byte[] value() + { + return "application/json".getBytes(StandardCharsets.UTF_8); + } + }, + new Header() + { + @Override + public String key() + { + return "kafkapkc"; + } + + @Override + public byte[] value() + { + return "pkc-bar".getBytes(StandardCharsets.UTF_8); + } + } + ); + private KafkaRecordEntity inputEntity; + private long timestamp = DateTimes.of("2021-06-24T00:00:00.000Z").getMillis(); + + @Test + public void testSerde() throws JsonProcessingException + { + Assert.assertEquals( + KAFKAHEADERNOENCODE, + KAFKAHEADERNOENCODE + ); + Assert.assertEquals( + KAFKAHEADERNOENCODE, + MAPPER.readValue(MAPPER.writeValueAsString(KAFKAHEADERNOENCODE), KafkaStringHeaderFormat.class) + ); + final KafkaStringHeaderFormat kafkaAsciiHeader = new KafkaStringHeaderFormat("US-ASCII"); + Assert.assertNotEquals( + KAFKAHEADERNOENCODE, + kafkaAsciiHeader + ); + } + + @Test + public void testDefaultHeaderFormat() + { + String headerLabelPrefix = "test.kafka.header."; + Headers headers = new RecordHeaders(SAMPLE_HEADERS); + inputEntity = new KafkaRecordEntity(new ConsumerRecord<byte[], byte[]>( + "sample", 0, 0, timestamp, + null, null, 0, 0, + null, "sampleValue".getBytes(StandardCharsets.UTF_8), headers + )); + List<Pair<String, Object>> expectedResults = Arrays.asList( + Pair.of("test.kafka.header.encoding", "application/json"), + Pair.of("test.kafka.header.kafkapkc", "pkc-bar") + ); + + KafkaHeaderFormat headerInput = new KafkaStringHeaderFormat(null); + KafkaHeaderReader headerParser = headerInput.createReader(inputEntity.getRecord().headers(), headerLabelPrefix); + Assert.assertEquals(expectedResults, headerParser.read()); + } + + @Test + public void testASCIIHeaderFormat() + { + Iterable<Header> header = ImmutableList.of( + new Header() + { + @Override + public String key() + { + return "encoding"; + } + + @Override + public byte[] value() + { + return "application/json".getBytes(StandardCharsets.US_ASCII); + } + }, + new Header() + { + @Override + public String key() + { + return "kafkapkc"; + } + + @Override + public byte[] value() + { + return "pkc-bar".getBytes(StandardCharsets.US_ASCII); + } + } + ); + + String headerLabelPrefix = "test.kafka.header."; + Headers headers = new RecordHeaders(header); + inputEntity = new KafkaRecordEntity(new ConsumerRecord<byte[], byte[]>( + "sample", 0, 0, timestamp, + null, null, 0, 0, + null, "sampleValue".getBytes(StandardCharsets.UTF_8), headers + )); + List<Pair<String, Object>> expectedResults = Arrays.asList( + Pair.of("test.kafka.header.encoding", "application/json"), + Pair.of("test.kafka.header.kafkapkc", "pkc-bar") + ); + + KafkaHeaderFormat headerInput = new KafkaStringHeaderFormat("US-ASCII"); + KafkaHeaderReader headerParser = headerInput.createReader(inputEntity.getRecord().headers(), headerLabelPrefix); + List<Pair<String, Object>> rows = headerParser.read(); + Assert.assertEquals(expectedResults, rows); + } + + @Test + public void testIllegalHeaderCharacter() + { + Iterable<Header> header = ImmutableList.of( + new Header() + { + @Override + public String key() + { + return "encoding"; + } + + @Override + public byte[] value() + { + return "€pplic€tion/json".getBytes(StandardCharsets.US_ASCII); + } + }, + new Header() + { + @Override + public String key() + { + return "kafkapkc"; + } + + @Override + public byte[] value() + { + return "pkc-bar".getBytes(StandardCharsets.US_ASCII); + } + } + ); + + String headerLabelPrefix = "test.kafka.header."; + Headers headers = new RecordHeaders(header); + inputEntity = new KafkaRecordEntity(new ConsumerRecord<byte[], byte[]>( + "sample", 0, 0, timestamp, + null, null, 0, 0, + null, "sampleValue".getBytes(StandardCharsets.UTF_8), headers + )); + List<Pair<String, Object>> expectedResults = Arrays.asList( + Pair.of("test.kafka.header.encoding", "?pplic?tion/json"), + Pair.of("test.kafka.header.kafkapkc", "pkc-bar") + ); + + KafkaHeaderFormat headerInput = new KafkaStringHeaderFormat("US-ASCII"); + KafkaHeaderReader headerParser = headerInput.createReader(inputEntity.getRecord().headers(), headerLabelPrefix); + List<Pair<String, Object>> rows = headerParser.read(); + Assert.assertEquals(expectedResults, rows); + } +} + 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 8732ed7..79b6f4b 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 @@ -53,6 +53,8 @@ import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.kafka.KafkaRecordEntity; +import org.apache.druid.data.input.kafkainput.KafkaInputFormat; +import org.apache.druid.data.input.kafkainput.KafkaStringHeaderFormat; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.discovery.LookupNodeService; @@ -201,6 +203,12 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase return "application/json".getBytes(StandardCharsets.UTF_8); } }); + private static final InputFormat KAFKA_INPUT_FORMAT = new KafkaInputFormat( + new KafkaStringHeaderFormat(null), + INPUT_FORMAT, + INPUT_FORMAT, + "kafka.testheader.", "kafka.key", "kafka.timestamp" + ); private static TestingCluster zkServer; private static TestBroker kafkaServer; @@ -1246,6 +1254,85 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase } @Test(timeout = 60_000L) + public void testKafkaInputFormat() throws Exception + { + // Insert data + insertData(Iterables.limit(records, 3)); + + final KafkaIndexTask task = createTask( + null, + new DataSchema( + "test_ds", + 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 StringDimensionSchema("kafka.testheader.encoding") + ), + null, + null + ), + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + }, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null + ), + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null, + KAFKA_INPUT_FORMAT + ) + ); + Assert.assertTrue(task.supportsQueries()); + + final ListenableFuture<TaskStatus> future = runTask(task); + + while (countEvents(task) != 3) { + Thread.sleep(25); + } + + Assert.assertEquals(Status.READING, task.getRunner().getStatus()); + + final QuerySegmentSpec interval = OBJECT_MAPPER.readValue( + "\"2008/2012\"", QuerySegmentSpec.class + ); + List<ScanResultValue> scanResultValues = scanData(task, interval); + //verify that there are no records indexed in the rollbacked time period + Assert.assertEquals(3, Iterables.size(scanResultValues)); + + int i = 0; + for (ScanResultValue result : scanResultValues) { + final Map<String, Object> event = ((List<Map<String, Object>>) result.getEvents()).get(0); + Assert.assertEquals("application/json", event.get("kafka.testheader.encoding")); + Assert.assertEquals("y", event.get("dim2")); + } + + // insert remaining data + insertData(Iterables.skip(records, 3)); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + } + + @Test(timeout = 60_000L) public void testRunOnNothing() throws Exception { // Insert data diff --git a/website/.spelling b/website/.spelling index d6e22f7..3f7690f 100644 --- a/website/.spelling +++ b/website/.spelling @@ -973,10 +973,21 @@ column_1 column_n com.opencsv ctrl +headerFormat +headerLabelPrefix jsonLowercase +kafka +KafkaStringHeaderFormat +kafka.header. +kafka.key +kafka.timestamp +keyColumnName +keyFormat listDelimiter +timestampColumnName timestampSpec urls +valueFormat - ../docs/ingestion/data-management.md 1GB IOConfig --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@druid.apache.org For additional commands, e-mail: commits-h...@druid.apache.org