Skip to content

Commit 7ef6a36

Browse files
committed
Raise Syntax error for duplicate consumer/producer constructor parameters
1 parent 7333f2a commit 7ef6a36

File tree

3 files changed

+51
-58
lines changed

3 files changed

+51
-58
lines changed

confluent_kafka/__init__.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
__all__ = ['cimpl', 'admin', 'avro', 'kafkatest']
1+
__all__ = ['cimpl', 'admin', 'avro', 'kafkatest', 'producer', 'consumer']
22
from .cimpl import (KafkaError, # noqa
33
KafkaException,
44
Message,

confluent_kafka/consumer.py

Lines changed: 29 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -21,14 +21,6 @@
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-
3224
class Consumer(_impl):
3325
"""
3426
Create a new Kafka Consumer instance.
@@ -53,15 +45,15 @@ class Consumer(_impl):
5345
"""
5446
def __new__(cls, *args, **kwargs):
5547
if 'key_deserializer' in kwargs or 'value_deserializer' in kwargs:
56-
return super(Consumer, cls).__new__(DeserializingConsumer, *args, **kwargs)
48+
return super(Consumer, cls).__new__(SerializingConsumer, *args, **kwargs)
5749
return super(Consumer, cls).__new__(cls, *args, **kwargs)
5850

5951

60-
class DeserializingConsumer(Consumer):
52+
class SerializingConsumer(Consumer):
6153
"""
62-
DeserializingConsumer extends Consumer with configurable key and value deserializer.
54+
SerializingConsumer extends Consumer with configurable key and value deserializer.
6355
64-
Instances of DeserializingConsumer cannot be created directly.
56+
Instances of SerializingConsumer cannot be created directly.
6557
To obtain an instance of this class instantiate a Consumer with a key and/or value deserializer.
6658
6759
Duplicate params have been omitted for brevity. See Consumer for class documentation.
@@ -73,10 +65,15 @@ class DeserializingConsumer(Consumer):
7365
__slots__ = ["_key_deserializer", "_value_deserializer"]
7466

7567
def __new__(cls, *args, **kwargs):
76-
raise TypeError("DeserializingConsumer is a non user-instantiable class")
68+
raise TypeError("SerializingConsumer is a non user-instantiable class")
69+
70+
@staticmethod
71+
def byteSerializer(topic, data):
72+
""" Pass-through serializer """
73+
return data
7774

7875
# conf must remain optional as long as kwargs are supported
79-
def __init__(self, conf={}, key_deserializer=byteDeserializer, value_deserializer=byteDeserializer,
76+
def __init__(self, conf={}, key_deserializer=None, value_deserializer=None,
8077
on_commit=None, stats_cb=None, throttle_cb=None, logger=None, **kwargs):
8178

8279
if not isinstance(conf, dict):
@@ -90,26 +87,27 @@ def __init__(self, conf={}, key_deserializer=byteDeserializer, value_deserialize
9087
"all keyword arguments must match the constructor signature explicitly.",
9188
category=DeprecationWarning, stacklevel=2)
9289

90+
# Ensure the default serializer cannot be overwritten with None on instantiation
91+
if key_deserializer is None:
92+
key_deserializer = SerializingConsumer.byteSerializer
93+
94+
if value_deserializer is None:
95+
value_deserializer = SerializingConsumer.byteSerializer
96+
9397
self._key_deserializer = key_deserializer
9498
self._value_deserializer = value_deserializer
9599

96100
# Callbacks can be set in the conf dict or *ideally* as parameters.
97-
# Handle both cases prior to passing along to _impl
98-
# If callbacks are configured in both places parameter values take precedence.
99-
if not on_commit:
100-
on_commit = conf.get('on_commit', None)
101-
102-
if not stats_cb:
103-
stats_cb = conf.get('stats_cb', None)
104-
105-
if not throttle_cb:
106-
throttle_cb = conf.get('throttle_cb', None)
107-
108-
if not logger:
109-
logger = conf.get('logger', None)
101+
# Raise a SyntaxError if a callback is set in both places.
102+
for var, name in [(logger, 'logger'), (on_commit, 'on_commit'),
103+
(stats_cb, 'stats_cb'), (throttle_cb, 'throttle_cb')]:
104+
if all([var, conf.get(name, None)]):
105+
raise SyntaxError("{} parameter repeated".format(name))
106+
if var is None:
107+
var = conf.get(name, None)
110108

111-
super(DeserializingConsumer, self).__init__(conf, on_commit=on_commit, stats_cb=stats_cb,
112-
throttle_cb=throttle_cb, logger=logger)
109+
super(SerializingConsumer, self).__init__(conf, on_commit=on_commit, stats_cb=stats_cb,
110+
throttle_cb=throttle_cb, logger=logger)
113111

114112
def poll(self, timeout=-1.0):
115113
"""
@@ -127,7 +125,7 @@ def poll(self, timeout=-1.0):
127125
:raises RuntimeError: If called on a closed consumer.
128126
"""
129127

130-
msg = super(DeserializingConsumer, self).poll(timeout)
128+
msg = super(SerializingConsumer, self).poll(timeout)
131129

132130
if msg is None or msg.error():
133131
return msg
@@ -157,7 +155,7 @@ def consume(self, num_messages=1, timeout=-1):
157155
:raises ValueError: If num_messages > 1M.
158156
"""
159157

160-
msgset = super(DeserializingConsumer, self).consume(num_messages, timeout)
158+
msgset = super(SerializingConsumer, self).consume(num_messages, timeout)
161159
for msg in msgset:
162160
if msg.error():
163161
continue

confluent_kafka/producer.py

Lines changed: 21 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -21,14 +21,6 @@
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-
3224
class Producer(_impl):
3325
"""
3426
Create a new Kafka Producer instance with or without serializer support.
@@ -78,8 +70,13 @@ class SerializingProducer(Producer):
7870
def __new__(cls, *args, **kwargs):
7971
raise TypeError("SerializingProducer is a non user-instantiable class")
8072

73+
@staticmethod
74+
def byteSerializer(topic, data):
75+
""" Pass-through serializer """
76+
return data
77+
8178
# conf must remain optional as long as kwargs are supported
82-
def __init__(self, conf={}, key_serializer=byteSerializer, value_serializer=byteSerializer,
79+
def __init__(self, conf={}, key_serializer=None, value_serializer=None,
8380
error_cb=None, stats_cb=None, throttle_cb=None, logger=None, **kwargs):
8481

8582
if not isinstance(conf, dict):
@@ -93,23 +90,24 @@ def __init__(self, conf={}, key_serializer=byteSerializer, value_serializer=byte
9390
"all keyword arguments must match the constructor signature explicitly.",
9491
category=DeprecationWarning, stacklevel=2)
9592

93+
# Ensure the default serializer cannot be overwritten with None on instantiation
94+
if key_serializer is None:
95+
key_serializer = SerializingProducer.byteSerializer
96+
97+
if value_serializer is None:
98+
value_serializer = SerializingProducer.byteSerializer
99+
96100
self._key_serializer = key_serializer
97101
self._value_serializer = value_serializer
98102

99103
# Callbacks can be set in the conf dict or *ideally* as parameters.
100-
# Handle both cases prior to passing along to _impl
101-
# If callbacks are configured in both places parameter values take precedence.
102-
if not error_cb:
103-
error_cb = conf.get('error_cb', None)
104-
105-
if not stats_cb:
106-
stats_cb = conf.get('stats_cb', None)
107-
108-
if not throttle_cb:
109-
throttle_cb = conf.get('throttle_cb', None)
110-
111-
if not logger:
112-
logger = conf.get('logger', None)
104+
# Raise a SyntaxError if a callback is set in both places.
105+
for var, name in [(logger, 'logger'), (error_cb, 'error_cb'),
106+
(stats_cb, 'stats_cb'), (throttle_cb, 'throttle_cb')]:
107+
if all([var, conf.get(name, None)]):
108+
raise SyntaxError("{} parameter repeated".format(name))
109+
if var is None:
110+
var = conf.get(name, None)
113111

114112
super(SerializingProducer, self).__init__(conf, error_cb=error_cb, stats_cb=stats_cb,
115113
throttle_cb=throttle_cb, logger=logger)
@@ -141,8 +139,5 @@ def produce(self, topic, value=None, key=None, partition=PARTITION_UA,
141139
142140
"""
143141

144-
key = self._key_serializer(topic, key)
145-
value = self._value_serializer(topic, value)
146-
147-
super(SerializingProducer, self).produce(topic, value, key, partition, on_delivery=on_delivery,
142+
super(SerializingProducer, self).produce(topic, self._value_serializer(topic, value), self._key_serializer(topic, key), partition, on_delivery=on_delivery,
148143
callback=callback, timestamp=timestamp, headers=headers)

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