Hi Sachin,

I think Matthias meant that you can chance the compaction configuration 
parameters when you create the topic, but you're right, by default you 
shouldn't need to do anything since the topic will be eventually compacted 
automatically.

Eno

> On 17 Dec 2016, at 08:23, Sachin Mittal <sjmit...@gmail.com> wrote:
> 
> Understood.
> What I understand so far is that by default changelog topics are compacted.
> So why do we need to compact these topics ourselves.
> 
> Won't stream application take care of this.
> 
> Thanks
> Sachin
> 
> On 17 Dec 2016 11:47 a.m., "Matthias J. Sax" <matth...@confluent.io> wrote:
> 
>> If a Streams application is shutdown via KafkaStream#close() it should
>> be able to reuse local store on restart.
>> 
>> As you mention that you application fails, I assume that some exception
>> happens within Streams and you catch this exception with an uncaught
>> exception handler to trigger some shutdown for the overall application.
>> 
>> For this scenario, a local store is most likely not reusable, because
>> the exception prevent a clean flush to disk and Streams does not know
>> what data did get flushed to disk and what not -- thus, the local store
>> is in an "dirty" state and needs to get rebuild from scratch by reading
>> the backing changelog topic.
>> 
>> If you want to speed up this process, you might want to compact the
>> changelog topic before your restart -- this limits the recovery time to
>> be linear in the number of unique keys in you store (compared to the
>> number of update over all keys). How often log compaction is triggered
>> is a broker configuration that you can also set for individual topics.
>> 
>> 
>> -Matthias
>> 
>> On 12/16/16 9:23 PM, Sachin Mittal wrote:
>>> Hi
>>> I had a question.
>>> Say stream application fails. We handle the shutdown gracefully.
>>> We fix the issue and simply restart it without any application reset. So
>> we
>>> don't delete any internal changelog table or local state stores.
>>> So once it is restarted does it create a new internal store by replaying
>>> the changelog messages?
>>> Or does it simply continue from last committed offsets.
>>> 
>>> I am asking this is that some users have complained that it takes kafka a
>>> long time to re-create state store when application is started and I was
>>> wondering what falls in that case.
>>> 
>>> Thanks
>>> Sachin
>>> 
>> 
>> 

Reply via email to