@@ -649,6 +649,55 @@ static PyObject *Consumer_position (Handle *self, PyObject *args,
649
649
return plist ;
650
650
}
651
651
652
+ static PyObject * Consumer_pause (Handle * self , PyObject * args ,
653
+ PyObject * kwargs ) {
654
+
655
+ PyObject * plist ;
656
+ rd_kafka_topic_partition_list_t * c_parts ;
657
+ rd_kafka_resp_err_t err ;
658
+ static char * kws [] = {"partitions" , NULL };
659
+
660
+ if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O" , kws , & plist ))
661
+ return NULL ;
662
+
663
+ if (!(c_parts = py_to_c_parts (plist )))
664
+ return NULL ;
665
+
666
+ err = rd_kafka_pause_partitions (self -> rk , c_parts );
667
+ rd_kafka_topic_partition_list_destroy (c_parts );
668
+ if (err ) {
669
+ cfl_PyErr_Format (err ,
670
+ "Failed to pause partitions: %s" ,
671
+ rd_kafka_err2str (err ));
672
+ return NULL ;
673
+ }
674
+ Py_RETURN_NONE ;
675
+ }
676
+
677
+ static PyObject * Consumer_resume (Handle * self , PyObject * args ,
678
+ PyObject * kwargs ) {
679
+
680
+ PyObject * plist ;
681
+ rd_kafka_topic_partition_list_t * c_parts ;
682
+ rd_kafka_resp_err_t err ;
683
+ static char * kws [] = {"partitions" , NULL };
684
+
685
+ if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O" , kws , & plist ))
686
+ return NULL ;
687
+
688
+ if (!(c_parts = py_to_c_parts (plist )))
689
+ return NULL ;
690
+
691
+ err = rd_kafka_resume_partitions (self -> rk , c_parts );
692
+ rd_kafka_topic_partition_list_destroy (c_parts );
693
+ if (err ) {
694
+ cfl_PyErr_Format (err ,
695
+ "Failed to resume partitions: %s" ,
696
+ rd_kafka_err2str (err ));
697
+ return NULL ;
698
+ }
699
+ Py_RETURN_NONE ;
700
+ }
652
701
653
702
static PyObject * Consumer_get_watermark_offsets (Handle * self , PyObject * args ,
654
703
PyObject * kwargs ) {
@@ -1063,6 +1112,30 @@ static PyMethodDef Consumer_methods[] = {
1063
1112
" :raises: RuntimeError if called on a closed consumer\n"
1064
1113
"\n"
1065
1114
},
1115
+ { "pause" , (PyCFunction )Consumer_pause ,
1116
+ METH_VARARGS |METH_KEYWORDS ,
1117
+ ".. py:function:: pause(partitions)\n"
1118
+ "\n"
1119
+ " Pause consumption for the provided list of partitions.\n"
1120
+ "\n"
1121
+ " :param list(TopicPartition) partitions: List of topic+partitions "
1122
+ "to pause.\n"
1123
+ " :rtype: None\n"
1124
+ " :raises: KafkaException\n"
1125
+ "\n"
1126
+ },
1127
+ { "resume" , (PyCFunction )Consumer_resume ,
1128
+ METH_VARARGS |METH_KEYWORDS ,
1129
+ ".. py:function:: resume(partitions)\n"
1130
+ "\n"
1131
+ " Resume consumption for the provided list of partitions.\n"
1132
+ "\n"
1133
+ " :param list(TopicPartition) partitions: List of topic+partitions "
1134
+ "to resume.\n"
1135
+ " :rtype: None\n"
1136
+ " :raises: KafkaException\n"
1137
+ "\n"
1138
+ },
1066
1139
{ "get_watermark_offsets" , (PyCFunction )Consumer_get_watermark_offsets ,
1067
1140
METH_VARARGS |METH_KEYWORDS ,
1068
1141
".. py:function:: get_watermark_offsets(partition, [timeout=None], [cached=False])\n"
0 commit comments