-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Kafka Input Format for headers, key and payload parsing. #11630
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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 hidden or 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 hidden or 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 hidden or 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 hidden or 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.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 | ||
| ); | ||
| } | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.