-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Introduce KafkaRecordEntity to support Kafka headers in InputFormats #10730
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
Changes from all commits
b8bbb1e
b687f16
809de31
0b0911e
a42d71d
36e51e0
876d455
99c37e0
abd1425
f35f7d4
025822b
680709b
9ef2256
c70418d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,53 @@ | ||
| /* | ||
| * 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.kafka; | ||
|
|
||
| import org.apache.druid.data.input.InputFormat; | ||
| import org.apache.druid.data.input.impl.ByteEntity; | ||
| import org.apache.druid.indexing.kafka.KafkaRecordSupplier; | ||
| import org.apache.kafka.clients.consumer.ConsumerRecord; | ||
|
|
||
| /** | ||
| * A {@link ByteEntity} generated by {@link KafkaRecordSupplier} and fed to any {@link InputFormat} used by Kafka | ||
| * indexing tasks. | ||
| * <p> | ||
| * It can be used as a regular ByteEntity, in which case the Kafka record value is returned, but the {@link #getRecord} | ||
| * method also allows Kafka-aware {@link InputFormat} implementations to read the full Kafka record, including headers, | ||
| * key, and timestamp. | ||
| * <p> | ||
| * NOTE: Any records with null values will be skipped, even if they contain non-null keys, or headers | ||
| * <p> | ||
| * This functionality is not yet exposed through any built-in InputFormats, but is available for use in extensions. | ||
| */ | ||
| public class KafkaRecordEntity extends ByteEntity | ||
| { | ||
| private final ConsumerRecord<byte[], byte[]> record; | ||
|
|
||
| public KafkaRecordEntity(ConsumerRecord<byte[], byte[]> record) | ||
| { | ||
| super(record.value()); | ||
| this.record = record; | ||
| } | ||
|
|
||
| public ConsumerRecord<byte[], byte[]> getRecord() | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. not used anywhere , is this only to allow writing extensions with custom InputFormat which can take advantage of the extra metadata available ?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. that's correct. We could have a sample InputFormat for documentation purposes, though I'm not sure if there is much value. A more generic input format for Kafka that wraps multiple input-formats would require a lot more thought, and I couldn't think of a one-size-fits-all approach that seemed very useful or didn't have additional complexities to deal with. |
||
| { | ||
| return record; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,6 +22,7 @@ | |
| import com.fasterxml.jackson.databind.ObjectMapper; | ||
| import com.google.common.annotations.VisibleForTesting; | ||
| import com.google.common.collect.ImmutableList; | ||
| import org.apache.druid.data.input.kafka.KafkaRecordEntity; | ||
| import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; | ||
| import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; | ||
| import org.apache.druid.indexing.seekablestream.common.RecordSupplier; | ||
|
|
@@ -51,7 +52,7 @@ | |
| import java.util.concurrent.Callable; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| public class KafkaRecordSupplier implements RecordSupplier<Integer, Long> | ||
| public class KafkaRecordSupplier implements RecordSupplier<Integer, Long, KafkaRecordEntity> | ||
| { | ||
| private final KafkaConsumer<byte[], byte[]> consumer; | ||
| private boolean closed; | ||
|
|
@@ -119,15 +120,16 @@ public Set<StreamPartition<Integer>> getAssignment() | |
|
|
||
| @Nonnull | ||
| @Override | ||
| public List<OrderedPartitionableRecord<Integer, Long>> poll(long timeout) | ||
| public List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> poll(long timeout) | ||
| { | ||
| List<OrderedPartitionableRecord<Integer, Long>> polledRecords = new ArrayList<>(); | ||
| List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> polledRecords = new ArrayList<>(); | ||
| for (ConsumerRecord<byte[], byte[]> record : consumer.poll(Duration.ofMillis(timeout))) { | ||
|
|
||
| polledRecords.add(new OrderedPartitionableRecord<>( | ||
| record.topic(), | ||
| record.partition(), | ||
| record.offset(), | ||
| record.value() == null ? null : ImmutableList.of(record.value()) | ||
| record.value() == null ? null : ImmutableList.of(new KafkaRecordEntity(record)) | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @himanshug I reverted this change to keep the existing behavior
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We should put in a comment or javadoc noting that records with null values will be skipped, even if the other stuff (keys, timestamp, headers) are not null. Here's a suggestion:
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done, KafkaRecordEntity will already barf on null values when instantiating ByteEntity |
||
| )); | ||
| } | ||
| return polledRecords; | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you please add javadocs explaining how this class is meant to be used? Something like: