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

Julius Michaelis edited comment on FLINK-18150 at 7/10/20, 2:54 AM:
--------------------------------------------------------------------

For Flink 1.10 / Kafka 2.2.0, the {{default.api.timeout.ms}} / 
{{request.timeout.ms}} works, but that only prevents the initial job recovery 
after the broker failure. If you trigger a job restart for a different reason 
(e.g. {{docker-compose restart taskmanager1}} in the reproducer), it will show 
the same failing behavior unless you set {{max.block.ms}}, too. (My math 
doesn't quite checkout yet. I calculated that I need around 10 tries for it to 
not fail with acceptable probability, but it seems to be good at around 5.)
-For 1.11, I haven't confirmed a good parameter combination yet.- (That was 
with the RC. For the Release, the 10 tries seem to check out.)

Either way, these kinds of workarounds aren't exactly the rock-solid behavior I 
have in mind for our applications. What do you think, should I raise an issue 
(feature request?) on the Kafka bugtracker? I think if I turn up there with 
things like "I have 600 producers running in my application", I might get some 
funny looks...


was (Author: caesar):
For Flink 1.10 / Kafka 2.2.0, the {{default.api.timeout.ms}} / 
{{request.timeout.ms}} works, but that only prevents the initial job recovery 
after the broker failure. If you trigger a job restart for a different reason 
(e.g. {{docker-compose restart taskmanager1}} in the reproducer), it will show 
the same failing behavior unless you set {{max.block.ms}}, too. (My math 
doesn't quite checkout yet. I calculated that I need around 10 tries for it to 
not fail with acceptable probability, but it seems to be good at around 5.)
For 1.11, I haven't confirmed a good parameter combination yet.

Either way, these kinds of workarounds aren't exactly the rock-solid behavior I 
have in mind for our applications. What do you think, should I raise an issue 
(feature request?) on the Kafka bugtracker? I think if I turn up there with 
things like "I have 600 producers running in my application", I might get some 
funny looks...

> A single failing Kafka broker may cause jobs to fail indefinitely with 
> TimeoutException: Timeout expired while fetching topic metadata
> --------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-18150
>                 URL: https://issues.apache.org/jira/browse/FLINK-18150
>             Project: Flink
>          Issue Type: Bug
>          Components: Connectors / Kafka
>    Affects Versions: 1.10.1
>         Environment: It is a bit unclear to me under what circumstances this 
> can be reproduced. I created a "minimum" non-working example at 
> https://github.com/jcaesar/flink-kafka-ha-failure. Note that this deploys the 
> minimum number of Kafka brokers, but it works just as well with replication 
> factor 3 and 8 brokers, e.g.
> I run this with
> {code:bash}
> docker-compose kill; and docker-compose rm -vf; and docker-compose up 
> --abort-on-container-exit --build
> {code}
> The exception should appear on the webui after 5~6 minutes.
> To make sure that this isn't dependent on my machine, I've also checked 
> reproducibility on a m5a.2xlarge EC2 instance.
> You verify that the Kafka cluster is running "normally" e.g. with:
> {code:bash}
> kafkacat -b localhost,localhost:9093 -L
> {code}
> So far, I only know that
> * {{flink.partition-discovery.interval-millis}} must be set.
> * The broker that failed must be part of the {{bootstrap.servers}}
> * There needs to be a certain amount of topics or producers, but I'm unsure 
> which is crucial
> * Changing the values of {{metadata.request.timeout.ms}} or 
> {{flink.partition-discovery.interval-millis}} does not seem to have any 
> effect.
>            Reporter: Julius Michaelis
>            Assignee: Aljoscha Krettek
>            Priority: Major
>
> When a Kafka broker fails that is listed among the bootstrap servers and 
> partition discovery is active, the Flink job reading from that Kafka may 
> enter a failing loop.
> At first, the job seems to react normally without failure with only a short 
> latency spike when switching Kafka leaders.
> Then, it fails with a
> {code:none}
> org.apache.flink.streaming.connectors.kafka.internal.Handover$ClosedException
>         at 
> org.apache.flink.streaming.connectors.kafka.internal.Handover.close(Handover.java:182)
>         at 
> org.apache.flink.streaming.connectors.kafka.internal.KafkaFetcher.cancel(KafkaFetcher.java:175)
>         at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.cancel(FlinkKafkaConsumerBase.java:821)
>         at 
> org.apache.flink.streaming.api.operators.StreamSource.cancel(StreamSource.java:147)
>         at 
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.cancelTask(SourceStreamTask.java:136)
>         at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.cancel(StreamTask.java:602)
>         at 
> org.apache.flink.runtime.taskmanager.Task$TaskCanceler.run(Task.java:1355)
>         at java.lang.Thread.run(Thread.java:748)
> {code}
> It recovers, but processes fewer than the expected amount of records.
> Finally,  the job fails with
> {code:none}
> 2020-06-05 13:59:37
> org.apache.kafka.common.errors.TimeoutException: Timeout expired while 
> fetching topic metadata
> {code}
> and repeats doing so while not processing any records. (The exception comes 
> without any backtrace or otherwise interesting information)
> I have also observed this behavior with partition-discovery turned off, but 
> only when the Flink job failed (after a broker failure) and had to run 
> checkpoint recovery for some other reason.
> Please see the [Environment] description for information on how to reproduce 
> the issue.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to