Skip to content

Commit 829277d

Browse files
committed
Add a new Avro Serializer/Deserialzer and Generic Avro data type
1 parent 7ef6a36 commit 829277d

File tree

3 files changed

+311
-89
lines changed

3 files changed

+311
-89
lines changed

confluent_kafka/avro/schema.py

Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,36 @@
1+
from avro.schema import PrimitiveSchema
2+
3+
# Python 2 considers int an instance of str
4+
try:
5+
string_type = basestring # noqa
6+
except NameError:
7+
string_type = str
8+
9+
10+
class GenericAvroRecord(dict):
11+
__slots__ = ['schema']
12+
13+
def __init__(self, schema, datum=None):
14+
self.schema = schema
15+
if datum is not None:
16+
self.update(datum)
17+
18+
def put(self, key, value):
19+
self[key] = value
20+
21+
22+
def get_schema(datum):
23+
if isinstance(datum, GenericAvroRecord):
24+
return datum.schema
25+
elif (datum is None):
26+
return PrimitiveSchema("null")
27+
elif isinstance(datum, basestring):
28+
return PrimitiveSchema('string')
29+
elif isinstance(datum, bool):
30+
return PrimitiveSchema('boolean')
31+
elif isinstance(datum, int):
32+
return PrimitiveSchema('int')
33+
elif isinstance(datum, float):
34+
return PrimitiveSchema('float')
35+
else:
36+
raise ValueError("Unsupported Avro type {}.".format(type(datum)))

confluent_kafka/avro/serializer/__init__.py

Lines changed: 215 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -15,10 +15,38 @@
1515
# limitations under the License.
1616
#
1717

18+
import avro
19+
import avro.io
20+
import io
21+
import logging
22+
import struct
23+
import sys
24+
import traceback
25+
26+
from confluent_kafka.avro.schema import GenericAvroRecord, get_schema
27+
from confluent_kafka.avro import ClientError
28+
29+
log = logging.getLogger(__name__)
30+
31+
MAGIC_BYTE = 0
32+
33+
HAS_FAST = False
34+
try:
35+
from fastavro import schemaless_reader, schemaless_writer
36+
37+
HAS_FAST = True
38+
except ImportError:
39+
pass
40+
1841

1942
class SerializerError(Exception):
2043
"""Generic error from serializer package"""
2144

45+
def __new__(cls, message, is_key=False):
46+
if is_key:
47+
return super(SerializerError, cls).__new__(KeySerializerError, message)
48+
return super(SerializerError, cls).__new__(ValueSerializerError, message)
49+
2250
def __init__(self, message):
2351
self.message = message
2452

@@ -38,3 +66,190 @@ class KeySerializerError(SerializerError):
3866

