Skip to content

KIP-932 : Implement Share consumer interface with poll API#2217

Open
Kaushik Raina (k-raina) wants to merge 2 commits intodev_kip-932_queues-for-kafkafrom
dev_kip-932_share_consumer_poll
Open

KIP-932 : Implement Share consumer interface with poll API#2217
Kaushik Raina (k-raina) wants to merge 2 commits intodev_kip-932_queues-for-kafkafrom
dev_kip-932_share_consumer_poll

Conversation

@k-raina
Copy link
Member

@k-raina Kaushik Raina (k-raina) commented Mar 12, 2026

Summary

  • Implements ShareConsumer, a new Kafka client type for KIP-932 share group consumption
  • Added New file: src/confluent_kafka/src/ShareConsumer.c
  • ShareConsumerHandle struct inheriting from Handle via first-member embedding — enables safe (Handle *)self casts and reuses all common callback/TLS infrastructure
  • subscribe(topics) / unsubscribe() / subscription() — topic subscription management via rd_kafka_share_* APIs
  • consume_batch(timeout=-1) — batch-only consumption with chunked polling for Ctrl+C interruptibility; uses - CallState_begin/end and check_signals_between_chunks for correct GIL and TLS lifecycle management
  • close() — graceful shutdown: attempts broker close then always destroys handle

Known limitations (TODOs in code)

Item Blocked On
rd_kafka_set_log_queue() Needs a specific rd_kafka_share_set_log_queue() wrapper to handle the share consumer handle.
OAuth Background Callbacks Missing rd_kafka_share_sasl_background_callbacks_enable() export in the C API.
max_poll_records Currently hardcoded to 10005; requires updating to the librdkafka double-pointer API for dynamic config reads.

Additional Changes

  • Additional commit bc6101a to build python binding with kip-932 branch for librdkafka. We can remove this commit in future before merging to master.

Copilot AI review requested due to automatic review settings March 12, 2026 17:24
@confluent-cla-assistant
Copy link

🎉 All Contributor License Agreements have been signed. Ready to merge.
Please push an empty commit if you would like to re-run the checks to verify CLA status for all contributors.

Copy link

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

This PR introduces a new ShareConsumer Python API backed by a C-extension wrapper around librdkafka’s KIP-932 Share Consumer functionality, and wires it into the package distribution and type hints.

Changes:

  • Add a new C-extension type cimpl.ShareConsumer with subscribe(), unsubscribe(), subscription(), consume_batch(), and close().
  • Export ShareConsumer through confluent_kafka.cimpl and confluent_kafka.__init__, and add corresponding type stubs.
  • Add unit tests for the new ShareConsumer API (skipped when ShareConsumer/librdkafka support is unavailable).

Reviewed changes

Copilot reviewed 7 out of 7 changed files in this pull request and generated 5 comments.

Show a summary per file
File Description
tests/test_ShareConsumer.py Adds test coverage for ShareConsumer construction, subscription handling, consume_batch behavior, and close semantics (with skip guard).
src/confluent_kafka/src/ShareConsumer.c Implements the new ShareConsumer C-extension type and its methods.
src/confluent_kafka/src/confluent_kafka.h Declares ShareConsumerType for module initialization.
src/confluent_kafka/src/confluent_kafka.c Registers the ShareConsumer Python type in the cimpl module init.
src/confluent_kafka/cimpl.pyi Adds ShareConsumer type stubs.
src/confluent_kafka/init.py Exposes ShareConsumer at the package top-level and in __all__.
setup.py Adds ShareConsumer.c to the extension sources for building.

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

You can also share your feedback on Copilot code review. Take the survey.

Comment on lines +463 to +468
self->base.type = RD_KAFKA_CONSUMER;

if (!(conf = common_conf_setup(RD_KAFKA_CONSUMER, (Handle *)self, args,
kwargs)))
return -1; /* Exception raised by common_conf_setup() */

Copy link

Copilot AI Mar 12, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ShareConsumer uses common_conf_setup(), which installs shared callbacks (error_cb/throttle_cb/stats_cb/logger/oauth). Those callbacks assume Handle.rk is a valid rd_kafka_t* and call rd_kafka_yield(h->rk) on callback crash/fatal paths. Since ShareConsumer never sets base.rk, a fatal error or a Python callback exception can end up calling rd_kafka_yield(NULL) and crash the process. Fix by either (a) updating the shared callbacks to yield using the rk argument they already receive (or rd_kafka_opaque(rk)), not h->rk, and using rk in oauth_cb as well, or (b) setting base.rk to an actual rd_kafka_t* for share consumers (if/when librdkafka exposes it).

Copilot uses AI. Check for mistakes.
Comment on lines +469 to +471
/* TODO: Remove after interface of librdkafka is updated to return double pointer */
self->max_poll_records = 10005;

Copy link

Copilot AI Mar 12, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

max_poll_records is hardcoded to 10005 and consume_batch() allocates an array based on it on every call. This both diverges from the docstring (which says batch size is controlled by the max.poll.records config) and can cause unnecessary per-call allocation overhead. Suggest reading max.poll.records from the provided config dict during init (with a sane default matching librdkafka/Kafka, e.g. 500), validating bounds, and storing it in self->max_poll_records.

