Merge pull request #56548 from ClickHouse/system-symbols-pick

Add a table `system.symbols`
This commit is contained in:
Alexander Gololobov 2023-11-10 10:37:11 +01:00 committed by GitHub
commit 0a1078bb37
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 192 additions and 1 deletions

View 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 │
└───────────────┴──────────┴───────────────────────────────────────────────────────────────────────────────────┘
```

View 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

View 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; }
};
}

View File

@ -86,6 +86,10 @@
#include <Storages/System/StorageSystemScheduler.h>
#include <Storages/System/StorageSystemS3Queue.h>
#if defined(__ELF__) && !defined(OS_FREEBSD)
#include <Storages/System/StorageSystemSymbols.h>
#endif
#if USE_RDKAFKA
#include <Storages/System/StorageSystemKafkaConsumers.h>
#endif
@ -151,6 +155,9 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database)
attach<StorageSystemSchemaInferenceCache>(context, system_database, "schema_inference_cache");
attach<StorageSystemDroppedTables>(context, system_database, "dropped_tables");
attach<StorageSystemScheduler>(context, system_database, "scheduler");
#if defined(__ELF__) && !defined(OS_FREEBSD)
attach<StorageSystemSymbols>(context, system_database, "symbols");
#endif
#if USE_RDKAFKA
attach<StorageSystemKafkaConsumers>(context, system_database, "kafka_consumers");
#endif

View File

@ -23,7 +23,7 @@ function run_selects()
for t in "${tables_arr[@]}"
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
}

View File

@ -0,0 +1 @@
DB::StorageSystemSymbols::StorageSystemSymbols(DB::StorageID const&)

View 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;