@@ -453,7 +453,8 @@ def _close_consumer(self):
453453class FromKafkaBatched (Stream ):
454454 """Base class for both local and cluster-based batched kafka processing"""
455455 def __init__ (self , topic , consumer_params , poll_interval = '1s' ,
456- npartitions = None , max_batch_size = 10000 , keys = False ,
456+ npartitions = None , refresh_partitions = False ,
457+ max_batch_size = 10000 , keys = False ,
457458 engine = None , ** kwargs ):
458459 self .consumer_params = consumer_params
459460 # Override the auto-commit config to enforce custom streamz checkpointing
@@ -462,6 +463,7 @@ def __init__(self, topic, consumer_params, poll_interval='1s',
462463 consumer_params ['auto.offset.reset' ] = 'latest'
463464 self .topic = topic
464465 self .npartitions = npartitions
466+ self .refresh_partitions = refresh_partitions
465467 if self .npartitions is not None and self .npartitions <= 0 :
466468 raise ValueError ("Number of Kafka topic partitions must be > 0." )
467469 self .poll_interval = convert_interval (poll_interval )
@@ -511,6 +513,17 @@ def checkpoint_emit(_part):
511513 try :
512514 while not self .stopped :
513515 out = []
516+
517+ if self .refresh_partitions :
518+ kafka_cluster_metadata = self .consumer .list_topics (self .topic )
519+ if self .engine == "cudf" : # pragma: no cover
520+ new_partitions = len (kafka_cluster_metadata [self .topic .encode ('utf-8' )])
521+ else :
522+ new_partitions = len (kafka_cluster_metadata .topics [self .topic ].partitions )
523+ if new_partitions > self .npartitions :
524+ self .positions .extend ([- 1001 ] * (new_partitions - self .npartitions ))
525+ self .npartitions = new_partitions
526+
514527 for partition in range (self .npartitions ):
515528 tp = ck .TopicPartition (self .topic , partition , 0 )
516529 try :
@@ -562,7 +575,8 @@ def start(self):
562575
563576@Stream .register_api (staticmethod )
564577def from_kafka_batched (topic , consumer_params , poll_interval = '1s' ,
565- npartitions = None , start = False , dask = False ,
578+ npartitions = None , refresh_partitions = False ,
579+ start = False , dask = False ,
566580 max_batch_size = 10000 , keys = False ,
567581 engine = None , ** kwargs ):
568582 """ Get messages and keys (optional) from Kafka in batches
@@ -598,8 +612,13 @@ def from_kafka_batched(topic, consumer_params, poll_interval='1s',
598612 npartitions: int (None)
599613 | Number of partitions in the topic.
600614 | If None, streamz will poll Kafka to get the number of partitions.
601- | As of now, streamz does not support changing number of partitions on the fly.
602- | It is recommended to restart the stream after changing the number of partitions.
615+ refresh_partitions: bool (False)
616+ | Useful if the user expects to increase the number of topic partitions on the
617+ | fly, maybe to handle spikes in load. Streamz polls Kafka in every batch to
618+ | determine the current number of partitions. If partitions have been added,
619+ | streamz will automatically start reading data from the new partitions as well.
620+ | If set to False, streamz will not accommodate adding partitions on the fly.
621+ | It is recommended to restart the stream after decreasing the number of partitions.
603622 start: bool (False)
604623 Whether to start polling upon instantiation
605624 max_batch_size: int
@@ -631,7 +650,6 @@ def from_kafka_batched(topic, consumer_params, poll_interval='1s',
631650 | More information at: https://rapids.ai/start.html
632651
633652 Important Kafka Configurations
634- ----------
635653 By default, a stream will start reading from the latest offsets
636654 available. Please set 'auto.offset.reset': 'earliest' in the
637655 consumer configs, if the stream needs to start processing from
@@ -651,6 +669,7 @@ def from_kafka_batched(topic, consumer_params, poll_interval='1s',
651669 source = FromKafkaBatched (topic , consumer_params ,
652670 poll_interval = poll_interval ,
653671 npartitions = npartitions ,
672+ refresh_partitions = refresh_partitions ,
654673 max_batch_size = max_batch_size ,
655674 keys = keys ,
656675 engine = engine ,
0 commit comments