Skip to content

Commit 6ef1ae4

Browse files
committed
Make Consumer.commit(..,async=False) return offset commit results
1 parent 1bcc4c4 commit 6ef1ae4

File tree

2 files changed

+145
-61
lines changed

2 files changed

+145
-61
lines changed

confluent_kafka/src/Consumer.c

Lines changed: 141 additions & 60 deletions
Original file line numberDiff line numberDiff line change
@@ -285,6 +285,84 @@ static PyObject *Consumer_assignment (Handle *self, PyObject *args,
285285
}
286286

287287

288+
/**
289+
* @brief Global offset commit on_commit callback trampoline triggered
290+
* from poll() et.al
291+
*/
292+
static void Consumer_offset_commit_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
293+
rd_kafka_topic_partition_list_t *c_parts,
294+
void *opaque) {
295+
Handle *self = opaque;
296+
PyObject *parts, *k_err, *args, *result;
297+
CallState *cs;
298+
299+
if (!self->u.Consumer.on_commit)
300+
return;
301+
302+
cs = CallState_get(self);
303+
304+
/* Insantiate error object */
305+
k_err = KafkaError_new_or_None(err, NULL);
306+
307+
/* Construct list of TopicPartition based on 'c_parts' */
308+
if (c_parts)
309+
parts = c_parts_to_py(c_parts);
310+
else
311+
parts = PyList_New(0);
312+
313+
args = Py_BuildValue("(OO)", k_err, parts);
314+
315+
Py_DECREF(k_err);
316+
Py_DECREF(parts);
317+
318+
if (!args) {
319+
cfl_PyErr_Format(RD_KAFKA_RESP_ERR__FAIL,
320+
"Unable to build callback args");
321+
CallState_crash(cs);
322+
CallState_resume(cs);
323+
return;
324+
}
325+
326+
result = PyObject_CallObject(self->u.Consumer.on_commit, args);
327+
328+
Py_DECREF(args);
329+
330+
if (result)
331+
Py_DECREF(result);
332+
else {
333+
CallState_crash(cs);
334+
rd_kafka_yield(rk);
335+
}
336+
337+
CallState_resume(cs);
338+
}
339+
340+
/**
341+
* @brief Simple struct to pass results from commit from offset_commit_return_cb
342+
* back to offset_commit() return value.
343+
*/
344+
struct commit_return {
345+
rd_kafka_resp_err_t err;
346+
rd_kafka_topic_partition_list_t *c_parts;
347+
};
348+
349+
/**
350+
* @brief Simple offset_commit_cb to pass the callback information
351+
* as return value from commit() through the commit_return struct.
352+
* Triggered from rd_kafka_commit_queue().
353+
*/
354+
static void
355+
Consumer_offset_commit_return_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
356+
rd_kafka_topic_partition_list_t *c_parts,
357+
void *opaque) {
358+
struct commit_return *commit_return = opaque;
359+
360+
commit_return->err = err;
361+
if (c_parts)
362+
commit_return->c_parts =
363+
rd_kafka_topic_partition_list_copy(c_parts);
364+
}
365+
288366

