Hi Manu, That is a good question, currently if users change their application logic while upgrading, the intermediate state stores may not be valid anymore, and hence the change log. In this case we need to wipe out the invalid internal data and restart from scratch.
We are working on a better re-processing user experience now and once there is a proposal we will present it to the community. How's everything on the Gearpump side? Looking forward to hear from you for any news :) Guozhang On Mon, May 23, 2016 at 8:50 AM, Manu Zhang <owenzhang1...@gmail.com> wrote: > That's one case. Well, if I get it right, the change log's topic name is > bound to applicationId, so what I want to ask is how change log works for > scenarios like application upgrade (with new applicationId, I guess). Does > the system handle that for user ? > > On Mon, May 23, 2016 at 9:21 PM Matthias J. Sax <matth...@confluent.io> > wrote: > > > If I understand correctly, you want to have a non-Kafka-Streams consumer > > to read a change-log topic that was written by a Kafka-Streams > application? > > > > That is certainly possible. Kafka is agnostic to Kafka Stream, ie, all > > topics are regular topic and can be read by any consumer. > > > > > > -Matthias > > > > On 05/23/2016 10:48 AM, Manu Zhang wrote: > > > Thanks Matthias. Is there a way to allow users to read change logs > from a > > > previous application ? > > > > > > On Mon, May 23, 2016 at 3:57 PM Matthias J. Sax <matth...@confluent.io > > > > > wrote: > > > > > >> Hi Manu, > > >> > > >> Yes. If a StreamTask recovers, it will write to the same change log's > > >> topic partition. Log compaction is enable per default for those > topics. > > >> You still might see some duplicates in your output. Currently, Kafka > > >> Streams guarantees at-least-once processing (exactly-once processing > is > > >> on the roadmap though). > > >> > > >> > > >> -Matthias > > >> > > >> On 05/23/2016 06:10 AM, Manu Zhang wrote: > > >>> Hi All, > > >>> > > >>> I'm new to Kafka Streams and have a question on change log. > > >>> If a StreamTask fails and is restarted, will the change log be > written > > to > > >>> the old change log's topic partition ? Is it possible for some > change > > >> log > > >>> topic partition to have duplicate records so that log compaction is > > >>> required ? > > >>> > > >>> Thanks, > > >>> Manu Zhang > > >>> > > >> > > >> > > > > > > > > -- -- Guozhang