[ 
https://issues.apache.org/jira/browse/FLINK-15670?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17035182#comment-17035182
 ] 

Yuan Mei commented on FLINK-15670:
----------------------------------

I have spent a couple of days hacking a prototype using existing Kafka Producer 
and Consumer to get some hands-on experiences, code link here: 
[https://github.com/apache/flink/compare/master...curcur:kafka_avoid_shuffle?expand=1]

The prototype works as follows:
 # writes data to specific Kafka partitions based on the first field (key)
 # read data through KafkaClient to make sure that each partition contains the 
data as expected (this section is commented)
 # consume data from Kafka partitions to make sure each sub-task reads data as 
expected (keyed)

 

 

We can wrap item 1 to a Kafka sink, and item 3 to a Kafka source to hide 
details how the partition work is done. We can wrap 
`KeyGroupRangeAssignment.assignKeyToParallelOperator` in both the write and the 
read side.

 

Two few more questions:
 # I am using `FlinkKafkaPartitioner<>` to assign a key to a specific 
partition, and hence am using KeyedSerializationSchema correspondingly. 
However, I found all `FlinkKafkaProducer` constructors with 
`FlinkKafkaPartitioner<>` are deprecated (at least for the universal version of 
Kafka connector). Am I doing anything wrong here? Or better ways of doing so?
 # How to avoid a `real` shuffle. 

```

DataStream<T> fromKafka = env.addKafkaSource(new FlinkKafkaConsumer<>(...), 
keyfields1);

fromKafka.keyby(keyfields1).sum(...)

```

In the above example, does the planner knows that `fromKafka` datastream is 
ready to go and do not need a shuffle? If not, we do need some way to tell the 
planner to avoid a shuffle?

 

 

> Provide a Kafka Source/Sink pair that aligns Kafka's Partitions and Flink's 
> KeyGroups
> -------------------------------------------------------------------------------------
>
>                 Key: FLINK-15670
>                 URL: https://issues.apache.org/jira/browse/FLINK-15670
>             Project: Flink
>          Issue Type: New Feature
>          Components: API / DataStream, Connectors / Kafka
>            Reporter: Stephan Ewen
>            Priority: Major
>              Labels: usability
>             Fix For: 1.11.0
>
>
> This Source/Sink pair would serve two purposes:
> 1. You can read topics that are already partitioned by key and process them 
> without partitioning them again (avoid shuffles)
> 2. You can use this to shuffle through Kafka, thereby decomposing the job 
> into smaller jobs and independent pipelined regions that fail over 
> independently.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to