Merge pull request #19526 from taiyang-li/disk_reload

Enable local disk config reload
This commit is contained in:
Kseniia Sumarokova 2021-08-03 08:37:39 +03:00 committed by GitHub
commit 98326ec028
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 138 additions and 46 deletions

View File

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

View File

@ -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");
};
}

View File

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

View File

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

View File

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