-
Notifications
You must be signed in to change notification settings - Fork 917
Add timestamp() to librdkafka messages #50
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -326,6 +326,13 @@ static PyObject *Message_offset (Message *self, PyObject *ignore) { | |
} | ||
|
||
|
||
static PyObject *Message_timestamp (Message *self, PyObject *ignore) { | ||
return Py_BuildValue("iL", | ||
self->tstype, | ||
self->timestamp); | ||
} | ||
|
||
|
||
static PyMethodDef Message_methods[] = { | ||
{ "error", (PyCFunction)Message_error, METH_NOARGS, | ||
" The message object is also used to propagate errors and events, " | ||
|
@@ -362,6 +369,11 @@ static PyMethodDef Message_methods[] = { | |
" :rtype: int or None\n" | ||
"\n" | ||
}, | ||
{ "timestamp", (PyCFunction)Message_timestamp, METH_NOARGS, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It needs to return the timestamp type as well (none, append, create). There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please add the timestamp type as a class level constant, e.g.: |
||
" :returns: tuple of message tstype, and timestamp.\n" | ||
" :rtype: (int, int)\n" | ||
"\n" | ||
}, | ||
{ NULL } | ||
}; | ||
|
||
|
@@ -495,6 +507,8 @@ PyObject *Message_new0 (const rd_kafka_message_t *rkm) { | |
self->partition = rkm->partition; | ||
self->offset = rkm->offset; | ||
|
||
self->timestamp = rd_kafka_message_timestamp(rkm, &self->tstype); | ||
|
||
return (PyObject *)self; | ||
} | ||
|
||
|
@@ -1419,6 +1433,10 @@ static PyObject *_init_cimpl (void) { | |
Py_INCREF(KafkaException); | ||
PyModule_AddObject(m, "KafkaException", KafkaException); | ||
|
||
PyModule_AddIntConstant(m, "TIMESTAMP_NOT_AVAILABLE", RD_KAFKA_TIMESTAMP_NOT_AVAILABLE); | ||
PyModule_AddIntConstant(m, "TIMESTAMP_CREATE_TIME", RD_KAFKA_TIMESTAMP_CREATE_TIME); | ||
PyModule_AddIntConstant(m, "TIMESTAMP_LOG_APPEND_TIME", RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME); | ||
|
||
return m; | ||
} | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think it would be better to have a single timestamp() method that returns a tuple: tstype, timestamp