[
https://issues.apache.org/jira/browse/FLINK-29398?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17608699#comment-17608699
]
Qingsheng Ren commented on FLINK-29398:
---------------------------------------
Thanks for starting the discussion [~jeremy.degroot] ! This is a very
interesting also useful feature as you described.
Under the design of FLIP-27 Source API we do expose host name of source readers
when registering readers on the split enumerator, so it's possible to let split
enumerator to make assignments according to the mapping of rack and hostname of
readers. File source has already implemented this feature (see
{{{}LocalityAwareSplitAssigner{}}}). Currently the split assigning strategy of
Kafka source is a hard-coded one, so in order to achieve this in KafkaSource we
need to design a new API to let users provide pluggable split assigner for
split enumerator in Kafka source.
Moreover a fully optimized solution would be that Flink scheduler could also
schedule tasks based on locality, but this is beyond the discussion of this
ticket.
[~jeremy.degroot] I'm not sure if your solution is based on the new KafkaSource
instead of the deprecated FlinkKafkaConsumer (we won't add new features to the
deprecated one). Would you like to lead the design of this feature? I think a
new FLIP is expected for this as we are introducing a new feature to Kafka
source.
> Utilize Rack Awareness in Flink Consumer
> ----------------------------------------
>
> Key: FLINK-29398
> URL: https://issues.apache.org/jira/browse/FLINK-29398
> Project: Flink
> Issue Type: Improvement
> Components: Connectors / Kafka
> Reporter: Jeremy DeGroot
> Priority: Major
>
> [KIP-708|https://cwiki.apache.org/confluence/display/KAFKA/KIP-708%3A+Rack+awareness+for+Kafka+Streams]
> was implemented some time ago in Kafka. This allows brokers and consumers to
> communicate about the rack (or AWS Availability Zone) they're located in.
> Reading from a local broker can save money in bandwidth and improve latency
> for your consumers.
> Flink Kafka consumers currently cannot easily rack awareness if they're
> deployed across multiple racks or availability zones, because they have no
> control over which rack the Task Manager they'll be assigned to may be in.
> This improvement proposes that a Kafka Consumer could be configured with a
> callback or Future that could be run when it's being configured on the task
> manager, that will set the appropriate value at runtime if a value is
> provided.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)