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:
Azat Khuzhin 2021-07-26 21:17:39 +03:00
parent 9282e0f5b7
commit c7d2a5aad8
8 changed files with 197 additions and 1 deletions

View File

@ -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-->

View File

@ -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)
{

View File

@ -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_,

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

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

View File

@ -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)

View File

@ -1,3 +1,4 @@
10000
123 Hello, world (123)
--
--

View File

@ -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 '--';