Skip to content

Commit 514f6a8

Browse files
johnistanedenhill
authored andcommitted
Expose offsets_for_times consumer method. closes confluentinc#224 (confluentinc#268, @johnistan)
1 parent e9d7932 commit 514f6a8

File tree

3 files changed

+98
-0
lines changed

3 files changed

+98
-0
lines changed

confluent_kafka/src/Consumer.c

Lines changed: 68 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -704,6 +704,56 @@ static PyObject *Consumer_get_watermark_offsets (Handle *self, PyObject *args,
704704
}
705705

706706

707+
static PyObject *Consumer_offsets_for_times (Handle *self, PyObject *args,
708+
PyObject *kwargs) {
709+
#if RD_KAFKA_VERSION < 0x000b0000
710+
PyErr_Format(PyExc_NotImplementedError,
711+
"Consumer offsets_for_times require "
712+
"confluent-kafka-python built for librdkafka "
713+
"version >=v0.11.0 (librdkafka runtime 0x%x, "
714+
"buildtime 0x%x)",
715+
rd_kafka_version(), RD_KAFKA_VERSION);
716+
return NULL;
717+
#else
718+
719+
PyObject *plist;
720+
double tmout = -1.0f;
721+
rd_kafka_topic_partition_list_t *c_parts;
722+
rd_kafka_resp_err_t err;
723+
static char *kws[] = { "partitions", "timeout", NULL };
724+
725+
if (!self->rk) {
726+
PyErr_SetString(PyExc_RuntimeError,
727+
"Consumer closed");
728+
return NULL;
729+
}
730+
731+
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "O|d", kws,
732+
&plist, &tmout))
733+
return NULL;
734+
735+
if (!(c_parts = py_to_c_parts(plist)))
736+
return NULL;
737+
738+
err = rd_kafka_offsets_for_times(self->rk,
739+
c_parts,
740+
tmout >= 0 ? (int)(tmout * 1000.0f) : -1);
741+
742+
if (err) {
743+
rd_kafka_topic_partition_list_destroy(c_parts);
744+
cfl_PyErr_Format(err,
745+
"Failed to get offsets: %s",
746+
rd_kafka_err2str(err));
747+
return NULL;
748+
}
749+
750+
plist = c_parts_to_py(c_parts);
751+
rd_kafka_topic_partition_list_destroy(c_parts);
752+
753+
return plist;
754+
#endif
755+
}
756+
707757

708758
static PyObject *Consumer_poll (Handle *self, PyObject *args,
709759
PyObject *kwargs) {
@@ -1030,6 +1080,24 @@ static PyMethodDef Consumer_methods[] = {
10301080
" :raises: RuntimeError if called on a closed consumer\n"
10311081
"\n"
10321082
},
1083+
{ "offsets_for_times", (PyCFunction)Consumer_offsets_for_times,
1084+
METH_VARARGS|METH_KEYWORDS,
1085+
".. py:function:: offsets_for_times(partitions, [timeout=None])\n"
1086+
"\n"
1087+
" offsets_for_times looks up offsets by timestamp for the given partitions.\n"
1088+
"\n"
1089+
" The returned offsets for each partition is the earliest offset whose\n"
1090+
" timestamp is greater than or equal to the given timestamp in the\n"
1091+
" corresponding partition.\n"
1092+
"\n"
1093+
" :param list(TopicPartition) partitions: topic+partitions with timestamps in the TopicPartition.offset field."
1094+
" :param float timeout: Request timeout.\n"
1095+
" :returns: list of topic+partition with offset field set and possibly error set\n"
1096+
" :rtype: list(TopicPartition)\n"
1097+
" :raises: KafkaException\n"
1098+
" :raises: RuntimeError if called on a closed consumer\n"
1099+
"\n"
1100+
},
10331101
{ "close", (PyCFunction)Consumer_close, METH_NOARGS,
10341102
"\n"
10351103
" Close down and terminate the Kafka Consumer.\n"

examples/integration_test.py

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -471,10 +471,18 @@ def verify_consumer():
471471
lo, hi = c.get_watermark_offsets(assignment[0], cached=True)
472472
print('Cached offsets for %s: %d - %d' % (assignment[0], lo, hi))
473473

474+
474475
# Query broker for offsets
475476
lo, hi = c.get_watermark_offsets(assignment[0], timeout=1.0)
476477
print('Queried offsets for %s: %d - %d' % (assignment[0], lo, hi))
477478

479+
# Query offsets for timestamps by setting the topic partition offset to a timestamp. 123456789000 + 1
480+
topic_partions_to_search = list(map(lambda p: confluent_kafka.TopicPartition(topic, p, 123456789001), range(0, 3)))
481+
print("Searching for offsets with %s" % topic_partions_to_search)
482+
483+
offsets = c.offsets_for_times(topic_partions_to_search, timeout=1.0)
484+
print("offsets_for_times results: %s" % offsets)
485+
478486
# Close consumer
479487
c.close()
480488

tests/test_Consumer.py

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -171,6 +171,24 @@ def poll(self, somearg):
171171
sc.close()
172172

173173

174+
@pytest.mark.skipif(libversion()[1] < 0x000b0000,
175+
reason="requires librdkafka >=0.11.0")
176+
def test_offsets_for_times():
177+
c = Consumer({'group.id': 'test',
178+
'enable.auto.commit': True,
179+
'enable.auto.offset.store': False,
180+
'socket.timeout.ms': 50,
181+
'session.timeout.ms': 100})
182+
# Query broker for timestamps for partition
183+
try:
184+
test_topic_partition = TopicPartition("test", 0, 100)
185+
c.offsets_for_times([test_topic_partition], timeout=0.1)
186+
except KafkaException as e:
187+
assert e.args[0].code() in (KafkaError._TIMED_OUT, KafkaError._WAIT_COORD, KafkaError.LEADER_NOT_AVAILABLE),\
188+
str(e.args([0]))
189+
c.close()
190+
191+
174192
def test_multiple_close_throw_exception():
175193
""" Calling Consumer.close() multiple times should throw Runtime Exception
176194
"""
@@ -266,3 +284,7 @@ def test_calling_store_offsets_after_close_throws_erro():
266284
with pytest.raises(RuntimeError) as ex:
267285
c.store_offsets(offsets=[TopicPartition("test", 0, 42)])
268286
assert 'Consumer closed' == str(ex.value)
287+
288+
with pytest.raises(RuntimeError) as ex:
289+
c.offsets_for_times([TopicPartition("test", 0)])
290+
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