Skip to content

Commit 320b5f7

Browse files
committed
Added Consumer.get_watermark_offsets() (confluentinc#31)
1 parent d12c588 commit 320b5f7

File tree

6 files changed

+167
-59
lines changed

6 files changed

+167
-59
lines changed

confluent_kafka/__init__.py

Lines changed: 15 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,16 @@
11
__all__ = ['cimpl', 'avro', 'kafkatest']
2-
from .cimpl import (Consumer, KafkaError, KafkaException, Message, Producer, TopicPartition, libversion, version,
3-
TIMESTAMP_NOT_AVAILABLE, TIMESTAMP_CREATE_TIME, TIMESTAMP_LOG_APPEND_TIME)
2+
from .cimpl import (Consumer,
3+
KafkaError,
4+
KafkaException,
5+
Message,
6+
Producer,
7+
TopicPartition,
8+
libversion,
9+
version,
10+
TIMESTAMP_NOT_AVAILABLE,
11+
TIMESTAMP_CREATE_TIME,
12+
TIMESTAMP_LOG_APPEND_TIME,
13+
OFFSET_BEGINNING,
14+
OFFSET_END,
15+
OFFSET_STORED,
16+
OFFSET_INVALID)

confluent_kafka/src/Consumer.c

Lines changed: 109 additions & 45 deletions
Original file line numberDiff line numberDiff line change
@@ -232,25 +232,25 @@ static PyObject *Consumer_unassign (Handle *self, PyObject *ignore) {
232232
}
233233

234234
static PyObject *Consumer_assignment (Handle *self, PyObject *args,
235-
PyObject *kwargs) {
236-
237-
PyObject *plist;
238-
rd_kafka_topic_partition_list_t *c_parts;
239-
rd_kafka_resp_err_t err;
240-
241-
err = rd_kafka_assignment(self->rk, &c_parts);
242-
if (err) {
243-
cfl_PyErr_Format(err,
244-
"Failed to get assignment: %s",
245-
rd_kafka_err2str(err));
246-
return NULL;
247-
}
235+
PyObject *kwargs) {
236+
237+
PyObject *plist;
238+
rd_kafka_topic_partition_list_t *c_parts;
239+
rd_kafka_resp_err_t err;
240+
241+
err = rd_kafka_assignment(self->rk, &c_parts);
242+
if (err) {
243+
cfl_PyErr_Format(err,
244+
"Failed to get assignment: %s",
245+
rd_kafka_err2str(err));
246+
return NULL;
247+
}
248248

249249

250-
plist = c_parts_to_py(c_parts);
251-
rd_kafka_topic_partition_list_destroy(c_parts);
250+
plist = c_parts_to_py(c_parts);
251+
rd_kafka_topic_partition_list_destroy(c_parts);
252252

253-
return plist;
253+
return plist;
254254
}
255255

256256

@@ -392,33 +392,81 @@ static PyObject *Consumer_position (Handle *self, PyObject *args,
392392
}
393393

394394

395+
static PyObject *Consumer_get_watermark_offsets (Handle *self, PyObject *args,
396+
PyObject *kwargs) {
397+
398+
TopicPartition *tp;
399+
rd_kafka_resp_err_t err;
400+
double tmout = -1.0f;
401+
int cached = 0;
402+
int64_t low = RD_KAFKA_OFFSET_INVALID, high = RD_KAFKA_OFFSET_INVALID;
403+
static char *kws[] = { "partition", "timeout", "cached", NULL };
404+
PyObject *rtup;
405+
406+
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "O|db", kws,
407+
(PyObject **)&tp, &tmout, &cached))
408+
return NULL;
409+
410+
411+
if (PyObject_Type((PyObject *)tp) != (PyObject *)&TopicPartitionType) {
412+
PyErr_Format(PyExc_TypeError,
413+
"expected %s", TopicPartitionType.tp_name);
414+
return NULL;
415+
}
416+
417+
if (cached) {
418+
err = rd_kafka_get_watermark_offsets(self->rk,
419+
tp->topic, tp->partition,
420+
&low, &high);
421+
} else {
422+
err = rd_kafka_query_watermark_offsets(self->rk,
423+
tp->topic, tp->partition,
424+
&low, &high,
425+
tmout >= 0 ? (int)(tmout * 1000.0f) : -1);
426+
}
427+
428+
if (err) {
429+
cfl_PyErr_Format(err,
430+
"Failed to get watermark offsets: %s",
431+
rd_kafka_err2str(err));
432+
return NULL;
433+
}
434+
435+
rtup = PyTuple_New(2);
436+
PyTuple_SetItem(rtup, 0, PyLong_FromLongLong(low));
437+
PyTuple_SetItem(rtup, 1, PyLong_FromLongLong(high));
438+
439+
return rtup;
440+
}
441+
442+
395443

