Merge pull request #57682 from azat/system.stack_trace/analyzer

Add support for system.stack_trace filtering optimizations for analyzer
This commit is contained in:
Alexey Milovidov 2023-12-22 16:28:28 +01:00 committed by GitHub
commit 08ff37f64e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 100 additions and 16 deletions

View File

@ -6,6 +6,7 @@
#include <mutex>
#include <filesystem>
#include <unordered_map>
#include <memory>
#include <base/scope_guard.h>
@ -24,9 +25,14 @@
#include <Common/HashTable/Hash.h>
#include <Common/logger_useful.h>
#include <Common/Stopwatch.h>
#include <Core/ColumnsWithTypeAndName.h>
#include <Interpreters/Context.h>
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Processors/QueryPlan/SourceStepWithFilter.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <QueryPipeline/Pipe.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <base/getThreadId.h>
@ -162,7 +168,7 @@ bool wait(int timeout_ms)
}
using ThreadIdToName = std::unordered_map<UInt64, String, DefaultHash<UInt64>>;
ThreadIdToName getFilteredThreadNames(ASTPtr query, ContextPtr context, const PaddedPODArray<UInt64> & thread_ids, Poco::Logger * log)
ThreadIdToName getFilteredThreadNames(const ActionsDAG::Node * predicate, ContextPtr context, const PaddedPODArray<UInt64> & thread_ids, Poco::Logger * log)
{
ThreadIdToName tid_to_name;
MutableColumnPtr all_thread_names = ColumnString::create();
@ -193,7 +199,7 @@ ThreadIdToName getFilteredThreadNames(ASTPtr query, ContextPtr context, const Pa
LOG_TRACE(log, "Read {} thread names for {} threads, took {} ms", tid_to_name.size(), thread_ids.size(), watch.elapsedMilliseconds());
Block block { ColumnWithTypeAndName(std::move(all_thread_names), std::make_shared<DataTypeString>(), "thread_name") };
VirtualColumnUtils::filterBlockWithQuery(query, block, context);
VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context);
ColumnPtr thread_names = std::move(block.getByPosition(0).column);
std::unordered_set<String> filtered_thread_names;
@ -218,14 +224,16 @@ ThreadIdToName getFilteredThreadNames(ASTPtr query, ContextPtr context, const Pa
/// We must wait for every thread one by one sequentially,
/// because there is a limit on number of queued signals in OS and otherwise signals may get lost.
/// Also, non-RT signals are not delivered if previous signal is handled right now (by default; but we use RT signals).
class StorageSystemStackTraceSource : public ISource
class StackTraceSource : public ISource
{
public:
StorageSystemStackTraceSource(const Names & column_names, Block header_, const ASTPtr query_, ContextPtr context_, UInt64 max_block_size_, Poco::Logger * log_)
StackTraceSource(const Names & column_names, Block header_, ASTPtr && query_, ActionsDAGPtr && filter_dag_, ContextPtr context_, UInt64 max_block_size_, Poco::Logger * log_)
: ISource(header_)
, context(context_)
, header(std::move(header_))
, query(query_)
, query(std::move(query_))
, filter_dag(std::move(filter_dag_))
, predicate(filter_dag ? filter_dag->getOutputs().at(0) : nullptr)
, max_block_size(max_block_size_)
, pipe_read_timeout_ms(static_cast<int>(context->getSettingsRef().storage_system_stack_trace_pipe_read_timeout_ms.totalMilliseconds()))
, log(log_)
@ -259,7 +267,7 @@ protected:
ThreadIdToName thread_names;
if (read_thread_names)
thread_names = getFilteredThreadNames(query, context, thread_ids_data, log);
thread_names = getFilteredThreadNames(predicate, context, thread_ids_data, log);
for (UInt64 tid : thread_ids_data)
{
@ -343,6 +351,8 @@ private:
ContextPtr context;
Block header;
const ASTPtr query;
const ActionsDAGPtr filter_dag;
const ActionsDAG::Node * predicate;
const size_t max_block_size;
const int pipe_read_timeout_ms;
@ -372,11 +382,55 @@ private:
}
Block block { ColumnWithTypeAndName(std::move(all_thread_ids), std::make_shared<DataTypeUInt64>(), "thread_id") };
VirtualColumnUtils::filterBlockWithQuery(query, block, context);
VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context);
return block.getByPosition(0).column;
}
};
class ReadFromSystemStackTrace : public SourceStepWithFilter
{
public:
std::string getName() const override { return "ReadFromSystemStackTrace"; }
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override
{
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
Pipe pipe(std::make_shared<StackTraceSource>(
column_names,
getOutputStream().header,
std::move(query),
std::move(filter_actions_dag),
context,
max_block_size,
log));
pipeline.init(std::move(pipe));
}
ReadFromSystemStackTrace(
const Names & column_names_,
Block sample_block,
ASTPtr && query_,
ContextPtr context_,
size_t max_block_size_,
Poco::Logger * log_)
: SourceStepWithFilter(DataStream{.header = std::move(sample_block)})
, column_names(column_names_)
, query(query_)
, context(std::move(context_))
, max_block_size(max_block_size_)
, log(log_)
{
}
private:
Names column_names;
ASTPtr query;
ContextPtr context;
size_t max_block_size;
Poco::Logger * log;
};
}
@ -412,23 +466,27 @@ StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_)
}
Pipe StorageSystemStackTrace::read(
void StorageSystemStackTrace::read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum /*processed_stage*/,
const size_t max_block_size,
const size_t /*num_streams*/)
size_t max_block_size,
size_t /*num_streams*/)
{
storage_snapshot->check(column_names);
return Pipe(std::make_shared<StorageSystemStackTraceSource>(
Block sample_block = storage_snapshot->metadata->getSampleBlock();
auto reading = std::make_unique<ReadFromSystemStackTrace>(
column_names,
storage_snapshot->metadata->getSampleBlock(),
sample_block,
query_info.query->clone(),
context,
max_block_size,
log));
log);
query_plan.addStep(std::move(reading));
}
}

View File

@ -25,14 +25,15 @@ public:
String getName() const override { return "SystemStackTrace"; }
Pipe read(
void read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size,
size_t num_streams) override;
size_t /*num_streams*/) override;
bool isSystemStorage() const override { return true; }

View File

@ -0,0 +1,5 @@
thread = 0
thread != 0
Send signal to
thread_name = 'foo'
Send signal to 0 threads (total)

View File

@ -0,0 +1,20 @@
#!/usr/bin/env bash
# Tags: no-parallel
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
# NOTE: due to grep "Cannot obtain a stack trace for thread {}' will be ignored automatically, which is the intention.
# no message at all
echo "thread = 0"
$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -nm -q "select * from system.stack_trace where thread_id = 0" |& grep -F -o 'Send signal to'
# send messages to some threads
echo "thread != 0"
$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -nm -q "select * from system.stack_trace where thread_id != 0 format Null" |& grep -F -o 'Send signal to' | grep -v 'Send signal to 0 threads (total)'
# there is no thread with comm="foo", so no signals will be sent
echo "thread_name = 'foo'"
$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -nm -q "select * from system.stack_trace where thread_name = 'foo' format Null" |& grep -F -o 'Send signal to 0 threads (total)'