289367
static PyObject *Consumer_commit (Handle *self, PyObject *args,
290368
PyObject *kwargs) {
@@ -294,6 +372,9 @@ static PyObject *Consumer_commit (Handle *self, PyObject *args,
294372
rd_kafka_topic_partition_list_t *c_offsets;
295373
int async = 1;
296374
static char *kws[] = { "message", "offsets", "async",NULL };
375+
rd_kafka_queue_t *rkqu = NULL;
376+
struct commit_return commit_return;
377+
PyThreadState *thread_state;
297378

298379
if (!self->rk) {
299380
PyErr_SetString(PyExc_RuntimeError,
@@ -342,28 +423,73 @@ static PyObject *Consumer_commit (Handle *self, PyObject *args,
342423
c_offsets = NULL;
343424
}
344425

345-
346-
err = rd_kafka_commit(self->rk, c_offsets, async);
426+
if (async) {
427+
/* Async mode: Use consumer queue for offset commit callback,
428+
* served by consumer_poll() */
429+
rkqu = rd_kafka_queue_get_consumer(self->rk);
347430

348-
if (c_offsets)
349-
rd_kafka_topic_partition_list_destroy(c_offsets);
431+
} else {
432+
/* Sync mode: Let commit_queue() trigger the callback. */
433+
memset(&commit_return, 0, sizeof(commit_return));
350434

435+
/* Unlock GIL while we are blocking. */
436+
thread_state = PyEval_SaveThread();
437+
}
351438

439+
err = rd_kafka_commit_queue(self->rk, c_offsets, rkqu,
440+
async ?
441+
Consumer_offset_commit_cb :
442+
Consumer_offset_commit_return_cb,
443+
async ?
444+
(void *)self : (void *)&commit_return);
352445

353-
if (err) {
354-
cfl_PyErr_Format(err,
355-
"Commit failed: %s", rd_kafka_err2str(err));
356-
return NULL;
357-
}
446+
if (c_offsets)
447+
rd_kafka_topic_partition_list_destroy(c_offsets);
358448

359-
Py_RETURN_NONE;
449+
if (async) {
450+
/* Loose reference to consumer queue */
451+
rd_kafka_queue_destroy(rkqu);
452+
453+
} else {
454+
/* Re-lock GIL */
455+
PyEval_RestoreThread(thread_state);
456+
457+
/* Honour inner error (richer) from offset_commit_return_cb */
458+
if (commit_return.err)
459+
err = commit_return.err;
460+
}
461+
462+
if (err) {
463+
/* Outer error from commit_queue() */
464+
if (!async && commit_return.c_parts)
465+
rd_kafka_topic_partition_list_destroy(commit_return.c_parts);
466+
467+
cfl_PyErr_Format(err,
468+
"Commit failed: %s", rd_kafka_err2str(err));
469+
return NULL;
470+
}
471+
472+
if (async) {
473+
/* async commit returns None when commit is in progress */
474+
Py_RETURN_NONE;
475+
476+
} else {
477+
PyObject *plist;
478+
479+
/* sync commit returns the topic,partition,offset,err list */
480+
assert(commit_return.c_parts);
481+
482+
plist = c_parts_to_py(commit_return.c_parts);
483+
rd_kafka_topic_partition_list_destroy(commit_return.c_parts);
484+
485+
return plist;
486+
}
360487
}
361488

362489

363490

364491
static PyObject *Consumer_store_offsets (Handle *self, PyObject *args,
365492
PyObject *kwargs) {
366-
367493
#if RD_KAFKA_VERSION < 0x000b0000
368494
PyErr_Format(PyExc_NotImplementedError,
369495
"Consumer store_offsets require "
@@ -752,8 +878,10 @@ static PyMethodDef Consumer_methods[] = {
752878
"\n"
753879
" :param confluent_kafka.Message message: Commit message's offset+1.\n"
754880
" :param list(TopicPartition) offsets: List of topic+partitions+offsets to commit.\n"
755-
" :param bool async: Asynchronous commit, return immediately.\n"
756-
" :rtype: None\n"
881+
" :param bool async: Asynchronous commit, return None immediately. "
882+
"If False the commit() call will block until the commit succeeds or "
883+
"fails and the committed offsets will be returned (on success). Note that specific partitions may have failed and the .err field of each partition will need to be checked for success.\n"
884+
" :rtype: None|list(TopicPartition)\n"
757885
" :raises: KafkaException\n"
758886
" :raises: RuntimeError if called on a closed consumer\n"
759887
"\n"
@@ -888,53 +1016,6 @@ static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
8881016
}
8891017

8901018

891-
static void Consumer_offset_commit_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
892-
rd_kafka_topic_partition_list_t *c_parts,
893-
void *opaque) {
894-
Handle *self = opaque;
895-
PyObject *parts, *k_err, *args, *result;
896-
CallState *cs;
897-
898-
if (!self->u.Consumer.on_commit)
899-
return;
900-
901-
cs = CallState_get(self);
902-
903-
/* Insantiate error object */
904-
k_err = KafkaError_new_or_None(err, NULL);
905-
906-
/* Construct list of TopicPartition based on 'c_parts' */
907-
if (c_parts)
908-
parts = c_parts_to_py(c_parts);
909-
else
910-
parts = PyList_New(0);
911-
912-
args = Py_BuildValue("(OO)", k_err, parts);
913-
914-
Py_DECREF(k_err);
915-
Py_DECREF(parts);
916-
917-
if (!args) {
918-
cfl_PyErr_Format(RD_KAFKA_RESP_ERR__FAIL,
919-
"Unable to build callback args");
920-
CallState_crash(cs);
921-
CallState_resume(cs);
922-
return;
923-
}
924-
925-
result = PyObject_CallObject(self->u.Consumer.on_commit, args);
926-
927-
Py_DECREF(args);
928-
929-
if (result)
930-
Py_DECREF(result);
931-
else {
932-
CallState_crash(cs);
933-
rd_kafka_yield(rk);
934-
}
935-
936-
CallState_resume(cs);
937-
}
9381019

9391020

9401021

examples/integration_test.py

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -451,7 +451,10 @@ def verify_consumer():
451451
# Async commit
452452
c.commit(msg, async=True)
453453
elif (msg.offset() % 4) == 0:
454-
c.commit(msg, async=False)
454+
offsets = c.commit(msg, async=False)
455+
assert len(offsets) == 1, 'expected 1 offset, not %s' % (offsets)
456+
assert offsets[0].offset == msg.offset()+1, 'expected offset %d to be committed, not %s' % (msg.offset(), offsets)
457+
print('Sync committed offset: %s' % offsets)
455458

456459
msgcnt += 1
457460
if msgcnt >= max_msgcnt:

0 commit comments

Comments
 (0)
pFad - Phonifier reborn

Pfad - The Proxy pFad of © 2024 Garber Painting. All rights reserved.

Note: This service is not intended for secure transactions such as banking, social media, email, or purchasing. Use at your own risk. We assume no liability whatsoever for broken pages.


Alternative Proxies:

Alternative Proxy

pFad Proxy

pFad v3 Proxy

pFad v4 Proxy