10
10
from quixstreams .models .serializers import (
11
11
DESERIALIZERS ,
12
12
SERIALIZERS ,
13
- BytesDeserializer ,
14
- BytesSerializer ,
15
13
Deserializer ,
16
- DeserializerIsNotProvidedError ,
17
14
DeserializerType ,
18
15
IgnoreMessage ,
19
16
MessageField ,
20
17
SerializationContext ,
21
18
Serializer ,
22
- SerializerIsNotProvidedError ,
23
19
SerializerType ,
24
20
)
25
21
from quixstreams .models .timestamps import TimestampType
@@ -57,29 +53,39 @@ def as_dict(self):
57
53
return dataclasses .asdict (self )
58
54
59
55
60
- def _get_serializer (serializer : Optional [ SerializerType ] ) -> Optional [ Serializer ] :
56
+ def _resolve_serializer (serializer : SerializerType ) -> Serializer :
61
57
if isinstance (serializer , str ):
62
58
try :
63
59
return SERIALIZERS [serializer ]()
64
60
except KeyError :
65
61
raise ValueError (
66
- f" Unknown deserializer option ' { serializer } '; "
62
+ f' Unknown serializer option " { serializer } "; '
67
63
f"valid options are { list (SERIALIZERS .keys ())} "
68
64
)
65
+ elif not isinstance (serializer , Serializer ):
66
+ raise ValueError (
67
+ f"Serializer must be either one of { list (SERIALIZERS .keys ())} "
68
+ f'or a subclass of Serializer; got "{ serializer } "'
69
+ )
69
70
return serializer
70
71
71
72
72
- def _get_deserializer (
73
- deserializer : Optional [ DeserializerType ] ,
74
- ) -> Optional [ Deserializer ] :
73
+ def _resolve_deserializer (
74
+ deserializer : DeserializerType ,
75
+ ) -> Deserializer :
75
76
if isinstance (deserializer , str ):
76
77
try :
77
78
return DESERIALIZERS [deserializer ]()
78
79
except KeyError :
79
80
raise ValueError (
80
- f" Unknown deserializer option ' { deserializer } '; "
81
+ f' Unknown deserializer option " { deserializer } "; '
81
82
f"valid options are { list (DESERIALIZERS .keys ())} "
82
83
)
84
+ elif not isinstance (deserializer , Deserializer ):
85
+ raise ValueError (
86
+ f"Deserializer must be either one of { list (DESERIALIZERS .keys ())} "
87
+ f'or a subclass of Deserializer; got "{ deserializer } "'
88
+ )
83
89
return deserializer
84
90
85
91
@@ -103,10 +109,10 @@ def __init__(
103
109
name : str ,
104
110
topic_type : TopicType = TopicType .REGULAR ,
105
111
create_config : Optional [TopicConfig ] = None ,
106
- value_deserializer : Optional [ DeserializerType ] = None ,
107
- key_deserializer : Optional [ DeserializerType ] = BytesDeserializer () ,
108
- value_serializer : Optional [ SerializerType ] = None ,
109
- key_serializer : Optional [ SerializerType ] = BytesSerializer () ,
112
+ value_deserializer : DeserializerType = "json" ,
113
+ key_deserializer : DeserializerType = "bytes" ,
114
+ value_serializer : SerializerType = "json" ,
115
+ key_serializer : SerializerType = "bytes" ,
110
116
timestamp_extractor : Optional [TimestampExtractor ] = None ,
111
117
quix_name : str = "" ,
112
118
):
@@ -133,10 +139,10 @@ def __init__(
133
139
self .quix_name = quix_name or name
134
140
self ._create_config = copy .deepcopy (create_config )
135
141
self ._broker_config : Optional [TopicConfig ] = None
136
- self ._value_deserializer = _get_deserializer (value_deserializer )
137
- self ._key_deserializer = _get_deserializer (key_deserializer )
138
- self ._value_serializer = _get_serializer (value_serializer )
139
- self ._key_serializer = _get_serializer (key_serializer )
142
+ self ._value_deserializer = _resolve_deserializer (value_deserializer )
143
+ self ._key_deserializer = _resolve_deserializer (key_deserializer )
144
+ self ._value_serializer = _resolve_serializer (value_serializer )
145
+ self ._key_serializer = _resolve_serializer (key_serializer )
140
146
self ._timestamp_extractor = timestamp_extractor
141
147
self ._type = topic_type
142
148
@@ -202,36 +208,27 @@ def row_serialize(self, row: Row, key: Any) -> KafkaMessage:
202
208
:param key: message key to serialize
203
209
:return: KafkaMessage object with serialized values
204
210
"""
205
- if self ._key_serializer is None :
206
- raise SerializerIsNotProvidedError (
207
- f'Key serializer is not provided for topic "{ self .name } "'
208
- )
209
- if self ._value_serializer is None :
210
- raise SerializerIsNotProvidedError (
211
- f'Value serializer is not provided for topic "{ self .name } "'
212
- )
213
211
212
+ serialization_ctx = SerializationContext (
213
+ topic = self .name , field = MessageField .KEY , headers = row .headers
214
+ )
214
215
# Try to serialize the key only if it's not None
215
216
# If key is None then pass it as is
216
217
# Otherwise, different serializers may serialize None differently
217
218
if key is None :
218
219
key_serialized = None
219
220
else :
220
- key_ctx = SerializationContext (
221
- topic = self .name , field = MessageField .KEY , headers = row .headers
222
- )
223
- key_serialized = self ._key_serializer (key , ctx = key_ctx )
221
+ key_serialized = self ._key_serializer (key , ctx = serialization_ctx )
224
222
225
223
# Update message headers with headers supplied by the value serializer.
226
224
extra_headers = self ._value_serializer .extra_headers
227
225
headers = merge_headers (row .headers , extra_headers )
228
- value_ctx = SerializationContext (
229
- topic = self .name , field = MessageField .VALUE , headers = row .headers
230
- )
226
+ serialization_ctx .field = MessageField .VALUE
227
+ value_serialized = self ._value_serializer (row .value , ctx = serialization_ctx )
231
228
232
229
return KafkaMessage (
233
230
key = key_serialized ,
234
- value = self . _value_serializer ( row . value , ctx = value_ctx ) ,
231
+ value = value_serialized ,
235
232
headers = headers ,
236
233
)
237
234
@@ -244,50 +241,46 @@ def row_deserialize(
244
241
:param message: an object with interface of `confluent_kafka.Message`
245
242
:return: Row, list of Rows or None if the message is ignored.
246
243
"""
247
- if self ._key_deserializer is None :
248
- raise DeserializerIsNotProvidedError (
249
- f'Key deserializer is not provided for topic "{ self .name } "'
250
- )
251
- if self ._value_deserializer is None :
252
- raise DeserializerIsNotProvidedError (
253
- f'Value deserializer is not provided for topic "{ self .name } "'
254
- )
255
-
256
244
headers = message .headers ()
245
+ topic = message .topic ()
246
+ partition = message .partition ()
247
+ offset = message .offset ()
257
248
249
+ serialization_ctx = SerializationContext (
250
+ topic = topic , field = MessageField .KEY , headers = headers
251
+ )
258
252
if (key_bytes := message .key ()) is None :
259
253
key_deserialized = None
260
254
else :
261
- key_ctx = SerializationContext (
262
- topic = message . topic (), field = MessageField . KEY , headers = headers
255
+ key_deserialized = self . _key_deserializer (
256
+ value = key_bytes , ctx = serialization_ctx
263
257
)
264
- key_deserialized = self ._key_deserializer (value = key_bytes , ctx = key_ctx )
265
258
266
259
if (value_bytes := message .value ()) is None :
267
260
value_deserialized = None
268
261
else :
269
- value_ctx = SerializationContext (
270
- topic = message . topic (), field = MessageField . VALUE , headers = headers
271
- )
262
+ # Reuse the SerializationContext object here to avoid creating a new
263
+ # one with almost the same fields
264
+ serialization_ctx . field = MessageField . VALUE
272
265
try :
273
266
value_deserialized = self ._value_deserializer (
274
- value = value_bytes , ctx = value_ctx
267
+ value = value_bytes , ctx = serialization_ctx
275
268
)
276
269
except IgnoreMessage :
277
270
# Ignore message completely if deserializer raised IgnoreValueError.
278
271
logger .debug (
279
272
'Ignore incoming message: partition="%s[%s]" offset="%s"' ,
280
- message . topic () ,
281
- message . partition () ,
282
- message . offset () ,
273
+ topic ,
274
+ partition ,
275
+ offset ,
283
276
)
284
277
return None
285
278
286
279
timestamp_type , timestamp_ms = message .timestamp ()
287
280
message_context = MessageContext (
288
- topic = message . topic () ,
289
- partition = message . partition () ,
290
- offset = message . offset () ,
281
+ topic = topic ,
282
+ partition = partition ,
283
+ offset = offset ,
291
284
size = len (message ),
292
285
leader_epoch = message .leader_epoch (),
293
286
)
@@ -312,8 +305,8 @@ def row_deserialize(
312
305
)
313
306
return rows
314
307
315
- if self ._timestamp_extractor :
316
- timestamp_ms = self . _timestamp_extractor (
308
+ if ( timestamp_extractor := self ._timestamp_extractor ) is not None :
309
+ timestamp_ms = timestamp_extractor (
317
310
value_deserialized , headers , timestamp_ms , TimestampType (timestamp_type )
318
311
)
319
312
@@ -332,24 +325,13 @@ def serialize(
332
325
headers : Optional [Headers ] = None ,
333
326
timestamp_ms : Optional [int ] = None ,
334
327
) -> KafkaMessage :
335
- if self ._key_serializer :
336
- key_ctx = SerializationContext (
337
- topic = self .name , field = MessageField .KEY , headers = headers
338
- )
339
- key = self ._key_serializer (key , ctx = key_ctx )
340
- elif key is not None :
341
- raise SerializerIsNotProvidedError (
342
- f'Key serializer is not provided for topic "{ self .name } "'
343
- )
344
- if self ._value_serializer :
345
- value_ctx = SerializationContext (
346
- topic = self .name , field = MessageField .VALUE , headers = headers
347
- )
348
- value = self ._value_serializer (value , ctx = value_ctx )
349
- elif value is not None :
350
- raise SerializerIsNotProvidedError (
351
- f'Value serializer is not provided for topic "{ self .name } "'
352
- )
328
+ serialization_ctx = SerializationContext (
329
+ topic = self .name , field = MessageField .KEY , headers = headers
330
+ )
331
+ key = self ._key_serializer (key , ctx = serialization_ctx )
332
+ serialization_ctx .field = MessageField .VALUE
333
+ value = self ._value_serializer (value , ctx = serialization_ctx )
334
+
353
335
return KafkaMessage (
354
336
key = key ,
355
337
value = value ,
@@ -358,30 +340,18 @@ def serialize(
358
340
)
359
341
360
342
def deserialize (self , message : SuccessfulConfluentKafkaMessageProto ):
343
+ serialization_ctx = SerializationContext (
344
+ topic = message .topic (),
345
+ field = MessageField .KEY ,
346
+ headers = message .headers (),
347
+ )
361
348
if (key := message .key ()) is not None :
362
- if self ._key_deserializer :
363
- key_ctx = SerializationContext (
364
- topic = message .topic (),
365
- field = MessageField .KEY ,
366
- headers = message .headers (),
367
- )
368
- key = self ._key_deserializer (key , ctx = key_ctx )
369
- else :
370
- raise DeserializerIsNotProvidedError (
371
- f'Key deserializer is not provided for topic "{ self .name } "'
372
- )
349
+ key = self ._key_deserializer (key , ctx = serialization_ctx )
350
+
373
351
if (value := message .value ()) is not None :
374
- if self ._value_deserializer :
375
- value_ctx = SerializationContext (
376
- topic = message .topic (),
377
- field = MessageField .VALUE ,
378
- headers = message .headers (),
379
- )
380
- value = self ._value_deserializer (value , ctx = value_ctx )
381
- else :
382
- raise DeserializerIsNotProvidedError (
383
- f'Value deserializer is not provided for topic "{ self .name } "'
384
- )
352
+ serialization_ctx .field = MessageField .VALUE
353
+ value = self ._value_deserializer (value , ctx = serialization_ctx )
354
+
385
355
return KafkaMessage (
386
356
key = key ,
387
357
value = value ,
0 commit comments