Some improvements about names of span logs (#47667)

This commit is contained in:
Frank Chen 2023-03-29 22:28:36 +08:00 committed by GitHub
parent c5b6ed410d
commit 87d235e842
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 36 additions and 10 deletions

View File

@ -68,7 +68,8 @@ bool Span::addAttribute(const Exception & e) noexcept
if (!this->isTraceEnabled())
return false;
return addAttributeImpl("clickhouse.exception", getExceptionMessage(e, false));
return addAttributeImpl("clickhouse.exception", getExceptionMessage(e, false))
&& addAttributeImpl("clickhouse.exception_code", toString(e.code()));
}
bool Span::addAttribute(std::exception_ptr e) noexcept
@ -79,6 +80,15 @@ bool Span::addAttribute(std::exception_ptr e) noexcept
return addAttributeImpl("clickhouse.exception", getExceptionMessage(e, false));
}
bool Span::addAttribute(const ExecutionStatus & e) noexcept
{
if (!this->isTraceEnabled())
return false;
return addAttributeImpl("clickhouse.exception", e.message)
&& addAttributeImpl("clickhouse.exception_code", toString(e.code));
}
bool Span::addAttributeImpl(std::string_view name, std::string_view value) noexcept
{
try

View File

@ -9,6 +9,7 @@ struct Settings;
class OpenTelemetrySpanLog;
class WriteBuffer;
class ReadBuffer;
struct ExecutionStatus;
namespace OpenTelemetry
{
@ -57,6 +58,7 @@ struct Span
bool addAttribute(std::string_view name, std::function<String()> value_supplier) noexcept;
bool addAttribute(const Exception & e) noexcept;
bool addAttribute(std::exception_ptr e) noexcept;
bool addAttribute(const ExecutionStatus & e) noexcept;
bool isTraceEnabled() const
{

View File

@ -11,6 +11,7 @@
#include <Poco/Util/Application.h>
#include <Poco/Util/LayeredConfiguration.h>
#include <base/demangle.h>
namespace DB
{
@ -29,6 +30,7 @@ namespace CurrentMetrics
extern const Metric LocalThreadActive;
}
static constexpr auto DEFAULT_THREAD_NAME = "ThreadPool";
template <typename Thread>
ThreadPoolImpl<Thread>::ThreadPoolImpl()
@ -342,7 +344,7 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
while (true)
{
/// This is inside the loop to also reset previous thread names set inside the jobs.
setThreadName("ThreadPool");
setThreadName(DEFAULT_THREAD_NAME);
/// A copy of parent trace context
DB::OpenTelemetry::TracingContextOnThread parent_thead_trace_context;
@ -389,10 +391,17 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
if (thread_trace_context.root_span.isTraceEnabled())
{
/// Use the thread name as operation name so that the tracing log will be more clear.
/// The thread name is usually set in the jobs, we can only get the name after the job finishes
/// The thread name is usually set in jobs, we can only get the name after the job finishes
std::string thread_name = getThreadName();
if (!thread_name.empty())
if (!thread_name.empty() && thread_name != DEFAULT_THREAD_NAME)
{
thread_trace_context.root_span.operation_name = thread_name;
}
else
{
/// If the thread name is not set, use the type name of the job instead
thread_trace_context.root_span.operation_name = demangle(job.target_type().name());
}
}
/// job should be reset before decrementing scheduled_jobs to

View File

@ -75,7 +75,7 @@ bool ExecutionThreadContext::executeTask()
if (trace_processors)
{
span = std::make_unique<OpenTelemetry::SpanHolder>("ExecutionThreadContext::executeTask() " + node->processor->getName());
span = std::make_unique<OpenTelemetry::SpanHolder>(node->processor->getName());
span->addAttribute("thread_number", thread_number);
}
std::optional<Stopwatch> execution_time_watch;

View File

@ -10,6 +10,8 @@
#include <Interpreters/ProcessList.h>
#include <Interpreters/Context.h>
#include <Common/scope_guard_safe.h>
#include <Common/Exception.h>
#include <Common/OpenTelemetryTraceContext.h>
#ifndef NDEBUG
#include <Common/Stopwatch.h>
@ -94,6 +96,9 @@ void PipelineExecutor::execute(size_t num_threads)
if (num_threads < 1)
num_threads = 1;
OpenTelemetry::SpanHolder span("PipelineExecutor::execute()");
span.addAttribute("clickhouse.thread_num", num_threads);
try
{
executeImpl(num_threads);
@ -108,6 +113,8 @@ void PipelineExecutor::execute(size_t num_threads)
}
catch (...)
{
span.addAttribute(ExecutionStatus::fromCurrentException());
#ifndef NDEBUG
LOG_TRACE(log, "Exception while executing query. Current state:\n{}", dumpPipeline());
#endif

View File

@ -1059,13 +1059,11 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
/** If exception is received from remote server, then stack trace is embedded in message.
* If exception is thrown on local server, then stack trace is in separate field.
*/
std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true);
int exception_code = getCurrentExceptionCode();
trySendExceptionToClient(exception_message, exception_code, request, response, used_output);
ExecutionStatus status = ExecutionStatus::fromCurrentException("", with_stacktrace);
trySendExceptionToClient(status.message, status.code, request, response, used_output);
if (thread_trace_context)
thread_trace_context->root_span.addAttribute("clickhouse.exception_code", exception_code);
thread_trace_context->root_span.addAttribute(status);
}
used_output.finalize();