33import os
44import time
55from tornado import gen
6+ import weakref
67
78from .core import Stream , convert_interval , RefCounter
89
@@ -446,6 +447,7 @@ def start(self):
446447 self .stopped = False
447448 self .consumer = ck .Consumer (self .cpars )
448449 self .consumer .subscribe (self .topics )
450+ weakref .finalize (self , self .consumer .close )
449451 tp = ck .TopicPartition (self .topics [0 ], 0 , 0 )
450452
451453 # blocks for consumer thread to come up
@@ -496,7 +498,7 @@ def commit(_part):
496498
497499 @gen .coroutine
498500 def checkpoint_emit (_part ):
499- ref = RefCounter (cb = lambda : commit (_part ))
501+ ref = RefCounter (cb = lambda : commit (_part ), loop = self . loop )
500502 yield self ._emit (_part , metadata = [{'ref' : ref }])
501503
502504 if self .npartitions is None :
@@ -521,61 +523,58 @@ def checkpoint_emit(_part):
521523 self .positions [tp .partition ] = tp .offset
522524 break
523525
524- try :
525- while not self .stopped :
526- out = []
527-
528- if self .refresh_partitions :
529- kafka_cluster_metadata = self .consumer .list_topics (self .topic )
530- if self .engine == "cudf" : # pragma: no cover
531- new_partitions = len (kafka_cluster_metadata [self .topic .encode ('utf-8' )])
532- else :
533- new_partitions = len (kafka_cluster_metadata .topics [self .topic ].partitions )
534- if new_partitions > self .npartitions :
535- self .positions .extend ([- 1001 ] * (new_partitions - self .npartitions ))
536- self .npartitions = new_partitions
526+ while not self .stopped :
527+ out = []
537528
538- for partition in range (self .npartitions ):
539- tp = ck .TopicPartition (self .topic , partition , 0 )
540- try :
541- low , high = self .consumer .get_watermark_offsets (
542- tp , timeout = 0.1 )
543- except (RuntimeError , ck .KafkaException ):
544- continue
545- self .started = True
546- if 'auto.offset.reset' in self .consumer_params .keys ():
547- if self .consumer_params ['auto.offset.reset' ] == 'latest' and \
548- self .positions [partition ] == - 1001 :
549- self .positions [partition ] = high
550- current_position = self .positions [partition ]
551- lowest = max (current_position , low )
552- if high > lowest + self .max_batch_size :
553- high = lowest + self .max_batch_size
554- if high > lowest :
555- out .append ((self .consumer_params , self .topic , partition ,
556- self .keys , lowest , high - 1 ))
529+ if self .refresh_partitions :
530+ kafka_cluster_metadata = self .consumer .list_topics (self .topic )
531+ if self .engine == "cudf" : # pragma: no cover
532+ new_partitions = len (kafka_cluster_metadata [self .topic .encode ('utf-8' )])
533+ else :
534+ new_partitions = len (kafka_cluster_metadata .topics [self .topic ].partitions )
535+ if new_partitions > self .npartitions :
536+ self .positions .extend ([- 1001 ] * (new_partitions - self .npartitions ))
537+ self .npartitions = new_partitions
538+
539+ for partition in range (self .npartitions ):
540+ tp = ck .TopicPartition (self .topic , partition , 0 )
541+ try :
542+ low , high = self .consumer .get_watermark_offsets (
543+ tp , timeout = 0.1 )
544+ except (RuntimeError , ck .KafkaException ):
545+ continue
546+ self .started = True
547+ if 'auto.offset.reset' in self .consumer_params .keys ():
548+ if self .consumer_params ['auto.offset.reset' ] == 'latest' and \
549+ self .positions [partition ] == - 1001 :
557550 self .positions [partition ] = high
558- self .consumer_params ['auto.offset.reset' ] = 'earliest'
551+ current_position = self .positions [partition ]
552+ lowest = max (current_position , low )
553+ if high > lowest + self .max_batch_size :
554+ high = lowest + self .max_batch_size
555+ if high > lowest :
556+ out .append ((self .consumer_params , self .topic , partition ,
557+ self .keys , lowest , high - 1 ))
558+ self .positions [partition ] = high
559+ self .consumer_params ['auto.offset.reset' ] = 'earliest'
560+
561+ for part in out :
562+ yield self .loop .add_callback (checkpoint_emit , part )
559563
560- for part in out :
561- yield self .loop .add_callback (checkpoint_emit , part )
562-
563- else :
564- yield gen .sleep (self .poll_interval )
565- finally :
566- self .consumer .unsubscribe ()
567- self .consumer .close ()
564+ else :
565+ yield gen .sleep (self .poll_interval )
568566
569567 def start (self ):
570568 import confluent_kafka as ck
571569 if self .engine == "cudf" : # pragma: no cover
572570 from custreamz import kafka
573571
574572 if self .stopped :
575- if self .engine == "cudf" : # pragma: no cover
573+ if self .engine == "cudf" : # pragma: no cover
576574 self .consumer = kafka .Consumer (self .consumer_params )
577575 else :
578576 self .consumer = ck .Consumer (self .consumer_params )
577+ weakref .finalize (self , self .consumer .close )
579578 self .stopped = False
580579 tp = ck .TopicPartition (self .topic , 0 , 0 )
581580
0 commit comments