Skip to content

Commit d874996

Browse files
fkauferRyan P
authored andcommitted
Add ability to specify reader schema (confluentinc#470)
1 parent d0496eb commit d874996

File tree

5 files changed

+161
-18
lines changed

5 files changed

+161
-18
lines changed

confluent_kafka/avro/__init__.py

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -98,9 +98,11 @@ class AvroConsumer(Consumer):
9898
Constructor takes below parameters
9999
100100
:param dict config: Config parameters containing url for schema registry (``schema.registry.url``)
101-
and the standard Kafka client configuration (``bootstrap.servers`` et.al).
101+
and the standard Kafka client configuration (``bootstrap.servers`` et.al)
102+
:param optional a reader schema for the message key
103+
:param optional a reader schema for the message value
102104
"""
103-
def __init__(self, config, schema_registry=None):
105+
def __init__(self, config, schema_registry=None, reader_key_schema=None, reader_value_schema=None):
104106

105107
schema_registry_url = config.pop("schema.registry.url", None)
106108
schema_registry_ca_location = config.pop("schema.registry.ssl.ca.location", None)
@@ -119,7 +121,7 @@ def __init__(self, config, schema_registry=None):
119121
raise ValueError("Cannot pass schema_registry along with schema.registry.url config")
120122

121123
super(AvroConsumer, self).__init__(config)
122-
self._serializer = MessageSerializer(schema_registry)
124+
self._serializer = MessageSerializer(schema_registry, reader_key_schema, reader_value_schema)
123125

124126
def poll(self, timeout=None):
125127
"""
@@ -139,9 +141,9 @@ def poll(self, timeout=None):
139141
return message
140142
if not message.error():
141143
if message.value() is not None:
142-
decoded_value = self._serializer.decode_message(message.value())
144+
decoded_value = self._serializer.decode_message(message.value(), is_key=False)
143145
message.set_value(decoded_value)
144146
if message.key() is not None:
145-
decoded_key = self._serializer.decode_message(message.key())
147+
decoded_key = self._serializer.decode_message(message.key(), is_key=True)
146148
message.set_key(decoded_key)
147149
return message

confluent_kafka/avro/serializer/message_serializer.py

Lines changed: 24 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -68,10 +68,12 @@ class MessageSerializer(object):
6868
All decode_* methods expect a buffer received from kafka.
6969
"""
7070

71-
def __init__(self, registry_client):
71+
def __init__(self, registry_client, reader_key_schema=None, reader_value_schema=None):
7272
self.registry_client = registry_client
7373
self.id_to_decoder_func = {}
7474
self.id_to_writers = {}
75+
self.reader_key_schema = reader_key_schema
76+
self.reader_value_schema = reader_value_schema
7577

7678
'''
7779
@@ -151,33 +153,38 @@ def encode_record_with_schema_id(self, schema_id, record, is_key=False):
151153
return outf.getvalue()
152154

153155
# Decoder support
154-
def _get_decoder_func(self, schema_id, payload):
156+
def _get_decoder_func(self, schema_id, payload, is_key=False):
155157
if schema_id in self.id_to_decoder_func:
156158
return self.id_to_decoder_func[schema_id]
157159

158-
# fetch from schema reg
160+
# fetch writer schema from schema reg
159161
try:
160-
schema = self.registry_client.get_by_id(schema_id)
162+
writer_schema_obj = self.registry_client.get_by_id(schema_id)
161163
except ClientError as e:
162164
raise SerializerError("unable to fetch schema with id %d: %s" % (schema_id, str(e)))
163165

164-
if schema is None:
166+
if writer_schema_obj is None:
165167
raise SerializerError("unable to fetch schema with id %d" % (schema_id))
166168

167169
curr_pos = payload.tell()
170+
171+
reader_schema_obj = self.reader_key_schema if is_key else self.reader_value_schema
172+
168173
if HAS_FAST:
169174
# try to use fast avro
170175
try:
171-
schema_dict = schema.to_json()
172-
schemaless_reader(payload, schema_dict)
176+
writer_schema = writer_schema_obj.to_json()
177+
reader_schema = reader_schema_obj.to_json()
178+
schemaless_reader(payload, writer_schema)
173179

174180
# If we reach this point, this means we have fastavro and it can
175181
# do this deserialization. Rewind since this method just determines
176182
# the reader function and we need to deserialize again along the
177183
# normal path.
178184
payload.seek(curr_pos)
179185

180-
self.id_to_decoder_func[schema_id] = lambda p: schemaless_reader(p, schema_dict)
186+
self.id_to_decoder_func[schema_id] = lambda p: schemaless_reader(
187+
p, writer_schema, reader_schema)
181188
return self.id_to_decoder_func[schema_id]
182189
except Exception:
183190
# Fast avro failed, fall thru to standard avro below.
@@ -186,7 +193,13 @@ def _get_decoder_func(self, schema_id, payload):
186193
# here means we should just delegate to slow avro
187194
# rewind
188195
payload.seek(curr_pos)
189-
avro_reader = avro.io.DatumReader(schema)
196+
# Avro DatumReader py2/py3 inconsistency, hence no param keywords
197+
# should be revisited later
198+
# https://github.com/apache/avro/blob/master/lang/py3/avro/io.py#L459
199+
# https://github.com/apache/avro/blob/master/lang/py/src/avro/io.py#L423
200+
# def __init__(self, writers_schema=None, readers_schema=None)
201+
# def __init__(self, writer_schema=None, reader_schema=None)
202+
avro_reader = avro.io.DatumReader(writer_schema_obj, reader_schema_obj)
190203

191204
def decoder(p):
192205
bin_decoder = avro.io.BinaryDecoder(p)
@@ -195,7 +208,7 @@ def decoder(p):
195208
self.id_to_decoder_func[schema_id] = decoder
196209
return self.id_to_decoder_func[schema_id]
197210

198-
def decode_message(self, message):
211+
def decode_message(self, message, is_key=False):
199212
"""
200213
Decode a message from kafka that has been encoded for use with
201214
the schema registry.
@@ -212,5 +225,5 @@ def decode_message(self, message):
212225
magic, schema_id = struct.unpack('>bI', payload.read(5))
213226
if magic != MAGIC_BYTE:
214227
raise SerializerError("message does not start with magic byte")
215-
decoder_func = self._get_decoder_func(schema_id, payload)
228+
decoder_func = self._get_decoder_func(schema_id, payload, is_key)
216229
return decoder_func(payload)

examples/integration_test.py

Lines changed: 112 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -328,7 +328,6 @@ def verify_avro():
328328
for i, combo in enumerate(combinations):
329329
combo['topic'] = str(uuid.uuid4())
330330
p.produce(**combo)
331-
p.poll(0)
332331
p.flush()
333332

334333
# Create consumer
@@ -1202,9 +1201,117 @@ def verify_config(expconfig, configs):
12021201
print("Topic {} marked for deletion".format(our_topic))
12031202

12041203

1205-
# Exclude throttle since from default list
1204+
def verify_avro_explicit_read_schema():
1205+
""" verify that reading Avro with explicit reader schema works"""
1206+
from confluent_kafka import avro
1207+
avsc_dir = os.path.join(os.path.dirname(__file__), os.pardir, 'tests', 'avro')
1208+
1209+
# Producer config
1210+
conf = {'bootstrap.servers': bootstrap_servers,
1211+
'error_cb': error_cb,
1212+
'api.version.request': api_version_request,
1213+
'default.topic.config': {'produce.offset.report': True}}
1214+
1215+
# Create producer
1216+
if schema_registry_url:
1217+
conf['schema.registry.url'] = schema_registry_url
1218+
p = avro.AvroProducer(conf)
1219+
else:
1220+
p = avro.AvroProducer(conf, schema_registry=InMemorySchemaRegistry())
1221+
1222+
key_schema = avro.load(os.path.join(avsc_dir, "primitive_float.avsc"))
1223+
schema1 = avro.load(os.path.join(avsc_dir, "user_v1.avsc"))
1224+
schema2 = avro.load(os.path.join(avsc_dir, "user_v2.avsc"))
1225+
float_value = 32.
1226+
val = {
1227+
"name": "abc",
1228+
"favorite_number": 42,
1229+
"favorite_colo": "orange"
1230+
}
1231+
val1 = {
1232+
"name": "abc"
1233+
}
1234+
1235+
combinations = [
1236+
dict(value=val, value_schema=schema2, key=float_value, key_schema=key_schema,
1237+
reader_value_schema=schema1, reader_key_schema=key_schema),
1238+
dict(value=val1, value_schema=schema1, key=float_value, key_schema=key_schema,
1239+
reader_value_schema=schema2, reader_key_schema=key_schema),
1240+
]
1241+
1242+
# Consumer config
1243+
cons_conf = {'bootstrap.servers': bootstrap_servers,
1244+
'group.id': 'test.py',
1245+
'session.timeout.ms': 6000,
1246+
'enable.auto.commit': False,
1247+
'api.version.request': api_version_request,
1248+
'on_commit': print_commit_result,
1249+
'error_cb': error_cb,
1250+
'default.topic.config': {
1251+
'auto.offset.reset': 'earliest'
1252+
}}
1253+
1254+
for i, combo in enumerate(combinations):
1255+
reader_key_schema = combo.pop("reader_key_schema")
1256+
reader_value_schema = combo.pop("reader_value_schema")
1257+
combo['topic'] = str(uuid.uuid4())
1258+
p.produce(**combo)
1259+
p.poll(0)
1260+
p.flush()
1261+
1262+
# Create consumer
1263+
conf = copy(cons_conf)
1264+
if schema_registry_url:
1265+
conf['schema.registry.url'] = schema_registry_url
1266+
c = avro.AvroConsumer(
1267+
conf,
1268+
reader_key_schema=reader_key_schema,
1269+
reader_value_schema=reader_value_schema)
1270+
else:
1271+
c = avro.AvroConsumer(
1272+
conf,
1273+
schema_registry=InMemorySchemaRegistry(),
1274+
reader_key_schema=reader_key_schema,
1275+
reader_value_schema=reader_value_schema)
1276+
1277+
c.subscribe([combo['topic']])
1278+
1279+
while True:
1280+
msg = c.poll(0)
1281+
if msg is None:
1282+
continue
1283+
1284+
if msg.error():
1285+
if msg.error().code() == confluent_kafka.KafkaError._PARTITION_EOF:
1286+
break
1287+
else:
1288+
continue
1289+
1290+
tstype, timestamp = msg.timestamp()
1291+
print('%s[%d]@%d: key=%s, value=%s, tstype=%d, timestamp=%s' %
1292+
(msg.topic(), msg.partition(), msg.offset(),
1293+
msg.key(), msg.value(), tstype, timestamp))
1294+
1295+
# omit empty Avro fields from payload for comparison
1296+
record_key = msg.key()
1297+
record_value = msg.value()
1298+
if isinstance(msg.key(), dict):
1299+
record_key = {k: v for k, v in msg.key().items() if v is not None}
1300+
1301+
if isinstance(msg.value(), dict):
1302+
record_value = {k: v for k, v in msg.value().items() if v is not None}
1303+
1304+
assert combo.get('key') == record_key
1305+
assert combo.get('value')['name'] == record_value['name']
1306+
c.commit(msg, asynchronous=False)
1307+
# Close consumer
1308+
c.close()
1309+
pass
1310+
1311+
12061312
default_modes = ['consumer', 'producer', 'avro', 'performance', 'admin']
12071313
all_modes = default_modes + ['throttle', 'avro-https', 'none']
1314+
12081315
"""All test modes"""
12091316

12101317

@@ -1315,6 +1422,9 @@ def resolve_envs(_conf):
13151422
print('=' * 30, 'Verifying AVRO', '=' * 30)
13161423
verify_avro()
13171424

1425+
print('=' * 30, 'Verifying AVRO with explicit reader schema', '=' * 30)
1426+
verify_avro_explicit_read_schema()
1427+
13181428
if 'avro-https' in modes:
13191429
print('=' * 30, 'Verifying AVRO with HTTPS', '=' * 30)
13201430
verify_avro_https()

tests/avro/user_v1.avsc

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,8 @@
1+
{
2+
"type": "record",
3+
"name": "UserKey",
4+
"aliases": ["User"],
5+
"fields": [
6+
{"name": "name", "type": "string"}
7+
]
8+
}

tests/avro/user_v2.avsc

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,10 @@
1+
{
2+
"type": "record",
3+
"name": "User",
4+
"aliases": ["UserKey"],
5+
"fields": [
6+
{"name": "name", "type": "string"},
7+
{"name": "favorite_number", "type": ["null", "int"], "default": null},
8+
{"name": "favorite_color", "type": ["null", "string"], "default": null}
9+
]
10+
}

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