@@ -3059,6 +3059,104 @@ const char Admin_delete_records_doc[] = PyDoc_STR(
30593059 "\n"
30603060 " This method should not be used directly, use confluent_kafka.AdminClient.delete_records()\n" );
30613061
3062+ /**
3063+ * @brief Elect leaders
3064+ */
3065+ PyObject * Admin_elect_leaders (Handle * self , PyObject * args , PyObject * kwargs ) {
3066+ PyObject * election_type = NULL , * partitions = NULL , * future ;
3067+ rd_kafka_ElectLeaders_t * c_elect_leaders = NULL ;
3068+ rd_kafka_ElectionType_t c_election_type ;
3069+ struct Admin_options options = Admin_options_INITIALIZER ;
3070+ rd_kafka_AdminOptions_t * c_options = NULL ;
3071+ rd_kafka_topic_partition_list_t * c_partitions = NULL ;
3072+ CallState cs ;
3073+ rd_kafka_queue_t * rkqu ;
3074+
3075+ static char * kws [] = {"election_type" ,
3076+ "partitions"
3077+ "future" ,
3078+ /* options */
3079+ "request_timeout" , "operation_timeout" , NULL };
3080+
3081+ if (!PyArg_ParseTupleAndKeywords (args , kwargs , "OOO|ff" , kws ,
3082+ & election_type , & partitions , & future ,
3083+ & options .request_timeout ,
3084+ & options .operation_timeout )) {
3085+ goto err ;
3086+ }
3087+
3088+ c_options = Admin_options_to_c (self , RD_KAFKA_ADMIN_OP_ELECTLEADERS ,
3089+ & options , future );
3090+ if (!c_options ) {
3091+ goto err ; /* Exception raised by options_to_c() */
3092+ }
3093+
3094+ /* options_to_c() sets future as the opaque, which is used in the
3095+ * background_event_cb to set the results on the future as the
3096+ * admin operation is finished, so we need to keep our own refcount. */
3097+ Py_INCREF (future );
3098+
3099+ c_election_type = (rd_kafka_ElectionType_t )cfl_PyInt_AsInt (election_type );
3100+
3101+ if (partitions != Py_None && !PyList_Check (partitions )) {
3102+ PyErr_SetString (PyExc_ValueError , "partitions must be None or a list" );
3103+ goto err ;
3104+ }
3105+
3106+ if (partitions != Py_None ) {
3107+ c_partitions = py_to_c_parts (partitions );
3108+ }
3109+
3110+ c_elect_leaders = rd_kafka_ElectLeaders_new (c_election_type , c_partitions );
3111+
3112+ if (c_partitions ) {
3113+ rd_kafka_topic_partition_list_destroy (c_partitions );
3114+ }
3115+
3116+ /* Use librdkafka's background thread queue to automatically dispatch
3117+ * Admin_background_event_cb() when the admin operation is finished. */
3118+ rkqu = rd_kafka_queue_get_background (self -> rk );
3119+
3120+ /**
3121+ *
3122+ * Call ElectLeaders
3123+ *
3124+ * We need to set up a CallState and release GIL here since
3125+ * the event_cb may be triggered immediately.
3126+ *
3127+ */
3128+ CallState_begin (self , & cs );
3129+ rd_kafka_ElectLeaders (self -> rk , c_elect_leaders , c_options , rkqu );
3130+ CallState_end (self , & cs );
3131+
3132+ rd_kafka_queue_destroy (rkqu ); /* drop reference from get_background */
3133+
3134+ rd_kafka_AdminOptions_destroy (c_options );
3135+ rd_kafka_ElectLeaders_destroy (c_elect_leaders );
3136+
3137+ Py_RETURN_NONE ;
3138+
3139+ err :
3140+ if (c_elect_leaders ) {
3141+ rd_kafka_ElectLeaders_destroy (c_elect_leaders );
3142+ }
3143+ if (c_options ) {
3144+ rd_kafka_AdminOptions_destroy (c_options );
3145+ Py_DECREF (future );
3146+ }
3147+ return NULL ;
3148+ }
3149+
3150+ const char Admin_elect_leaders_doc [] = PyDoc_STR (
3151+ ".. py:function:: elect_leaders(election_type, partitions, "
3152+ "future, [request_timeout, operation_timeout])\n"
3153+ "\n"
3154+ " Perform Preferred or Unclean election for the specified "
3155+ "Topic Partitions.\n"
3156+ "\n"
3157+ " This method should not be used directly, use "
3158+ "confluent_kafka.AdminClient.elect_leaders()\n" );
3159+
30623160/**
30633161 * @brief Call rd_kafka_poll() and keep track of crashing callbacks.
30643162 * @returns -1 if callback crashed (or poll() failed), else the number
@@ -3225,6 +3323,10 @@ static PyMethodDef Admin_methods[] = {
32253323 Admin_delete_records_doc
32263324 },
32273325
3326+ { "elect_leaders" , (PyCFunction )Admin_elect_leaders , METH_VARARGS | METH_KEYWORDS ,
3327+ Admin_elect_leaders_doc
3328+ },
3329+
32283330 { NULL }
32293331};
32303332
@@ -4875,6 +4977,23 @@ static void Admin_background_event_cb (rd_kafka_t *rk, rd_kafka_event_t *rkev,
48754977 break ;
48764978 }
48774979
4980+ case RD_KAFKA_EVENT_ELECTLEADERS_RESULT :
4981+ {
4982+ size_t c_result_cnt ;
4983+
4984+ const rd_kafka_ElectLeaders_result_t
4985+ * c_elect_leaders_res_event =
4986+ rd_kafka_event_ElectLeaders_result (rkev );
4987+
4988+ const rd_kafka_topic_partition_result_t * * partition_results =
4989+ rd_kafka_ElectLeaders_result_partitions (
4990+ c_elect_leaders_res_event , & c_result_cnt );
4991+
4992+ result = c_topic_partition_result_to_py_dict (partition_results , c_result_cnt );
4993+
4994+ break ;
4995+ }
4996+
48784997 default :
48794998 Py_DECREF (error ); /* Py_None */
48804999 error = KafkaError_new0 (RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE ,
0 commit comments