3967
class ValueSerializerError(SerializerError):
4068
pass
69+
70+
71+
class ContextStringIO(io.BytesIO):
72+
"""
73+
Wrapper to allow use of StringIO via 'with' constructs.
74+
"""
75+
76+
def __enter__(self):
77+
return self
78+
79+
def __exit__(self, *args):
80+
self.close()
81+
return False
82+
83+
84+
class AvroSerializer(object):
85+
86+
__slots__ = ["registry_client", "codec_cache", "is_key"]
87+
88+
def __init__(self, registry_client, is_key=False):
89+
self.registry_client = registry_client
90+
self.codec_cache = {}
91+
self.is_key = is_key
92+
93+
def __call__(self, topic, record):
94+
"""
95+
Given a parsed avro schema, encode a record for the given topic.
96+
97+
The schema is registered with the subject of 'topic-value'
98+
:param str topic: Topic name
99+
:param GenericAvroRecord record: An object to serialize
100+
:returns: Encoded record with schema ID as bytes
101+
:rtype: bytes
102+
"""
103+
104+
if record is None:
105+
return None
106+
107+
subject_suffix = '-key' if self.is_key else '-value'
108+
subject = topic + subject_suffix
109+
110+
schema_id = self.registry_client.register(subject, get_schema(record))
111+
if not schema_id:
112+
message = "Failed to register schema with subject {}.".format(subject)
113+
raise SerializerError(message, is_key=self.is_key)
114+
115+
if schema_id not in self.codec_cache:
116+
self.codec_cache[schema_id] = self._get_encoder_func(get_schema(record))
117+
118+
return self._encode_record_with_schema_id(schema_id, record)
119+
120+
def _get_encoder_func(self, writer_schema):
121+
if HAS_FAST:
122+
return lambda record, fp: schemaless_writer(fp, writer_schema.to_json(), record)
123+
writer = avro.io.DatumWriter(writer_schema)
124+
return lambda record, fp: writer.write(record, avro.io.BinaryEncoder(fp))
125+
126+
def _encode_record_with_schema_id(self, schema_id, record):
127+
"""
128+
Encode a record with a given schema id. The record must
129+
be a python dictionary.
130+
:param int schema_id: integer ID
131+
:param dict record: An object to serialize
132+
:param bool is_key: If the record is a key
133+
:param SerializerErr err_type: Error type to raise on serialization exception
134+
:returns: decoder function
135+
:rtype: func
136+
"""
137+
138+
if schema_id not in self.codec_cache:
139+
try:
140+
schema = self.registry_client.get_by_id(schema_id)
141+
if not schema:
142+
raise SerializerError("Schema does not exist", self.is_key)
143+
self.codec_cache[schema_id] = self._get_encoder_func(schema)
144+
except ClientError:
145+
exc_type, exc_value, exc_traceback = sys.exc_info()
146+
raise SerializerError(
147+
repr(traceback.format_exception(exc_type, exc_value, exc_traceback)),
148+
self.is_key)
149+
150+
# get the writer
151+
writer = self.codec_cache[schema_id]
152+
with ContextStringIO() as outf:
153+
# Write the magic byte and schema ID in network byte order (big endian)
154+
outf.write(struct.pack('>bI', MAGIC_BYTE, schema_id))
155+
156+
# write the record to the rest of the buffer
157+
writer(record, outf)
158+
return outf.getvalue()
159+
160+
161+
class AvroDeserializer(object):
162+
163+
__slots__ = ["registry_client", "codec_cache", "is_key", "reader_schema"]
164+
165+
def __init__(self, registry_client, is_key=False, reader_schema=None):
166+
self.registry_client = registry_client
167+
self.codec_cache = {}
168+
self.is_key = is_key
169+
self.reader_schema = reader_schema
170+
171+
def __call__(self, topic, message):
172+
"""
173+
Decode a message from kafka that has been encoded for use with
174+
the schema registry.
175+
:param str|bytes or None message: message key or value to be decoded
176+
:returns: Decoded message contents.
177+
:rtype GenericAvroRecord:
178+
"""
179+
180+
if message is None:
181+
return None
182+
183+
if len(message) <= 5:
184+
raise SerializerError("message is too small to decode")
185+
186+
with ContextStringIO(message) as payload:
187+
magic, schema_id = struct.unpack('>bI', payload.read(5))
188+
if magic != MAGIC_BYTE:
189+
raise SerializerError("message does not start with magic byte", self.is_key)
190+
191+
decoder_func = self._get_decoder_func(schema_id, payload)
192+
return decoder_func(payload)
193+
194+
def _get_decoder_func(self, schema_id, payload):
195+
if schema_id in self.codec_cache:
196+
return self.codec_cache[schema_id]
197+
198+
try:
199+
writer_schema = self.registry_client.get_by_id(schema_id)
200+
except ClientError as e:
201+
raise SerializerError("unable to fetch schema with id %d: %s" % (schema_id, str(e)))
202+
203+
if writer_schema is None:
204+
raise SerializerError("unable to fetch schema with id %d" % (schema_id))
205+
206+
reader_schema = writer_schema if self.reader_schema is None else self.reader_schema
207+
208+
curr_pos = payload.tell()
209+
210+
if HAS_FAST:
211+
# try to use fast avro
212+
try:
213+
schemaless_reader(payload, writer_schema.to_json())
214+
215+
# If we reach this point, this means we have fastavro and it can
216+
# do this deserialization. Rewind since this method just determines
217+
# the reader function and we need to deserialize again along the
218+
# normal path.
219+
payload.seek(curr_pos)
220+
221+
def fast_decoder(p):
222+
return schemaless_reader(p, writer_schema.to_json(), reader_schema.to_json())
223+
224+
def fast_record_decoder(p):
225+
return GenericAvroRecord(reader_schema,
226+
schemaless_reader(p, writer_schema.to_json(), reader_schema.to_json()))
227+
228+
if writer_schema.type is 'record':
229+
self.codec_cache[schema_id] = fast_record_decoder
230+
else:
231+
self.codec_cache[schema_id] = fast_decoder
232+
233+
return self.codec_cache[schema_id]
234+
except Exception:
235+
# Fast avro failed, fall thru to standard avro below.
236+
pass
237+
238+
payload.seek(curr_pos)
239+
240+
avro_reader = avro.io.DatumReader(writer_schema, reader_schema)
241+
242+
def record_decoder(p):
243+
bin_decoder = avro.io.BinaryDecoder(p)
244+
return GenericAvroRecord(reader_schema, avro_reader.read(bin_decoder))
245+
246+
def decoder(p):
247+
bin_decoder = avro.io.BinaryDecoder(p)
248+
return avro_reader.read(bin_decoder)
249+
250+
if writer_schema.get_prop('type') is 'record':
251+
self.codec_cache[schema_id] = record_decoder
252+
else:
253+
self.codec_cache[schema_id] = decoder
254+
255+
return self.codec_cache[schema_id]

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