Thanks for letting me know. From what you said it seems a bug in the Kafka Streams, would you like to create JIRA with your topology code? I'll start the investigation right away.
Guozhang On Wed, Feb 3, 2016 at 3:45 PM, Tom Dearman <tom.dear...@gmail.com> wrote: > I built trunk and pulled source yesterday. There are two topics created one > called [store-name] the other as you describe > [Job-Id]-[Store-name]-"changelog". > Only the one called [store-name] contained the store state, the other topic > contained nothing so to get this to work I had to compile the code with the > restore from [store-name]. > On Wed, 3 Feb 2016, 18:50 Guozhang Wang <wangg...@gmail.com> wrote: > > > Hello Tom, > > > > Thanks for trying out Kafka Streams. A quick question: are you using > Kafka > > Streams built from trunk? > > > > In the latest trunk we changed changelog topic name to be the store-name > + > > "-changelog" suffix. So when your processing job starts the topology > > created state should have the topic name as > > [Job-Id]-[Store-name]-"changelog". > > > > Guozhang > > > > On Wed, Feb 3, 2016 at 7:33 AM, Tom Dearman <tom.dear...@gmail.com> > wrote: > > > > > I am trying to use kafka-streams (I know it isn’t released yet but > seems > > > to be fairly far on in its implementation and I would like to use that > > > rather than samza for this project as my project doesn’t need to go > live > > > yet). My set up has a single source, producer, in memory state store > and > > > sink. The state store is called local-state. I can see that there is a > > > kafka topic created which does contain my state and the topic name is > the > > > same as the state store name. When I stop and restart the kafka stream > > > code, it restores the state from a topic with a name from a call to > > > storeChangelogTopic(jobId, storeName), there is a topic that exists > with > > > this name but it doesn’t have the state in it, the state is in a topic > > > called storeName. If I change the code so it restores from changeName > it > > > works fine. Am I doing something wrong and if so any pointers as to > > what I > > > should do. > > > > > > Thanks. > > > > > > > > > > -- > > -- Guozhang > > > -- -- Guozhang