Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
Next Next commit
Refactor Consumer and Producer into common Handle for code reuse.
  • Loading branch information
edenhill committed Jul 13, 2016
commit fdaa0d4aa109639924bc44e6e0e0ce0ac75369d2
111 changes: 60 additions & 51 deletions confluent_kafka/src/Consumer.c
Original file line number Diff line number Diff line change
Expand Up @@ -28,23 +28,26 @@
****************************************************************************/


static int Consumer_clear (Consumer *self) {
if (self->on_assign) {
Py_DECREF(self->on_assign);
self->on_assign = NULL;
static int Consumer_clear (Handle *self) {
if (self->u.Consumer.on_assign) {
Py_DECREF(self->u.Consumer.on_assign);
self->u.Consumer.on_assign = NULL;
}
if (self->on_revoke) {
Py_DECREF(self->on_revoke);
self->on_revoke = NULL;
if (self->u.Consumer.on_revoke) {
Py_DECREF(self->u.Consumer.on_revoke);
self->u.Consumer.on_revoke = NULL;
}
if (self->on_commit) {
Py_DECREF(self->on_commit);
self->on_commit = NULL;
if (self->u.Consumer.on_commit) {
Py_DECREF(self->u.Consumer.on_commit);
self->u.Consumer.on_commit = NULL;
}

Handle_clear(self);

return 0;
}

static void Consumer_dealloc (Consumer *self) {
static void Consumer_dealloc (Handle *self) {
PyObject_GC_UnTrack(self);

Consumer_clear(self);
Expand All @@ -55,12 +58,15 @@ static void Consumer_dealloc (Consumer *self) {
Py_TYPE(self)->tp_free((PyObject *)self);
}

static int Consumer_traverse (Consumer *self,
visitproc visit, void *arg) {
if (self->on_assign)
Py_VISIT(self->on_assign);
if (self->on_revoke)
Py_VISIT(self->on_revoke);
static int Consumer_traverse (Handle *self,
visitproc visit, void *arg) {
if (self->u.Consumer.on_assign)
Py_VISIT(self->u.Consumer.on_assign);
if (self->u.Consumer.on_revoke)
Py_VISIT(self->u.Consumer.on_revoke);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are we missing visiting on_commit?


Handle_traverse(self, visit, arg);

return 0;
}

Expand All @@ -69,7 +75,7 @@ static int Consumer_traverse (Consumer *self,



static PyObject *Consumer_subscribe (Consumer *self, PyObject *args,
static PyObject *Consumer_subscribe (Handle *self, PyObject *args,
PyObject *kwargs) {

rd_kafka_topic_partition_list_t *topics;
Expand Down Expand Up @@ -130,29 +136,29 @@ static PyObject *Consumer_subscribe (Consumer *self, PyObject *args,
/*
* Update rebalance callbacks
*/
if (self->on_assign) {
Py_DECREF(self->on_assign);
self->on_assign = NULL;
if (self->u.Consumer.on_assign) {
Py_DECREF(self->u.Consumer.on_assign);
self->u.Consumer.on_assign = NULL;
}
if (on_assign) {
self->on_assign = on_assign;
Py_INCREF(self->on_assign);
self->u.Consumer.on_assign = on_assign;
Py_INCREF(self->u.Consumer.on_assign);
}

if (self->on_revoke) {
Py_DECREF(self->on_revoke);
self->on_revoke = NULL;
if (self->u.Consumer.on_revoke) {
Py_DECREF(self->u.Consumer.on_revoke);
self->u.Consumer.on_revoke = NULL;
}
if (on_revoke) {
self->on_revoke = on_revoke;
Py_INCREF(self->on_revoke);
self->u.Consumer.on_revoke = on_revoke;
Py_INCREF(self->u.Consumer.on_revoke);
}

Py_RETURN_NONE;
}


static PyObject *Consumer_unsubscribe (Consumer *self,
static PyObject *Consumer_unsubscribe (Handle *self,
PyObject *ignore) {

rd_kafka_resp_err_t err;
Expand All @@ -169,15 +175,15 @@ static PyObject *Consumer_unsubscribe (Consumer *self,
}


static PyObject *Consumer_assign (Consumer *self, PyObject *tlist) {
static PyObject *Consumer_assign (Handle *self, PyObject *tlist) {

rd_kafka_topic_partition_list_t *c_parts;
rd_kafka_resp_err_t err;

if (!(c_parts = py_to_c_parts(tlist)))
return NULL;

self->rebalance_assigned++;
self->u.Consumer.rebalance_assigned++;

err = rd_kafka_assign(self->rk, c_parts);

Expand All @@ -194,11 +200,11 @@ static PyObject *Consumer_assign (Consumer *self, PyObject *tlist) {
}


static PyObject *Consumer_unassign (Consumer *self, PyObject *ignore) {
static PyObject *Consumer_unassign (Handle *self, PyObject *ignore) {

rd_kafka_resp_err_t err;

self->rebalance_assigned++;
self->u.Consumer.rebalance_assigned++;

err = rd_kafka_assign(self->rk, NULL);
if (err) {
Expand All @@ -213,7 +219,7 @@ static PyObject *Consumer_unassign (Consumer *self, PyObject *ignore) {



static PyObject *Consumer_commit (Consumer *self, PyObject *args,
static PyObject *Consumer_commit (Handle *self, PyObject *args,
PyObject *kwargs) {

rd_kafka_resp_err_t err;
Expand Down Expand Up @@ -281,7 +287,7 @@ static PyObject *Consumer_commit (Consumer *self, PyObject *args,



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

PyObject *plist;
Expand Down Expand Up @@ -317,7 +323,7 @@ static PyObject *Consumer_committed (Consumer *self, PyObject *args,
}


static PyObject *Consumer_position (Consumer *self, PyObject *args,
static PyObject *Consumer_position (Handle *self, PyObject *args,
PyObject *kwargs) {

PyObject *plist;
Expand Down Expand Up @@ -352,7 +358,7 @@ static PyObject *Consumer_position (Consumer *self, PyObject *args,



static PyObject *Consumer_poll (Consumer *self, PyObject *args,
static PyObject *Consumer_poll (Handle *self, PyObject *args,
PyObject *kwargs) {
double tmout = -1.0f;
static char *kws[] = { "timeout", NULL };
Expand Down Expand Up @@ -384,7 +390,7 @@ static PyObject *Consumer_poll (Consumer *self, PyObject *args,
}


static PyObject *Consumer_close (Consumer *self, PyObject *ignore) {
static PyObject *Consumer_close (Handle *self, PyObject *ignore) {
self->thread_state = PyEval_SaveThread();
rd_kafka_consumer_close(self->rk);
PyEval_RestoreThread(self->thread_state);
Expand Down Expand Up @@ -523,7 +529,7 @@ static PyObject *Consumer_new (PyTypeObject *type, PyObject *args,
PyTypeObject ConsumerType = {
PyVarObject_HEAD_INIT(NULL, 0)
"cimpl.Consumer", /*tp_name*/
sizeof(Consumer), /*tp_basicsize*/
sizeof(Handle), /*tp_basicsize*/
0, /*tp_itemsize*/
(destructor)Consumer_dealloc, /*tp_dealloc*/
0, /*tp_print*/
Expand Down Expand Up @@ -584,14 +590,16 @@ PyTypeObject ConsumerType = {
static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
rd_kafka_topic_partition_list_t *c_parts,
void *opaque) {
Consumer *self = opaque;
Handle *self = opaque;

PyEval_RestoreThread(self->thread_state);

self->rebalance_assigned = 0;
self->u.Consumer.rebalance_assigned = 0;

if ((err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS && self->on_assign) ||
(err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS && self->on_revoke)) {
if ((err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS &&
self->u.Consumer.on_assign) ||
(err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS &&
self->u.Consumer.on_revoke)) {
PyObject *parts;
PyObject *args, *result;

Expand All @@ -612,7 +620,8 @@ static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,

result = PyObject_CallObject(
err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS ?
self->on_assign : self->on_revoke, args);
self->u.Consumer.on_assign :
self->u.Consumer.on_revoke, args);

Py_DECREF(args);

Expand All @@ -628,7 +637,7 @@ static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
* to synchronize state, if the user did not do this from callback,
* or there was no callback, or the callback failed, then we perform
* that assign() call here instead. */
if (!self->rebalance_assigned) {
if (!self->u.Consumer.rebalance_assigned) {
if (err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS)
rd_kafka_assign(rk, c_parts);
else
Expand All @@ -642,10 +651,10 @@ static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
static void Consumer_offset_commit_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
rd_kafka_topic_partition_list_t *c_parts,
void *opaque) {
Consumer *self = opaque;
Handle *self = opaque;
PyObject *parts, *k_err, *args, *result;

if (!self->on_commit)
if (!self->u.Consumer.on_commit)
return;

PyEval_RestoreThread(self->thread_state);
Expand All @@ -669,7 +678,7 @@ static void Consumer_offset_commit_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
return;
}

result = PyObject_CallObject(self->on_commit, args);
result = PyObject_CallObject(self->u.Consumer.on_commit, args);

Py_DECREF(args);

Expand All @@ -687,16 +696,16 @@ static void Consumer_offset_commit_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,

static PyObject *Consumer_new (PyTypeObject *type, PyObject *args,
PyObject *kwargs) {
Consumer *self;
Handle *self;
char errstr[256];
rd_kafka_conf_t *conf;

self = (Consumer *)ConsumerType.tp_alloc(&ConsumerType, 0);
self = (Handle *)ConsumerType.tp_alloc(&ConsumerType, 0);
if (!self)
return NULL;

if (!(conf = common_conf_setup(RD_KAFKA_CONSUMER, self,
args, kwargs))) {
args, kwargs))) {
Py_DECREF(self);
return NULL;
}
Expand Down
Loading