Suggested change
/* TODO: Remove after interface of librdkafka is updated to return double pointer */
self->max_poll_records = 10005;
/* Initialize max_poll_records from config (if provided), else use sane default. */
self->max_poll_records = 500; /* default, aligned with typical max.poll.records */
if (args && PyTuple_Check(args) && PyTuple_Size(args) >= 1) {
PyObject *conf_dict = PyTuple_GetItem(args, 0); /* borrowed ref */
if (conf_dict && PyDict_Check(conf_dict)) {
PyObject *mpr_obj =
PyDict_GetItemString(conf_dict, "max.poll.records"); /* borrowed */
if (mpr_obj) {
long mpr_val = PyLong_AsLong(mpr_obj);
if (mpr_val == -1 && PyErr_Occurred()) {
/* Non-integer or conversion error */
PyErr_SetString(PyExc_TypeError,
"max.poll.records must be an integer");
rd_kafka_conf_destroy(conf);
return -1;
}
/* Validate bounds: require a positive, reasonably bounded value. */
if (mpr_val < 1 || mpr_val > 1000000) {
PyErr_SetString(
PyExc_ValueError,
"max.poll.records must be between 1 and 1000000");
rd_kafka_conf_destroy(conf);
return -1;
}
self->max_poll_records = (size_t)mpr_val;
}
}
}

Copilot uses AI. Check for mistakes.
Comment on lines +349 to +357
CallState_begin((Handle *)self, &cs);
err = rd_kafka_share_consumer_close(self->rkshare);
CallState_end((Handle *)self, &cs);

/* Always destroy handle, to ensures resources are freed even if broker communication fails. */
CallState_begin((Handle *)self, &cs);
rd_kafka_share_destroy(self->rkshare);
CallState_end((Handle *)self, &cs);

Copy link

Copilot AI Mar 12, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ShareConsumer_close() ignores the return value of CallState_end(). If a Python signal is raised or a callback crashes during rd_kafka_share_consumer_close()/rd_kafka_share_destroy(), CallState_end() will return 0 and set an exception, but this code will continue and may mask the error or run with an inconsistent CallState. Mirror Consumer_close() by checking CallState_end() after each CallState_begin()/external call and returning NULL immediately on failure.

Copilot uses AI. Check for mistakes.
Comment on lines +291 to +296
if (!CallState_end((Handle *)self, &cs)) {
free(rkmessages);
if (error)
rd_kafka_error_destroy(error);
return NULL;
}
Copy link

Copilot AI Mar 12, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If CallState_end() fails in consume_batch(), the code frees the rkmessages pointer array (and destroys error), but it does not destroy any messages that may already have been returned into rkmessages (rkmessages_size > 0). This leaks rd_kafka_message_t objects and differs from Consumer_consume(), which destroys messages before returning NULL. Ensure any populated messages are destroyed on this failure path before freeing the array.

Copilot uses AI. Check for mistakes.
" This method should be called to properly clean up the share consumer\n"
" and leave the share group.\n"
"\n"
" :raises RuntimeError: on error\n"
Copy link

Copilot AI Mar 12, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The close() docstring says it raises RuntimeError on error, but the implementation uses cfl_PyErr_Format(err, ...) which raises KafkaException (with the underlying rd_kafka_resp_err_t) on non-zero err. Please align the docstring with actual behavior (or change the exception type if RuntimeError is intended).

Suggested change
" :raises RuntimeError: on error\n"
" :raises KafkaException: on error\n"

Copilot uses AI. Check for mistakes.
@k-raina Kaushik Raina (k-raina) changed the base branch from master to dev_kip-932_queues-for-kafka March 13, 2026 06:13
@airlock-confluentinc airlock-confluentinc bot force-pushed the dev_kip-932_share_consumer_poll branch 3 times, most recently from a7d9785 to bc6101a Compare March 16, 2026 06:12
@airlock-confluentinc airlock-confluentinc bot force-pushed the dev_kip-932_share_consumer_poll branch from bc6101a to fc5ef4f Compare March 16, 2026 07:02
@sonarqube-confluent
Copy link

Quality Gate failed Quality Gate failed

Failed conditions
26.7% Coverage on New Code (required ≥ 80%)

See analysis details on SonarQube

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

First pass comments

rd_kafka_share_t *rkshare;

/* TODO: Remove after interface of librdkafka is updated to return double pointer */
size_t max_poll_records;

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lets name it batch_size so as to not confuse with the actual configuration property max.poll.records

0, /*tp_as_buffer*/
Py_TPFLAGS_DEFAULT | Py_TPFLAGS_BASETYPE |
Py_TPFLAGS_HAVE_GC, /*tp_flags*/
"A high-level Apache Kafka share consumer (KIP-932)\n"

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
"A high-level Apache Kafka share consumer (KIP-932)\n"
"A high-level Apache Kafka share consumer\n"

def subscribe(self, topics: List[str]) -> None: ...
def unsubscribe(self) -> None: ...
def subscription(self) -> List[str]: ...
def consume_batch(self, timeout: float = -1) -> List[Message]: ...

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should name this Poll only to match the signature of the Java Client atleast in Higher level clients

*
*/
static PyObject *
ShareConsumer_consume_batch(ShareConsumerHandle *self,

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reduce the cyclomatic complexity of this function

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants