ClickHouse/src/Disks/StoragePolicy.cpp

378 lines
12 KiB
C++
Raw Normal View History

#include "StoragePolicy.h"
2019-11-27 09:39:44 +00:00
#include "DiskFactory.h"
#include "DiskLocal.h"
#include <Interpreters/Context.h>
2019-09-24 00:45:40 +00:00
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
2014-03-13 12:48:07 +00:00
#include <set>
#include <Poco/File.h>
2019-09-24 00:45:40 +00:00
namespace
{
const auto DEFAULT_STORAGE_POLICY_NAME = "default";
const auto DEFAULT_VOLUME_NAME = "default";
const auto DEFAULT_DISK_NAME = "default";
}
2014-03-13 12:48:07 +00:00
namespace DB
{
2020-02-25 12:19:47 +00:00
namespace ErrorCodes
{
2020-03-05 08:21:53 +00:00
extern const int BAD_ARGUMENTS;
2020-02-25 12:19:47 +00:00
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int NO_ELEMENTS_IN_CONFIG;
2020-02-25 12:19:47 +00:00
extern const int UNKNOWN_DISK;
extern const int UNKNOWN_POLICY;
extern const int UNKNOWN_VOLUME;
2020-02-25 12:19:47 +00:00
extern const int LOGICAL_ERROR;
}
2019-09-04 16:00:20 +00:00
StoragePolicy::StoragePolicy(
String name_,
const Poco::Util::AbstractConfiguration & config,
2019-11-27 09:39:44 +00:00
const String & config_prefix,
DiskSelectorPtr disks)
2019-09-04 16:00:20 +00:00
: name(std::move(name_))
2019-04-05 17:37:27 +00:00
{
Poco::Util::AbstractConfiguration::Keys keys;
2019-07-16 18:20:47 +00:00
String volumes_prefix = config_prefix + ".volumes";
if (!config.has(volumes_prefix))
{
if (name != DEFAULT_STORAGE_POLICY_NAME)
throw Exception("Storage policy " + backQuote(name) + " must contain at least one volume (.volumes)", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
}
else
{
config.keys(volumes_prefix, keys);
}
2019-05-22 19:20:10 +00:00
for (const auto & attr_name : keys)
{
2019-07-25 11:42:48 +00:00
if (!std::all_of(attr_name.begin(), attr_name.end(), isWordCharASCII))
throw Exception(
"Volume name can contain only alphanumeric and '_' in storage policy " + backQuote(name) + " (" + attr_name + ")", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
volumes.emplace_back(createVolumeFromConfig(attr_name, config, volumes_prefix + "." + attr_name, disks));
}
2019-07-16 18:20:47 +00:00
if (volumes.empty() && name == DEFAULT_STORAGE_POLICY_NAME)
{
auto default_volume = std::make_shared<VolumeJBOD>(DEFAULT_VOLUME_NAME, std::vector<DiskPtr>{disks->get(DEFAULT_DISK_NAME)}, 0, false);
volumes.emplace_back(std::move(default_volume));
}
2019-08-01 10:29:14 +00:00
if (volumes.empty())
throw Exception("Storage policy " + backQuote(name) + " must contain at least one volume.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
const double default_move_factor = volumes.size() > 1 ? 0.1 : 0.0;
move_factor = config.getDouble(config_prefix + ".move_factor", default_move_factor);
2019-08-16 09:20:44 +00:00
if (move_factor > 1)
throw Exception("Disk move factor have to be in [0., 1.] interval, but set to " + toString(move_factor) + " in storage policy " + backQuote(name), ErrorCodes::LOGICAL_ERROR);
buildVolumeIndices();
}
StoragePolicy::StoragePolicy(String name_, Volumes volumes_, double move_factor_)
: volumes(std::move(volumes_)), name(std::move(name_)), move_factor(move_factor_)
2019-08-14 15:20:52 +00:00
{
if (volumes.empty())
throw Exception("Storage policy " + backQuote(name) + " must contain at least one Volume.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
2019-08-14 15:20:52 +00:00
2019-08-16 09:20:44 +00:00
if (move_factor > 1)
throw Exception("Disk move factor have to be in [0., 1.] interval, but set to " + toString(move_factor) + " in storage policy " + backQuote(name), ErrorCodes::LOGICAL_ERROR);
buildVolumeIndices();
}
2019-08-16 09:20:44 +00:00
StoragePolicy::StoragePolicy(const StoragePolicy & storage_policy,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
DiskSelectorPtr disks)
: StoragePolicy(storage_policy.getName(), config, config_prefix, disks)
{
for (auto & volume : volumes)
2019-08-14 15:20:52 +00:00
{
if (storage_policy.volume_index_by_volume_name.count(volume->getName()) > 0)
{
auto old_volume = storage_policy.getVolumeByName(volume->getName());
try
{
auto new_volume = updateVolumeFromConfig(old_volume, config, config_prefix + ".volumes." + volume->getName(), disks);
volume = std::move(new_volume);
}
catch (Exception & e)
{
/// Default policies are allowed to be missed in configuration.
if (e.code() != ErrorCodes::NO_ELEMENTS_IN_CONFIG || storage_policy.getName() != DEFAULT_STORAGE_POLICY_NAME)
throw;
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
if (!keys.empty())
throw;
}
}
2019-08-14 15:20:52 +00:00
}
}
bool StoragePolicy::isDefaultPolicy() const
{
/// Guessing if this policy is default, not 100% correct though.
if (getName() != DEFAULT_STORAGE_POLICY_NAME)
return false;
2019-12-27 19:44:54 +00:00
if (volumes.size() != 1)
return false;
if (volumes[0]->getName() != DEFAULT_VOLUME_NAME)
return false;
const auto & disks = volumes[0]->getDisks();
2019-12-27 19:44:54 +00:00
if (disks.size() != 1)
return false;
if (disks[0]->getName() != DEFAULT_DISK_NAME)
return false;
return true;
}
Disks StoragePolicy::getDisks() const
2019-04-05 17:37:27 +00:00
{
Disks res;
2019-04-05 17:37:27 +00:00
for (const auto & volume : volumes)
for (const auto & disk : volume->getDisks())
res.push_back(disk);
return res;
}
DiskPtr StoragePolicy::getAnyDisk() const
2019-05-13 20:58:22 +00:00
{
/// StoragePolicy must contain at least one Volume
2019-05-13 20:58:22 +00:00
/// Volume must contain at least one Disk
2019-05-22 19:20:10 +00:00
if (volumes.empty())
throw Exception("Storage policy " + backQuote(name) + " has no volumes. It's a bug.", ErrorCodes::LOGICAL_ERROR);
2019-09-04 16:00:20 +00:00
if (volumes[0]->getDisks().empty())
throw Exception("Volume " + backQuote(name) + "." + backQuote(volumes[0]->getName()) + " has no disks. It's a bug.", ErrorCodes::LOGICAL_ERROR);
2019-09-04 16:00:20 +00:00
return volumes[0]->getDisks()[0];
2019-05-13 20:58:22 +00:00
}
2019-06-09 12:31:03 +00:00
DiskPtr StoragePolicy::getDiskByName(const String & disk_name) const
{
2019-06-07 19:16:42 +00:00
for (auto && volume : volumes)
for (auto && disk : volume->getDisks())
2019-06-07 19:16:42 +00:00
if (disk->getName() == disk_name)
return disk;
return {};
}
UInt64 StoragePolicy::getMaxUnreservedFreeSpace() const
2019-04-05 17:37:27 +00:00
{
UInt64 res = 0;
2019-04-05 17:37:27 +00:00
for (const auto & volume : volumes)
res = std::max(res, volume->getMaxUnreservedFreeSpace());
return res;
}
2020-03-08 22:38:12 +00:00
ReservationPtr StoragePolicy::reserve(UInt64 bytes, size_t min_volume_index) const
2019-04-05 17:37:27 +00:00
{
2019-07-16 11:07:04 +00:00
for (size_t i = min_volume_index; i < volumes.size(); ++i)
{
2019-06-19 17:56:41 +00:00
const auto & volume = volumes[i];
2020-03-08 22:38:12 +00:00
auto reservation = volume->reserve(bytes);
2019-04-05 17:37:27 +00:00
if (reservation)
return reservation;
}
return {};
}
2020-03-08 22:38:12 +00:00
ReservationPtr StoragePolicy::reserve(UInt64 bytes) const
2019-05-22 19:20:10 +00:00
{
2020-03-08 22:38:12 +00:00
return reserve(bytes, 0);
2019-05-22 19:20:10 +00:00
}
2019-09-06 15:09:20 +00:00
ReservationPtr StoragePolicy::makeEmptyReservationOnLargestDisk() const
{
UInt64 max_space = 0;
DiskPtr max_disk;
for (const auto & volume : volumes)
{
for (const auto & disk : volume->getDisks())
{
auto avail_space = disk->getAvailableSpace();
if (avail_space > max_space)
{
max_space = avail_space;
max_disk = disk;
}
}
}
return max_disk->reserve(0);
}
VolumePtr StoragePolicy::getVolume(size_t index) const
{
if (index < volume_index_by_volume_name.size())
return volumes[index];
else
throw Exception("No volume with index " + std::to_string(index) + " in storage policy " + backQuote(name), ErrorCodes::UNKNOWN_VOLUME);
}
VolumePtr StoragePolicy::getVolumeByName(const String & volume_name) const
{
auto it = volume_index_by_volume_name.find(volume_name);
if (it == volume_index_by_volume_name.end())
throw Exception("No such volume " + backQuote(volume_name) + " in storage policy " + backQuote(name), ErrorCodes::UNKNOWN_VOLUME);
return getVolume(it->second);
}
void StoragePolicy::checkCompatibleWith(const StoragePolicyPtr & new_storage_policy) const
{
std::unordered_set<String> new_volume_names;
for (const auto & volume : new_storage_policy->getVolumes())
new_volume_names.insert(volume->getName());
for (const auto & volume : getVolumes())
{
if (new_volume_names.count(volume->getName()) == 0)
throw Exception("New storage policy " + backQuote(name) + " shall contain volumes of old one", ErrorCodes::BAD_ARGUMENTS);
std::unordered_set<String> new_disk_names;
for (const auto & disk : new_storage_policy->getVolumeByName(volume->getName())->getDisks())
new_disk_names.insert(disk->getName());
for (const auto & disk : volume->getDisks())
if (new_disk_names.count(disk->getName()) == 0)
throw Exception("New storage policy " + backQuote(name) + " shall contain disks of old one", ErrorCodes::BAD_ARGUMENTS);
}
}
2019-09-10 11:21:59 +00:00
size_t StoragePolicy::getVolumeIndexByDisk(const DiskPtr & disk_ptr) const
2019-08-14 15:20:52 +00:00
{
auto it = volume_index_by_disk_name.find(disk_ptr->getName());
if (it != volume_index_by_disk_name.end())
return it->second;
else
throw Exception("No disk " + backQuote(disk_ptr->getName()) + " in policy " + backQuote(name), ErrorCodes::UNKNOWN_DISK);
}
void StoragePolicy::buildVolumeIndices()
{
for (size_t index = 0; index < volumes.size(); ++index)
2019-08-14 15:20:52 +00:00
{
const VolumePtr & volume = volumes[index];
if (volume_index_by_volume_name.find(volume->getName()) != volume_index_by_volume_name.end())
throw Exception("Volume names must be unique in storage policy "
+ backQuote(name) + " (" + backQuote(volume->getName()) + " is duplicated)"
, ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
volume_index_by_volume_name[volume->getName()] = index;
for (const auto & disk : volume->getDisks())
{
const String & disk_name = disk->getName();
if (volume_index_by_disk_name.find(disk_name) != volume_index_by_disk_name.end())
throw Exception("Disk names must be unique in storage policy "
+ backQuote(name) + " (" + backQuote(disk_name) + " is duplicated)"
, ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
volume_index_by_disk_name[disk_name] = index;
}
2019-08-14 15:20:52 +00:00
}
}
bool StoragePolicy::hasAnyVolumeWithDisabledMerges() const
{
for (const auto & volume : volumes)
if (volume->areMergesAvoided())
return true;
return false;
}
2019-08-14 15:20:52 +00:00
StoragePolicySelector::StoragePolicySelector(
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
DiskSelectorPtr disks)
2019-04-05 19:58:59 +00:00
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
for (const auto & name : keys)
{
2019-05-11 18:00:43 +00:00
if (!std::all_of(name.begin(), name.end(), isWordCharASCII))
throw Exception(
"Storage policy name can contain only alphanumeric and '_' (" + backQuote(name) + ")", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
2019-08-14 15:20:52 +00:00
policies.emplace(name, std::make_shared<StoragePolicy>(name, config, config_prefix + "." + name, disks));
2020-05-30 21:57:37 +00:00
LOG_INFO(&Poco::Logger::get("StoragePolicySelector"), "Storage policy {} loaded", backQuote(name));
}
/// Add default policy if it isn't explicitly specified.
if (policies.find(DEFAULT_STORAGE_POLICY_NAME) == policies.end())
2019-08-14 15:20:52 +00:00
{
auto default_policy = std::make_shared<StoragePolicy>(DEFAULT_STORAGE_POLICY_NAME, config, config_prefix + "." + DEFAULT_STORAGE_POLICY_NAME, disks);
policies.emplace(DEFAULT_STORAGE_POLICY_NAME, std::move(default_policy));
2019-08-14 15:20:52 +00:00
}
}
StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks) const
{
std::shared_ptr<StoragePolicySelector> result = std::make_shared<StoragePolicySelector>(config, config_prefix, disks);
/// First pass, check.
for (const auto & [name, policy] : policies)
{
if (result->policies.count(name) == 0)
throw Exception("Storage policy " + backQuote(name) + " is missing in new configuration", ErrorCodes::BAD_ARGUMENTS);
policy->checkCompatibleWith(result->policies[name]);
}
/// Second pass, load.
for (const auto & [name, policy] : policies)
{
result->policies[name] = std::make_shared<StoragePolicy>(*policy, config, config_prefix + "." + name, disks);
}
return result;
}
StoragePolicyPtr StoragePolicySelector::get(const String & name) const
2019-04-05 17:37:27 +00:00
{
auto it = policies.find(name);
if (it == policies.end())
throw Exception("Unknown storage policy " + backQuote(name), ErrorCodes::UNKNOWN_POLICY);
return it->second;
}
2014-03-13 12:48:07 +00:00
}