> Hi Mangat,
>
> thanks for clarification. So to my knowledge exactly-once is configured
> using the 'processing.guarantee=exactly_once_v2' setting? Is the
> configuration setting 'acks=all' somehow related and would you advise
> setting that as well?
>
> Bes
[Consumer
>
> clientId=kstreams-folder-aggregator-a38397c2-d30a-437e-9817-baa605d49e23-StreamThread-4-consumer,
> groupId=kstreams-folder-aggregator] Error while fetching metadata with
> correlation id 69 :
>
> {kstreams-folder-aggregator-folder-to-agency-subscription-response-top
Hey Karsten,
There could be several reasons this could happen.
1. Did you check the error logs? There are several reasons why the Kafka
stream app may drop incoming messages. Use exactly-once semantics to limit
such cases.
2. Are you sure there was no error when deserializing the records from
`fol
Hey All,
We are using the low level processor API to create kafka stream
applications. Each app has 1 or more in-memory state stores with caching
disabled and changelog enabled. Some of the apps also have global stores.
We noticed from the node metrics (kubernetes) that the stream applications
are
Urko,
You can enable changelog topics for your state store. This will enable the
application to persist the data to a Kafka topic. Next time when
application start, it will first build it's state by using this topic. Are
you using Kstreams or the low-level processor API?
Regards,
Mangat
On Mon,
Hey Lorenzo Rovere,
Consider the case where you want to reprocess all the data. Let's say your
process had a bug. You fixed it and now you want to reprocess everything to
produce the correct output.
Similarly, there can be other use cases for resetting the consumer offsets
and reprocessing the in
e" already. This
> case would still persist even with incremental rebalancing.
>
> I've filed https://issues.apache.org/jira/browse/KAFKA-12693 to summarize
> the situation. Please LMK if that explanation is clear to you.
>
> On Mon, Apr 19, 2021 at 12:58 AM mangat rai wrote:
&g
that was intended
> for a record A, but when it was written interleaving and there’s another
> record B in between, that tombstone would effectively delete record B. The
> key here is that, when we replay the changelogs, we replay it completely
> following offset ordering.
>
>
sor API, then I'm assuming you did your own
> group-by processor right? In that case, the partition key would just be the
> record key when you are sending to the repartition topic.
>
>
> Guozhang
>
>
>
>
> On Thu, Apr 8, 2021 at 9:00 AM mangat rai wrote:
&
partition key for the changelog topic?
Note that we use low-level processor API and don't commit ourselves.
Regards,
Mangat
On Thu, Apr 8, 2021 at 5:37 PM Guozhang Wang wrote:
> Hi Mangat,
>
> Please see my replies inline below.
>
> On Thu, Apr 8, 2021 at 5:34
es, how many records from the changelog was replayed to
> restore the store, and from which offset on the input topic does Streams
> resume processing. You may also consider upgrading to 2.6.x or higher
> version and see if this issue goes away.
>
>
> Guozhang
>
> On Tue, Ap
Hey,
We have the following setup in our infrastructure.
1. Kafka - 2.5.1
2. Apps use kafka streams `org.apache.kafka` version 2.5.1 library
3. Low level processor API is used with *atleast-once* semantics
4. State stores are *in-memory* with *caching disabled* and *changelog
enable
12 matches
Mail list logo