2
2
import functools
3
3
import logging
4
4
import os
5
+ import time
5
6
import signal
6
7
import warnings
7
8
from typing import Optional , List , Callable , Union , Literal , Tuple , Type
44
45
from .state import StateStoreManager
45
46
from .state .recovery import RecoveryManager
46
47
from .state .rocksdb import RocksDBOptionsType
48
+ from .sources .manager import SourceManager , BaseSource , SourceException
47
49
from .utils .settings import BaseSettings
48
50
49
51
__all__ = ("Application" , "ApplicationConfig" )
@@ -114,7 +116,7 @@ def __init__(
114
116
on_message_processed : Optional [MessageProcessedCallback ] = None ,
115
117
consumer_poll_timeout : float = 1.0 ,
116
118
producer_poll_timeout : float = 0.0 ,
117
- loglevel : Optional [LogLevel ] = "INFO" ,
119
+ loglevel : Optional [Union [ int , LogLevel ] ] = "INFO" ,
118
120
auto_create_topics : bool = True ,
119
121
use_changelog_topics : bool = True ,
120
122
quix_config_builder : Optional [QuixKafkaConfigsBuilder ] = None ,
@@ -289,13 +291,7 @@ def __init__(
289
291
extra_config = self ._config .consumer_extra_config ,
290
292
on_error = on_consumer_error ,
291
293
)
292
- self ._producer = RowProducer (
293
- broker_address = self ._config .broker_address ,
294
- extra_config = self ._config .producer_extra_config ,
295
- on_error = on_producer_error ,
296
- flush_timeout = self ._config .flush_timeout ,
297
- transactional = self ._config .exactly_once ,
298
- )
294
+ self ._producer = self ._get_rowproducer (on_error = on_producer_error )
299
295
self ._running = False
300
296
self ._failed = False
301
297
@@ -328,6 +324,7 @@ def __init__(
328
324
recovery_manager = recovery_manager ,
329
325
)
330
326
327
+ self ._source_manager = SourceManager ()
331
328
self ._sink_manager = SinkManager ()
332
329
self ._pausing_manager = PausingManager (consumer = self ._consumer )
333
330
self ._processing_context = ProcessingContext (
@@ -364,7 +361,7 @@ def Quix(
364
361
on_message_processed : Optional [MessageProcessedCallback ] = None ,
365
362
consumer_poll_timeout : float = 1.0 ,
366
363
producer_poll_timeout : float = 0.0 ,
367
- loglevel : Optional [LogLevel ] = "INFO" ,
364
+ loglevel : Optional [Union [ int , LogLevel ] ] = "INFO" ,
368
365
quix_config_builder : Optional [QuixKafkaConfigsBuilder ] = None ,
369
366
auto_create_topics : bool = True ,
370
367
use_changelog_topics : bool = True ,
@@ -573,7 +570,8 @@ def custom_ts_extractor(
573
570
574
571
def dataframe (
575
572
self ,
576
- topic : Topic ,
573
+ topic : Optional [Topic ] = None ,
574
+ source : Optional [BaseSource ] = None ,
577
575
) -> StreamingDataFrame :
578
576
"""
579
577
A simple helper method that generates a `StreamingDataFrame`, which is used
@@ -603,6 +601,12 @@ def dataframe(
603
601
to be used as an input topic.
604
602
:return: `StreamingDataFrame` object
605
603
"""
604
+ if not source and not topic :
605
+ raise ValueError ("one of `source` or `topic` is required" )
606
+
607
+ if source :
608
+ topic = self .add_source (source , topic )
609
+
606
610
sdf = StreamingDataFrame (
607
611
topic = topic ,
608
612
topic_manager = self ._topic_manager ,
@@ -633,6 +637,28 @@ def stop(self, fail: bool = False):
633
637
if self ._state_manager .using_changelogs :
634
638
self ._state_manager .stop_recovery ()
635
639
640
+ def _get_rowproducer (
641
+ self ,
642
+ on_error : Optional [ProducerErrorCallback ] = None ,
643
+ transactional : Optional [bool ] = None ,
644
+ ) -> RowProducer :
645
+ """
646
+ Create a RowProducer using the application config
647
+
648
+ Used to create the application producer as well as the sources producers
649
+ """
650
+
651
+ if transactional is None :
652
+ transactional = self ._config .exactly_once
653
+
654
+ return RowProducer (
655
+ broker_address = self ._config .broker_address ,
656
+ extra_config = self ._config .producer_extra_config ,
657
+ flush_timeout = self ._config .flush_timeout ,
658
+ on_error = on_error ,
659
+ transactional = transactional ,
660
+ )
661
+
636
662
def get_producer (self ) -> Producer :
637
663
"""
638
664
Create and return a pre-configured Producer instance.
@@ -716,6 +742,25 @@ def clear_state(self):
716
742
"""
717
743
self ._state_manager .clear_stores ()
718
744
745
+ def add_source (self , source : BaseSource , topic : Optional [Topic ] = None ) -> Topic :
746
+ """
747
+ Add a source to the application.
748
+
749
+ See :class:`quixstreams.sources.base.BaseSource` for more details.
750
+
751
+ :param source: a :class:`quixstreams.sources.BaseSource` instance
752
+ :param topic: the :class:`quixstreams.models.Topic` instance the source will produce to
753
+ Default: the source default
754
+ """
755
+ if not topic :
756
+ topic = source .default_topic ()
757
+ self ._topic_manager .register (topic )
758
+
759
+ producer = self ._get_rowproducer (transactional = False )
760
+ source .configure (topic , producer )
761
+ self ._source_manager .register (source )
762
+ return topic
763
+
719
764
def run (
720
765
self ,
721
766
dataframe : StreamingDataFrame ,
@@ -745,6 +790,19 @@ def run(
745
790
746
791
:param dataframe: instance of `StreamingDataFrame`
747
792
"""
793
+ self ._run (dataframe )
794
+
795
+ def _exception_handler (self , exc_type , exc_val , exc_tb ):
796
+ fail = False
797
+
798
+ # Sources and the application are independent.
799
+ # If a source fails, the application can shutdown gracefully.
800
+ if exc_val is not None and exc_type is not SourceException :
801
+ fail = True
802
+
803
+ self .stop (fail = fail )
804
+
805
+ def _run (self , dataframe : Optional [StreamingDataFrame ] = None ):
748
806
self ._setup_signal_handlers ()
749
807
750
808
logger .info (
@@ -765,36 +823,52 @@ def run(
765
823
exit_stack .enter_context (self ._processing_context )
766
824
exit_stack .enter_context (self ._state_manager )
767
825
exit_stack .enter_context (self ._consumer )
768
- exit_stack .push (
769
- lambda exc_type , exc_val , exc_tb : self .stop (fail = exc_val is not None )
770
- )
826
+ exit_stack .enter_context (self ._source_manager )
827
+ exit_stack .push (self ._exception_handler )
771
828
772
829
with exit_stack :
773
830
# Subscribe to topics in Kafka and start polling
774
- self ._consumer .subscribe (
775
- dataframe .topics_to_subscribe ,
776
- on_assign = self ._on_assign ,
777
- on_revoke = self ._on_revoke ,
778
- on_lost = self ._on_lost ,
779
- )
780
- logger .info ("Waiting for incoming messages" )
781
- # Start polling Kafka for messages and callbacks
782
- self ._running = True
831
+ if dataframe is not None :
832
+ self ._run_dataframe (dataframe )
833
+ else :
834
+ self ._run_sources ()
835
+
836
+ def _run_dataframe (self , dataframe ):
837
+ self ._consumer .subscribe (
838
+ dataframe .topics_to_subscribe ,
839
+ on_assign = self ._on_assign ,
840
+ on_revoke = self ._on_revoke ,
841
+ on_lost = self ._on_lost ,
842
+ )
843
+ logger .info ("Waiting for incoming messages" )
844
+ # Start polling Kafka for messages and callbacks
845
+ self ._running = True
846
+
847
+ # Initialize the checkpoint
848
+ self ._processing_context .init_checkpoint ()
849
+
850
+ dataframe_composed = dataframe .compose ()
851
+
852
+ while self ._running :
853
+ if self ._state_manager .recovery_required :
854
+ self ._state_manager .do_recovery ()
855
+ else :
856
+ self ._process_message (dataframe_composed )
857
+ self ._processing_context .commit_checkpoint ()
858
+ self ._processing_context .resume_ready_partitions ()
859
+ self ._source_manager .raise_for_error ()
783
860
784
- # Initialize the checkpoint
785
- self ._processing_context .init_checkpoint ()
861
+ logger .info ("Stop processing of StreamingDataFrame" )
786
862
787
- dataframe_composed = dataframe .compose ()
863
+ def _run_sources (self ):
864
+ self ._running = True
865
+ while self ._running :
866
+ self ._source_manager .raise_for_error ()
788
867
789
- while self ._running :
790
- if self ._state_manager .recovery_required :
791
- self ._state_manager .do_recovery ()
792
- else :
793
- self ._process_message (dataframe_composed )
794
- self ._processing_context .commit_checkpoint ()
795
- self ._processing_context .resume_ready_partitions ()
868
+ if not self ._source_manager .is_alive ():
869
+ self .stop ()
796
870
797
- logger . info ( "Stop processing of StreamingDataFrame" )
871
+ time . sleep ( 1 )
798
872
799
873
def _quix_runtime_init (self ):
800
874
"""
0 commit comments