Skip to content

Commit 77305ff

Browse files
committed
Added (global) on_commit callback.
1 parent 5a579d8 commit 77305ff

File tree

6 files changed

+122
-16
lines changed

6 files changed

+122
-16
lines changed

Consumer.c

Lines changed: 62 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,10 @@ static int Consumer_clear (Consumer *self) {
3737
Py_DECREF(self->on_revoke);
3838
self->on_revoke = NULL;
3939
}
40+
if (self->on_commit) {
41+
Py_DECREF(self->on_commit);
42+
self->on_commit = NULL;
43+
}
4044
return 0;
4145
}
4246

@@ -543,6 +547,18 @@ PyTypeObject ConsumerType = {
543547
"\n"
544548
" Create new Consumer instance using provided configuration dict.\n"
545549
"\n"
550+
" Special configuration properties:\n"
551+
" ``on_commit``: Optional callback will be called when a commit "
552+
"request has succeeded or failed.\n"
553+
"\n"
554+
"\n"
555+
".. py:function:: on_commit(consumer, err, partitions)\n"
556+
"\n"
557+
" :param Consumer consumer: Consumer instance.\n"
558+
" :param KafkaError err: Commit error object, or None on success.\n"
559+
" :param list(TopicPartition) partitions: List of partitions with "
560+
"their committed offsets or per-partition errors.\n"
561+
"\n"
546562
"\n", /*tp_doc*/
547563
(traverseproc)Consumer_traverse, /* tp_traverse */
548564
(inquiry)Consumer_clear, /* tp_clear */
@@ -622,6 +638,51 @@ static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
622638
}
623639

624640

641+
static void Consumer_offset_commit_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
642+
rd_kafka_topic_partition_list_t *c_parts,
643+
void *opaque) {
644+
Consumer *self = opaque;
645+
PyObject *parts, *k_err, *args, *result;
646+
647+
if (!self->on_commit)
648+
return;
649+
650+
PyEval_RestoreThread(self->thread_state);
651+
652+
/* Insantiate error object */
653+
k_err = KafkaError_new_or_None(err, NULL);
654+
655+
/* Construct list of TopicPartition based on 'c_parts' */
656+
parts = c_parts_to_py(c_parts);
657+
658+
args = Py_BuildValue("(OOO)", self, k_err, parts);
659+
660+
Py_DECREF(k_err);
661+
Py_DECREF(parts);
662+
663+
if (!args) {
664+
cfl_PyErr_Format(RD_KAFKA_RESP_ERR__FAIL,
665+
"Unable to build callback args");
666+
self->thread_state = PyEval_SaveThread();
667+
self->callback_crashed++;
668+
return;
669+
}
670+
671+
result = PyObject_CallObject(self->on_commit, args);
672+
673+
Py_DECREF(args);
674+
675+
if (result)
676+
Py_DECREF(result);
677+
else {
678+
self->callback_crashed++;
679+
rd_kafka_yield(rk);
680+
}
681+
682+
self->thread_state = PyEval_SaveThread();
683+
}
684+
685+
625686

626687
static PyObject *Consumer_new (PyTypeObject *type, PyObject *args,
627688
PyObject *kwargs) {
@@ -640,6 +701,7 @@ static PyObject *Consumer_new (PyTypeObject *type, PyObject *args,
640701
}
641702

642703
rd_kafka_conf_set_rebalance_cb(conf, Consumer_rebalance_cb);
704+
rd_kafka_conf_set_offset_commit_cb(conf, Consumer_offset_commit_cb);
643705

644706
self->rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf,
645707
errstr, sizeof(errstr));

confluent_kafka.c

Lines changed: 42 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -966,6 +966,34 @@ static int producer_conf_set_special (Producer *self, rd_kafka_conf_t *conf,
966966
}
967967

968968

