Skip to content

Commit d09e7fa

Browse files
authored
Kafka Replicator Source & Quix Environment Source (#448)
1 parent 102494e commit d09e7fa

File tree

20 files changed

+1104
-46
lines changed

20 files changed

+1104
-46
lines changed

docs/build/build.py

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -123,6 +123,9 @@
123123
k: None
124124
for k in [
125125
"quixstreams.sources.base",
126+
"quixstreams.sources.csv",
127+
"quixstreams.sources.kafka.kafka",
128+
"quixstreams.sources.kafka.quix",
126129
]
127130
},
128131
}

docs/connectors/sources/README.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,8 @@ if __name__ == "__main__":
2626
Quix streams provide a source out of the box.
2727

2828
* [CSVSource](./csv-source.md): A source that reads data from a single CSV file.
29+
* [KafkaReplicatorSource](./kafka-source.md): A source that replicates a topic from a Kafka broker to your application broker.
30+
* [QuixEnvironmentSource](./quix-source.md): A source that replicates a topic from a Quix Cloud environment to your application broker.
2931

3032
You can also implement your own, have a look at [Creating a Custom Source](custom-sources.md) for documentation on how to do that.
3133

Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,42 @@
1+
# Kafka Replicator Source
2+
3+
A source that reads data from a Kafka topic and produce it to another Kafka topic. The two topics can be located on different Kafka clusters.
4+
5+
This source supports exactly-once guarantees.
6+
7+
## How to use the Kafka Replicator Source
8+
9+
To use a Kafka Replicator source, you need to create an instance of `KafkaReplicatorSource` and pass it to the `app.dataframe()` method.
10+
11+
```python
12+
from quixstreams import Application
13+
from quixstreams.sources import KafkaReplicatorSource
14+
15+
def main():
16+
app = Application()
17+
source = KafkaReplicatorSource(
18+
name="my-source",
19+
app_config=app.config,
20+
topic="source-topic",
21+
broker_address="source-broker-address"
22+
)
23+
24+
sdf = app.dataframe(source=source)
25+
sdf.print(metadata=True)
26+
27+
app.run(sdf)
28+
29+
if __name__ == "__main__":
30+
main()
31+
```
32+
33+
## Topic
34+
35+
The Kafka Replicator source only deals with bytes. It reads the remote keys and values as bytes and produces them directly as bytes.
36+
You can configure the key and value deserializer used by the Streaming Dataframe with the `key_deserializer` and `value_deserializer` paramaters.
37+
38+
## Consumer group
39+
40+
The Kafka Replicator consumer group is the source name prefixed by `source-`. Changing the name will reset the source state and it will re-replicate the data based on the configured `auto_offset_reset`. It is not based on the application consumer group, changing the application consumer group will not reset the source.
41+
42+
For more information about consumer group [see the glosary](https://quix.io/docs/kb/glossary.html?h=consumer+group#consumer-group)
Lines changed: 34 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,34 @@
1+
# Quix Environment Source
2+
3+
A specialised [Kafka Source](kafka-source.md) that simplify copying data from a Quix environment.
4+
5+
## How to use the Quix Environment Source
6+
7+
To use a Quix Environment source, you need to create an instance of `QuixEnvironmentSource` and pass it to the `app.dataframe()` method.
8+
9+
```python
10+
from quixstreams import Application
11+
from quixstreams.sources import QuixEnvironmentSource
12+
13+
def main():
14+
app = Application()
15+
source = QuixEnvironmentSource(
16+
name="my-source",
17+
app_config=app.config,
18+
topic="source-topic",
19+
quix_sdk_token="quix-sdk-token",
20+
quix_workspace_id="quix-workspace-id",
21+
)
22+
23+
sdf = app.dataframe(source=source)
24+
sdf.print(metadata=True)
25+
26+
app.run(sdf)
27+
28+
if __name__ == "__main__":
29+
main()
30+
```
31+
32+
## Token
33+
34+
The Quix Environment Source requires the sdk token of the source environment. [Click here](../../../develop/authentication/streaming-token.md) for more information on SDK tokens.

mkdocs.yml

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -52,6 +52,8 @@ nav:
5252
- Sources:
5353
- 'connectors/sources/README.md'
5454
- CSV Source: connectors/sources/csv-source.md
55+
- Kafka Replicator Source: connectors/sources/kafka-source.md
56+
- Quix Source: connectors/sources/quix-source.md
5557
- Creating a Custom Source: connectors/sources/custom-sources.md
5658
- Upgrading Guide:
5759
- Upgrading from Quix Streams v0.5: upgrading-legacy.md

quixstreams/app.py

Lines changed: 11 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -682,7 +682,7 @@ def get_producer(self) -> Producer:
682682
producer.produce(topic=topic.name, key=b"key", value=b"value")
683683
```
684684
"""
685-
self._setup_topics()
685+
self.setup_topics()
686686

687687
return Producer(
688688
broker_address=self._config.broker_address,
@@ -725,8 +725,11 @@ def get_consumer(self, auto_commit_enable: bool = True) -> Consumer:
725725
# consumer.store_offsets(msg)
726726
727727
```
728+
729+
:param auto_commit_enable: Enable or disable auto commit
730+
Default - True
728731
"""
729-
self._setup_topics()
732+
self.setup_topics()
730733

731734
return Consumer(
732735
broker_address=self._config.broker_address,
@@ -817,7 +820,7 @@ def _run(self, dataframe: Optional[StreamingDataFrame] = None):
817820
if self.is_quix_app:
818821
self._quix_runtime_init()
819822

820-
self._setup_topics()
823+
self.setup_topics()
821824

822825
exit_stack = contextlib.ExitStack()
823826
exit_stack.enter_context(self._processing_context)
@@ -881,7 +884,11 @@ def _quix_runtime_init(self):
881884
if self._state_manager.stores:
882885
check_state_management_enabled()
883886

884-
def _setup_topics(self):
887+
def setup_topics(self):
888+
"""
889+
Validate and create the topics
890+
"""
891+
885892
topics_list = ", ".join(
886893
f'"{topic}"' for topic in self._topic_manager.all_topics
887894
)

quixstreams/checkpointing/__init__.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,2 +1,2 @@
1-
from .checkpoint import Checkpoint
1+
from .checkpoint import Checkpoint, BaseCheckpoint
22
from .exceptions import InvalidStoredOffset

quixstreams/checkpointing/checkpoint.py

Lines changed: 53 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
import logging
22
import time
3+
from abc import abstractmethod
34
from typing import Dict, Tuple
45

56
from confluent_kafka import TopicPartition, KafkaException
@@ -24,21 +25,19 @@
2425
logger = logging.getLogger(__name__)
2526

2627

27-
class Checkpoint:
28+
class BaseCheckpoint:
2829
"""
29-
Class to keep track of state updates and consumer offsets and to checkpoint these
30+
Base class to keep track of state updates and consumer offsets and to checkpoint these
3031
updates on schedule.
32+
33+
Two implementations exist:
34+
* one for checkpointing the Application in quixstreams/checkpoint/checkpoint.py
35+
* one for checkpointing the kafka source in quixstreams/sources/kafka/checkpoint.py
3136
"""
3237

3338
def __init__(
3439
self,
3540
commit_interval: float,
36-
producer: RowProducer,
37-
consumer: Consumer,
38-
state_manager: StateStoreManager,
39-
sink_manager: SinkManager,
40-
pausing_manager: PausingManager,
41-
exactly_once: bool = False,
4241
commit_every: int = 0,
4342
):
4443
self._created_at = time.monotonic()
@@ -52,18 +51,9 @@ def __init__(
5251
# Passing zero or lower will flush the checkpoint after each processed message
5352
self._commit_interval = max(commit_interval, 0)
5453

55-
self._state_manager = state_manager
56-
self._consumer = consumer
57-
self._producer = producer
58-
self._exactly_once = exactly_once
59-
self._sink_manager = sink_manager
60-
self._pausing_manager = pausing_manager
6154
self._commit_every = commit_every
6255
self._total_offsets_processed = 0
6356

64-
if self._exactly_once:
65-
self._producer.begin_transaction()
66-
6757
def expired(self) -> bool:
6858
"""
6959
Returns `True` if checkpoint deadline has expired OR
@@ -107,6 +97,52 @@ def store_offset(self, topic: str, partition: int, offset: int):
10797
self._starting_tp_offsets[tp] = offset
10898
self._total_offsets_processed += 1
10999

100+
@abstractmethod
101+
def close(self):
102+
"""
103+
Perform cleanup (when the checkpoint is empty) instead of committing.
104+
105+
Needed for exactly-once, as Kafka transactions are timeboxed.
106+
"""
107+
108+
@abstractmethod
109+
def commit(self):
110+
"""
111+
Commit the checkpoint.
112+
"""
113+
pass
114+
115+
116+
class Checkpoint(BaseCheckpoint):
117+
"""
118+
Checkpoint implementation used by the application
119+
"""
120+
121+
def __init__(
122+
self,
123+
commit_interval: float,
124+
producer: RowProducer,
125+
consumer: Consumer,
126+
state_manager: StateStoreManager,
127+
sink_manager: SinkManager,
128+
pausing_manager: PausingManager,
129+
exactly_once: bool = False,
130+
commit_every: int = 0,
131+
):
132+
super().__init__(
133+
commit_interval=commit_interval,
134+
commit_every=commit_every,
135+
)
136+
137+
self._state_manager = state_manager
138+
self._consumer = consumer
139+
self._producer = producer
140+
self._sink_manager = sink_manager
141+
self._pausing_manager = pausing_manager
142+
self._exactly_once = exactly_once
143+
if self._exactly_once:
144+
self._producer.begin_transaction()
145+
110146
def get_store_transaction(
111147
self, topic: str, partition: int, store_name: str = DEFAULT_STATE_STORE_NAME
112148
) -> PartitionTransaction:

quixstreams/models/topics/admin.py

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -209,3 +209,13 @@ def create_topics(
209209
),
210210
finalize_timeout=finalize_timeout,
211211
)
212+
213+
# support pickling by dropping the inner admin
214+
def __getstate__(self) -> object:
215+
state = self.__dict__.copy()
216+
state.pop("_inner_admin", None)
217+
return state
218+
219+
def __setstate__(self, state):
220+
self.__dict__.update(state)
221+
self._inner_admin = None

