-
Notifications
You must be signed in to change notification settings - Fork 917
AvroConsumer for handling schema registry #80
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 2 commits
550688d
5b88a9e
f8ec601
4bac7cd
21c0b64
badd3bb
9797fef
95a9dbb
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 |
---|---|---|
|
@@ -60,6 +60,26 @@ avroProducer = AvroProducer({'bootstrap.servers': 'mybroker,mybroker2', 'schema. | |
avroProducer.produce(topic='my_topic', value=value, key=key) | ||
``` | ||
|
||
**AvroConsumer** | ||
``` | ||
from confluent_kafka.avro import AvroConsumer | ||
|
||
c = AvroConsumer({'bootstrap.servers': 'mybroker,mybroker2', 'group.id': 'groupid', 'schema.registry.url': 'http://127.0.0.1:8081'}) | ||
c.subscribe(['my_topic']) | ||
running = True | ||
while running: | ||
msg = c.poll(10) | ||
if msg: | ||
if not msg.error(): | ||
if (msg.value() && isinstance(msg.value(), dict)): | ||
print("Got back deserialized dict object") | ||
print(msg.value()) | ||
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. Do we want an example that actually uses the deserialized type? This looks just like a standard non-avro consumer. 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. @edenhill How do we show the type of msg? msg value is going to be a dict object. 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. No idea, but it is boring if the example is no different than the vanilla example, right? :) |
||
elif msg.error().code() != KafkaError._PARTITION_EOF: | ||
print(msg.error()) | ||
running = False | ||
c.close() | ||
``` | ||
|
||
See [examples](examples) for more examples. | ||
|
||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -4,7 +4,7 @@ | |
""" | ||
import sys | ||
|
||
from confluent_kafka import Producer | ||
from confluent_kafka import Producer, Consumer | ||
|
||
VALID_LEVELS = ['NONE', 'FULL', 'FORWARD', 'BACKWARD'] | ||
|
||
|
@@ -112,3 +112,43 @@ def produce(self, **kwargs): | |
raise SerializerError("Avro schema required for key") | ||
|
||
super(AvroProducer, self).produce(topic, value, key, **kwargs) | ||
|
||
|
||
class AvroConsumer(Consumer): | ||
""" | ||
Kafka Consumer client which does avro schema decoding of messages. | ||
Handles message deserialization. | ||
|
||
Constructor takes below parameters | ||
|
||
@:param: config: dict object with config parameters containing url for schema registry (schema.registry.url). | ||
""" | ||
def __init__(self, config): | ||
|
||
if ('schema.registry.url' not in config.keys()): | ||
raise ValueError("Missing parameter: schema.registry.url") | ||
schem_registry_url = config["schema.registry.url"] | ||
del config["schema.registry.url"] | ||
|
||
super(AvroConsumer, self).__init__(config) | ||
self._serializer = MessageSerializer(CachedSchemaRegistryClient(url=schem_registry_url)) | ||
|
||
def poll(self, timeout): | ||
""" | ||
This is an overriden method from confluent_kafka.Consumer class. This handles message | ||
deserialization using avro schema | ||
|
||
@:param timeout | ||
@:return message object with deserialized key and value as dict objects | ||
""" | ||
message = super(AvroConsumer, self).poll(timeout) | ||
if not message: | ||
return message | ||
if not message.error(): | ||
if message.value(): | ||
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. I think you want to check |
||
decoded_value = self._serializer.decode_message(message.value()) | ||
message.set_value(decoded_value) | ||
if message.key(): | ||
decoded_key = self._serializer.decode_message(message.key()) | ||
message.set_key(decoded_key) | ||
return message |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -332,6 +332,23 @@ static PyObject *Message_timestamp (Message *self, PyObject *ignore) { | |
self->timestamp); | ||
} | ||
|
||
static PyObject *Message_set_value (Message *self, PyObject *new_val) { | ||
if (self->value) { | ||
Py_DECREF(self->value); | ||
self->value = new_val; | ||
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. This only sets a new value if there was an old one, which isnt right. |
||
Py_INCREF(self->value); | ||
} | ||
Py_RETURN_NONE; | ||
} | ||
|
||
static PyObject *Message_set_key (Message *self, PyObject *new_key) { | ||
if (self->key) { | ||
Py_DECREF(self->key); | ||
self->key = new_key; | ||
Py_INCREF(self->key); | ||
} | ||
Py_RETURN_NONE; | ||
} | ||
|
||
static PyMethodDef Message_methods[] = { | ||
{ "error", (PyCFunction)Message_error, METH_NOARGS, | ||
|
@@ -391,6 +408,18 @@ static PyMethodDef Message_methods[] = { | |
" :rtype: (int, int)\n" | ||
"\n" | ||
}, | ||
{ "set_value", (PyCFunction)Message_set_value, METH_O, | ||
" Set the field 'Message.value' with new value.\n" | ||
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. Document param and type, see an example here: |
||
" :returns: None.\n" | ||
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. Missing docstring saying what the method actually does |
||
" :rtype: None\n" | ||
"\n" | ||
}, | ||
{ "set_key", (PyCFunction)Message_set_key, METH_O, | ||
" Set the field 'Message.key' with new value.\n" | ||
" :returns: None.\n" | ||
" :rtype: None\n" | ||
"\n" | ||
}, | ||
{ NULL } | ||
}; | ||
|
||
|
@@ -1217,7 +1246,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype, | |
Py_DECREF(ks); | ||
return NULL; | ||
} | ||
|
||
if (h->stats_cb) { | ||
Py_DECREF(h->stats_cb); | ||
h->stats_cb = NULL; | ||
|
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.
Is this ever not true?
From what I understand the deserializer will raise an exception if it can't deserialize a message.
Speaking of which, this example should probably have a proper try:..except block catching such errors, right?