@@ -1076,6 +1076,39 @@ static int stats_cb(rd_kafka_t *rk, char *json, size_t json_len, void *opaque) {
10761076 return 0 ;
10771077}
10781078
1079+ static void log_cb (const rd_kafka_t * rk , int level ,
1080+ const char * fac , const char * buf ) {
1081+ Handle * h = rd_kafka_opaque (rk );
1082+ PyObject * result ;
1083+ CallState * cs ;
1084+ static const int level_map [8 ] = {
1085+ /* Map syslog levels to python logging levels */
1086+ [0 ] = 50 , /* LOG_EMERG -> logging.CRITICAL */
1087+ [1 ] = 50 , /* LOG_ALERT -> logging.CRITICAL */
1088+ [2 ] = 50 , /* LOG_CRIT -> logging.CRITICAL */
1089+ [3 ] = 40 , /* LOG_ERR -> logging.ERROR */
1090+ [4 ] = 30 , /* LOG_WARNING -> logging.WARNING */
1091+ [5 ] = 20 , /* LOG_NOTICE -> logging.INFO */
1092+ [6 ] = 20 , /* LOG_INFO -> logging.INFO */
1093+ [7 ] = 10 , /* LOG_DEBUG -> logging.DEBUG */
1094+ };
1095+
1096+ cs = CallState_get (h );
1097+ result = PyObject_CallMethod (h -> logger , "log" , "issss" ,
1098+ level_map [level ],
1099+ "%s [%s] %s" ,
1100+ fac , rd_kafka_name (rk ), buf );
1101+
1102+ if (result )
1103+ Py_DECREF (result );
1104+ else {
1105+ CallState_crash (cs );
1106+ rd_kafka_yield (h -> rk );
1107+ }
1108+
1109+ CallState_resume (cs );
1110+ }
1111+
10791112/****************************************************************************
10801113 *
10811114 *
@@ -1098,6 +1131,8 @@ void Handle_clear (Handle *h) {
10981131 if (h -> stats_cb )
10991132 Py_DECREF (h -> stats_cb );
11001133
1134+ Py_XDECREF (h -> logger );
1135+
11011136 if (h -> initiated )
11021137 PyThread_delete_key (h -> tlskey );
11031138}
@@ -1443,7 +1478,20 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
14431478 Py_XDECREF (ks8 );
14441479 Py_DECREF (ks );
14451480 continue ;
1446- }
1481+ } else if (!strcmp (k , "logger" )) {
1482+ if (h -> logger ) {
1483+ Py_DECREF (h -> logger );
1484+ h -> logger = NULL ;
1485+ }
1486+
1487+ if (vo != Py_None ) {
1488+ h -> logger = vo ;
1489+ Py_INCREF (h -> logger );
1490+ }
1491+ Py_XDECREF (ks8 );
1492+ Py_DECREF (ks );
1493+ continue ;
1494+ }
14471495
14481496 /* Special handling for certain config keys. */
14491497 if (ktype == RD_KAFKA_PRODUCER )
@@ -1509,6 +1557,13 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
15091557 if (h -> stats_cb )
15101558 rd_kafka_conf_set_stats_cb (conf , stats_cb );
15111559
1560+ if (h -> logger ) {
1561+ /* Write logs to log queue (which is forwarded
1562+ * to the polled queue in the Producer/Consumer constructors) */
1563+ rd_kafka_conf_set (conf , "log.queue" , "true" , NULL , 0 );
1564+ rd_kafka_conf_set_log_cb (conf , log_cb );
1565+ }
1566+
15121567 rd_kafka_topic_conf_set_opaque (tconf , h );
15131568 rd_kafka_conf_set_default_topic_conf (conf , tconf );
15141569
0 commit comments