quixstreams/rowproducer.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -158,7 +158,7 @@ def produce_row(
158158
return
159159
raise
160160

161-
def poll(self, timeout: float = None):
161+
def poll(self, timeout: float = 0):
162162
"""
163163
Polls the producer for events and calls `on_delivery` callbacks.
164164

quixstreams/sources/__init__.py

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2,3 +2,4 @@
22
from .manager import SourceException
33
from .multiprocessing import multiprocessing
44
from .csv import CSVSource
5+
from .kafka import KafkaReplicatorSource, QuixEnvironmentSource

quixstreams/sources/kafka/__init__.py

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,2 @@
1+
from .kafka import *
2+
from .quix import *
Lines changed: 88 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,88 @@
1+
from confluent_kafka import TopicPartition, KafkaException
2+
3+
from typing import List
4+
5+
from quixstreams.checkpointing import BaseCheckpoint
6+
from quixstreams.checkpointing.exceptions import (
7+
CheckpointProducerTimeout,
8+
CheckpointConsumerCommitError,
9+
)
10+
from quixstreams.models.topics import Topic
11+
from quixstreams.rowconsumer import Consumer
12+
from quixstreams.rowproducer import RowProducer
13+
14+
15+
class Checkpoint(BaseCheckpoint):
16+
"""
17+
Checkpoint implementation used by the KafkaReplicatorSource
18+
"""
19+
20+
def __init__(
21+
self,
22+
producer: RowProducer,
23+
producer_topic: Topic,
24+
consumer: Consumer,
25+
commit_interval: float,
26+
commit_every: int = 0,
27+
flush_timeout: float = 10,
28+
exactly_once: bool = False,
29+
):
30+
super().__init__(commit_interval, commit_every)
31+
32+
self._producer = producer
33+
self._producer_topic = producer_topic
34+
self._consumer = consumer
35+
self._flush_timeout = flush_timeout
36+
self._exactly_once = exactly_once
37+
38+
if self._exactly_once:
39+
self._producer.begin_transaction()
40+
41+
def close(self):
42+
"""
43+
Perform cleanup (when the checkpoint is empty) instead of committing.
44+
45+
Needed for exactly-once, as Kafka transactions are timeboxed.
46+
"""
47+
if self._exactly_once:
48+
self._producer.abort_transaction()
49+
50+
def commit(self):
51+
"""
52+
Commit the checkpoint.
53+
54+
This method will:
55+
1. Flush the producer to ensure everything is delivered.
56+
2. Commit topic offsets.
57+
"""
58+
unproduced_msg_count = self._producer.flush(self._flush_timeout)
59+
if unproduced_msg_count > 0:
60+
raise CheckpointProducerTimeout(
61+
f"'{unproduced_msg_count}' messages failed to be produced before the producer flush timeout"
62+
)
63+
64+
offsets = [
65+
TopicPartition(
66+
topic=self._producer_topic.name,
67+
partition=partition,
68+
offset=offset + 1,
69+
)
70+
for (_, partition), offset in self._tp_offsets.items()
71+
]
72+
self._tp_offsets = {}
73+
74+
try:
75+
self._commit(offsets=offsets)
76+
except KafkaException as e:
77+
raise CheckpointConsumerCommitError(e.args[0]) from None
78+
79+
def _commit(self, offsets: List[TopicPartition]):
80+
if self._exactly_once:
81+
self._producer.commit_transaction(
82+
offsets, self._consumer.consumer_group_metadata()
83+
)
84+
else:
85+
partitions = self._consumer.commit(offsets=offsets, asynchronous=False)
86+
for partition in partitions:
87+
if partition.error:
88+
raise CheckpointConsumerCommitError(partition.error)

0 commit comments

Comments
 (0)