-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-8040: Streams retry initTransactions on timeout #6372
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -218,11 +218,20 @@ public void onCompletion(final RecordMetadata metadata, | |
| } | ||
| }); | ||
| } catch (final TimeoutException e) { | ||
| log.error("Timeout exception caught when sending record to topic {}. " + | ||
| "This might happen if the producer cannot send data to the Kafka cluster and thus, " + | ||
| "its internal buffer fills up. " + | ||
| "You can increase producer parameter `max.block.ms` to increase this timeout.", topic); | ||
| throw new StreamsException(String.format("%sFailed to send record to topic %s due to timeout.", logPrefix, topic)); | ||
| log.error( | ||
| "Timeout exception caught when sending record to topic {}. " + | ||
| "This might happen if the producer cannot send data to the Kafka cluster and thus, " + | ||
| "its internal buffer fills up. " + | ||
| "This can also happen if the broker is slow to respond, if the network connection to " + | ||
| "the broker was interrupted, or if similar circumstances arise. " + | ||
| "You can increase producer parameter `max.block.ms` to increase this timeout.", | ||
| topic, | ||
| e | ||
| ); | ||
| throw new StreamsException( | ||
| String.format("%sFailed to send record to topic %s due to timeout.", logPrefix, topic), | ||
| e | ||
| ); | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I added the I also added some more failure modes to the log message; I felt the existing message could be misleading if the problem was actually just a network interruption. |
||
| } catch (final Exception uncaughtException) { | ||
| if (uncaughtException instanceof KafkaException && | ||
| uncaughtException.getCause() instanceof ProducerFencedException) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,6 +25,7 @@ | |
| import org.apache.kafka.common.MetricName; | ||
| import org.apache.kafka.common.TopicPartition; | ||
| import org.apache.kafka.common.errors.ProducerFencedException; | ||
| import org.apache.kafka.common.errors.TimeoutException; | ||
| import org.apache.kafka.common.metrics.Sensor; | ||
| import org.apache.kafka.common.metrics.stats.Avg; | ||
| import org.apache.kafka.common.metrics.stats.Count; | ||
|
|
@@ -246,7 +247,7 @@ public StreamTask(final TaskId id, | |
| // initialize transactions if eos is turned on, which will block if the previous transaction has not | ||
| // completed yet; do not start the first transaction until the topology has been initialized later | ||
| if (eosEnabled) { | ||
| this.producer.initTransactions(); | ||
| initializeTransactions(); | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -298,7 +299,7 @@ public void resume() { | |
| throw new IllegalStateException("Task producer should be null."); | ||
| } | ||
| producer = producerSupplier.get(); | ||
| producer.initTransactions(); | ||
| initializeTransactions(); | ||
| recordCollector.init(producer); | ||
|
|
||
| if (stateMgr.checkpoint != null) { | ||
|
|
@@ -872,4 +873,23 @@ RecordCollector recordCollector() { | |
| Producer<byte[], byte[]> getProducer() { | ||
| return producer; | ||
| } | ||
|
|
||
| private void initializeTransactions() { | ||
| try { | ||
| producer.initTransactions(); | ||
| } catch (final TimeoutException retriable) { | ||
| log.error( | ||
| "Timeout exception caught when initializing transactions for task {}. " + | ||
| "This might happen if the broker is slow to respond, if the network connection to " + | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: as above |
||
| "the broker was interrupted, or if similar circumstances arise. " + | ||
| "You can increase producer parameter `max.block.ms` to increase this timeout.", | ||
| id, | ||
| retriable | ||
| ); | ||
| throw new StreamsException( | ||
| format("%sFailed to initialize task %s due to timeout.", logPrefix, id), | ||
| retriable | ||
| ); | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: why one more indent ?
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's just how Idea seems to format multiline expressions... e.g.,
instead of
Is it undesirable?
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not important -- just look funky to me. We concatenate multiple strings and thus all should have the same indent -- why would we indent the first differently? I would format as follows (even if I know that you don't like that the plus goes into the next line):
This avoid the "ambiguity", of multiple string parameters, vs one concatenated parameter:
Thirst and second hard hard to distinguish (where do parameters start/stop), but third makes it clear, that it's two parameters but not one or four, what is hard to tell in the middle case. Of course, double indent also fixes this but it's weird to me:
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I remember a while back someone on the internet trying to get everyone to always put operators (and commas) at the beginning of the line for this reason. I think it makes sense. I don't think it caught on in general because it creates syntactic ambiguity in Javascript, but since Java requires semicolons to end a line, it should be fine.
Do you have your IDE set up to create this formatting? Maybe it sounds lazy, but the reason I've formatted it this way is that that's what IDEA does by default. I don't want to spend time curating the number of indent spaces by hand on every code change. I couldn't figure out how to get rid of the extra indent in the multi-line string concatenation. Eg, it even does this:
which is like the worst outcome.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't know the IDE setting -- this case is rare enough that I "fix" fit manually if it happens.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let me know if you plan to address or ignore this -- I am fine either way.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ok, I agree with you in principle, but I think I'll just leave it as-is, until I can figure out a way to get the IDE to do it for me.