Re: ProducerFencedException when running 2 jobs with FlinkKafkaProducer

2019-02-18 Thread Tzu-Li (Gordon) Tai
Hi, I just saw a JIRA opened for this: https://issues.apache.org/jira/browse/FLINK-11654. The JIRA ticket's description matches what I had in mind and can confirm the bug assessment. Unfortunately, I currently do not have the capacity to provide a fix and test for this. For the meantime, I've mad

Re: ProducerFencedException when running 2 jobs with FlinkKafkaProducer

2019-02-18 Thread Rohan Thimmappa
Hi Tzu-Li, Any updated on this. This is consistently reproducible. Same jar - Separate source topic to Separate destination topic. This sort of blocker for flink upgrada. i tried with 1.7.2 but no luck. org.apache.flink.streaming.connectors.kafka.FlinkKafka011Exception: Failed to send data to

Re: ProducerFencedException when running 2 jobs with FlinkKafkaProducer

2019-02-12 Thread Tzu-Li (Gordon) Tai
Hi, I think this is unexpected. The generated transactional ids should not be clashing. Looking at the FlinkKafkaProducer code, it seems like the generation is only a function of the subtask id of the FlinkKafkaProducer, which could be the same across 2 different Kafka sources. I'm not completely

ProducerFencedException when running 2 jobs with FlinkKafkaProducer

2019-02-12 Thread Slotterback, Chris
Hey all, I am running into an issue where if I run 2 flink jobs (same jar, different configuration), that produce to different kafka topics on the same broker, using the 1.7 FlinkKafkaProducer set with EXACTLY_ONCE semantics, both jobs go into a checkpoint exception loop every 15 seconds or so: