Skip to content

Commit

Permalink
Changelog update for latest transactional updates
Browse files Browse the repository at this point in the history
  • Loading branch information
edenhill committed Jan 25, 2021
1 parent 157d7bf commit 7d5ae63
Showing 1 changed file with 27 additions and 0 deletions.
27 changes: 27 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ librdkafka v1.6.0 is feature release:
* [KIP-447 Producer scalability for exactly once semantics](https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+scalability+for+exactly+once+semantics) -
allows a single transactional producer to be used for multiple input
partitions. Requires Apache Kafka 2.5 or later.
* Transactional producer fixes and improvements, see **Transactional Producer fixes** below.
* The [librdkafka.redist](https://www.nuget.org/packages/librdkafka.redist/)
NuGet package now supports Linux ARM64/Aarch64.

Expand All @@ -29,6 +30,17 @@ librdkafka v1.6.0 is feature release:
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.
* Error handling for the transactional producer has been improved, see
the **Transactional Producer fixes** below for more information.


## Known issues

* The Transactional Producer's API timeout handling is inconsistent with the
underlying protocol requests, it is therefore strongly recommended that
applications call `rd_kafka_commit_transaction()` and
`rd_kafka_abort_transaction()` with the `timeout_ms` parameter
set to `-1`, which will use the remaining transaction timeout.


## Enhancements
Expand Down Expand Up @@ -80,6 +92,21 @@ librdkafka v1.6.0 is feature release:
connection is down, which was not previously the case.


### Transactional Producer fixes

* Transaction commit or abort failures on the broker, such as when the
producer was fenced by a newer instance, were not propagated to the
application resulting in failed commits seeming successful.
This was a critical race condition for applications that had a delay after
producing messages (or sendings offsets) before committing or
aborting the transaction. This issue has now been fixed and test coverage
improved.
* The transactional producer API would return `RD_KAFKA_RESP_ERR__STATE`
when API calls were attempted after the transaction had failed, we now
try to return the error that caused the transaction to fail in the first
place, such as `RD_KAFKA_RESP_ERR__FENCED` when the producer has
been fenced, or `RD_KAFKA_RESP_ERR__TIMED_OUT` when the transaction
has timed out.
* Transactional producer retry count for transactional control protocol
requests has been increased from 3 to infinite, retriable errors
are now automatically retried by the producer until success or the
Expand Down

0 comments on commit 7d5ae63

Please sign in to comment.