Store offset commit metadata when calling rd_kafka_offsets_store#4084
Conversation
|
Hey Magnus Edenhill (@edenhill) or Ismael Juma (@ijuma), |
|
Hey folks, |
There's no CHANGELOG yet for the release.
Milind L (milindl)
left a comment
There was a problem hiding this comment.
Looks good to me, besides the tiny issue with the test. I'd like Emanuele Sabellico (@emasab) to weigh in.
We should add back the CHANGELOG, but under the next version, (we can use 2.1.0, or just call it vNext or something).
| "Retrieving committed offsets to verify committed offset " | ||
| "metadata\n"); | ||
| rd_kafka_topic_partition_list_t *committed_toppar; | ||
| committed_toppar = rd_kafka_topic_partition_list_new(1); |
There was a problem hiding this comment.
This needs to be destroyed at the end of the test, rd_kafka_topic_partition_list_destroy(committed_toppar);
|
Mathis (@mathispesch) Thanks for the contribution! For security reason currently it's not allowed to run SemaphoreCI on forked PRs. I merge it into an internal branch and open a new PR. If you want to do additional changes before merging to master please open a new PR based on that branch. |
5458230
into
confluentinc:feature/store-offsets-metadata
|
Mathis (@mathispesch) Emanuele Sabellico (@emasab) should it also store metadata when using |
Fixes #3927.
Offset commit metadata can be used to send additional data to the broker when making an offset commit. See this doc from the Java consumer.
Librdkafka currently sends the offset commit metadata when the commit is requested by calling
rd_kafka_commit_queuebut not when callingrd_kafka_offsets_storeto use the auto-commit. This PR fixes that by storing the offset commit metadata when callingrd_kafka_offsets_store.