From 7cae2c6c9ea406361915e3459e0d75e559d2f5ea Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Sun, 23 Jun 2013 23:43:31 +0200 Subject: [PATCH] rd_kafka_set_logger() is now per rd_kafka_t handle. --- examples/rdkafka_performance.c | 3 ++- rdkafka.c | 28 ++++++++++++++++------------ rdkafka.h | 9 ++++++--- rdkafka_int.h | 6 ++++++ 4 files changed, 30 insertions(+), 16 deletions(-) diff --git a/examples/rdkafka_performance.c b/examples/rdkafka_performance.c index 181c58c7..f023600b 100644 --- a/examples/rdkafka_performance.c +++ b/examples/rdkafka_performance.c @@ -312,7 +312,8 @@ int main (int argc, char **argv) { exit(1); } - rd_kafka_dbg_set(rk, debug); + if (debug) + rd_kafka_set_log_level(rk, 7); /* Add broker(s) */ if (rd_kafka_brokers_add(rk, brokers) < 1) { diff --git a/rdkafka.c b/rdkafka.c index a0f1696c..60d528fb 100644 --- a/rdkafka.c +++ b/rdkafka.c @@ -41,9 +41,6 @@ static pthread_once_t rd_kafka_global_init_once = PTHREAD_ONCE_INIT; -static void (*rd_kafka_log_cb) (const rd_kafka_t *rk, int level, - const char *fac, - const char *buf) = rd_kafka_log_print; @@ -79,7 +76,7 @@ void rd_kafka_log0 (const rd_kafka_t *rk, const char *extra, int level, va_list ap; int elen = 0; - if (!rd_kafka_log_cb) + if (!rk->rk_log_cb || level > rk->rk_log_level) return; if (extra) { @@ -92,7 +89,7 @@ void rd_kafka_log0 (const rd_kafka_t *rk, const char *extra, int level, vsnprintf(buf+elen, sizeof(buf)-elen, fmt, ap); va_end(ap); - rd_kafka_log_cb(rk, level, fac, buf); + rk->rk_log_cb(rk, level, fac, buf); } @@ -122,11 +119,19 @@ void rd_kafka_log_syslog (const rd_kafka_t *rk, int level, } -void rd_kafka_set_logger (void (*func) (const rd_kafka_t *rk, int level, +void rd_kafka_set_logger (rd_kafka_t *rk, + void (*func) (const rd_kafka_t *rk, int level, const char *fac, const char *buf)) { - rd_kafka_log_cb = func; + rk->rk_log_cb = func; } +void rd_kafka_set_log_level (rd_kafka_t *rk, int level) { + rk->rk_log_level = level; +} + + + + @@ -460,6 +465,10 @@ rd_kafka_t *rd_kafka_new (rd_kafka_type_t type, const rd_kafka_conf_t *conf, TAILQ_INIT(&rk->rk_brokers); TAILQ_INIT(&rk->rk_topics); + rk->rk_log_cb = rd_kafka_log_print; + rk->rk_log_level = 6; + + switch (rk->rk_type) { case RD_KAFKA_CONSUMER: @@ -590,11 +599,6 @@ int rd_kafka_poll (rd_kafka_t *rk, int timeout_ms) { -void rd_kafka_dbg_set (rd_kafka_t *rk, int onoff) { - rd_dbg_set(onoff); -} - - static void rd_kafka_toppar_dump (FILE *fp, const char *indent, rd_kafka_toppar_t *rktp) { diff --git a/rdkafka.h b/rdkafka.h index d23adee1..d9a4c3cd 100644 --- a/rdkafka.h +++ b/rdkafka.h @@ -607,14 +607,17 @@ void rd_kafka_log_syslog (const rd_kafka_t *rk, int level, * * NOTE: 'rk' may be passed as NULL. */ -void rd_kafka_set_logger (void (*func) (const rd_kafka_t *rk, int level, +void rd_kafka_set_logger (rd_kafka_t *rk, + void (*func) (const rd_kafka_t *rk, int level, const char *fac, const char *buf)); /** - * Enable/disable rdkafka internal debugging + * Specifies the maximum logging level produced by + * internal kafka logging and debugging. + * Set to LOG_DEBUG (7) to enable debugging. */ -void rd_kafka_dbg_set (rd_kafka_t *rk, int onoff); +void rd_kafka_set_log_level (rd_kafka_t *rk, int level); diff --git a/rdkafka_int.h b/rdkafka_int.h index c566182a..238086fd 100644 --- a/rdkafka_int.h +++ b/rdkafka_int.h @@ -365,6 +365,12 @@ typedef struct rd_kafka_s { } rk_u; #define rk_consumer rk_u.consumer #define rk_producer rk_u.producer + + void (*rk_log_cb) (const rd_kafka_t *rk, int level, + const char *fac, + const char *buf); + int rk_log_level; + struct { char msg[512]; int err; /* errno */