Kafka Streams LEFT JOIN multiple tables

2024-01-24 Thread Karsten Stöckmann
Hi, we're currently in the process of evaluating Debezium and Kafka as a CDC system for our Postgres database in order to build an indexing solution (i.e. Solr or OpenSearch). Debezium captures changes per table and propagates them into dedicated Kafka topics each. The ingested tables originally

Re: Kafka Streams LEFT JOIN multiple tables

2024-01-24 Thread Karsten Stöckmann
hah1234/the-hidden-complexity-of-kafka-streams-dsl-solving-scaling-issues-8620fd344a6d > > > article. > > Thanks, > Dharin > > On Wed, Jan 24, 2024 at 12:55 PM Karsten Stöckmann < > karsten.stoeckm...@gmail.com> wrote: > > > > Hi, > > > > we'

Re-key by multiple properties without composite key

2024-01-28 Thread Karsten Stöckmann
Hi all, just stumbled upon another Kafka Streams issue that keeps me busy these days. Assume a (simplified) class A like this: class A { private Long id; private String someContent; private Long fk1; private Long fk2; // Getters and setters accordingly } Both fk1 and fk2 poi

Re: Re-key by multiple properties without composite key

2024-01-30 Thread Karsten Stöckmann
2k). > > Your last step seems to join fk1-fk2 -- is this on purpose? I guess it's > unclear what you try to actually do to begin with? It sound like it's > overall a self-join of the input topic on fk1 and fk2 ? > > > -Matthias > > On 1/28/24 2:24 AM, Karsten Stöck

Re: Re-key by multiple properties without composite key

2024-02-01 Thread Karsten Stöckmann
> Think of `folderTable` as fact-table and `personTable` as dimension table. > > > KS will take care of everything else under the hood automatically. > > > -Matthias > > On 1/30/24 11:25 AM, Karsten Stöckmann wrote: > > Matthias, thanks for getting back on this

Re: Re-key by multiple properties without composite key

2024-02-07 Thread Karsten Stöckmann
ectKey()` nor a `repartition()` as explained in my > previous reply. However, doing a `table.groupBy(...)` will set a new key > and repartition the data to your needs. > > > -Matthias > > > On 2/1/24 1:12 AM, Karsten Stöckmann wrote: > > Thanks so much for takin

Re: Re-key by multiple properties without composite key

2024-02-07 Thread Karsten Stöckmann
gt; > Of course, there might also be ways to play with configs to cut down on > latency to some extend, if e2e latency is your main concern. Again, I > don't know the use case: for many case, sub-second latency is actually > sufficient. > > HTH. > > -Matthias > > On 2/

Kafka Streams output topic configuration (specifically: Quarkus)

2024-02-14 Thread Karsten Stöckmann
Hi, is anyone here familiar with Quarkus Kafka Streams applications? If so - is there a way to control output topic configuration when streaming aggregate data into a sink like so: KTable aggregate = ...; aggregate.toStream().to("topic", ); -> Can I programmatically (or by application config) de

Kafka Streams: understanding re-key operations for joins

2024-02-22 Thread Karsten Stöckmann
Hi, I am observing somewhat unexpected (from my point of view) behaviour while ke-key / re-partitioning operations in order to prepare a KTable-KTable join. Assume two (simplified) source data structures from two respective topics: class User { Long id; // PK String name; } class Attribute

Re: Kafka Streams: understanding re-key operations for joins

2024-02-23 Thread Karsten Stöckmann
Kharpude > > On Fri, Feb 23, 2024 at 1:14 PM Karsten Stöckmann < > karsten.stoeckm...@gmail.com> wrote: > > > Hi, > > > > I am observing somewhat unexpected (from my point of view) behaviour > > while ke-key / re-partitioning operations in order to pr

Messages disappearing from Kafka Streams topology

2024-03-25 Thread Karsten Stöckmann
Hi, are there circumstances that might lead to messages silently (i.e. without any logged warnings or errors) disappearing from a topology? Specifically, I've got a rather simple topology doing a series of FK left joins and notice severe message loss in case the application is fired up for the fi

Re: Messages disappearing from Kafka Streams topology

2024-03-26 Thread Karsten Stöckmann
another table snapshot to the input topic, things seem (!) to work as expected... Best wishes, Karsten Bruno Cadonna schrieb am Mo., 25. März 2024, 17:01: > Hi, > > That sounds worrisome! > > Could you please provide us with a minimal example that shows the issue > you describe

Re: Messages disappearing from Kafka Streams topology

2024-03-26 Thread Karsten Stöckmann
rializing the records from > `folderTopicName`. You mentioned that it happens only when you start > processing and the other table snapshot works fine. This gives me a feeling > that the first records in the topic might not be deserialized properly. > > Regards, > Mangat > >

Re: Messages disappearing from Kafka Streams topology

2024-03-27 Thread Karsten Stöckmann
p will commit in the source topics. If there is no errors, then it should > be persisted eventually. > > However, overall exactly-once provides a much tighter and better commit > control. If you don't have scaling issue, I will strongly advise you to use > EOS. > > Thanks,

Re: Messages disappearing from Kafka Streams topology

2024-03-28 Thread Karsten Stöckmann
the favour of exactly-once. > > Do let me know if that solves your problem. I am curious. if yes, then I > would ask you to create an issue. > > Regards, > Mangat > > On Wed, Mar 27, 2024 at 10:49 AM Karsten Stöckmann < > karsten.stoeckm...@gmail.com> wrote: &g

Re: Messages disappearing from Kafka Streams topology

2024-04-10 Thread Karsten Stöckmann
reduce the StatefulSet to only one replica, the changelog topics as well as the eventual output topic show the expected message count. So - can the issue somehow be related to rebalancing? Best wishes Karsten Am Do., 28. März 2024 um 08:25 Uhr schrieb Karsten Stöckmann : > > Hi Mangat, &

Implications and root causes for UNSTABLE_OFFSET_COMMIT

2025-05-27 Thread Karsten Stöckmann
Hi there, our Kafka Streams applications occasionally show messages like the following: [AdminClient clientId=prod.index-document-aggregator-0-admin] Skipping return offset for prod.document-20 due to error UNSTABLE_OFFSET_COMMIT. As far as I understand, this originates from Kafka brokers rathe