Desai | Senior Software Developer | *ude...@itrsgroup.com*
>
> *www.itrsgroup.com* <https://www.itrsgroup.com/>
> <https://www.itrsgroup.com/>
>
> *From: *Guozhang Wang
> *Date: *Tuesday, March 30, 2021 at 2:10 PM
> *To: *Users
> *Cc: *Bart Lilje
> *Subject: *Re: Ka
From: *Guozhang Wang
> *Date: *Tuesday, March 30, 2021 at 1:00 PM
> *To: *Users
> *Cc: *Bart Lilje
> *Subject: *Re: Kafka Streams Processor API state stores not restored via
> changelog topics
>
> Hello Upesh,
>
> These are super helpful logs, and I think I'm very
gt; log? This was the first and only record written to the store/changelog.
> Is
> > there anything else we can try to resolve this issue or give us more
> > insight into where this issue could originate from?
> >
> >
> >
> > Thanks,
> > Upesh
> >
> &g
Desai | Senior Software Developer | ude...@itrsgroup.com
www.itrsgroup.com
From: Guozhang Wang
Date: Tuesday, March 30, 2021 at 1:00 PM
To: Users
Cc: Bart Lilje
Subject: Re: Kafka Streams Processor API state stores not restored via
changelog topics
Hello Upesh,
These are super helpful logs
lmost
> instantly.
>
>
>
> Best,
>
> Upesh
>
>
>
> *Upesh Desai***
>
> * | *
>
> Senior Software Developer
>
> |
>
> *ude...@itrsgroup.com *
>
> *www.itrsgroup.com* <https://www.itrsgroup.com/>
>
> <https://www.itrsgroup
Date: Thursday, March 25, 2021 at 6:46 PM
To: users@kafka.apache.org
Cc: Bart Lilje
Subject: Re: Kafka Streams Processor API state stores not restored via
changelog topics
We have not tried running 2.6 brokers and 2.7 client, I will try and get back
to you.
We are not enabling EOS on the streams
instantly.
Best,
Upesh
Upesh Desai | Senior Software Developer | ude...@itrsgroup.com
www.itrsgroup.com
From: Guozhang Wang
Date: Thursday, March 25, 2021 at 6:31 PM
To: Users
Subject: Re: Kafka Streams Processor API state stores not restored via
changelog topics
BTW, yes that indicates the
ppen when we run everything on Kafka version 2.6.
>>
>>
>>
>> Thanks,
>>
>> Upesh
>>
>>
>> Upesh Desai | Senior Software Developer | *ude...@itrsgroup.com*
>>
>> *www.itrsgroup.com* <https://www.itrsgroup.com/>
>> <https:
ww.itrsgroup.com/>
> <https://www.itrsgroup.com/>
>
> *From: *Guozhang Wang
> *Date: *Thursday, March 25, 2021 at 4:01 PM
> *To: *Users
> *Cc: *Bart Lilje
> *Subject: *Re: Kafka Streams Processor API state stores not restored via
> changelog topics
>
>
Subject: Re: Kafka Streams Processor API state stores not restored via
changelog topics
Hello Upesh,
Could you confirm a few more things for me:
1. After you stopped the application, and wiped out the state dir; check if
the corresponding changelog topic has one record indeed at offset 0 ---
this
gt; Upesh Desai | Senior Software Developer | *ude...@itrsgroup.com*
>
> *www.itrsgroup.com* <https://www.itrsgroup.com/>
> <https://www.itrsgroup.com/>
>
> *From: *Guozhang Wang
> *Date: *Wednesday, March 24, 2021 at 1:37 PM
> *To: *Users
> *Cc: *Bart Lilje
> *S
...@itrsgroup.com
www.itrsgroup.com
From: Guozhang Wang
Date: Wednesday, March 24, 2021 at 1:37 PM
To: Users
Cc: Bart Lilje
Subject: Re: Kafka Streams Processor API state stores not restored via
changelog topics
Hello Upesh,
Thanks for the detailed report. I looked through the code and tried to
Hello Upesh,
Thanks for the detailed report. I looked through the code and tried to
reproduce the issue, but so far have not succeeded. I think I may need some
further information from you to help my further investigation.
1) The `bufferedLimitIndex == 0` itself does not necessarily mean there's
Hi all,
Our team think we discovered a bug over the weekend withing the Kafka Streams /
Processor API. We are running 2.7.0.
When configuring a state store backed by a changelog topic with the cleanup
policy configuration set to “compact,delete”:
final StoreBuilder> store = Stores
.keyValueS
14 matches
Mail list logo