Skip to content

Commit c8ae724

Browse files
committed
Add auto promototing client class
1 parent 4827c5a commit c8ae724

File tree

4 files changed

+114
-44
lines changed

4 files changed

+114
-44
lines changed

confluent_kafka/consumer.py

Lines changed: 44 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,34 @@
2222

2323

2424
class Consumer(_impl):
25+
"""
26+
Create a new Kafka Consumer instance with or without serializer support.
27+
28+
To avoid spontaneous calls from non-Python threads all callbacks will only be served upon
29+
calling ```client.poll()``` or ```client.flush()```.
30+
31+
:param dict conf: Configuration properties.
32+
See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md for more information.
33+
:param func key_deserializer(topic, key): Converts message key bytes to object.
34+
**note** deserializers are responsible for handling NULL keys
35+
:param func value_deserializer(topic, value): Converts message value bytes to object.
36+
**note** deserializers are responsible for handling NULL values
37+
:param func on_commit(err, [partitions]): Callback used to indicate success or failure
38+
of an offset commit.
39+
:param func stats_cb(json_str): Callback for statistics emitted every ``statistics.interval.ms``.
40+
See https://github.com/edenhill/librdkafka/wiki/Statistics” for more information.
41+
:param func throttle_cb(confluent_kafka.ThrottleEvent): Callback for throttled request reporting.
42+
:param logging.handlers logger: Forwards logs from the Kafka client to the provided handler instance.
43+
Log messages will only be forwarded when ``client.poll()`` or ``producer.flush()`` are called.
44+
:raises TypeError: If conf is not a dict.
45+
"""
46+
def __new__(cls, *args, **kwargs):
47+
if 'key_deserializer' in kwargs or 'value_deserializer' in kwargs:
48+
return super(Consumer, cls).__new__(DeserializingConsumer, *args, **kwargs)
49+
return super(Consumer, cls).__new__(cls, *args, **kwargs)
50+
51+
52+
class DeserializingConsumer(Consumer):
2553
"""
2654
Create a new Kafka Consumer instance.
2755
@@ -78,10 +106,11 @@ def __init__(self, conf={}, key_deserializer=None, value_deserializer=None,
78106

79107
if not logger:
80108
logger = conf.get('logger', None)
81-
super(Consumer, self).__init__(conf, on_commit=on_commit, stats_cb=stats_cb,
82-
throttle_cb=throttle_cb, logger=logger)
83109

84-
def poll(self, timeout=-1.0, key_deserializer=None, value_deserializer=None):
110+
super(DeserializingConsumer, self).__init__(conf, on_commit=on_commit, stats_cb=stats_cb,
111+
throttle_cb=throttle_cb, logger=logger)
112+
113+
def poll(self, timeout=-1.0):
85114
"""
86115
Consumes a message, triggers callbacks, returns an event.
87116
@@ -97,25 +126,15 @@ def poll(self, timeout=-1.0, key_deserializer=None, value_deserializer=None):
97126
:raises RuntimeError: If called on a closed consumer.
98127
"""
99128

100-
msg = super(Consumer, self).poll(timeout)
129+
msg = super(DeserializingConsumer, self).poll(timeout)
101130

102-
if not msg or msg.error():
131+
if msg is None or msg.error():
103132
return msg
104133

105134
topic = msg.topic()
106135

107-
# parameter overrides take precedence over instance functions
108-
if not key_deserializer:
109-
key_deserializer = self._key_deserializer
110-
111-
if key_deserializer:
112-
msg.set_key(key_deserializer(topic, msg.key()))
113-
114-
if not value_deserializer:
115-
value_deserializer = self._value_deserializer
116-
117-
if value_deserializer:
118-
msg.set_value(value_deserializer(topic, msg.value()))
136+
msg.set_key(self._key_deserializer(topic, msg.key()))
137+
msg.set_value(self._value_deserializer(topic, msg.value()))
119138

120139
return msg
121140

@@ -138,8 +157,12 @@ def consume(self, num_messages=1, timeout=-1):
138157
:raises ValueError: If num_messages > 1M.
139158
"""
140159

141-
# Disable consume() method when serializers are in use.
142-
if self._key_deserializer or self._value_deserializer:
143-
raise(NotImplementedError, "Batch consumption does not support the use of deserializers")
160+
msgset = super(DeserializingConsumer, self).consume(num_messages, timeout)
161+
for msg in msgset:
162+
if msg.error():
163+
continue
164+
165+
msg.set_key(self._key_deserializer(topic, msg.key()))
166+
msg.set_value(self._value_deserializer(topic, msg.value()))
144167

145-
return super(Consumer, self).consume(num_messages, timeout)
168+
return msgset

confluent_kafka/producer.py

Lines changed: 36 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,35 @@
2323

