Re: Kafka Producer avoid sending more records when met exception while also keeping performance

2024-03-14 Thread Haruki Okada
Hi. > By setting max.in.flight.requests.per.connection to 1, I'm concerned that this could become a performance bottleneck As Greg pointed out, this is a trade-off between the ordering-guarantee and the throughput. So you should first measure the throughput of

Re: Kafka Producer avoid sending more records when met exception while also keeping performance

2024-03-13 Thread William Lee
Hi Richard, Thanks for replying. > but I close the KafkaProducer inside the send > callback. > ... > Combined with idempotence enabled > and max inflight set to 5 (the maximum for idempotence tracking) it gave me > relatively good performance. Yes, I also find that closing the KafkaProducer

Re: Kafka Producer avoid sending more records when met exception while also keeping performance

2024-03-13 Thread Richard Bosch
Hi WIlliam, I see from your example that you close the kafka producer in the send loop, based on the content of sendException that is used in the callback of the KafkaProducer send. Since your send loop is a different thread than the KafkaProducer uses to send you will encounter race conditions

Re: Kafka Producer avoid sending more records when met exception while also keeping performance

2024-03-11 Thread William Lee
Hi Greg, Thanks for replying. > From your description, it sounds like you want the success/failure of > a callback to influence whether that record (and later records) are > present in the topic. Is this correct? Yes > The solution that you posted does actually write a record that has an >

Re: Kafka Producer avoid sending more records when met exception while also keeping performance

2024-03-11 Thread Greg Harris
Hi William, >From your description, it sounds like you want the success/failure of a callback to influence whether that record (and later records) are present in the topic. Is this correct? The solution that you posted does actually write a record that has an erroneous callback, is that

Re: Kafka Producer avoid sending more records when met exception while also keeping performance

2024-03-11 Thread William Lee
Hi Haruki, Thanks for your answer. > I still don't get why you need this behavior though The reason is I have to ensure message ordering per partition strictly. Once there is an exception in the producer callback, it indicates that the exception is not a retryable exception(from kafka producer's

Re: Kafka Producer avoid sending more records when met exception while also keeping performance

2024-03-11 Thread Haruki Okada
Hi. > I immediately stop sending more new records and stop the kafka producer, but some extra records were still sent I still don't get why you need this behavior though, as long as you set max.in.flight.requests.per.connection to greater than 1, it's impossible to avoid this because

Kafka Producer avoid sending more records when met exception while also keeping performance

2024-03-11 Thread William Lee
Hi all, I am facing a problem when I detect an exception in kafka producer callback, I immediately stop sending more new records and stop the kafka producer, but some extra records were still sent. I found a way to resolve this issue: setting max.in.flight.requests.per.connection to 1 and closing