mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-17 20:02:05 +00:00
Add ability to set any rocksdb option via config.xml
v2: Cover rocksdb options in ClickHouse config v3: add missing __init__.py v4: Rework rocksdb options from config v5: add column_family_options support
This commit is contained in:
parent
c7d2a5aad8
commit
8ef677b15f
@ -55,4 +55,30 @@ FROM system.rocksdb
|
||||
└───────────────────────────┴───────┘
|
||||
```
|
||||
|
||||
## Configuration
|
||||
|
||||
You can also change any [rocksdb options](https://github.com/facebook/rocksdb/wiki/Option-String-and-Option-Map) using config:
|
||||
|
||||
```xml
|
||||
<rocksdb>
|
||||
<options>
|
||||
<max_background_jobs>8</max_background_jobs>
|
||||
</options>
|
||||
<column_family_options>
|
||||
<num_levels>2</num_levels>
|
||||
</column_family_options>
|
||||
<tables>
|
||||
<table>
|
||||
<name>TABLE</name>
|
||||
<options>
|
||||
<max_background_jobs>8</max_background_jobs>
|
||||
</options>
|
||||
<column_family_options>
|
||||
<num_levels>2</num_levels>
|
||||
</column_family_options>
|
||||
</table>
|
||||
</tables>
|
||||
</rocksdb>
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/engines/table-engines/integrations/embedded-rocksdb/) <!--hide-->
|
||||
|
@ -1156,4 +1156,27 @@
|
||||
|
||||
<!-- Uncomment to disable ClickHouse internal DNS caching. -->
|
||||
<!-- <disable_internal_dns_cache>1</disable_internal_dns_cache> -->
|
||||
|
||||
<!-- You can also configure rocksdb like this: -->
|
||||
<!--
|
||||
<rocksdb>
|
||||
<options>
|
||||
<max_background_jobs>8</max_background_jobs>
|
||||
</options>
|
||||
<column_family_options>
|
||||
<num_levels>2</num_levels>
|
||||
</column_family_options>
|
||||
<tables>
|
||||
<table>
|
||||
<name>TABLE</name>
|
||||
<options>
|
||||
<max_background_jobs>8</max_background_jobs>
|
||||
</options>
|
||||
<column_family_options>
|
||||
<num_levels>2</num_levels>
|
||||
</column_family_options>
|
||||
</table>
|
||||
</tables>
|
||||
</rocksdb>
|
||||
-->
|
||||
</yandex>
|
||||
|
@ -28,10 +28,12 @@
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <rocksdb/db.h>
|
||||
#include <rocksdb/table.h>
|
||||
#include <rocksdb/convenience.h>
|
||||
|
||||
#include <filesystem>
|
||||
|
||||
@ -49,6 +51,24 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
using FieldVectorPtr = std::shared_ptr<FieldVector>;
|
||||
using RocksDBOptions = std::unordered_map<std::string, std::string>;
|
||||
|
||||
|
||||
static RocksDBOptions getOptionsFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & path)
|
||||
{
|
||||
RocksDBOptions options;
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(path, keys);
|
||||
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
const String key_path = path + "." + key;
|
||||
options[key] = config.getString(key_path);
|
||||
}
|
||||
|
||||
return options;
|
||||
}
|
||||
|
||||
|
||||
// returns keys may be filter by condition
|
||||
@ -250,7 +270,9 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_,
|
||||
bool attach,
|
||||
ContextPtr context_,
|
||||
const String & primary_key_)
|
||||
: IStorage(table_id_), primary_key{primary_key_}
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, primary_key{primary_key_}
|
||||
{
|
||||
setInMemoryMetadata(metadata_);
|
||||
rocksdb_dir = context_->getPath() + relative_data_path_;
|
||||
@ -271,19 +293,86 @@ void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr &
|
||||
|
||||
void StorageEmbeddedRocksDB::initDb()
|
||||
{
|
||||
rocksdb::Options options;
|
||||
rocksdb::Status status;
|
||||
rocksdb::Options base;
|
||||
rocksdb::DB * db;
|
||||
options.create_if_missing = true;
|
||||
options.compression = rocksdb::CompressionType::kZSTD;
|
||||
options.statistics = rocksdb::CreateDBStatistics();
|
||||
|
||||
base.create_if_missing = true;
|
||||
base.compression = rocksdb::CompressionType::kZSTD;
|
||||
base.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;
|
||||
base.info_log_level = rocksdb::ERROR_LEVEL;
|
||||
|
||||
rocksdb::Status status = rocksdb::DB::Open(options, rocksdb_dir, &db);
|
||||
rocksdb::Options merged = base;
|
||||
|
||||
if (status != rocksdb::Status::OK())
|
||||
throw Exception("Fail to open rocksdb path at: " + rocksdb_dir + ": " + status.ToString(), ErrorCodes::ROCKSDB_ERROR);
|
||||
const auto & config = getContext()->getConfigRef();
|
||||
if (config.has("rocksdb.options"))
|
||||
{
|
||||
auto config_options = getOptionsFromConfig(config, "rocksdb.options");
|
||||
status = rocksdb::GetDBOptionsFromMap(merged, config_options, &merged);
|
||||
if (!status.ok())
|
||||
{
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.options' at: {}: {}",
|
||||
rocksdb_dir, status.ToString());
|
||||
}
|
||||
}
|
||||
if (config.has("rocksdb.column_family_options"))
|
||||
{
|
||||
auto column_family_options = getOptionsFromConfig(config, "rocksdb.column_family_options");
|
||||
status = rocksdb::GetColumnFamilyOptionsFromMap(merged, column_family_options, &merged);
|
||||
if (!status.ok())
|
||||
{
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.options' at: {}: {}",
|
||||
rocksdb_dir, status.ToString());
|
||||
}
|
||||
}
|
||||
|
||||
if (config.has("rocksdb.tables"))
|
||||
{
|
||||
auto table_name = getStorageID().getTableName();
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys("rocksdb.tables", keys);
|
||||
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
const String key_prefix = "rocksdb.tables." + key;
|
||||
if (config.getString(key_prefix + ".name") != table_name)
|
||||
continue;
|
||||
|
||||
String config_key = key_prefix + ".options";
|
||||
if (config.has(config_key))
|
||||
{
|
||||
auto table_config_options = getOptionsFromConfig(config, config_key);
|
||||
status = rocksdb::GetDBOptionsFromMap(merged, table_config_options, &merged);
|
||||
if (!status.ok())
|
||||
{
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from '{}' at: {}: {}",
|
||||
config_key, rocksdb_dir, status.ToString());
|
||||
}
|
||||
}
|
||||
|
||||
config_key = key_prefix + ".column_family_options";
|
||||
if (config.has(config_key))
|
||||
{
|
||||
auto table_column_family_options = getOptionsFromConfig(config, config_key);
|
||||
status = rocksdb::GetColumnFamilyOptionsFromMap(merged, table_column_family_options, &merged);
|
||||
if (!status.ok())
|
||||
{
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from '{}' at: {}: {}",
|
||||
config_key, rocksdb_dir, status.ToString());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
status = rocksdb::DB::Open(merged, rocksdb_dir, &db);
|
||||
|
||||
if (!status.ok())
|
||||
{
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to open rocksdb path at: {}: {}",
|
||||
rocksdb_dir, status.ToString());
|
||||
}
|
||||
rocksdb_ptr = std::unique_ptr<rocksdb::DB>(db);
|
||||
}
|
||||
|
||||
|
@ -17,7 +17,7 @@ namespace DB
|
||||
|
||||
class Context;
|
||||
|
||||
class StorageEmbeddedRocksDB final : public shared_ptr_helper<StorageEmbeddedRocksDB>, public IStorage
|
||||
class StorageEmbeddedRocksDB final : public shared_ptr_helper<StorageEmbeddedRocksDB>, public IStorage, WithContext
|
||||
{
|
||||
friend struct shared_ptr_helper<StorageEmbeddedRocksDB>;
|
||||
friend class EmbeddedRocksDBSource;
|
||||
|
0
tests/integration/test_rocksdb_options/__init__.py
Normal file
0
tests/integration/test_rocksdb_options/__init__.py
Normal file
13
tests/integration/test_rocksdb_options/configs/rocksdb.yml
Normal file
13
tests/integration/test_rocksdb_options/configs/rocksdb.yml
Normal file
@ -0,0 +1,13 @@
|
||||
---
|
||||
rocksdb:
|
||||
options:
|
||||
max_background_jobs: 8
|
||||
column_family_options:
|
||||
num_levels: 2
|
||||
tables:
|
||||
- table:
|
||||
name: test
|
||||
options:
|
||||
max_open_files: 10000
|
||||
column_family_options:
|
||||
max_bytes_for_level_base: 14
|
85
tests/integration/test_rocksdb_options/test.py
Normal file
85
tests/integration/test_rocksdb_options/test.py
Normal file
@ -0,0 +1,85 @@
|
||||
# pylint: disable=unused-argument
|
||||
# pylint: disable=redefined-outer-name
|
||||
# pylint: disable=line-too-long
|
||||
|
||||
import pytest
|
||||
|
||||
from helpers.client import QueryRuntimeException
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node = cluster.add_instance('node', main_configs=['configs/rocksdb.yml'], stay_alive=True)
|
||||
|
||||
|
||||
@pytest.fixture(scope='module', autouse=True)
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
def test_valid_options():
|
||||
node.query("""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
DROP TABLE test;
|
||||
""")
|
||||
|
||||
def test_invalid_options():
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/max_background_jobs/no_such_option/' /etc/clickhouse-server/config.d/rocksdb.yml"])
|
||||
node.restart_clickhouse()
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node.query("""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
""")
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/no_such_option/max_background_jobs/' /etc/clickhouse-server/config.d/rocksdb.yml"])
|
||||
node.restart_clickhouse()
|
||||
|
||||
def test_table_valid_options():
|
||||
node.query("""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
DROP TABLE test;
|
||||
""")
|
||||
|
||||
def test_table_invalid_options():
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/max_open_files/no_such_table_option/' /etc/clickhouse-server/config.d/rocksdb.yml"])
|
||||
node.restart_clickhouse()
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node.query("""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
""")
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/no_such_table_option/max_open_files/' /etc/clickhouse-server/config.d/rocksdb.yml"])
|
||||
node.restart_clickhouse()
|
||||
|
||||
def test_valid_column_family_options():
|
||||
node.query("""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
DROP TABLE test;
|
||||
""")
|
||||
|
||||
def test_invalid_column_family_options():
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/num_levels/no_such_column_family_option/' /etc/clickhouse-server/config.d/rocksdb.yml"])
|
||||
node.restart_clickhouse()
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node.query("""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
""")
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/no_such_column_family_option/num_levels/' /etc/clickhouse-server/config.d/rocksdb.yml"])
|
||||
node.restart_clickhouse()
|
||||
|
||||
def test_table_valid_column_family_options():
|
||||
node.query("""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
DROP TABLE test;
|
||||
""")
|
||||
|
||||
def test_table_invalid_column_family_options():
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/max_bytes_for_level_base/no_such_table_column_family_option/' /etc/clickhouse-server/config.d/rocksdb.yml"])
|
||||
node.restart_clickhouse()
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node.query("""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
""")
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/no_such_table_column_family_option/max_bytes_for_level_base/' /etc/clickhouse-server/config.d/rocksdb.yml"])
|
||||
node.restart_clickhouse()
|
Loading…
Reference in New Issue
Block a user