@@ -595,8 +595,10 @@ static PyObject *Consumer_committed (Handle *self, PyObject *args,
595595 if (!(c_parts = py_to_c_parts (plist )))
596596 return NULL ;
597597
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 ;
600602
601603 if (err ) {
602604 rd_kafka_topic_partition_list_destroy (c_parts );
@@ -737,7 +739,9 @@ static PyObject *Consumer_seek (Handle *self, PyObject *args, PyObject *kwargs)
737739 return NULL ;
738740 }
739741
742+ Py_BEGIN_ALLOW_THREADS ;
740743 err = rd_kafka_seek (rkt , tp -> partition , tp -> offset , -1 );
744+ Py_END_ALLOW_THREADS ;
741745
742746 rd_kafka_topic_destroy (rkt );
743747
@@ -785,10 +789,12 @@ static PyObject *Consumer_get_watermark_offsets (Handle *self, PyObject *args,
785789 tp -> topic , tp -> partition ,
786790 & low , & high );
787791 } else {
792+ Py_BEGIN_ALLOW_THREADS ;
788793 err = rd_kafka_query_watermark_offsets (self -> rk ,
789794 tp -> topic , tp -> partition ,
790795 & low , & high ,
791796 tmout >= 0 ? (int )(tmout * 1000.0f ) : -1 );
797+ Py_END_ALLOW_THREADS ;
792798 }
793799
794800 if (err ) {
@@ -837,9 +843,11 @@ static PyObject *Consumer_offsets_for_times (Handle *self, PyObject *args,
837843 if (!(c_parts = py_to_c_parts (plist )))
838844 return NULL ;
839845
846+ Py_BEGIN_ALLOW_THREADS ;
840847 err = rd_kafka_offsets_for_times (self -> rk ,
841848 c_parts ,
842849 tmout >= 0 ? (int )(tmout * 1000.0f ) : -1 );
850+ Py_END_ALLOW_THREADS ;
843851
844852 if (err ) {
845853 rd_kafka_topic_partition_list_destroy (c_parts );
0 commit comments