Skip to content

Added Consumer.store_offsets() API #245

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 4 commits into from
Sep 15, 2017
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
92 changes: 92 additions & 0 deletions confluent_kafka/src/Consumer.c
Original file line number Diff line number Diff line change
Expand Up @@ -325,6 +325,82 @@ static PyObject *Consumer_commit (Handle *self, PyObject *args,



static PyObject *Consumer_store_offsets (Handle *self, PyObject *args,
PyObject *kwargs) {

#if RD_KAFKA_VERSION < 0x000b0000
PyErr_Format(PyExc_NotImplementedError,
"Consumer store_offsets require "
"confluent-kafka-python built for librdkafka "
"version >=v0.11.0 (librdkafka runtime 0x%x, "
"buildtime 0x%x)",
rd_kafka_version(), RD_KAFKA_VERSION);
return NULL;
#else
rd_kafka_resp_err_t err;
PyObject *msg = NULL, *offsets = NULL;
rd_kafka_topic_partition_list_t *c_offsets;
static char *kws[] = { "message", "offsets", NULL };


if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|OO", kws,
&msg, &offsets))
return NULL;

if (msg && offsets) {
PyErr_SetString(PyExc_ValueError,
"message and offsets are mutually exclusive");
return NULL;
}

if (!msg && !offsets) {
PyErr_SetString(PyExc_ValueError,
"expected either message or offsets");
return NULL;
}

if (offsets) {

if (!(c_offsets = py_to_c_parts(offsets)))
return NULL;
} else {
Message *m;
PyObject *uo8;

if (PyObject_Type((PyObject *)msg) !=
(PyObject *)&MessageType) {
PyErr_Format(PyExc_TypeError,
"expected %s", MessageType.tp_name);
return NULL;
}

m = (Message *)msg;

c_offsets = rd_kafka_topic_partition_list_new(1);
rd_kafka_topic_partition_list_add(
c_offsets, cfl_PyUnistr_AsUTF8(m->topic, &uo8),
m->partition)->offset = m->offset + 1;
Py_XDECREF(uo8);
}


err = rd_kafka_offsets_store(self->rk, c_offsets);
rd_kafka_topic_partition_list_destroy(c_offsets);



if (err) {
cfl_PyErr_Format(err,
"StoreOffsets failed: %s", rd_kafka_err2str(err));
return NULL;
}

Py_RETURN_NONE;
#endif
}



static PyObject *Consumer_committed (Handle *self, PyObject *args,
PyObject *kwargs) {

Expand Down Expand Up @@ -570,6 +646,22 @@ static PyMethodDef Consumer_methods[] = {
" :raises: KafkaException\n"
"\n"
},
{ "store_offsets", (PyCFunction)Consumer_store_offsets, METH_VARARGS|METH_KEYWORDS,
".. py:function:: store_offsets([message=None], [offsets=None])\n"
"\n"
" Store offsets for a message or a list of offsets.\n"
"\n"
" ``message`` and ``offsets`` are mutually exclusive. "
"The stored offsets will be committed according to 'auto.commit.interval.ms' or manual "
"offset-less :py:meth:`commit`. "
"Note that 'enable.auto.offset.store' must be set to False when using this API.\n"
"\n"
" :param confluent_kafka.Message message: Store message's offset+1.\n"
" :param list(TopicPartition) offsets: List of topic+partitions+offsets to store.\n"
" :rtype: None\n"
" :raises: KafkaException\n"
"\n"
},
{ "commit", (PyCFunction)Consumer_commit, METH_VARARGS|METH_KEYWORDS,
".. py:function:: commit([message=None], [offsets=None], [async=True])\n"
"\n"
Expand Down
22 changes: 22 additions & 0 deletions tests/test_Consumer.py
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,28 @@ def dummy_assign_revoke(consumer, partitions):
kc.close()


@pytest.mark.skipif(libversion()[1] < 0x000b0000,
reason="requires librdkafka >=0.11.0")
def test_store_offsets():
""" Basic store_offsets() tests """

c = Consumer({'group.id': 'test',
'enable.auto.commit': True,
'enable.auto.offset.store': False,
'socket.timeout.ms': 50,
'session.timeout.ms': 100})

c.subscribe(["test"])

try:
c.store_offsets(offsets=[TopicPartition("test", 0, 42)])
except KafkaException as e:
assert e.args[0].code() == KafkaError._UNKNOWN_PARTITION

c.unsubscribe()
c.close()


# librdkafka <=0.9.2 has a race-issue where it will hang indefinately
# if a commit is issued when no coordinator is available.
@pytest.mark.skipif(libversion()[1] <= 0x000902ff,
Expand Down