[ 
https://issues.apache.org/jira/browse/FLINK-27963?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17552970#comment-17552970
 ] 

Dmytro commented on FLINK-27963:
--------------------------------

Hi [~martijnvisser], thank you for being involved! 

I absolutely agree that a Flink message should be produced without exceptions 
otherwise it shouldn't be produced at all. 

My concern is still the same, it must be a robust fault-tolerant system. Not 
all potential issues could be checked/covered before a message is going to be 
produced. Exceptions may (even should) happen. Now they are re-processed 
(almost) indefinitely stopping the whole message processing. It should be 
delegated to system design architectures, at higher lever than Kafka connectors.

>From my prospective of view, I would like to analyze failed messages, skip 
>them if they cannot be re-processed, save non producible messages somewhere 
>for the future analysis,  log errors, post alerts if it is required. I don't 
>see how and where runtime exceptions can be suppressed and managed properly 
>when Kafka connectors throw them categorically. It is why I still think that 
>the best place to manage exceptions is the place where they are coming from. I 
>believe that a possibility to overwrite the call back will be really useful 
>for developers who use Kafka connectors.

Could you please advise how this  Restarting-Failed-Restarting loop could be 
avoided if an exception is happens in the Kafka connector?  

Thank you!

 

> FlinkRuntimeException in KafkaSink causes a Flink job to hang
> -------------------------------------------------------------
>
>                 Key: FLINK-27963
>                 URL: https://issues.apache.org/jira/browse/FLINK-27963
>             Project: Flink
>          Issue Type: Bug
>          Components: Connectors / Kafka
>    Affects Versions: 1.15.0, 1.14.4
>            Reporter: Dmytro
>            Priority: Major
>              Labels: FlinkRuntimeException, KafkaSink
>
> If FlinkRuntimeException occurs in the 
> [KafkaSink|https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/datastream/kafka/#kafka-sink]
>  then the Flink job tries to re-send failed data  again and gets into endless 
> loop "exception->send again"
> *Code sample which throws the FlinkRuntimeException:*
> {code:java}
> int numberOfRows = 1;
>     int rowsPerSecond = 1;
>     DataStream<String> stream = environment.addSource(
>                     new DataGeneratorSource<>(
>                             RandomGenerator.stringGenerator(1050000), // 
> max.message.bytes=1048588
>                             rowsPerSecond,
>                             (long) numberOfRows),
>                     TypeInformation.of(String.class))
>             .setParallelism(1)
>             .name("string-generator");
>     KafkaSinkBuilder<String> builder = KafkaSink.<String>builder()
>             .setBootstrapServers("localhost:9092")
>             .setDeliverGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
>             .setRecordSerializer(
>                     
> KafkaRecordSerializationSchema.builder().setTopic("test.output")
>                             .setValueSerializationSchema(new 
> SimpleStringSchema())
>                             .build());
>     KafkaSink<String> sink = builder.build();
>     stream.sinkTo(sink).setParallelism(1).name("output-producer"); {code}
> *Exception Stack Trace:*
> {code:java}
> 2022-06-02/14:01:45.066/PDT [flink-akka.actor.default-dispatcher-4] INFO 
> output-producer: Writer -> output-producer: Committer (1/1) 
> (a66beca5a05c1c27691f7b94ca6ac025) switched from RUNNING to FAILED on 
> 271b1b90-7d6b-4a34-8116-3de6faa8a9bf @ 127.0.0.1 (dataPort=-1). 
> org.apache.flink.util.FlinkRuntimeException: Failed to send data to Kafka 
> null with FlinkKafkaInternalProducer{transactionalId='null', 
> inTransaction=false, closed=false} at 
> org.apache.flink.connector.kafka.sink.KafkaWriter$WriterCallback.throwException(KafkaWriter.java:440)
>  ~[flink-connector-kafka-1.15.0.jar:1.15.0] at 
> org.apache.flink.connector.kafka.sink.KafkaWriter$WriterCallback.lambda$onCompletion$0(KafkaWriter.java:421)
>  ~[flink-connector-kafka-1.15.0.jar:1.15.0] at 
> org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50)
>  ~[flink-streaming-java-1.15.0.jar:1.15.0] at 
> org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90) 
> ~[flink-streaming-java-1.15.0.jar:1.15.0] at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMailsNonBlocking(MailboxProcessor.java:353)
>  ~[flink-streaming-java-1.15.0.jar:1.15.0] at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:317)
>  ~[flink-streaming-java-1.15.0.jar:1.15.0] at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:201)
>  ~[flink-streaming-java-1.15.0.jar:1.15.0] at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:804)
>  ~[flink-streaming-java-1.15.0.jar:1.15.0] at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:753)
>  ~[flink-streaming-java-1.15.0.jar:1.15.0] at 
> org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:948)
>  ~[flink-runtime-1.15.0.jar:1.15.0] at 
> org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:927) 
> ~[flink-runtime-1.15.0.jar:1.15.0] at 
> org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:741) 
> ~[flink-runtime-1.15.0.jar:1.15.0] at 
> org.apache.flink.runtime.taskmanager.Task.run(Task.java:563) 
> ~[flink-runtime-1.15.0.jar:1.15.0] at java.lang.Thread.run(Thread.java:748) 
> ~[?:1.8.0_292] Caused by: 
> org.apache.kafka.common.errors.RecordTooLargeException: The message is 
> 1050088 bytes when serialized which is larger than 1048576, which is the 
> value of the max.request.size configuration. {code}
> **



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to