Skip to content

Commit 7333f2a

Browse files
committed
Make derived producer/consumer non-instantiable
1 parent c8ae724 commit 7333f2a

File tree

4 files changed

+60
-37
lines changed

4 files changed

+60
-37
lines changed

confluent_kafka/consumer.py

Lines changed: 22 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -21,9 +21,17 @@
2121
from warnings import warn
2222

2323

24+
def byteDeserializer(topic, payload):
25+
"""
26+
byteDeserializer returns an unaltered payload to the caller
27+
"""
28+
29+
return payload
30+
31+
2432
class Consumer(_impl):
2533
"""
26-
Create a new Kafka Consumer instance with or without serializer support.
34+
Create a new Kafka Consumer instance.
2735
2836
To avoid spontaneous calls from non-Python threads all callbacks will only be served upon
2937
calling ```client.poll()``` or ```client.flush()```.
@@ -51,31 +59,24 @@ def __new__(cls, *args, **kwargs):
5159

5260
class DeserializingConsumer(Consumer):
5361
"""
54-
Create a new Kafka Consumer instance.
62+
DeserializingConsumer extends Consumer with configurable key and value deserializer.
5563
56-
To avoid spontaneous calls from non-Python threads all callbacks will only be served upon
57-
calling ```client.poll()``` or ```client.flush()```.
64+
Instances of DeserializingConsumer cannot be created directly.
65+
To obtain an instance of this class instantiate a Consumer with a key and/or value deserializer.
66+
67+
Duplicate params have been omitted for brevity. See Consumer for class documentation.
5868
59-
:param dict conf: Configuration properties.
60-
See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md for more information.
61-
:param func key_deserializer(topic, key): Converts message key bytes to object.
62-
**note** deserializers are responsible for handling NULL keys
63-
:param func value_deserializer(topic, value): Converts message value bytes to object.
64-
**note** deserializers are responsible for handling NULL values
65-
:param func on_commit(err, [partitions]): Callback used to indicate success or failure
66-
of an offset commit.
67-
:param func stats_cb(json_str): Callback for statistics emitted every ``statistics.interval.ms``.
68-
See https://github.com/edenhill/librdkafka/wiki/Statistics” for more information.
69-
:param func throttle_cb(confluent_kafka.ThrottleEvent): Callback for throttled request reporting.
70-
:param logging.handlers logger: Forwards logs from the Kafka client to the provided handler instance.
71-
Log messages will only be forwarded when ``client.poll()`` or ``producer.flush()`` are called.
7269
:raises TypeError: If conf is not a dict.
70+
:raises TypeError: If instantiated directly.
7371
"""
7472

7573
__slots__ = ["_key_deserializer", "_value_deserializer"]
7674

75+
def __new__(cls, *args, **kwargs):
76+
raise TypeError("DeserializingConsumer is a non user-instantiable class")
77+
7778
# conf must remain optional as long as kwargs are supported
78-
def __init__(self, conf={}, key_deserializer=None, value_deserializer=None,
79+
def __init__(self, conf={}, key_deserializer=byteDeserializer, value_deserializer=byteDeserializer,
7980
on_commit=None, stats_cb=None, throttle_cb=None, logger=None, **kwargs):
8081

8182
if not isinstance(conf, dict):
@@ -143,15 +144,14 @@ def consume(self, num_messages=1, timeout=-1):
143144
Consume messages, calls callbacks and returns a list of messages. (possibly empty on timeout)
144145
145146
The application must check Message.error() to distinguish between
146-
proper messages, an error(see error().code() for specifics), or an event. for each
147+
proper messages, an error(see error().code() for specifics), or an event for each
147148
Message in the list.
148149
149150
:param int num_messages: Maximum number of messages to return (default: 1)
150151
:param float timeout: Maximum time in seconds to block waiting for message, event or callback.
151152
(default: infinite (-1))
152153
:returns: A list of Message objects (possibly empty on timeout)
153154
:rtype: list(Message)
154-
:raises NotImplementedError: If used with key/value serializers.
155155
:raises RuntimeError: If called on a closed consumer.
156156
:raises KafkaError: In case of internal error.
157157
:raises ValueError: If num_messages > 1M.
@@ -162,7 +162,7 @@ def consume(self, num_messages=1, timeout=-1):
162162
if msg.error():
163163
continue
164164

165-
msg.set_key(self._key_deserializer(topic, msg.key()))
166-
msg.set_value(self._value_deserializer(topic, msg.value()))
165+
msg.set_key(self._key_deserializer(msg.topic(), msg.key()))
166+
msg.set_value(self._value_deserializer(msg.topic(), msg.value()))
167167

168168
return msgset

confluent_kafka/producer.py

Lines changed: 20 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,14 @@
2121
from warnings import warn
2222

2323

24+
def byteSerializer(topic, payload):
25+
"""
26+
byteSerializer returns an unaltered payload to the caller
27+
"""
28+
29+
return payload
30+
31+
2432
class Producer(_impl):
2533
"""
2634
Create a new Kafka Producer instance with or without serializer support.
@@ -51,30 +59,27 @@ def __new__(cls, *args, **kwargs):
5159

