From 660b23ac7a451b9ee0607093ab1f2ba92c6571be Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 27 Dec 2023 09:45:27 +0800 Subject: [PATCH] Log Improvement Back Port (#8569) close pingcap/tiflash#8568 --- .../CreatingSetsBlockInputStream.cpp | 49 ++++++++----------- .../Coprocessor/DAGStorageInterpreter.cpp | 4 +- dbms/src/Flash/EstablishCall.cpp | 8 ++- dbms/src/Flash/FlashService.cpp | 36 +++++++++----- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 23 +++------ dbms/src/Flash/Mpp/ExchangeReceiver.h | 3 +- dbms/src/Flash/Mpp/MPPTask.cpp | 17 +++++-- dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp | 7 +-- dbms/src/Flash/Mpp/MPPTaskStatistics.cpp | 4 +- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 15 ++++-- dbms/src/Interpreters/Join.cpp | 2 +- dbms/src/Interpreters/executeQuery.cpp | 2 +- libs/libdaemon/src/BaseDaemon.cpp | 2 +- 13 files changed, 95 insertions(+), 77 deletions(-) diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp index ae2626feafe..8c11aef85fd 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -161,7 +161,7 @@ void CreatingSetsBlockInputStream::createAll() exception_from_workers.size()); std::rethrow_exception(exception_from_workers.front()); } - LOG_DEBUG( + LOG_INFO( log, "Creating all tasks takes {} sec. ", watch.elapsedSeconds()); @@ -184,7 +184,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) Stopwatch watch; try { - LOG_DEBUG(log, "{}", gen_log_msg()); + LOG_INFO(log, "{}", gen_log_msg()); BlockOutputStreamPtr table_out; if (subquery.table) table_out = subquery.table->write({}, {}); @@ -203,7 +203,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) { if (isCancelled()) { - LOG_DEBUG(log, "Query was cancelled during set / join or temporary table creation."); + LOG_WARNING(log, "Query was cancelled during set / join or temporary table creation."); return; } @@ -263,31 +263,24 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) if (subquery.join) head_rows = subquery.join->getTotalBuildInputRows(); - if (head_rows != 0) - { - // avoid generate log message when log level > DEBUG. - auto gen_debug_log_msg = [&] { - FmtBuffer msg; - msg.append("Created. "); - - if (subquery.set) - msg.fmtAppend("Set with {} entries from {} rows. ", subquery.set->getTotalRowCount(), head_rows); - if (subquery.join) - msg.fmtAppend("Join with {} entries from {} rows. ", subquery.join->getTotalRowCount(), head_rows); - if (subquery.table) - msg.fmtAppend("Table with {} rows. ", head_rows); - - msg.fmtAppend("In {:.3f} sec. ", watch.elapsedSeconds()); - msg.fmtAppend("using {} threads.", subquery.join ? subquery.join->getBuildConcurrency() : 1); - return msg.toString(); - }; - - LOG_DEBUG(log, "{}", gen_debug_log_msg()); - } - else - { - LOG_DEBUG(log, "Subquery has empty result."); - } + // avoid generate log message when log level > INFO. + auto gen_finish_log_msg = [&] { + FmtBuffer msg; + msg.append("Created. "); + + if (subquery.set) + msg.fmtAppend("Set with {} entries from {} rows. ", head_rows > 0 ? subquery.set->getTotalRowCount() : 0, head_rows); + if (subquery.join) + msg.fmtAppend("Join with {} entries from {} rows. ", head_rows > 0 ? subquery.join->getTotalRowCount() : 0, head_rows); + if (subquery.table) + msg.fmtAppend("Table with {} rows. ", head_rows); + + msg.fmtAppend("In {:.3f} sec. ", watch.elapsedSeconds()); + msg.fmtAppend("using {} threads.", subquery.join ? subquery.join->getBuildConcurrency() : 1); + return msg.toString(); + }; + + LOG_INFO(log, "{}", gen_finish_log_msg()); } catch (...) { diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 46c82135a40..473fcbf5c4f 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -998,7 +998,7 @@ std::unordered_map DAG auto [storages, locks, storage_schema_versions, ok] = get_and_lock_storages(false); if (ok) { - LOG_INFO(log, "{}", log_schema_version("OK, no syncing required.", storage_schema_versions)); + LOG_DEBUG(log, "{}", log_schema_version("OK, no syncing required.", storage_schema_versions)); } else /// If first try failed, sync schema and try again. @@ -1010,7 +1010,7 @@ std::unordered_map DAG std::tie(storages, locks, storage_schema_versions, ok) = get_and_lock_storages(true); if (ok) { - LOG_INFO(log, "{}", log_schema_version("OK after syncing.", storage_schema_versions)); + LOG_DEBUG(log, "{}", log_schema_version("OK after syncing.", storage_schema_versions)); } else throw TiFlashException("Shouldn't reach here", Errors::Coprocessor::Internal); diff --git a/dbms/src/Flash/EstablishCall.cpp b/dbms/src/Flash/EstablishCall.cpp index a04e5fae2c3..13c7dbce901 100644 --- a/dbms/src/Flash/EstablishCall.cpp +++ b/dbms/src/Flash/EstablishCall.cpp @@ -206,7 +206,13 @@ void EstablishCallData::writeDone(String msg, const grpc::Status & status) if (async_tunnel_sender) { - LOG_INFO(async_tunnel_sender->getLogger(), "connection for {} cost {} ms, including {} ms to waiting task.", async_tunnel_sender->getTunnelId(), stopwatch->elapsedMilliseconds(), waiting_task_time_ms); + auto time = stopwatch->elapsedMilliseconds(); + LOG_IMPL(async_tunnel_sender->getLogger(), + msg.empty() && time < 1000 ? Poco::Message::PRIO_DEBUG : Poco::Message::PRIO_INFORMATION, + "connection for {} cost {} ms, including {} ms to waiting task.", + async_tunnel_sender->getTunnelId(), + time, + waiting_task_time_ms); RUNTIME_ASSERT(!async_tunnel_sender->isConsumerFinished(), async_tunnel_sender->getLogger(), "tunnel {} consumer finished in advance", async_tunnel_sender->getTunnelId()); diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index fa232780f41..8a1e441cb15 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -113,13 +113,14 @@ grpc::Status FlashService::Coprocessor( coprocessor::Response * response) { CPUAffinityManager::getInstance().bindSelfGrpcThread(); - LOG_DEBUG(log, "Handling coprocessor request: {}", request->DebugString()); + bool is_remote_read = getClientMetaVarWithDefault(grpc_context, "is_remote_read", "") == "true"; + auto log_level = is_remote_read ? Poco::Message::PRIO_INFORMATION : Poco::Message::PRIO_DEBUG; + LOG_IMPL(log, log_level, "Handling coprocessor request, is_remote_read: {}, start ts: {}, region id: {}, region epoch: {}", is_remote_read, request->start_ts(), request->context().region_id(), request->context().region_epoch().DebugString()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) return check_result; - bool is_remote_read = getClientMetaVarWithDefault(grpc_context, "is_remote_read", "") == "true"; GET_METRIC(tiflash_coprocessor_request_count, type_cop).Increment(); GET_METRIC(tiflash_coprocessor_handling_request_count, type_cop).Increment(); if (is_remote_read) @@ -140,7 +141,7 @@ grpc::Status FlashService::Coprocessor( const auto & settings = context->getSettingsRef(); auto handle_limit = settings.cop_pool_handle_limit != 0 ? settings.cop_pool_handle_limit.get() : 10 * cop_pool->size(); - auto max_queued_duration_seconds = std::min(settings.cop_pool_max_queued_seconds, 20); + auto max_queued_duration_ms = std::min(settings.cop_pool_max_queued_seconds, 20) * 1000; if (handle_limit > 0) { @@ -155,14 +156,20 @@ grpc::Status FlashService::Coprocessor( grpc::Status ret = executeInThreadPool(*cop_pool, [&] { - if (max_queued_duration_seconds > 0) + auto wait_ms = watch.elapsedMilliseconds(); + if (max_queued_duration_ms > 0) { - if (auto current = watch.elapsedSeconds(); current > max_queued_duration_seconds) + if (wait_ms > static_cast(max_queued_duration_ms)) { - response->mutable_region_error()->mutable_server_is_busy()->set_reason(fmt::format("this task queued in tiflash cop pool too long, current = {}, limit = {}", current, max_queued_duration_seconds)); + response->mutable_region_error()->mutable_server_is_busy()->set_reason(fmt::format("this task queued in tiflash cop pool too long, current = {}ms, limit = {}ms", wait_ms, max_queued_duration_ms)); return grpc::Status::OK; } } + + if (wait_ms > 1000) + log_level = Poco::Message::PRIO_INFORMATION; + LOG_IMPL(log, log_level, "Begin process cop request after wait {} ms, start ts: {}, region id: {}, region epoch: {}", wait_ms, request->start_ts(), request->context().region_id(), request->context().region_epoch().DebugString()); + auto [db_context, status] = createDBContext(grpc_context); if (!status.ok()) { @@ -179,14 +186,14 @@ grpc::Status FlashService::Coprocessor( return cop_handler.execute(); }); - LOG_DEBUG(log, "Handle coprocessor request done: {}, {}", ret.error_code(), ret.error_message()); + LOG_IMPL(log, log_level, "Handle coprocessor request done: {}, {}", ret.error_code(), ret.error_message()); return ret; } grpc::Status FlashService::BatchCoprocessor(grpc::ServerContext * grpc_context, const coprocessor::BatchRequest * request, grpc::ServerWriter * writer) { CPUAffinityManager::getInstance().bindSelfGrpcThread(); - LOG_DEBUG(log, "Handling coprocessor request: {}", request->DebugString()); + LOG_INFO(log, "Handling batch coprocessor request, start ts: {}", request->start_ts()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) @@ -202,6 +209,9 @@ grpc::Status FlashService::BatchCoprocessor(grpc::ServerContext * grpc_context, }); grpc::Status ret = executeInThreadPool(*batch_cop_pool, [&] { + auto wait_ms = watch.elapsedMilliseconds(); + LOG_INFO(log, "Begin process batch cop request after wait {} ms, start ts: {}", wait_ms, request->start_ts()); + auto [db_context, status] = createDBContext(grpc_context); if (!status.ok()) { @@ -212,7 +222,7 @@ grpc::Status FlashService::BatchCoprocessor(grpc::ServerContext * grpc_context, return cop_handler.execute(); }); - LOG_DEBUG(log, "Handle coprocessor request done: {}, {}", ret.error_code(), ret.error_message()); + LOG_INFO(log, "Handle batch coprocessor request done: {}, {}", ret.error_code(), ret.error_message()); return ret; } @@ -222,7 +232,7 @@ grpc::Status FlashService::DispatchMPPTask( mpp::DispatchTaskResponse * response) { CPUAffinityManager::getInstance().bindSelfGrpcThread(); - LOG_DEBUG(log, "Handling mpp dispatch request: {}", request->DebugString()); + LOG_INFO(log, "Handling mpp dispatch request, task meta: {}", request->meta().DebugString()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) return check_result; @@ -278,7 +288,7 @@ grpc::Status AsyncFlashService::establishMPPConnectionAsync(grpc::ServerContext CPUAffinityManager::getInstance().bindSelfGrpcThread(); // Establish a pipe for data transferring. The pipes have registered by the task in advance. // We need to find it out and bind the grpc stream with it. - LOG_DEBUG(log, "Handling establish mpp connection request: {}", request->DebugString()); + LOG_INFO(log, "Handling establish mpp connection request: {}", request->DebugString()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) @@ -297,7 +307,7 @@ grpc::Status FlashService::EstablishMPPConnection(grpc::ServerContext * grpc_con CPUAffinityManager::getInstance().bindSelfGrpcThread(); // Establish a pipe for data transferring. The pipes have registered by the task in advance. // We need to find it out and bind the grpc stream with it. - LOG_DEBUG(log, "Handling establish mpp connection request: {}", request->DebugString()); + LOG_INFO(log, "Handling establish mpp connection request: {}", request->DebugString()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) @@ -350,7 +360,7 @@ grpc::Status FlashService::CancelMPPTask( { CPUAffinityManager::getInstance().bindSelfGrpcThread(); // CancelMPPTask cancels the query of the task. - LOG_DEBUG(log, "cancel mpp task request: {}", request->DebugString()); + LOG_INFO(log, "cancel mpp task request: {}", request->DebugString()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 3dcdfc5770e..71dc2eaeae3 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -550,7 +550,7 @@ void ExchangeReceiverBase::reactor(const std::vector & asyn if (handler->finished()) { --alive_async_connections; - connectionDone(handler->meetError(), handler->getErrMsg(), handler->getLog()); + connectionDone(handler->meetError(), handler->getErrMsg()); } } else @@ -650,7 +650,7 @@ void ExchangeReceiverBase::readLoop(const Request & req) meet_error = true; local_err_msg = getCurrentExceptionMessage(false); } - connectionDone(meet_error, local_err_msg, log); + connectionDone(meet_error, local_err_msg); } template @@ -804,8 +804,7 @@ String ExchangeReceiverBase::getStatusString() template void ExchangeReceiverBase::connectionDone( bool meet_error, - const String & local_err_msg, - const LoggerPtr & log) + const String & local_err_msg) { Int32 copy_live_conn = -1; { @@ -819,22 +818,16 @@ void ExchangeReceiverBase::connectionDone( } copy_live_conn = --live_connections; } - LOG_DEBUG( - log, - "connection end. meet error: {}, err msg: {}, current alive connections: {}", - meet_error, - local_err_msg, - copy_live_conn); - if (copy_live_conn == 0) - { - LOG_DEBUG(log, "All threads end in ExchangeReceiver"); - } - else if (copy_live_conn < 0) + if (copy_live_conn < 0) throw Exception("live_connections should not be less than 0!"); if (meet_error || copy_live_conn == 0) + { + auto log_level = meet_error ? Poco::Message::PRIO_WARNING : Poco::Message::PRIO_INFORMATION; + LOG_IMPL(exc_log, log_level, "Finish receiver channels, meet error: {}, error message: {}", meet_error, local_err_msg); finishAllMsgChannels(); + } } template diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.h b/dbms/src/Flash/Mpp/ExchangeReceiver.h index 7bfce910894..afcfd054e8d 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.h +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.h @@ -176,8 +176,7 @@ class ExchangeReceiverBase void connectionDone( bool meet_error, - const String & local_err_msg, - const LoggerPtr & log); + const String & local_err_msg); void finishAllMsgChannels(); void cancelAllMsgChannels(); diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 69acbfadf43..14a5a1bc962 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -71,7 +71,7 @@ MPPTask::~MPPTask() if (process_list_entry != nullptr && current_memory_tracker != process_list_entry->get().getMemoryTrackerPtr().get()) current_memory_tracker = process_list_entry->get().getMemoryTrackerPtr().get(); abortTunnels("", true); - LOG_DEBUG(log, "finish MPPTask: {}", id.toString()); + LOG_INFO(log, "finish MPPTask: {}", id.toString()); } void MPPTask::abortTunnels(const String & message, bool wait_sender_finish) @@ -361,14 +361,17 @@ void MPPTask::runImpl() String err_msg; try { - LOG_INFO(log, "task starts preprocessing"); + LOG_DEBUG(log, "task starts preprocessing"); preprocess(); + auto time_cost_in_preprocess_ms = stopwatch.elapsedMilliseconds(); + LOG_DEBUG(log, "task preprocess done"); schedule_entry.setNeededThreads(estimateCountOfNewThreads()); LOG_DEBUG(log, "Estimate new thread count of query: {} including tunnel_threads: {}, receiver_threads: {}", schedule_entry.getNeededThreads(), dag_context->tunnel_set->getExternalThreadCnt(), new_thread_count_of_mpp_receiver); scheduleOrWait(); - LOG_INFO(log, "task starts running"); + auto time_cost_in_schedule_ms = stopwatch.elapsedMilliseconds() - time_cost_in_preprocess_ms; + LOG_INFO(log, "task starts running, time cost in schedule: {} ms, time cost in preprocess {} ms", time_cost_in_schedule_ms, time_cost_in_preprocess_ms); if (status.load() != RUNNING) { /// when task is in running state, canceling the task will call sendCancelToQuery to do the cancellation, however @@ -379,6 +382,10 @@ void MPPTask::runImpl() mpp_task_statistics.start(); auto result = query_executor_holder->execute(); + auto log_level = Poco::Message::PRIO_DEBUG; + if (!result.is_success || status != RUNNING) + log_level = Poco::Message::PRIO_INFORMATION; + LOG_IMPL(log, log_level, "mpp task finish execute, success: {}, status: {}", result.is_success, magic_enum::enum_name(status.load())); if (likely(result.is_success)) { // finish receiver @@ -408,7 +415,7 @@ void MPPTask::runImpl() if (err_msg.empty()) { if (switchStatus(RUNNING, FINISHED)) - LOG_INFO(log, "finish task"); + LOG_DEBUG(log, "finish task"); else LOG_WARNING(log, "finish task which is in {} state", magic_enum::enum_name(status.load())); if (status == FINISHED) @@ -442,7 +449,7 @@ void MPPTask::runImpl() mpp_task_statistics.end(status.load(), err_string); mpp_task_statistics.logTracingJson(); - LOG_INFO(log, "task ends, time cost is {} ms.", stopwatch.elapsedMilliseconds()); + LOG_DEBUG(log, "task ends, time cost is {} ms.", stopwatch.elapsedMilliseconds()); unregisterTask(); } diff --git a/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp b/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp index 7ebf65f401c..83b4d24c9a2 100644 --- a/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskScheduleEntry.cpp @@ -41,7 +41,8 @@ bool MPPTaskScheduleEntry::schedule(ScheduleState state) std::unique_lock lock(schedule_mu); if (schedule_state == ScheduleState::WAITING) { - LOG_INFO(log, "task is {}.", state == ScheduleState::SCHEDULED ? "scheduled" : " failed to schedule"); + auto log_level = state == ScheduleState::SCHEDULED ? Poco::Message::PRIO_DEBUG : Poco::Message::PRIO_WARNING; + LOG_IMPL(log, log_level, "task is {}.", state == ScheduleState::SCHEDULED ? "scheduled" : " failed to schedule"); schedule_state = state; schedule_cv.notify_one(); return true; @@ -51,7 +52,7 @@ bool MPPTaskScheduleEntry::schedule(ScheduleState state) void MPPTaskScheduleEntry::waitForSchedule() { - LOG_INFO(log, "task waits for schedule"); + LOG_DEBUG(log, "task waits for schedule"); Stopwatch stopwatch; double time_cost = 0; { @@ -69,7 +70,7 @@ void MPPTaskScheduleEntry::waitForSchedule() throw Exception(fmt::format("{} is failed to schedule because of being cancelled in min-tso scheduler after waiting for {}s.", id.toString(), time_cost)); } } - LOG_INFO(log, "task waits for {} s to schedule and starts to run in parallel.", time_cost); + LOG_DEBUG(log, "task waits for {} s to schedule and starts to run in parallel.", time_cost); } const MPPTaskId & MPPTaskScheduleEntry::getMPPTaskId() const diff --git a/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp b/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp index f0436278d12..289c53dc70c 100644 --- a/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp @@ -94,8 +94,10 @@ const BaseRuntimeStatistics & MPPTaskStatistics::collectRuntimeStatistics() void MPPTaskStatistics::logTracingJson() { - LOG_INFO( + LOG_IMPL( logger, + /// don't use info log for initializing status since it does not contains too many information + status == INITIALIZING ? Poco::Message::PRIO_DEBUG : Poco::Message::PRIO_INFORMATION, R"({{"query_tso":{},"task_id":{},"is_root":{},"sender_executor_id":"{}","executors":{},"host":"{}")" R"(,"task_init_timestamp":{},"task_start_timestamp":{},"task_end_timestamp":{})" R"(,"compile_start_timestamp":{},"compile_end_timestamp":{})" diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index aedaf0c0112..536cc875842 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -44,7 +44,7 @@ MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 ac } if (isDisabled()) { - LOG_INFO(log, "MinTSOScheduler is disabled!"); + LOG_WARNING(log, "MinTSOScheduler is disabled!"); } else { @@ -52,7 +52,7 @@ MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 ac { thread_hard_limit = 10000; thread_soft_limit = 5000; - LOG_INFO(log, "hard limit {} should > soft limit {} and under maximum {}, so MinTSOScheduler set them as {}, {} by default, and active_set_soft_limit is {}.", hard_limit, soft_limit, OS_THREAD_SOFT_LIMIT, thread_hard_limit, thread_soft_limit, active_set_soft_limit); + LOG_WARNING(log, "hard limit {} should > soft limit {} and under maximum {}, so MinTSOScheduler set them as {}, {} by default, and active_set_soft_limit is {}.", hard_limit, soft_limit, OS_THREAD_SOFT_LIMIT, thread_hard_limit, thread_soft_limit, active_set_soft_limit); } else { @@ -203,7 +203,7 @@ bool MinTSOScheduler::scheduleImp(const UInt64 tso, const MPPQueryTaskSetPtr & q } GET_METRIC(tiflash_task_scheduler, type_active_queries_count).Set(active_set.size()); GET_METRIC(tiflash_task_scheduler, type_estimated_thread_usage).Set(estimated_thread_usage); - LOG_INFO(log, "{} is scheduled (active set size = {}) due to available threads {}, after applied for {} threads, used {} of the thread {} limit {}.", schedule_entry.getMPPTaskId().toString(), active_set.size(), isWaiting ? "from the waiting set" : "directly", needed_threads, estimated_thread_usage, min_tso == tso ? "hard" : "soft", min_tso == tso ? thread_hard_limit : thread_soft_limit); + LOG_DEBUG(log, "{} is scheduled (active set size = {}) due to available threads {}, after applied for {} threads, used {} of the thread {} limit {}.", schedule_entry.getMPPTaskId().toString(), active_set.size(), isWaiting ? "from the waiting set" : "directly", needed_threads, estimated_thread_usage, min_tso == tso ? "hard" : "soft", min_tso == tso ? thread_hard_limit : thread_soft_limit); return true; } else @@ -235,7 +235,14 @@ bool MinTSOScheduler::scheduleImp(const UInt64 tso, const MPPQueryTaskSetPtr & q GET_METRIC(tiflash_task_scheduler, type_waiting_queries_count).Set(waiting_set.size()); GET_METRIC(tiflash_task_scheduler, type_waiting_tasks_count).Increment(); } - LOG_INFO(log, "threads are unavailable for the query {} or active set is full (size = {}), need {}, but used {} of the thread soft limit {},{} waiting set size = {}", tso, active_set.size(), needed_threads, estimated_thread_usage, thread_soft_limit, isWaiting ? "" : " put into", waiting_set.size()); + LOG_INFO(log, "Resource temporary not available for query with start_ts {}(is first schedule: {}), available threads count are {}, available active set size = {}, " + "required threads count are {}, waiting set size = {}", + tso, + !isWaiting, + thread_soft_limit - estimated_thread_usage, + active_set_soft_limit - active_set.size(), + needed_threads, + waiting_set.size()); return false; } } diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 087c83c6c47..3beab46584d 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -162,7 +162,7 @@ Join::Join( throw Exception("Not supported: non left join with left conditions"); if (unlikely(!right_filter_column.empty() && !isRightJoin(kind))) throw Exception("Not supported: non right join with right conditions"); - LOG_INFO(log, "FineGrainedShuffle flag {}, stream count {}", enable_fine_grained_shuffle, fine_grained_shuffle_count); + LOG_DEBUG(log, "FineGrainedShuffle flag {}, stream count {}", enable_fine_grained_shuffle, fine_grained_shuffle_count); } void Join::setBuildTableState(BuildTableState state_) diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index ce3ec0107ac..a3852c6224e 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -436,7 +436,7 @@ void logQueryPipeline(const LoggerPtr & logger, const BlockInputStreamPtr & in) in->dumpTree(log_buffer); return log_buffer.toString(); }; - LOG_DEBUG(logger, pipeline_log_str()); + LOG_INFO(logger, pipeline_log_str()); } BlockIO executeQuery( diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index c65487d0849..0238ecdfe83 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -760,7 +760,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) // Split log, error log and tracing log. Poco::AutoPtr split = new Poco::ReloadableSplitterChannel; - auto log_level = normalize(config.getString("logger.level", "debug")); + auto log_level = normalize(config.getString("logger.level", "info")); const auto log_path = config.getString("logger.log", ""); if (!log_path.empty()) {