mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #19526 from taiyang-li/disk_reload
Enable local disk config reload
This commit is contained in:
commit
98326ec028
@ -31,6 +31,56 @@ std::mutex DiskLocal::reservation_mutex;
|
||||
|
||||
using DiskLocalPtr = std::shared_ptr<DiskLocal>;
|
||||
|
||||
static void loadDiskLocalConfig(const String & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
ContextPtr context,
|
||||
String & path,
|
||||
UInt64 & keep_free_space_bytes)
|
||||
{
|
||||
path = config.getString(config_prefix + ".path", "");
|
||||
if (name == "default")
|
||||
{
|
||||
if (!path.empty())
|
||||
throw Exception(
|
||||
"\"default\" disk path should be provided in <path> not it <storage_configuration>",
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
path = context->getPath();
|
||||
}
|
||||
else
|
||||
{
|
||||
if (path.empty())
|
||||
throw Exception("Disk path can not be empty. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
if (path.back() != '/')
|
||||
throw Exception("Disk path must end with /. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
}
|
||||
|
||||
if (!FS::canRead(path) || !FS::canWrite(path))
|
||||
throw Exception("There is no RW access to the disk " + name + " (" + path + ")", ErrorCodes::PATH_ACCESS_DENIED);
|
||||
|
||||
bool has_space_ratio = config.has(config_prefix + ".keep_free_space_ratio");
|
||||
|
||||
if (config.has(config_prefix + ".keep_free_space_bytes") && has_space_ratio)
|
||||
throw Exception(
|
||||
"Only one of 'keep_free_space_bytes' and 'keep_free_space_ratio' can be specified",
|
||||
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
|
||||
keep_free_space_bytes = config.getUInt64(config_prefix + ".keep_free_space_bytes", 0);
|
||||
|
||||
if (has_space_ratio)
|
||||
{
|
||||
auto ratio = config.getDouble(config_prefix + ".keep_free_space_ratio");
|
||||
if (ratio < 0 || ratio > 1)
|
||||
throw Exception("'keep_free_space_ratio' have to be between 0 and 1", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
String tmp_path = path;
|
||||
if (tmp_path.empty())
|
||||
tmp_path = context->getPath();
|
||||
|
||||
// Create tmp disk for getting total disk space.
|
||||
keep_free_space_bytes = static_cast<UInt64>(DiskLocal("tmp", tmp_path, 0).getTotalSpace() * ratio);
|
||||
}
|
||||
}
|
||||
|
||||
class DiskLocalReservation : public IReservation
|
||||
{
|
||||
public:
|
||||
@ -317,6 +367,21 @@ SyncGuardPtr DiskLocal::getDirectorySyncGuard(const String & path) const
|
||||
return std::make_unique<LocalDirectorySyncGuard>(fs::path(disk_path) / path);
|
||||
}
|
||||
|
||||
|
||||
void DiskLocal::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap &)
|
||||
{
|
||||
String new_disk_path;
|
||||
UInt64 new_keep_free_space_bytes;
|
||||
|
||||
loadDiskLocalConfig(name, config, config_prefix, context, new_disk_path, new_keep_free_space_bytes);
|
||||
|
||||
if (disk_path != new_disk_path)
|
||||
throw Exception("Disk path can't be updated from config " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
if (keep_free_space_bytes != new_keep_free_space_bytes)
|
||||
keep_free_space_bytes = new_keep_free_space_bytes;
|
||||
}
|
||||
|
||||
DiskPtr DiskLocalReservation::getDisk(size_t i) const
|
||||
{
|
||||
if (i != 0)
|
||||
@ -334,7 +399,6 @@ void DiskLocalReservation::update(UInt64 new_size)
|
||||
disk->reserved_bytes += size;
|
||||
}
|
||||
|
||||
|
||||
DiskLocalReservation::~DiskLocalReservation()
|
||||
{
|
||||
try
|
||||
@ -369,48 +433,9 @@ void registerDiskLocal(DiskFactory & factory)
|
||||
const String & config_prefix,
|
||||
ContextPtr context,
|
||||
const DisksMap & /*map*/) -> DiskPtr {
|
||||
String path = config.getString(config_prefix + ".path", "");
|
||||
if (name == "default")
|
||||
{
|
||||
if (!path.empty())
|
||||
throw Exception(
|
||||
"\"default\" disk path should be provided in <path> not it <storage_configuration>",
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
path = context->getPath();
|
||||
}
|
||||
else
|
||||
{
|
||||
if (path.empty())
|
||||
throw Exception("Disk path can not be empty. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
if (path.back() != '/')
|
||||
throw Exception("Disk path must end with /. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
}
|
||||
|
||||
if (!FS::canRead(path) || !FS::canWrite(path))
|
||||
throw Exception("There is no RW access to the disk " + name + " (" + path + ")", ErrorCodes::PATH_ACCESS_DENIED);
|
||||
|
||||
bool has_space_ratio = config.has(config_prefix + ".keep_free_space_ratio");
|
||||
|
||||
if (config.has(config_prefix + ".keep_free_space_bytes") && has_space_ratio)
|
||||
throw Exception(
|
||||
"Only one of 'keep_free_space_bytes' and 'keep_free_space_ratio' can be specified",
|
||||
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
|
||||
UInt64 keep_free_space_bytes = config.getUInt64(config_prefix + ".keep_free_space_bytes", 0);
|
||||
|
||||
if (has_space_ratio)
|
||||
{
|
||||
auto ratio = config.getDouble(config_prefix + ".keep_free_space_ratio");
|
||||
if (ratio < 0 || ratio > 1)
|
||||
throw Exception("'keep_free_space_ratio' have to be between 0 and 1", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
String tmp_path = path;
|
||||
if (tmp_path.empty())
|
||||
tmp_path = context->getPath();
|
||||
|
||||
// Create tmp disk for getting total disk space.
|
||||
keep_free_space_bytes = static_cast<UInt64>(DiskLocal("tmp", tmp_path, 0).getTotalSpace() * ratio);
|
||||
}
|
||||
|
||||
String path;
|
||||
UInt64 keep_free_space_bytes;
|
||||
loadDiskLocalConfig(name, config, config_prefix, context, path, keep_free_space_bytes);
|
||||
return std::make_shared<DiskLocal>(name, path, keep_free_space_bytes);
|
||||
};
|
||||
factory.registerDiskType("local", creator);
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -104,13 +105,15 @@ public:
|
||||
|
||||
SyncGuardPtr getDirectorySyncGuard(const String & path) const override;
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap &) override;
|
||||
|
||||
private:
|
||||
bool tryReserve(UInt64 bytes);
|
||||
|
||||
private:
|
||||
const String name;
|
||||
const String disk_path;
|
||||
const UInt64 keep_free_space_bytes;
|
||||
std::atomic<UInt64> keep_free_space_bytes;
|
||||
|
||||
UInt64 reserved_bytes = 0;
|
||||
UInt64 reservation_count = 0;
|
||||
@ -120,4 +123,5 @@ private:
|
||||
Poco::Logger * log = &Poco::Logger::get("DiskLocal");
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
|
||||
/// Get all disks with names
|
||||
const DisksMap & getDisksMap() const { return disks; }
|
||||
void addToDiskMap(String name, DiskPtr disk)
|
||||
void addToDiskMap(const String & name, DiskPtr disk)
|
||||
{
|
||||
disks.emplace(name, disk);
|
||||
}
|
||||
|
@ -13,9 +13,9 @@
|
||||
#include <mutex>
|
||||
#include <utility>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include "Poco/Util/AbstractConfiguration.h"
|
||||
#include <Poco/Timestamp.h>
|
||||
#include <filesystem>
|
||||
#include "Poco/Util/AbstractConfiguration.h"
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
|
@ -7,8 +7,10 @@ import xml.etree.ElementTree as ET
|
||||
|
||||
import helpers.client
|
||||
import helpers.cluster
|
||||
from helpers.test_tools import TSV
|
||||
import pytest
|
||||
|
||||
|
||||
cluster = helpers.cluster.ClickHouseCluster(__file__)
|
||||
|
||||
node1 = cluster.add_instance('node1',
|
||||
@ -76,6 +78,37 @@ def add_disk(node, name, path, separate_file=False):
|
||||
else:
|
||||
tree.write(os.path.join(node.config_d_dir, "storage_configuration.xml"))
|
||||
|
||||
def update_disk(node, name, path, keep_free_space_bytes, separate_file=False):
|
||||
separate_configuration_path = os.path.join(node.config_d_dir,
|
||||
"separate_configuration.xml")
|
||||
|
||||
try:
|
||||
if separate_file:
|
||||
tree = ET.parse(separate_configuration_path)
|
||||
else:
|
||||
tree = ET.parse(
|
||||
os.path.join(node.config_d_dir, "storage_configuration.xml"))
|
||||
except:
|
||||
tree = ET.ElementTree(
|
||||
ET.fromstring('<yandex><storage_configuration><disks/><policies/></storage_configuration></yandex>'))
|
||||
|
||||
root = tree.getroot()
|
||||
disk = root.find("storage_configuration").find("disks").find(name)
|
||||
assert disk is not None
|
||||
|
||||
new_path = disk.find("path")
|
||||
assert new_path is not None
|
||||
new_path.text = path
|
||||
|
||||
new_keep_free_space_bytes = disk.find("keep_free_space_bytes")
|
||||
assert new_keep_free_space_bytes is not None
|
||||
new_keep_free_space_bytes.text = keep_free_space_bytes
|
||||
|
||||
if separate_file:
|
||||
tree.write(separate_configuration_path)
|
||||
else:
|
||||
tree.write(os.path.join(node.config_d_dir, "storage_configuration.xml"))
|
||||
|
||||
|
||||
def add_policy(node, name, volumes):
|
||||
tree = ET.parse(os.path.join(node.config_d_dir, "storage_configuration.xml"))
|
||||
@ -123,6 +156,36 @@ def test_add_disk(started_cluster):
|
||||
except:
|
||||
""""""
|
||||
|
||||
def test_update_disk(started_cluster):
|
||||
try:
|
||||
name = "test_update_disk"
|
||||
engine = "MergeTree()"
|
||||
|
||||
start_over()
|
||||
node1.restart_clickhouse(kill=True)
|
||||
time.sleep(2)
|
||||
|
||||
node1.query("""
|
||||
CREATE TABLE {name} (
|
||||
d UInt64
|
||||
) ENGINE = {engine}
|
||||
ORDER BY d
|
||||
SETTINGS storage_policy='jbods_with_external'
|
||||
""".format(name=name, engine=engine))
|
||||
|
||||
assert node1.query("SELECT path, keep_free_space FROM system.disks where name = 'jbod2'") == TSV([
|
||||
["/jbod2/", "10485760"]])
|
||||
|
||||
update_disk(node1, "jbod2", "/jbod2/", "20971520")
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
|
||||
assert node1.query("SELECT path, keep_free_space FROM system.disks where name = 'jbod2'") == TSV([
|
||||
["/jbod2/", "20971520"]])
|
||||
finally:
|
||||
try:
|
||||
node1.query("DROP TABLE IF EXISTS {}".format(name))
|
||||
except:
|
||||
""""""
|
||||
|
||||
def test_add_disk_to_separate_config(started_cluster):
|
||||
try:
|
||||
|
Loading…
Reference in New Issue
Block a user