396444
static PyObject *Consumer_poll (Handle *self, PyObject *args,
397-
PyObject *kwargs) {
398-
double tmout = -1.0f;
399-
static char *kws[] = { "timeout", NULL };
400-
rd_kafka_message_t *rkm;
401-
PyObject *msgobj;
402-
CallState cs;
445+
PyObject *kwargs) {
446+
double tmout = -1.0f;
447+
static char *kws[] = { "timeout", NULL };
448+
rd_kafka_message_t *rkm;
449+
PyObject *msgobj;
450+
CallState cs;
403451

404-
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|d", kws, &tmout))
405-
return NULL;
452+
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|d", kws, &tmout))
453+
return NULL;
406454

407-
CallState_begin(self, &cs);
455+
CallState_begin(self, &cs);
408456

409-
rkm = rd_kafka_consumer_poll(self->rk, tmout >= 0 ?
410-
(int)(tmout * 1000.0f) : -1);
457+
rkm = rd_kafka_consumer_poll(self->rk, tmout >= 0 ?
458+
(int)(tmout * 1000.0f) : -1);
411459

412-
if (!CallState_end(self, &cs))
413-
return NULL;
460+
if (!CallState_end(self, &cs))
461+
return NULL;
414462

415-
if (!rkm)
416-
Py_RETURN_NONE;
463+
if (!rkm)
464+
Py_RETURN_NONE;
417465

418-
msgobj = Message_new0(rkm);
419-
rd_kafka_message_destroy(rkm);
466+
msgobj = Message_new0(rkm);
467+
rd_kafka_message_destroy(rkm);
420468

421-
return msgobj;
469+
return msgobj;
422470
}
423471

424472

