Re: [DISCUSS] Add async backpressure support to FlinkKafkaProducer

2021-05-05 Thread Piotr Nowojski
no problem and I'm glad that it's solving your problem :) Piotrek śr., 5 maj 2021 o 08:58 Wenhao Ji napisał(a): > Thanks Piotr for your reply! > It is a nice solution! By restricting the buffer using these properties, I > think maxConcurrentRequests attribute is indeed not necessary anymore. >

Re: [DISCUSS] Add async backpressure support to FlinkKafkaProducer

2021-05-04 Thread Wenhao Ji
Thanks Piotr for your reply! It is a nice solution! By restricting the buffer using these properties, I think maxConcurrentRequests attribute is indeed not necessary anymore. On Tue, May 4, 2021 at 11:52 PM Piotr Nowojski wrote: > Hi Wenhao, > > As far as I know this is different compared to FLI

Re: [DISCUSS] Add async backpressure support to FlinkKafkaProducer

2021-05-04 Thread Piotr Nowojski
Hi Wenhao, As far as I know this is different compared to FLINK-9083, as KafkaProducer itself can back pressure writes if internal buffers are exhausted [1]. > The buffer.memory controls the total amount of memory available to the producer for buffering. If records are sent faster than they can b

[DISCUSS] Add async backpressure support to FlinkKafkaProducer

2021-04-23 Thread Wenhao Ji
Hi everyone, I recently came across the following exception when dealing with a job failure, which uses the Flink as its sink. ``` org.apache.flink.streaming.connectors.kafka.FlinkKafkaException: Failed to send data to Kafka: Expiring #N record(s) for TOPIC-PARTITION:#N ms has passed since batch