[ https://issues.apache.org/jira/browse/KAFKA-8147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17332711#comment-17332711 ]
A. Sophie Blee-Goldman commented on KAFKA-8147: ----------------------------------------------- [~philbour] that would be a bug, you should be able to set these configs in any order. Seems like BufferConfigInternal#emitEarlyWhenFull creates a new EagerBufferConfigImpl and passes the two original configs (maxRecords and maxBytes) in to the constructor, but loses the logging configs at that point. Same thing for BufferConfigInternal#shutDownWhenFull. Looks like the PR for this feature just missed updating this, I notice that it did remember to add this parameter in the constructor calls inside EagerBufferConfigImpl and StrictBufferConfigImpl. That said, this looks like kind of an abuse of this pattern so I'm not surprised bugs slipped through. Maybe instead of just patching the current problem by adding this parameter to the constructor calls in BufferConfigInternal we can try to refactor things a bit so we aren't calling constructors all over the place and making things vulnerable to future changes. For example in Materialized all of the non-static .withXXX methods just set that parameter directly instead of creating a new Materialized object every time you set some configuration. But I'm sure there was a reason to do it this way initially... [~philbour] can you file a separate ticket for this? And would you be interested in submitting a PR to fix the bug you found? > Add changelog topic configuration to KTable suppress > ---------------------------------------------------- > > Key: KAFKA-8147 > URL: https://issues.apache.org/jira/browse/KAFKA-8147 > Project: Kafka > Issue Type: Improvement > Components: streams > Affects Versions: 2.1.1 > Reporter: Maarten > Assignee: highluck > Priority: Minor > Labels: kip > Fix For: 2.6.0 > > > The streams DSL does not provide a way to configure the changelog topic > created by KTable.suppress. > From the perspective of an external user this could be implemented similar to > the configuration of aggregate + materialized, i.e., > {code:java} > changelogTopicConfigs = // Configs > materialized = Materialized.as(..).withLoggingEnabled(changelogTopicConfigs) > .. > KGroupedStream.aggregate(..,materialized) > {code} > [KIP-446: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-446%3A+Add+changelog+topic+configuration+to+KTable+suppress|https://cwiki.apache.org/confluence/display/KAFKA/KIP-446%3A+Add+changelog+topic+configuration+to+KTable+suppress] -- This message was sent by Atlassian Jira (v8.3.4#803005)