@@ -511,17 +559,17 @@ static PyMethodDef Consumer_methods[] = {
511559
" :raises: KafkaException\n"
512560
"\n"
513561
},
514-
{ "assignment", (PyCFunction)Consumer_assignment,
515-
METH_VARARGS|METH_KEYWORDS,
516-
".. py:function:: assignment()\n"
517-
"\n"
518-
" Returns the current partition assignment.\n"
519-
"\n"
520-
" :returns: List of assigned topic+partitions.\n"
521-
" :rtype: list(TopicPartition)\n"
522-
" :raises: KafkaException\n"
523-
"\n"
524-
},
562+
{ "assignment", (PyCFunction)Consumer_assignment,
563+
METH_VARARGS|METH_KEYWORDS,
564+
".. py:function:: assignment()\n"
565+
"\n"
566+
" Returns the current partition assignment.\n"
567+
"\n"
568+
" :returns: List of assigned topic+partitions.\n"
569+
" :rtype: list(TopicPartition)\n"
570+
" :raises: KafkaException\n"
571+
"\n"
572+
},
525573
{ "commit", (PyCFunction)Consumer_commit, METH_VARARGS|METH_KEYWORDS,
526574
".. py:function:: commit([message=None], [offsets=None], [async=True])\n"
527575
"\n"
@@ -566,6 +614,22 @@ static PyMethodDef Consumer_methods[] = {
566614
" :raises: KafkaException\n"
567615
"\n"
568616
},
617+
{ "get_watermark_offsets", (PyCFunction)Consumer_get_watermark_offsets,
618+
METH_VARARGS|METH_KEYWORDS,
619+
".. py:function:: get_watermark_offsets(partition, [timeout=None], [cached=False])\n"
620+
"\n"
621+
" Retrieve low and high offsets for partition.\n"
622+
"\n"
623+
" :param TopicPartition partition: Topic+partition to return offsets for."
624+
" :param float timeout: Request timeout (when cached=False).\n"
625+
" :param bool cached: Instead of querying the broker used cached information. "
626+
"Cached values: The low offset is updated periodically (if statistics.interval.ms is set) while "
627+
"the high offset is updated on each message fetched from the broker for this partition."
628+
" :returns: Tuple of (low,high) on success or None on timeout.\n"
629+
" :rtype: tuple(int,int)\n"
630+
" :raises: KafkaException\n"
631+
"\n"
632+
},
569633
{ "close", (PyCFunction)Consumer_close, METH_NOARGS,
570634
"\n"
571635
" Close down and terminate the Kafka Consumer.\n"

confluent_kafka/src/confluent_kafka.c

Lines changed: 1 addition & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -573,15 +573,6 @@ PyObject *Message_new0 (const rd_kafka_message_t *rkm) {
573573
*
574574
*
575575
****************************************************************************/
576-
typedef struct {
577-
PyObject_HEAD
578-
char *topic;
579-
int partition;
580-
int64_t offset;
581-
PyObject *error;
582-
} TopicPartition;
583-
584-
585576
static int TopicPartition_clear (TopicPartition *self) {
586577
if (self->topic) {
587578
free(self->topic);
@@ -681,8 +672,6 @@ static PyObject *TopicPartition_str0 (TopicPartition *self) {
681672
}
682673

683674

684-
static PyTypeObject TopicPartitionType;
685-
686675
static PyObject *
687676
TopicPartition_richcompare (TopicPartition *self, PyObject *o2,
688677
int op) {
@@ -739,7 +728,7 @@ static long TopicPartition_hash (TopicPartition *self) {
739728
}
740729

741730

742-
static PyTypeObject TopicPartitionType = {
731+
PyTypeObject TopicPartitionType = {
743732
PyVarObject_HEAD_INIT(NULL, 0)
744733
"cimpl.TopicPartition", /*tp_name*/
745734
sizeof(TopicPartition), /*tp_basicsize*/

confluent_kafka/src/confluent_kafka.h

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -191,6 +191,26 @@ void CallState_resume (CallState *cs);
191191
void CallState_crash (CallState *cs);
192192

193193

194+
/****************************************************************************
195+
*
196+
*
197+
* TopicPartition
198+
*
199+
*
200+
*
201+
*
202+
****************************************************************************/
203+
typedef struct {
204+
PyObject_HEAD
205+
char *topic;
206+
int partition;
207+
int64_t offset;
208+
PyObject *error;
209+
} TopicPartition;
210+
211+
extern PyTypeObject TopicPartitionType;
212+
213+
194214
/****************************************************************************
195215
*
196216
*

examples/integration_test.py

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -359,6 +359,18 @@ def verify_consumer():
359359
print('max_msgcnt %d reached' % msgcnt)
360360
break
361361

362+
# Get current assignment
363+
assignment = c.assignment()
364+
365+
# Get cached watermark offsets
366+
# Since we're not making use of statistics the low offset is not known so ignore it.
367+
lo,hi = c.get_watermark_offsets(assignment[0], cached=True)
368+
print('Cached offsets for %s: %d - %d' % (assignment[0], lo, hi))
369+
370+
# Query broker for offsets
371+
lo,hi = c.get_watermark_offsets(assignment[0], timeout=1.0)
372+
print('Queried offsets for %s: %d - %d' % (assignment[0], lo, hi))
373+
362374

363375
# Close consumer
364376
c.close()

tests/test_Consumer.py

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,16 @@ def dummy_assign_revoke (consumer, partitions):
4747
assignment = kc.assignment()
4848
assert partitions == assignment
4949

50+
# Get cached watermarks, should all be invalid.
51+
lo, hi = kc.get_watermark_offsets(partitions[0], cached=True)
52+
assert lo == -1001 and hi == -1001
53+
54+
# Query broker for watermarks, should raise an exception.
55+
try:
56+
lo, hi = kc.get_watermark_offsets(partitions[0], timeout=0.5, cached=False)
57+
except KafkaException as e:
58+
assert e.args[0].code() in (KafkaError._TIMED_OUT, KafkaError._WAIT_COORD, KafkaError.LEADER_NOT_AVAILABLE), str(e.args([0]))
59+
5060
kc.unassign()
5161

5262
kc.commit(async=True)

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