[ https://issues.apache.org/jira/browse/KAFKA-6761?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16509742#comment-16509742 ]
ASF GitHub Bot commented on KAFKA-6761: --------------------------------------- bbejeck opened a new pull request #5201: KAFKA-6761: Construct Physical Plan using Graph, Reduce streams footprint part III URL: https://github.com/apache/kafka/pull/5201 Sorry for the massive PR, but at this point it's very difficult to break up into smaller parts now that we are building the logical and physical plan. It's worth noting at them moment this PR does not include optimizations, for two reasons 1. We need a solution to providing properties for the topology build process. 2. Some of the repartition optimizations need more work The specific changes in this PR from the second PR include 1. Moved all graph objects into a separate package. This requires the graph objects to have public access but after looking at the code for some time it's worth the trade-off to have a cleaner code-base 2. Changed the types of graph nodes to names conveying more context 2. Build the entire physical plan from the graph, after `StreamsBuilder.build()` is called. 3. I'm currently working on applying optimizations and they will be applied soon. Those optimizations will include: 1. Re-using source topics as changelogs for KTables 2. Re-using sink topics as changelogs for statestore that output directly to sinks 3. Automatically repartition for key-changing operations followed by other DSL operations that would repartition on their own, reducing repartitions from N to 1. Other changes are addressed directly as review comments on the PR. Testing consists of using all existing streams tests to validate building the physical plan with graph ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade notes) ---------------------------------------------------------------- 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 > Reduce Kafka Streams Footprint > ------------------------------ > > Key: KAFKA-6761 > URL: https://issues.apache.org/jira/browse/KAFKA-6761 > Project: Kafka > Issue Type: Improvement > Components: streams > Reporter: Bill Bejeck > Assignee: Bill Bejeck > Priority: Major > Fix For: 2.1.0 > > > The persistent storage footprint of a Kafka Streams application contains the > following aspects: > # The internal topics created on the Kafka cluster side. > # The materialized state stores on the Kafka Streams application instances > side. > There have been some questions about reducing these footprints, especially > since many of them are not necessary. For example, there are redundant > internal topics, as well as unnecessary state stores that takes up space but > also affect performance. When people are pushing Streams to production with > high traffic, this issue would be more common and severe. Reducing the > footprint of Streams have clear benefits for reducing resource utilization of > Kafka Streams applications, and also not creating pressure on broker's > capacities. -- This message was sent by Atlassian JIRA (v7.6.3#76005)