Hi Nick,

Thank you for the KIP!

I agree on points 1, 2, and 3. I am not sure about point 4. I agree that we should update the docs for commit.interval.ms but I am not sure if this needs to mentioned in the KIP. That seems to me more a PR discussion. Also on point 2, I agree that we need to add a doc string but the content should be exemplary not binding. What I want to say is that, we do not need a KIP to change docs.

Here my points:

5. Could you specify in the motivation that the KIP is about deleting records from repartition topics? Maybe with a short description when why and when records are deleted from the repartition topics. For us it might be clear, but IMO we should try to write KIPs so that someone that is relatively new to Kafka Streams can understand the KIP without needing to know too much background.

6. Does the config need to be validated? For example, does delete.interval.ms need to be greater than or equal to commit.interval.ms?

7. Should the default value for non-EOS be 30s or the same value as commit.interval.ms? I am just thinking about the case where a user explicitly changes commit.interval.ms but not delete.interval.ms (or whatever name you come up for it). Once delete.interval.ms is set explicitly it is decoupled from commit.interval.ms. Similar could be done for the EOS case. Alternatively, we could also define delete.interval.ms to take a integral number without a unit that specifies after how many commit intervals the records in repartition topics should be deleted. This would make sense since delete.interval.ms is tightly bound to commit.interval.ms. Additionally, it would make the semantics of the config simpler. The name of the config should definitely change if we go down this way.

Best,
Bruno



On 21.12.21 11:14, Luke Chen wrote:
Hi Nick,

Thanks for the KIP!

In addition to Sophie's comments, I have one more to this KIP:
3. I think you should mention the behavior change *explicitly* in
"Compatibility" section. I know you already mentioned it in KIP, in the
benefit way. But I think in this section, we should clearly point out which
behavior will be change after this KIP. That is, you should put it clear
that the delete record interval will change from 100ms to 30s with EOS
enabled. And it should also be mentioned in doc/upgrade.html doc.
4. Since this new config has some relationship with commit.interval.ms, I
think we should also update the doc description for `commit.interval.ms`,
to let user know there's another config to control delete interval and
should be greater than commit.interval.ms. Something like that. WDYT? (You
should put this change in the KIP as Sophie mentioned)

Thank you.
Luke

On Tue, Dec 21, 2021 at 9:27 AM Sophie Blee-Goldman
<sop...@confluent.io.invalid> wrote:

Hey Nick,

I think you forgot to link to the KIP document, but I take it this is
it: KIP-811:
Add separate delete.interval.ms to Kafka Streams
<https://cwiki.apache.org/confluence/x/JY-kCw>

The overall proposal sounds good to me, just a few minor things:

    1. Please specify everything needed to define this config explicitly, ie
    all the arguments that will be passed in to the
    StreamsConfig's ConfigDef: in addition to the default value, we need the
    config type (presumably a Long), the doc
    string, and the importance (probably "low", similar to
commit.interval.ms
    )
    2. Might be worth considering a slightly more descriptive name for this
    config. Most users probably don't think about,
    or may not even be aware of, the deletion of consumed records by Kafka
    Streams, so calling it something along
    the lines of "repartition.records.delete.interval.ms" or "
    consumed.records.deletion.interval.ms" or so on will help
    make it clear what the config refers to and whether or not they need to
    care. Maybe you can come up with better
    and/or shorter names, just wanted to suggest some example names that I
    think sufficiently get the point across

Other than that I'm +1 -- thanks for the KIP!

Sophie



On Mon, Dec 20, 2021 at 9:15 AM Nick Telford <nick.telf...@gmail.com>
wrote:

This is a KIP for a proposed solution to KAFKA-13549
<https://issues.apache.org/jira/browse/KAFKA-13549>. The solution is
very
simple, so the KIP is pretty short.

The suggested changes are implemented by this PR
<https://github.com/apache/kafka/pull/11610>.
--
Nick Telford



Reply via email to