5260
class SerializingProducer(Producer):
5361
"""
54-
Create a new Kafka Producer instance with custom serialization support.
62+
SerializingProducer extends Producer with configurable key and value serializer.
5563
5664
To avoid spontaneous calls from non-Python threads all callbacks will only be served upon
5765
calling ```client.poll()``` or ```client.flush()```.
5866
59-
:param dict conf: Configuration properties. At a minimum ``bootstrap.servers`` **should** be set.
60-
See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md for more information.
61-
:param func key_serializer(topic, key): Converts key to bytes.
62-
**note** serializers are responsible for handling NULL keys
63-
:param func value_serializer(topic, value): Converts value to bytes.
64-
**note** serializers are responsible for handling NULL keys
65-
:param func error_cb(kafka.KafkaError): Callback for generic/global error events.
66-
:param func stats_cb(json_str): Callback for statistics emitted every ``statistics.interval.ms``.
67-
See https://github.com/edenhill/librdkafka/wiki/Statistics” for more information.
68-
:param func throttle_cb(confluent_kafka.ThrottleEvent): Callback for throttled request reporting.
69-
:param logging.handlers logger: Forwards logs from the Kafka client to the provided handler instance.
70-
Log messages will only be forwarded when ``client.poll()`` or ``producer.flush()`` are called.
71-
:raises TypeError: If conf is not a dict object.
67+
Instances of SerializingProducer cannot be created directly.
68+
To obtain an instance of this class instantiate a Consumer with a key and/or value deserializer.
69+
70+
Duplicate params have been omitted for brevity. See Consumer for class documentation.
71+
72+
:raises TypeError: If conf is not a dict.
73+
:raises TypeError: If instantiated directly.
7274
"""
7375

7476
__slots__ = ["_key_serializer", "_value_serializer"]
7577

78+
def __new__(cls, *args, **kwargs):
79+
raise TypeError("SerializingProducer is a non user-instantiable class")
80+
7681
# conf must remain optional as long as kwargs are supported
77-
def __init__(self, conf={}, key_serializer=None, value_serializer=None,
82+
def __init__(self, conf={}, key_serializer=byteSerializer, value_serializer=byteSerializer,
7883
error_cb=None, stats_cb=None, throttle_cb=None, logger=None, **kwargs):
7984

8085
if not isinstance(conf, dict):

tests/test_Consumer.py

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -338,3 +338,12 @@ def test_producer_factory(test_param):
338338

339339
c = Consumer({'group.id': 'wrapper'}, **test_param[0])
340340
assert(type(c) is test_param[1])
341+
342+
343+
def test_deserializing_consumer_instantiation():
344+
"""
345+
Assert DeserializingConsumer instances cannot be created directly
346+
"""
347+
with pytest.raises(TypeError) as e:
348+
DeserializingConsumer()
349+
assert 'DeserializingConsumer is a non user-instantiable class' in str(e)

tests/test_Producer.py

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -208,3 +208,12 @@ def test_producer_factory(test_param):
208208

209209
p = Producer(**test_param[0])
210210
assert(type(p) is test_param[1])
211+
212+
213+
def test_serializing_producer_instantiation():
214+
"""
215+
Assert SerializingProducer instances cannot be created directly
216+
"""
217+
with pytest.raises(TypeError) as e:
218+
SerializingProducer()
219+
assert 'SerializingProducer is a non user-instantiable class' in str(e)

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