/
topic.py
284 lines (248 loc) · 9.37 KB
/
topic.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
import dataclasses
import logging
from typing import List, Optional, Any, Callable, Union
from confluent_kafka.admin import NewTopic, ConfigResource # type: ignore
from quixstreams.models.messagecontext import MessageContext
from quixstreams.models.messages import KafkaMessage
from quixstreams.models.rows import Row
from quixstreams.models.serializers import (
SerializationContext,
DeserializerIsNotProvidedError,
SerializerIsNotProvidedError,
BytesSerializer,
BytesDeserializer,
IgnoreMessage,
SERIALIZERS,
DESERIALIZERS,
SerializerType,
DeserializerType,
Serializer,
Deserializer,
)
from quixstreams.models.timestamps import MessageTimestamp, TimestampType
from quixstreams.models.types import (
ConfluentKafkaMessageProto,
Headers,
MessageHeadersTuples,
)
__all__ = ("Topic", "TopicConfig", "TimestampExtractor")
logger = logging.getLogger(__name__)
TimestampExtractor = Callable[
[Any, Optional[MessageHeadersTuples], int, TimestampType],
int,
]
@dataclasses.dataclass(eq=True)
class TopicConfig:
"""
Represents all kafka-level configuration for a kafka topic.
Generally used by Topic and any topic creation procedures.
"""
num_partitions: int
replication_factor: int
extra_config: dict = dataclasses.field(default_factory=dict)
def as_dict(self):
return dataclasses.asdict(self)
def _get_serializer(serializer: Optional[SerializerType]) -> Optional[Serializer]:
if isinstance(serializer, str):
try:
return SERIALIZERS[serializer]()
except KeyError:
raise ValueError(
f"Unknown deserializer option '{serializer}'; "
f"valid options are {list(SERIALIZERS.keys())}"
)
return serializer
def _get_deserializer(
deserializer: Optional[DeserializerType],
) -> Optional[Deserializer]:
if isinstance(deserializer, str):
try:
return DESERIALIZERS[deserializer]()
except KeyError:
raise ValueError(
f"Unknown deserializer option '{deserializer}'; "
f"valid options are {list(DESERIALIZERS.keys())}"
)
return deserializer
class Topic:
"""
A definition of a Kafka topic.
Typically created with an `app = quixstreams.app.Application()` instance via
`app.topic()`, and used by `quixstreams.dataframe.StreamingDataFrame`
instance.
"""
def __init__(
self,
name: str,
config: TopicConfig,
value_deserializer: Optional[DeserializerType] = None,
key_deserializer: Optional[DeserializerType] = BytesDeserializer(),
value_serializer: Optional[SerializerType] = None,
key_serializer: Optional[SerializerType] = BytesSerializer(),
timestamp_extractor: Optional[TimestampExtractor] = None,
):
"""
:param name: topic name
:param value_deserializer: a deserializer type for values
:param key_deserializer: a deserializer type for keys
:param value_serializer: a serializer type for values
:param key_serializer: a serializer type for keys
:param config: optional topic configs via `TopicConfig` (creation/validation)
:param timestamp_extractor: a callable that returns a timestamp in
milliseconds from a deserialized message.
"""
self._name = name
self._config = config
self._key_serializer = _get_serializer(key_serializer)
self._key_deserializer = _get_deserializer(key_deserializer)
self._value_serializer = _get_serializer(value_serializer)
self._value_deserializer = _get_deserializer(value_deserializer)
self._timestamp_extractor = timestamp_extractor
@property
def name(self) -> str:
"""
Topic name
"""
return self._name
@property
def config(self) -> TopicConfig:
return self._config
def row_serialize(self, row: Row, key: Optional[Any] = None) -> KafkaMessage:
"""
Serialize Row to a Kafka message structure
:param row: Row to serialize
:param key: message key to serialize, optional. Default - current Row key.
:return: KafkaMessage object with serialized values
"""
ctx = SerializationContext(topic=self.name, headers=row.headers)
if self._key_serializer is None:
raise SerializerIsNotProvidedError(
f'Key serializer is not provided for topic "{self.name}"'
)
if self._value_serializer is None:
raise SerializerIsNotProvidedError(
f'Value serializer is not provided for topic "{self.name}"'
)
return KafkaMessage(
key=self._key_serializer(key or row.key, ctx=ctx),
value=self._value_serializer(row.value, ctx=ctx),
headers=self._value_serializer.extra_headers,
)
def row_deserialize(
self, message: ConfluentKafkaMessageProto
) -> Union[Row, List[Row], None]:
"""
Deserialize incoming Kafka message to a Row.
:param message: an object with interface of `confluent_kafka.Message`
:return: Row, list of Rows or None if the message is ignored.
"""
if self._key_deserializer is None:
raise DeserializerIsNotProvidedError(
f'Key deserializer is not provided for topic "{self.name}"'
)
if self._value_deserializer is None:
raise DeserializerIsNotProvidedError(
f'Value deserializer is not provided for topic "{self.name}"'
)
headers = message.headers()
ctx = SerializationContext(topic=message.topic(), headers=headers)
key = self._key_deserializer(value=message.key(), ctx=ctx)
try:
value = self._value_deserializer(value=message.value(), ctx=ctx)
except IgnoreMessage:
# Ignore message completely if deserializer raised IgnoreValueError.
logger.debug(
'Ignore incoming message: partition="%s[%s]" offset="%s"',
message.topic(),
message.partition(),
message.offset(),
)
return
if self._value_deserializer.split_values:
# The expected value from this serializer is Iterable and each item
# should be processed as a separate message
rows = []
for item in value:
rows.append(
Row(
value=item,
context=self._create_message_context(
message=message, key=key, headers=headers, value=item
),
)
)
return rows
return Row(
value=value,
context=self._create_message_context(
message=message, key=key, headers=headers, value=value
),
)
def serialize(
self,
key: Optional[object] = None,
value: Optional[object] = None,
headers: Optional[Headers] = None,
timestamp_ms: Optional[int] = None,
) -> KafkaMessage:
ctx = SerializationContext(topic=self.name, headers=headers)
if self._key_serializer:
key = self._key_serializer(key, ctx=ctx)
elif key is not None:
raise SerializerIsNotProvidedError(
f'Key serializer is not provided for topic "{self.name}"'
)
if self._value_serializer:
value = self._value_serializer(value, ctx=ctx)
elif value is not None:
raise SerializerIsNotProvidedError(
f'Value serializer is not provided for topic "{self.name}"'
)
return KafkaMessage(
key=key,
value=value,
headers=headers,
timestamp=timestamp_ms,
)
def deserialize(self, message: ConfluentKafkaMessageProto):
ctx = SerializationContext(topic=message.topic(), headers=message.headers())
return KafkaMessage(
key=(
self._key_deserializer(key, ctx=ctx) if (key := message.key()) else None
),
value=(
self._value_serializer(value, ctx=ctx)
if (value := message.value())
else None
),
headers=message.headers(),
timestamp=message.timestamp()[1],
)
def __repr__(self):
return f'<{self.__class__.__name__} name="{self._name}"> '
def _create_message_context(
self,
message: ConfluentKafkaMessageProto,
key: Any,
headers: Optional[MessageHeadersTuples],
value: Any,
) -> MessageContext:
timestamp_type, timestamp_ms = message.timestamp()
if self._timestamp_extractor:
timestamp_ms = self._timestamp_extractor(
value, headers, timestamp_ms, TimestampType(timestamp_type)
)
timestamp = MessageTimestamp.create(
timestamp_type=timestamp_type, milliseconds=timestamp_ms
)
return MessageContext(
key=key,
headers=headers,
topic=message.topic(),
partition=message.partition(),
offset=message.offset(),
size=len(message),
timestamp=timestamp,
latency=message.latency(),
leader_epoch=message.leader_epoch(),
)