@@ -595,8 +595,10 @@ static PyObject *Consumer_committed (Handle *self, PyObject *args,
595
595
if (!(c_parts = py_to_c_parts (plist )))
596
596
return NULL ;
597
597
598
- err = rd_kafka_committed (self -> rk , c_parts ,
599
- tmout >= 0 ? (int )(tmout * 1000.0f ) : -1 );
598
+ Py_BEGIN_ALLOW_THREADS ;
599
+ err = rd_kafka_committed (self -> rk , c_parts ,
600
+ tmout >= 0 ? (int )(tmout * 1000.0f ) : -1 );
601
+ Py_END_ALLOW_THREADS ;
600
602
601
603
if (err ) {
602
604
rd_kafka_topic_partition_list_destroy (c_parts );
@@ -737,7 +739,9 @@ static PyObject *Consumer_seek (Handle *self, PyObject *args, PyObject *kwargs)
737
739
return NULL ;
738
740
}
739
741
742
+ Py_BEGIN_ALLOW_THREADS ;
740
743
err = rd_kafka_seek (rkt , tp -> partition , tp -> offset , -1 );
744
+ Py_END_ALLOW_THREADS ;
741
745
742
746
rd_kafka_topic_destroy (rkt );
743
747
@@ -785,10 +789,12 @@ static PyObject *Consumer_get_watermark_offsets (Handle *self, PyObject *args,
785
789
tp -> topic , tp -> partition ,
786
790
& low , & high );
787
791
} else {
792
+ Py_BEGIN_ALLOW_THREADS ;
788
793
err = rd_kafka_query_watermark_offsets (self -> rk ,
789
794
tp -> topic , tp -> partition ,
790
795
& low , & high ,
791
796
tmout >= 0 ? (int )(tmout * 1000.0f ) : -1 );
797
+ Py_END_ALLOW_THREADS ;
792
798
}
793
799
794
800
if (err ) {
@@ -837,9 +843,11 @@ static PyObject *Consumer_offsets_for_times (Handle *self, PyObject *args,
837
843
if (!(c_parts = py_to_c_parts (plist )))
838
844
return NULL ;
839
845
846
+ Py_BEGIN_ALLOW_THREADS ;
840
847
err = rd_kafka_offsets_for_times (self -> rk ,
841
848
c_parts ,
842
849
tmout >= 0 ? (int )(tmout * 1000.0f ) : -1 );
850
+ Py_END_ALLOW_THREADS ;
843
851
844
852
if (err ) {
845
853
rd_kafka_topic_partition_list_destroy (c_parts );
0 commit comments