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

ASF GitHub Bot commented on FLINK-10122:
----------------------------------------

tzulitai commented on a change in pull request #6537: [FLINK-10122] 
KafkaConsumer should use partitionable state over union state if partition 
discovery is not active
URL: https://github.com/apache/flink/pull/6537#discussion_r210808619
 
 

 ##########
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
 ##########
 @@ -480,7 +477,7 @@ public void open(Configuration configuration) throws 
Exception {
                        }
 
                        for (Map.Entry<KafkaTopicPartition, Long> 
restoredStateEntry : restoredState.entrySet()) {
-                               if (!restoredFromOldState) {
+                               if (discoveryIntervalMillis != 
PARTITION_DISCOVERY_DISABLED) {
 
 Review comment:
   This, therefore, is a bit confusing.
   
   If we decide to change it so that whether or not partition discovery is 
enabled is determined at the very beginning and cannot be changed from them on 
(even on restarts), then there shouldn't be a case to check 
`discoveryIntervalMillis != PARTITION_DISCOVERY_DISABLED` here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> KafkaConsumer should use partitionable state over union state if partition 
> discovery is not active
> --------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-10122
>                 URL: https://issues.apache.org/jira/browse/FLINK-10122
>             Project: Flink
>          Issue Type: Improvement
>          Components: Kafka Connector
>            Reporter: Stefan Richter
>            Assignee: Stefan Richter
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.7.0
>
>
> KafkaConsumer store its offsets state always as union state. I think this is 
> only required in the case that partition discovery is active. For jobs with a 
> very high parallelism, the union state can lead to prohibitively expensive 
> deployments. For example, a job with 2000 source and a total of 10MB 
> checkpointed union state offsets state would have to ship ~ 2000 x 10MB = 
> 20GB of state. With partitionable state, it would have to ship ~10MB.
> For now, I would suggest to go back to partitionable state in case that 
> partition discovery is not active. In the long run, I have some ideas for 
> more efficient partitioning schemes that would also work for active discovery.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to