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.retention.X`
do not apply. Those setting only apply if cleanup.policy is set to "delete".

The size of a compacted topic depends on the number of unique keys --
there will be one message per key -- if newer message with the same key
are written, older message with this key can be garbage collected. If a
message is never updated (and not explicitly deleted with a tomstone
records, ie, a record with null-value), the record will never be deleted
for a compacted topic.


-Matthias

On 4/6/18 2:10 PM, Björn Häuser wrote:
> 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.retention.hours=720
> 
> Though one of internal kafka stream topics (with cleanup.policy = compact) 
> grew to ~40gb today. 
> 
> What am I missing? I thought these settings should take care that a given 
> topic is never >15 gb, right?
> 
> Thanks
> Björn
> 
>> On 29. Mar 2018, at 00:05, Guozhang Wang <wangg...@gmail.com> wrote:
>>
>> 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 <bjoernhaeu...@gmail.com>
>> wrote:
>>
>>> 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 retention and went ahead and set it
>>> retention.bytes to 15gb. Unfortunately this does not seem to apply and the
>>> topic size is still around 140gb.
>>>
>>> Is this intended? I could not find any documentation about setting the
>>> retention size for the internal topics.
>>>
>>> Thanks
>>> Björn
>>>
>>>
>>>
>>
>>
>> -- 
>> -- Guozhang
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to