[
https://issues.apache.org/jira/browse/KAFKA-19873?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18038414#comment-18038414
]
Matthias J. Sax commented on KAFKA-19873:
-----------------------------------------
Arvid, I agree to the problem, and I would like to support something like this.
In Kafka Streams, the "there is no new data and time does not advance" issue is
also there...
But that would be an entirely different feature you are asking for, so let's
keep this ticket as-is, as it's goal is something totally different than what
you are asking for. We can of course file a new ticket for what you are asking
for. (I have already ideas how to do this...)
> Add explicit liveness check for transactional producers
> -------------------------------------------------------
>
> Key: KAFKA-19873
> URL: https://issues.apache.org/jira/browse/KAFKA-19873
> Project: Kafka
> Issue Type: Improvement
> Components: clients, producer
> Reporter: Matthias J. Sax
> Priority: Major
> Labels: needs-kip
>
> The producer does not have an explicit liveness check like the consumer,
> which sends periodic heartbeats if it's part of a consumer group. Because
> there is no "producer group" this is fine in general.
> However, for transactional producers, the missing liveness check has quite
> some downsides (for example KAFKA-19853).
> The problem is, that there is only an indirect liveness check via
> `transaction.timeout.ms` config. The purpose of `transaction.timeout.ms` is
> to avoid head-of-line blocking for read-committed consumers though, and it's
> just a side effect that a crashed producer does also hit this timeout
> eventually, too. The transaction timeout by itself, is not a liveness check.
> For the Kafka Streams case in particular, to react to a failed producers more
> quickly, we set an aggressive default transaction timeout of only 10 seconds,
> allowing the broker to abort a transaction quickly, allowing some other
> consumer to fetch offset quickly after a rebalance (otherwise, fetching
> offset is blocked on an open TX – cf
> [KIP-447|https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+scalability+for+exactly+once+semantics]).
> However, in many cases (not limited to Kafka Streams), it is desirable to
> actually allow transaction to take more time, but this implies that the
> producer error detection and failover mechanism gets slowed down. For this
> reason, users are hesitant to increase the transaction timeout, what may fire
> back by getting TX aborted too aggressively causing unwanted errors (it's
> particularly problematic for Kafka Streams, because we can't re-use previous
> `transaction.id` to fence off a pending TX pro-actively, as we moved off
> EOSv1 to EOSv2 implementation).
> Thus, for transactional producers, it would make sense to follow the consumer
> model, which allows for aggressive hard failure detection via
> `session.timeout.ms` plus longer processing loops via `max.poll.interval.ms`
> decoupling liveness check and "max processing" time. – We propose to add a
> new producer `session.timeout.ms` plus a new heartbeat RPC for transactional
> producers. If a tx-producer has a hard failure and stops sending heartbeats
> to the broker side transaction coordinator, the coordinator can abort the TX
> right away without the need to wait for the TX timeout. This allows uses to
> configure a low session timeout in combination with a larger transaction
> timeout, providing swift hard error detection plus longer transaction times.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)