Skip to content

Commit

Permalink
Change transaction.timeout.ms default from 60 to 10 s (KIP-447)
Browse files Browse the repository at this point in the history
  • Loading branch information
edenhill committed Dec 23, 2020
1 parent 7d42487 commit a0f7546
Show file tree
Hide file tree
Showing 3 changed files with 10 additions and 3 deletions.
5 changes: 5 additions & 0 deletions CHANGELOG.md
Expand Up @@ -29,6 +29,11 @@ and the sticky consumer group partition assignor.
supported on Apache Kafka 2.5 or later, on earlier releases you will
need to use one producer per input partition for EOS. This limitation
is not enforced by the producer or broker.
* The default for the producer configuration property
`transaction.timeout.ms` has been decreased from 60 to 10 seconds as
recommended by KIP-447. Transactional producer users on older versions of
Apache Kafka (< 2.5) should configure this property to 60 seconds (60000).



## Enhancements
Expand Down
2 changes: 1 addition & 1 deletion CONFIGURATION.md
Expand Up @@ -119,7 +119,7 @@ check.crcs | C | true, false | false
allow.auto.create.topics | C | true, false | false | low | Allow automatic topic creation on the broker when subscribing to or assigning non-existent topics. The broker must also be configured with `auto.create.topics.enable=true` for this configuraiton to take effect. Note: The default value (false) is different from the Java consumer (true). Requires broker version >= 0.11.0.0, for older broker versions only the broker configuration applies. <br>*Type: boolean*
client.rack | * | | | low | A rack identifier for this client. This can be any string value which indicates where this client is physically located. It corresponds with the broker config `broker.rack`. <br>*Type: string*
transactional.id | P | | | high | Enables the transactional producer. The transactional.id is used to identify the same transactional producer instance across process restarts. It allows the producer to guarantee that transactions corresponding to earlier instances of the same producer have been finalized prior to starting any new transactions, and that any zombie instances are fenced off. If no transactional.id is provided, then the producer is limited to idempotent delivery (if enable.idempotence is set). Requires broker version >= 0.11.0. <br>*Type: string*
transaction.timeout.ms | P | 1000 .. 2147483647 | 60000 | medium | The maximum amount of time in milliseconds that the transaction coordinator will wait for a transaction status update from the producer before proactively aborting the ongoing transaction. If this value is larger than the `transaction.max.timeout.ms` setting in the broker, the init_transactions() call will fail with ERR_INVALID_TRANSACTION_TIMEOUT. The transaction timeout automatically adjusts `message.timeout.ms` and `socket.timeout.ms`, unless explicitly configured in which case they must not exceed the transaction timeout (`socket.timeout.ms` must be at least 100ms lower than `transaction.timeout.ms`). This is also the default timeout value if no timeout (-1) is supplied to the transactional API methods. <br>*Type: integer*
transaction.timeout.ms | P | 1000 .. 2147483647 | 10000 | medium | The maximum amount of time in milliseconds that the transaction coordinator will wait for a transaction status update from the producer before proactively aborting the ongoing transaction. If this value is larger than the `transaction.max.timeout.ms` setting in the broker, the init_transactions() call will fail with ERR_INVALID_TRANSACTION_TIMEOUT. The transaction timeout automatically adjusts `message.timeout.ms` and `socket.timeout.ms`, unless explicitly configured in which case they must not exceed the transaction timeout (`socket.timeout.ms` must be at least 100ms lower than `transaction.timeout.ms`). This is also the default timeout value if no timeout (-1) is supplied to the transactional API methods. Users on older versions of Apache Kafka (older than 2.5) are recommended to set this value to 60000. <br>*Type: integer*
enable.idempotence | P | true, false | false | high | When set to `true`, the producer will ensure that messages are successfully produced exactly once and in the original produce order. The following configuration properties are adjusted automatically (if not modified by the user) when idempotence is enabled: `max.in.flight.requests.per.connection=5` (must be less than or equal to 5), `retries=INT32_MAX` (must be greater than 0), `acks=all`, `queuing.strategy=fifo`. Producer instantation will fail if user-supplied configuration is incompatible. <br>*Type: boolean*
enable.gapless.guarantee | P | true, false | false | low | **EXPERIMENTAL**: subject to change or removal. When set to `true`, any error that could result in a gap in the produced message series when a batch of messages fails, will raise a fatal error (ERR__GAPLESS_GUARANTEE) and stop the producer. Messages failing due to `message.timeout.ms` are not covered by this guarantee. Requires `enable.idempotence=true`. <br>*Type: boolean*
queue.buffering.max.messages | P | 1 .. 10000000 | 100000 | high | Maximum number of messages allowed on the producer queue. This queue is shared by all topics and partitions. <br>*Type: integer*
Expand Down
6 changes: 4 additions & 2 deletions src/rdkafka_conf.c
Expand Up @@ -1253,8 +1253,10 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"transaction timeout (`socket.timeout.ms` must be at least 100ms "
"lower than `transaction.timeout.ms`). "
"This is also the default timeout value if no timeout (-1) is "
"supplied to the transactional API methods.",
1000, INT_MAX, 60000 },
"supplied to the transactional API methods. "
"Users on older versions of Apache Kafka (older than 2.5) "
"are recommended to set this value to 60000.",
1000, INT_MAX, 10000 },
{ _RK_GLOBAL|_RK_PRODUCER|_RK_HIGH, "enable.idempotence", _RK_C_BOOL,
_RK(eos.idempotence),
"When set to `true`, the producer will ensure that messages are "
Expand Down

0 comments on commit a0f7546

Please sign in to comment.