mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Some improvements about names of span logs (#47667)
This commit is contained in:
parent
c5b6ed410d
commit
87d235e842
@ -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
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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();
|
||||
|
Loading…
Reference in New Issue
Block a user