|
| 1 | +import logging |
| 2 | + |
| 3 | +from abc import ABC, abstractmethod |
| 4 | +from typing import Optional, Union, Any, TYPE_CHECKING |
| 5 | + |
| 6 | +from quixstreams.models import ConfluentKafkaMessageProto |
| 7 | +from quixstreams.state.exceptions import ColumnFamilyHeaderMissing |
| 8 | +from quixstreams.state.metadata import ( |
| 9 | + CHANGELOG_CF_MESSAGE_HEADER, |
| 10 | + CHANGELOG_PROCESSED_OFFSET_MESSAGE_HEADER, |
| 11 | +) |
| 12 | +from quixstreams.state.serialization import DumpsFunc, LoadsFunc |
| 13 | +from quixstreams.utils.json import loads as json_loads |
| 14 | + |
| 15 | +from .transaction import PartitionTransaction, CACHE_TYPE |
| 16 | + |
| 17 | +if TYPE_CHECKING: |
| 18 | + from quixstreams.state.recovery import ChangelogProducer |
| 19 | + |
| 20 | + |
| 21 | +__all__ = ("StorePartition",) |
| 22 | + |
| 23 | +logger = logging.getLogger(__name__) |
| 24 | + |
| 25 | + |
| 26 | +class StorePartition(ABC): |
| 27 | + """ |
| 28 | + A base class to access state in the underlying storage. |
| 29 | + It represents a single instance of some storage (e.g. a single database for |
| 30 | + the persistent storage). |
| 31 | + """ |
| 32 | + |
| 33 | + def __init__( |
| 34 | + self, |
| 35 | + dumps: DumpsFunc, |
| 36 | + loads: LoadsFunc, |
| 37 | + changelog_producer: Optional["ChangelogProducer"], |
| 38 | + ) -> None: |
| 39 | + super().__init__() |
| 40 | + self._dumps = dumps |
| 41 | + self._loads = loads |
| 42 | + self._changelog_producer = changelog_producer |
| 43 | + |
| 44 | + @abstractmethod |
| 45 | + def close(self): ... |
| 46 | + |
| 47 | + @abstractmethod |
| 48 | + def _recover_from_changelog_message( |
| 49 | + self, |
| 50 | + changelog_message: ConfluentKafkaMessageProto, |
| 51 | + cf_name: str, |
| 52 | + processed_offset: Optional[int], |
| 53 | + committed_offset: int, |
| 54 | + ): ... |
| 55 | + |
| 56 | + @abstractmethod |
| 57 | + def get_processed_offset(self) -> Optional[int]: |
| 58 | + """ |
| 59 | + Get last processed offset for the given partition |
| 60 | + :return: offset or `None` if there's no processed offset yet |
| 61 | + """ |
| 62 | + ... |
| 63 | + |
| 64 | + @abstractmethod |
| 65 | + def get_changelog_offset(self) -> Optional[int]: |
| 66 | + """ |
| 67 | + Get offset that the changelog is up-to-date with. |
| 68 | + :return: offset or `None` if there's no processed offset yet |
| 69 | + """ |
| 70 | + ... |
| 71 | + |
| 72 | + @abstractmethod |
| 73 | + def write( |
| 74 | + self, |
| 75 | + data: CACHE_TYPE, |
| 76 | + processed_offset: Optional[int], |
| 77 | + changelog_offset: Optional[int], |
| 78 | + ): |
| 79 | + """ |
| 80 | + Update the state with data |
| 81 | +
|
| 82 | + :param data: The modified data |
| 83 | + :param processed_offset: The offset processed to generate the data. |
| 84 | + :param changelog_offset: The changelog message offset of the data. |
| 85 | + """ |
| 86 | + |
| 87 | + @abstractmethod |
| 88 | + def get( |
| 89 | + self, key: bytes, default: Any = None, cf_name: str = "default" |
| 90 | + ) -> Union[None, bytes, Any]: |
| 91 | + """ |
| 92 | + Get a key from the store |
| 93 | +
|
| 94 | + :param key: a key encoded to `bytes` |
| 95 | + :param default: a default value to return if the key is not found. |
| 96 | + :param cf_name: rocksdb column family name. Default - "default" |
| 97 | + :return: a value if the key is present in the store. Otherwise, `default` |
| 98 | + """ |
| 99 | + |
| 100 | + @abstractmethod |
| 101 | + def exists(self, key: bytes, cf_name: str = "default") -> bool: |
| 102 | + """ |
| 103 | + Check if a key is present in the store. |
| 104 | +
|
| 105 | + :param key: a key encoded to `bytes`. |
| 106 | + :param cf_name: rocksdb column family name. Default - "default" |
| 107 | + :return: `True` if the key is present, `False` otherwise. |
| 108 | + """ |
| 109 | + |
| 110 | + def begin(self) -> PartitionTransaction: |
| 111 | + """ |
| 112 | + Start a new `PartitionTransaction` |
| 113 | +
|
| 114 | + Using `PartitionTransaction` is a recommended way for accessing the data. |
| 115 | + """ |
| 116 | + return PartitionTransaction( |
| 117 | + partition=self, |
| 118 | + dumps=self._dumps, |
| 119 | + loads=self._loads, |
| 120 | + changelog_producer=self._changelog_producer, |
| 121 | + ) |
| 122 | + |
| 123 | + def recover_from_changelog_message( |
| 124 | + self, changelog_message: ConfluentKafkaMessageProto, committed_offset: int |
| 125 | + ): |
| 126 | + """ |
| 127 | + Updates state from a given changelog message. |
| 128 | +
|
| 129 | + :param changelog_message: A raw Confluent message read from a changelog topic. |
| 130 | + :param committed_offset: latest committed offset for the partition |
| 131 | + """ |
| 132 | + headers = dict(changelog_message.headers() or ()) |
| 133 | + # Parse the column family name from message headers |
| 134 | + cf_name = headers.get(CHANGELOG_CF_MESSAGE_HEADER, b"").decode() |
| 135 | + if not cf_name: |
| 136 | + raise ColumnFamilyHeaderMissing( |
| 137 | + f"Header '{CHANGELOG_CF_MESSAGE_HEADER}' missing from changelog message" |
| 138 | + ) |
| 139 | + |
| 140 | + # Parse the processed topic-partition-offset info from the changelog message |
| 141 | + # headers to determine whether the update should be applied or skipped. |
| 142 | + # It can be empty if the message was produced by the older version of the lib. |
| 143 | + processed_offset = json_loads( |
| 144 | + headers.get(CHANGELOG_PROCESSED_OFFSET_MESSAGE_HEADER, b"null") |
| 145 | + ) |
| 146 | + |
| 147 | + self._recover_from_changelog_message( |
| 148 | + changelog_message, |
| 149 | + cf_name, |
| 150 | + processed_offset, |
| 151 | + committed_offset, |
| 152 | + ) |
| 153 | + |
| 154 | + def _should_apply_changelog( |
| 155 | + self, processed_offset: Optional[int], committed_offset: int |
| 156 | + ) -> bool: |
| 157 | + """ |
| 158 | + Determine whether the changelog update should be skipped. |
| 159 | +
|
| 160 | + :param processed_offset: changelog message processed offset. |
| 161 | + :param committed_offset: latest committed offset of the source topic partition |
| 162 | + :return: True if update should be applied, else False. |
| 163 | + """ |
| 164 | + if processed_offset is not None: |
| 165 | + # Skip recovering from the message if its processed offset is ahead of the |
| 166 | + # current committed offset. |
| 167 | + # This way it will recover to a consistent state if the checkpointing code |
| 168 | + # produced the changelog messages but failed to commit |
| 169 | + # the source topic offset. |
| 170 | + return processed_offset < committed_offset |
| 171 | + return True |
| 172 | + |
| 173 | + def __enter__(self): |
| 174 | + return self |
| 175 | + |
| 176 | + def __exit__(self, exc_type, exc_val, exc_tb): |
| 177 | + self.close() |
0 commit comments