mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge pull request #56548 from ClickHouse/system-symbols-pick
Add a table `system.symbols`
This commit is contained in:
commit
0a1078bb37
35
docs/en/operations/system-tables/symbols.md
Normal file
35
docs/en/operations/system-tables/symbols.md
Normal file
@ -0,0 +1,35 @@
|
|||||||
|
---
|
||||||
|
slug: /en/operations/system-tables/symbols
|
||||||
|
---
|
||||||
|
# symbols
|
||||||
|
|
||||||
|
Contains information for introspection of `clickhouse` binary. It requires the introspection privilege to access.
|
||||||
|
This table is only useful for C++ experts and ClickHouse engineers.
|
||||||
|
|
||||||
|
Columns:
|
||||||
|
|
||||||
|
- `symbol` ([String](../../sql-reference/data-types/string.md)) — Symbol name in the binary. It is mangled. You can apply `demangle(symbol)` to obtain a readable name.
|
||||||
|
- `address_begin` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Start address of the symbol in the binary.
|
||||||
|
- `address_end` ([UInt64](../../sql-reference/data-types/int-uint.md)) — End address of the symbol in the binary.
|
||||||
|
- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for `event`.
|
||||||
|
|
||||||
|
**Example**
|
||||||
|
|
||||||
|
``` sql
|
||||||
|
SELECT address_begin, address_end - address_begin AS size, demangle(symbol) FROM system.symbols ORDER BY size DESC LIMIT 10
|
||||||
|
```
|
||||||
|
|
||||||
|
``` text
|
||||||
|
┌─address_begin─┬─────size─┬─demangle(symbol)──────────────────────────────────────────────────────────────────┐
|
||||||
|
│ 25000976 │ 29466000 │ icudt70_dat │
|
||||||
|
│ 400605288 │ 2097272 │ arena_emap_global │
|
||||||
|
│ 18760592 │ 1048576 │ CLD2::kQuadChrome1015_2 │
|
||||||
|
│ 9807152 │ 884808 │ TopLevelDomainLookupHash::isValid(char const*, unsigned long)::wordlist │
|
||||||
|
│ 57442432 │ 850608 │ llvm::X86Insts │
|
||||||
|
│ 55682944 │ 681360 │ (anonymous namespace)::X86DAGToDAGISel::SelectCode(llvm::SDNode*)::MatcherTable │
|
||||||
|
│ 55130368 │ 502840 │ (anonymous namespace)::X86InstructionSelector::getMatchTable() const::MatchTable0 │
|
||||||
|
│ 402930616 │ 404032 │ qpl::ml::dispatcher::hw_dispatcher::get_instance()::instance │
|
||||||
|
│ 274131872 │ 356795 │ DB::SettingsTraits::Accessor::instance()::$_0::operator()() const │
|
||||||
|
│ 58293040 │ 249424 │ llvm::X86InstrNameData │
|
||||||
|
└───────────────┴──────────┴───────────────────────────────────────────────────────────────────────────────────┘
|
||||||
|
```
|
114
src/Storages/System/StorageSystemSymbols.cpp
Normal file
114
src/Storages/System/StorageSystemSymbols.cpp
Normal file
@ -0,0 +1,114 @@
|
|||||||
|
#if defined(__ELF__) && !defined(OS_FREEBSD)
|
||||||
|
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <Storages/System/StorageSystemSymbols.h>
|
||||||
|
#include <Storages/System/getQueriedColumnsMaskAndHeader.h>
|
||||||
|
#include <Access/ContextAccess.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <Processors/ISource.h>
|
||||||
|
#include <QueryPipeline/Pipe.h>
|
||||||
|
#include <Common/SymbolIndex.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
StorageSystemSymbols::StorageSystemSymbols(const StorageID & table_id_)
|
||||||
|
: IStorage(table_id_)
|
||||||
|
{
|
||||||
|
StorageInMemoryMetadata storage_metadata;
|
||||||
|
storage_metadata.setColumns(ColumnsDescription(
|
||||||
|
{
|
||||||
|
{"symbol", std::make_shared<DataTypeString>()},
|
||||||
|
{"address_begin", std::make_shared<DataTypeUInt64>()},
|
||||||
|
{"address_end", std::make_shared<DataTypeUInt64>()},
|
||||||
|
}));
|
||||||
|
setInMemoryMetadata(storage_metadata);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
class SymbolsBlockSource : public ISource
|
||||||
|
{
|
||||||
|
private:
|
||||||
|
using Iterator = std::vector<SymbolIndex::Symbol>::const_iterator;
|
||||||
|
Iterator it;
|
||||||
|
const Iterator end;
|
||||||
|
std::vector<UInt8> columns_mask;
|
||||||
|
UInt64 max_block_size;
|
||||||
|
|
||||||
|
public:
|
||||||
|
SymbolsBlockSource(
|
||||||
|
Iterator begin_,
|
||||||
|
Iterator end_,
|
||||||
|
std::vector<UInt8> columns_mask_,
|
||||||
|
Block header,
|
||||||
|
UInt64 max_block_size_)
|
||||||
|
: ISource(std::move(header))
|
||||||
|
, it(begin_), end(end_), columns_mask(std::move(columns_mask_)), max_block_size(max_block_size_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
String getName() const override { return "Symbols"; }
|
||||||
|
|
||||||
|
protected:
|
||||||
|
Chunk generate() override
|
||||||
|
{
|
||||||
|
if (it == end)
|
||||||
|
return {};
|
||||||
|
|
||||||
|
MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns();
|
||||||
|
|
||||||
|
size_t rows_count = 0;
|
||||||
|
while (rows_count < max_block_size && it != end)
|
||||||
|
{
|
||||||
|
size_t src_index = 0;
|
||||||
|
size_t res_index = 0;
|
||||||
|
|
||||||
|
if (columns_mask[src_index++])
|
||||||
|
res_columns[res_index++]->insert(it->name);
|
||||||
|
if (columns_mask[src_index++])
|
||||||
|
res_columns[res_index++]->insert(reinterpret_cast<uintptr_t>(it->address_begin));
|
||||||
|
if (columns_mask[src_index++])
|
||||||
|
res_columns[res_index++]->insert(reinterpret_cast<uintptr_t>(it->address_end));
|
||||||
|
|
||||||
|
++rows_count;
|
||||||
|
++it;
|
||||||
|
}
|
||||||
|
|
||||||
|
return Chunk(std::move(res_columns), rows_count);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
Pipe StorageSystemSymbols::read(
|
||||||
|
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*/)
|
||||||
|
{
|
||||||
|
context->getAccess()->checkAccess(AccessType::INTROSPECTION);
|
||||||
|
|
||||||
|
storage_snapshot->check(column_names);
|
||||||
|
Block sample_block = storage_snapshot->metadata->getSampleBlock();
|
||||||
|
auto [columns_mask, res_block] = getQueriedColumnsMaskAndHeader(sample_block, column_names);
|
||||||
|
|
||||||
|
const auto & symbols = SymbolIndex::instance().symbols();
|
||||||
|
|
||||||
|
return Pipe(std::make_shared<SymbolsBlockSource>(
|
||||||
|
symbols.cbegin(), symbols.cend(), std::move(columns_mask), std::move(res_block), max_block_size));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
#endif
|
33
src/Storages/System/StorageSystemSymbols.h
Normal file
33
src/Storages/System/StorageSystemSymbols.h
Normal file
@ -0,0 +1,33 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Storages/IStorage.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class Context;
|
||||||
|
|
||||||
|
|
||||||
|
/** Implements the system table `symbols` for introspection of symbols in the ClickHouse binary.
|
||||||
|
*/
|
||||||
|
class StorageSystemSymbols final : public IStorage
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit StorageSystemSymbols(const StorageID & table_id_);
|
||||||
|
|
||||||
|
std::string getName() const override { return "SystemSymbols"; }
|
||||||
|
|
||||||
|
Pipe read(
|
||||||
|
const Names & column_names,
|
||||||
|
const StorageSnapshotPtr & storage_snapshot,
|
||||||
|
SelectQueryInfo & query_info,
|
||||||
|
ContextPtr context,
|
||||||
|
QueryProcessingStage::Enum processed_stage,
|
||||||
|
size_t max_block_size,
|
||||||
|
size_t num_streams) override;
|
||||||
|
|
||||||
|
bool isSystemStorage() const override { return true; }
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -86,6 +86,10 @@
|
|||||||
#include <Storages/System/StorageSystemScheduler.h>
|
#include <Storages/System/StorageSystemScheduler.h>
|
||||||
#include <Storages/System/StorageSystemS3Queue.h>
|
#include <Storages/System/StorageSystemS3Queue.h>
|
||||||
|
|
||||||
|
#if defined(__ELF__) && !defined(OS_FREEBSD)
|
||||||
|
#include <Storages/System/StorageSystemSymbols.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_RDKAFKA
|
#if USE_RDKAFKA
|
||||||
#include <Storages/System/StorageSystemKafkaConsumers.h>
|
#include <Storages/System/StorageSystemKafkaConsumers.h>
|
||||||
#endif
|
#endif
|
||||||
@ -151,6 +155,9 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database)
|
|||||||
attach<StorageSystemSchemaInferenceCache>(context, system_database, "schema_inference_cache");
|
attach<StorageSystemSchemaInferenceCache>(context, system_database, "schema_inference_cache");
|
||||||
attach<StorageSystemDroppedTables>(context, system_database, "dropped_tables");
|
attach<StorageSystemDroppedTables>(context, system_database, "dropped_tables");
|
||||||
attach<StorageSystemScheduler>(context, system_database, "scheduler");
|
attach<StorageSystemScheduler>(context, system_database, "scheduler");
|
||||||
|
#if defined(__ELF__) && !defined(OS_FREEBSD)
|
||||||
|
attach<StorageSystemSymbols>(context, system_database, "symbols");
|
||||||
|
#endif
|
||||||
#if USE_RDKAFKA
|
#if USE_RDKAFKA
|
||||||
attach<StorageSystemKafkaConsumers>(context, system_database, "kafka_consumers");
|
attach<StorageSystemKafkaConsumers>(context, system_database, "kafka_consumers");
|
||||||
#endif
|
#endif
|
||||||
|
@ -23,7 +23,7 @@ function run_selects()
|
|||||||
|
|
||||||
for t in "${tables_arr[@]}"
|
for t in "${tables_arr[@]}"
|
||||||
do
|
do
|
||||||
${CLICKHOUSE_CLIENT} -q "SELECT * FROM $t LIMIT $LIMIT FORMAT Null" # Suppress style check: database=$CLICKHOUSE_DATABASEs
|
${CLICKHOUSE_CLIENT} -q "SELECT * FROM $t LIMIT $LIMIT SETTINGS allow_introspection_functions = 1 FORMAT Null" # Suppress style check: database=$CLICKHOUSE_DATABASEs
|
||||||
done
|
done
|
||||||
}
|
}
|
||||||
|
|
||||||
|
1
tests/queries/0_stateless/02911_system_symbols.reference
Normal file
1
tests/queries/0_stateless/02911_system_symbols.reference
Normal file
@ -0,0 +1 @@
|
|||||||
|
DB::StorageSystemSymbols::StorageSystemSymbols(DB::StorageID const&)
|
1
tests/queries/0_stateless/02911_system_symbols.sql
Normal file
1
tests/queries/0_stateless/02911_system_symbols.sql
Normal file
@ -0,0 +1 @@
|
|||||||
|
SELECT x FROM (SELECT demangle(symbol) AS x FROM system.symbols WHERE symbol LIKE '%StorageSystemSymbols%') WHERE x LIKE '%DB::StorageSystemSymbols::StorageSystemSymbols%' ORDER BY x LIMIT 1 SETTINGS allow_introspection_functions = 1;
|
Loading…
Reference in New Issue
Block a user