mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Add system.rocksdb table (with various rocksdb internals)
v2: fix USE_ROCKSDB check (there is #cmakedefine01 so it is either 0/1, so we should not check with #ifdef)
This commit is contained in:
parent
9282e0f5b7
commit
c7d2a5aad8
@ -39,4 +39,20 @@ ENGINE = EmbeddedRocksDB
|
||||
PRIMARY KEY key
|
||||
```
|
||||
|
||||
## Metrics
|
||||
|
||||
There is also `system.rocksdb` table, that expose rocksdb statistics:
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
name,
|
||||
value
|
||||
FROM system.rocksdb
|
||||
|
||||
┌─name──────────────────────┬─value─┐
|
||||
│ no.file.opens │ 1 │
|
||||
│ number.block.decompressed │ 1 │
|
||||
└───────────────────────────┴───────┘
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/engines/table-engines/integrations/embedded-rocksdb/) <!--hide-->
|
||||
|
@ -275,6 +275,7 @@ void StorageEmbeddedRocksDB::initDb()
|
||||
rocksdb::DB * db;
|
||||
options.create_if_missing = true;
|
||||
options.compression = rocksdb::CompressionType::kZSTD;
|
||||
options.statistics = rocksdb::CreateDBStatistics();
|
||||
|
||||
/// It is too verbose by default, and in fact we don't care about rocksdb logs at all.
|
||||
options.info_log_level = rocksdb::ERROR_LEVEL;
|
||||
@ -368,6 +369,10 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
return StorageEmbeddedRocksDB::create(args.table_id, args.relative_data_path, metadata, args.attach, args.getContext(), primary_key_names[0]);
|
||||
}
|
||||
|
||||
std::shared_ptr<rocksdb::Statistics> StorageEmbeddedRocksDB::getRocksDBStatistics() const
|
||||
{
|
||||
return rocksdb_ptr->GetOptions().statistics;
|
||||
}
|
||||
|
||||
void registerStorageEmbeddedRocksDB(StorageFactory & factory)
|
||||
{
|
||||
|
@ -8,6 +8,7 @@
|
||||
namespace rocksdb
|
||||
{
|
||||
class DB;
|
||||
class Statistics;
|
||||
}
|
||||
|
||||
|
||||
@ -48,6 +49,8 @@ public:
|
||||
bool storesDataOnDisk() const override { return true; }
|
||||
Strings getDataPaths() const override { return {rocksdb_dir}; }
|
||||
|
||||
std::shared_ptr<rocksdb::Statistics> getRocksDBStatistics() const;
|
||||
|
||||
protected:
|
||||
StorageEmbeddedRocksDB(const StorageID & table_id_,
|
||||
const String & relative_data_path_,
|
||||
|
129
src/Storages/RocksDB/StorageSystemRocksDB.cpp
Normal file
129
src/Storages/RocksDB/StorageSystemRocksDB.cpp
Normal file
@ -0,0 +1,129 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Storages/RocksDB/StorageSystemRocksDB.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/RocksDB/StorageEmbeddedRocksDB.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <rocksdb/statistics.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
NamesAndTypesList StorageSystemRocksDB::getNamesAndTypes()
|
||||
{
|
||||
return {
|
||||
{ "database", std::make_shared<DataTypeString>() },
|
||||
{ "table", std::make_shared<DataTypeString>() },
|
||||
{ "name", std::make_shared<DataTypeString>() },
|
||||
{ "value", std::make_shared<DataTypeUInt64>() },
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const
|
||||
{
|
||||
const auto access = context->getAccess();
|
||||
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
|
||||
|
||||
std::map<String, std::map<String, StoragePtr>> tables;
|
||||
for (const auto & db : DatabaseCatalog::instance().getDatabases())
|
||||
{
|
||||
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first);
|
||||
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
StoragePtr table = iterator->table();
|
||||
if (!table)
|
||||
continue;
|
||||
|
||||
if (!dynamic_cast<const StorageEmbeddedRocksDB *>(table.get()))
|
||||
continue;
|
||||
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name()))
|
||||
continue;
|
||||
tables[db.first][iterator->name()] = table;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr col_database_mut = ColumnString::create();
|
||||
MutableColumnPtr col_table_mut = ColumnString::create();
|
||||
|
||||
for (auto & db : tables)
|
||||
{
|
||||
for (auto & table : db.second)
|
||||
{
|
||||
col_database_mut->insert(db.first);
|
||||
col_table_mut->insert(table.first);
|
||||
}
|
||||
}
|
||||
|
||||
ColumnPtr col_database_to_filter = std::move(col_database_mut);
|
||||
ColumnPtr col_table_to_filter = std::move(col_table_mut);
|
||||
|
||||
/// Determine what tables are needed by the conditions in the query.
|
||||
{
|
||||
Block filtered_block
|
||||
{
|
||||
{ col_database_to_filter, std::make_shared<DataTypeString>(), "database" },
|
||||
{ col_table_to_filter, std::make_shared<DataTypeString>(), "table" },
|
||||
};
|
||||
|
||||
VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context);
|
||||
|
||||
if (!filtered_block.rows())
|
||||
return;
|
||||
|
||||
col_database_to_filter = filtered_block.getByName("database").column;
|
||||
col_table_to_filter = filtered_block.getByName("table").column;
|
||||
}
|
||||
|
||||
bool show_zeros = context->getSettingsRef().system_events_show_zero_values;
|
||||
for (size_t i = 0, tables_size = col_database_to_filter->size(); i < tables_size; ++i)
|
||||
{
|
||||
String database = (*col_database_to_filter)[i].safeGet<const String &>();
|
||||
String table = (*col_table_to_filter)[i].safeGet<const String &>();
|
||||
|
||||
auto & rocksdb_table = dynamic_cast<StorageEmbeddedRocksDB &>(*tables[database][table]);
|
||||
auto statistics = rocksdb_table.getRocksDBStatistics();
|
||||
if (!statistics)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "rocksdb statistics is not enabled");
|
||||
|
||||
for (auto [tick, name] : rocksdb::TickersNameMap)
|
||||
{
|
||||
UInt64 value = statistics->getTickerCount(tick);
|
||||
if (!value && !show_zeros)
|
||||
continue;
|
||||
|
||||
/// trim "rocksdb."
|
||||
if (startsWith(name, "rocksdb."))
|
||||
name = name.substr(strlen("rocksdb."));
|
||||
|
||||
size_t col_num = 0;
|
||||
res_columns[col_num++]->insert(database);
|
||||
res_columns[col_num++]->insert(table);
|
||||
|
||||
res_columns[col_num++]->insert(name);
|
||||
res_columns[col_num++]->insert(value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
29
src/Storages/RocksDB/StorageSystemRocksDB.h
Normal file
29
src/Storages/RocksDB/StorageSystemRocksDB.h
Normal file
@ -0,0 +1,29 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/shared_ptr_helper.h>
|
||||
#include <Storages/System/IStorageSystemOneBlock.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
|
||||
/** Implements the `rocksdb` system table, which expose various rocksdb metrics.
|
||||
*/
|
||||
class StorageSystemRocksDB final : public shared_ptr_helper<StorageSystemRocksDB>, public IStorageSystemOneBlock<StorageSystemRocksDB>
|
||||
{
|
||||
friend struct shared_ptr_helper<StorageSystemRocksDB>;
|
||||
public:
|
||||
std::string getName() const override { return "SystemRocksDB"; }
|
||||
|
||||
static NamesAndTypesList getNamesAndTypes();
|
||||
|
||||
protected:
|
||||
using IStorageSystemOneBlock::IStorageSystemOneBlock;
|
||||
|
||||
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;
|
||||
};
|
||||
|
||||
}
|
@ -1,3 +1,7 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
#include <Storages/System/attachSystemTablesImpl.h>
|
||||
@ -74,6 +78,10 @@
|
||||
#include <Storages/System/StorageSystemStackTrace.h>
|
||||
#endif
|
||||
|
||||
#if USE_ROCKSDB
|
||||
#include <Storages/RocksDB/StorageSystemRocksDB.h>
|
||||
#endif
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -126,6 +134,9 @@ void attachSystemTablesLocal(IDatabase & system_database)
|
||||
#ifdef OS_LINUX
|
||||
attach<StorageSystemStackTrace>(system_database, "stack_trace");
|
||||
#endif
|
||||
#if USE_ROCKSDB
|
||||
attach<StorageSystemRocksDB>(system_database, "rocksdb");
|
||||
#endif
|
||||
}
|
||||
|
||||
void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper)
|
||||
|
@ -1,3 +1,4 @@
|
||||
10000
|
||||
123 Hello, world (123)
|
||||
--
|
||||
--
|
||||
|
@ -2,7 +2,9 @@ DROP TABLE IF EXISTS 01686_test;
|
||||
|
||||
CREATE TABLE 01686_test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
|
||||
SELECT value FROM system.rocksdb WHERE database = currentDatabase() and table = '01686_test' and name = 'number.keys.written';
|
||||
INSERT INTO 01686_test SELECT number, format('Hello, world ({})', toString(number)) FROM numbers(10000);
|
||||
SELECT value FROM system.rocksdb WHERE database = currentDatabase() and table = '01686_test' and name = 'number.keys.written';
|
||||
|
||||
SELECT * FROM 01686_test WHERE key = 123;
|
||||
SELECT '--';
|
||||
|
Loading…
Reference in New Issue
Block a user