Skip to content

Commit c864b64

Browse files
tburmeisteredenhill
authored andcommitted
Add batch consume() API (closes confluentinc#252, confluentinc#282, @tburmeister)
1 parent 887331d commit c864b64

File tree

4 files changed

+281
-7
lines changed

4 files changed

+281
-7
lines changed

confluent_kafka/src/Consumer.c

Lines changed: 103 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,10 @@ static int Consumer_clear (Handle *self) {
4141
Py_DECREF(self->u.Consumer.on_commit);
4242
self->u.Consumer.on_commit = NULL;
4343
}
44+
if (self->u.Consumer.rkqu) {
45+
rd_kafka_queue_destroy(self->u.Consumer.rkqu);
46+
self->u.Consumer.rkqu = NULL;
47+
}
4448

4549
Handle_clear(self);
4650

@@ -424,9 +428,9 @@ static PyObject *Consumer_commit (Handle *self, PyObject *args,
424428
}
425429

426430
if (async) {
427-
/* Async mode: Use consumer queue for offset commit callback,
431+
/* Async mode: Use consumer queue for offset commit
428432
* served by consumer_poll() */
429-
rkqu = rd_kafka_queue_get_consumer(self->rk);
433+
rkqu = self->u.Consumer.rkqu;
430434

431435
} else {
432436
/* Sync mode: Let commit_queue() trigger the callback. */
@@ -446,11 +450,7 @@ static PyObject *Consumer_commit (Handle *self, PyObject *args,
446450
if (c_offsets)
447451
rd_kafka_topic_partition_list_destroy(c_offsets);
448452

449-
if (async) {
450-
/* Loose reference to consumer queue */
451-
rd_kafka_queue_destroy(rkqu);
452-
453-
} else {
453+
if (!async) {
454454
/* Re-lock GIL */
455455
PyEval_RestoreThread(thread_state);
456456

@@ -743,6 +743,71 @@ static PyObject *Consumer_poll (Handle *self, PyObject *args,
743743
}
744744

745745

746+
static PyObject *Consumer_consume (Handle *self, PyObject *args,
747+
PyObject *kwargs) {
748+
unsigned int num_messages = 1;
749+
double tmout = -1.0f;
750+
static char *kws[] = { "num_messages", "timeout", NULL };
751+
rd_kafka_message_t **rkmessages;
752+
PyObject *msglist;
753+
rd_kafka_queue_t *rkqu = self->u.Consumer.rkqu;
754+
CallState cs;
755+
Py_ssize_t i;
756+
757+
if (!self->rk) {
758+
PyErr_SetString(PyExc_RuntimeError,
759+
"Consumer closed");
760+
return NULL;
761+
}
762+
763+
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|Id", kws,
764+
&num_messages, &tmout))
765+
return NULL;
766+
767+
if (num_messages > 1000000) {
768+
PyErr_SetString(PyExc_ValueError,
769+
"num_messages must be between 0 and 1000000 (1M)");
770+
return NULL;
771+
}
772+
773+
CallState_begin(self, &cs);
774+
775+
rkmessages = malloc(num_messages * sizeof(rd_kafka_message_t *));
776+
777+
Py_ssize_t n = (Py_ssize_t)rd_kafka_consume_batch_queue(rkqu,
778+
tmout >= 0 ? (int)(tmout * 1000.0f) : -1,
779+
rkmessages,
780+
num_messages);
781+
782+
if (!CallState_end(self, &cs)) {
783+
for (i = 0; i < n; i++) {
784+
rd_kafka_message_destroy(rkmessages[i]);
785+
}
786+
free(rkmessages);
787+
return NULL;
788+
}
789+
790+
if (n < 0) {
791+
free(rkmessages);
792+
cfl_PyErr_Format(rd_kafka_last_error(),
793+
"%s", rd_kafka_err2str(rd_kafka_last_error()));
794+
return NULL;
795+
}
796+
797+
msglist = PyList_New(n);
798+
799+
for (i = 0; i < n; i++) {
800+
PyObject *msgobj = Message_new0(self, rkmessages[i]);
801+
PyList_SET_ITEM(msglist, i, msgobj);
802+
rd_kafka_message_destroy(rkmessages[i]);
803+
}
804+
805+
free(rkmessages);
806+
807+
return msglist;
808+
}
809+
810+
746811
static PyObject *Consumer_close (Handle *self, PyObject *ignore) {
747812
CallState cs;
748813

@@ -756,6 +821,11 @@ static PyObject *Consumer_close (Handle *self, PyObject *ignore) {
756821

757822
rd_kafka_consumer_close(self->rk);
758823

824+
if (self->u.Consumer.rkqu) {
825+
rd_kafka_queue_destroy(self->u.Consumer.rkqu);
826+
self->u.Consumer.rkqu = NULL;
827+
}
828+
759829
rd_kafka_destroy(self->rk);
760830
self->rk = NULL;
761831

@@ -825,6 +895,30 @@ static PyMethodDef Consumer_methods[] = {
825895
" :raises: RuntimeError if called on a closed consumer\n"
826896
"\n"
827897
},
898+
{ "consume", (PyCFunction)Consumer_consume,
899+
METH_VARARGS|METH_KEYWORDS,
900+
".. py:function:: consume([num_messages=1], [timeout=-1])\n"
901+
"\n"
902+
" Consume messages, calls callbacks and returns list of messages "
903+
"(possibly empty on timeout).\n"
904+
"\n"
905+
" The application must check the returned :py:class:`Message` "
906+
"object's :py:func:`Message.error()` method to distinguish "
907+
"between proper messages (error() returns None), or an event or "
908+
"error for each :py:class:`Message` in the list (see error().code() "
909+
"for specifics).\n"
910+
"\n"
911+
" .. note: Callbacks may be called from this method, "
912+
"such as ``on_assign``, ``on_revoke``, et.al.\n"
913+
"\n"
914+
" :param int num_messages: Maximum number of messages to return (default: 1).\n"
915+
" :param float timeout: Maximum time to block waiting for message, event or callback (default: infinite (-1)).\n"
916+
" :returns: A list of Message objects (possibly empty on timeout)\n"
917+
" :rtype: list(Message)\n"
918+
" :raises: RuntimeError if called on a closed consumer, KafkaError "
919+
"in case of internal error, or ValueError if num_messages > 1M.\n"
920+
"\n"
921+
},
828922
{ "assign", (PyCFunction)Consumer_assign, METH_O,
829923
".. py:function:: assign(partitions)\n"
830924
"\n"
@@ -1053,6 +1147,8 @@ static int Consumer_init (PyObject *selfobj, PyObject *args, PyObject *kwargs) {
10531147

10541148
rd_kafka_poll_set_consumer(self->rk);
10551149

1150+
self->u.Consumer.rkqu = rd_kafka_queue_get_consumer(self->rk);
1151+
10561152
return 0;
10571153
}
10581154

confluent_kafka/src/confluent_kafka.h

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -186,6 +186,7 @@ typedef struct {
186186
PyObject *on_assign; /* Rebalance: on_assign callback */
187187
PyObject *on_revoke; /* Rebalance: on_revoke callback */
188188
PyObject *on_commit; /* Commit callback */
189+
rd_kafka_queue_t *rkqu; /* Consumer queue */
189190

190191
} Consumer;
191192
} u;

examples/integration_test.py

Lines changed: 162 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -565,6 +565,162 @@ def my_on_revoke(consumer, partitions):
565565
c.close()
566566

567567

568+
def verify_batch_consumer():
569+
""" Verify basic batch Consumer functionality """
570+
571+
# Consumer config
572+
conf = {'bootstrap.servers': bootstrap_servers,
573+
'group.id': 'test.py',
574+
'session.timeout.ms': 6000,
575+
'enable.auto.commit': False,
576+
'api.version.request': api_version_request,
577+
'on_commit': print_commit_result,
578+
'error_cb': error_cb,
579+
'default.topic.config': {
580+
'auto.offset.reset': 'earliest'
581+
}}
582+
583+
# Create consumer
584+
c = confluent_kafka.Consumer(**conf)
585+
586+
# Subscribe to a list of topics
587+
c.subscribe([topic])
588+
589+
max_msgcnt = 1000
590+
batch_cnt = 100
591+
msgcnt = 0
592+
593+
while msgcnt < max_msgcnt:
594+
# Consume until we hit max_msgcnt
595+
596+
# Consume messages (error()==0) or event (error()!=0)
597+
msglist = c.consume(batch_cnt, 10.0)
598+
assert len(msglist) == batch_cnt
599+
600+
for msg in msglist:
601+
if msg.error():
602+
print('Consumer error: %s: ignoring' % msg.error())
603+
continue
604+
605+
tstype, timestamp = msg.timestamp()
606+
print('%s[%d]@%d: key=%s, value=%s, tstype=%d, timestamp=%s' %
607+
(msg.topic(), msg.partition(), msg.offset(),
608+
msg.key(), msg.value(), tstype, timestamp))
609+
610+
if (msg.offset() % 5) == 0:
611+
# Async commit
612+
c.commit(msg, async=True)
613+
elif (msg.offset() % 4) == 0:
614+
offsets = c.commit(msg, async=False)
615+
assert len(offsets) == 1, 'expected 1 offset, not %s' % (offsets)
616+
assert offsets[0].offset == msg.offset()+1, \
617+
'expected offset %d to be committed, not %s' % \
618+
(msg.offset(), offsets)
619+
print('Sync committed offset: %s' % offsets)
620+
621+
msgcnt += 1
622+
623+
print('max_msgcnt %d reached' % msgcnt)
624+
625+
# Get current assignment
626+
assignment = c.assignment()
627+
628+
# Get cached watermark offsets
629+
# Since we're not making use of statistics the low offset is not known so ignore it.
630+
lo, hi = c.get_watermark_offsets(assignment[0], cached=True)
631+
print('Cached offsets for %s: %d - %d' % (assignment[0], lo, hi))
632+
633+
# Query broker for offsets
634+
lo, hi = c.get_watermark_offsets(assignment[0], timeout=1.0)
635+
print('Queried offsets for %s: %d - %d' % (assignment[0], lo, hi))
636+
637+
# Close consumer
638+
c.close()
639+
640+
# Start a new client and get the committed offsets
641+
c = confluent_kafka.Consumer(**conf)
642+
offsets = c.committed(list(map(lambda p: confluent_kafka.TopicPartition(topic, p), range(0, 3))))
643+
for tp in offsets:
644+
print(tp)
645+
646+
c.close()
647+
648+
649+
def verify_batch_consumer_performance():
650+
""" Verify batch Consumer performance """
651+
652+
conf = {'bootstrap.servers': bootstrap_servers,
653+
'group.id': uuid.uuid1(),
654+
'session.timeout.ms': 6000,
655+
'error_cb': error_cb,
656+
'default.topic.config': {
657+
'auto.offset.reset': 'earliest'
658+
}}
659+
660+
c = confluent_kafka.Consumer(**conf)
661+
662+
def my_on_assign(consumer, partitions):
663+
print('on_assign:', len(partitions), 'partitions:')
664+
for p in partitions:
665+
print(' %s [%d] @ %d' % (p.topic, p.partition, p.offset))
666+
consumer.assign(partitions)
667+
668+
def my_on_revoke(consumer, partitions):
669+
print('on_revoke:', len(partitions), 'partitions:')
670+
for p in partitions:
671+
print(' %s [%d] @ %d' % (p.topic, p.partition, p.offset))
672+
consumer.unassign()
673+
674+
c.subscribe([topic], on_assign=my_on_assign, on_revoke=my_on_revoke)
675+
676+
max_msgcnt = 1000000
677+
bytecnt = 0
678+
msgcnt = 0
679+
batch_size = 1000
680+
681+
print('Will now consume %d messages' % max_msgcnt)
682+
683+
if with_progress:
684+
bar = Bar('Consuming', max=max_msgcnt,
685+
suffix='%(index)d/%(max)d [%(eta_td)s]')
686+
else:
687+
bar = None
688+
689+
while msgcnt < max_msgcnt:
690+
# Consume until we hit max_msgcnt
691+
692+
msglist = c.consume(num_messages=batch_size, timeout=20.0)
693+
694+
for msg in msglist:
695+
if msg.error():
696+
if msg.error().code() == confluent_kafka.KafkaError._PARTITION_EOF:
697+
# Reached EOF for a partition, ignore.
698+
continue
699+
else:
700+
raise confluent_kafka.KafkaException(msg.error())
701+
702+
bytecnt += len(msg)
703+
msgcnt += 1
704+
705+
if bar is not None and (msgcnt % 10000) == 0:
706+
bar.next(n=10000)
707+
708+
if msgcnt == 1:
709+
t_first_msg = time.time()
710+
711+
if bar is not None:
712+
bar.finish()
713+
714+
if msgcnt > 0:
715+
t_spent = time.time() - t_first_msg
716+
print('%d messages (%.2fMb) consumed in %.3fs: %d msgs/s, %.2f Mb/s' %
717+
(msgcnt, bytecnt / (1024*1024), t_spent, msgcnt / t_spent,
718+
(bytecnt / t_spent) / (1024*1024)))
719+
720+
print('closing consumer')
721+
c.close()
722+
723+
568724
def verify_stats_cb():
569725
""" Verify stats_cb """
570726

@@ -663,6 +819,9 @@ def stats_cb(stats_json_str):
663819
print('=' * 30, 'Verifying Consumer', '=' * 30)
664820
verify_consumer()
665821

822+
print('=' * 30, 'Verifying batch Consumer', '=' * 30)
823+
verify_batch_consumer()
824+
666825
print('=' * 30, 'Verifying Producer performance (with dr_cb)', '=' * 30)
667826
verify_producer_performance(with_dr_cb=True)
668827

@@ -672,6 +831,9 @@ def stats_cb(stats_json_str):
672831
print('=' * 30, 'Verifying Consumer performance', '=' * 30)
673832
verify_consumer_performance()
674833

834+
print('=' * 30, 'Verifying batch Consumer performance', '=' * 30)
835+
verify_batch_consumer_performance()
836+
675837
print('=' * 30, 'Verifying stats_cb', '=' * 30)
676838
verify_stats_cb()
677839

tests/test_Consumer.py

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,17 @@ def dummy_assign_revoke(consumer, partitions):
4141
if msg is not None:
4242
assert msg.timestamp() == (TIMESTAMP_NOT_AVAILABLE, -1)
4343

44+
msglist = kc.consume(num_messages=10, timeout=0.001)
45+
assert len(msglist) == 0, "expected 0 messages, not %d" % len(msglist)
46+
47+
with pytest.raises(ValueError) as ex:
48+
kc.consume(-100)
49+
assert 'num_messages must be between 0 and 1000000 (1M)' == str(ex.value)
50+
51+
with pytest.raises(ValueError) as ex:
52+
kc.consume(1000001)
53+
assert 'num_messages must be between 0 and 1000000 (1M)' == str(ex.value)
54+
4455
partitions = list(map(lambda part: TopicPartition("test", part), range(0, 100, 3)))
4556
kc.assign(partitions)
4657

@@ -204,6 +215,10 @@ def test_any_method_after_close_throws_exception():
204215
c.poll()
205216
assert 'Consumer closed' == str(ex.value)
206217

218+
with pytest.raises(RuntimeError) as ex:
219+
c.consume()
220+
assert 'Consumer closed' == str(ex.value)
221+
207222
with pytest.raises(RuntimeError) as ex:
208223
c.assign([TopicPartition('test', 0)])
209224
assert 'Consumer closed' == str(ex.value)

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