diff --git a/be/src/common/config.h b/be/src/common/config.h index f281ea1063657..d40d5b023b05f 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -23,8 +23,7 @@ #include "configbase.h" -namespace starrocks { -namespace config { +namespace starrocks::config { // The cluster id. CONF_Int32(cluster_id, "-1"); // The port on which ImpalaInternalService is exported. @@ -743,6 +742,13 @@ CONF_String(starmgr_addr, ""); CONF_Int32(starlet_port, "9070"); #endif -} // namespace config +CONF_mBool(dependency_librdkafka_debug_enable, "false"); -} // namespace starrocks +// A comma-separated list of debug contexts to enable. +// Producer debug context: broker, topic, msg +// Consumer debug context: consumer, cgrp, topic, fetch +// Other debug context: generic, metadata, feature, queue, protocol, security, interceptor, plugin +// admin, eos, mock, assigner, conf +CONF_String(dependency_librdkafka_debug, "all"); + +} // namespace starrocks::config diff --git a/be/src/runtime/routine_load/data_consumer.cpp b/be/src/runtime/routine_load/data_consumer.cpp index 92e22806bc5c4..b9885d06fd51d 100644 --- a/be/src/runtime/routine_load/data_consumer.cpp +++ b/be/src/runtime/routine_load/data_consumer.cpp @@ -89,6 +89,9 @@ Status KafkaDataConsumer::init(StreamLoadContext* ctx) { RETURN_IF_ERROR(set_conf("auto.offset.reset", "error")); RETURN_IF_ERROR(set_conf("api.version.request", "true")); RETURN_IF_ERROR(set_conf("api.version.fallback.ms", "0")); + if (config::dependency_librdkafka_debug_enable) { + RETURN_IF_ERROR(set_conf("debug", config::dependency_librdkafka_debug)); + } for (auto& item : ctx->kafka_info->properties) { if (boost::algorithm::starts_with(item.second, "FILE:")) {