Re: Flink-Kafka exactly once errors even with transaction.timeout.ms set

2020-11-12 Thread Arvid Heise
Hi Tim, afaik we are confusing two things here, there is a transaction timeout = how long the transaction lasts until aborted. And what you see here is some timeout while creating the transaction in the first place. A quick google search turned up [1], from which I'd infer that you need to set

Re: Flink-Kafka exactly once errors even with transaction.timeout.ms set

2020-11-12 Thread Tim Josefsson
To further add to this problem, I've now got our ops team to set transaction.max.timeout.ms on our Kafka brokers to 1 hour (as suggested by the Flink docs). However the problem persists and I'm still getting the same error message. I've confirmed that this config setting is actually set on the

Re: Flink-Kafka exactly once errors even with transaction.timeout.ms set

2020-11-12 Thread Tim Josefsson
Also realized I had a typo in the config dump I did in the previous email (the one from the 10th). If I don't do Properties producerProps = new Properties(); producerProps.setProperty("transaction.timeout.ms", "90"); Then the value reported from the ProducerConfig is 360 and not 6 as I

Re: Flink-Kafka exactly once errors even with transaction.timeout.ms set

2020-11-12 Thread Tim Josefsson
Sure, I've attached it to this email. The process seems to restart once the TimeoutException happens so it's repeated a couple of times. Thanks for looking at it! /Tim On Wed, 11 Nov 2020 at 10:37, Aljoscha Krettek wrote: > Hmm, could you please post the full stack trace that leads to the >

Re: Flink-Kafka exactly once errors even with transaction.timeout.ms set

2020-11-11 Thread Aljoscha Krettek
Hmm, could you please post the full stack trace that leads to the TimeoutException? Best, Aljoscha On 10.11.20 17:54, Tim Josefsson wrote: Hey Aljoscha, I'm setting the transaction.timeout.ms when I create the FlinkKafkaProducer: I create a Properties object and then set the property and

Re: Flink-Kafka exactly once errors even with transaction.timeout.ms set

2020-11-10 Thread Tim Josefsson
Hey Aljoscha, I'm setting the transaction.timeout.ms when I create the FlinkKafkaProducer: I create a Properties object and then set the property and finally add those properties when creating the producer. Properties producerProps = new Properties();

Re: Flink-Kafka exactly once errors even with transaction.timeout.ms set

2020-11-10 Thread Aljoscha Krettek
On 10.11.20 11:53, Tim Josefsson wrote: Also when checking my logs I see the following message: 11:41:56,345 INFO org.apache.kafka.clients.producer.ProducerConfig - ProducerConfig values: acks = 1 [omitted for brevity] transaction.timeout.ms = 90 transactional.id =