Skip to content

Commit 4827c5a

Browse files
committed
Add configurable key/value serializer support
1 parent 469f140 commit 4827c5a

File tree

5 files changed

+388
-6
lines changed

5 files changed

+388
-6
lines changed

confluent_kafka/__init__.py

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1,9 +1,7 @@
11
__all__ = ['cimpl', 'admin', 'avro', 'kafkatest']
2-
from .cimpl import (Consumer, # noqa
3-
KafkaError,
2+
from .cimpl import (KafkaError, # noqa
43
KafkaException,
54
Message,
6-
Producer,
75
TopicPartition,
86
libversion,
97
version,
@@ -15,6 +13,10 @@
1513
OFFSET_STORED,
1614
OFFSET_INVALID)
1715

16+
# High-level client wrappers used for extending cimpl.[Producer|Consumer] client functionality.
17+
from .producer import Producer # noqa
18+
from .consumer import Consumer # noqa
19+
1820
__version__ = version()[0]
1921

2022

confluent_kafka/consumer.py

Lines changed: 145 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,145 @@
1+
#!/usr/bin/env python
2+
# -*- coding: utf-8 -*-
3+
#
4+
#
5+
# Copyright 2018 Confluent Inc.
6+
#
7+
# Licensed under the Apache License, Version 2.0 (the "License");
8+
# you may not use this file except in compliance with the License.
9+
# You may obtain a copy of the License at
10+
#
11+
# http://www.apache.org/licenses/LICENSE-2.0
12+
#
13+
# Unless required by applicable law or agreed to in writing, software
14+
# distributed under the License is distributed on an "AS IS" BASIS,
15+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16+
# See the License for the specific language governing permissions and
17+
# limitations under the License.
18+
#
19+
20+
from .cimpl import Consumer as _impl
21+
from warnings import warn
22+
23+
24+
class Consumer(_impl):
25+
"""
26+
Create a new Kafka Consumer instance.
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+
47+
__slots__ = ["_key_deserializer", "_value_deserializer"]
48+
49+
# conf must remain optional as long as kwargs are supported
50+
def __init__(self, conf={}, key_deserializer=None, value_deserializer=None,
51+
on_commit=None, stats_cb=None, throttle_cb=None, logger=None, **kwargs):
52+
53+
if not isinstance(conf, dict):
54+
raise TypeError("expected configuration dict")
55+
56+
if kwargs:
57+
# Handle kwargs for backwards compatibility
58+
conf.update(kwargs)
59+
warn("The use of kwargs is being deprecated. "
60+
"In future releases `conf` will be mandatory and "
61+
"all keyword arguments must match the constructor signature explicitly.",
62+
category=DeprecationWarning, stacklevel=2)
63+
64+
self._key_deserializer = key_deserializer
65+
self._value_deserializer = value_deserializer
66+
67+
# Callbacks can be set in the conf dict or *ideally* as parameters.
68+
# Handle both cases prior to passing along to _impl
69+
# If callbacks are configured in both places parameter values take precedence.
70+
if not on_commit:
71+
on_commit = conf.get('on_commit', None)
72+
73+
if not stats_cb:
74+
stats_cb = conf.get('stats_cb', None)
75+
76+
if not throttle_cb:
77+
throttle_cb = conf.get('throttle_cb', None)
78+
79+
if not logger:
80+
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)
83+
84+
def poll(self, timeout=-1.0, key_deserializer=None, value_deserializer=None):
85+
"""
86+
Consumes a message, triggers callbacks, returns an event.
87+
88+
The application must check the returned Message object’s Message.error() method to distinguish
89+
between proper messages, an error(see error().code() for specifics), or an event.
90+
91+
:param float timeout: Maximum time in seconds to block waiting for message, event or callback.
92+
:param func key_deserializer(topic, key): Converts message key bytes to object.
93+
**note** deserializers are responsible for handling NULL keys
94+
:param func value_deserializer(topic, value): Converts message value bytes to object.
95+
**note** deserializers are responsible for handling NULL values
96+
:returns: A confluent_kafka.Message or None on timeout.
97+
:raises RuntimeError: If called on a closed consumer.
98+
"""
99+
100+
msg = super(Consumer, self).poll(timeout)
101+
102+
if not msg or msg.error():
103+
return msg
104+
105+
topic = msg.topic()
106+
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()))
119+
120+
return msg
121+
122+
def consume(self, num_messages=1, timeout=-1):
123+
"""
124+
Consume messages, calls callbacks and returns a list of messages. (possibly empty on timeout)
125+
126+
The application must check Message.error() to distinguish between
127+
proper messages, an error(see error().code() for specifics), or an event. for each
128+
Message in the list.
129+
130+
:param int num_messages: Maximum number of messages to return (default: 1)
131+
:param float timeout: Maximum time in seconds to block waiting for message, event or callback.
132+
(default: infinite (-1))
133+
:returns: A list of Message objects (possibly empty on timeout)
134+
:rtype: list(Message)
135+
:raises NotImplementedError: If used with key/value serializers.
136+
:raises RuntimeError: If called on a closed consumer.
137+
:raises KafkaError: In case of internal error.
138+
:raises ValueError: If num_messages > 1M.
139+
"""
140+
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")
144+
145+
return super(Consumer, self).consume(num_messages, timeout)

confluent_kafka/producer.py

Lines changed: 130 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,130 @@
1+
#!/usr/bin/env python
2+
# -*- coding: utf-8 -*-
3+
#
4+
#
5+
# Copyright 2018 Confluent Inc.
6+
#
7+
# Licensed under the Apache License, Version 2.0 (the "License");
8+
# you may not use this file except in compliance with the License.
9+
# You may obtain a copy of the License at
10+
#
11+
# http://www.apache.org/licenses/LICENSE-2.0
12+
#
13+
# Unless required by applicable law or agreed to in writing, software
14+
# distributed under the License is distributed on an "AS IS" BASIS,
15+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16+
# See the License for the specific language governing permissions and
17+
# limitations under the License.
18+
#
19+
20+
from .cimpl import Producer as _impl, PARTITION_UA
21+
from warnings import warn
22+
23+
24+
class Producer(_impl):
25+
"""
26+
Create a new Kafka Producer instance.
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+
__slots__ = ["_key_serializer", "_value_serializer"]
47+
48+
# conf must remain optional as long as kwargs are supported
49+
def __init__(self, conf={}, key_serializer=None, value_serializer=None,
50+
error_cb=None, stats_cb=None, throttle_cb=None, logger=None, **kwargs):
51+
52+
if not isinstance(conf, dict):
53+
raise TypeError("expected configuration dict")
54+
55+
if kwargs:
56+
# Handle kwargs for backwards compatibility
57+
conf.update(kwargs)
58+
warn("The use of kwargs is being deprecated. "
59+
"In future releases `conf` will be mandatory and "
60+
"all keyword arguments must match the constructor signature explicitly.",
61+
category=DeprecationWarning, stacklevel=2)
62+
63+
self._key_serializer = key_serializer
64+
self._value_serializer = value_serializer
65+
66+
# Callbacks can be set in the conf dict or *ideally* as parameters.
67+
# Handle both cases prior to passing along to _impl
68+
# If callbacks are configured in both places parameter values take precedence.
69+
if not error_cb:
70+
error_cb = conf.get('error_cb', None)
71+
72+
if not stats_cb:
73+
stats_cb = conf.get('stats_cb', None)
74+
75+
if not throttle_cb:
76+
throttle_cb = conf.get('throttle_cb', None)
77+
78+
if not logger:
79+
logger = conf.get('logger', None)
80+
81+
super(Producer, self).__init__(conf, error_cb=error_cb, stats_cb=stats_cb,
82+
throttle_cb=throttle_cb, logger=logger)
83+
84+
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):
87+
"""
88+
Produces message to Kafka
89+
90+
:param str topic: Topic to produce message to.
91+
:param str|bytes|obj value: Message payload; value_serializer required for non-character values.
92+
:param str|bytes|obj key: Message key payload; key_serializer required for non-character keys.
93+
:param int partition: Partition to produce to, else uses the configured built-in partitioner.
94+
Default value is PARTITION_UA(round-robin over all partitions).
95+
:param func on_delivery(confluent_kafka.KafkaError, confluent_kafka.Message):
96+
Delivery report callback to call on successful or failed delivery.
97+
:param func callback(confluent_kafka.KafkaError, confluent_kafka.Message):
98+
See on_delivery.
99+
:param int timestamp: Message timestamp (CreateTime) in milliseconds since epoch UTC
100+
(requires librdkafka >= v0.9.4, api.version.request=true, and broker >= 0.10.0.0).
101+
Default value is current time.
102+
:param dict|list headers: Message headers to set on the message. The header key must be a string while
103+
the value must be binary, unicode or None. Accepts a list of (key,value) or a dict.
104+
(Requires librdkafka >= v0.11.4 and broker version >= 0.11.0.0)
105+
:param func key_serializer(topic, key): Producer key_serializer override;
106+
Converts message key to bytes. **note** serializers are responsible for handling NULL keys
107+
:param func value_serializer(topic, value): Producer value_serializer override;
108+
Converts message value to bytes. **note** serializers are responsible for handling NULL values
109+
110+
"""
111+
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)
128+
129+
super(Producer, self).produce(topic, value, key, partition, on_delivery=on_delivery,
130+
timestamp=timestamp, headers=headers)

confluent_kafka/src/Producer.c

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -282,9 +282,10 @@ static PyObject *Producer_produce (Handle *self, PyObject *args,
282282
return NULL;
283283
}
284284
#else
285-
if (headers) {
286-
if(!(rd_headers = py_headers_to_c(headers)))
287-
return NULL;
285+
/* Treat Py_None as if it were NULL */
286+
if (headers && headers != Py_None) {
287+
if(!(rd_headers = py_headers_to_c(headers)))
288+
return NULL;
288289
}
289290
#endif
290291

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