[ https://issues.apache.org/jira/browse/KAFKA-10494?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Andy Coates updated KAFKA-10494: -------------------------------- Description: Calling `enableSendingOldValues` on many nodes, e.g. `KTableFilter`, is unnecessarily calling `enableSendingOldValues` on the parent, even when the processor itself is materialized. This can force the parent table to be materialized unnecessarily. For example: ``` StreamsBuilder builder = new StreamsBuilder(); builder .table("t1", Consumed.of(...)) .filter(predicate, Materialized.as("t2")) .<downStreamOps> ``` If `downStreamOps` result in a call to `enableSendingOldValues` on the table returned by the `filter` call, i.e. `t2`, then it will result in `t1` being materialized unnecessarily. This ticket was raised off the back of [comments in a PR|[https://github.com/apache/kafka/pull/9156#discussion_r490152263]|https://github.com/apache/kafka/pull/9156#discussion_r490152263] while working on https://issues.apache.org/jira/browse/KAFKA-10077. was: Calling `enableSendingOldValues` on many nodes, e.g. `KTableFilter`, is unnecessarily calling `enableSendingOldValues` on the parent, even when the processor itself is materialized. This can force the parent table to be materialized unnecessarily. For example: ``` StreamsBuilder builder = new StreamsBuilder(); builder .table("t1", Consumed.of(...)) .filter(predicate, Materialized.as("t2")) .<downStreamOps> ``` If `downStreamOps` result in a call to `enableSendingOldValues` on the table returned by the `filter` call, i.e. `t2`, then it will result in `t1` being materialized unnecessarily. This ticket was raised off the back of [comments in a PR](https://github.com/apache/kafka/pull/9156#discussion_r490152263) while working on https://issues.apache.org/jira/browse/KAFKA-10077. > Streams: enableSendingOldValues should not call parent if node is itself > materialized > ------------------------------------------------------------------------------------- > > Key: KAFKA-10494 > URL: https://issues.apache.org/jira/browse/KAFKA-10494 > Project: Kafka > Issue Type: Improvement > Components: streams > Reporter: Andy Coates > Priority: Major > > Calling `enableSendingOldValues` on many nodes, e.g. `KTableFilter`, is > unnecessarily calling `enableSendingOldValues` on the parent, even when the > processor itself is materialized. This can force the parent table to be > materialized unnecessarily. > > For example: > ``` > StreamsBuilder builder = new StreamsBuilder(); > builder > .table("t1", Consumed.of(...)) > .filter(predicate, Materialized.as("t2")) > .<downStreamOps> > ``` > If `downStreamOps` result in a call to `enableSendingOldValues` on the table > returned by the `filter` call, i.e. `t2`, then it will result in `t1` being > materialized unnecessarily. > This ticket was raised off the back of [comments in a > PR|[https://github.com/apache/kafka/pull/9156#discussion_r490152263]|https://github.com/apache/kafka/pull/9156#discussion_r490152263] > while working on https://issues.apache.org/jira/browse/KAFKA-10077. -- This message was sent by Atlassian Jira (v8.3.4#803005)