ktables 0.1.0__py3-none-any.whl

This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
ktables/__init__.py ADDED
@@ -0,0 +1,25 @@
1
+ """ktables — materialize Kafka topics into in-memory dicts (GlobalKTable for asyncio).
2
+
3
+ See the kafka_table module docstring for the consistency contract and
4
+ design notes, and README.md for usage.
5
+ """
6
+
7
+ from ktables.kafka_table import (
8
+ DEFAULT_TOPIC_CONFIGS,
9
+ KafkaTable,
10
+ KafkaTableWriter,
11
+ SupportsJsonModel,
12
+ TableStatus,
13
+ ViewStats,
14
+ ensure_topic,
15
+ )
16
+
17
+ __all__ = [
18
+ "DEFAULT_TOPIC_CONFIGS",
19
+ "KafkaTable",
20
+ "KafkaTableWriter",
21
+ "SupportsJsonModel",
22
+ "TableStatus",
23
+ "ViewStats",
24
+ "ensure_topic",
25
+ ]
ktables/kafka_table.py ADDED
@@ -0,0 +1,568 @@
1
+ """KafkaTable — materialize a Kafka topic into an in-memory dict.
2
+
3
+ A generic, self-contained "GlobalKTable at home": every process that runs a
4
+ table replays the topic from the beginning into a local last-write-wins dict
5
+ keyed by the Kafka message key, then keeps consuming for live updates. A
6
+ record with a null value is a tombstone and deletes the key.
7
+
8
+ Verified design facts (traced through aiokafka 0.13.0 source during review):
9
+ - ``group_id=None`` + constructor topic: assignment is populated synchronously
10
+ inside ``consumer.start()`` (NoGroupCoordinator) — no race with our seek.
11
+ - The catch-up gate (positions vs a start-time ``end_offsets`` snapshot)
12
+ survives compaction holes and transaction control markers: the fetcher
13
+ advances ``position()`` past compacted-away offsets and control batches.
14
+ - Partition assignment is NOT fixed at start: NoGroupCoordinator listens for
15
+ metadata changes and auto-assigns new partitions (they replay from earliest).
16
+ While still catching up, the gate extends itself to newly seen partitions;
17
+ after the latch, new partitions' records simply arrive as live updates.
18
+ - Broker restarts recover transparently (no group/session to lose). A
19
+ NON-retriable reader error (e.g. authorization) kills the reader task: that
20
+ failure is captured, logged loudly, and surfaced via ``status``/``failure``.
21
+
22
+ Consistency contract (the four guarantees):
23
+ 1. After ``start()``/``async with``: complete as of the start-time end offsets
24
+ (unless ``status == "degraded"`` — catch-up timed out, view may be partial).
25
+ 2. Thereafter: eventually consistent; publish→visible is typically a few ms.
26
+ 3. Contents are stable between *your* awaits (single event loop; only the
27
+ reader task mutates). ``snapshot()`` for copies you hold across awaits.
28
+ 4. NO read-your-own-writes: after ``await writer.set(k, v)``, a local
29
+ ``table.get(k)`` may return the old value until the broker round trip.
30
+
31
+ See README.md for usage. Integration tests need a Kafka broker on
32
+ localhost:9092: ``pytest tests``.
33
+ """
34
+
35
+ from __future__ import annotations
36
+
37
+ import asyncio
38
+ import logging
39
+ import time
40
+ from collections.abc import Callable, Iterator, Mapping
41
+ from dataclasses import dataclass
42
+ from types import MappingProxyType
43
+ from typing import TYPE_CHECKING, Generic, Literal, Protocol, TypeVar
44
+
45
+ if TYPE_CHECKING:
46
+ # Annotation-only (lazy annotations): no runtime typing_extensions
47
+ # dependency on 3.10.
48
+ from typing_extensions import Self
49
+
50
+ from aiokafka import AIOKafkaConsumer, AIOKafkaProducer, ConsumerRecord, TopicPartition
51
+ from aiokafka.admin import AIOKafkaAdminClient, NewTopic
52
+ from aiokafka.errors import TopicAlreadyExistsError
53
+
54
+ logger = logging.getLogger(__name__)
55
+
56
+ V = TypeVar("V")
57
+
58
+ DEFAULT_TOPIC_CONFIGS: Mapping[str, str] = MappingProxyType({"cleanup.policy": "compact"})
59
+
60
+ TableStatus = Literal["unstarted", "loading", "caught_up", "degraded", "failed"]
61
+ """``degraded``: catch-up timed out; serving possibly-partial data (loudly logged).
62
+ ``failed``: the reader task died; contents are frozen at the last applied state."""
63
+
64
+
65
+ class SupportsJsonModel(Protocol):
66
+ """The pydantic-v2 JSON surface the ``.json()`` presets rely on."""
67
+
68
+ def model_dump_json(self) -> str: ...
69
+
70
+ @classmethod
71
+ def model_validate_json(cls, json_data: str | bytes) -> Self: ...
72
+
73
+
74
+ JsonT = TypeVar("JsonT", bound=SupportsJsonModel)
75
+
76
+
77
+ async def ensure_topic(
78
+ bootstrap_servers: str,
79
+ topic: str,
80
+ *,
81
+ num_partitions: int = 1,
82
+ replication_factor: int = 1,
83
+ topic_configs: Mapping[str, str] | None = None,
84
+ ) -> bool:
85
+ """Idempotently create ``topic`` with an explicit config.
86
+
87
+ Returns True if this call created it, False if it already existed.
88
+ CreateTopics is atomic broker-side, so reader and writer racing to ensure
89
+ the same topic is benign — one wins, the other no-ops. This is the
90
+ EXPLICIT creation path; relying on broker auto-create is the bug (default
91
+ configs: cleanup.policy=delete), which this helper exists to make
92
+ unnecessary.
93
+
94
+ Any error other than already-exists (ACL denial, replication factor >
95
+ available brokers, broker unreachable) is logged with context and
96
+ re-raised — callers own retry/permission policy. The defaults
97
+ (1 partition, RF=1) are DEV-grade; production registries want RF>=3 with
98
+ min.insync.replicas=2 alongside an acks=all writer.
99
+ """
100
+ if num_partitions < 1 or replication_factor < 1:
101
+ raise ValueError(f"num_partitions and replication_factor must be >= 1, got {num_partitions}/{replication_factor}")
102
+ admin = AIOKafkaAdminClient(bootstrap_servers=bootstrap_servers)
103
+ await admin.start()
104
+ try:
105
+ await admin.create_topics(
106
+ [
107
+ NewTopic(
108
+ name=topic,
109
+ num_partitions=num_partitions,
110
+ replication_factor=replication_factor,
111
+ topic_configs=dict(topic_configs) if topic_configs is not None else dict(DEFAULT_TOPIC_CONFIGS),
112
+ )
113
+ ]
114
+ )
115
+ logger.info("created topic %s (partitions=%d, rf=%d)", topic, num_partitions, replication_factor)
116
+ return True
117
+ except TopicAlreadyExistsError:
118
+ logger.debug("topic %s already exists", topic)
119
+ return False
120
+ except Exception:
121
+ logger.exception(
122
+ "ensure_topic failed for topic=%s on %s (not an already-exists). If this process should not create topics, construct with ensure_topic=False.", # noqa: E501
123
+ topic,
124
+ bootstrap_servers,
125
+ )
126
+ raise
127
+ finally:
128
+ await admin.close()
129
+
130
+
131
+ def _utf8_decode(b: bytes) -> str:
132
+ return b.decode("utf-8")
133
+
134
+
135
+ def _utf8_encode(s: str) -> bytes:
136
+ return s.encode("utf-8")
137
+
138
+
139
+ @dataclass(frozen=True, slots=True)
140
+ class ViewStats:
141
+ """An immutable point-in-time snapshot of the reader's counters."""
142
+
143
+ records_applied: int = 0
144
+ tombstones_applied: int = 0
145
+ keyless_records: int = 0
146
+ key_decode_errors: int = 0
147
+ value_decode_errors: int = 0
148
+ catch_up_seconds: float | None = None
149
+ replayed_at_catch_up: int = 0
150
+
151
+
152
+ class _LiveStats:
153
+ """Internal mutable counters; ``freeze()`` produces the public snapshot."""
154
+
155
+ __slots__ = tuple(ViewStats.__dataclass_fields__)
156
+
157
+ def __init__(self) -> None:
158
+ self.records_applied = 0
159
+ self.tombstones_applied = 0
160
+ self.keyless_records = 0
161
+ self.key_decode_errors = 0
162
+ self.value_decode_errors = 0
163
+ self.catch_up_seconds: float | None = None
164
+ self.replayed_at_catch_up = 0
165
+
166
+ def freeze(self) -> ViewStats:
167
+ return ViewStats(**{f: getattr(self, f) for f in self.__slots__})
168
+
169
+
170
+ class KafkaTable(Mapping[str, V]):
171
+ """An IMMUTABLE Mapping materialized from a Kafka topic, LWW per key.
172
+
173
+ Honest dict-likeness: read-only ``Mapping`` only (``table[k]``, ``k in
174
+ table``, iteration, ``.get``) — deliberately NOT ``MutableMapping``; writes
175
+ go through :class:`KafkaTableWriter`, because there is no
176
+ read-your-own-writes (a just-published record is visible only after the
177
+ broker round trip).
178
+
179
+ A running table is a resource handle, not a value: equality is identity
180
+ (two tables with momentarily equal contents are not "the same table").
181
+ Not thread-safe; single event loop only. Reads before ``start()`` raise.
182
+ """
183
+
184
+ def __init__(
185
+ self,
186
+ *,
187
+ bootstrap_servers: str,
188
+ topic: str,
189
+ value_decoder: Callable[[bytes], V],
190
+ key_decoder: Callable[[bytes], str] = _utf8_decode,
191
+ catchup_timeout: float = 30.0,
192
+ poll_timeout_ms: int = 200,
193
+ ensure_topic: bool = True,
194
+ topic_configs: Mapping[str, str] | None = None,
195
+ ) -> None:
196
+ if not bootstrap_servers or not topic:
197
+ raise ValueError("bootstrap_servers and topic must be non-empty")
198
+ if catchup_timeout <= 0:
199
+ raise ValueError("catchup_timeout must be > 0")
200
+ if not callable(value_decoder) or not callable(key_decoder):
201
+ raise TypeError("value_decoder and key_decoder must be callable")
202
+ self._bootstrap_servers = bootstrap_servers
203
+ self._topic = topic
204
+ self._value_decoder = value_decoder
205
+ self._key_decoder = key_decoder
206
+ self._catchup_timeout = catchup_timeout
207
+ self._poll_timeout_ms = poll_timeout_ms
208
+ # Idempotent ensure on start: reader or writer may come up first.
209
+ # Disable where the app lacks topic-create ACLs (see ensure_topic()).
210
+ self._ensure_topic = ensure_topic
211
+ self._topic_configs = dict(topic_configs) if topic_configs is not None else dict(DEFAULT_TOPIC_CONFIGS)
212
+
213
+ self._live = _LiveStats()
214
+ self._caught_up = asyncio.Event()
215
+ self._failed = asyncio.Event() # wakes catch-up waiters on reader death
216
+ self._data: dict[str, V] = {}
217
+ self._task: asyncio.Task[None] | None = None
218
+ self._consumer: AIOKafkaConsumer | None = None
219
+ self._started = False
220
+ self._timed_out = False
221
+ self._failure: BaseException | None = None
222
+
223
+ # -- read API (Mapping) ----------------------------------------------------
224
+
225
+ # Mapping injects contents-__eq__ and __hash__=None; a table is a
226
+ # resource handle, so restore identity semantics.
227
+ def __eq__(self, other: object) -> bool:
228
+ return self is other
229
+
230
+ def __hash__(self) -> int:
231
+ return id(self)
232
+
233
+ def _require_started(self) -> None:
234
+ if not self._started:
235
+ raise RuntimeError("table not started — use 'async with table:' or call start()")
236
+
237
+ def __getitem__(self, key: str) -> V:
238
+ self._require_started()
239
+ return self._data[key]
240
+
241
+ def __iter__(self) -> Iterator[str]:
242
+ self._require_started()
243
+ return iter(self._data)
244
+
245
+ def __len__(self) -> int:
246
+ self._require_started()
247
+ return len(self._data)
248
+
249
+ def snapshot(self) -> dict[str, V]:
250
+ self._require_started()
251
+ return dict(self._data)
252
+
253
+ def __repr__(self) -> str:
254
+ return f"<KafkaTable topic={self._topic!r} status={self.status} size={len(self._data)}>"
255
+
256
+ @classmethod
257
+ def json(cls, *, bootstrap_servers: str, topic: str, model: type[JsonT], **kwargs: object) -> Self:
258
+ """Preset for pydantic-v2-shaped models (anything satisfying
259
+ :class:`SupportsJsonModel`); pydantic itself is NOT a dependency."""
260
+ return KafkaTable(bootstrap_servers=bootstrap_servers, topic=topic, value_decoder=model.model_validate_json, **kwargs) # type: ignore[arg-type]
261
+
262
+ # -- introspection ---------------------------------------------------------
263
+
264
+ @property
265
+ def topic(self) -> str:
266
+ return self._topic
267
+
268
+ @property
269
+ def stats(self) -> ViewStats:
270
+ """A frozen snapshot of the reader's counters (the live ones keep moving)."""
271
+ return self._live.freeze()
272
+
273
+ @property
274
+ def failure(self) -> BaseException | None:
275
+ """The exception that killed the reader task, if it died. See ``status``."""
276
+ return self._failure
277
+
278
+ @property
279
+ def status(self) -> TableStatus:
280
+ if self._failure is not None:
281
+ return "failed"
282
+ if not self._started:
283
+ return "unstarted"
284
+ if self._caught_up.is_set():
285
+ return "caught_up"
286
+ if self._timed_out:
287
+ return "degraded"
288
+ return "loading"
289
+
290
+ @property
291
+ def is_caught_up(self) -> bool:
292
+ return self._caught_up.is_set()
293
+
294
+ async def wait_until_caught_up(self, timeout: float | None = None) -> bool:
295
+ """Wait for replay to reach the start-time end offsets; True if reached.
296
+
297
+ Returns False on timeout OR if the reader has died (check ``status``).
298
+ """
299
+ if self._failure is not None:
300
+ return False
301
+ # Race catch-up against reader death so a death wakes this
302
+ # immediately. No shield: cancelling these waiters is harmless.
303
+ caught = asyncio.ensure_future(self._caught_up.wait())
304
+ failed = asyncio.ensure_future(self._failed.wait())
305
+ try:
306
+ await asyncio.wait({caught, failed}, timeout=timeout, return_when=asyncio.FIRST_COMPLETED)
307
+ finally:
308
+ for fut in (caught, failed):
309
+ fut.cancel()
310
+ return self._caught_up.is_set() and self._failure is None
311
+
312
+ # -- lifecycle -----------------------------------------------------------
313
+
314
+ async def start(self) -> None:
315
+ """Start the reader and wait (bounded) for catch-up.
316
+
317
+ Raises on double-start, on a missing topic, and if the reader dies
318
+ during catch-up. On catch-up *timeout* the table stays usable and
319
+ keeps consuming (``status == "degraded"``, loudly logged) — mirrors a
320
+ boot-gate policy of serve-degraded rather than crash-loop.
321
+
322
+ After a reader-death failure the table counts as started: reads serve
323
+ the frozen last-applied state (possibly empty) rather than raising
324
+ "not started" — gate on ``status``/``failure`` for liveness decisions.
325
+ """
326
+ if self._started:
327
+ raise RuntimeError(f"KafkaTable for topic={self._topic!r} already started")
328
+ if self._ensure_topic:
329
+ await ensure_topic(self._bootstrap_servers, self._topic, topic_configs=self._topic_configs)
330
+ # Constructor topic + group_id=None: all partitions are assigned
331
+ # synchronously inside start() — no group, no commits, no rebalance.
332
+ # (Manual assign()+partitions_for_topic() hits a stale partition
333
+ # cache on fresh topics.)
334
+ consumer = AIOKafkaConsumer(
335
+ self._topic,
336
+ bootstrap_servers=self._bootstrap_servers,
337
+ group_id=None,
338
+ enable_auto_commit=False,
339
+ auto_offset_reset="earliest",
340
+ )
341
+ await consumer.start()
342
+ self._consumer = consumer
343
+ try:
344
+ tps = sorted(consumer.assignment(), key=lambda tp: tp.partition)
345
+ if not tps:
346
+ raise RuntimeError(
347
+ f"topic {self._topic!r}: no partitions assigned. Topic missing — or, if ensure_topic ran, "
348
+ "check the ensure_topic log above for a masked creation error (e.g. ACLs)."
349
+ )
350
+ # Keep: auto_offset_reset only covers the no-valid-position
351
+ # path; this makes replay-from-zero unconditional.
352
+ await consumer.seek_to_beginning(*tps)
353
+ # Gate target: HWM at start; later records are live updates.
354
+ end_offsets: dict[TopicPartition, int] = await consumer.end_offsets(tps)
355
+ except BaseException:
356
+ await consumer.stop()
357
+ self._consumer = None
358
+ raise
359
+ self._started = True
360
+ task = asyncio.create_task(self._run(consumer, tps, end_offsets, time.perf_counter()), name=f"kafka-table:{self._topic}")
361
+ task.add_done_callback(self._on_reader_done)
362
+ self._task = task
363
+ if not await self.wait_until_caught_up(timeout=self._catchup_timeout):
364
+ if self._failure is not None:
365
+ # Reader died during boot: fail start() loudly, cleaned up.
366
+ failure = self._failure
367
+ await self.stop()
368
+ raise RuntimeError(f"KafkaTable reader for topic={self._topic!r} died during catch-up") from failure
369
+ self._timed_out = True
370
+ logger.error(
371
+ "table for topic=%s NOT caught up after %.1fs (applied=%d so far); continuing DEGRADED — data may be incomplete",
372
+ self._topic,
373
+ self._catchup_timeout,
374
+ self._live.records_applied,
375
+ )
376
+
377
+ def _on_reader_done(self, task: asyncio.Task[None]) -> None:
378
+ if task.cancelled():
379
+ return
380
+ exc = task.exception()
381
+ if exc is not None:
382
+ self._failure = exc
383
+ self._failed.set()
384
+ logger.error(
385
+ "KafkaTable reader for topic=%s DIED (%s); table is FROZEN at last applied state (size=%d, status=failed)",
386
+ self._topic,
387
+ type(exc).__name__,
388
+ len(self._data),
389
+ exc_info=exc,
390
+ )
391
+
392
+ async def stop(self) -> None:
393
+ task, self._task = self._task, None
394
+ if task is not None:
395
+ task.cancel()
396
+ try:
397
+ await task
398
+ except asyncio.CancelledError:
399
+ pass
400
+ except Exception:
401
+ # Logged by _on_reader_done; re-raising here would mask
402
+ # caller errors and skip consumer cleanup.
403
+ pass
404
+ consumer, self._consumer = self._consumer, None
405
+ if consumer is not None:
406
+ try:
407
+ await consumer.stop()
408
+ except Exception:
409
+ logger.exception("consumer.stop() failed for topic=%s during teardown", self._topic)
410
+
411
+ async def __aenter__(self) -> Self:
412
+ await self.start()
413
+ return self
414
+
415
+ async def __aexit__(self, *exc: object) -> None:
416
+ await self.stop()
417
+
418
+ # -- reader loop ----------------------------------------------------------
419
+
420
+ def _apply(self, record: ConsumerRecord) -> None:
421
+ where = f"topic={self._topic} partition={record.partition} offset={record.offset}"
422
+ key_bytes = record.key
423
+ if key_bytes is None:
424
+ self._live.keyless_records += 1
425
+ logger.warning("keyless record skipped (%s) — producer is writing unkeyed records to a keyed table topic", where)
426
+ return
427
+ try:
428
+ key = self._key_decoder(key_bytes)
429
+ except Exception:
430
+ self._live.key_decode_errors += 1
431
+ logger.exception("undecodable key skipped (%s)", where)
432
+ return
433
+ if record.value is None: # null value = tombstone (b"" is NOT a tombstone)
434
+ self._data.pop(key, None)
435
+ self._live.tombstones_applied += 1
436
+ return
437
+ try:
438
+ self._data[key] = self._value_decoder(record.value)
439
+ except Exception:
440
+ # Poison tolerance: keep the prior value, never kill the reader.
441
+ self._live.value_decode_errors += 1
442
+ logger.exception("undecodable value skipped (key=%s, %s)", key, where)
443
+ return
444
+ self._live.records_applied += 1
445
+
446
+ async def _run(
447
+ self,
448
+ consumer: AIOKafkaConsumer,
449
+ tps: list[TopicPartition],
450
+ end_offsets: dict[TopicPartition, int],
451
+ started: float,
452
+ ) -> None:
453
+ # Escaping exceptions are captured by _on_reader_done (status
454
+ # "failed"); transient outages don't raise — getmany returns empty.
455
+ while True:
456
+ batches = await consumer.getmany(timeout_ms=self._poll_timeout_ms)
457
+ for records in batches.values():
458
+ for record in records:
459
+ self._apply(record)
460
+ if not self._caught_up.is_set():
461
+ # aiokafka auto-assigns new partitions on metadata change;
462
+ # pre-latch, extend the gate — post-latch they arrive as
463
+ # live updates.
464
+ current = consumer.assignment()
465
+ if current and set(tps) != current:
466
+ new = sorted(current - set(tps), key=lambda tp: tp.partition)
467
+ if new:
468
+ end_offsets.update(await consumer.end_offsets(new))
469
+ tps = sorted(current, key=lambda tp: tp.partition)
470
+ positions = [await consumer.position(tp) for tp in tps]
471
+ if all(pos >= end_offsets[tp] for pos, tp in zip(positions, tps, strict=True)):
472
+ self._live.catch_up_seconds = time.perf_counter() - started
473
+ self._live.replayed_at_catch_up = self._live.records_applied + self._live.tombstones_applied
474
+ self._caught_up.set()
475
+
476
+
477
+ class KafkaTableWriter(Generic[V]):
478
+ """Writer counterpart of :class:`KafkaTable`: keyed upserts + tombstones.
479
+
480
+ ``set(key, value)`` publishes the encoded value under the key (the table's
481
+ LWW upsert); ``delete(key)`` publishes a null-value tombstone. A periodic
482
+ re-``set`` of the same value is a heartbeat — no separate API needed.
483
+
484
+ Registry-grade durability by default: ``enable_idempotence=True`` (which
485
+ implies ``acks=all``), so a leader failover can't drop an acked record and
486
+ producer retries can't duplicate or reorder. Opt out for throwaway data.
487
+
488
+ The key encoder must be deterministic and stable across processes and
489
+ versions — on a multi-partition topic, per-key LWW ordering holds only if
490
+ a key always hashes to the same partition.
491
+ """
492
+
493
+ def __init__(
494
+ self,
495
+ *,
496
+ bootstrap_servers: str,
497
+ topic: str,
498
+ value_encoder: Callable[[V], bytes],
499
+ key_encoder: Callable[[str], bytes] = _utf8_encode,
500
+ ensure_topic: bool = True,
501
+ topic_configs: Mapping[str, str] | None = None,
502
+ enable_idempotence: bool = True,
503
+ ) -> None:
504
+ if not bootstrap_servers or not topic:
505
+ raise ValueError("bootstrap_servers and topic must be non-empty")
506
+ if not callable(value_encoder) or not callable(key_encoder):
507
+ raise TypeError("value_encoder and key_encoder must be callable")
508
+ self._bootstrap_servers = bootstrap_servers
509
+ self._topic = topic
510
+ self._value_encoder = value_encoder
511
+ self._key_encoder = key_encoder
512
+ self._ensure_topic = ensure_topic
513
+ self._topic_configs = dict(topic_configs) if topic_configs is not None else dict(DEFAULT_TOPIC_CONFIGS)
514
+ self._enable_idempotence = enable_idempotence
515
+ self._producer: AIOKafkaProducer | None = None
516
+
517
+ def __repr__(self) -> str:
518
+ return f"<KafkaTableWriter topic={self._topic!r} started={self._producer is not None}>"
519
+
520
+ @classmethod
521
+ def json(cls, *, bootstrap_servers: str, topic: str, model: type[JsonT] | None = None, **kwargs: object) -> Self:
522
+ """Preset for pydantic-v2-shaped values (encodes via ``model_dump_json``).
523
+
524
+ ``model`` is typing/documentation-only; it is not used at runtime.
525
+ """
526
+
527
+ def encode(v: JsonT) -> bytes:
528
+ return v.model_dump_json().encode()
529
+
530
+ return KafkaTableWriter(bootstrap_servers=bootstrap_servers, topic=topic, value_encoder=encode, **kwargs) # type: ignore[arg-type]
531
+
532
+ async def start(self) -> None:
533
+ if self._producer is not None:
534
+ raise RuntimeError(f"KafkaTableWriter for topic={self._topic!r} already started")
535
+ if self._ensure_topic:
536
+ await ensure_topic(self._bootstrap_servers, self._topic, topic_configs=self._topic_configs)
537
+ producer = AIOKafkaProducer(bootstrap_servers=self._bootstrap_servers, enable_idempotence=self._enable_idempotence)
538
+ await producer.start()
539
+ self._producer = producer
540
+
541
+ async def stop(self) -> None:
542
+ producer, self._producer = self._producer, None
543
+ if producer is not None:
544
+ await producer.stop()
545
+
546
+ async def __aenter__(self) -> Self:
547
+ await self.start()
548
+ return self
549
+
550
+ async def __aexit__(self, *exc: object) -> None:
551
+ await self.stop()
552
+
553
+ def _require_producer(self) -> AIOKafkaProducer:
554
+ if self._producer is None:
555
+ raise RuntimeError("writer not started — use 'async with writer:' or call start()")
556
+ return self._producer
557
+
558
+ async def set(self, key: str, value: V) -> None:
559
+ """Upsert: publish ``value`` under ``key`` (awaits broker ack).
560
+
561
+ Note: no read-your-own-writes — a table in this same process sees this
562
+ record only after the broker round trip (~ms), not synchronously.
563
+ """
564
+ await self._require_producer().send_and_wait(self._topic, value=self._value_encoder(value), key=self._key_encoder(key))
565
+
566
+ async def delete(self, key: str) -> None:
567
+ """Tombstone: publish a null value under ``key`` (awaits broker ack)."""
568
+ await self._require_producer().send_and_wait(self._topic, value=None, key=self._key_encoder(key))
@@ -0,0 +1,200 @@
1
+ Metadata-Version: 2.4
2
+ Name: ktables
3
+ Version: 0.1.0
4
+ Summary: Materialize a Kafka topic into an in-memory, compacted dict — a GlobalKTable for asyncio Python.
5
+ License-Expression: MIT
6
+ License-File: LICENSE
7
+ Requires-Python: >=3.10
8
+ Requires-Dist: aiokafka>=0.10
9
+ Description-Content-Type: text/markdown
10
+
11
+ # ktables
12
+
13
+ [![Tests](https://github.com/ryan-yuuu/ktables/actions/workflows/test.yml/badge.svg)](https://github.com/ryan-yuuu/ktables/actions/workflows/test.yml)
14
+ ![Python 3.10+](https://img.shields.io/badge/python-3.10%2B-blue)
15
+ [![License: MIT](https://img.shields.io/badge/license-MIT-green)](LICENSE)
16
+
17
+ Materialize a Kafka topic into an in-memory, compacted dict — a GlobalKTable for asyncio Python.
18
+
19
+ Every process that opens a `KafkaTable` replays the topic from the beginning
20
+ into a local read-only mapping, then keeps consuming for live updates; a
21
+ `KafkaTableWriter` maintains the topic with keyed upserts and tombstones.
22
+ Built for small, broadly-needed reference data — service registries,
23
+ capability advertisements, feature flags, config maps — not for large or
24
+ high-churn state.
25
+
26
+ ## Table of Contents
27
+
28
+ - [Background](#background)
29
+ - [Install](#install)
30
+ - [Usage](#usage)
31
+ - [Consistency contract](#consistency-contract)
32
+ - [API](#api)
33
+ - [Contributing](#contributing)
34
+ - [License](#license)
35
+
36
+ ## Background
37
+
38
+ Kafka Streams (JVM) has two table abstractions over changelog topics: the
39
+ partition-sharded `KTable`, where each application instance holds a slice of
40
+ the keys, and `GlobalKTable`, where every instance bootstraps and maintains a
41
+ full local copy — the right shape for lookup data that any instance may need
42
+ at any moment. The Python ecosystem has several maintained stream-processing
43
+ frameworks, but all of them implement only the sharded shape, with
44
+ framework-owned changelog topics and their own process runtimes. ktables fills
45
+ the gap with just the global-table piece, as a plain asyncio library over
46
+ `aiokafka`: your topic, your message format, your event loop.
47
+
48
+ The Kafka semantics the implementation relies on (group-less consumers,
49
+ catch-up gating against end offsets, compaction independence) are documented,
50
+ with provenance, in the module docstring of
51
+ [`kafka_table.py`](./ktables/kafka_table.py).
52
+
53
+ ## Install
54
+
55
+ ktables is not yet published to PyPI. Until the first release, vendor the
56
+ package — copy the `ktables/` directory into your project — and install its
57
+ single runtime dependency:
58
+
59
+ ```sh
60
+ $ pip install aiokafka
61
+ ```
62
+
63
+ Requires Python 3.10+. Pydantic is **not** required — the `.json()` presets
64
+ accept any class with pydantic-v2's JSON methods.
65
+
66
+ ## Usage
67
+
68
+ Maintain a registry from one service:
69
+
70
+ ```python
71
+ from ktables import KafkaTableWriter
72
+
73
+ writer = KafkaTableWriter.json(
74
+ bootstrap_servers="localhost:9092", topic="my.registry", model=ServiceRecord
75
+ )
76
+ async with writer:
77
+ await writer.set("billing", record) # upsert (broker-acked)
78
+ ...
79
+ await writer.delete("billing") # tombstone: removes the key
80
+ ```
81
+
82
+ Consume it from any other process:
83
+
84
+ ```python
85
+ from ktables import KafkaTable
86
+
87
+ table = KafkaTable.json(
88
+ bootstrap_servers="localhost:9092", topic="my.registry", model=ServiceRecord
89
+ )
90
+ async with table: # replays the topic; returns once caught up
91
+ record = table.get("billing")
92
+ if table.status != "caught_up": # "degraded": catch-up timed out
93
+ ...
94
+ ```
95
+
96
+ Non-pydantic payloads: construct directly with your own codecs —
97
+ `KafkaTable(..., value_decoder=bytes_to_value)` /
98
+ `KafkaTableWriter(..., value_encoder=value_to_bytes)`.
99
+
100
+ ### Removing a key on clean shutdown
101
+
102
+ There is deliberately no `delete_on_close` option (shutdown-time deletion is
103
+ application policy, and no library can promise it on a crash). Compose it:
104
+
105
+ ```python
106
+ async with writer:
107
+ await writer.set(my_key, my_record)
108
+ try:
109
+ ... # serve
110
+ finally:
111
+ await writer.delete(my_key) # acked before the producer stops
112
+ ```
113
+
114
+ ### Locked-down clusters
115
+
116
+ Both classes ensure their topic exists at start (idempotent create,
117
+ compacted). If the application lacks topic-create ACLs, pass
118
+ `ensure_topic=False` and create the topic out-of-band (the module-level
119
+ `ensure_topic()` function is the deploy-time primitive).
120
+
121
+ ## Consistency contract
122
+
123
+ `KafkaTable` is eventually consistent. Precisely:
124
+
125
+ 1. When `start()` / `async with` returns, contents are complete as of the
126
+ topic's end offsets at start time — unless `status == "degraded"`
127
+ (catch-up timed out; data may be partial).
128
+ 2. Thereafter, updates appear within milliseconds of the broker write — but
129
+ **there is no read-your-own-writes**: after `await writer.set(k, v)`, a
130
+ table in the same process may briefly still return the old value.
131
+ 3. Contents are stable between your awaits (single event loop; only the
132
+ reader task mutates). Use `snapshot()` for a copy held across awaits.
133
+ 4. Correctness does not depend on broker-side compaction: last-write-wins
134
+ over the full log yields the same dict; compaction only bounds replay time.
135
+
136
+ A tombstone is a record with a **null** value (`b""` is data, not a tombstone).
137
+ If the background reader dies (non-retriable error, e.g. authorization),
138
+ contents freeze at the last applied state: `status` becomes `"failed"` and
139
+ `failure` holds the exception — gate liveness decisions on `status`, never on
140
+ reads alone. Transient broker outages do not kill the reader; it resumes.
141
+
142
+ ## API
143
+
144
+ ### `KafkaTable[V]` — read-only `Mapping[str, V]`
145
+
146
+ | Member | Description |
147
+ |---|---|
148
+ | `KafkaTable(*, bootstrap_servers, topic, value_decoder, key_decoder=utf-8, catchup_timeout=30.0, poll_timeout_ms=200, ensure_topic=True, topic_configs=None)` | Construct (does not connect). |
149
+ | `KafkaTable.json(*, bootstrap_servers, topic, model, **kwargs)` | Preset wiring `model.model_validate_json` as the decoder. |
150
+ | `start()` / `stop()` / `async with` | Lifecycle. `start()` raises on double-start, missing topic, or reader death during catch-up; on catch-up *timeout* it serves degraded. |
151
+ | `table[key]`, `key in table`, `iter`, `len`, `.get(key, default=None)` | Mapping reads. Raise `RuntimeError` before `start()`. |
152
+ | `snapshot()` | Shallow-copy dict, safe to hold across awaits. |
153
+ | `status` | `"unstarted" \| "loading" \| "caught_up" \| "degraded" \| "failed"`. |
154
+ | `failure` | Exception that killed the reader, else `None`. |
155
+ | `is_caught_up` / `wait_until_caught_up(timeout=None)` | Catch-up gate; the wait returns `False` on timeout or reader death. |
156
+ | `stats` | Frozen `ViewStats` snapshot (see below). |
157
+
158
+ Equality is **identity** and instances are hashable: a running table is a
159
+ resource handle, not a value.
160
+
161
+ ### `KafkaTableWriter[V]`
162
+
163
+ | Member | Description |
164
+ |---|---|
165
+ | `KafkaTableWriter(*, bootstrap_servers, topic, value_encoder, key_encoder=utf-8, ensure_topic=True, topic_configs=None, enable_idempotence=True)` | Construct. Idempotence implies `acks=all` (registry-grade durability); opt out for throwaway data. |
166
+ | `KafkaTableWriter.json(*, bootstrap_servers, topic, model=None, **kwargs)` | Preset encoding via `model_dump_json()` (`model` is typing-only). |
167
+ | `set(key, value)` | Keyed upsert; awaits broker ack. Re-`set` periodically as a heartbeat. |
168
+ | `delete(key)` | Publishes a null-value tombstone; awaits broker ack. |
169
+ | `start()` / `stop()` / `async with` | Lifecycle; `set`/`delete` before start raise `RuntimeError`. |
170
+
171
+ The key encoder must be deterministic and stable across processes — on a
172
+ multi-partition topic, per-key ordering holds only if a key always hashes to
173
+ the same partition.
174
+
175
+ ### Module level
176
+
177
+ | Member | Description |
178
+ |---|---|
179
+ | `ensure_topic(bootstrap_servers, topic, *, num_partitions=1, replication_factor=1, topic_configs=None) -> bool` | Idempotent explicit create; `True` if this call created it. Defaults are dev-grade — production registries want RF≥3, `min.insync.replicas=2`, `acks=all`. |
180
+ | `DEFAULT_TOPIC_CONFIGS` | `{"cleanup.policy": "compact"}` (read-only mapping). |
181
+ | `ViewStats` | Frozen counters: `records_applied`, `tombstones_applied`, `keyless_records`, `key_decode_errors`, `value_decode_errors`, `catch_up_seconds`, `replayed_at_catch_up`. |
182
+ | `SupportsJsonModel` | Protocol the `.json()` presets require (`model_dump_json` / `model_validate_json`). |
183
+ | `TableStatus` | The `status` literal type. |
184
+
185
+ ## Contributing
186
+
187
+ Questions and bug reports are welcome as issues, and PRs are accepted. Please
188
+ run the test suite before submitting:
189
+
190
+ ```sh
191
+ $ pytest tests
192
+ ```
193
+
194
+ Unit tests always run; integration tests need a Kafka broker on
195
+ `localhost:9092` and skip otherwise
196
+ (`docker run -d -p 9092:9092 apache/kafka:3.9.0`).
197
+
198
+ ## License
199
+
200
+ [MIT](LICENSE)
@@ -0,0 +1,6 @@
1
+ ktables/__init__.py,sha256=cJQ2oKxwKGnPSezV06bTtLZUMJBv2P2Bf6ANTo4ZI0g,545
2
+ ktables/kafka_table.py,sha256=LsaVKgg5VFjHc2812Dj3A6GmDhvio7rpTgxRXFiiF-4,23973
3
+ ktables-0.1.0.dist-info/METADATA,sha256=As1UFYF_GSmwvAIfFpmEqD-cQ492SD10Ya8U6xGWOQ4,8587
4
+ ktables-0.1.0.dist-info/WHEEL,sha256=mffPy8wBnZQn2VnJUU5jE99KsxaSfiyMHV9Yt0aLVxs,87
5
+ ktables-0.1.0.dist-info/licenses/LICENSE,sha256=h8B4q52QowRPNZ5rVj1Oav6nqgYIl89PIWGGh7OFZ5Y,1061
6
+ ktables-0.1.0.dist-info/RECORD,,
@@ -0,0 +1,4 @@
1
+ Wheel-Version: 1.0
2
+ Generator: hatchling 1.30.1
3
+ Root-Is-Purelib: true
4
+ Tag: py3-none-any
@@ -0,0 +1,21 @@
1
+ MIT License
2
+
3
+ Copyright (c) 2026 Ryan
4
+
5
+ Permission is hereby granted, free of charge, to any person obtaining a copy
6
+ of this software and associated documentation files (the "Software"), to deal
7
+ in the Software without restriction, including without limitation the rights
8
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
9
+ copies of the Software, and to permit persons to whom the Software is
10
+ furnished to do so, subject to the following conditions:
11
+
12
+ The above copyright notice and this permission notice shall be included in all
13
+ copies or substantial portions of the Software.
14
+
15
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
16
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
17
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
18
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
19
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
20
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
21
+ SOFTWARE.