969+
/**
970+
* @brief Set single special consumer config value.
971+
*
972+
* @returns 1 if handled, 0 if unknown, or -1 on failure (exception raised).
973+
*/
974+
static int consumer_conf_set_special (Consumer *self, rd_kafka_conf_t *conf,
975+
rd_kafka_topic_conf_t *tconf,
976+
const char *name, PyObject *valobj) {
977+
978+
if (!strcasecmp(name, "on_commit")) {
979+
if (!PyCallable_Check(valobj)) {
980+
cfl_PyErr_Format(
981+
RD_KAFKA_RESP_ERR__INVALID_ARG,
982+
"%s requires a callable "
983+
"object", name);
984+
return -1;
985+
}
986+
987+
self->on_commit = valobj;
988+
Py_INCREF(self->on_commit);
989+
990+
return 1;
991+
}
992+
993+
return 0;
994+
}
995+
996+
969997
/**
970998
* Common config setup for Kafka client handles.
971999
*
@@ -1004,6 +1032,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
10041032
const char *k;
10051033
const char *v;
10061034
char errstr[256];
1035+
int r;
10071036

10081037
if (!(ks = cfl_PyObject_Unistr(ko))) {
10091038
PyErr_SetString(PyExc_TypeError,
@@ -1028,24 +1057,22 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
10281057
}
10291058

10301059
/* Special handling for certain config keys. */
1031-
if (ktype == RD_KAFKA_PRODUCER) {
1032-
int r;
1033-
1060+
if (ktype == RD_KAFKA_PRODUCER)
10341061
r = producer_conf_set_special((Producer *)self0,
10351062
conf, tconf, k, vo);
1036-
if (r == -1) {
1037-
/* Error */
1038-
Py_DECREF(ks);
1039-
rd_kafka_topic_conf_destroy(tconf);
1040-
rd_kafka_conf_destroy(conf);
1041-
return NULL;
1042-
1043-
} else if (r == 1) {
1044-
/* Handled */
1045-
continue;
1046-
}
1063+
else
1064+
r = consumer_conf_set_special((Consumer *)self0,
1065+
conf, tconf, k, vo);
1066+
if (r == -1) {
1067+
/* Error */
1068+
Py_DECREF(ks);
1069+
rd_kafka_topic_conf_destroy(tconf);
1070+
rd_kafka_conf_destroy(conf);
1071+
return NULL;
10471072

1048-
/* FALLTHRU */
1073+
} else if (r == 1) {
1074+
/* Handled */
1075+
continue;
10491076
}
10501077

10511078

confluent_kafka.h

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -197,6 +197,7 @@ typedef struct {
197197
int rebalance_assigned; /* Rebalance: Callback performed assign() call.*/
198198
PyObject *on_assign; /* Rebalance: on_assign callback */
199199
PyObject *on_revoke; /* Rebalance: on_revoke callback */
200+
PyObject *on_commit; /* Commit callback */
200201
int callback_crashed;
201202
PyThreadState *thread_state;
202203
} Consumer;

docs/index.rst

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ providing a dict of configuration properties to the instance constructor, e.g.::
2121

2222
conf = {'bootstrap.servers': 'mybroker.com',
2323
'group.id': 'mygroup', 'session.timeout.ms': 6000,
24+
'on_commit': my_commit_callback,
2425
'default.topic.config': {'auto.offset.reset': 'smallest'}}
2526
consumer = confluent_kafka.Consumer(**conf)
2627

@@ -40,5 +41,8 @@ The Python bindings also provide some additional configuration properties:
4041
This property may also be set per-message by passing ``callback=somefunc``
4142
to the confluent_kafka.Producer.produce() function.
4243

44+
* ``on_commit`` (**Consumer**): Callback used to indicate success or failure
45+
of commit requests.
46+
4347

4448

integration_test.py

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -191,6 +191,13 @@ def verify_producer_performance(with_dr_cb=True):
191191
(t_delivery_spent - t_produce_spent))
192192

193193

194+
def print_commit_result (consumer, err, partitions):
195+
if err is not None:
196+
print('# Failed to commit offsets: %s: %s' % (err, partitions))
197+
else:
198+
print('# Committed offsets for: %s' % partitions)
199+
200+
194201
def verify_consumer():
195202
""" Verify basic Consumer functionality """
196203

@@ -199,6 +206,7 @@ def verify_consumer():
199206
'group.id': 'test.py',
200207
'session.timeout.ms': 6000,
201208
'enable.auto.commit': False,
209+
'on_commit': print_commit_result,
202210
'default.topic.config': {
203211
'auto.offset.reset': 'earliest'
204212
}}

tests/test_Consumer.py

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,7 +12,11 @@ def test_basic_api():
1212
except TypeError as e:
1313
assert str(e) == "expected configuration dict"
1414

15-
kc = Consumer({'group.id':'test', 'socket.timeout.ms':'100'})
15+
def dummy_commit_cb (consumer, err, partitions):
16+
pass
17+
18+
kc = Consumer({'group.id':'test', 'socket.timeout.ms':'100',
19+
'on_commit': dummy_commit_cb})
1620

1721
kc.subscribe(["test"])
1822
kc.unsubscribe()

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