Re: Kafka Streams Internal Topic Retention not applied

2018-04-07 Thread Matthias J. Sax
Björn, Couple of answers: >> So, a streams internal topic for aggregation will be of cleanup.policy = >> compact. Yes. (for non-windowed aggregation) However, in your case, you are using a windowed aggregation, and there the policy is "compact,delete". Because for window aggregation, the key s

Re: Kafka Streams Internal Topic Retention not applied

2018-04-07 Thread Björn Häuser
Hello Matthias, thank you very much for your patience. I am still trying to understand the complete picture here. So, a streams internal topic for aggregation will be of cleanup.policy = compact. Which means that while doing aggregation ignoring tombstones records will cause havoc, right? Th

Re: Kafka Streams Internal Topic Retention not applied

2018-04-06 Thread Matthias J. Sax
Björn, broker configs are default config but can be overwritten when a topic is created. And this happens when Kafka Streams creates internal topics. Thus, you need to change the setting Kafka Streams applies when creating topics. Also note: if cleanup.policy = compact, the setting of `log.retent

Re: Kafka Streams Internal Topic Retention not applied

2018-04-06 Thread Björn Häuser
Hello Guozhang thanks. So after reading much more docs I still do not have the complete picture. These are our relevant settings from kafka broker configuration: log.cleanup.policy=delete # set log.retention.bytes to 15 gb log.retention.bytes=16106127360 # set log.retention.hours to 30 days log

Re: Kafka Streams Internal Topic Retention not applied

2018-03-28 Thread Guozhang Wang
Hello, You can set the topic-level configs via the StreamsConfig#topicPrefix(String), please find the following web docs (search for KIP-173): https://kafka.apache.org/documentation/streams/upgrade-guide#streams_api_changes_100 Guozhang On Wed, Mar 28, 2018 at 3:23 AM, Björn Häuser wrote:

Kafka Streams Internal Topic Retention not applied

2018-03-28 Thread Björn Häuser
Hello Everyone, we are running a Kafka Streams Application with does time window aggregates (using kafka 1.0.0). Unfortunately one of the changelog topics is now growing quite a bit in size maxing out the brokers. I did not find any settings in the kafka stream properties to configure retentio