Re: Kafka streams dropping events in join when reading from earliest message on topic

2019-06-20 Thread Matthias J. Sax
> The two streams were read in separately: > instead of together: If you want to join two streams, reading both topic separately sound correct. > There are twenty partitions per topic. It seems as if it is not reading > equally fast from all topic partitions. This should not affect the corre

Re: Kafka streams dropping events in join when reading from earliest message on topic

2019-06-20 Thread John Roesler
Hi! You might also want to set MAX_TASK_IDLE_MS_CONFIG = "max.task.idle.ms" to a non-zero value. This will instruct Streams to wait the configured amount of time to buffer incoming events on all topics before choosing any records to process. In turn, this should cause records to be processed in ro

Re: Kafka streams dropping events in join when reading from earliest message on topic

2019-06-20 Thread giselle . vandongen
It seems like there were multiple issues: 1. The two streams were read in separately: val stream1: KStream[String, String] = builder.stream[String, String](Set("topic1")) val stream2: KStream[String, String] = builder.stream[String, String](Set("topic2")) instead of together: val raw

Re: Kafka streams dropping events in join when reading from earliest message on topic

2019-06-17 Thread Matthias J. Sax
> I verified keys and timestamps and they match. Did you verify the timestamps client side, ie, in your Streams application? > When is the watermark for the grace period advanced? There is nothing like a watermark. Time is tracked on a per-record basis. > the event time is the Kafka log append

Re: Kafka streams dropping events in join when reading from earliest message on topic

2019-06-17 Thread giselle . vandongen
I verified keys and timestamps and they match. If I start the publisher and processor at the same time, the join has entirely correct output with 6000 messages coming in and 3000 coming out. Putting the grace period to a higher value has no effect. When is the watermark for the grace period ad

Re: Kafka streams dropping events in join when reading from earliest message on topic

2019-06-14 Thread Matthias J. Sax
How do you know that the result should be 900,000 messages? Did you verify that the keys match and that the timestamps are correct? Did you try to remove grace-period or set a higher value? Maybe there is an issue with ouf-of-order data? -Matthias On 6/14/19 5:05 AM, giselle.vandon...@klarrio.co

Kafka streams dropping events in join when reading from earliest message on topic

2019-06-14 Thread giselle . vandongen
I have two streams of data flowing into a Kafka cluster. I want to process this data with Kafka streams. The stream producers are started at some time t. I start up the Kafka Streams job 5 minutes later and start reading from earliest from both topics (about 900 000 messages already on each topi