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

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

StefanRRichter opened a new 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
 
 
   ## What is the purpose of the change
   
   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.
   
   As an additional step to support backwards compatibility and avoid state 
blowup from the compatibility state, I also implemented FLINK-10121, so that 
operator state can be removed from the backend.
   
   Included also a hotfix that makes state assignment on the JM more memory 
friendly by trying to allocate all collections with good sizes.
   
   ## Verifying this change
   
   This change is already covered by existing tests.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency):  (no)
     - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (yes)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable)
   

----------------------------------------------------------------
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:
us...@infra.apache.org


> 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