Skip to content

Commit 20bb8bb

Browse files
authored
Merge pull request confluentinc#34 from confluentinc/watermark_offsets
Add assignment and watermark offsets APIs
2 parents 61ac655 + 8ea2fd4 commit 20bb8bb

File tree

8 files changed

+211
-38
lines changed

8 files changed

+211
-38
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: 115 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -231,6 +231,28 @@ static PyObject *Consumer_unassign (Handle *self, PyObject *ignore) {
231231
Py_RETURN_NONE;
232232
}
233233

234+
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+
}
248+
249+
250+
plist = c_parts_to_py(c_parts);
251+
rd_kafka_topic_partition_list_destroy(c_parts);
252+
253+
return plist;
254+
}
255+
234256

235257

236258
static PyObject *Consumer_commit (Handle *self, PyObject *args,
@@ -370,33 +392,81 @@ static PyObject *Consumer_position (Handle *self, PyObject *args,
370392
}
371393

372394

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+
373443

374444
static PyObject *Consumer_poll (Handle *self, PyObject *args,
375-
PyObject *kwargs) {
376-
double tmout = -1.0f;
377-
static char *kws[] = { "timeout", NULL };
378-
rd_kafka_message_t *rkm;
379-
PyObject *msgobj;
380-
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;
381451

382-
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|d", kws, &tmout))
383-
return NULL;
452+
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|d", kws, &tmout))
453+
return NULL;
384454

385-
CallState_begin(self, &cs);
455+
CallState_begin(self, &cs);
386456

387-
rkm = rd_kafka_consumer_poll(self->rk, tmout >= 0 ?
388-
(int)(tmout * 1000.0f) : -1);
457+
rkm = rd_kafka_consumer_poll(self->rk, tmout >= 0 ?
458+
(int)(tmout * 1000.0f) : -1);
389459

390-
if (!CallState_end(self, &cs))
391-
return NULL;
460+
if (!CallState_end(self, &cs))
461+
return NULL;
392462

393-
if (!rkm)
394-
Py_RETURN_NONE;
463+
if (!rkm)
464+
Py_RETURN_NONE;
395465

396-
msgobj = Message_new0(rkm);
397-
rd_kafka_message_destroy(rkm);
466+
msgobj = Message_new0(rkm);
467+
rd_kafka_message_destroy(rkm);
398468

399-
return msgobj;
469+
return msgobj;
400470
}
401471

402472

@@ -489,6 +559,17 @@ static PyMethodDef Consumer_methods[] = {
489559
" :raises: KafkaException\n"
490560
"\n"
491561
},
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+
},
492573
{ "commit", (PyCFunction)Consumer_commit, METH_VARARGS|METH_KEYWORDS,
493574
".. py:function:: commit([message=None], [offsets=None], [async=True])\n"
494575
"\n"
@@ -533,6 +614,22 @@ static PyMethodDef Consumer_methods[] = {
533614
" :raises: KafkaException\n"
534615
"\n"
535616
},
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+
},
536633
{ "close", (PyCFunction)Consumer_close, METH_NOARGS,
537634
"\n"
538635
" Close down and terminate the Kafka Consumer.\n"

confluent_kafka/src/confluent_kafka.c

Lines changed: 14 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -29,8 +29,6 @@
2929
* partitioner run without any locks taken.
3030
* Until this is fixed the partitioner is ignored and librdkafka's
3131
* default will be used.
32-
* - TopicPartition.offset should probably be None for the INVALID offset
33-
* rather than exposing the special value -1001.
3432
* - KafkaError type .tp_doc allocation is lost on exit.
3533
*
3634
*/
@@ -573,15 +571,6 @@ PyObject *Message_new0 (const rd_kafka_message_t *rkm) {
573571
*
574572
*
575573
****************************************************************************/
576-
typedef struct {
577-
PyObject_HEAD
578-
char *topic;
579-
int partition;
580-
int64_t offset;
581-
PyObject *error;
582-
} TopicPartition;
583-
584-
585574
static int TopicPartition_clear (TopicPartition *self) {
586575
if (self->topic) {
587576
free(self->topic);
@@ -652,11 +641,17 @@ static int TopicPartition_traverse (TopicPartition *self,
652641

653642
static PyMemberDef TopicPartition_members[] = {
654643
{ "topic", T_STRING, offsetof(TopicPartition, topic), READONLY,
655-
":py:attribute: Topic name (string)" },
644+
":py:attribute:topic - Topic name (string)" },
656645
{ "partition", T_INT, offsetof(TopicPartition, partition), 0,
657646
":py:attribute: Partition number (int)" },
658647
{ "offset", T_LONGLONG, offsetof(TopicPartition, offset), 0,
659-
":py:attribute: Offset (long)" }, /* FIXME: Possibly use None for INVALID offset (-1001) */
648+
" :py:attribute: Offset (long)\n"
649+
"Either an absolute offset (>=0) or a logical offset:"
650+
" :py:const:`OFFSET_BEGINNING`,"
651+
" :py:const:`OFFSET_END`,"
652+
" :py:const:`OFFSET_STORED`,"
653+
" :py:const:`OFFSET_INVALID`"
654+
},
660655
{ "error", T_OBJECT, offsetof(TopicPartition, error), READONLY,
661656
":py:attribute: Indicates an error (with :py:class:`KafkaError`) unless None." },
662657
{ NULL }
@@ -681,8 +676,6 @@ static PyObject *TopicPartition_str0 (TopicPartition *self) {
681676
}
682677

683678

684-
static PyTypeObject TopicPartitionType;
685-
686679
static PyObject *
687680
TopicPartition_richcompare (TopicPartition *self, PyObject *o2,
688681
int op) {
@@ -739,7 +732,7 @@ static long TopicPartition_hash (TopicPartition *self) {
739732
}
740733

741734

742-
static PyTypeObject TopicPartitionType = {
735+
PyTypeObject TopicPartitionType = {
743736
PyVarObject_HEAD_INIT(NULL, 0)
744737
"cimpl.TopicPartition", /*tp_name*/
745738
sizeof(TopicPartition), /*tp_basicsize*/
@@ -1569,6 +1562,11 @@ static PyObject *_init_cimpl (void) {
15691562
PyModule_AddIntConstant(m, "TIMESTAMP_CREATE_TIME", RD_KAFKA_TIMESTAMP_CREATE_TIME);
15701563
PyModule_AddIntConstant(m, "TIMESTAMP_LOG_APPEND_TIME", RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME);
15711564

1565+
PyModule_AddIntConstant(m, "OFFSET_BEGINNING", RD_KAFKA_OFFSET_BEGINNING);
1566+
PyModule_AddIntConstant(m, "OFFSET_END", RD_KAFKA_OFFSET_END);
1567+
PyModule_AddIntConstant(m, "OFFSET_STORED", RD_KAFKA_OFFSET_STORED);
1568+
PyModule_AddIntConstant(m, "OFFSET_INVALID", RD_KAFKA_OFFSET_INVALID);
1569+
15721570
return m;
15731571
}
15741572

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
*

docs/index.rst

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,17 @@ KafkaException
5555
.. autoclass:: confluent_kafka.KafkaException
5656
:members:
5757

58+
******
59+
Offset
60+
******
61+
62+
Logical offset constants:
63+
64+
* :py:const:`OFFSET_BEGINNING` - Beginning of partition (oldest offset)
65+
* :py:const:`OFFSET_END` - End of partition (next offset)
66+
* :py:const:`OFFSET_STORED` - Use stored/committed offset
67+
* :py:const:`OFFSET_INVALID` - Invalid/Default offset
68+
5869

5970

6071
Configuration

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: 17 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,6 @@
11
#!/usr/bin/env python
22

3-
from confluent_kafka import Consumer, TopicPartition, KafkaError, KafkaException, TIMESTAMP_NOT_AVAILABLE, libversion
3+
from confluent_kafka import Consumer, TopicPartition, KafkaError, KafkaException, TIMESTAMP_NOT_AVAILABLE, OFFSET_INVALID, libversion
44
import pytest
55

66

@@ -43,6 +43,21 @@ def dummy_assign_revoke (consumer, partitions):
4343
partitions = list(map(lambda p: TopicPartition("test", p), range(0,100,3)))
4444
kc.assign(partitions)
4545

46+
# Verify assignment
47+
assignment = kc.assignment()
48+
assert partitions == assignment
49+
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+
assert lo == OFFSET_INVALID and hi == OFFSET_INVALID
54+
55+
# Query broker for watermarks, should raise an exception.
56+
try:
57+
lo, hi = kc.get_watermark_offsets(partitions[0], timeout=0.5, cached=False)
58+
except KafkaException as e:
59+
assert e.args[0].code() in (KafkaError._TIMED_OUT, KafkaError._WAIT_COORD, KafkaError.LEADER_NOT_AVAILABLE), str(e.args([0]))
60+
4661
kc.unassign()
4762

4863
kc.commit(async=True)
@@ -54,7 +69,7 @@ def dummy_assign_revoke (consumer, partitions):
5469

5570
# Get current position, should all be invalid.
5671
kc.position(partitions)
57-
assert len([p for p in partitions if p.offset == -1001]) == len(partitions)
72+
assert len([p for p in partitions if p.offset == OFFSET_INVALID]) == len(partitions)
5873

5974
try:
6075
offsets = kc.committed(partitions, timeout=0.001)

tests/test_enums.py

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -13,3 +13,10 @@ def test_tstype_enums():
1313
assert confluent_kafka.TIMESTAMP_NOT_AVAILABLE == 0
1414
assert confluent_kafka.TIMESTAMP_CREATE_TIME == 1
1515
assert confluent_kafka.TIMESTAMP_LOG_APPEND_TIME == 2
16+
17+
def test_offset_consts():
18+
""" Make sure librdkafka's logical offsets are available. """
19+
assert confluent_kafka.OFFSET_BEGINNING == -2
20+
assert confluent_kafka.OFFSET_END == -1
21+
assert confluent_kafka.OFFSET_STORED == -1000
22+
assert confluent_kafka.OFFSET_INVALID == -1001

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