@@ -825,9 +825,9 @@ static void error_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque)
825825 result = PyObject_CallFunctionObjArgs (h -> error_cb , eo , NULL );
826826 Py_DECREF (eo );
827827
828- if (result ) {
828+ if (result )
829829 Py_DECREF (result );
830- } else {
830+ else {
831831 CallState_crash (cs );
832832 rd_kafka_yield (h -> rk );
833833 }
@@ -836,6 +836,32 @@ static void error_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque)
836836 CallState_resume (cs );
837837}
838838
839+ static int stats_cb (rd_kafka_t * rk , char * json , size_t json_len , void * opaque ) {
840+ Handle * h = opaque ;
841+ PyObject * eo = NULL , * result = NULL ;
842+ CallState * cs = NULL ;
843+
844+ cs = CallState_get (h );
845+ if (json_len == 0 ) {
846+ /* No data returned*/
847+ goto done ;
848+ }
849+
850+ eo = Py_BuildValue ("s" , json );
851+ result = PyObject_CallFunctionObjArgs (h -> stats_cb , eo , NULL );
852+ Py_DECREF (eo );
853+
854+ if (result )
855+ Py_DECREF (result );
856+ else {
857+ CallState_crash (cs );
858+ rd_kafka_yield (h -> rk );
859+ }
860+
861+ done :
862+ CallState_resume (cs );
863+ return 0 ;
864+ }
839865
840866/****************************************************************************
841867 *
@@ -853,9 +879,11 @@ static void error_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque)
853879 * Clear Python object references in Handle
854880 */
855881void Handle_clear (Handle * h ) {
856- if (h -> error_cb ) {
882+ if (h -> error_cb )
857883 Py_DECREF (h -> error_cb );
858- }
884+
885+ if (h -> stats_cb )
886+ Py_DECREF (h -> stats_cb );
859887
860888 PyThread_delete_key (h -> tlskey );
861889}
@@ -867,6 +895,9 @@ int Handle_traverse (Handle *h, visitproc visit, void *arg) {
867895 if (h -> error_cb )
868896 Py_VISIT (h -> error_cb );
869897
898+ if (h -> stats_cb )
899+ Py_VISIT (h -> stats_cb );
900+
870901 return 0 ;
871902}
872903
@@ -1113,6 +1144,15 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
11131144 continue ;
11141145
11151146 } else if (!strcmp (k , "error_cb" )) {
1147+ if (!PyCallable_Check (vo )) {
1148+ PyErr_SetString (PyExc_TypeError ,
1149+ "expected error_cb property "
1150+ "as a callable function" );
1151+ rd_kafka_topic_conf_destroy (tconf );
1152+ rd_kafka_conf_destroy (conf );
1153+ Py_DECREF (ks );
1154+ return NULL ;
1155+ }
11161156 if (h -> error_cb ) {
11171157 Py_DECREF (h -> error_cb );
11181158 h -> error_cb = NULL ;
@@ -1123,6 +1163,27 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
11231163 }
11241164 Py_DECREF (ks );
11251165 continue ;
1166+ } else if (!strcmp (k , "stats_cb" )) {
1167+ if (!PyCallable_Check (vo )) {
1168+ PyErr_SetString (PyExc_TypeError ,
1169+ "expected stats_cb property "
1170+ "as a callable function" );
1171+ rd_kafka_topic_conf_destroy (tconf );
1172+ rd_kafka_conf_destroy (conf );
1173+ Py_DECREF (ks );
1174+ return NULL ;
1175+ }
1176+
1177+ if (h -> stats_cb ) {
1178+ Py_DECREF (h -> stats_cb );
1179+ h -> stats_cb = NULL ;
1180+ }
1181+ if (vo != Py_None ) {
1182+ h -> stats_cb = vo ;
1183+ Py_INCREF (h -> stats_cb );
1184+ }
1185+ Py_DECREF (ks );
1186+ continue ;
11261187 }
11271188
11281189 /* Special handling for certain config keys. */
@@ -1174,6 +1235,10 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
11741235
11751236 if (h -> error_cb )
11761237 rd_kafka_conf_set_error_cb (conf , error_cb );
1238+
1239+ if (h -> stats_cb )
1240+ rd_kafka_conf_set_stats_cb (conf , stats_cb );
1241+
11771242 rd_kafka_topic_conf_set_opaque (tconf , h );
11781243 rd_kafka_conf_set_default_topic_conf (conf , tconf );
11791244
0 commit comments