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 Streams 3.5.1 based app seems to get stalled

2024-03-11 Thread Matthias J. Sax
Without detailed logs (maybe even DEBUG) hard to say. But from what you describe, it could be a metadata issue? Why are you setting METADATA_MAX_AGE_CONFIG (consumer and producer): 5 hours in millis (to make rebalances rare) Refreshing metadata has nothing to do with rebalances, and a

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

Issues while setting up RBAC for Apache Kafka using Ranger

2024-03-11 Thread Karthik Suvarnasa
Hi All, I'm working on setting up RBAC for Apache Kafka using Ranger. Right now, I'm facing an authorization issue while testing the console producer script in Kafka. I need help in properly configuring Kafka with Ranger. Below are the steps I performed. - I successfully installed the ranger