2424
class Producer(_impl):
2525
"""
26-
Create a new Kafka Producer instance.
26+
Create a new Kafka Producer instance with or without serializer support.
27+
28+
To avoid spontaneous calls from non-Python threads all callbacks will only be served upon
29+
calling ```client.poll()``` or ```client.flush()```.
30+
31+
:param dict conf: Configuration properties. At a minimum ``bootstrap.servers`` **should** be set.
32+
See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md for more information.
33+
:param func key_serializer(topic, key): Converts key to bytes.
34+
**note** serializers are responsible for handling NULL keys
35+
:param func value_serializer(topic, value): Converts value to bytes.
36+
**note** serializers are responsible for handling NULL keys
37+
:param func error_cb(kafka.KafkaError): Callback for generic/global error events.
38+
:param func stats_cb(json_str): Callback for statistics emitted every ``statistics.interval.ms``.
39+
See https://github.com/edenhill/librdkafka/wiki/Statistics” for more information.
40+
:param func throttle_cb(confluent_kafka.ThrottleEvent): Callback for throttled request reporting.
41+
:param logging.handlers logger: Forwards logs from the Kafka client to the provided handler instance.
42+
Log messages will only be forwarded when ``client.poll()`` or ``producer.flush()`` are called.
43+
:raises TypeError: If conf is not a dict object.
44+
"""
45+
46+
def __new__(cls, *args, **kwargs):
47+
if 'key_serializer' in kwargs or 'value_serializer' in kwargs:
48+
return super(Producer, cls).__new__(SerializingProducer, *args, **kwargs)
49+
return super(Producer, cls).__new__(cls, *args, **kwargs)
50+
51+
52+
class SerializingProducer(Producer):
53+
"""
54+
Create a new Kafka Producer instance with custom serialization support.
2755
2856
To avoid spontaneous calls from non-Python threads all callbacks will only be served upon
2957
calling ```client.poll()``` or ```client.flush()```.
@@ -78,12 +106,11 @@ def __init__(self, conf={}, key_serializer=None, value_serializer=None,
78106
if not logger:
79107
logger = conf.get('logger', None)
80108

81-
super(Producer, self).__init__(conf, error_cb=error_cb, stats_cb=stats_cb,
82-
throttle_cb=throttle_cb, logger=logger)
109+
super(SerializingProducer, self).__init__(conf, error_cb=error_cb, stats_cb=stats_cb,
110+
throttle_cb=throttle_cb, logger=logger)
83111

84112
def produce(self, topic, value=None, key=None, partition=PARTITION_UA,
85-
on_delivery=None, callback=None, timestamp=0, headers=None,
86-
key_serializer=None, value_serializer=None):
113+
on_delivery=None, callback=None, timestamp=0, headers=None):
87114
"""
88115
Produces message to Kafka
89116
@@ -109,22 +136,8 @@ def produce(self, topic, value=None, key=None, partition=PARTITION_UA,
109136
110137
"""
111138

112-
# on_delivery is an alias for callback and take precedence
113-
if callback and not on_delivery:
114-
on_delivery = callback
115-
116-
# parameter overrides take precedence over instance functions
117-
if not key_serializer:
118-
key_serializer = self._key_serializer
119-
120-
if key_serializer:
121-
key = key_serializer(topic, key)
122-
123-
if not value_serializer:
124-
value_serializer = self._value_serializer
125-
126-
if value_serializer:
127-
value = value_serializer(topic, value)
139+
key = self._key_serializer(topic, key)
140+
value = self._value_serializer(topic, value)
128141

129-
super(Producer, self).produce(topic, value, key, partition, on_delivery=on_delivery,
130-
timestamp=timestamp, headers=headers)
142+
super(SerializingProducer, self).produce(topic, value, key, partition, on_delivery=on_delivery,
143+
callback=callback, timestamp=timestamp, headers=headers)

tests/test_Consumer.py

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@
33
from confluent_kafka import (Consumer, TopicPartition, KafkaError,
44
KafkaException, TIMESTAMP_NOT_AVAILABLE,
55
OFFSET_INVALID, libversion)
6+
from confluent_kafka.consumer import DeserializingConsumer
67
import pytest
78

89

@@ -321,3 +322,19 @@ def test_consumer_withot_groupid():
321322
with pytest.raises(ValueError) as ex:
322323
Consumer({'bootstrap.servers': "mybroker:9092"})
323324
assert 'group.id must be set' in str(ex)
325+
326+
327+
@pytest.mark.parametrize("test_param", [
328+
({"key_deserializer": lambda key: key}, DeserializingConsumer),
329+
({"value_deserializer": lambda value: value}, DeserializingConsumer),
330+
({"key_deserializer": lambda key: key,
331+
"value_deserializer": lambda value: value}, DeserializingConsumer),
332+
({}, Consumer)
333+
])
334+
def test_producer_factory(test_param):
335+
"""
336+
Assert Consumer factory returns the correct class
337+
"""
338+
339+
c = Consumer({'group.id': 'wrapper'}, **test_param[0])
340+
assert(type(c) is test_param[1])

tests/test_Producer.py

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@
33
import pytest
44

55
from confluent_kafka import Producer, KafkaError, KafkaException, libversion
6+
from confluent_kafka.producer import SerializingProducer
67
from struct import pack
78

89

@@ -191,3 +192,19 @@ def test_set_invalid_partitioner_murmur():
191192
with pytest.raises(KafkaException) as e:
192193
Producer({'partitioner': 'murmur'})
193194
assert 'Invalid value for configuration property "partitioner": murmur' in str(e)
195+
196+
197+
@pytest.mark.parametrize("test_param", [
198+
({"key_serializer": lambda key: key}, SerializingProducer),
199+
({"value_serializer": lambda value: value}, SerializingProducer),
200+
({"key_serializer": lambda key: key,
201+
"value_serializer": lambda value: value}, SerializingProducer),
202+
({}, Producer)
203+
])
204+
def test_producer_factory(test_param):
205+
"""
206+
Assert Producer factory returns the correct class
207+
"""
208+
209+
p = Producer(**test_param[0])
210+
assert(type(p) is test_param[1])

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