forked from apache/druid
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Kafka Input Format for headers, key and payload parsing (apache#11630)
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 as useful dimension for aggregation and in turn bringing better insights. PR(apache#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 way to expose the data present in the key as druid columns. We need a generic way to express at configuration time what attributes from headers, key and payload need to be ingested into druid. We need to keep the design generic enough so that users can specify different parsers for headers, key and payload. 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". 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. (cherry picked from commit ad6609a)
- Loading branch information
1 parent
62087ce
commit 51de994
Showing
13 changed files
with
1,400 additions
and
2 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
37 changes: 37 additions & 0 deletions
37
...exing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaHeaderFormat.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 | ||
); | ||
} |
29 changes: 29 additions & 0 deletions
29
...exing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaHeaderReader.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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(); | ||
} |
175 changes: 175 additions & 0 deletions
175
...dexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputFormat.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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.getMetricNames()); | ||
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 | ||
); | ||
} | ||
} |
Oops, something went wrong.