Merge pull request #6489 from yandex/ObjatieGroba-feature_multiple_disks

Add multiple disk support to MergeTree storage engine.
This commit is contained in:
alesapin 2019-09-13 16:22:41 +03:00 committed by GitHub
commit ba8c11a026
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
74 changed files with 3996 additions and 763 deletions

View File

@ -88,6 +88,7 @@ namespace ErrorCodes
extern const int FAILED_TO_GETPWUID;
extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA;
extern const int NETWORK_ERROR;
extern const int PATH_ACCESS_DENIED;
}
@ -270,6 +271,15 @@ int Server::main(const std::vector<std::string> & /*args*/)
Poco::File(path + "data/" + default_database).createDirectories();
Poco::File(path + "metadata/" + default_database).createDirectories();
/// Check that we have read and write access to all data paths
auto disk_selector = global_context->getDiskSelector();
for (const auto & [name, disk] : disk_selector.getDisksMap())
{
Poco::File disk_path(disk->getPath());
if (!disk_path.canRead() || !disk_path.canWrite())
throw Exception("There is no RW access to disk " + name + " (" + disk->getPath() + ")", ErrorCodes::PATH_ACCESS_DENIED);
}
StatusFile status{path + "status"};
SCOPE_EXIT({

View File

@ -0,0 +1,538 @@
#include <Common/DiskSpaceMonitor.h>
#include <set>
#include <Common/escapeForFileName.h>
#include <Poco/File.h>
namespace DB
{
namespace DiskSpace
{
std::mutex Disk::mutex;
std::filesystem::path getMountPoint(std::filesystem::path absolute_path)
{
if (absolute_path.is_relative())
throw Exception("Path is relative. It's a bug.", ErrorCodes::LOGICAL_ERROR);
absolute_path = std::filesystem::canonical(absolute_path);
const auto get_device_id = [](const std::filesystem::path & p)
{
struct stat st;
if (stat(p.c_str(), &st))
throwFromErrnoWithPath("Cannot stat " + p.string(), p.string(), ErrorCodes::SYSTEM_ERROR);
return st.st_dev;
};
/// If /some/path/to/dir/ and /some/path/to/ have different device id,
/// then device which contains /some/path/to/dir/filename is mounted to /some/path/to/dir/
auto device_id = get_device_id(absolute_path);
while (absolute_path.has_relative_path())
{
auto parent = absolute_path.parent_path();
auto parent_device_id = get_device_id(parent);
if (device_id != parent_device_id)
return absolute_path;
absolute_path = parent;
device_id = parent_device_id;
}
return absolute_path;
}
/// Returns name of filesystem mounted to mount_point
#if !defined(__linux__)
[[noreturn]]
#endif
std::string getFilesystemName([[maybe_unused]] const std::string & mount_point)
{
#if defined(__linux__)
auto mounted_filesystems = setmntent("/etc/mtab", "r");
if (!mounted_filesystems)
throw DB::Exception("Cannot open /etc/mtab to get name of filesystem", ErrorCodes::SYSTEM_ERROR);
mntent fs_info;
constexpr size_t buf_size = 4096; /// The same as buffer used for getmntent in glibc. It can happen that it's not enough
char buf[buf_size];
while (getmntent_r(mounted_filesystems, &fs_info, buf, buf_size) && fs_info.mnt_dir != mount_point)
;
endmntent(mounted_filesystems);
if (fs_info.mnt_dir != mount_point)
throw DB::Exception("Cannot find name of filesystem by mount point " + mount_point, ErrorCodes::SYSTEM_ERROR);
return fs_info.mnt_fsname;
#else
throw DB::Exception("Supported on linux only", ErrorCodes::NOT_IMPLEMENTED);
#endif
}
ReservationPtr Disk::reserve(UInt64 bytes) const
{
if (!tryReserve(bytes))
return {};
return std::make_unique<Reservation>(bytes, std::static_pointer_cast<const Disk>(shared_from_this()));
}
bool Disk::tryReserve(UInt64 bytes) const
{
std::lock_guard lock(mutex);
if (bytes == 0)
{
LOG_DEBUG(&Logger::get("DiskSpaceMonitor"), "Reserving 0 bytes on disk " << name);
++reservation_count;
return true;
}
auto available_space = getAvailableSpace();
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
if (unreserved_space >= bytes)
{
LOG_DEBUG(
&Logger::get("DiskSpaceMonitor"),
"Reserving " << bytes << " bytes on disk " << name << " having unreserved " << unreserved_space << " bytes.");
++reservation_count;
reserved_bytes += bytes;
return true;
}
return false;
}
UInt64 Disk::getUnreservedSpace() const
{
std::lock_guard lock(mutex);
auto available_space = getSpaceInformation().getAvailableSpace();
available_space -= std::min(available_space, reserved_bytes);
return available_space;
}
UInt64 Disk::Stat::getTotalSpace() const
{
UInt64 total_size = fs.f_blocks * fs.f_bsize;
if (total_size < keep_free_space_bytes)
return 0;
return total_size - keep_free_space_bytes;
}
UInt64 Disk::Stat::getAvailableSpace() const
{
/// we use f_bavail, because part of b_free space is
/// available for superuser only and for system purposes
UInt64 total_size = fs.f_bavail * fs.f_bsize;
if (total_size < keep_free_space_bytes)
return 0;
return total_size - keep_free_space_bytes;
}
Reservation::~Reservation()
{
try
{
std::lock_guard lock(Disk::mutex);
if (disk_ptr->reserved_bytes < size)
{
disk_ptr->reserved_bytes = 0;
LOG_ERROR(&Logger::get("DiskSpaceMonitor"), "Unbalanced reservations size for disk '" + disk_ptr->getName() + "'.");
}
else
{
disk_ptr->reserved_bytes -= size;
}
if (disk_ptr->reservation_count == 0)
LOG_ERROR(&Logger::get("DiskSpaceMonitor"), "Unbalanced reservation count for disk '" + disk_ptr->getName() + "'.");
else
--disk_ptr->reservation_count;
}
catch (...)
{
tryLogCurrentException("~DiskSpaceMonitor");
}
}
void Reservation::update(UInt64 new_size)
{
std::lock_guard lock(Disk::mutex);
disk_ptr->reserved_bytes -= size;
size = new_size;
disk_ptr->reserved_bytes += size;
}
DiskSelector::DiskSelector(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const String & default_path)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
constexpr auto default_disk_name = "default";
bool has_default_disk = false;
for (const auto & disk_name : keys)
{
if (!std::all_of(disk_name.begin(), disk_name.end(), isWordCharASCII))
throw Exception("Disk name can contain only alphanumeric and '_' (" + disk_name + ")",
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
auto disk_config_prefix = config_prefix + "." + disk_name;
bool has_space_ratio = config.has(disk_config_prefix + ".keep_free_space_ratio");
if (config.has(disk_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(disk_config_prefix + ".keep_free_space_bytes", 0);
String path;
if (config.has(disk_config_prefix + ".path"))
path = config.getString(disk_config_prefix + ".path");
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 = default_path;
// Create tmp disk for getting total disk space.
keep_free_space_bytes = static_cast<UInt64>(Disk("tmp", tmp_path, 0).getTotalSpace() * ratio);
}
if (disk_name == default_disk_name)
{
has_default_disk = true;
if (!path.empty())
throw Exception("\"default\" disk path should be provided in <path> not it <storage_configuration>",
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
disks.emplace(disk_name, std::make_shared<const Disk>(disk_name, default_path, keep_free_space_bytes));
}
else
{
if (path.empty())
throw Exception("Disk path can not be empty. Disk " + disk_name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
if (path.back() != '/')
throw Exception("Disk path must end with /. Disk " + disk_name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
disks.emplace(disk_name, std::make_shared<const Disk>(disk_name, path, keep_free_space_bytes));
}
}
if (!has_default_disk)
disks.emplace(default_disk_name, std::make_shared<const Disk>(default_disk_name, default_path, 0));
}
const DiskPtr & DiskSelector::operator[](const String & name) const
{
auto it = disks.find(name);
if (it == disks.end())
throw Exception("Unknown disk " + name, ErrorCodes::UNKNOWN_DISK);
return it->second;
}
Volume::Volume(
String name_,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const DiskSelector & disk_selector)
: name(std::move(name_))
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
Logger * logger = &Logger::get("StorageConfiguration");
for (const auto & disk : keys)
{
if (startsWith(disk, "disk"))
{
auto disk_name = config.getString(config_prefix + "." + disk);
disks.push_back(disk_selector[disk_name]);
}
}
if (disks.empty())
throw Exception("Volume must contain at least one disk.", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
auto has_max_bytes = config.has(config_prefix + ".max_data_part_size_bytes");
auto has_max_ratio = config.has(config_prefix + ".max_data_part_size_ratio");
if (has_max_bytes && has_max_ratio)
throw Exception("Only one of 'max_data_part_size_bytes' and 'max_data_part_size_ratio' should be specified.",
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
if (has_max_bytes)
{
max_data_part_size = config.getUInt64(config_prefix + ".max_data_part_size_bytes", 0);
}
else if (has_max_ratio)
{
auto ratio = config.getDouble(config_prefix + ".max_data_part_size_ratio");
if (ratio < 0)
throw Exception("'max_data_part_size_ratio' have to be not less then 0.",
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
UInt64 sum_size = 0;
std::vector<UInt64> sizes;
for (const auto & disk : disks)
{
sizes.push_back(disk->getTotalSpace());
sum_size += sizes.back();
}
max_data_part_size = static_cast<decltype(max_data_part_size)>(sum_size * ratio / disks.size());
for (size_t i = 0; i < disks.size(); ++i)
if (sizes[i] < max_data_part_size)
LOG_WARNING(logger, "Disk " << disks[i]->getName() << " on volume " << config_prefix <<
" have not enough space (" << sizes[i] <<
") for containing part the size of max_data_part_size (" <<
max_data_part_size << ")");
}
constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u;
if (max_data_part_size < MIN_PART_SIZE)
LOG_WARNING(logger, "Volume '" << name << "' max_data_part_size is too low ("
<< formatReadableSizeWithBinarySuffix(max_data_part_size) << " < "
<< formatReadableSizeWithBinarySuffix(MIN_PART_SIZE) << ")");
}
ReservationPtr Volume::reserve(UInt64 expected_size) const
{
/// This volume can not store files which size greater than max_data_part_size
if (max_data_part_size != 0 && expected_size > max_data_part_size)
return {};
size_t start_from = last_used.fetch_add(1u, std::memory_order_relaxed);
size_t disks_num = disks.size();
for (size_t i = 0; i < disks_num; ++i)
{
size_t index = (start_from + i) % disks_num;
auto reservation = disks[index]->reserve(expected_size);
if (reservation)
return reservation;
}
return {};
}
UInt64 Volume::getMaxUnreservedFreeSpace() const
{
UInt64 res = 0;
for (const auto & disk : disks)
res = std::max(res, disk->getUnreservedSpace());
return res;
}
StoragePolicy::StoragePolicy(
String name_,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const DiskSelector & disks)
: name(std::move(name_))
{
String volumes_prefix = config_prefix + ".volumes";
if (!config.has(volumes_prefix))
throw Exception("StoragePolicy must contain at least one volume (.volumes)", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(volumes_prefix, keys);
for (const auto & attr_name : keys)
{
if (!std::all_of(attr_name.begin(), attr_name.end(), isWordCharASCII))
throw Exception("Volume name can contain only alphanumeric and '_' (" + attr_name + ")", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
volumes.push_back(std::make_shared<Volume>(attr_name, config, volumes_prefix + "." + attr_name, disks));
if (volumes_names.find(attr_name) != volumes_names.end())
throw Exception("Volumes names must be unique (" + attr_name + " duplicated)", ErrorCodes::UNKNOWN_POLICY);
volumes_names[attr_name] = volumes.size() - 1;
}
if (volumes.empty())
throw Exception("StoragePolicy must contain at least one volume.", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
/// Check that disks are unique in Policy
std::set<String> disk_names;
for (const auto & volume : volumes)
{
for (const auto & disk : volume->disks)
{
if (disk_names.find(disk->getName()) != disk_names.end())
throw Exception("Duplicate disk '" + disk->getName() + "' in storage policy '" + name + "'", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
disk_names.insert(disk->getName());
}
}
move_factor = config.getDouble(config_prefix + ".move_factor", 0.1);
if (move_factor > 1)
throw Exception("Disk move factor have to be in [0., 1.] interval, but set to " + toString(move_factor),
ErrorCodes::LOGICAL_ERROR);
}
StoragePolicy::StoragePolicy(String name_, Volumes volumes_, double move_factor_)
: volumes(std::move(volumes_))
, name(std::move(name_))
, move_factor(move_factor_)
{
if (volumes.empty())
throw Exception("StoragePolicy must contain at least one Volume.", ErrorCodes::UNKNOWN_POLICY);
if (move_factor > 1)
throw Exception("Disk move factor have to be in [0., 1.] interval, but set to " + toString(move_factor),
ErrorCodes::LOGICAL_ERROR);
for (size_t i = 0; i < volumes.size(); ++i)
{
if (volumes_names.find(volumes[i]->getName()) != volumes_names.end())
throw Exception("Volumes names must be unique (" + volumes[i]->getName() + " duplicated).", ErrorCodes::UNKNOWN_POLICY);
volumes_names[volumes[i]->getName()] = i;
}
}
Disks StoragePolicy::getDisks() const
{
Disks res;
for (const auto & volume : volumes)
for (const auto & disk : volume->disks)
res.push_back(disk);
return res;
}
DiskPtr StoragePolicy::getAnyDisk() const
{
/// StoragePolicy must contain at least one Volume
/// Volume must contain at least one Disk
if (volumes.empty())
throw Exception("StoragePolicy has no volumes. It's a bug.", ErrorCodes::NOT_ENOUGH_SPACE);
if (volumes[0]->disks.empty())
throw Exception("Volume '" + volumes[0]->getName() + "' has no disks. It's a bug.", ErrorCodes::NOT_ENOUGH_SPACE);
return volumes[0]->disks[0];
}
DiskPtr StoragePolicy::getDiskByName(const String & disk_name) const
{
for (auto && volume : volumes)
for (auto && disk : volume->disks)
if (disk->getName() == disk_name)
return disk;
return {};
}
UInt64 StoragePolicy::getMaxUnreservedFreeSpace() const
{
UInt64 res = 0;
for (const auto & volume : volumes)
res = std::max(res, volume->getMaxUnreservedFreeSpace());
return res;
}
ReservationPtr StoragePolicy::reserve(UInt64 expected_size, size_t min_volume_index) const
{
for (size_t i = min_volume_index; i < volumes.size(); ++i)
{
const auto & volume = volumes[i];
auto reservation = volume->reserve(expected_size);
if (reservation)
return reservation;
}
return {};
}
ReservationPtr StoragePolicy::reserve(UInt64 expected_size) const
{
return reserve(expected_size, 0);
}
ReservationPtr StoragePolicy::makeEmptyReservationOnLargestDisk() const
{
UInt64 max_space = 0;
DiskPtr max_disk;
for (const auto & volume : volumes)
{
for (const auto & disk : volume->disks)
{
auto avail_space = disk->getAvailableSpace();
if (avail_space > max_space)
{
max_space = avail_space;
max_disk = disk;
}
}
}
return max_disk->reserve(0);
}
size_t StoragePolicy::getVolumeIndexByDisk(const DiskPtr & disk_ptr) const
{
for (size_t i = 0; i < volumes.size(); ++i)
{
const auto & volume = volumes[i];
for (const auto & disk : volume->disks)
if (disk->getName() == disk_ptr->getName())
return i;
}
throw Exception("No disk " + disk_ptr->getName() + " in policy " + name, ErrorCodes::UNKNOWN_DISK);
}
StoragePolicySelector::StoragePolicySelector(
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
const DiskSelector & disks)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
for (const auto & name : keys)
{
if (!std::all_of(name.begin(), name.end(), isWordCharASCII))
throw Exception("StoragePolicy name can contain only alphanumeric and '_' (" + name + ")",
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
policies.emplace(name, std::make_shared<StoragePolicy>(name, config, config_prefix + "." + name, disks));
LOG_INFO(&Logger::get("StoragePolicySelector"), "Storage policy " << name << " loaded");
}
constexpr auto default_storage_policy_name = "default";
constexpr auto default_volume_name = "default";
constexpr auto default_disk_name = "default";
/// Add default policy if it's not specified explicetly
if (policies.find(default_storage_policy_name) == policies.end())
{
auto default_volume = std::make_shared<Volume>(
default_volume_name,
std::vector<DiskPtr>{disks[default_disk_name]},
0);
auto default_policy = std::make_shared<StoragePolicy>(default_storage_policy_name, Volumes{default_volume}, 0.0);
policies.emplace(default_storage_policy_name, default_policy);
}
}
const StoragePolicyPtr & StoragePolicySelector::operator[](const String & name) const
{
auto it = policies.find(name);
if (it == policies.end())
throw Exception("Unknown StoragePolicy " + name, ErrorCodes::UNKNOWN_POLICY);
return it->second;
}
}
}

View File

@ -0,0 +1,359 @@
#pragma once
#include <mutex>
#include <sys/statvfs.h>
#include <sys/types.h>
#include <sys/stat.h>
#include <unistd.h>
#if defined(__linux__)
#include <cstdio>
#include <mntent.h>
#endif
#include <memory>
#include <filesystem>
#include <boost/noncopyable.hpp>
#include <Poco/Util/AbstractConfiguration.h>
#include <common/logger_useful.h>
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
#include <Common/formatReadable.h>
#include <Common/CurrentMetrics.h>
namespace CurrentMetrics
{
extern const Metric DiskSpaceReservedForMerge;
}
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int CANNOT_STATVFS;
extern const int NOT_ENOUGH_SPACE;
extern const int SYSTEM_ERROR;
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int UNKNOWN_POLICY;
extern const int UNKNOWN_DISK;
}
namespace DiskSpace
{
class Reservation;
using ReservationPtr = std::unique_ptr<Reservation>;
/// Returns mount point of filesystem where absoulte_path (must exist) is located
std::filesystem::path getMountPoint(std::filesystem::path absolute_path);
/// Returns name of filesystem mounted to mount_point
#if !defined(__linux__)
[[noreturn]]
#endif
std::string getFilesystemName([[maybe_unused]] const std::string & mount_point);
inline struct statvfs getStatVFS(const std::string & path)
{
struct statvfs fs;
if (statvfs(path.c_str(), &fs) != 0)
throwFromErrnoWithPath(
"Could not calculate available disk space (statvfs)", path, ErrorCodes::CANNOT_STATVFS);
return fs;
}
/**
* Provide interface for reservation
*/
class Space : public std::enable_shared_from_this<Space>
{
public:
virtual ReservationPtr reserve(UInt64 bytes) const = 0;
virtual const String & getName() const = 0;
virtual ~Space() = default;
};
using SpacePtr = std::shared_ptr<const Space>;
/** Disk - Smallest space unit.
* path - Path to space. Ends with /
* name - Unique key using for disk space reservation.
*/
class Disk : public Space
{
public:
friend class Reservation;
/// Snapshot of disk space state (free and total space)
class Stat
{
struct statvfs fs{};
UInt64 keep_free_space_bytes;
public:
explicit Stat(const Disk & disk)
{
if (statvfs(disk.path.c_str(), &fs) != 0)
throwFromErrno("Could not calculate available disk space (statvfs)", ErrorCodes::CANNOT_STATVFS);
keep_free_space_bytes = disk.keep_free_space_bytes;
}
/// Total space on disk using information from statvfs
UInt64 getTotalSpace() const;
/// Available space on disk using information from statvfs
UInt64 getAvailableSpace() const;
};
Disk(const String & name_, const String & path_, UInt64 keep_free_space_bytes_)
: name(name_)
, path(path_)
, keep_free_space_bytes(keep_free_space_bytes_)
{
if (path.back() != '/')
throw Exception("Disk path must ends with '/', but '" + path + "' doesn't.", ErrorCodes::LOGICAL_ERROR);
}
/// Reserves bytes on disk, if not possible returns nullptr.
ReservationPtr reserve(UInt64 bytes) const override;
/// Disk name from configuration;
const String & getName() const override { return name; }
/// Path on fs to disk
const String & getPath() const { return path; }
/// Path to clickhouse data directory on this disk
String getClickHouseDataPath() const { return path + "data/"; }
/// Amount of bytes which should be kept free on this disk
UInt64 getKeepingFreeSpace() const { return keep_free_space_bytes; }
/// Snapshot of disk space state (free and total space)
Stat getSpaceInformation() const { return Stat(*this); }
/// Total available space on disk
UInt64 getTotalSpace() const { return getSpaceInformation().getTotalSpace(); }
/// Space currently available on disk, take information from statvfs call
UInt64 getAvailableSpace() const { return getSpaceInformation().getAvailableSpace(); }
/// Currently available (prev method) minus already reserved space
UInt64 getUnreservedSpace() const;
private:
const String name;
const String path;
const UInt64 keep_free_space_bytes;
/// Used for reservation counters modification
static std::mutex mutex;
mutable UInt64 reserved_bytes = 0;
mutable UInt64 reservation_count = 0;
private:
/// Reserves bytes on disk, if not possible returns false
bool tryReserve(UInt64 bytes) const;
};
/// It is not possible to change disk runtime.
using DiskPtr = std::shared_ptr<const Disk>;
using Disks = std::vector<DiskPtr>;
/** Information about reserved size on concrete disk.
* Unreserve on destroy. Doesn't reserve bytes in constructor.
*/
class Reservation final : private boost::noncopyable
{
public:
Reservation(UInt64 size_, DiskPtr disk_ptr_)
: size(size_)
, metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size)
, disk_ptr(disk_ptr_)
{
}
/// Unreserves reserved space and decrement reservations count on disk
~Reservation();
/// Changes amount of reserved space. When new_size is greater than before,
/// availability of free space is not checked.
void update(UInt64 new_size);
/// Get reservation size
UInt64 getSize() const { return size; }
/// Get disk where reservation take place
const DiskPtr & getDisk() const { return disk_ptr; }
private:
UInt64 size;
CurrentMetrics::Increment metric_increment;
DiskPtr disk_ptr;
};
/// Parse .xml configuration and store information about disks
/// Mostly used for introspection.
class DiskSelector
{
public:
DiskSelector(const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const String & default_path);
/// Get disk by name
const DiskPtr & operator[](const String & name) const;
/// Get all disks name
const auto & getDisksMap() const { return disks; }
private:
std::map<String, DiskPtr> disks;
};
/**
* Disks group by some (user) criteria. For example,
* - Volume("slow_disks", [d1, d2], 100)
* - Volume("fast_disks", [d3, d4], 200)
* Cannot store parts larger than max_data_part_size.
*/
class Volume : public Space
{
friend class StoragePolicy;
public:
Volume(String name_, std::vector<DiskPtr> disks_, UInt64 max_data_part_size_)
: max_data_part_size(max_data_part_size_)
, disks(std::move(disks_))
, name(std::move(name_))
{
}
Volume(String name_, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const DiskSelector & disk_selector);
/// Uses Round-robin to choose disk for reservation.
/// Returns valid reservation or nullptr if there is no space left on any disk.
ReservationPtr reserve(UInt64 bytes) const override;
/// Return biggest unreserved space across all disks
UInt64 getMaxUnreservedFreeSpace() const;
/// Volume name from config
const String & getName() const override { return name; }
/// Max size of reservation
UInt64 max_data_part_size = 0;
/// Disks in volume
Disks disks;
private:
mutable std::atomic<size_t> last_used = 0;
const String name;
};
using VolumePtr = std::shared_ptr<const Volume>;
using Volumes = std::vector<VolumePtr>;
/**
* Contains all information about volumes configuration for Storage.
* Can determine appropriate Volume and Disk for each reservation.
*/
class StoragePolicy : public Space
{
public:
StoragePolicy(String name_, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const DiskSelector & disks);
StoragePolicy(String name_, Volumes volumes_, double move_factor_);
/// Returns disks ordered by volumes priority
Disks getDisks() const;
/// Returns any disk
/// Used when it's not important, for example for
/// mutations files
DiskPtr getAnyDisk() const;
DiskPtr getDiskByName(const String & disk_name) const;
/// Get free space from most free disk
UInt64 getMaxUnreservedFreeSpace() const;
const String & getName() const override { return name; }
/// Returns valid reservation or null
ReservationPtr reserve(UInt64 bytes) const override;
/// Reserve space on any volume with index > min_volume_index
ReservationPtr reserve(UInt64 bytes, size_t min_volume_index) const;
/// Find volume index, which contains disk
size_t getVolumeIndexByDisk(const DiskPtr & disk_ptr) const;
/// Reserves 0 bytes on disk with max available space
/// Do not use this function when it is possible to predict size.
ReservationPtr makeEmptyReservationOnLargestDisk() const;
const Volumes & getVolumes() const { return volumes; }
/// Returns number [0., 1.] -- fraction of free space on disk
/// which should be kept with help of background moves
double getMoveFactor() const { return move_factor; }
/// Get volume by index from storage_policy
VolumePtr getVolume(size_t i) const { return (i < volumes_names.size() ? volumes[i] : VolumePtr()); }
VolumePtr getVolumeByName(const String & volume_name) const
{
auto it = volumes_names.find(volume_name);
if (it == volumes_names.end())
return {};
return getVolume(it->second);
}
private:
Volumes volumes;
const String name;
std::map<String, size_t> volumes_names;
/// move_factor from interval [0., 1.]
/// We move something if disk from this policy
/// filled more than total_size * move_factor
double move_factor = 0.1; /// by default move factor is 10%
};
using StoragePolicyPtr = std::shared_ptr<const StoragePolicy>;
/// Parse .xml configuration and store information about policies
/// Mostly used for introspection.
class StoragePolicySelector
{
public:
StoragePolicySelector(const Poco::Util::AbstractConfiguration & config,
const String & config_prefix, const DiskSelector & disks);
/// Policy by name
const StoragePolicyPtr & operator[](const String & name) const;
/// All policies
const std::map<String, StoragePolicyPtr> & getPoliciesMap() const { return policies; }
private:
std::map<String, StoragePolicyPtr> policies;
};
}
}

View File

@ -452,6 +452,11 @@ namespace ErrorCodes
extern const int INVALID_WITH_FILL_EXPRESSION = 475;
extern const int WITH_TIES_WITHOUT_ORDER_BY = 476;
extern const int INVALID_USAGE_OF_INPUT = 477;
extern const int UNKNOWN_POLICY = 478;
extern const int UNKNOWN_DISK = 479;
extern const int UNKNOWN_PROTOCOL = 480;
extern const int PATH_ACCESS_DENIED = 481;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -10,7 +10,7 @@
#include <common/demangle.h>
#include <Common/config_version.h>
#include <Common/formatReadable.h>
#include <Storages/MergeTree/DiskSpaceMonitor.h>
#include <Common/DiskSpaceMonitor.h>
#include <filesystem>
namespace DB
@ -84,16 +84,16 @@ void getNoSpaceLeftInfoMessage(std::filesystem::path path, std::string & msg)
while (!std::filesystem::exists(path) && path.has_relative_path())
path = path.parent_path();
auto fs = DiskSpaceMonitor::getStatVFS(path);
auto fs = DiskSpace::getStatVFS(path);
msg += "\nTotal space: " + formatReadableSizeWithBinarySuffix(fs.f_blocks * fs.f_bsize)
+ "\nAvailable space: " + formatReadableSizeWithBinarySuffix(fs.f_bavail * fs.f_bsize)
+ "\nTotal inodes: " + formatReadableQuantity(fs.f_files)
+ "\nAvailable inodes: " + formatReadableQuantity(fs.f_favail);
auto mount_point = DiskSpaceMonitor::getMountPoint(path).string();
auto mount_point = DiskSpace::getMountPoint(path).string();
msg += "\nMount point: " + mount_point;
#if defined(__linux__)
msg += "\nFilesystem: " + DiskSpaceMonitor::getFilesystemName(mount_point);
msg += "\nFilesystem: " + DiskSpace::getFilesystemName(mount_point);
#endif
}

View File

@ -38,20 +38,22 @@ namespace detail
SessionPtr session;
std::istream * istr; /// owned by session
std::unique_ptr<ReadBuffer> impl;
std::vector<Poco::Net::HTTPCookie> cookies;
public:
using OutStreamCallback = std::function<void(std::ostream &)>;
explicit ReadWriteBufferFromHTTPBase(SessionPtr session_,
explicit ReadWriteBufferFromHTTPBase(
SessionPtr session_,
Poco::URI uri_,
const std::string & method_ = {},
OutStreamCallback out_stream_callback = {},
const Poco::Net::HTTPBasicCredentials & credentials = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE)
: ReadBuffer(nullptr, 0)
, uri {uri_}
, method {!method_.empty() ? method_ : out_stream_callback ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET}
, session {session_}
, uri{uri_}
, method{!method_.empty() ? method_ : out_stream_callback ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET}
, session{session_}
{
// With empty path poco will send "POST HTTP/1.1" its bug.
if (uri.getPath().empty())
@ -78,6 +80,7 @@ namespace detail
out_stream_callback(stream_out);
istr = receiveResponse(*session, request, response);
response.getCookies(cookies);
impl = std::make_unique<ReadBufferFromIStream>(*istr, buffer_size_);
}
@ -90,7 +93,6 @@ namespace detail
}
}
bool nextImpl() override
{
if (!impl->next())
@ -99,6 +101,14 @@ namespace detail
working_buffer = internal_buffer;
return true;
}
std::string getResponseCookie(const std::string & name, const std::string & def) const
{
for (const auto & cookie : cookies)
if (cookie.getName() == name)
return cookie.getValue();
return def;
}
};
}

View File

@ -15,6 +15,7 @@ namespace ActionLocks
extern const StorageActionBlockType ReplicationQueue = 4;
extern const StorageActionBlockType DistributedSend = 5;
extern const StorageActionBlockType PartsTTLMerge = 6;
extern const StorageActionBlockType PartsMove = 7;
}

View File

@ -143,6 +143,11 @@ struct ContextShared
std::unique_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression settings, depending on the size of the part.
mutable std::unique_ptr<CompressionCodecSelector> compression_codec_selector;
/// Storage disk chooser
mutable std::unique_ptr<DiskSpace::DiskSelector> merge_tree_disk_selector;
/// Storage policy chooser
mutable std::unique_ptr<DiskSpace::StoragePolicySelector> merge_tree_storage_policy_selector;
std::optional<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
@ -1759,6 +1764,56 @@ CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double par
}
const DiskSpace::DiskPtr & Context::getDisk(const String & name) const
{
auto lock = getLock();
const auto & disk_selector = getDiskSelector();
return disk_selector[name];
}
DiskSpace::DiskSelector & Context::getDiskSelector() const
{
auto lock = getLock();
if (!shared->merge_tree_disk_selector)
{
constexpr auto config_name = "storage_configuration.disks";
auto & config = getConfigRef();
shared->merge_tree_disk_selector = std::make_unique<DiskSpace::DiskSelector>(config, config_name, getPath());
}
return *shared->merge_tree_disk_selector;
}
const DiskSpace::StoragePolicyPtr & Context::getStoragePolicy(const String & name) const
{
auto lock = getLock();
auto & policy_selector = getStoragePolicySelector();
return policy_selector[name];
}
DiskSpace::StoragePolicySelector & Context::getStoragePolicySelector() const
{
auto lock = getLock();
if (!shared->merge_tree_storage_policy_selector)
{
constexpr auto config_name = "storage_configuration.policies";
auto & config = getConfigRef();
shared->merge_tree_storage_policy_selector = std::make_unique<DiskSpace::StoragePolicySelector>(config, config_name, getDiskSelector());
}
return *shared->merge_tree_storage_policy_selector;
}
const MergeTreeSettings & Context::getMergeTreeSettings() const
{
auto lock = getLock();

View File

@ -13,6 +13,7 @@
#include <Common/ThreadPool.h>
#include "config_core.h"
#include <Storages/IStorage_fwd.h>
#include <Common/DiskSpaceMonitor.h>
#include <atomic>
#include <chrono>
#include <condition_variable>
@ -485,6 +486,16 @@ public:
/// Lets you select the compression codec according to the conditions described in the configuration file.
std::shared_ptr<ICompressionCodec> chooseCompressionCodec(size_t part_size, double part_size_ratio) const;
DiskSpace::DiskSelector & getDiskSelector() const;
/// Provides storage disks
const DiskSpace::DiskPtr & getDisk(const String & name) const;
DiskSpace::StoragePolicySelector & getStoragePolicySelector() const;
/// Provides storage politics schemes
const DiskSpace::StoragePolicyPtr & getStoragePolicy(const String &name) const;
/// Get the server uptime in seconds.
time_t getUptimeSeconds() const;

View File

@ -49,6 +49,7 @@ namespace ActionLocks
extern StorageActionBlockType ReplicationQueue;
extern StorageActionBlockType DistributedSend;
extern StorageActionBlockType PartsTTLMerge;
extern StorageActionBlockType PartsMove;
}
@ -189,6 +190,12 @@ BlockIO InterpreterSystemQuery::execute()
case Type::START_TTL_MERGES:
startStopAction(context, query, ActionLocks::PartsTTLMerge, true);
break;
case Type::STOP_MOVES:
startStopAction(context, query, ActionLocks::PartsMove, false);
break;
case Type::START_MOVES:
startStopAction(context, query, ActionLocks::PartsMove, true);
break;
case Type::STOP_FETCHES:
startStopAction(context, query, ActionLocks::PartsFetch, false);
break;

View File

@ -27,7 +27,9 @@ Block PartLogElement::createBlock()
{"DownloadPart", static_cast<Int8>(DOWNLOAD_PART)},
{"RemovePart", static_cast<Int8>(REMOVE_PART)},
{"MutatePart", static_cast<Int8>(MUTATE_PART)},
});
{"MovePart", static_cast<Int8>(MOVE_PART)},
}
);
return
{
@ -40,6 +42,7 @@ Block PartLogElement::createBlock()
{ColumnString::create(), std::make_shared<DataTypeString>(), "table"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "part_name"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "partition_id"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "path_on_disk"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "rows"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "size_in_bytes"}, // On disk
@ -71,6 +74,7 @@ void PartLogElement::appendToBlock(Block & block) const
columns[i++]->insert(table_name);
columns[i++]->insert(part_name);
columns[i++]->insert(partition_id);
columns[i++]->insert(path_on_disk);
columns[i++]->insert(rows);
columns[i++]->insert(bytes_compressed_on_disk);
@ -124,6 +128,7 @@ bool PartLog::addNewParts(Context & current_context, const PartLog::MutableDataP
elem.table_name = part->storage.getTableName();
elem.partition_id = part->info.partition_id;
elem.part_name = part->name;
elem.path_on_disk = part->getFullPath();
elem.bytes_compressed_on_disk = part->bytes_on_disk;
elem.rows = part->rows_count;

View File

@ -15,6 +15,7 @@ struct PartLogElement
DOWNLOAD_PART = 3,
REMOVE_PART = 4,
MUTATE_PART = 5,
MOVE_PART = 6,
};
Type event_type = NEW_PART;
@ -26,6 +27,7 @@ struct PartLogElement
String table_name;
String part_name;
String partition_id;
String path_on_disk;
/// Size of the part
UInt64 rows = 0;

View File

@ -1,5 +1,6 @@
#include <Parsers/ASTAlterQuery.h>
#include <iomanip>
#include <IO/WriteHelpers.h>
namespace DB
@ -167,6 +168,25 @@ void ASTAlterCommand::formatImpl(
<< (part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::MOVE_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MOVE "
<< (part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame);
settings.ostr << " TO ";
switch (move_destination_type)
{
case MoveDestinationType::DISK:
settings.ostr << "DISK ";
break;
case MoveDestinationType::VOLUME:
settings.ostr << "VOLUME ";
break;
}
WriteBufferFromOwnString move_destination_name_buf;
writeQuoted(move_destination_name, move_destination_name_buf);
settings.ostr << move_destination_name_buf.str();
}
else if (type == ASTAlterCommand::REPLACE_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << (replace ? "REPLACE" : "ATTACH") << " PARTITION "

View File

@ -42,6 +42,7 @@ public:
DROP_PARTITION,
DROP_DETACHED_PARTITION,
ATTACH_PARTITION,
MOVE_PARTITION,
REPLACE_PARTITION,
FETCH_PARTITION,
FREEZE_PARTITION,
@ -117,7 +118,7 @@ public:
bool detach = false; /// true for DETACH PARTITION
bool part = false; /// true for ATTACH PART and DROP DETACHED PART
bool part = false; /// true for ATTACH PART, DROP DETACHED PART and MOVE
bool clear_column = false; /// for CLEAR COLUMN (do not drop column from metadata)
@ -127,6 +128,16 @@ public:
bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
enum MoveDestinationType
{
DISK,
VOLUME,
};
MoveDestinationType move_destination_type;
String move_destination_name;
/** For FETCH PARTITION - the path in ZK to the shard, from which to download the partition.
*/
String from;

View File

@ -59,6 +59,10 @@ const char * ASTSystemQuery::typeToString(Type type)
return "STOP TTL MERGES";
case Type::START_TTL_MERGES:
return "START TTL MERGES";
case Type::STOP_MOVES:
return "STOP MOVES";
case Type::START_MOVES:
return "START MOVES";
case Type::STOP_FETCHES:
return "STOP FETCHES";
case Type::START_FETCHES:
@ -106,6 +110,8 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|| type == Type::START_MERGES
|| type == Type::STOP_TTL_MERGES
|| type == Type::START_TTL_MERGES
|| type == Type::STOP_MOVES
|| type == Type::START_MOVES
|| type == Type::STOP_FETCHES
|| type == Type::START_FETCHES
|| type == Type::STOP_REPLICATED_SENDS

View File

@ -37,6 +37,8 @@ public:
START_TTL_MERGES,
STOP_FETCHES,
START_FETCHES,
STOP_MOVES,
START_MOVES,
STOP_REPLICATED_SENDS,
START_REPLICATED_SENDS,
STOP_REPLICATION_QUEUES,

View File

@ -49,9 +49,11 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserKeyword s_attach_partition("ATTACH PARTITION");
ParserKeyword s_detach_partition("DETACH PARTITION");
ParserKeyword s_drop_partition("DROP PARTITION");
ParserKeyword s_move_partition("MOVE PARTITION");
ParserKeyword s_drop_detached_partition("DROP DETACHED PARTITION");
ParserKeyword s_drop_detached_part("DROP DETACHED PART");
ParserKeyword s_attach_part("ATTACH PART");
ParserKeyword s_move_part("MOVE PART");
ParserKeyword s_fetch_partition("FETCH PARTITION");
ParserKeyword s_replace_partition("REPLACE PARTITION");
ParserKeyword s_freeze("FREEZE");
@ -65,6 +67,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserKeyword s_with("WITH");
ParserKeyword s_name("NAME");
ParserKeyword s_to_disk("TO DISK");
ParserKeyword s_to_volume("TO VOLUME");
ParserKeyword s_delete_where("DELETE WHERE");
ParserKeyword s_update("UPDATE");
ParserKeyword s_where("WHERE");
@ -222,6 +227,47 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
return false;
}
}
else if (s_move_part.ignore(pos, expected))
{
if (!parser_string_literal.parse(pos, command->partition, expected))
return false;
command->type = ASTAlterCommand::MOVE_PARTITION;
command->part = true;
if (s_to_disk.ignore(pos))
command->move_destination_type = ASTAlterCommand::MoveDestinationType::DISK;
else if (s_to_volume.ignore(pos))
command->move_destination_type = ASTAlterCommand::MoveDestinationType::VOLUME;
else
return false;
ASTPtr ast_space_name;
if (!parser_string_literal.parse(pos, ast_space_name, expected))
return false;
command->move_destination_name = ast_space_name->as<ASTLiteral &>().value.get<const String &>();
}
else if (s_move_partition.ignore(pos, expected))
{
if (!parser_partition.parse(pos, command->partition, expected))
return false;
command->type = ASTAlterCommand::MOVE_PARTITION;
if (s_to_disk.ignore(pos))
command->move_destination_type = ASTAlterCommand::MoveDestinationType::DISK;
else if (s_to_volume.ignore(pos))
command->move_destination_type = ASTAlterCommand::MoveDestinationType::VOLUME;
else
return false;
ASTPtr ast_space_name;
if (!parser_string_literal.parse(pos, ast_space_name, expected))
return false;
command->move_destination_name = ast_space_name->as<ASTLiteral &>().value.get<const String &>();
}
else if (s_add_constraint.ignore(pos, expected))
{
if (s_if_not_exists.ignore(pos, expected))

View File

@ -58,6 +58,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
case Type::START_MERGES:
case Type::STOP_TTL_MERGES:
case Type::START_TTL_MERGES:
case Type::STOP_MOVES:
case Type::START_MOVES:
case Type::STOP_FETCHES:
case Type::START_FETCHES:
case Type::STOP_REPLICATED_SENDS:

View File

@ -365,8 +365,8 @@ public:
/** Notify engine about updated dependencies for this storage. */
virtual void updateDependencies() {}
/// Returns data path if storage supports it, empty string otherwise.
virtual String getDataPath() const { return {}; }
/// Returns data paths if storage supports it, empty vector otherwise.
virtual Strings getDataPaths() const { return {}; }
/// Returns ASTExpressionList of partition key expression for storage or nullptr if there is none.
virtual ASTPtr getPartitionKeyAST() const { return nullptr; }
@ -398,6 +398,8 @@ public:
/// Returns names of primary key + secondary sorting columns
virtual Names getSortingKeyColumns() const { return {}; }
/// Returns storage policy if storage supports it
virtual DiskSpace::StoragePolicyPtr getStoragePolicy() const { return {}; }
private:
/// You always need to take the next three locks in this order.

View File

@ -4,7 +4,6 @@
#include <Common/NetException.h>
#include <Common/typeid_cast.h>
#include <IO/HTTPCommon.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Poco/File.h>
#include <ext/scope_guard.h>
#include <Poco/Net/HTTPServerResponse.h>
@ -27,6 +26,7 @@ namespace ErrorCodes
extern const int CANNOT_WRITE_TO_OSTREAM;
extern const int CHECKSUM_DOESNT_MATCH;
extern const int UNKNOWN_TABLE;
extern const int UNKNOWN_PROTOCOL;
extern const int INSECURE_PATH;
}
@ -36,6 +36,9 @@ namespace DataPartsExchange
namespace
{
static constexpr auto REPLICATION_PROTOCOL_VERSION_WITHOUT_PARTS_SIZE = "0";
static constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE = "1";
std::string getEndpointId(const std::string & node_id)
{
return "DataPartsExchange:" + node_id;
@ -53,7 +56,14 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
if (blocker.isCancelled())
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
String client_protocol_version = params.get("client_protocol_version", REPLICATION_PROTOCOL_VERSION_WITHOUT_PARTS_SIZE);
String part_name = params.get("part");
if (client_protocol_version != REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE && client_protocol_version != REPLICATION_PROTOCOL_VERSION_WITHOUT_PARTS_SIZE)
throw Exception("Unsupported fetch protocol version", ErrorCodes::UNKNOWN_PROTOCOL);
const auto data_settings = data.getSettings();
/// Validation of the input that may come from malicious replica.
@ -70,6 +80,8 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
response.setChunkedTransferEncoding(false);
return;
}
response.addCookie({"server_protocol_version", REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE});
++total_sends;
SCOPE_EXIT({--total_sends;});
@ -100,12 +112,16 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
MergeTreeData::DataPart::Checksums data_checksums;
if (client_protocol_version == REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE)
writeBinary(checksums.getTotalSizeOnDisk(), out);
writeBinary(checksums.files.size(), out);
for (const auto & it : checksums.files)
{
String file_name = it.first;
String path = data.getFullPath() + part_name + "/" + file_name;
String path = part->getFullPath() + file_name;
UInt64 size = Poco::File(path).getSize();
@ -183,9 +199,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
uri.setPort(port);
uri.setQueryParameters(
{
{"endpoint", getEndpointId(replica_path)},
{"part", part_name},
{"compress", "false"}
{"endpoint", getEndpointId(replica_path)},
{"part", part_name},
{"client_protocol_version", REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE},
{"compress", "false"}
});
Poco::Net::HTTPBasicCredentials creds{};
@ -205,11 +222,42 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
data_settings->replicated_max_parallel_fetches_for_host
};
auto server_protocol_version = in.getResponseCookie("server_protocol_version", REPLICATION_PROTOCOL_VERSION_WITHOUT_PARTS_SIZE);
DiskSpace::ReservationPtr reservation;
if (server_protocol_version == REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE)
{
size_t sum_files_size;
readBinary(sum_files_size, in);
reservation = data.reserveSpace(sum_files_size);
}
else
{
/// We don't know real size of part because sender server version is too old
reservation = data.makeEmptyReservationOnLargestDisk();
}
return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in);
}
MergeTreeData::MutableDataPartPtr Fetcher::downloadPart(
const String & part_name,
const String & replica_path,
bool to_detached,
const String & tmp_prefix_,
const DiskSpace::ReservationPtr reservation,
PooledReadWriteBufferFromHTTP & in)
{
size_t files;
readBinary(files, in);
static const String TMP_PREFIX = "tmp_fetch_";
String tmp_prefix = tmp_prefix_.empty() ? TMP_PREFIX : tmp_prefix_;
String relative_part_path = String(to_detached ? "detached/" : "") + tmp_prefix + part_name;
String absolute_part_path = Poco::Path(data.getFullPath() + relative_part_path + "/").absolute().toString();
String absolute_part_path = Poco::Path(data.getFullPathOnDisk(reservation->getDisk()) + relative_part_path + "/").absolute().toString();
Poco::File part_file(absolute_part_path);
if (part_file.exists())
@ -219,12 +267,11 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
part_file.createDirectory();
MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(data, part_name);
MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(data, reservation->getDisk(), part_name);
new_data_part->relative_path = relative_part_path;
new_data_part->is_temp = true;
size_t files;
readBinary(files, in);
MergeTreeData::DataPart::Checksums checksums;
for (size_t i = 0; i < files; ++i)
{

View File

@ -6,6 +6,7 @@
#include <IO/HashingWriteBuffer.h>
#include <IO/copyData.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/ReadWriteBufferFromHTTP.h>
namespace DB
@ -64,6 +65,14 @@ public:
ActionBlocker blocker;
private:
MergeTreeData::MutableDataPartPtr downloadPart(
const String & part_name,
const String & replica_path,
bool to_detached,
const String & tmp_prefix_,
const DiskSpace::ReservationPtr reservation,
PooledReadWriteBufferFromHTTP & in);
MergeTreeData & data;
Logger * log;
};

View File

@ -1,10 +0,0 @@
#include <Storages/MergeTree/DiskSpaceMonitor.h>
namespace DB
{
UInt64 DiskSpaceMonitor::reserved_bytes;
UInt64 DiskSpaceMonitor::reservation_count;
std::mutex DiskSpaceMonitor::mutex;
}

View File

@ -1,220 +0,0 @@
#pragma once
#include <mutex>
#include <sys/statvfs.h>
#include <sys/types.h>
#include <sys/stat.h>
#include <unistd.h>
#if defined(__linux__)
#include <cstdio>
#include <mntent.h>
#endif
#include <memory>
#include <filesystem>
#include <boost/noncopyable.hpp>
#include <common/logger_useful.h>
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
#include <Common/formatReadable.h>
#include <Common/CurrentMetrics.h>
namespace CurrentMetrics
{
extern const Metric DiskSpaceReservedForMerge;
}
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_STATVFS;
extern const int NOT_ENOUGH_SPACE;
extern const int SYSTEM_ERROR;
}
/** Determines amount of free space in filesystem.
* Could "reserve" space, for different operations to plan disk space usage.
* Reservations are not separated for different filesystems,
* instead it is assumed, that all reservations are done within same filesystem.
*/
class DiskSpaceMonitor
{
public:
class Reservation : private boost::noncopyable
{
public:
~Reservation()
{
try
{
std::lock_guard lock(DiskSpaceMonitor::mutex);
if (DiskSpaceMonitor::reserved_bytes < size)
{
DiskSpaceMonitor::reserved_bytes = 0;
LOG_ERROR(&Logger::get("DiskSpaceMonitor"), "Unbalanced reservations size; it's a bug");
}
else
{
DiskSpaceMonitor::reserved_bytes -= size;
}
if (DiskSpaceMonitor::reservation_count == 0)
{
LOG_ERROR(&Logger::get("DiskSpaceMonitor"), "Unbalanced reservation count; it's a bug");
}
else
{
--DiskSpaceMonitor::reservation_count;
}
}
catch (...)
{
tryLogCurrentException("~DiskSpaceMonitor");
}
}
/// Change amount of reserved space. When new_size is greater than before, availability of free space is not checked.
void update(UInt64 new_size)
{
std::lock_guard lock(DiskSpaceMonitor::mutex);
DiskSpaceMonitor::reserved_bytes -= size;
size = new_size;
DiskSpaceMonitor::reserved_bytes += size;
}
UInt64 getSize() const
{
return size;
}
Reservation(UInt64 size_)
: size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size)
{
std::lock_guard lock(DiskSpaceMonitor::mutex);
DiskSpaceMonitor::reserved_bytes += size;
++DiskSpaceMonitor::reservation_count;
}
private:
UInt64 size;
CurrentMetrics::Increment metric_increment;
};
using ReservationPtr = std::unique_ptr<Reservation>;
inline static struct statvfs getStatVFS(const std::string & path)
{
struct statvfs fs;
if (statvfs(path.c_str(), &fs) != 0)
throwFromErrnoWithPath("Could not calculate available disk space (statvfs)", path,
ErrorCodes::CANNOT_STATVFS);
return fs;
}
static UInt64 getUnreservedFreeSpace(const std::string & path)
{
struct statvfs fs = getStatVFS(path);
UInt64 res = fs.f_bfree * fs.f_bsize;
/// Heuristic by Michael Kolupaev: reserve 30 MB more, because statvfs shows few megabytes more space than df.
res -= std::min(res, static_cast<UInt64>(30 * (1ul << 20)));
std::lock_guard lock(mutex);
if (reserved_bytes > res)
res = 0;
else
res -= reserved_bytes;
return res;
}
static UInt64 getReservedSpace()
{
std::lock_guard lock(mutex);
return reserved_bytes;
}
static UInt64 getReservationCount()
{
std::lock_guard lock(mutex);
return reservation_count;
}
/// If not enough (approximately) space, throw an exception.
static ReservationPtr reserve(const std::string & path, UInt64 size)
{
UInt64 free_bytes = getUnreservedFreeSpace(path);
if (free_bytes < size)
throw Exception("Not enough free disk space to reserve: " + formatReadableSizeWithBinarySuffix(free_bytes) + " available, "
+ formatReadableSizeWithBinarySuffix(size) + " requested", ErrorCodes::NOT_ENOUGH_SPACE);
return std::make_unique<Reservation>(size);
}
/// Returns mount point of filesystem where absoulte_path (must exist) is located
static std::filesystem::path getMountPoint(std::filesystem::path absolute_path)
{
if (absolute_path.is_relative())
throw Exception("Path is relative. It's a bug.", ErrorCodes::LOGICAL_ERROR);
absolute_path = std::filesystem::canonical(absolute_path);
const auto get_device_id = [](const std::filesystem::path & p)
{
struct stat st;
if (stat(p.c_str(), &st))
throwFromErrnoWithPath("Cannot stat " + p.string(), p.string(), ErrorCodes::SYSTEM_ERROR);
return st.st_dev;
};
/// If /some/path/to/dir/ and /some/path/to/ have different device id,
/// then device which contains /some/path/to/dir/filename is mounted to /some/path/to/dir/
auto device_id = get_device_id(absolute_path);
while (absolute_path.has_relative_path())
{
auto parent = absolute_path.parent_path();
auto parent_device_id = get_device_id(parent);
if (device_id != parent_device_id)
return absolute_path;
absolute_path = parent;
device_id = parent_device_id;
}
return absolute_path;
}
/// Returns name of filesystem mounted to mount_point
#if !defined(__linux__)
[[noreturn]]
#endif
static std::string getFilesystemName([[maybe_unused]] const std::string & mount_point)
{
#if defined(__linux__)
auto mounted_filesystems = setmntent("/etc/mtab", "r");
if (!mounted_filesystems)
throw DB::Exception("Cannot open /etc/mtab to get name of filesystem", ErrorCodes::SYSTEM_ERROR);
mntent fs_info;
constexpr size_t buf_size = 4096; /// The same as buffer used for getmntent in glibc. It can happen that it's not enough
char buf[buf_size];
while (getmntent_r(mounted_filesystems, &fs_info, buf, buf_size) && fs_info.mnt_dir != mount_point)
;
endmntent(mounted_filesystems);
if (fs_info.mnt_dir != mount_point)
throw DB::Exception("Cannot find name of filesystem by mount point " + mount_point, ErrorCodes::SYSTEM_ERROR);
return fs_info.mnt_fsname;
#else
throw DB::Exception("Supported on linux only", ErrorCodes::NOT_IMPLEMENTED);
#endif
}
private:
static UInt64 reserved_bytes;
static UInt64 reservation_count;
static std::mutex mutex;
};
}

View File

@ -27,8 +27,8 @@ void MergeTreeBlockOutputStream::write(const Block & block)
PartLog::addNewPart(storage.global_context, part, watch.elapsed());
/// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'.
if (storage.background_task_handle)
storage.background_task_handle->wake();
if (storage.merging_mutating_task_handle)
storage.merging_mutating_task_handle->wake();
}
}

View File

@ -92,12 +92,14 @@ namespace ErrorCodes
extern const int BAD_DATA_PART_NAME;
extern const int UNKNOWN_SETTING;
extern const int READONLY_SETTING;
extern const int ABORTED;
}
MergeTreeData::MergeTreeData(
const String & database_, const String & table_,
const String & full_path_, const ColumnsDescription & columns_,
const String & database_,
const String & table_,
const ColumnsDescription & columns_,
const IndicesDescription & indices_,
const ConstraintsDescription & constraints_,
Context & context_,
@ -112,19 +114,22 @@ MergeTreeData::MergeTreeData(
bool require_part_metadata_,
bool attach,
BrokenPartCallback broken_part_callback_)
: global_context(context_),
merging_params(merging_params_),
partition_by_ast(partition_by_ast_),
sample_by_ast(sample_by_ast_),
ttl_table_ast(ttl_table_ast_),
require_part_metadata(require_part_metadata_),
database_name(database_), table_name(table_),
full_path(full_path_),
broken_part_callback(broken_part_callback_),
log_name(database_name + "." + table_name), log(&Logger::get(log_name)),
storage_settings(std::move(storage_settings_)),
data_parts_by_info(data_parts_indexes.get<TagByInfo>()),
data_parts_by_state_and_info(data_parts_indexes.get<TagByStateAndInfo>())
: global_context(context_)
, merging_params(merging_params_)
, partition_by_ast(partition_by_ast_)
, sample_by_ast(sample_by_ast_)
, ttl_table_ast(ttl_table_ast_)
, require_part_metadata(require_part_metadata_)
, database_name(database_)
, table_name(table_)
, broken_part_callback(broken_part_callback_)
, log_name(database_name + "." + table_name)
, log(&Logger::get(log_name))
, storage_settings(std::move(storage_settings_))
, storage_policy(context_.getStoragePolicy(getSettings()->storage_policy_name))
, data_parts_by_info(data_parts_indexes.get<TagByInfo>())
, data_parts_by_state_and_info(data_parts_indexes.get<TagByStateAndInfo>())
, parts_mover(this)
{
const auto settings = getSettings();
setProperties(order_by_ast_, primary_key_ast_, columns_, indices_, constraints_);
@ -143,6 +148,7 @@ MergeTreeData::MergeTreeData(
auto syntax = SyntaxAnalyzer(global_context).analyze(sample_by_ast, getColumns().getAllPhysical());
columns_required_for_sampling = syntax->requiredSourceColumns();
}
MergeTreeDataFormatVersion min_format_version(0);
if (!date_column_name.empty())
{
@ -170,22 +176,40 @@ MergeTreeData::MergeTreeData(
setTTLExpressions(columns_.getColumnTTLs(), ttl_table_ast_);
auto path_exists = Poco::File(full_path).exists();
// format_file always contained on any data path
String version_file_path;
/// Creating directories, if not exist.
Poco::File(full_path).createDirectories();
auto paths = getDataPaths();
for (const String & path : paths)
{
Poco::File(path).createDirectories();
Poco::File(path + "detached").createDirectory();
if (Poco::File{path + "format_version.txt"}.exists())
{
if (!version_file_path.empty())
{
LOG_ERROR(log, "Duplication of version file " << version_file_path << " and " << path << "format_file.txt");
throw Exception("Multiple format_version.txt file", ErrorCodes::CORRUPTED_DATA);
}
version_file_path = path + "format_version.txt";
}
}
Poco::File(full_path + "detached").createDirectory();
/// If not choose any
if (version_file_path.empty())
version_file_path = getFullPathOnDisk(storage_policy->getAnyDisk()) + "format_version.txt";
bool version_file_exists = Poco::File(version_file_path).exists();
String version_file_path = full_path + "format_version.txt";
auto version_file_exists = Poco::File(version_file_path).exists();
// When data path or file not exists, ignore the format_version check
if (!attach || !path_exists || !version_file_exists)
if (!attach || !version_file_exists)
{
format_version = min_format_version;
WriteBufferFromFile buf(version_file_path);
writeIntText(format_version.toUnderType(), buf);
}
else if (Poco::File(version_file_path).exists())
else
{
ReadBufferFromFile buf(version_file_path);
UInt32 read_format_version;
@ -194,8 +218,6 @@ MergeTreeData::MergeTreeData(
if (!buf.eof())
throw Exception("Bad version file: " + version_file_path, ErrorCodes::CORRUPTED_DATA);
}
else
format_version = 0;
if (format_version < min_format_version)
{
@ -734,28 +756,38 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
LOG_DEBUG(log, "Loading data parts");
const auto settings = getSettings();
std::vector<std::pair<String, DiskSpace::DiskPtr>> part_names_with_disks;
Strings part_file_names;
Poco::DirectoryIterator end;
for (Poco::DirectoryIterator it(full_path); it != end; ++it)
{
/// Skip temporary directories.
if (startsWith(it.name(), "tmp"))
continue;
part_file_names.push_back(it.name());
auto disks = storage_policy->getDisks();
/// Reversed order to load part from low priority disks firstly.
/// Used for keep part on low priority disk if duplication found
for (auto disk_it = disks.rbegin(); disk_it != disks.rend(); ++disk_it)
{
auto disk_ptr = *disk_it;
for (Poco::DirectoryIterator it(getFullPathOnDisk(disk_ptr)); it != end; ++it)
{
/// Skip temporary directories.
if (startsWith(it.name(), "tmp"))
continue;
part_names_with_disks.emplace_back(it.name(), disk_ptr);
}
}
auto part_lock = lockParts();
data_parts_indexes.clear();
if (part_file_names.empty())
if (part_names_with_disks.empty())
{
LOG_DEBUG(log, "There is no data parts");
return;
}
/// Parallel loading of data parts.
size_t num_threads = std::min(size_t(settings->max_part_loading_threads), part_file_names.size());
size_t num_threads = std::min(size_t(settings->max_part_loading_threads), part_names_with_disks.size());
std::mutex mutex;
@ -768,16 +800,18 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
ThreadPool pool(num_threads);
for (const String & file_name : part_file_names)
for (size_t i = 0; i < part_names_with_disks.size(); ++i)
{
pool.schedule([&]
pool.schedule([&, i]
{
const auto & part_name = part_names_with_disks[i].first;
const auto part_disk_ptr = part_names_with_disks[i].second;
MergeTreePartInfo part_info;
if (!MergeTreePartInfo::tryParsePartName(file_name, &part_info, format_version))
if (!MergeTreePartInfo::tryParsePartName(part_name, &part_info, format_version))
return;
MutableDataPartPtr part = std::make_shared<DataPart>(*this, file_name, part_info);
part->relative_path = file_name;
MutableDataPartPtr part = std::make_shared<DataPart>(*this, part_disk_ptr, part_name, part_info);
part->relative_path = part_name;
bool broken = false;
try
@ -810,7 +844,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
if (part->info.level == 0)
{
/// It is impossible to restore level 0 parts.
LOG_ERROR(log, "Considering to remove broken part " << full_path << file_name << " because it's impossible to repair.");
LOG_ERROR(log, "Considering to remove broken part " << getFullPathOnDisk(part_disk_ptr) << part_name << " because it's impossible to repair.");
std::lock_guard loading_lock(mutex);
broken_parts_to_remove.push_back(part);
}
@ -821,11 +855,11 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
/// delete it.
size_t contained_parts = 0;
LOG_ERROR(log, "Part " << full_path << file_name << " is broken. Looking for parts to replace it.");
LOG_ERROR(log, "Part " << getFullPathOnDisk(part_disk_ptr) << part_name << " is broken. Looking for parts to replace it.");
for (const String & contained_name : part_file_names)
for (const auto & [contained_name, contained_disk_ptr] : part_names_with_disks)
{
if (contained_name == file_name)
if (contained_name == part_name)
continue;
MergeTreePartInfo contained_part_info;
@ -834,20 +868,20 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
if (part->info.contains(contained_part_info))
{
LOG_ERROR(log, "Found part " << full_path << contained_name);
LOG_ERROR(log, "Found part " << getFullPathOnDisk(contained_disk_ptr) << contained_name);
++contained_parts;
}
}
if (contained_parts >= 2)
{
LOG_ERROR(log, "Considering to remove broken part " << full_path << file_name << " because it covers at least 2 other parts");
LOG_ERROR(log, "Considering to remove broken part " << getFullPathOnDisk(part_disk_ptr) << part_name << " because it covers at least 2 other parts");
std::lock_guard loading_lock(mutex);
broken_parts_to_remove.push_back(part);
}
else
{
LOG_ERROR(log, "Detaching broken part " << full_path << file_name
LOG_ERROR(log, "Detaching broken part " << getFullPathOnDisk(part_disk_ptr) << part_name
<< " because it covers less than 2 parts. You need to resolve this manually");
std::lock_guard loading_lock(mutex);
broken_parts_to_detach.push_back(part);
@ -862,7 +896,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
else
has_adaptive_parts.store(true, std::memory_order_relaxed);
part->modification_time = Poco::File(full_path + file_name).getLastModified().epochTime();
part->modification_time = Poco::File(getFullPathOnDisk(part_disk_ptr) + part_name).getLastModified().epochTime();
/// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later
part->state = DataPartState::Committed;
@ -972,25 +1006,30 @@ void MergeTreeData::clearOldTemporaryDirectories(ssize_t custom_directories_life
? current_time - custom_directories_lifetime_seconds
: current_time - settings->temporary_directories_lifetime.totalSeconds();
const auto full_paths = getDataPaths();
/// Delete temporary directories older than a day.
Poco::DirectoryIterator end;
for (Poco::DirectoryIterator it{full_path}; it != end; ++it)
for (auto && full_data_path : full_paths)
{
if (startsWith(it.name(), "tmp_"))
for (Poco::DirectoryIterator it{full_data_path}; it != end; ++it)
{
Poco::File tmp_dir(full_path + it.name());
if (startsWith(it.name(), "tmp_"))
{
Poco::File tmp_dir(full_data_path + it.name());
try
{
if (tmp_dir.isDirectory() && isOldPartDirectory(tmp_dir, deadline))
try
{
LOG_WARNING(log, "Removing temporary directory " << full_path << it.name());
Poco::File(full_path + it.name()).remove(true);
if (tmp_dir.isDirectory() && isOldPartDirectory(tmp_dir, deadline))
{
LOG_WARNING(log, "Removing temporary directory " << full_data_path << it.name());
Poco::File(full_data_path + it.name()).remove(true);
}
}
catch (const Poco::FileNotFoundException &)
{
/// If the file is already deleted, do nothing.
}
}
catch (const Poco::FileNotFoundException &)
{
/// If the file is already deleted, do nothing.
}
}
}
@ -1135,15 +1174,42 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_re
}
}
void MergeTreeData::setPath(const String & new_full_path)
void MergeTreeData::rename(
const String & /*new_path_to_db*/, const String & new_database_name,
const String & new_table_name, TableStructureWriteLockHolder &)
{
if (Poco::File{new_full_path}.exists())
throw Exception{"Target path already exists: " + new_full_path, ErrorCodes::DIRECTORY_ALREADY_EXISTS};
auto old_file_db_name = escapeForFileName(database_name);
auto new_file_db_name = escapeForFileName(new_database_name);
auto old_file_table_name = escapeForFileName(table_name);
auto new_file_table_name = escapeForFileName(new_table_name);
Poco::File(full_path).renameTo(new_full_path);
auto disks = storage_policy->getDisks();
for (const auto & disk : disks)
{
auto new_full_path = disk->getClickHouseDataPath() + new_file_db_name + '/' + new_file_table_name + '/';
if (Poco::File{new_full_path}.exists())
throw Exception{"Target path already exists: " + new_full_path, ErrorCodes::DIRECTORY_ALREADY_EXISTS};
}
for (const auto & disk : disks)
{
auto full_path = disk->getClickHouseDataPath() + old_file_db_name + '/' + old_file_table_name + '/';
auto new_db_path = disk->getClickHouseDataPath() + new_file_db_name + '/';
Poco::File db_file{new_db_path};
if (!db_file.exists())
db_file.createDirectory();
auto new_full_path = new_db_path + new_file_table_name + '/';
Poco::File{full_path}.renameTo(new_full_path);
}
global_context.dropCaches();
full_path = new_full_path;
database_name = new_database_name;
table_name = new_table_name;
}
void MergeTreeData::dropAllData()
@ -1166,7 +1232,10 @@ void MergeTreeData::dropAllData()
/// Removing of each data part before recursive removal of directory is to speed-up removal, because there will be less number of syscalls.
clearPartsFromFilesystem(all_parts);
Poco::File(full_path).remove(true);
auto full_paths = getDataPaths();
for (auto && full_data_path : full_paths)
Poco::File(full_data_path).remove(true);
LOG_TRACE(log, "dropAllData: done.");
}
@ -1554,7 +1623,7 @@ void MergeTreeData::alterDataPart(
exception_message
<< ") need to be "
<< (forbidden_because_of_modify ? "modified" : "removed")
<< " in part " << part->name << " of table at " << full_path << ". Aborting just in case."
<< " in part " << part->name << " of table at " << part->getFullPath() << ". Aborting just in case."
<< " If it is not an error, you could increase merge_tree/"
<< (forbidden_because_of_modify ? "max_files_to_modify_in_alter_columns" : "max_files_to_remove_in_alter_columns")
<< " parameter in configuration file (current value: "
@ -1592,10 +1661,11 @@ void MergeTreeData::alterDataPart(
* will have old name of shared offsets for arrays.
*/
IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets;
MergedColumnOnlyOutputStream out(
*this,
in.getHeader(),
full_path + part->name + '/',
part->getFullPath(),
true /* sync */,
compression_codec,
true /* skip_offsets */,
@ -1629,7 +1699,7 @@ void MergeTreeData::alterDataPart(
if (!part->checksums.empty())
{
transaction->new_checksums = new_checksums;
WriteBufferFromFile checksums_file(full_path + part->name + "/checksums.txt.tmp", 4096);
WriteBufferFromFile checksums_file(part->getFullPath() + "checksums.txt.tmp", 4096);
new_checksums.write(checksums_file);
transaction->rename_map["checksums.txt.tmp"] = "checksums.txt";
}
@ -1637,7 +1707,7 @@ void MergeTreeData::alterDataPart(
/// Write the new column list to the temporary file.
{
transaction->new_columns = new_columns.filter(part->columns.getNames());
WriteBufferFromFile columns_file(full_path + part->name + "/columns.txt.tmp", 4096);
WriteBufferFromFile columns_file(part->getFullPath() + "columns.txt.tmp", 4096);
transaction->new_columns.writeText(columns_file);
transaction->rename_map["columns.txt.tmp"] = "columns.txt";
}
@ -1721,7 +1791,7 @@ void MergeTreeData::AlterDataPartTransaction::commit()
{
std::unique_lock<std::shared_mutex> lock(data_part->columns_lock);
String path = data_part->storage.full_path + data_part->name + "/";
String path = data_part->getFullPath();
/// NOTE: checking that a file exists before renaming or deleting it
/// is justified by the fact that, when converting an ordinary column
@ -1811,6 +1881,19 @@ MergeTreeData::AlterDataPartTransaction::~AlterDataPartTransaction()
void MergeTreeData::PartsTemporaryRename::addPart(const String & old_name, const String & new_name)
{
old_and_new_names.push_back({old_name, new_name});
const auto paths = storage.getDataPaths();
for (const auto & full_path : paths)
{
for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
{
String name = it.name();
if (name == old_name)
{
old_part_name_to_full_path[old_name] = full_path;
break;
}
}
}
}
void MergeTreeData::PartsTemporaryRename::tryRenameAll()
@ -1823,7 +1906,8 @@ void MergeTreeData::PartsTemporaryRename::tryRenameAll()
const auto & names = old_and_new_names[i];
if (names.first.empty() || names.second.empty())
throw DB::Exception("Empty part name. Most likely it's a bug.", ErrorCodes::INCORRECT_FILE_NAME);
Poco::File(base_dir + names.first).renameTo(base_dir + names.second);
const auto full_path = old_part_name_to_full_path[names.first] + source_dir; /// old_name
Poco::File(full_path + names.first).renameTo(full_path + names.second);
}
catch (...)
{
@ -1843,9 +1927,11 @@ MergeTreeData::PartsTemporaryRename::~PartsTemporaryRename()
{
if (names.first.empty())
continue;
try
{
Poco::File(base_dir + names.second).renameTo(base_dir + names.first);
const auto full_path = old_part_name_to_full_path[names.first] + source_dir; /// old_name
Poco::File(full_path + names.second).renameTo(full_path + names.first);
}
catch (...)
{
@ -2425,6 +2511,29 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(
return nullptr;
}
void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy)
{
auto lock = lockParts();
for (const auto & original_active_part : getDataPartsStateRange(DataPartState::Committed))
{
if (part_copy->name == original_active_part->name)
{
auto active_part_it = data_parts_by_info.find(original_active_part->info);
if (active_part_it == data_parts_by_info.end())
throw Exception("Cannot swap part '" + part_copy->name + "', no such active part.", ErrorCodes::NO_SUCH_DATA_PART);
modifyPartState(original_active_part, DataPartState::DeleteOnDestroy);
data_parts_indexes.erase(active_part_it);
auto part_it = data_parts_indexes.insert(part_copy).first;
modifyPartState(part_it, DataPartState::Committed);
return;
}
}
throw Exception("Cannot swap part '" + part_copy->name + "', no such active part.", ErrorCodes::NO_SUCH_DATA_PART);
}
MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const MergeTreePartInfo & part_info)
{
auto lock = lockParts();
@ -2472,9 +2581,9 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_na
}
MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const String & relative_path)
MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const DiskSpace::DiskPtr & disk, const String & relative_path)
{
MutableDataPartPtr part = std::make_shared<DataPart>(*this, Poco::Path(relative_path).getFileName());
MutableDataPartPtr part = std::make_shared<DataPart>(*this, disk, Poco::Path(relative_path).getFileName());
part->relative_path = relative_path;
loadPartAndFixMetadata(part);
return part;
@ -2591,6 +2700,74 @@ void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String &
}
void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & name, bool moving_part, const Context & context)
{
String partition_id;
if (moving_part)
partition_id = partition->as<ASTLiteral &>().value.safeGet<String>();
else
partition_id = getPartitionIDFromQuery(partition, context);
DataPartsVector parts;
if (moving_part)
{
parts.push_back(getActiveContainingPart(partition_id));
if (!parts.back())
throw Exception("Part " + partition_id + " is not exists or not active", ErrorCodes::NO_SUCH_DATA_PART);
}
else
parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
auto disk = storage_policy->getDiskByName(name);
if (!disk)
throw Exception("Disk " + name + " does not exists on policy " + storage_policy->getName(), ErrorCodes::UNKNOWN_DISK);
for (const auto & part : parts)
{
if (part->disk->getName() == disk->getName())
throw Exception("Part " + part->name + " already on disk " + name, ErrorCodes::UNKNOWN_DISK);
}
if (!movePartsToSpace(parts, std::static_pointer_cast<const DiskSpace::Space>(disk)))
throw Exception("Cannot move parts because moves are manually disabled.", ErrorCodes::ABORTED);
}
void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String & name, bool moving_part, const Context & context)
{
String partition_id;
if (moving_part)
partition_id = partition->as<ASTLiteral &>().value.safeGet<String>();
else
partition_id = getPartitionIDFromQuery(partition, context);
DataPartsVector parts;
if (moving_part)
{
parts.push_back(getActiveContainingPart(partition_id));
if (!parts.back())
throw Exception("Part " + partition_id + " is not exists or not active", ErrorCodes::NO_SUCH_DATA_PART);
}
else
parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
auto volume = storage_policy->getVolumeByName(name);
if (!volume)
throw Exception("Volume " + name + " does not exists on policy " + storage_policy->getName(), ErrorCodes::UNKNOWN_DISK);
for (const auto & part : parts)
for (const auto & disk : volume->disks)
if (part->disk->getName() == disk->getName())
throw Exception("Part " + part->name + " already on volume '" + name + "'", ErrorCodes::UNKNOWN_DISK);
if (!movePartsToSpace(parts, std::static_pointer_cast<const DiskSpace::Space>(volume)))
throw Exception("Cannot move parts because moves are manually disabled.", ErrorCodes::ABORTED);
}
String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context & context)
{
const auto & partition_ast = ast->as<ASTPartition &>();
@ -2714,15 +2891,18 @@ MergeTreeData::getDetachedParts() const
{
std::vector<DetachedPartInfo> res;
for (Poco::DirectoryIterator it(full_path + "detached");
it != Poco::DirectoryIterator(); ++it)
for (const String & path : getDataPaths())
{
auto dir_name = it.name();
for (Poco::DirectoryIterator it(path + "detached");
it != Poco::DirectoryIterator(); ++it)
{
auto dir_name = it.name();
res.emplace_back();
auto & part = res.back();
res.emplace_back();
auto & part = res.back();
DetachedPartInfo::tryParseDetachedPartName(dir_name, part, format_version);
DetachedPartInfo::tryParseDetachedPartName(dir_name, part, format_version);
}
}
return res;
}
@ -2730,10 +2910,11 @@ MergeTreeData::getDetachedParts() const
void MergeTreeData::validateDetachedPartName(const String & name) const
{
if (name.find('/') != std::string::npos || name == "." || name == "..")
throw DB::Exception("Invalid part name", ErrorCodes::INCORRECT_FILE_NAME);
throw DB::Exception("Invalid part name '" + name + "'", ErrorCodes::INCORRECT_FILE_NAME);
Poco::File detached_part_dir(full_path + "detached/" + name);
if (!detached_part_dir.exists())
String full_path = getFullPathForPart(name, "detached/");
if (full_path.empty() || !Poco::File(full_path + name).exists())
throw DB::Exception("Detached part \"" + name + "\" not found" , ErrorCodes::BAD_DATA_PART_NAME);
if (startsWith(name, "attaching_") || startsWith(name, "deleting_"))
@ -2744,7 +2925,7 @@ void MergeTreeData::validateDetachedPartName(const String & name) const
void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, const Context & context)
{
PartsTemporaryRename renamed_parts(*this, full_path + "detached/");
PartsTemporaryRename renamed_parts(*this, "detached/");
if (part)
{
@ -2766,11 +2947,11 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, const Cont
renamed_parts.tryRenameAll();
for (auto & names : renamed_parts.old_and_new_names)
for (auto & [old_name, new_name] : renamed_parts.old_and_new_names)
{
Poco::File(renamed_parts.base_dir + names.second).remove(true);
LOG_DEBUG(log, "Dropped detached part " << names.first);
names.first.clear();
Poco::File(renamed_parts.old_part_name_to_full_path[old_name] + "detached/" + new_name).remove(true);
LOG_DEBUG(log, "Dropped detached part " << old_name);
old_name.clear();
}
}
@ -2779,6 +2960,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const
{
String source_dir = "detached/";
std::map<String, DiskSpace::DiskPtr> name_to_disk;
/// Let's compose a list of parts that should be added.
if (attach_part)
{
@ -2792,35 +2974,44 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const
LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir);
ActiveDataPartSet active_parts(format_version);
std::set<String> part_names;
for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
const auto disks = storage_policy->getDisks();
for (const DiskSpace::DiskPtr & disk : disks)
{
String name = it.name();
MergeTreePartInfo part_info;
// TODO what if name contains "_tryN" suffix?
/// Parts with prefix in name (e.g. attaching_1_3_3_0, deleting_1_3_3_0) will be ignored
if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version))
continue;
if (part_info.partition_id != partition_id)
continue;
LOG_DEBUG(log, "Found part " << name);
active_parts.add(name);
part_names.insert(name);
const auto full_path = getFullPathOnDisk(disk);
for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
{
const String & name = it.name();
MergeTreePartInfo part_info;
// TODO what if name contains "_tryN" suffix?
/// Parts with prefix in name (e.g. attaching_1_3_3_0, deleting_1_3_3_0) will be ignored
if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version)
|| part_info.partition_id != partition_id)
{
continue;
}
LOG_DEBUG(log, "Found part " << name);
active_parts.add(name);
name_to_disk[name] = disk;
}
}
LOG_DEBUG(log, active_parts.size() << " of them are active");
/// Inactive parts rename so they can not be attached in case of repeated ATTACH.
for (const auto & name : part_names)
for (const auto & [name, disk] : name_to_disk)
{
String containing_part = active_parts.getContainingPart(name);
if (!containing_part.empty() && containing_part != name)
{
auto full_path = getFullPathOnDisk(disk);
// TODO maybe use PartsTemporaryRename here?
Poco::File(full_path + source_dir + name).renameTo(full_path + source_dir + "inactive_" + name);
Poco::File(full_path + source_dir + name)
.renameTo(full_path + source_dir + "inactive_" + name);
}
else
renamed_parts.addPart(name, "attaching_" + name);
}
}
/// Try to rename all parts before attaching to prevent race with DROP DETACHED and another ATTACH.
renamed_parts.tryRenameAll();
@ -2831,7 +3022,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const
for (const auto & part_names : renamed_parts.old_and_new_names)
{
LOG_DEBUG(log, "Checking part " << part_names.second);
MutableDataPartPtr part = std::make_shared<DataPart>(*this, part_names.first);
MutableDataPartPtr part = std::make_shared<DataPart>(*this, name_to_disk[part_names.first], part_names.first);
part->relative_path = source_dir + part_names.second;
loadPartAndFixMetadata(part);
loaded_parts.push_back(part);
@ -2840,6 +3031,20 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const
return loaded_parts;
}
DiskSpace::ReservationPtr MergeTreeData::reserveSpace(UInt64 expected_size)
{
constexpr UInt64 RESERVATION_MIN_ESTIMATION_SIZE = 1u * 1024u * 1024u; /// 1MB
expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size);
auto reservation = storage_policy->reserve(expected_size);
if (reservation)
return reservation;
throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(expected_size) + ", not enought space.",
ErrorCodes::NOT_ENOUGH_SPACE);
}
MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affordable_states) const
{
DataParts res;
@ -3019,7 +3224,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPart(const Merg
String dst_part_name = src_part->getNewName(dst_part_info);
String tmp_dst_part_name = tmp_part_prefix + dst_part_name;
Poco::Path dst_part_absolute_path = Poco::Path(full_path + tmp_dst_part_name).absolute();
auto reservation = reserveSpace(src_part->bytes_on_disk);
String dst_part_path = getFullPathOnDisk(reservation->getDisk());
Poco::Path dst_part_absolute_path = Poco::Path(dst_part_path + tmp_dst_part_name).absolute();
Poco::Path src_part_absolute_path = Poco::Path(src_part->getFullPath()).absolute();
if (Poco::File(dst_part_absolute_path).exists())
@ -3028,7 +3235,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPart(const Merg
LOG_DEBUG(log, "Cloning part " << src_part_absolute_path.toString() << " to " << dst_part_absolute_path.toString());
localBackup(src_part_absolute_path, dst_part_absolute_path);
MergeTreeData::MutableDataPartPtr dst_data_part = std::make_shared<MergeTreeData::DataPart>(*this, dst_part_name, dst_part_info);
MergeTreeData::MutableDataPartPtr dst_data_part = std::make_shared<MergeTreeData::DataPart>(
*this, reservation->getDisk(), dst_part_name, dst_part_info);
dst_data_part->relative_path = tmp_dst_part_name;
dst_data_part->is_temp = true;
@ -3037,18 +3246,49 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPart(const Merg
return dst_data_part;
}
String MergeTreeData::getFullPathOnDisk(const DiskSpace::DiskPtr & disk) const
{
return disk->getClickHouseDataPath() + escapeForFileName(database_name) + '/' + escapeForFileName(table_name) + '/';
}
DiskSpace::DiskPtr MergeTreeData::getDiskForPart(const String & part_name, const String & relative_path) const
{
const auto disks = storage_policy->getDisks();
for (const DiskSpace::DiskPtr & disk : disks)
{
const auto disk_path = getFullPathOnDisk(disk);
for (Poco::DirectoryIterator it = Poco::DirectoryIterator(disk_path + relative_path); it != Poco::DirectoryIterator(); ++it)
if (it.name() == part_name)
return disk;
}
return nullptr;
}
String MergeTreeData::getFullPathForPart(const String & part_name, const String & relative_path) const
{
auto disk = getDiskForPart(part_name, relative_path);
if (disk)
return getFullPathOnDisk(disk) + relative_path;
return "";
}
Strings MergeTreeData::getDataPaths() const
{
Strings res;
auto disks = storage_policy->getDisks();
for (const auto & disk : disks)
res.push_back(getFullPathOnDisk(disk));
return res;
}
void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & with_name, const Context & context)
{
String clickhouse_path = Poco::Path(context.getPath()).makeAbsolute().toString();
String shadow_path = clickhouse_path + "shadow/";
Poco::File(shadow_path).createDirectories();
String backup_path = shadow_path
+ (!with_name.empty()
? escapeForFileName(with_name)
: toString(Increment(shadow_path + "increment.txt").get(true)))
+ "/";
LOG_DEBUG(log, "Snapshot will be placed at " + backup_path);
String default_shadow_path = clickhouse_path + "shadow/";
Poco::File(default_shadow_path).createDirectories();
auto increment = Increment(default_shadow_path + "increment.txt").get(true);
/// Acquire a snapshot of active data parts to prevent removing while doing backup.
const auto data_parts = getDataParts();
@ -3059,14 +3299,19 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String &
if (!matcher(part))
continue;
LOG_DEBUG(log, "Freezing part " << part->name);
String shadow_path = part->disk->getPath() + "shadow/";
Poco::File(shadow_path).createDirectories();
String backup_path = shadow_path
+ (!with_name.empty()
? escapeForFileName(with_name)
: toString(increment))
+ "/";
LOG_DEBUG(log, "Freezing part " << part->name << " snapshot will be placed at " + backup_path);
String part_absolute_path = Poco::Path(part->getFullPath()).absolute().toString();
if (!startsWith(part_absolute_path, clickhouse_path))
throw Exception("Part path " + part_absolute_path + " is not inside " + clickhouse_path, ErrorCodes::LOGICAL_ERROR);
String backup_part_absolute_path = part_absolute_path;
backup_part_absolute_path.replace(0, clickhouse_path.size(), backup_path);
String backup_part_absolute_path = backup_path + "data/" + getDatabaseName() + "/" + getTableName() + "/" + part->relative_path;
localBackup(part_absolute_path, backup_part_absolute_path);
part->is_frozen.store(true, std::memory_order_relaxed);
++parts_processed;
@ -3089,4 +3334,199 @@ bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const
return true;
}
void MergeTreeData::writePartLog(
PartLogElement::Type type,
const ExecutionStatus & execution_status,
UInt64 elapsed_ns,
const String & new_part_name,
const DataPartPtr & result_part,
const DataPartsVector & source_parts,
const MergeListEntry * merge_entry)
try
{
auto part_log = global_context.getPartLog(database_name);
if (!part_log)
return;
PartLogElement part_log_elem;
part_log_elem.event_type = type;
part_log_elem.error = static_cast<UInt16>(execution_status.code);
part_log_elem.exception = execution_status.message;
part_log_elem.event_time = time(nullptr);
/// TODO: Stop stopwatch in outer code to exclude ZK timings and so on
part_log_elem.duration_ms = elapsed_ns / 10000000;
part_log_elem.database_name = database_name;
part_log_elem.table_name = table_name;
part_log_elem.partition_id = MergeTreePartInfo::fromPartName(new_part_name, format_version).partition_id;
part_log_elem.part_name = new_part_name;
if (result_part)
{
part_log_elem.path_on_disk = result_part->getFullPath();
part_log_elem.bytes_compressed_on_disk = result_part->bytes_on_disk;
part_log_elem.rows = result_part->rows_count;
}
part_log_elem.source_part_names.reserve(source_parts.size());
for (const auto & source_part : source_parts)
part_log_elem.source_part_names.push_back(source_part->name);
if (merge_entry)
{
part_log_elem.rows_read = (*merge_entry)->rows_read;
part_log_elem.bytes_read_uncompressed = (*merge_entry)->bytes_read_uncompressed;
part_log_elem.rows = (*merge_entry)->rows_written;
part_log_elem.bytes_uncompressed = (*merge_entry)->bytes_written_uncompressed;
}
part_log->add(part_log_elem);
}
catch (...)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
}
MergeTreeData::CurrentlyMovingPartsTagger::CurrentlyMovingPartsTagger(MergeTreeMovingParts && moving_parts_, MergeTreeData & data_)
: parts_to_move(std::move(moving_parts_)), data(data_)
{
for (const auto & moving_part : parts_to_move)
if (!data.currently_moving_parts.emplace(moving_part.part).second)
throw Exception("Cannot move part '" + moving_part.part->name + "'. It's already moving.", ErrorCodes::LOGICAL_ERROR);
}
MergeTreeData::CurrentlyMovingPartsTagger::~CurrentlyMovingPartsTagger()
{
std::lock_guard lock(data.moving_parts_mutex);
for (const auto & moving_part : parts_to_move)
{
/// Something went completely wrong
if (!data.currently_moving_parts.count(moving_part.part))
std::terminate();
data.currently_moving_parts.erase(moving_part.part);
}
}
bool MergeTreeData::selectPartsAndMove()
{
if (parts_mover.moves_blocker.isCancelled())
return false;
auto moving_tagger = selectPartsForMove();
if (moving_tagger.parts_to_move.empty())
return false;
return moveParts(std::move(moving_tagger));
}
bool MergeTreeData::movePartsToSpace(const DataPartsVector & parts, DiskSpace::SpacePtr space)
{
if (parts_mover.moves_blocker.isCancelled())
return false;
auto moving_tagger = checkPartsForMove(parts, space);
if (moving_tagger.parts_to_move.empty())
return false;
return moveParts(std::move(moving_tagger));
}
MergeTreeData::CurrentlyMovingPartsTagger MergeTreeData::selectPartsForMove()
{
MergeTreeMovingParts parts_to_move;
auto can_move = [this](const DataPartPtr & part, String * reason) -> bool
{
if (partIsAssignedToBackgroundOperation(part))
{
*reason = "part already assigned to background operation.";
return false;
}
if (currently_moving_parts.count(part))
{
*reason = "part is already moving.";
return false;
}
return true;
};
std::lock_guard moving_lock(moving_parts_mutex);
parts_mover.selectPartsForMove(parts_to_move, can_move, moving_lock);
return CurrentlyMovingPartsTagger(std::move(parts_to_move), *this);
}
MergeTreeData::CurrentlyMovingPartsTagger MergeTreeData::checkPartsForMove(const DataPartsVector & parts, DiskSpace::SpacePtr space)
{
std::lock_guard moving_lock(moving_parts_mutex);
MergeTreeMovingParts parts_to_move;
for (const auto & part : parts)
{
auto reservation = space->reserve(part->bytes_on_disk);
if (!reservation)
throw Exception("Move is not possible. Not enough space on '" + space->getName() + "'", ErrorCodes::NOT_ENOUGH_SPACE);
auto & reserved_disk = reservation->getDisk();
String path_to_clone = getFullPathOnDisk(reserved_disk);
if (Poco::File(path_to_clone + part->name).exists())
throw Exception(
"Move is not possible: " + path_to_clone + part->name + " already exists",
ErrorCodes::DIRECTORY_ALREADY_EXISTS);
if (currently_moving_parts.count(part) || partIsAssignedToBackgroundOperation(part))
throw Exception(
"Cannot move part '" + part->name + "' because it's participating in background process",
ErrorCodes::PART_IS_TEMPORARILY_LOCKED);
parts_to_move.emplace_back(part, std::move(reservation));
}
return CurrentlyMovingPartsTagger(std::move(parts_to_move), *this);
}
bool MergeTreeData::moveParts(CurrentlyMovingPartsTagger && moving_tagger)
{
LOG_INFO(log, "Got " << moving_tagger.parts_to_move.size() << " parts to move.");
for (const auto & moving_part : moving_tagger.parts_to_move)
{
Stopwatch stopwatch;
DataPartPtr cloned_part;
auto write_part_log = [&](const ExecutionStatus & execution_status)
{
writePartLog(
PartLogElement::Type::MOVE_PART,
execution_status,
stopwatch.elapsed(),
moving_part.part->name,
cloned_part,
{moving_part.part},
nullptr);
};
try
{
cloned_part = parts_mover.clonePart(moving_part);
parts_mover.swapClonedPart(cloned_part);
write_part_log({});
}
catch (...)
{
write_part_log(ExecutionStatus::fromCurrentException());
if (cloned_part)
cloned_part->remove();
throw;
}
}
return true;
}
}

View File

@ -8,6 +8,7 @@
#include <Storages/MergeTree/MergeTreePartInfo.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <Storages/MergeTree/MergeTreeMutationStatus.h>
#include <Storages/MergeTree/MergeList.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFile.h>
@ -16,6 +17,9 @@
#include <DataStreams/GraphiteRollupSortedBlockInputStream.h>
#include <Storages/MergeTree/MergeTreeDataPart.h>
#include <Storages/IndicesDescription.h>
#include <Storages/MergeTree/MergeTreePartsMover.h>
#include <Interpreters/PartLog.h>
#include <Common/DiskSpaceMonitor.h>
#include <boost/multi_index_container.hpp>
#include <boost/multi_index/ordered_index.hpp>
@ -26,7 +30,9 @@
namespace DB
{
class MergeListEntry;
class AlterCommands;
class MergeTreePartsMover;
namespace ErrorCodes
{
@ -251,7 +257,13 @@ public:
struct PartsTemporaryRename : private boost::noncopyable
{
PartsTemporaryRename(const MergeTreeData & storage_, const String & base_dir_) : storage(storage_), base_dir(base_dir_) {}
PartsTemporaryRename(
const MergeTreeData & storage_,
const String & source_dir_)
: storage(storage_)
, source_dir(source_dir_)
{
}
void addPart(const String & old_name, const String & new_name);
@ -262,8 +274,9 @@ public:
~PartsTemporaryRename();
const MergeTreeData & storage;
String base_dir;
const String source_dir;
std::vector<std::pair<String, String>> old_and_new_names;
std::unordered_map<String, String> old_part_name_to_full_path;
bool renamed = false;
};
@ -302,7 +315,7 @@ public:
String getModeName() const;
};
/// Attach the table corresponding to the directory in full_path (must end with /), with the given columns.
/// Attach the table corresponding to the directory in full_path inside policy (must end with /), with the given columns.
/// Correctness of names and paths is not checked.
///
/// date_column_name - if not empty, the name of the Date column used for partitioning by month.
@ -319,7 +332,6 @@ public:
/// require_part_metadata - should checksums.txt and columns.txt exist in the part directory.
/// attach - whether the existing table is attached or the new table is created.
MergeTreeData(const String & database_, const String & table_,
const String & full_path_,
const ColumnsDescription & columns_,
const IndicesDescription & indices_,
const ConstraintsDescription & constraints_,
@ -348,6 +360,8 @@ public:
Names getColumnsRequiredForFinal() const override { return sorting_key_expr->getRequiredColumns(); }
Names getSortingKeyColumns() const override { return sorting_key_columns; }
DiskSpace::StoragePolicyPtr getStoragePolicy() const override { return storage_policy; }
bool supportsPrewhere() const override { return true; }
bool supportsSampling() const override { return sample_by_ast != nullptr; }
@ -393,7 +407,6 @@ public:
/// Load the set of data parts from disk. Call once - immediately after the object is created.
void loadDataParts(bool skip_sanity_checks);
String getFullPath() const { return full_path; }
String getLogName() const { return log_name; }
Int64 getMaxBlockNumber() const;
@ -424,7 +437,11 @@ public:
/// Returns a committed part with the given name or a part containing it. If there is no such part, returns nullptr.
DataPartPtr getActiveContainingPart(const String & part_name);
DataPartPtr getActiveContainingPart(const MergeTreePartInfo & part_info);
DataPartPtr getActiveContainingPart(const MergeTreePartInfo & part_info, DataPartState state, DataPartsLock &lock);
DataPartPtr getActiveContainingPart(const MergeTreePartInfo & part_info, DataPartState state, DataPartsLock & lock);
/// Swap part with it's identical copy (possible with another path on another disk).
/// If original part is not active or doesn't exist exception will be thrown.
void swapActivePart(MergeTreeData::DataPartPtr part_copy);
/// Returns all parts in specified partition
DataPartsVector getDataPartsVectorInPartition(DataPartState state, const String & partition_id);
@ -518,7 +535,8 @@ public:
/// Moves the entire data directory.
/// Flushes the uncompressed blocks cache and the marks cache.
/// Must be called with locked lockStructureForAlter().
void setPath(const String & full_path);
void rename(const String & new_path_to_db, const String & new_database_name,
const String & new_table_name, TableStructureWriteLockHolder &) override;
/// Check if the ALTER can be performed:
/// - all needed columns are present.
@ -570,7 +588,7 @@ public:
bool hasAnyColumnTTL() const { return !ttl_entries_by_name.empty(); }
/// Check that the part is not broken and calculate the checksums for it if they are not present.
MutableDataPartPtr loadPartAndFixMetadata(const String & relative_path);
MutableDataPartPtr loadPartAndFixMetadata(const DiskSpace::DiskPtr & disk, const String & relative_path);
void loadPartAndFixMetadata(MutableDataPartPtr part);
/** Create local backup (snapshot) for parts with specified prefix.
@ -579,6 +597,14 @@ public:
*/
void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context, TableStructureReadLockHolder & table_lock_holder);
public:
/// Moves partition to specified Disk
void movePartitionToDisk(const ASTPtr & partition, const String & name, bool moving_part, const Context & context);
/// Moves partition to specified Volume
void movePartitionToVolume(const ASTPtr & partition, const String & name, bool moving_part, const Context & context);
size_t getColumnCompressedSize(const std::string & name) const
{
auto lock = lockParts();
@ -607,8 +633,8 @@ public:
/// Tables structure should be locked.
MergeTreeData & checkStructureAndGetMergeTreeData(const StoragePtr & source_table) const;
MergeTreeData::MutableDataPartPtr cloneAndLoadDataPart(const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix,
const MergeTreePartInfo & dst_part_info);
MergeTreeData::MutableDataPartPtr cloneAndLoadDataPart(
const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info);
virtual std::vector<MergeTreeMutationStatus> getMutationsStatus() const = 0;
@ -629,6 +655,25 @@ public:
return storage_settings.get();
}
/// Get table path on disk
String getFullPathOnDisk(const DiskSpace::DiskPtr & disk) const;
/// Get disk for part. Looping through directories on FS because some parts maybe not in
/// active dataparts set (detached)
DiskSpace::DiskPtr getDiskForPart(const String & part_name, const String & relative_path = "") const;
/// Get full path for part. Uses getDiskForPart and returns the full path
String getFullPathForPart(const String & part_name, const String & relative_path = "") const;
Strings getDataPaths() const override;
/// Reserves space at least 1MB
DiskSpace::ReservationPtr reserveSpace(UInt64 expected_size);
/// Choose disk with max available free space
/// Reserves 0 bytes
DiskSpace::ReservationPtr makeEmptyReservationOnLargestDisk() { return storage_policy->makeEmptyReservationOnLargestDisk(); }
MergeTreeDataFormatVersion format_version;
Context global_context;
@ -686,6 +731,16 @@ public:
bool has_non_adaptive_index_granularity_parts = false;
/// Parts that currently moving from disk/volume to another.
/// This set have to be used with `currently_processing_in_background_mutex`.
/// Moving may conflict with merges and mutations, but this is OK, because
/// if we decide to move some part to another disk, than we
/// assuredly will choose this disk for containing part, which will appear
/// as result of merge or mutation.
DataParts currently_moving_parts;
/// Mutex for currently_moving_parts
mutable std::mutex moving_parts_mutex;
protected:
@ -705,7 +760,7 @@ protected:
String database_name;
String table_name;
String full_path;
/// Current column sizes in compressed and uncompressed form.
ColumnSizeByName column_sizes;
@ -720,6 +775,8 @@ protected:
/// Use get and set to receive readonly versions.
MultiVersion<MergeTreeSettings> storage_settings;
DiskSpace::StoragePolicyPtr storage_policy;
/// Work with data parts
struct TagByInfo{};
@ -757,6 +814,8 @@ protected:
DataPartsIndexes::index<TagByInfo>::type & data_parts_by_info;
DataPartsIndexes::index<TagByStateAndInfo>::type & data_parts_by_state_and_info;
MergeTreePartsMover parts_mover;
using DataPartIteratorByInfo = DataPartsIndexes::index<TagByInfo>::type::iterator;
using DataPartIteratorByStateAndInfo = DataPartsIndexes::index<TagByStateAndInfo>::type::iterator;
@ -801,7 +860,6 @@ protected:
throw Exception("Can't modify " + (*it)->getNameWithState(), ErrorCodes::LOGICAL_ERROR);
}
/// Used to serialize calls to grabOldParts.
std::mutex grab_old_parts_mutex;
/// The same for clearOldTemporaryDirectories.
@ -855,6 +913,48 @@ protected:
bool canReplacePartition(const DataPartPtr & data_part) const;
void writePartLog(
PartLogElement::Type type,
const ExecutionStatus & execution_status,
UInt64 elapsed_ns,
const String & new_part_name,
const DataPartPtr & result_part,
const DataPartsVector & source_parts,
const MergeListEntry * merge_entry);
/// If part is assigned to merge or mutation (possibly replicated)
/// Should be overriden by childs, because they can have different
/// mechanisms for parts locking
virtual bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const = 0;
/// Moves part to specified space, used in ALTER ... MOVE ... queries
bool movePartsToSpace(const DataPartsVector & parts, DiskSpace::SpacePtr space);
/// Selects parts for move and moves them, used in background process
bool selectPartsAndMove();
private:
/// RAII Wrapper for atomic work with currently moving parts
/// Acuire them in constructor and remove them in destructor
/// Uses data.currently_moving_parts_mutex
struct CurrentlyMovingPartsTagger
{
MergeTreeMovingParts parts_to_move;
MergeTreeData & data;
CurrentlyMovingPartsTagger(MergeTreeMovingParts && moving_parts_, MergeTreeData & data_);
CurrentlyMovingPartsTagger(const CurrentlyMovingPartsTagger & other) = delete;
~CurrentlyMovingPartsTagger();
};
/// Move selected parts to corresponding disks
bool moveParts(CurrentlyMovingPartsTagger && parts_to_move);
/// Select parts for move and disks for them. Used in background moving processes.
CurrentlyMovingPartsTagger selectPartsForMove();
/// Check selected parts for movements. Used by ALTER ... MOVE queries.
CurrentlyMovingPartsTagger checkPartsForMove(const DataPartsVector & parts, DiskSpace::SpacePtr space);
};
}

View File

@ -3,13 +3,12 @@
#include <Storages/MergeTree/MergeTreeSequentialBlockInputStream.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Storages/MergeTree/MergedColumnOnlyOutputStream.h>
#include <Storages/MergeTree/DiskSpaceMonitor.h>
#include <Common/DiskSpaceMonitor.h>
#include <Storages/MergeTree/SimpleMergeSelector.h>
#include <Storages/MergeTree/AllMergeSelector.h>
#include <Storages/MergeTree/TTLMergeSelector.h>
#include <Storages/MergeTree/MergeList.h>
#include <Storages/MergeTree/StorageFromMergeTreeDataPart.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h>
#include <DataStreams/TTLBlockInputStream.h>
#include <DataStreams/DistinctSortedBlockInputStream.h>
#include <DataStreams/ExpressionBlockInputStream.h>
@ -120,18 +119,17 @@ void FutureMergedMutatedPart::assign(MergeTreeData::DataPartsVector parts_)
name = part_info.getPartName();
}
MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, const BackgroundProcessingPool & pool_)
: data(data_), pool(pool_), log(&Logger::get(data.getLogName() + " (MergerMutator)"))
MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, size_t background_pool_size_)
: data(data_), background_pool_size(background_pool_size_), log(&Logger::get(data.getLogName() + " (MergerMutator)"))
{
}
UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge()
{
size_t total_threads_in_pool = pool.getNumberOfThreads();
size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed);
return getMaxSourcePartsSizeForMerge(total_threads_in_pool, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1); /// 1 is current thread
return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1); /// 1 is current thread
}
@ -152,20 +150,18 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz
data_settings->max_bytes_to_merge_at_max_space_in_pool,
static_cast<double>(free_entries) / data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge);
return std::min(max_size, static_cast<UInt64>(DiskSpaceMonitor::getUnreservedFreeSpace(data.full_path) / DISK_USAGE_COEFFICIENT_TO_SELECT));
return std::min(max_size, static_cast<UInt64>(data.storage_policy->getMaxUnreservedFreeSpace() / DISK_USAGE_COEFFICIENT_TO_SELECT));
}
UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation()
{
const auto data_settings = data.getSettings();
size_t total_threads_in_pool = pool.getNumberOfThreads();
size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed);
/// Allow mutations only if there are enough threads, leave free threads for merges else
if (total_threads_in_pool - busy_threads_in_pool >= data_settings->number_of_free_entries_in_pool_to_execute_mutation)
return static_cast<UInt64>(DiskSpaceMonitor::getUnreservedFreeSpace(data.full_path) / DISK_USAGE_COEFFICIENT_TO_RESERVE);
if (background_pool_size - busy_threads_in_pool >= data_settings->number_of_free_entries_in_pool_to_execute_mutation)
return static_cast<UInt64>(data.storage_policy->getMaxUnreservedFreeSpace() / DISK_USAGE_COEFFICIENT_TO_RESERVE);
return 0;
}
@ -276,7 +272,6 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge(
return true;
}
bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition(
FutureMergedMutatedPart & future_part,
UInt64 & available_disk_space,
@ -325,9 +320,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition(
disk_space_warning_time = now;
LOG_WARNING(log, "Won't merge parts from " << parts.front()->name << " to " << (*prev_it)->name
<< " because not enough free space: "
<< formatReadableSizeWithBinarySuffix(available_disk_space) << " free and unreserved "
<< "(" << formatReadableSizeWithBinarySuffix(DiskSpaceMonitor::getReservedSpace()) << " reserved in "
<< DiskSpaceMonitor::getReservationCount() << " chunks), "
<< formatReadableSizeWithBinarySuffix(available_disk_space) << " free and unreserved, "
<< formatReadableSizeWithBinarySuffix(sum_bytes)
<< " required now (+" << static_cast<int>((DISK_USAGE_COEFFICIENT_TO_SELECT - 1.0) * 100)
<< "% on overhead); suppressing similar warnings for the next hour");
@ -536,7 +529,7 @@ public:
/// parts should be sorted.
MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart(
const FutureMergedMutatedPart & future_part, MergeList::Entry & merge_entry, TableStructureReadLockHolder &,
time_t time_of_merge, DiskSpaceMonitor::Reservation * disk_reservation, bool deduplicate, bool force_ttl)
time_t time_of_merge, DiskSpace::Reservation * space_reservation, bool deduplicate, bool force_ttl)
{
static const String TMP_PREFIX = "tmp_merge_";
@ -549,7 +542,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
<< parts.front()->name << " to " << parts.back()->name
<< " into " << TMP_PREFIX + future_part.name);
String new_part_tmp_path = data.getFullPath() + TMP_PREFIX + future_part.name + "/";
String part_path = data.getFullPathOnDisk(space_reservation->getDisk());
String new_part_tmp_path = part_path + TMP_PREFIX + future_part.name + "/";
if (Poco::File(new_part_tmp_path).exists())
throw Exception("Directory " + new_part_tmp_path + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS);
@ -569,7 +563,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names);
MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(
data, future_part.name, future_part.part_info);
data, space_reservation->getDisk(), future_part.name, future_part.part_info);
new_data_part->partition.assign(future_part.getPartition());
new_data_part->relative_path = TMP_PREFIX + future_part.name;
new_data_part->is_temp = true;
@ -743,7 +737,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
to.writePrefix();
size_t rows_written = 0;
const size_t initial_reservation = disk_reservation ? disk_reservation->getSize() : 0;
const size_t initial_reservation = space_reservation ? space_reservation->getSize() : 0;
auto is_cancelled = [&]() { return merges_blocker.isCancelled()
|| (need_remove_expired_values && ttl_merges_blocker.isCancelled()); };
@ -759,7 +753,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
merge_entry->bytes_written_uncompressed = merged_stream->getProfileInfo().bytes;
/// Reservation updates is not performed yet, during the merge it may lead to higher free space requirements
if (disk_reservation && sum_input_rows_upper_bound)
if (space_reservation && sum_input_rows_upper_bound)
{
/// The same progress from merge_entry could be used for both algorithms (it should be more accurate)
/// But now we are using inaccurate row-based estimation in Horizontal case for backward compatibility
@ -767,9 +761,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
? std::min(1., 1. * rows_written / sum_input_rows_upper_bound)
: std::min(1., merge_entry->progress.load(std::memory_order_relaxed));
disk_reservation->update(static_cast<size_t>((1. - progress) * initial_reservation));
space_reservation->update(static_cast<size_t>((1. - progress) * initial_reservation));
}
}
merged_stream->readSuffix();
merged_stream.reset();
@ -904,6 +899,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
const std::vector<MutationCommand> & commands,
MergeListEntry & merge_entry,
const Context & context,
DiskSpace::Reservation * space_reservation,
TableStructureReadLockHolder & table_lock_holder)
{
auto check_not_cancelled = [&]()
@ -950,7 +946,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
LOG_TRACE(log, "Mutating part " << source_part->name << " to mutation version " << future_part.part_info.mutation);
MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(
data, future_part.name, future_part.part_info);
data, space_reservation->getDisk(), future_part.name, future_part.part_info);
new_data_part->relative_path = "tmp_mut_" + future_part.name;
new_data_part->is_temp = true;
new_data_part->ttl_infos = source_part->ttl_infos;
@ -1239,6 +1235,7 @@ MergeTreeData::DataPartPtr MergeTreeDataMergerMutator::renameMergedTemporaryPart
return new_data_part;
}
size_t MergeTreeDataMergerMutator::estimateNeededDiskSpace(const MergeTreeData::DataPartsVector & source_parts)
{
size_t res = 0;

View File

@ -1,7 +1,6 @@
#pragma once
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/DiskSpaceMonitor.h>
#include <Storages/MutationCommands.h>
#include <atomic>
#include <functional>
@ -32,15 +31,17 @@ struct FutureMergedMutatedPart
void assign(MergeTreeData::DataPartsVector parts_);
};
/** Can select the parts to merge and merge them.
*/
/** Can select parts for background processes and do them.
* Currently helps with merges, mutations and moves
*/
class MergeTreeDataMergerMutator
{
public:
using AllowedMergingPredicate = std::function<bool (const MergeTreeData::DataPartPtr &, const MergeTreeData::DataPartPtr &, String * reason)>;
public:
MergeTreeDataMergerMutator(MergeTreeData & data_, const BackgroundProcessingPool & pool_);
MergeTreeDataMergerMutator(MergeTreeData & data_, size_t background_pool_size);
/** Get maximum total size of parts to do merge, at current moment of time.
* It depends on number of free threads in background_pool and amount of free space in disk.
@ -71,6 +72,7 @@ public:
const AllowedMergingPredicate & can_merge,
String * out_disable_reason = nullptr);
/** Select all the parts in the specified partition for merge, if possible.
* final - choose to merge even a single part - that is, allow to merge one part "with itself".
*/
@ -95,19 +97,22 @@ public:
MergeTreeData::MutableDataPartPtr mergePartsToTemporaryPart(
const FutureMergedMutatedPart & future_part,
MergeListEntry & merge_entry, TableStructureReadLockHolder & table_lock_holder, time_t time_of_merge,
DiskSpaceMonitor::Reservation * disk_reservation, bool deduplication, bool force_ttl);
DiskSpace::Reservation * disk_reservation, bool deduplication, bool force_ttl);
/// Mutate a single data part with the specified commands. Will create and return a temporary part.
MergeTreeData::MutableDataPartPtr mutatePartToTemporaryPart(
const FutureMergedMutatedPart & future_part,
const std::vector<MutationCommand> & commands,
MergeListEntry & merge_entry, const Context & context, TableStructureReadLockHolder & table_lock_holder);
MergeListEntry & merge_entry, const Context & context,
DiskSpace::Reservation * disk_reservation,
TableStructureReadLockHolder & table_lock_holder);
MergeTreeData::DataPartPtr renameMergedTemporaryPart(
MergeTreeData::MutableDataPartPtr & new_data_part,
const MergeTreeData::DataPartsVector & parts,
MergeTreeData::Transaction * out_transaction = nullptr);
/// The approximate amount of disk space needed for merge or mutation. With a surplus.
static size_t estimateNeededDiskSpace(const MergeTreeData::DataPartsVector & source_parts);
@ -137,7 +142,7 @@ private:
private:
MergeTreeData & data;
const BackgroundProcessingPool & pool;
const size_t background_pool_size;
Logger * log;

View File

@ -134,16 +134,22 @@ void MergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other)
}
MergeTreeDataPart::MergeTreeDataPart(MergeTreeData & storage_, const String & name_)
MergeTreeDataPart::MergeTreeDataPart(MergeTreeData & storage_, const DiskSpace::DiskPtr & disk_, const String & name_)
: storage(storage_)
, disk(disk_)
, name(name_)
, info(MergeTreePartInfo::fromPartName(name_, storage.format_version))
, index_granularity_info(storage)
{
}
MergeTreeDataPart::MergeTreeDataPart(const MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_)
MergeTreeDataPart::MergeTreeDataPart(
const MergeTreeData & storage_,
const DiskSpace::DiskPtr & disk_,
const String & name_,
const MergeTreePartInfo & info_)
: storage(storage_)
, disk(disk_)
, name(name_)
, info(info_)
, index_granularity_info(storage)
@ -240,9 +246,9 @@ String MergeTreeDataPart::getColumnNameWithMinumumCompressedSize() const
String MergeTreeDataPart::getFullPath() const
{
if (relative_path.empty())
throw Exception("Part relative_path cannot be empty. This is bug.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR);
return storage.full_path + relative_path + "/";
return storage.getFullPathOnDisk(disk) + relative_path + "/";
}
String MergeTreeDataPart::getNameWithPrefix() const
@ -308,7 +314,7 @@ time_t MergeTreeDataPart::getMaxTime() const
MergeTreeDataPart::~MergeTreeDataPart()
{
if (is_temp)
if (state == State::DeleteOnDestroy || is_temp)
{
try
{
@ -318,11 +324,14 @@ MergeTreeDataPart::~MergeTreeDataPart()
if (!dir.exists())
return;
if (!startsWith(getNameWithPrefix(), "tmp"))
if (is_temp)
{
LOG_ERROR(storage.log, "~DataPart() should remove part " << path
<< " but its name doesn't start with tmp. Too suspicious, keeping the part.");
return;
if (!startsWith(getNameWithPrefix(), "tmp"))
{
LOG_ERROR(storage.log, "~DataPart() should remove part " << path
<< " but its name doesn't start with tmp. Too suspicious, keeping the part.");
return;
}
}
dir.remove(true);
@ -364,10 +373,11 @@ void MergeTreeDataPart::remove() const
* And a race condition can happen that will lead to "File not found" error here.
*/
String full_path = storage.getFullPathOnDisk(disk);
String from = full_path + relative_path;
String to = full_path + "delete_tmp_" + name;
// TODO directory delete_tmp_<name> is never removed if server crashes before returning from this function
String from = storage.full_path + relative_path;
String to = storage.full_path + "delete_tmp_" + name;
Poco::File from_dir{from};
Poco::File to_dir{to};
@ -447,7 +457,7 @@ void MergeTreeDataPart::remove() const
void MergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const
{
String from = getFullPath();
String to = storage.full_path + new_relative_path + "/";
String to = storage.getFullPathOnDisk(disk) + new_relative_path + "/";
Poco::File from_file(from);
if (!from_file.exists())
@ -468,7 +478,7 @@ void MergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_n
}
else
{
throw Exception("part directory " + to + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS);
throw Exception("Part directory " + to + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS);
}
}
@ -495,7 +505,7 @@ String MergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix)
res = "detached/" + (prefix.empty() ? "" : prefix + "_")
+ name + (try_no ? "_try" + DB::toString(try_no) : "");
if (!Poco::File(storage.full_path + res).exists())
if (!Poco::File(storage.getFullPathOnDisk(disk) + res).exists())
return res;
LOG_WARNING(storage.log, "Directory " << res << " (to detach to) already exists."
@ -519,11 +529,27 @@ UInt64 MergeTreeDataPart::getMarksCount() const
void MergeTreeDataPart::makeCloneInDetached(const String & prefix) const
{
Poco::Path src(getFullPath());
Poco::Path dst(storage.full_path + getRelativePathForDetachedPart(prefix));
Poco::Path dst(storage.getFullPathOnDisk(disk) + getRelativePathForDetachedPart(prefix));
/// Backup is not recursive (max_level is 0), so do not copy inner directories
localBackup(src, dst, 0);
}
void MergeTreeDataPart::makeCloneOnDiskDetached(const DiskSpace::ReservationPtr & reservation) const
{
auto & reserved_disk = reservation->getDisk();
if (reserved_disk->getName() == disk->getName())
throw Exception("Can not clone data part " + name + " to same disk " + disk->getName(), ErrorCodes::LOGICAL_ERROR);
String path_to_clone = storage.getFullPathOnDisk(reserved_disk) + "detached/";
if (Poco::File(path_to_clone + relative_path).exists())
throw Exception("Path " + path_to_clone + relative_path + " already exists. Can not clone ", ErrorCodes::DIRECTORY_ALREADY_EXISTS);
Poco::File(path_to_clone).createDirectory();
Poco::File cloning_directory(getFullPath());
cloning_directory.copyTo(path_to_clone);
}
void MergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency)
{
/// Memory should not be limited during ATTACH TABLE query.
@ -636,10 +662,10 @@ void MergeTreeDataPart::loadPartitionAndMinMaxIndex()
}
else
{
String full_path = getFullPath();
partition.load(storage, full_path);
String path = getFullPath();
partition.load(storage, path);
if (!isEmpty())
minmax_idx.load(storage, full_path);
minmax_idx.load(storage, path);
}
String calculated_partition_id = partition.getID(storage.partition_key_sample);
@ -955,6 +981,8 @@ String MergeTreeDataPart::stateToString(MergeTreeDataPart::State state)
return "Outdated";
case State::Deleting:
return "Deleting";
case State::DeleteOnDestroy:
return "DeleteOnDestroy";
}
__builtin_unreachable();

View File

@ -32,9 +32,9 @@ struct MergeTreeDataPart
using Checksums = MergeTreeDataPartChecksums;
using Checksum = MergeTreeDataPartChecksums::Checksum;
MergeTreeDataPart(const MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_);
MergeTreeDataPart(const MergeTreeData & storage_, const DiskSpace::DiskPtr & disk_, const String & name_, const MergeTreePartInfo & info_);
MergeTreeDataPart(MergeTreeData & storage_, const String & name_);
MergeTreeDataPart(MergeTreeData & storage_, const DiskSpace::DiskPtr & disk_, const String & name_);
/// Returns the name of a column with minimum compressed size (as returned by getColumnSize()).
/// If no checksums are present returns the name of the first physically existing column.
@ -73,6 +73,7 @@ struct MergeTreeDataPart
const MergeTreeData & storage;
DiskSpace::DiskPtr disk;
String name;
MergeTreePartInfo info;
@ -102,20 +103,22 @@ struct MergeTreeDataPart
* Part state should be modified under data_parts mutex.
*
* Possible state transitions:
* Temporary -> Precommitted: we are trying to commit a fetched, inserted or merged part to active set
* Precommitted -> Outdated: we could not to add a part to active set and doing a rollback (for example it is duplicated part)
* Precommitted -> Commited: we successfully committed a part to active dataset
* Precommitted -> Outdated: a part was replaced by a covering part or DROP PARTITION
* Outdated -> Deleting: a cleaner selected this part for deletion
* Deleting -> Outdated: if an ZooKeeper error occurred during the deletion, we will retry deletion
* Temporary -> Precommitted: we are trying to commit a fetched, inserted or merged part to active set
* Precommitted -> Outdated: we could not to add a part to active set and doing a rollback (for example it is duplicated part)
* Precommitted -> Commited: we successfully committed a part to active dataset
* Precommitted -> Outdated: a part was replaced by a covering part or DROP PARTITION
* Outdated -> Deleting: a cleaner selected this part for deletion
* Deleting -> Outdated: if an ZooKeeper error occurred during the deletion, we will retry deletion
* Committed -> DeleteOnDestroy if part was moved to another disk
*/
enum class State
{
Temporary, /// the part is generating now, it is not in data_parts list
PreCommitted, /// the part is in data_parts, but not used for SELECTs
Committed, /// active data part, used by current and upcoming SELECTs
Outdated, /// not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes
Deleting /// not active data part with identity refcounter, it is deleting right now by a cleaner
Temporary, /// the part is generating now, it is not in data_parts list
PreCommitted, /// the part is in data_parts, but not used for SELECTs
Committed, /// active data part, used by current and upcoming SELECTs
Outdated, /// not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes
Deleting, /// not active data part with identity refcounter, it is deleting right now by a cleaner
DeleteOnDestroy, /// part was moved to another disk and should be deleted in own destructor
};
using TTLInfo = MergeTreeDataPartTTLInfo;
@ -256,6 +259,9 @@ struct MergeTreeDataPart
/// Makes clone of a part in detached/ directory via hard links
void makeCloneInDetached(const String & prefix) const;
/// Makes full clone of part in detached/ on another disk
void makeCloneOnDiskDetached(const DiskSpace::ReservationPtr & reservation) const;
/// Populates columns_to_size map (compressed size).
void accumulateColumnSizes(ColumnToSize & column_to_size) const;

View File

@ -198,7 +198,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
else
part_name = new_part_info.getPartName();
MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(data, part_name, new_part_info);
/// Size of part would not be grater than block.bytes() + epsilon
size_t expected_size = block.bytes();
auto reservation = data.reserveSpace(expected_size);
MergeTreeData::MutableDataPartPtr new_data_part =
std::make_shared<MergeTreeData::DataPart>(data, reservation->getDisk(), part_name, new_part_info);
new_data_part->partition = std::move(partition);
new_data_part->minmax_idx = std::move(minmax_idx);
new_data_part->relative_path = TMP_PREFIX + part_name;

View File

@ -9,6 +9,7 @@ namespace DB
{
/// A mutation entry for non-replicated MergeTree storage engines.
/// Stores information about mutation in file mutation_*.txt.
struct MergeTreeMutationEntry
{
time_t create_time = 0;

View File

@ -0,0 +1,178 @@
#include <Storages/MergeTree/MergeTreePartsMover.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <set>
#include <boost/algorithm/string/join.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int ABORTED;
extern const int NO_SUCH_DATA_PART;
extern const int LOGICAL_ERROR;
}
namespace
{
/// Contains minimal number of heaviest parts, which sum size on disk is greater than required.
/// If there are not enough summary size, than contains all parts.
class LargestPartsWithRequiredSize
{
struct PartsSizeOnDiskComparator
{
bool operator()(const MergeTreeData::DataPartPtr & f, const MergeTreeData::DataPartPtr & s) const
{
/// If parts have equal sizes, than order them by names (names are unique)
return std::tie(f->bytes_on_disk, f->name) < std::tie(s->bytes_on_disk, s->name);
}
};
std::set<MergeTreeData::DataPartPtr, PartsSizeOnDiskComparator> elems;
UInt64 required_size_sum;
UInt64 current_size_sum = 0;
public:
LargestPartsWithRequiredSize(UInt64 required_sum_size_) : required_size_sum(required_sum_size_) {}
void add(MergeTreeData::DataPartPtr part)
{
if (current_size_sum < required_size_sum)
{
elems.emplace(part);
current_size_sum += part->bytes_on_disk;
return;
}
/// Adding smaller element
if (!elems.empty() && (*elems.begin())->bytes_on_disk >= part->bytes_on_disk)
return;
elems.emplace(part);
current_size_sum += part->bytes_on_disk;
while (!elems.empty() && (current_size_sum - (*elems.begin())->bytes_on_disk >= required_size_sum))
{
current_size_sum -= (*elems.begin())->bytes_on_disk;
elems.erase(elems.begin());
}
}
/// Returns parts ordered by size
MergeTreeData::DataPartsVector getAccumulatedParts()
{
MergeTreeData::DataPartsVector res;
for (const auto & elem : elems)
res.push_back(elem);
return res;
}
};
}
bool MergeTreePartsMover::selectPartsForMove(
MergeTreeMovingParts & parts_to_move,
const AllowedMovingPredicate & can_move,
const std::lock_guard<std::mutex> & /* moving_parts_lock */)
{
MergeTreeData::DataPartsVector data_parts = data->getDataPartsVector();
if (data_parts.empty())
return false;
std::unordered_map<DiskSpace::DiskPtr, LargestPartsWithRequiredSize> need_to_move;
const auto & policy = data->getStoragePolicy();
const auto & volumes = policy->getVolumes();
/// Do not check if policy has one volume
if (volumes.size() == 1)
return false;
/// Do not check last volume
for (size_t i = 0; i != volumes.size() - 1; ++i)
{
for (const auto & disk : volumes[i]->disks)
{
UInt64 required_available_space = disk->getTotalSpace() * policy->getMoveFactor();
UInt64 unreserved_space = disk->getUnreservedSpace();
if (required_available_space > unreserved_space)
need_to_move.emplace(disk, required_available_space - unreserved_space);
}
}
for (const auto & part : data_parts)
{
String reason;
/// Don't report message to log, because logging is excessive
if (!can_move(part, &reason))
continue;
auto to_insert = need_to_move.find(part->disk);
if (to_insert != need_to_move.end())
to_insert->second.add(part);
}
for (auto && move : need_to_move)
{
auto min_volume_priority = policy->getVolumeIndexByDisk(move.first) + 1;
for (auto && part : move.second.getAccumulatedParts())
{
auto reservation = policy->reserve(part->bytes_on_disk, min_volume_priority);
if (!reservation)
{
/// Next parts to move from this disk has greater size and same min volume priority
/// There are no space for them
/// But it can be possible to move data from other disks
break;
}
parts_to_move.emplace_back(part, std::move(reservation));
}
}
return !parts_to_move.empty();
}
MergeTreeData::DataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEntry & moving_part) const
{
if (moves_blocker.isCancelled())
throw Exception("Cancelled moving parts.", ErrorCodes::ABORTED);
LOG_TRACE(log, "Cloning part " << moving_part.part->name);
moving_part.part->makeCloneOnDiskDetached(moving_part.reserved_space);
MergeTreeData::MutableDataPartPtr cloned_part =
std::make_shared<MergeTreeData::DataPart>(*data, moving_part.reserved_space->getDisk(), moving_part.part->name);
cloned_part->relative_path = "detached/" + moving_part.part->name;
LOG_TRACE(log, "Part " << moving_part.part->name << " was cloned to " << cloned_part->getFullPath());
cloned_part->loadColumnsChecksumsIndexes(true, true);
return cloned_part;
}
void MergeTreePartsMover::swapClonedPart(const MergeTreeData::DataPartPtr & cloned_part) const
{
if (moves_blocker.isCancelled())
throw Exception("Cancelled moving parts.", ErrorCodes::ABORTED);
auto active_part = data->getActiveContainingPart(cloned_part->name);
/// It's ok, because we don't block moving parts for merges or mutations
if (!active_part || active_part->name != cloned_part->name)
{
LOG_INFO(log, "Failed to swap " << cloned_part->name << ". Active part doesn't exist."
<< " Possible it was merged or mutated. Will remove copy on path '" << cloned_part->getFullPath() << "'.");
return;
}
cloned_part->renameTo(active_part->name);
/// TODO what happen if server goes down here?
data->swapActivePart(cloned_part);
LOG_TRACE(log, "Part " << cloned_part->name << " was moved to " << cloned_part->getFullPath());
}
}

View File

@ -0,0 +1,73 @@
#pragma once
#include <functional>
#include <vector>
#include <optional>
#include <Storages/MergeTree/MergeTreeDataPart.h>
#include <Common/ActionBlocker.h>
#include <Common/DiskSpaceMonitor.h>
namespace DB
{
/// Active part from storage and destination reservation where
/// it have to be moved.
struct MergeTreeMoveEntry
{
std::shared_ptr<const MergeTreeDataPart> part;
DiskSpace::ReservationPtr reserved_space;
MergeTreeMoveEntry(const std::shared_ptr<const MergeTreeDataPart> & part_, DiskSpace::ReservationPtr reservation_)
: part(part_), reserved_space(std::move(reservation_))
{
}
};
using MergeTreeMovingParts = std::vector<MergeTreeMoveEntry>;
/** Can select parts for background moves, clone them to appropriate disks into
* /detached directory and replace them into active parts set
*/
class MergeTreePartsMover
{
private:
/// Callback tells that part is not participating in background process
using AllowedMovingPredicate = std::function<bool(const std::shared_ptr<const MergeTreeDataPart> &, String * reason)>;
public:
MergeTreePartsMover(MergeTreeData * data_)
: data(data_)
, log(&Poco::Logger::get("MergeTreePartsMover"))
{
}
/// Select parts for background moves according to storage_policy configuration.
/// Returns true if at least one part was selected for move.
bool selectPartsForMove(
MergeTreeMovingParts & parts_to_move,
const AllowedMovingPredicate & can_move,
const std::lock_guard<std::mutex> & moving_parts_lock);
/// Copies part to selected reservation in detached folder. Throws exception if part alredy exists.
std::shared_ptr<const MergeTreeDataPart> clonePart(const MergeTreeMoveEntry & moving_part) const;
/// Replaces cloned part from detached directory into active data parts set.
/// Replacing part changes state to DeleteOnDestroy and will be removed from disk after destructor of
/// MergeTreeDataPart called. If replacing part doesn't exists or not active (commited) than
/// cloned part will be removed and loge message will be reported. It may happen in case of concurrent
/// merge or mutation.
void swapClonedPart(const std::shared_ptr<const MergeTreeDataPart> & cloned_parts) const;
public:
/// Can stop background moves and moves from queries
ActionBlocker moves_blocker;
private:
MergeTreeData * data;
Logger * log;
};
}

View File

@ -88,6 +88,7 @@ struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
M(SettingMaxThreads, max_part_loading_threads, 0, "The number of theads to load data parts at startup.") \
M(SettingMaxThreads, max_part_removal_threads, 0, "The number of theads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).") \
M(SettingUInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.") \
M(SettingString, storage_policy_name, "default", "Name of storage disk policy")
DECLARE_SETTINGS_COLLECTION(LIST_OF_MERGE_TREE_SETTINGS)
@ -103,7 +104,7 @@ struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
/// We check settings after storage creation
static bool isReadonlySetting(const String & name)
{
return name == "index_granularity" || name == "index_granularity_bytes";
return name == "index_granularity" || name == "index_granularity_bytes" || name == "storage_policy_name";
}
};

View File

@ -85,8 +85,9 @@ void ReplicatedMergeTreeAlterThread::run()
auto metadata_in_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str);
auto metadata_diff = ReplicatedMergeTreeTableMetadata(storage).checkAndFindDiff(metadata_in_zk, /* allow_alter = */ true);
/// If you need to lock table structure, then suspend merges.
/// If you need to lock table structure, then suspend merges and moves.
ActionLock merge_blocker = storage.merger_mutator.merges_blocker.cancel();
ActionLock moves_blocker = storage.parts_mover.moves_blocker.cancel();
MergeTreeData::DataParts parts;

View File

@ -15,6 +15,7 @@ namespace ErrorCodes
{
extern const int UNEXPECTED_NODE_IN_ZOOKEEPER;
extern const int UNFINISHED;
extern const int PART_IS_TEMPORARILY_LOCKED;
}
@ -30,7 +31,7 @@ void ReplicatedMergeTreeQueue::addVirtualParts(const MergeTreeData::DataParts &
{
std::lock_guard lock(state_mutex);
for (const auto & part : parts)
for (auto part : parts)
{
current_parts.add(part->name);
virtual_parts.add(part->name);
@ -38,6 +39,12 @@ void ReplicatedMergeTreeQueue::addVirtualParts(const MergeTreeData::DataParts &
}
bool ReplicatedMergeTreeQueue::isVirtualPart(const MergeTreeData::DataPartPtr & data_part) const
{
std::lock_guard lock(state_mutex);
return virtual_parts.getContainingPart(data_part->info) != data_part->name;
}
bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper)
{
auto queue_path = replica_path + "/queue";
@ -379,11 +386,10 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri
bool ReplicatedMergeTreeQueue::removeFromVirtualParts(const MergeTreePartInfo & part_info)
{
std::unique_lock lock(state_mutex);
std::lock_guard lock(state_mutex);
return virtual_parts.remove(part_info);
}
void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback)
{
std::lock_guard lock(pull_logs_to_queue_mutex);
@ -763,7 +769,10 @@ bool ReplicatedMergeTreeQueue::checkReplaceRangeCanBeRemoved(const MergeTreePart
return true;
}
void ReplicatedMergeTreeQueue::removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, const ReplicatedMergeTreeLogEntryData & current)
void ReplicatedMergeTreeQueue::removePartProducingOpsInRange(
zkutil::ZooKeeperPtr zookeeper,
const MergeTreePartInfo & part_info,
const ReplicatedMergeTreeLogEntryData & current)
{
Queue to_wait;
size_t removed_entries = 0;
@ -1312,7 +1321,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep
}
void ReplicatedMergeTreeQueue::disableMergesInRange(const String & part_name)
void ReplicatedMergeTreeQueue::disableMergesInBlockRange(const String & part_name)
{
std::lock_guard lock(state_mutex);
virtual_parts.add(part_name);
@ -1630,25 +1639,30 @@ bool ReplicatedMergeTreeMergePredicate::operator()(
if (left_max_block + 1 < right_min_block)
{
/// Fake part which will appear as merge result
MergeTreePartInfo gap_part_info(
left->info.partition_id, left_max_block + 1, right_min_block - 1,
MergeTreePartInfo::MAX_LEVEL, MergeTreePartInfo::MAX_BLOCK_NUMBER);
/// We don't select parts if any smaller part covered by our merge must exist after
/// processing replication log up to log_pointer.
Strings covered = queue.virtual_parts.getPartsCoveredBy(gap_part_info);
if (!covered.empty())
{
if (out_reason)
*out_reason = "There are " + toString(covered.size()) + " parts (from " + covered.front()
+ " to " + covered.back() + ") that are still not present on this replica between "
+ left->name + " and " + right->name;
+ " to " + covered.back() + ") that are still not present or beeing processed by "
+ " other background process on this replica between " + left->name + " and " + right->name;
return false;
}
}
Int64 left_mutation_ver = queue.getCurrentMutationVersionImpl(
left->info.partition_id, left->info.getDataVersion(), lock);
Int64 right_mutation_ver = queue.getCurrentMutationVersionImpl(
left->info.partition_id, right->info.getDataVersion(), lock);
if (left_mutation_ver != right_mutation_ver)
{
if (out_reason)

View File

@ -229,6 +229,7 @@ public:
~ReplicatedMergeTreeQueue();
void initialize(const String & zookeeper_path_, const String & replica_path_, const String & logger_name_,
const MergeTreeData::DataParts & parts);
@ -304,6 +305,7 @@ public:
/// Count the total number of active mutations that are finished (is_done = true).
size_t countFinishedMutations() const;
/// Returns functor which used by MergeTreeMergerMutator to select parts for merge
ReplicatedMergeTreeMergePredicate getMergePredicate(zkutil::ZooKeeperPtr & zookeeper);
/// Return the version (block number) of the last mutation that we don't need to apply to the part
@ -318,12 +320,17 @@ public:
/// (because some mutations are probably done but we are not sure yet), returns true.
bool tryFinalizeMutations(zkutil::ZooKeeperPtr zookeeper);
/// Prohibit merges in the specified range.
void disableMergesInRange(const String & part_name);
/// Prohibit merges in the specified blocks range.
/// Add part to virtual_parts, which means that part must exist
/// after processing replication log up to log_pointer.
/// Part maybe fake (look at ReplicatedMergeTreeMergePredicate).
void disableMergesInBlockRange(const String & part_name);
/** Check that part isn't in currently generating parts and isn't covered by them and add it to future_parts.
* Locks queue's mutex.
*/
/// Cheks that part is already in virtual parts
bool isVirtualPart(const MergeTreeData::DataPartPtr & data_part) const;
/// Check that part isn't in currently generating parts and isn't covered by them and add it to future_parts.
/// Locks queue's mutex.
bool addFuturePartIfNotCoveredByThem(const String & part_name, LogEntry & entry, String & reject_reason);
/// A blocker that stops selects from the queue

View File

@ -639,14 +639,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (replicated)
return StorageReplicatedMergeTree::create(
zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name,
zookeeper_path, replica_name, args.attach, args.database_name, args.table_name,
args.columns, indices_description, args.constraints,
args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast,
sample_by_ast, ttl_table_ast, merging_params, std::move(storage_settings),
args.has_force_restore_data_flag);
else
return StorageMergeTree::create(
args.data_path, args.database_name, args.table_name, args.columns, indices_description,
args.database_name, args.table_name, args.columns, indices_description,
args.constraints, args.attach, args.context, date_column_name, partition_by_ast, order_by_ast,
primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, std::move(storage_settings),
args.has_force_restore_data_flag);

View File

@ -39,6 +39,24 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
res.part = command_ast->part;
return res;
}
else if (command_ast->type == ASTAlterCommand::MOVE_PARTITION)
{
PartitionCommand res;
res.type = MOVE_PARTITION;
res.partition = command_ast->partition;
res.part = command_ast->part;
switch (command_ast->move_destination_type)
{
case ASTAlterCommand::MoveDestinationType::DISK:
res.move_destination_type = PartitionCommand::MoveDestinationType::DISK;
break;
case ASTAlterCommand::MoveDestinationType::VOLUME:
res.move_destination_type = PartitionCommand::MoveDestinationType::VOLUME;
break;
}
res.move_destination_name = command_ast->move_destination_name;
return res;
}
else if (command_ast->type == ASTAlterCommand::REPLACE_PARTITION)
{
PartitionCommand res;

View File

@ -19,6 +19,7 @@ struct PartitionCommand
enum Type
{
ATTACH_PARTITION,
MOVE_PARTITION,
CLEAR_COLUMN,
CLEAR_INDEX,
DROP_PARTITION,
@ -52,6 +53,16 @@ struct PartitionCommand
/// For FREEZE PARTITION
String with_name;
enum MoveDestinationType
{
DISK,
VOLUME,
};
MoveDestinationType move_destination_type;
String move_destination_name;
static std::optional<PartitionCommand> parse(const ASTAlterCommand * command);
};

View File

@ -96,7 +96,7 @@ public:
void startup() override;
void shutdown() override;
String getDataPath() const override { return path; }
Strings getDataPaths() const override { return {path}; }
const ExpressionActionsPtr & getShardingKeyExpr() const { return sharding_key_expr; }
const String & getShardingKeyColumnName() const { return sharding_key_column_name; }

View File

@ -325,11 +325,11 @@ BlockOutputStreamPtr StorageFile::write(
return std::make_shared<StorageFileBlockOutputStream>(*this);
}
String StorageFile::getDataPath() const
Strings StorageFile::getDataPaths() const
{
if (paths.empty())
throw Exception("Table '" + table_name + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED);
return paths[0];
return paths;
}
void StorageFile::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)

View File

@ -40,7 +40,7 @@ public:
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
String getDataPath() const override;
Strings getDataPaths() const override;
protected:
friend class StorageFileBlockInputStream;

View File

@ -44,9 +44,9 @@ public:
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
std::string full_path() const { return path + escapeForFileName(table_name) + '/';}
std::string fullPath() const { return path + escapeForFileName(table_name) + '/';}
String getDataPath() const override { return full_path(); }
Strings getDataPaths() const override { return {fullPath()}; }
protected:
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),

View File

@ -341,10 +341,10 @@ StoragePtr StorageMaterializedView::tryGetTargetTable() const
return global_context.tryGetTable(target_database_name, target_table_name);
}
String StorageMaterializedView::getDataPath() const
Strings StorageMaterializedView::getDataPaths() const
{
if (auto table = tryGetTargetTable())
return table->getDataPath();
return table->getDataPaths();
return {};
}

View File

@ -65,7 +65,7 @@ public:
size_t max_block_size,
unsigned num_streams) override;
String getDataPath() const override;
Strings getDataPaths() const override;
private:
String select_database_name;

View File

@ -18,7 +18,7 @@
#include <Storages/AlterCommands.h>
#include <Storages/PartitionCommands.h>
#include <Storages/MergeTree/MergeTreeBlockOutputStream.h>
#include <Storages/MergeTree/DiskSpaceMonitor.h>
#include <Common/DiskSpaceMonitor.h>
#include <Storages/MergeTree/MergeList.h>
#include <Storages/MergeTree/checkDataPart.h>
#include <Poco/DirectoryIterator.h>
@ -38,6 +38,7 @@ namespace ErrorCodes
extern const int INCORRECT_FILE_NAME;
extern const int CANNOT_ASSIGN_OPTIMIZE;
extern const int INCOMPATIBLE_COLUMNS;
extern const int PART_IS_TEMPORARILY_LOCKED;
extern const int UNKNOWN_SETTING;
extern const int TOO_BIG_AST;
}
@ -46,11 +47,11 @@ namespace ActionLocks
{
extern const StorageActionBlockType PartsMerge;
extern const StorageActionBlockType PartsTTLMerge;
extern const StorageActionBlockType PartsMove;
}
StorageMergeTree::StorageMergeTree(
const String & path_,
const String & database_name_,
const String & table_name_,
const ColumnsDescription & columns_,
@ -68,18 +69,13 @@ StorageMergeTree::StorageMergeTree(
std::unique_ptr<MergeTreeSettings> storage_settings_,
bool has_force_restore_data_flag)
: MergeTreeData(database_name_, table_name_,
path_ + escapeForFileName(table_name_) + '/',
columns_, indices_, constraints_,
context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_,
sample_by_ast_, ttl_table_ast_, merging_params_,
std::move(storage_settings_), false, attach),
path(path_),
background_pool(context_.getBackgroundPool()),
reader(*this), writer(*this), merger_mutator(*this, global_context.getBackgroundPool())
reader(*this), writer(*this),
merger_mutator(*this, global_context.getBackgroundPool().getNumberOfThreads())
{
if (path.empty())
throw Exception("MergeTree require data path", ErrorCodes::INCORRECT_FILE_NAME);
loadDataParts(has_force_restore_data_flag);
if (!attach && !getDataParts().empty())
@ -101,7 +97,8 @@ void StorageMergeTree::startup()
/// NOTE background task will also do the above cleanups periodically.
time_after_previous_cleanup.restart();
background_task_handle = background_pool.addTask([this] { return backgroundTask(); });
merging_mutating_task_handle = global_context.getBackgroundPool().addTask([this] { return mergeMutateTask(); });
moving_task_handle = global_context.getBackgroundPool().addTask([this] { return movePartsTask(); });
}
@ -111,8 +108,13 @@ void StorageMergeTree::shutdown()
return;
shutdown_called = true;
merger_mutator.merges_blocker.cancelForever();
if (background_task_handle)
background_pool.removeTask(background_task_handle);
parts_mover.moves_blocker.cancelForever();
if (merging_mutating_task_handle)
global_context.getBackgroundPool().removeTask(merging_mutating_task_handle);
if (moving_task_handle)
global_context.getBackgroundPool().removeTask(moving_task_handle);
}
@ -184,20 +186,6 @@ void StorageMergeTree::truncate(const ASTPtr &, const Context &, TableStructureW
clearOldPartsFromFilesystem();
}
void StorageMergeTree::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
{
std::string new_full_path = new_path_to_db + escapeForFileName(new_table_name) + '/';
setPath(new_full_path);
path = new_path_to_db;
table_name = new_table_name;
database_name = new_database_name;
full_path = new_full_path;
/// NOTE: Logger names are not updated.
}
std::vector<MergeTreeData::AlterDataPartTransactionPtr> StorageMergeTree::prepareAlterTransactions(
const ColumnsDescription & new_columns, const IndicesDescription & new_indices, const Context & context)
@ -276,7 +264,9 @@ void StorageMergeTree::alter(
}
/// NOTE: Here, as in ReplicatedMergeTree, you can do ALTER which does not block the writing of data for a long time.
/// Also block moves, because they can replace part with old state
auto merge_blocker = merger_mutator.merges_blocker.cancel();
auto moves_blocked = parts_mover.moves_blocker.cancel();
lockNewDataStructureExclusively(table_lock_holder, context.getCurrentQueryId());
checkAlter(params, context);
@ -334,11 +324,11 @@ void StorageMergeTree::alter(
}
/// While exists, marks parts as 'currently_merging' and reserves free space on filesystem.
/// While exists, marks parts as 'currently_merging_mutating_parts' and reserves free space on filesystem.
struct CurrentlyMergingPartsTagger
{
FutureMergedMutatedPart future_part;
DiskSpaceMonitor::ReservationPtr reserved_space;
DiskSpace::ReservationPtr reserved_space;
bool is_successful = false;
String exception_message;
@ -346,29 +336,41 @@ struct CurrentlyMergingPartsTagger
StorageMergeTree & storage;
public:
CurrentlyMergingPartsTagger(const FutureMergedMutatedPart & future_part_, size_t total_size, StorageMergeTree & storage_)
CurrentlyMergingPartsTagger(const FutureMergedMutatedPart & future_part_, size_t total_size, StorageMergeTree & storage_, bool is_mutation)
: future_part(future_part_), storage(storage_)
{
/// Assume mutex is already locked, because this method is called from mergeTask.
reserved_space = DiskSpaceMonitor::reserve(storage.full_path, total_size); /// May throw.
/// if we mutate part, than we should reserve space on the same disk, because mutations possible can create hardlinks
if (is_mutation)
reserved_space = future_part_.parts[0]->disk->reserve(total_size);
else
reserved_space = storage.reserveSpace(total_size);
if (!reserved_space)
{
if (is_mutation)
throw Exception("Not enough space for mutating part '" + future_part_.parts[0]->name + "'", ErrorCodes::NOT_ENOUGH_SPACE);
else
throw Exception("Not enough space for merging parts", ErrorCodes::NOT_ENOUGH_SPACE);
}
for (const auto & part : future_part.parts)
{
if (storage.currently_merging.count(part))
throw Exception("Tagging alreagy tagged part " + part->name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
if (storage.currently_merging_mutating_parts.count(part))
throw Exception("Tagging already tagged part " + part->name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
}
storage.currently_merging.insert(future_part.parts.begin(), future_part.parts.end());
storage.currently_merging_mutating_parts.insert(future_part.parts.begin(), future_part.parts.end());
}
~CurrentlyMergingPartsTagger()
{
std::lock_guard lock(storage.currently_merging_mutex);
std::lock_guard lock(storage.currently_processing_in_background_mutex);
for (const auto & part : future_part.parts)
{
if (!storage.currently_merging.count(part))
if (!storage.currently_merging_mutating_parts.count(part))
std::terminate();
storage.currently_merging.erase(part);
storage.currently_merging_mutating_parts.erase(part);
}
/// Update the information about failed parts in the system.mutations table.
@ -410,10 +412,13 @@ public:
void StorageMergeTree::mutate(const MutationCommands & commands, const Context &)
{
MergeTreeMutationEntry entry(commands, full_path, insert_increment.get());
/// Choose any disk, because when we load mutations we search them at each disk
/// where storage can be placed. See loadMutations().
auto disk = storage_policy->getAnyDisk();
MergeTreeMutationEntry entry(commands, getFullPathOnDisk(disk), insert_increment.get());
String file_name;
{
std::lock_guard lock(currently_merging_mutex);
std::lock_guard lock(currently_processing_in_background_mutex);
Int64 version = increment.get();
entry.commit(version);
@ -423,12 +428,12 @@ void StorageMergeTree::mutate(const MutationCommands & commands, const Context &
}
LOG_INFO(log, "Added mutation: " << file_name);
background_task_handle->wake();
merging_mutating_task_handle->wake();
}
std::vector<MergeTreeMutationStatus> StorageMergeTree::getMutationsStatus() const
{
std::lock_guard lock(currently_merging_mutex);
std::lock_guard lock(currently_processing_in_background_mutex);
std::vector<Int64> part_data_versions;
auto data_parts = getDataPartsVector();
@ -476,7 +481,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id)
std::optional<MergeTreeMutationEntry> to_kill;
{
std::lock_guard lock(currently_merging_mutex);
std::lock_guard lock(currently_processing_in_background_mutex);
auto it = current_mutations_by_id.find(mutation_id);
if (it != current_mutations_by_id.end())
{
@ -494,7 +499,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id)
LOG_TRACE(log, "Cancelled part mutations and removed mutation file " << mutation_id);
/// Maybe there is another mutation that was blocked by the killed one. Try to execute it immediately.
background_task_handle->wake();
merging_mutating_task_handle->wake();
return CancellationCode::CancelSent;
}
@ -503,18 +508,22 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id)
void StorageMergeTree::loadMutations()
{
Poco::DirectoryIterator end;
for (auto it = Poco::DirectoryIterator(full_path); it != end; ++it)
const auto full_paths = getDataPaths();
for (const String & full_path : full_paths)
{
if (startsWith(it.name(), "mutation_"))
for (auto it = Poco::DirectoryIterator(full_path); it != end; ++it)
{
MergeTreeMutationEntry entry(full_path, it.name());
Int64 block_number = entry.block_number;
auto insertion = current_mutations_by_id.emplace(it.name(), std::move(entry));
current_mutations_by_version.emplace(block_number, insertion.first->second);
}
else if (startsWith(it.name(), "tmp_mutation_"))
{
it->remove();
if (startsWith(it.name(), "mutation_"))
{
MergeTreeMutationEntry entry(full_path, it.name());
Int64 block_number = entry.block_number;
auto insertion = current_mutations_by_id.emplace(it.name(), std::move(entry));
current_mutations_by_version.emplace(block_number, insertion.first->second);
}
else if (startsWith(it.name(), "tmp_mutation_"))
{
it->remove();
}
}
}
@ -534,15 +543,15 @@ bool StorageMergeTree::merge(
FutureMergedMutatedPart future_part;
/// You must call destructor with unlocked `currently_merging_mutex`.
/// You must call destructor with unlocked `currently_processing_in_background_mutex`.
std::optional<CurrentlyMergingPartsTagger> merging_tagger;
{
std::lock_guard lock(currently_merging_mutex);
std::lock_guard lock(currently_processing_in_background_mutex);
auto can_merge = [this, &lock] (const DataPartPtr & left, const DataPartPtr & right, String *)
{
return !currently_merging.count(left) && !currently_merging.count(right)
return !currently_merging_mutating_parts.count(left) && !currently_merging_mutating_parts.count(right)
&& getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock);
};
@ -558,7 +567,7 @@ bool StorageMergeTree::merge(
}
else
{
UInt64 disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
UInt64 disk_space = storage_policy->getMaxUnreservedFreeSpace();
selected = merger_mutator.selectAllPartsToMergeWithinPartition(future_part, disk_space, can_merge, partition_id, final, out_disable_reason);
}
@ -569,7 +578,7 @@ bool StorageMergeTree::merge(
return false;
}
merging_tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this);
merging_tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, false);
}
MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(database_name, table_name, future_part);
@ -580,45 +589,14 @@ bool StorageMergeTree::merge(
auto write_part_log = [&] (const ExecutionStatus & execution_status)
{
try
{
auto part_log = global_context.getPartLog(database_name);
if (!part_log)
return;
PartLogElement part_log_elem;
part_log_elem.event_type = PartLogElement::MERGE_PARTS;
part_log_elem.event_time = time(nullptr);
part_log_elem.duration_ms = stopwatch.elapsed() / 1000000;
part_log_elem.database_name = database_name;
part_log_elem.table_name = table_name;
part_log_elem.partition_id = future_part.part_info.partition_id;
part_log_elem.part_name = future_part.name;
if (new_part)
part_log_elem.bytes_compressed_on_disk = new_part->bytes_on_disk;
part_log_elem.source_part_names.reserve(future_part.parts.size());
for (const auto & source_part : future_part.parts)
part_log_elem.source_part_names.push_back(source_part->name);
part_log_elem.rows_read = (*merge_entry)->rows_read;
part_log_elem.bytes_read_uncompressed = (*merge_entry)->bytes_read_uncompressed;
part_log_elem.rows = (*merge_entry)->rows_written;
part_log_elem.bytes_uncompressed = (*merge_entry)->bytes_written_uncompressed;
part_log_elem.error = static_cast<UInt16>(execution_status.code);
part_log_elem.exception = execution_status.message;
part_log->add(part_log_elem);
}
catch (...)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
}
writePartLog(
PartLogElement::MERGE_PARTS,
execution_status,
stopwatch.elapsed(),
future_part.name,
new_part,
future_part.parts,
merge_entry.get());
};
try
@ -647,6 +625,30 @@ bool StorageMergeTree::merge(
}
bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & part) const
{
std::lock_guard background_processing_lock(currently_processing_in_background_mutex);
return currently_merging_mutating_parts.count(part);
}
BackgroundProcessingPoolTaskResult StorageMergeTree::movePartsTask()
{
try
{
if (!selectPartsAndMove())
return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO;
return BackgroundProcessingPoolTaskResult::SUCCESS;
}
catch (...)
{
tryLogCurrentException(log);
return BackgroundProcessingPoolTaskResult::ERROR;
}
}
bool StorageMergeTree::tryMutatePart()
{
auto table_lock_holder = lockStructureForShare(true, RWLockImpl::NO_QUERY);
@ -654,10 +656,13 @@ bool StorageMergeTree::tryMutatePart()
FutureMergedMutatedPart future_part;
MutationCommands commands;
/// You must call destructor with unlocked `currently_merging_mutex`.
/// You must call destructor with unlocked `currently_processing_in_background_mutex`.
std::optional<CurrentlyMergingPartsTagger> tagger;
{
std::lock_guard lock(currently_merging_mutex);
/// DataPart can be store only at one disk. Get Max of free space at all disks
UInt64 disk_space = storage_policy->getMaxUnreservedFreeSpace();
std::lock_guard lock(currently_processing_in_background_mutex);
if (current_mutations_by_version.empty())
return false;
@ -665,14 +670,14 @@ bool StorageMergeTree::tryMutatePart()
auto mutations_end_it = current_mutations_by_version.end();
for (const auto & part : getDataPartsVector())
{
if (currently_merging.count(part))
if (currently_merging_mutating_parts.count(part))
continue;
auto mutations_begin_it = current_mutations_by_version.upper_bound(part->info.getDataVersion());
if (mutations_begin_it == mutations_end_it)
continue;
if (merger_mutator.getMaxSourcePartSizeForMutation() < part->bytes_on_disk)
if (merger_mutator.getMaxSourcePartSizeForMutation() > disk_space)
continue;
size_t current_ast_elements = 0;
@ -695,7 +700,7 @@ bool StorageMergeTree::tryMutatePart()
future_part.part_info = new_part_info;
future_part.name = part->getNewName(new_part_info);
tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this);
tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, true);
break;
}
}
@ -710,51 +715,21 @@ bool StorageMergeTree::tryMutatePart()
auto write_part_log = [&] (const ExecutionStatus & execution_status)
{
try
{
auto part_log = global_context.getPartLog(database_name);
if (!part_log)
return;
PartLogElement part_log_elem;
part_log_elem.event_type = PartLogElement::MUTATE_PART;
part_log_elem.error = static_cast<UInt16>(execution_status.code);
part_log_elem.exception = execution_status.message;
part_log_elem.event_time = time(nullptr);
part_log_elem.duration_ms = stopwatch.elapsed() / 1000000;
part_log_elem.database_name = database_name;
part_log_elem.table_name = table_name;
part_log_elem.partition_id = future_part.part_info.partition_id;
part_log_elem.part_name = future_part.name;
part_log_elem.rows_read = (*merge_entry)->rows_read;
part_log_elem.bytes_read_uncompressed = (*merge_entry)->bytes_read_uncompressed;
part_log_elem.rows = (*merge_entry)->rows_written;
part_log_elem.bytes_uncompressed = (*merge_entry)->bytes_written_uncompressed;
if (new_part)
part_log_elem.bytes_compressed_on_disk = new_part->bytes_on_disk;
part_log_elem.source_part_names.reserve(future_part.parts.size());
for (const auto & source_part : future_part.parts)
part_log_elem.source_part_names.push_back(source_part->name);
part_log->add(part_log_elem);
}
catch (...)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
}
writePartLog(
PartLogElement::MUTATE_PART,
execution_status,
stopwatch.elapsed(),
future_part.name,
new_part,
future_part.parts,
merge_entry.get());
};
try
{
new_part = merger_mutator.mutatePartToTemporaryPart(future_part, commands, *merge_entry, global_context, table_lock_holder);
new_part = merger_mutator.mutatePartToTemporaryPart(future_part, commands, *merge_entry, global_context,
tagger->reserved_space.get(), table_lock_holder);
renameTempPartAndReplace(new_part);
tagger->is_successful = true;
write_part_log({});
@ -770,13 +745,13 @@ bool StorageMergeTree::tryMutatePart()
}
BackgroundProcessingPoolTaskResult StorageMergeTree::backgroundTask()
BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask()
{
if (shutdown_called)
return BackgroundProcessingPoolTaskResult::ERROR;
if (merger_mutator.merges_blocker.isCancelled())
return BackgroundProcessingPoolTaskResult::ERROR;
return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO;
try
{
@ -796,10 +771,11 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::backgroundTask()
if (merge(false /*aggressive*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/))
return BackgroundProcessingPoolTaskResult::SUCCESS;
if (tryMutatePart())
return BackgroundProcessingPoolTaskResult::SUCCESS;
else
return BackgroundProcessingPoolTaskResult::ERROR;
return BackgroundProcessingPoolTaskResult::ERROR;
}
catch (const Exception & e)
{
@ -815,7 +791,7 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::backgroundTask()
Int64 StorageMergeTree::getCurrentMutationVersion(
const DataPartPtr & part,
std::lock_guard<std::mutex> & /* currently_merging_mutex_lock */) const
std::lock_guard<std::mutex> & /* currently_processing_in_background_mutex_lock */) const
{
auto it = current_mutations_by_version.upper_bound(part->info.getDataVersion());
if (it == current_mutations_by_version.begin())
@ -832,7 +808,7 @@ void StorageMergeTree::clearOldMutations(bool truncate)
std::vector<MergeTreeMutationEntry> mutations_to_delete;
{
std::lock_guard lock(currently_merging_mutex);
std::lock_guard lock(currently_processing_in_background_mutex);
if (!truncate && current_mutations_by_version.size() <= settings->finished_mutations_to_keep)
return;
@ -872,9 +848,10 @@ void StorageMergeTree::clearOldMutations(bool truncate)
void StorageMergeTree::clearColumnOrIndexInPartition(const ASTPtr & partition, const AlterCommand & alter_command, const Context & context)
{
/// Asks to complete merges and does not allow them to start.
/// Asks to complete merges and moves and does not allow them to start.
/// This protects against "revival" of data for a removed partition after completion of merge.
auto merge_blocker = merger_mutator.merges_blocker.cancel();
auto move_blocker = parts_mover.moves_blocker.cancel();
/// We don't change table structure, only data in some parts, parts are locked inside alterDataPart() function
auto lock_read_structure = lockStructureForShare(false, context.getCurrentQueryId());
@ -1000,6 +977,22 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma
attachPartition(command.partition, command.part, context);
break;
case PartitionCommand::MOVE_PARTITION:
{
switch (command.move_destination_type)
{
case PartitionCommand::MoveDestinationType::DISK:
movePartitionToDisk(command.partition, command.move_destination_name, command.part, context);
break;
case PartitionCommand::MoveDestinationType::VOLUME:
movePartitionToVolume(command.partition, command.move_destination_name, command.part, context);
break;
}
}
break;
case PartitionCommand::REPLACE_PARTITION:
{
checkPartitionCanBeDropped(command.partition);
@ -1084,7 +1077,7 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par
{
// TODO: should get some locks to prevent race with 'alter … modify column'
PartsTemporaryRename renamed_parts(*this, full_path + "detached/");
PartsTemporaryRename renamed_parts(*this, "detached/");
MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, context, renamed_parts);
for (size_t i = 0; i < loaded_parts.size(); ++i)
@ -1177,7 +1170,9 @@ ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type)
if (action_type == ActionLocks::PartsMerge)
return merger_mutator.merges_blocker.cancel();
else if (action_type == ActionLocks::PartsTTLMerge)
return merger_mutator.ttl_merges_blocker.cancel();
return merger_mutator.ttl_merges_blocker.cancel();
else if (action_type == ActionLocks::PartsMove)
return parts_mover.moves_blocker.cancel();
return {};
}

View File

@ -9,11 +9,13 @@
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <Storages/MergeTree/MergeTreeDataWriter.h>
#include <Storages/MergeTree/MergeTreeDataMergerMutator.h>
#include <Storages/MergeTree/MergeTreePartsMover.h>
#include <Storages/MergeTree/MergeTreeMutationEntry.h>
#include <Storages/MergeTree/MergeTreeMutationStatus.h>
#include <Storages/MergeTree/DiskSpaceMonitor.h>
#include <Common/DiskSpaceMonitor.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h>
#include <Common/SimpleIncrement.h>
#include <Core/BackgroundSchedulePool.h>
namespace DB
@ -58,8 +60,6 @@ public:
void drop(TableStructureWriteLockHolder &) override;
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
void checkTableCanBeDropped() const override;
@ -68,14 +68,9 @@ public:
ActionLock getActionLock(StorageActionBlockType action_type) override;
String getDataPath() const override { return full_path; }
CheckResults checkData(const ASTPtr & query, const Context & context) override;
private:
String path;
BackgroundProcessingPool & background_pool;
MergeTreeDataSelectExecutor reader;
MergeTreeDataWriter writer;
@ -87,14 +82,23 @@ private:
/// For clearOldParts, clearOldTemporaryDirectories.
AtomicStopwatch time_after_previous_cleanup;
mutable std::mutex currently_merging_mutex;
DataParts currently_merging;
/// Mutex for parts currently processing in background
/// merging (also with TTL), mutating or moving.
mutable std::mutex currently_processing_in_background_mutex;
/// Parts that currently participate in merge or mutation.
/// This set have to be used with `currently_processing_in_background_mutex`.
DataParts currently_merging_mutating_parts;
std::map<String, MergeTreeMutationEntry> current_mutations_by_id;
std::multimap<Int64, MergeTreeMutationEntry &> current_mutations_by_version;
std::atomic<bool> shutdown_called {false};
BackgroundProcessingPool::TaskHandle background_task_handle;
/// Task handler for merges, mutations and moves.
BackgroundProcessingPool::TaskHandle merging_mutating_task_handle;
BackgroundProcessingPool::TaskHandle moving_task_handle;
std::vector<MergeTreeData::AlterDataPartTransactionPtr> prepareAlterTransactions(
const ColumnsDescription & new_columns, const IndicesDescription & new_indices, const Context & context);
@ -107,14 +111,16 @@ private:
*/
bool merge(bool aggressive, const String & partition_id, bool final, bool deduplicate, String * out_disable_reason = nullptr);
BackgroundProcessingPoolTaskResult movePartsTask();
/// Try and find a single part to mutate and mutate it. If some part was successfully mutated, return true.
bool tryMutatePart();
BackgroundProcessingPoolTaskResult backgroundTask();
BackgroundProcessingPoolTaskResult mergeMutateTask();
Int64 getCurrentMutationVersion(
const DataPartPtr & part,
std::lock_guard<std::mutex> & /* currently_merging_mutex_lock */) const;
std::lock_guard<std::mutex> & /* currently_processing_in_background_mutex_lock */) const;
void clearOldMutations(bool truncate = false);
@ -123,6 +129,8 @@ private:
void clearColumnOrIndexInPartition(const ASTPtr & partition, const AlterCommand & alter_command, const Context & context);
void attachPartition(const ASTPtr & partition, bool part, const Context & context);
void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context);
bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override;
friend class MergeTreeBlockOutputStream;
friend class MergeTreeData;
@ -137,7 +145,6 @@ protected:
* See MergeTreeData constructor for comments on parameters.
*/
StorageMergeTree(
const String & path_,
const String & database_name_,
const String & table_name_,
const ColumnsDescription & columns_,

View File

@ -8,6 +8,7 @@
#include <Common/typeid_cast.h>
#include <Common/thread_local_rng.h>
#include <Common/ThreadPool.h>
#include <Common/DiskSpaceMonitor.h>
#include <Storages/AlterCommands.h>
#include <Storages/PartitionCommands.h>
@ -119,6 +120,7 @@ namespace ActionLocks
extern const StorageActionBlockType PartsSend;
extern const StorageActionBlockType ReplicationQueue;
extern const StorageActionBlockType PartsTTLMerge;
extern const StorageActionBlockType PartsMove;
}
@ -188,7 +190,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
const String & zookeeper_path_,
const String & replica_name_,
bool attach,
const String & path_,
const String & database_name_,
const String & table_name_,
const ColumnsDescription & columns_,
@ -205,7 +206,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
std::unique_ptr<MergeTreeSettings> settings_,
bool has_force_restore_data_flag)
: MergeTreeData(database_name_, table_name_,
path_ + escapeForFileName(table_name_) + '/',
columns_, indices_, constraints_,
context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_,
sample_by_ast_, ttl_table_ast_, merging_params_,
@ -213,12 +213,10 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
[this] (const std::string & name) { enqueuePartForCheck(name); }),
zookeeper_path(global_context.getMacros()->expand(zookeeper_path_, database_name_, table_name_)),
replica_name(global_context.getMacros()->expand(replica_name_, database_name_, table_name_)),
reader(*this), writer(*this), merger_mutator(*this, global_context.getBackgroundPool()), queue(*this), fetcher(*this),
cleanup_thread(*this), alter_thread(*this), part_check_thread(*this), restarting_thread(*this)
reader(*this), writer(*this), merger_mutator(*this, global_context.getBackgroundPool().getNumberOfThreads()),
queue(*this), fetcher(*this), cleanup_thread(*this), alter_thread(*this),
part_check_thread(*this), restarting_thread(*this)
{
if (path_.empty())
throw Exception("ReplicatedMergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME);
if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
zookeeper_path.resize(zookeeper_path.size() - 1);
/// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it.
@ -944,64 +942,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry)
return true;
}
void StorageReplicatedMergeTree::writePartLog(
PartLogElement::Type type, const ExecutionStatus & execution_status, UInt64 elapsed_ns,
const String & new_part_name,
const DataPartPtr & result_part,
const DataPartsVector & source_parts,
const MergeListEntry * merge_entry)
{
try
{
auto part_log = global_context.getPartLog(database_name);
if (!part_log)
return;
PartLogElement part_log_elem;
part_log_elem.event_type = type;
part_log_elem.error = static_cast<UInt16>(execution_status.code);
part_log_elem.exception = execution_status.message;
part_log_elem.event_time = time(nullptr);
/// TODO: Stop stopwatch in outer code to exclude ZK timings and so on
part_log_elem.duration_ms = elapsed_ns / 10000000;
part_log_elem.database_name = database_name;
part_log_elem.table_name = table_name;
part_log_elem.partition_id = MergeTreePartInfo::fromPartName(new_part_name, format_version).partition_id;
part_log_elem.part_name = new_part_name;
if (result_part)
{
part_log_elem.bytes_compressed_on_disk = result_part->bytes_on_disk;
part_log_elem.rows = result_part->rows_count;
}
part_log_elem.source_part_names.reserve(source_parts.size());
for (const auto & source_part : source_parts)
part_log_elem.source_part_names.push_back(source_part->name);
if (merge_entry)
{
part_log_elem.rows_read = (*merge_entry)->rows_read;
part_log_elem.bytes_read_uncompressed = (*merge_entry)->bytes_read_uncompressed;
part_log_elem.rows = (*merge_entry)->rows_written;
part_log_elem.bytes_uncompressed = (*merge_entry)->bytes_written_uncompressed;
}
part_log->add(part_log_elem);
}
catch (...)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
}
}
bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
{
// Log source part names just in case
@ -1063,11 +1003,10 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
}
/// Start to make the main work
size_t estimated_space_for_merge = MergeTreeDataMergerMutator::estimateNeededDiskSpace(parts);
/// Can throw an exception.
DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, estimated_space_for_merge);
DiskSpace::ReservationPtr reserved_space = reserveSpace(estimated_space_for_merge);
auto table_lock = lockStructureForShare(false, RWLockImpl::NO_QUERY);
@ -1194,12 +1133,13 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM
}
}
MergeTreePartInfo new_part_info = MergeTreePartInfo::fromPartName(
entry.new_part_name, format_version);
MutationCommands commands = queue.getMutationCommands(source_part, new_part_info.mutation);
/// Can throw an exception.
DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, estimated_space_for_result);
DiskSpace::ReservationPtr reserved_space = reserveSpace(estimated_space_for_result);
auto table_lock = lockStructureForShare(false, RWLockImpl::NO_QUERY);
@ -1225,7 +1165,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM
try
{
new_part = merger_mutator.mutatePartToTemporaryPart(future_mutated_part, commands, *merge_entry, global_context, table_lock);
new_part = merger_mutator.mutatePartToTemporaryPart(future_mutated_part, commands, *merge_entry, global_context, reserved_space.get(), table_lock);
renameTempPartAndReplace(new_part, nullptr, &transaction);
try
@ -1833,7 +1773,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
String interserver_scheme = global_context.getInterserverScheme();
if (interserver_scheme != address.scheme)
throw Exception("Interserver schemes are different '" + interserver_scheme + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::LOGICAL_ERROR);
throw Exception("Interserver schemas are different '" + interserver_scheme + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::LOGICAL_ERROR);
part_desc->res_part = fetcher.fetchPart(part_desc->found_new_part_name, source_replica_path,
address.host, address.replication_port, timeouts, user, password, interserver_scheme, false, TMP_PREFIX + "fetch_");
@ -2208,6 +2148,28 @@ BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask()
}
bool StorageReplicatedMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & part) const
{
return queue.isVirtualPart(part);
}
BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::movePartsTask()
{
try
{
if (!selectPartsAndMove())
return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO;
return BackgroundProcessingPoolTaskResult::SUCCESS;
}
catch (...)
{
tryLogCurrentException(log);
return BackgroundProcessingPoolTaskResult::ERROR;
}
}
void StorageReplicatedMergeTree::mergeSelectingTask()
{
if (!is_leader)
@ -2910,6 +2872,7 @@ void StorageReplicatedMergeTree::startup()
data_parts_exchange_endpoint->getId(replica_path), data_parts_exchange_endpoint, global_context.getInterserverIOHandler());
queue_task_handle = global_context.getBackgroundPool().addTask([this] { return queueTask(); });
move_parts_task_handle = global_context.getBackgroundPool().addTask([this] { return movePartsTask(); });
/// In this thread replica will be activated.
restarting_thread.start();
@ -2924,6 +2887,7 @@ void StorageReplicatedMergeTree::shutdown()
/// Cancel fetches, merges and mutations to force the queue_task to finish ASAP.
fetcher.blocker.cancelForever();
merger_mutator.merges_blocker.cancelForever();
parts_mover.moves_blocker.cancelForever();
restarting_thread.shutdown();
@ -2931,6 +2895,10 @@ void StorageReplicatedMergeTree::shutdown()
global_context.getBackgroundPool().removeTask(queue_task_handle);
queue_task_handle.reset();
if (move_parts_task_handle)
global_context.getBackgroundPool().removeTask(move_parts_task_handle);
move_parts_task_handle.reset();
if (data_parts_exchange_endpoint_holder)
{
data_parts_exchange_endpoint_holder->getBlocker().cancelForever();
@ -3077,7 +3045,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
for (const DataPartPtr & part : data_parts)
partition_ids.emplace(part->info.partition_id);
UInt64 disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
UInt64 disk_space = storage_policy->getMaxUnreservedFreeSpace();
for (const String & partition_id : partition_ids)
{
@ -3104,7 +3072,8 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
}
else
{
UInt64 disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
UInt64 disk_space = storage_policy->getMaxUnreservedFreeSpace();
String partition_id = getPartitionIDFromQuery(partition, query_context);
selected = merger_mutator.selectAllPartsToMergeWithinPartition(
future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason);
@ -3466,6 +3435,19 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part
case PartitionCommand::ATTACH_PARTITION:
attachPartition(command.partition, command.part, query_context);
break;
case PartitionCommand::MOVE_PARTITION:
{
switch (command.move_destination_type)
{
case PartitionCommand::MoveDestinationType::DISK:
movePartitionToDisk(command.partition, command.move_destination_name, command.part, query_context);
break;
case PartitionCommand::MoveDestinationType::VOLUME:
movePartitionToVolume(command.partition, command.move_destination_name, command.part, query_context);
break;
}
}
break;
case PartitionCommand::REPLACE_PARTITION:
{
@ -3664,7 +3646,7 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool
assertNotReadonly();
PartsTemporaryRename renamed_parts(*this, full_path + "detached/");
PartsTemporaryRename renamed_parts(*this, "detached/");
MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts);
ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, 0, false); /// TODO Allow to use quorum here.
@ -3733,15 +3715,10 @@ void StorageReplicatedMergeTree::drop(TableStructureWriteLockHolder &)
void StorageReplicatedMergeTree::rename(
const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
const String & new_path_to_db, const String & new_database_name,
const String & new_table_name, TableStructureWriteLockHolder & lock)
{
std::string new_full_path = new_path_to_db + escapeForFileName(new_table_name) + '/';
setPath(new_full_path);
database_name = new_database_name;
table_name = new_table_name;
full_path = new_full_path;
MergeTreeData::rename(new_path_to_db, new_database_name, new_table_name, lock);
/// Update table name in zookeeper
auto zookeeper = getZooKeeper();
@ -4238,12 +4215,16 @@ void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const
* Unreliable (there is a race condition) - such a partition may appear a little later.
*/
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator dir_it{getFullPath() + "detached/"}; dir_it != dir_end; ++dir_it)
for (const std::string & path : getDataPaths())
{
MergeTreePartInfo part_info;
if (MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_info, format_version)
&& part_info.partition_id == partition_id)
throw Exception("Detached partition " + partition_id + " already exists.", ErrorCodes::PARTITION_ALREADY_EXISTS);
for (Poco::DirectoryIterator dir_it{path + "detached/"}; dir_it != dir_end; ++dir_it)
{
MergeTreePartInfo part_info;
if (MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_info, format_version)
&& part_info.partition_id == partition_id)
throw Exception("Detached partition " + partition_id + " already exists.", ErrorCodes::PARTITION_ALREADY_EXISTS);
}
}
zkutil::Strings replicas;
@ -4854,7 +4835,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
{
std::lock_guard merge_selecting_lock(merge_selecting_mutex);
queue.disableMergesInRange(drop_range_fake_part_name);
queue.disableMergesInBlockRange(drop_range_fake_part_name);
}
}
@ -5120,6 +5101,9 @@ ActionLock StorageReplicatedMergeTree::getActionLock(StorageActionBlockType acti
if (action_type == ActionLocks::ReplicationQueue)
return queue.actions_blocker.cancel();
if (action_type == ActionLocks::PartsMove)
return parts_mover.moves_blocker.cancel();
return {};
}
@ -5179,7 +5163,7 @@ bool StorageReplicatedMergeTree::dropPartsInPartition(
String drop_range_fake_part_name = getPartNamePossiblyFake(format_version, drop_range_info);
{
std::lock_guard merge_selecting_lock(merge_selecting_mutex);
queue.disableMergesInRange(drop_range_fake_part_name);
queue.disableMergesInBlockRange(drop_range_fake_part_name);
}
LOG_DEBUG(log, "Disabled merges covered by range " << drop_range_fake_part_name);

View File

@ -6,6 +6,7 @@
#include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeDataMergerMutator.h>
#include <Storages/MergeTree/MergeTreePartsMover.h>
#include <Storages/MergeTree/MergeTreeDataWriter.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <Storages/MergeTree/ReplicatedMergeTreeLogEntry.h>
@ -167,8 +168,6 @@ public:
part_check_thread.enqueuePart(part_name, delay_to_check_seconds);
}
String getDataPath() const override { return full_path; }
CheckResults checkData(const ASTPtr & query, const Context & context) override;
/// Checks ability to use granularity
@ -264,6 +263,10 @@ private:
/// A task that performs actions from the queue.
BackgroundProcessingPool::TaskHandle queue_task_handle;
/// A task which move parts to another disks/volumes
/// Transparent for replication.
BackgroundProcessingPool::TaskHandle move_parts_task_handle;
/// A task that selects parts to merge.
BackgroundSchedulePool::TaskHolder merge_selecting_task;
/// It is acquired for each iteration of the selection of parts to merge or each OPTIMIZE query.
@ -334,10 +337,9 @@ private:
DataPartsVector checkPartChecksumsAndCommit(Transaction & transaction,
const DataPartPtr & part);
void getCommitPartOps(
Coordination::Requests & ops,
MutableDataPartPtr & part,
const String & block_id_path = "") const;
bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override;
void getCommitPartOps(Coordination::Requests & ops, MutableDataPartPtr & part, const String & block_id_path = "") const;
/// Updates info about part columns and checksums in ZooKeeper and commits transaction if successful.
void updatePartHeaderInZooKeeperAndCommit(
@ -365,12 +367,6 @@ private:
*/
bool executeLogEntry(LogEntry & entry);
void writePartLog(
PartLogElement::Type type, const ExecutionStatus & execution_status, UInt64 elapsed_ns,
const String & new_part_name,
const DataPartPtr & result_part,
const DataPartsVector & source_parts,
const MergeListEntry * merge_entry);
void executeDropRange(const LogEntry & entry);
@ -379,6 +375,7 @@ private:
bool tryExecutePartMutation(const LogEntry & entry);
bool executeFetch(LogEntry & entry);
void executeClearColumnOrIndexInPartition(const LogEntry & entry);
@ -403,6 +400,11 @@ private:
*/
BackgroundProcessingPoolTaskResult queueTask();
/// Perform moves of parts to another disks.
/// Local operation, doesn't interact with replicationg queue.
BackgroundProcessingPoolTaskResult movePartsTask();
/// Postcondition:
/// either leader_election is fully initialized (node in ZK is created and the watching thread is launched)
/// or an exception is thrown and leader_election is destroyed.
@ -459,6 +461,7 @@ private:
std::unordered_set<String> currently_fetching_parts;
std::mutex currently_fetching_parts_mutex;
/// With the quorum being tracked, add a replica to the quorum for the part.
void updateQuorum(const String & part_name);
@ -527,7 +530,7 @@ protected:
const String & zookeeper_path_,
const String & replica_name_,
bool attach,
const String & path_, const String & database_name_, const String & name_,
const String & database_name_, const String & name_,
const ColumnsDescription & columns_,
const IndicesDescription & indices_,
const ConstraintsDescription & constraints_,

View File

@ -26,7 +26,7 @@ public:
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
String getDataPath() const override { return path; }
Strings getDataPaths() const override { return {path}; }
protected:
StorageSetOrJoinBase(

View File

@ -53,7 +53,7 @@ public:
std::string full_path() const { return path + escapeForFileName(table_name) + '/';}
String getDataPath() const override { return full_path(); }
Strings getDataPaths() const override { return {full_path()}; }
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;

View File

@ -186,7 +186,7 @@ Block TinyLogBlockInputStream::readImpl()
{
/// if there are no files in the folder, it means that the table is empty
if (Poco::DirectoryIterator(storage.full_path()) == Poco::DirectoryIterator())
if (Poco::DirectoryIterator(storage.fullPath()) == Poco::DirectoryIterator())
return res;
}
@ -200,7 +200,7 @@ Block TinyLogBlockInputStream::readImpl()
}
catch (Exception & e)
{
e.addMessage("while reading column " + name_type.name + " at " + storage.full_path());
e.addMessage("while reading column " + name_type.name + " at " + storage.fullPath());
throw;
}

View File

@ -50,9 +50,9 @@ public:
};
using Files_t = std::map<String, ColumnData>;
std::string full_path() const { return path + escapeForFileName(table_name) + '/';}
std::string fullPath() const { return path + escapeForFileName(table_name) + '/';}
String getDataPath() const override { return full_path(); }
Strings getDataPaths() const override { return {fullPath()}; }
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;

View File

@ -0,0 +1,63 @@
#include <DataStreams/OneBlockInputStream.h>
#include <Storages/System/StorageSystemDisks.h>
namespace DB
{
namespace ErrorCodes
{
}
StorageSystemDisks::StorageSystemDisks(const std::string & name_)
: name(name_)
{
setColumns(ColumnsDescription(
{
{"name", std::make_shared<DataTypeString>()},
{"path", std::make_shared<DataTypeString>()},
{"free_space", std::make_shared<DataTypeUInt64>()},
{"total_space", std::make_shared<DataTypeUInt64>()},
{"keep_free_space", std::make_shared<DataTypeUInt64>()},
}));
}
BlockInputStreams StorageSystemDisks::read(
const Names & column_names,
const SelectQueryInfo & /*query_info*/,
const Context & context,
QueryProcessingStage::Enum /*processed_stage*/,
const size_t /*max_block_size*/,
const unsigned /*num_streams*/)
{
check(column_names);
MutableColumnPtr col_name = ColumnString::create();
MutableColumnPtr col_path = ColumnString::create();
MutableColumnPtr col_free = ColumnUInt64::create();
MutableColumnPtr col_total = ColumnUInt64::create();
MutableColumnPtr col_keep = ColumnUInt64::create();
const auto & disk_selector = context.getDiskSelector();
for (const auto & [disk_name, disk_ptr] : disk_selector.getDisksMap())
{
col_name->insert(disk_name);
col_path->insert(disk_ptr->getPath());
col_free->insert(disk_ptr->getAvailableSpace());
col_total->insert(disk_ptr->getTotalSpace());
col_keep->insert(disk_ptr->getKeepingFreeSpace());
}
Block res = getSampleBlock().cloneEmpty();
size_t col_num = 0;
res.getByPosition(col_num++).column = std::move(col_name);
res.getByPosition(col_num++).column = std::move(col_path);
res.getByPosition(col_num++).column = std::move(col_free);
res.getByPosition(col_num++).column = std::move(col_total);
res.getByPosition(col_num++).column = std::move(col_keep);
return BlockInputStreams(1, std::make_shared<OneBlockInputStream>(res));
}
}

View File

@ -0,0 +1,40 @@
#pragma once
#include <ext/shared_ptr_helper.h>
#include <Formats/FormatSettings.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeData.h>
namespace DB
{
class Context;
/** Implements the system table `disks`, which allows you to get information about all disks.
*/
class StorageSystemDisks : public ext::shared_ptr_helper<StorageSystemDisks>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemDisks>;
public:
std::string getName() const override { return "SystemDisks"; }
std::string getTableName() const override { return name; }
std::string getDatabaseName() const override { return "system"; }
BlockInputStreams read(
const Names & column_names,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams) override;
private:
const std::string name;
protected:
StorageSystemDisks(const std::string & name_);
};
}

View File

@ -45,6 +45,7 @@ StorageSystemParts::StorageSystemParts(const std::string & name_)
{"database", std::make_shared<DataTypeString>()},
{"table", std::make_shared<DataTypeString>()},
{"engine", std::make_shared<DataTypeString>()},
{"disk_name", std::make_shared<DataTypeString>()},
{"path", std::make_shared<DataTypeString>()},
{"hash_of_all_files", std::make_shared<DataTypeString>()},
@ -108,6 +109,7 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns_, const Sto
columns_[i++]->insert(info.database);
columns_[i++]->insert(info.table);
columns_[i++]->insert(info.engine);
columns_[i++]->insert(part->disk->getName());
columns_[i++]->insert(part->getFullPath());
if (has_state_column)

View File

@ -43,6 +43,7 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const std::string & name_)
{"database", std::make_shared<DataTypeString>()},
{"table", std::make_shared<DataTypeString>()},
{"engine", std::make_shared<DataTypeString>()},
{"disk_name", std::make_shared<DataTypeString>()},
{"path", std::make_shared<DataTypeString>()},
{"column", std::make_shared<DataTypeString>()},
@ -135,7 +136,9 @@ void StorageSystemPartsColumns::processNextStorage(MutableColumns & columns_, co
columns_[j++]->insert(info.database);
columns_[j++]->insert(info.table);
columns_[j++]->insert(info.engine);
columns_[j++]->insert(part->disk->getName());
columns_[j++]->insert(part->getFullPath());
columns_[j++]->insert(column.name);
columns_[j++]->insert(column.type->getName());

View File

@ -0,0 +1,80 @@
#include <Columns/ColumnArray.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Storages/System/StorageSystemStoragePolicies.h>
#include <DataTypes/DataTypeArray.h>
namespace DB
{
namespace ErrorCodes
{
}
StorageSystemStoragePolicies::StorageSystemStoragePolicies(const std::string & name_)
: name(name_)
{
setColumns(
ColumnsDescription({
{"policy_name", std::make_shared<DataTypeString>()},
{"volume_name", std::make_shared<DataTypeString>()},
{"volume_priority", std::make_shared<DataTypeUInt64>()},
{"disks", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{"max_data_part_size", std::make_shared<DataTypeUInt64>()},
{"move_factor", std::make_shared<DataTypeFloat32>()}
}));
}
BlockInputStreams StorageSystemStoragePolicies::read(
const Names & column_names,
const SelectQueryInfo & /*query_info*/,
const Context & context,
QueryProcessingStage::Enum /*processed_stage*/,
const size_t /*max_block_size*/,
const unsigned /*num_streams*/)
{
check(column_names);
MutableColumnPtr col_policy_name = ColumnString::create();
MutableColumnPtr col_volume_name = ColumnString::create();
MutableColumnPtr col_priority = ColumnUInt64::create();
MutableColumnPtr col_disks = ColumnArray::create(ColumnString::create());
MutableColumnPtr col_max_part_size = ColumnUInt64::create();
MutableColumnPtr col_move_factor = ColumnFloat32::create();
const auto & policy_selector = context.getStoragePolicySelector();
for (const auto & [policy_name, policy_ptr] : policy_selector.getPoliciesMap())
{
const auto & volumes = policy_ptr->getVolumes();
for (size_t i = 0; i != volumes.size(); ++i)
{
col_policy_name->insert(policy_name);
col_volume_name->insert(volumes[i]->getName());
col_priority->insert(i + 1);
Array disks;
disks.reserve(volumes[i]->disks.size());
for (const auto & disk_ptr : volumes[i]->disks)
disks.push_back(disk_ptr->getName());
col_disks->insert(disks);
col_max_part_size->insert(volumes[i]->max_data_part_size);
col_move_factor->insert(policy_ptr->getMoveFactor());
}
}
Block res = getSampleBlock().cloneEmpty();
size_t col_num = 0;
res.getByPosition(col_num++).column = std::move(col_policy_name);
res.getByPosition(col_num++).column = std::move(col_volume_name);
res.getByPosition(col_num++).column = std::move(col_priority);
res.getByPosition(col_num++).column = std::move(col_disks);
res.getByPosition(col_num++).column = std::move(col_max_part_size);
res.getByPosition(col_num++).column = std::move(col_move_factor);
return BlockInputStreams(1, std::make_shared<OneBlockInputStream>(res));
}
}

View File

@ -0,0 +1,40 @@
#pragma once
#include <ext/shared_ptr_helper.h>
#include <Formats/FormatSettings.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeData.h>
namespace DB
{
class Context;
/** Implements the system table `storage`, which allows you to get information about all disks.
*/
class StorageSystemStoragePolicies : public ext::shared_ptr_helper<StorageSystemStoragePolicies>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemStoragePolicies>;
public:
std::string getName() const override { return "SystemStoragePolicies"; }
std::string getTableName() const override { return name; }
std::string getDatabaseName() const override { return "system"; }
BlockInputStreams read(
const Names & column_names,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams) override;
private:
const std::string name;
protected:
StorageSystemStoragePolicies(const std::string & name_);
};
}

View File

@ -34,7 +34,7 @@ StorageSystemTables::StorageSystemTables(const std::string & name_)
{"name", std::make_shared<DataTypeString>()},
{"engine", std::make_shared<DataTypeString>()},
{"is_temporary", std::make_shared<DataTypeUInt8>()},
{"data_path", std::make_shared<DataTypeString>()},
{"data_paths", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{"metadata_path", std::make_shared<DataTypeString>()},
{"metadata_modification_time", std::make_shared<DataTypeDateTime>()},
{"dependencies_database", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
@ -45,6 +45,7 @@ StorageSystemTables::StorageSystemTables(const std::string & name_)
{"sorting_key", std::make_shared<DataTypeString>()},
{"primary_key", std::make_shared<DataTypeString>()},
{"sampling_key", std::make_shared<DataTypeString>()},
{"storage_policy", std::make_shared<DataTypeString>()},
}));
}
@ -161,6 +162,9 @@ protected:
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
}
}
@ -208,7 +212,14 @@ protected:
res_columns[res_index++]->insert(0u); // is_temporary
if (columns_mask[src_index++])
res_columns[res_index++]->insert(table->getDataPath());
{
Array table_paths_array;
auto paths = table->getDataPaths();
table_paths_array.reserve(paths.size());
for (const String & path : paths)
table_paths_array.push_back(path);
res_columns[res_index++]->insert(table_paths_array);
}
if (columns_mask[src_index++])
res_columns[res_index++]->insert(database->getTableMetadataPath(table_name));
@ -301,6 +312,15 @@ protected:
else
res_columns[res_index++]->insertDefault();
}
if (columns_mask[src_index++])
{
auto policy = table->getStoragePolicy();
if (policy)
res_columns[res_index++]->insert(policy->getName());
else
res_columns[res_index++]->insertDefault();
}
}
}

View File

@ -34,6 +34,8 @@
#include <Storages/System/StorageSystemTables.h>
#include <Storages/System/StorageSystemZooKeeper.h>
#include <Storages/System/StorageSystemContributors.h>
#include <Storages/System/StorageSystemDisks.h>
#include <Storages/System/StorageSystemStoragePolicies.h>
namespace DB
@ -67,6 +69,8 @@ void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper)
system_database.attachTable("parts", StorageSystemParts::create("parts"));
system_database.attachTable("detached_parts", createDetachedPartsTable());
system_database.attachTable("parts_columns", StorageSystemPartsColumns::create("parts_columns"));
system_database.attachTable("disks", StorageSystemDisks::create("disks"));
system_database.attachTable("storage_policies", StorageSystemStoragePolicies::create("storage_policies"));
system_database.attachTable("processes", StorageSystemProcesses::create("processes"));
system_database.attachTable("metrics", StorageSystemMetrics::create("metrics"));
system_database.attachTable("merges", StorageSystemMerges::create("merges"));

View File

@ -114,7 +114,7 @@ class ClickHouseCluster:
cmd += " client"
return cmd
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False):
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]):
"""Add an instance to the cluster.
name - the name of the instance directory and the value of the 'instance' macro in ClickHouse.
@ -135,7 +135,7 @@ class ClickHouseCluster:
self.zookeeper_config_path, with_mysql, with_kafka, with_mongo, self.base_configs_dir, self.server_bin_path,
self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname,
env_variables=env_variables, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, ipv6_address=ipv6_address,
with_installed_binary=with_installed_binary)
with_installed_binary=with_installed_binary, tmpfs=tmpfs)
self.instances[name] = instance
if ipv4_address is not None or ipv6_address is not None:
@ -395,6 +395,10 @@ class ClickHouseCluster:
instance.client = None
def open_bash_shell(self, instance_name):
os.system(' '.join(self.base_cmd + ['exec', instance_name, '/bin/bash']))
def get_kazoo_client(self, zoo_instance_name):
zk = KazooClient(hosts=self.get_instance_ip(zoo_instance_name))
zk.start()
@ -435,6 +439,7 @@ services:
{odbc_bridge_volume}
{odbc_ini_path}
entrypoint: {entrypoint_cmd}
tmpfs: {tmpfs}
cap_add:
- SYS_PTRACE
depends_on: {depends_on}
@ -456,7 +461,7 @@ class ClickHouseInstance:
self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros,
with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_mongo, base_configs_dir, server_bin_path, odbc_bridge_bin_path,
clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test",
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False):
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]):
self.name = name
self.base_cmd = cluster.base_cmd[:]
@ -464,6 +469,7 @@ class ClickHouseInstance:
self.cluster = cluster
self.hostname = hostname if hostname is not None else self.name
self.tmpfs = tmpfs[:]
self.custom_config_dir = p.abspath(p.join(base_path, custom_config_dir)) if custom_config_dir else None
self.custom_main_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_main_configs]
self.custom_user_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_user_configs]
@ -811,6 +817,7 @@ class ClickHouseInstance:
configs_dir=configs_dir,
config_d_dir=config_d_dir,
db_dir=db_dir,
tmpfs=str(self.tmpfs),
logs_dir=logs_dir,
depends_on=str(depends_on),
user=os.getuid(),

View File

@ -0,0 +1,16 @@
<yandex>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</yandex>

View File

@ -0,0 +1,81 @@
<yandex>
<storage_configuration>
<disks>
<default>
<keep_free_space_bytes>1024</keep_free_space_bytes>
</default>
<jbod1>
<path>/jbod1/</path>
</jbod1>
<jbod2>
<path>/jbod2/</path>
<keep_free_space_bytes>10485760</keep_free_space_bytes>
<!-- 10MB -->
</jbod2>
<external>
<path>/external/</path>
</external>
</disks>
<policies>
<small_jbod_with_external>
<volumes>
<main>
<disk>jbod1</disk>
</main>
<external>
<disk>external</disk>
</external>
</volumes>
</small_jbod_with_external>
<!-- store on JBOD by default (round-robin), store big parts on external -->
<jbods_with_external>
<volumes>
<main>
<disk>jbod1</disk>
<disk>jbod2</disk>
<max_data_part_size_bytes>10485760</max_data_part_size_bytes>
<!-- 10MB -->
</main>
<external>
<disk>external</disk>
</external>
</volumes>
</jbods_with_external>
<!-- Moving all parts jbod1 if accuired more than 70% -->
<moving_jbod_with_external>
<volumes>
<main>
<disk>jbod1</disk>
</main>
<external>
<disk>external</disk>
</external>
</volumes>
<move_factor>0.7</move_factor>
</moving_jbod_with_external>
<!-- store local by default, store big parts on external -->
<default_disk_with_external>
<volumes>
<small>
<disk>default</disk>
<max_data_part_size_bytes>2097152</max_data_part_size_bytes>
<!-- 2MB -->
</small>
<big>
<disk>external</disk>
<max_data_part_size_bytes>20971520</max_data_part_size_bytes>
<!-- 20MB -->
</big>
</volumes>
</default_disk_with_external>
</policies>
</storage_configuration>
</yandex>

View File

@ -0,0 +1,17 @@
<yandex>
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/log.log</log>
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
<size>1000M</size>
<count>10</count>
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
</logger>
<part_log>
<database>system</database>
<table>part_log</table>
<flush_interval_milliseconds>500</flush_interval_milliseconds>
</part_log>
</yandex>

View File

@ -0,0 +1,892 @@
import time
import pytest
import random
import string
import json
from multiprocessing.dummy import Pool
from helpers.client import QueryRuntimeException
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1',
config_dir='configs',
main_configs=['configs/logs_config.xml'],
with_zookeeper=True,
tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'],
macros={"shard": 0, "replica": 1} )
node2 = cluster.add_instance('node2',
config_dir='configs',
main_configs=['configs/logs_config.xml'],
with_zookeeper=True,
tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'],
macros={"shard": 0, "replica": 2} )
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_system_tables(start_cluster):
expected_disks_data = [
{
"name": "default",
"path": "/var/lib/clickhouse/",
"keep_free_space": '1024',
},
{
"name": "jbod1",
"path": "/jbod1/",
"keep_free_space": '0',
},
{
"name": "jbod2",
"path": "/jbod2/",
"keep_free_space": '10485760',
},
{
"name": "external",
"path": "/external/",
"keep_free_space": '0',
}
]
click_disk_data = json.loads(node1.query("SELECT name, path, keep_free_space FROM system.disks FORMAT JSON"))["data"]
assert sorted(click_disk_data, key=lambda x: x["name"]) == sorted(expected_disks_data, key=lambda x: x["name"])
expected_policies_data = [
{
"policy_name": "small_jbod_with_external",
"volume_name": "main",
"volume_priority": "1",
"disks": ["jbod1"],
"max_data_part_size": "0",
"move_factor": 0.1,
},
{
"policy_name": "small_jbod_with_external",
"volume_name": "external",
"volume_priority": "2",
"disks": ["external"],
"max_data_part_size": "0",
"move_factor": 0.1,
},
{
"policy_name": "jbods_with_external",
"volume_name": "main",
"volume_priority": "1",
"disks": ["jbod1", "jbod2"],
"max_data_part_size": "10485760",
"move_factor": 0.1,
},
{
"policy_name": "jbods_with_external",
"volume_name": "external",
"volume_priority": "2",
"disks": ["external"],
"max_data_part_size": "0",
"move_factor": 0.1,
},
{
"policy_name": "moving_jbod_with_external",
"volume_name": "main",
"volume_priority": "1",
"disks": ["jbod1"],
"max_data_part_size": "0",
"move_factor": 0.7,
},
{
"policy_name": "moving_jbod_with_external",
"volume_name": "external",
"volume_priority": "2",
"disks": ["external"],
"max_data_part_size": "0",
"move_factor": 0.7,
},
{
"policy_name": "default_disk_with_external",
"volume_name": "small",
"volume_priority": "1",
"disks": ["default"],
"max_data_part_size": "2097152",
"move_factor": 0.1,
},
{
"policy_name": "default_disk_with_external",
"volume_name": "big",
"volume_priority": "2",
"disks": ["external"],
"max_data_part_size": "20971520",
"move_factor": 0.1,
},
]
clickhouse_policies_data = json.loads(node1.query("SELECT * FROM system.storage_policies WHERE policy_name != 'default' FORMAT JSON"))["data"]
def key(x):
return (x["policy_name"], x["volume_name"], x["volume_priority"])
assert sorted(clickhouse_policies_data, key=key) == sorted(expected_policies_data, key=key)
def test_query_parser(start_cluster):
try:
with pytest.raises(QueryRuntimeException):
node1.query("""
CREATE TABLE table_with_absent_policy (
d UInt64
) ENGINE = MergeTree()
ORDER BY d
SETTINGS storage_policy_name='very_exciting_policy'
""")
with pytest.raises(QueryRuntimeException):
node1.query("""
CREATE TABLE table_with_absent_policy (
d UInt64
) ENGINE = MergeTree()
ORDER BY d
SETTINGS storage_policy_name='jbod1'
""")
node1.query("""
CREATE TABLE table_with_normal_policy (
d UInt64
) ENGINE = MergeTree()
ORDER BY d
SETTINGS storage_policy_name='default'
""")
node1.query("INSERT INTO table_with_normal_policy VALUES (5)")
with pytest.raises(QueryRuntimeException):
node1.query("ALTER TABLE table_with_normal_policy MOVE PARTITION 'all' TO VOLUME 'some_volume'")
with pytest.raises(QueryRuntimeException):
node1.query("ALTER TABLE table_with_normal_policy MOVE PARTITION 'all' TO DISK 'some_volume'")
with pytest.raises(QueryRuntimeException):
node1.query("ALTER TABLE table_with_normal_policy MOVE PART 'xxxxx' TO DISK 'jbod1'")
with pytest.raises(QueryRuntimeException):
node1.query("ALTER TABLE table_with_normal_policy MOVE PARTITION 'yyyy' TO DISK 'jbod1'")
with pytest.raises(QueryRuntimeException):
node1.query("ALTER TABLE table_with_normal_policy MODIFY SETTING storage_policy_name='moving_jbod_with_external'")
finally:
node1.query("DROP TABLE IF EXISTS table_with_normal_policy")
def get_random_string(length):
return ''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(length))
def get_used_disks_for_table(node, table_name):
return node.query("select disk_name from system.parts where table == '{}' and active=1 order by modification_time".format(table_name)).strip().split('\n')
@pytest.mark.parametrize("name,engine", [
("mt_on_jbod","MergeTree()"),
("replicated_mt_on_jbod","ReplicatedMergeTree('/clickhouse/replicated_mt_on_jbod', '1')",),
])
def test_round_robin(start_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
d UInt64
) ENGINE = {engine}
ORDER BY d
SETTINGS storage_policy_name='jbods_with_external'
""".format(name=name, engine=engine))
# first should go to the jbod1
node1.query("insert into {} select * from numbers(10000)".format(name))
used_disk = get_used_disks_for_table(node1, name)
assert len(used_disk) == 1, 'More than one disk used for single insert'
node1.query("insert into {} select * from numbers(10000, 10000)".format(name))
used_disks = get_used_disks_for_table(node1, name)
assert len(used_disks) == 2, 'Two disks should be used for two parts'
assert used_disks[0] != used_disks[1], "Should write to different disks"
node1.query("insert into {} select * from numbers(20000, 10000)".format(name))
used_disks = get_used_disks_for_table(node1, name)
# jbod1 -> jbod2 -> jbod1 -> jbod2 ... etc
assert len(used_disks) == 3
assert used_disks[0] != used_disks[1]
assert used_disks[2] == used_disks[0]
finally:
node1.query("DROP TABLE IF EXISTS {}".format(name))
@pytest.mark.parametrize("name,engine", [
("mt_with_huge_part","MergeTree()"),
("replicated_mt_with_huge_part","ReplicatedMergeTree('/clickhouse/replicated_mt_with_huge_part', '1')",),
])
def test_max_data_part_size(start_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
s1 String
) ENGINE = {engine}
ORDER BY tuple()
SETTINGS storage_policy_name='jbods_with_external'
""".format(name=name, engine=engine))
data = [] # 10MB in total
for i in range(10):
data.append(get_random_string(1024 * 1024)) # 1MB row
node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data])))
used_disks = get_used_disks_for_table(node1, name)
assert len(used_disks) == 1
assert used_disks[0] == 'external'
finally:
node1.query("DROP TABLE IF EXISTS {}".format(name))
@pytest.mark.parametrize("name,engine", [
("mt_with_overflow","MergeTree()"),
("replicated_mt_with_overflow","ReplicatedMergeTree('/clickhouse/replicated_mt_with_overflow', '1')",),
])
def test_jbod_overflow(start_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
s1 String
) ENGINE = {engine}
ORDER BY tuple()
SETTINGS storage_policy_name='small_jbod_with_external'
""".format(name=name, engine=engine))
node1.query("SYSTEM STOP MERGES")
# small jbod size is 40MB, so lets insert 5MB batch 7 times
for i in range(7):
data = [] # 5MB in total
for i in range(5):
data.append(get_random_string(1024 * 1024)) # 1MB row
node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data])))
used_disks = get_used_disks_for_table(node1, name)
assert all(disk == 'jbod1' for disk in used_disks)
# should go to the external disk (jbod is overflown)
data = [] # 10MB in total
for i in range(10):
data.append(get_random_string(1024 * 1024)) # 1MB row
node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data])))
used_disks = get_used_disks_for_table(node1, name)
assert used_disks[-1] == 'external'
node1.query("SYSTEM START MERGES")
time.sleep(1)
node1.query("OPTIMIZE TABLE {} FINAL".format(name))
time.sleep(2)
disks_for_merges = node1.query("SELECT disk_name FROM system.parts WHERE table == '{}' AND level >= 1 and active = 1 ORDER BY modification_time".format(name)).strip().split('\n')
assert all(disk == 'external' for disk in disks_for_merges)
finally:
node1.query("DROP TABLE IF EXISTS {}".format(name))
@pytest.mark.parametrize("name,engine", [
("moving_mt","MergeTree()"),
("moving_replicated_mt","ReplicatedMergeTree('/clickhouse/moving_replicated_mt', '1')",),
])
def test_background_move(start_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
s1 String
) ENGINE = {engine}
ORDER BY tuple()
SETTINGS storage_policy_name='moving_jbod_with_external'
""".format(name=name, engine=engine))
for i in range(5):
data = [] # 5MB in total
for i in range(5):
data.append(get_random_string(1024 * 1024)) # 1MB row
# small jbod size is 40MB, so lets insert 5MB batch 5 times
node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data])))
used_disks = get_used_disks_for_table(node1, name)
retry = 20
i = 0
while not sum(1 for x in used_disks if x == 'jbod1') <= 2 and i < retry:
time.sleep(0.5)
used_disks = get_used_disks_for_table(node1, name)
i += 1
assert sum(1 for x in used_disks if x == 'jbod1') <= 2
# first (oldest) part was moved to external
assert used_disks[0] == 'external'
path = node1.query("SELECT path_on_disk FROM system.part_log WHERE table = '{}' AND event_type='MovePart' ORDER BY event_time LIMIT 1".format(name))
# first (oldest) part was moved to external
assert path.startswith("/external")
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
@pytest.mark.parametrize("name,engine", [
("stopped_moving_mt","MergeTree()"),
("stopped_moving_replicated_mt","ReplicatedMergeTree('/clickhouse/stopped_moving_replicated_mt', '1')",),
])
def test_start_stop_moves(start_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
s1 String
) ENGINE = {engine}
ORDER BY tuple()
SETTINGS storage_policy_name='moving_jbod_with_external'
""".format(name=name, engine=engine))
node1.query("INSERT INTO {} VALUES ('HELLO')".format(name))
node1.query("INSERT INTO {} VALUES ('WORLD')".format(name))
used_disks = get_used_disks_for_table(node1, name)
assert all(d == "jbod1" for d in used_disks), "All writes shoud go to jbods"
first_part = node1.query("SELECT name FROM system.parts WHERE table = '{}' and active = 1 ORDER BY modification_time LIMIT 1".format(name)).strip()
node1.query("SYSTEM STOP MOVES")
with pytest.raises(QueryRuntimeException):
node1.query("ALTER TABLE {} MOVE PART '{}' TO VOLUME 'external'".format(name, first_part))
used_disks = get_used_disks_for_table(node1, name)
assert all(d == "jbod1" for d in used_disks), "Blocked moves doesn't actually move something"
node1.query("SYSTEM START MOVES")
node1.query("ALTER TABLE {} MOVE PART '{}' TO VOLUME 'external'".format(name, first_part))
disk = node1.query("SELECT disk_name FROM system.parts WHERE table = '{}' and name = '{}' and active = 1".format(name, first_part)).strip()
assert disk == "external"
node1.query("TRUNCATE TABLE {}".format(name))
node1.query("SYSTEM STOP MOVES {}".format(name))
node1.query("SYSTEM STOP MERGES {}".format(name))
for i in range(5):
data = [] # 5MB in total
for i in range(5):
data.append(get_random_string(1024 * 1024)) # 1MB row
# jbod size is 40MB, so lets insert 5MB batch 7 times
node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data])))
used_disks = get_used_disks_for_table(node1, name)
retry = 5
i = 0
while not sum(1 for x in used_disks if x == 'jbod1') <= 2 and i < retry:
time.sleep(0.1)
used_disks = get_used_disks_for_table(node1, name)
i += 1
# first (oldest) part doesn't move anywhere
assert used_disks[0] == 'jbod1'
node1.query("SYSTEM START MOVES {}".format(name))
node1.query("SYSTEM START MERGES {}".format(name))
# wait sometime until background backoff finishes
retry = 30
i = 0
while not sum(1 for x in used_disks if x == 'jbod1') <= 2 and i < retry:
time.sleep(1)
used_disks = get_used_disks_for_table(node1, name)
i += 1
assert sum(1 for x in used_disks if x == 'jbod1') <= 2
# first (oldest) part moved to external
assert used_disks[0] == 'external'
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
def get_path_for_part_from_part_log(node, table, part_name):
node.query("SYSTEM FLUSH LOGS")
path = node.query("SELECT path_on_disk FROM system.part_log WHERE table = '{}' and part_name = '{}' ORDER BY event_time DESC LIMIT 1".format(table, part_name))
return path.strip()
def get_paths_for_partition_from_part_log(node, table, partition_id):
node.query("SYSTEM FLUSH LOGS")
paths = node.query("SELECT path_on_disk FROM system.part_log WHERE table = '{}' and partition_id = '{}' ORDER BY event_time DESC".format(table, partition_id))
return paths.strip().split('\n')
@pytest.mark.parametrize("name,engine", [
("altering_mt","MergeTree()"),
#("altering_replicated_mt","ReplicatedMergeTree('/clickhouse/altering_replicated_mt', '1')",),
# SYSTEM STOP MERGES doesn't disable merges assignments
])
def test_alter_move(start_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
EventDate Date,
number UInt64
) ENGINE = {engine}
ORDER BY tuple()
PARTITION BY toYYYYMM(EventDate)
SETTINGS storage_policy_name='jbods_with_external'
""".format(name=name, engine=engine))
node1.query("SYSTEM STOP MERGES {}".format(name)) # to avoid conflicts
node1.query("INSERT INTO {} VALUES(toDate('2019-03-15'), 65)".format(name))
node1.query("INSERT INTO {} VALUES(toDate('2019-03-16'), 66)".format(name))
node1.query("INSERT INTO {} VALUES(toDate('2019-04-10'), 42)".format(name))
node1.query("INSERT INTO {} VALUES(toDate('2019-04-11'), 43)".format(name))
used_disks = get_used_disks_for_table(node1, name)
assert all(d.startswith("jbod") for d in used_disks), "All writes shoud go to jbods"
first_part = node1.query("SELECT name FROM system.parts WHERE table = '{}' and active = 1 ORDER BY modification_time LIMIT 1".format(name)).strip()
time.sleep(1)
node1.query("ALTER TABLE {} MOVE PART '{}' TO VOLUME 'external'".format(name, first_part))
disk = node1.query("SELECT disk_name FROM system.parts WHERE table = '{}' and name = '{}' and active = 1".format(name, first_part)).strip()
assert disk == 'external'
assert get_path_for_part_from_part_log(node1, name, first_part).startswith("/external")
time.sleep(1)
node1.query("ALTER TABLE {} MOVE PART '{}' TO DISK 'jbod1'".format(name, first_part))
disk = node1.query("SELECT disk_name FROM system.parts WHERE table = '{}' and name = '{}' and active = 1".format(name, first_part)).strip()
assert disk == 'jbod1'
assert get_path_for_part_from_part_log(node1, name, first_part).startswith("/jbod1")
time.sleep(1)
node1.query("ALTER TABLE {} MOVE PARTITION 201904 TO VOLUME 'external'".format(name))
disks = node1.query("SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201904' and active = 1".format(name)).strip().split('\n')
assert len(disks) == 2
assert all(d == "external" for d in disks)
assert all(path.startswith("/external") for path in get_paths_for_partition_from_part_log(node1, name, '201904')[:2])
time.sleep(1)
node1.query("ALTER TABLE {} MOVE PARTITION 201904 TO DISK 'jbod2'".format(name))
disks = node1.query("SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201904' and active = 1".format(name)).strip().split('\n')
assert len(disks) == 2
assert all(d == "jbod2" for d in disks)
assert all(path.startswith("/jbod2") for path in get_paths_for_partition_from_part_log(node1, name, '201904')[:2])
assert node1.query("SELECT COUNT() FROM {}".format(name)) == "4\n"
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
def produce_alter_move(node, name):
move_type = random.choice(["PART", "PARTITION"])
if move_type == "PART":
for _ in range(10):
try:
parts = node1.query("SELECT name from system.parts where table = '{}' and active = 1".format(name)).strip().split('\n')
break
except QueryRuntimeException:
pass
else:
raise Exception("Cannot select from system.parts")
move_part = random.choice(["'" + part + "'" for part in parts])
else:
move_part = random.choice([201903, 201904])
move_disk = random.choice(["DISK", "VOLUME"])
if move_disk == "DISK":
move_volume = random.choice(["'external'", "'jbod1'", "'jbod2'"])
else:
move_volume = random.choice(["'main'", "'external'"])
try:
node1.query("ALTER TABLE {} MOVE {mt} {mp} TO {md} {mv}".format(
name, mt=move_type, mp=move_part, md=move_disk, mv=move_volume))
except QueryRuntimeException as ex:
pass
@pytest.mark.parametrize("name,engine", [
("concurrently_altering_mt","MergeTree()"),
("concurrently_altering_replicated_mt","ReplicatedMergeTree('/clickhouse/concurrently_altering_replicated_mt', '1')",),
])
def test_concurrent_alter_move(start_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
EventDate Date,
number UInt64
) ENGINE = {engine}
ORDER BY tuple()
PARTITION BY toYYYYMM(EventDate)
SETTINGS storage_policy_name='jbods_with_external'
""".format(name=name, engine=engine))
def insert(num):
for i in range(num):
day = random.randint(11, 30)
value = random.randint(1, 1000000)
month = '0' + str(random.choice([3, 4]))
node1.query("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value))
def alter_move(num):
for i in range(num):
produce_alter_move(node1, name)
def alter_update(num):
for i in range(num):
node1.query("ALTER TABLE {} UPDATE number = number + 1 WHERE 1".format(name))
def optimize_table(num):
for i in range(num):
node1.query("OPTIMIZE TABLE {} FINAL".format(name))
p = Pool(15)
tasks = []
for i in range(5):
tasks.append(p.apply_async(insert, (100,)))
tasks.append(p.apply_async(alter_move, (100,)))
tasks.append(p.apply_async(alter_update, (100,)))
tasks.append(p.apply_async(optimize_table, (100,)))
for task in tasks:
task.get(timeout=60)
assert node1.query("SELECT 1") == "1\n"
assert node1.query("SELECT COUNT() FROM {}".format(name)) == "500\n"
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
@pytest.mark.parametrize("name,engine", [
("concurrently_dropping_mt","MergeTree()"),
("concurrently_dropping_replicated_mt","ReplicatedMergeTree('/clickhouse/concurrently_dropping_replicated_mt', '1')",),
])
def test_concurrent_alter_move_and_drop(start_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
EventDate Date,
number UInt64
) ENGINE = {engine}
ORDER BY tuple()
PARTITION BY toYYYYMM(EventDate)
SETTINGS storage_policy_name='jbods_with_external'
""".format(name=name, engine=engine))
def insert(num):
for i in range(num):
day = random.randint(11, 30)
value = random.randint(1, 1000000)
month = '0' + str(random.choice([3, 4]))
node1.query("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value))
def alter_move(num):
for i in range(num):
produce_alter_move(node1, name)
def alter_drop(num):
for i in range(num):
partition = random.choice([201903, 201904])
drach = random.choice(["drop", "detach"])
node1.query("ALTER TABLE {} {} PARTITION {}".format(name, drach, partition))
insert(100)
p = Pool(15)
tasks = []
for i in range(5):
tasks.append(p.apply_async(insert, (100,)))
tasks.append(p.apply_async(alter_move, (100,)))
tasks.append(p.apply_async(alter_drop, (100,)))
for task in tasks:
task.get(timeout=60)
assert node1.query("SELECT 1") == "1\n"
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
@pytest.mark.parametrize("name,engine", [
("mutating_mt","MergeTree()"),
("replicated_mutating_mt","ReplicatedMergeTree('/clickhouse/replicated_mutating_mt', '1')",),
])
def test_mutate_to_another_disk(start_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
s1 String
) ENGINE = {engine}
ORDER BY tuple()
SETTINGS storage_policy_name='moving_jbod_with_external'
""".format(name=name, engine=engine))
for i in range(5):
data = [] # 5MB in total
for i in range(5):
data.append(get_random_string(1024 * 1024)) # 1MB row
node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data])))
node1.query("ALTER TABLE {} UPDATE s1 = concat(s1, 'x') WHERE 1".format(name))
retry = 20
while node1.query("SELECT * FROM system.mutations WHERE is_done = 0") != "" and retry > 0:
retry -= 1
time.sleep(0.5)
if node1.query("SELECT latest_fail_reason FROM system.mutations WHERE table = '{}'".format(name)) == "":
assert node1.query("SELECT sum(endsWith(s1, 'x')) FROM {}".format(name)) == "25\n"
else: # mutation failed, let's try on another disk
print "Mutation failed"
node1.query("OPTIMIZE TABLE {} FINAL".format(name))
node1.query("ALTER TABLE {} UPDATE s1 = concat(s1, 'x') WHERE 1".format(name))
retry = 20
while node1.query("SELECT * FROM system.mutations WHERE is_done = 0") != "" and retry > 0:
retry -= 1
time.sleep(0.5)
assert node1.query("SELECT sum(endsWith(s1, 'x')) FROM {}".format(name)) == "25\n"
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
@pytest.mark.parametrize("name,engine", [
("alter_modifying_mt","MergeTree()"),
("replicated_alter_modifying_mt","ReplicatedMergeTree('/clickhouse/replicated_alter_modifying_mt', '1')",),
])
def test_concurrent_alter_modify(start_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
EventDate Date,
number UInt64
) ENGINE = {engine}
ORDER BY tuple()
PARTITION BY toYYYYMM(EventDate)
SETTINGS storage_policy_name='jbods_with_external'
""".format(name=name, engine=engine))
def insert(num):
for i in range(num):
day = random.randint(11, 30)
value = random.randint(1, 1000000)
month = '0' + str(random.choice([3, 4]))
node1.query("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value))
def alter_move(num):
for i in range(num):
produce_alter_move(node1, name)
def alter_modify(num):
for i in range(num):
column_type = random.choice(["UInt64", "String"])
node1.query("ALTER TABLE {} MODIFY COLUMN number {}".format(name, column_type))
insert(100)
assert node1.query("SELECT COUNT() FROM {}".format(name)) == "100\n"
p = Pool(50)
tasks = []
for i in range(5):
tasks.append(p.apply_async(alter_move, (100,)))
tasks.append(p.apply_async(alter_modify, (100,)))
for task in tasks:
task.get(timeout=60)
assert node1.query("SELECT 1") == "1\n"
assert node1.query("SELECT COUNT() FROM {}".format(name)) == "100\n"
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
def test_simple_replication_and_moves(start_cluster):
try:
for i, node in enumerate([node1, node2]):
node.query("""
CREATE TABLE replicated_table_for_moves (
s1 String
) ENGINE = ReplicatedMergeTree('/clickhouse/replicated_table_for_moves', '{}')
ORDER BY tuple()
SETTINGS storage_policy_name='moving_jbod_with_external', old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=2
""".format(i + 1))
def insert(num):
for i in range(num):
node = random.choice([node1, node2])
data = [] # 1MB in total
for i in range(2):
data.append(get_random_string(512 * 1024)) # 500KB value
node.query("INSERT INTO replicated_table_for_moves VALUES {}".format(','.join(["('" + x + "')" for x in data])))
def optimize(num):
for i in range(num):
node = random.choice([node1, node2])
node.query("OPTIMIZE TABLE replicated_table_for_moves FINAL")
p = Pool(60)
tasks = []
tasks.append(p.apply_async(insert, (20,)))
tasks.append(p.apply_async(optimize, (20,)))
for task in tasks:
task.get(timeout=60)
node1.query("SYSTEM SYNC REPLICA replicated_table_for_moves", timeout=5)
node2.query("SYSTEM SYNC REPLICA replicated_table_for_moves", timeout=5)
node1.query("SELECT COUNT() FROM replicated_table_for_moves") == "40\n"
node2.query("SELECT COUNT() FROM replicated_table_for_moves") == "40\n"
data = [] # 1MB in total
for i in range(2):
data.append(get_random_string(512 * 1024)) # 500KB value
time.sleep(3) # wait until old parts will be deleted
node1.query("SYSTEM STOP MERGES")
node2.query("SYSTEM STOP MERGES")
node1.query("INSERT INTO replicated_table_for_moves VALUES {}".format(','.join(["('" + x + "')" for x in data])))
node2.query("INSERT INTO replicated_table_for_moves VALUES {}".format(','.join(["('" + x + "')" for x in data])))
time.sleep(3) # nothing was moved
disks1 = get_used_disks_for_table(node1, "replicated_table_for_moves")
disks2 = get_used_disks_for_table(node2, "replicated_table_for_moves")
node1.query("SYSTEM START MERGES")
node2.query("SYSTEM START MERGES")
set(disks1) == set(["jbod1", "external"])
set(disks2) == set(["jbod1", "external"])
finally:
for node in [node1, node2]:
node.query("DROP TABLE IF EXISTS replicated_table_for_moves")
def test_download_appropriate_disk(start_cluster):
try:
for i, node in enumerate([node1, node2]):
node.query("""
CREATE TABLE replicated_table_for_download (
s1 String
) ENGINE = ReplicatedMergeTree('/clickhouse/replicated_table_for_download', '{}')
ORDER BY tuple()
SETTINGS storage_policy_name='moving_jbod_with_external', old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=2
""".format(i + 1))
data = []
for i in range(50):
data.append(get_random_string(1024 * 1024)) # 1MB value
node1.query("INSERT INTO replicated_table_for_download VALUES {}".format(','.join(["('" + x + "')" for x in data])))
for _ in range(10):
try:
print "Syncing replica"
node2.query("SYSTEM SYNC REPLICA replicated_table_for_download")
break
except:
time.sleep(0.5)
disks2 = get_used_disks_for_table(node2, "replicated_table_for_download")
assert set(disks2) == set(["external"])
finally:
for node in [node1, node2]:
node.query("DROP TABLE IF EXISTS replicated_table_for_download")
def test_rename(start_cluster):
try:
node1.query("""
CREATE TABLE default.renaming_table (
s String
) ENGINE = MergeTree
ORDER BY tuple()
SETTINGS storage_policy_name='small_jbod_with_external'
""")
for _ in range(5):
data = []
for i in range(10):
data.append(get_random_string(1024 * 1024)) # 1MB value
node1.query("INSERT INTO renaming_table VALUES {}".format(','.join(["('" + x + "')" for x in data])))
disks = get_used_disks_for_table(node1, "renaming_table")
assert len(disks) > 1
assert node1.query("SELECT COUNT() FROM default.renaming_table") == "50\n"
node1.query("RENAME TABLE default.renaming_table TO default.renaming_table1")
assert node1.query("SELECT COUNT() FROM default.renaming_table1") == "50\n"
with pytest.raises(QueryRuntimeException):
node1.query("SELECT COUNT() FROM default.renaming_table")
node1.query("CREATE DATABASE IF NOT EXISTS test")
node1.query("RENAME TABLE default.renaming_table1 TO test.renaming_table2")
assert node1.query("SELECT COUNT() FROM test.renaming_table2") == "50\n"
with pytest.raises(QueryRuntimeException):
node1.query("SELECT COUNT() FROM default.renaming_table1")
finally:
node1.query("DROP TABLE IF EXISTS default.renaming_table")
node1.query("DROP TABLE IF EXISTS default.renaming_table1")
node1.query("DROP TABLE IF EXISTS test.renaming_table2")
def test_freeze(start_cluster):
try:
node1.query("""
CREATE TABLE default.freezing_table (
d Date,
s String
) ENGINE = MergeTree
ORDER BY tuple()
PARTITION BY toYYYYMM(d)
SETTINGS storage_policy_name='small_jbod_with_external'
""")
for _ in range(5):
data = []
dates = []
for i in range(10):
data.append(get_random_string(1024 * 1024)) # 1MB value
dates.append("toDate('2019-03-05')")
node1.query("INSERT INTO freezing_table VALUES {}".format(','.join(["(" + d + ", '" + s + "')" for d, s in zip(dates, data)])))
disks = get_used_disks_for_table(node1, "freezing_table")
assert len(disks) > 1
assert node1.query("SELECT COUNT() FROM default.freezing_table") == "50\n"
node1.query("ALTER TABLE freezing_table FREEZE PARTITION 201903")
# check shadow files (backups) exists
node1.exec_in_container(["bash", "-c", "find /jbod1/shadow -name '*.mrk2' | grep '.*'"])
node1.exec_in_container(["bash", "-c", "find /external/shadow -name '*.mrk2' | grep '.*'"])
finally:
node1.query("DROP TABLE IF EXISTS default.freezing_table")

View File

@ -1,12 +1,12 @@
<yandex>
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/log.log</log>
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
<size>1000M</size>
<count>10</count>
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/log.log</log>
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
<size>1000M</size>
<count>10</count>
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
</logger>
</yandex>
</yandex>

View File

@ -2,7 +2,7 @@
1
1
1
t_00693 Memory 1 0000-00-00 00:00:00 [] [] Memory
t_00693 Memory 1 [] 0000-00-00 00:00:00 [] [] Memory
1
1
1

View File

@ -280,7 +280,9 @@ SELECT * FROM system.metrics LIMIT 10
- `primary_key_bytes_in_memory_allocated` (UInt64) - выделенный с резервом объем памяти (в байтах) для размещения первичных ключей;
- `database (String)` - имя базы данных;
- `table (String)` - имя таблицы;
- `engine (String)` - имя движка таблицы, без параметров.
- `engine (String)` - имя движка таблицы, без параметров;
- `path (String)` - путь к куску на диске;
- `disk (String)` - имя диска, на котором находится кусок;
- `is_frozen (UInt8)` Признак, показывающий существование бэкапа партиции. 1, бэкап есть. 0, бэкапа нет. Смотрите раздел [FREEZE PARTITION](../query_language/alter.md#alter_freeze-partition)
## system.part_log {#system_tables-part-log}
@ -291,7 +293,7 @@ SELECT * FROM system.metrics LIMIT 10
Столбцы:
- `event_type` (Enum) — тип события. Столбец может содержать одно из следующих значений: `NEW_PART` — вставка нового куска; `MERGE_PARTS` — слияние кусков; `DOWNLOAD_PART` — загрузка с реплики; `REMOVE_PART` — удаление или отсоединение из таблицы с помощью [DETACH PARTITION](../query_language/alter.md#alter_detach-partition); `MUTATE_PART` — изменение куска.
- `event_type` (Enum) — тип события. Столбец может содержать одно из следующих значений: `NEW_PART` — вставка нового куска; `MERGE_PARTS` — слияние кусков; `DOWNLOAD_PART` — загрузка с реплики; `REMOVE_PART` — удаление или отсоединение из таблицы с помощью [DETACH PARTITION](../query_language/alter.md#alter_detach-partition); `MUTATE_PART` — изменение куска; `MOVE_PART` — перемещение куска между дисками.
- `event_date` (Date) — дата события;
- `event_time` (DateTime) — время события;
- `duration_ms` (UInt64) — длительность;
@ -703,3 +705,25 @@ path: /clickhouse/tables/01-08/visits/replicas
**latest_fail_reason** — причина последней ошибки мутации.
[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/system_tables/) <!--hide-->
## system.disks {#system_tables-disks}
Таблица содержит информацию о дисках, заданных в [конфигурации сервера](#table_engine/mergetree.md#table_engine-mergetree-multiple-volumes_configure). Имеет следующие столбцы:
- `name String` — имя диска в конфигурации сервера.
- `path String` — путь к точке монтирования на файловой системе.
- `free_space UInt64` — свободное место на диске в данный момент времени в байтах.
- `total_space UInt64` — общее количество места на диске в данный момент времени в байтах.
- `keep_free_space UInt64` — количество байт, которое должно оставаться свободным (задается в конфигурации).
## system.storage_policies {#system_tables-storage_policies}
Таблица содержит информацию о политиках хранения и томах, заданных в [конфигурации сервера](#table_engine/mergetree.md#table_engine-mergetree-multiple-volumes_configure). Данные в таблице денормализованны, имя одной политики хранения может содержаться несколько раз, по количеству томов в ней. Имеет следующие столбцы:
- `policy_name String` — имя политики хранения в конфигурации сервера.
- `volume_name String` — имя тома, который содержится в данной политике хранения.
- `volume_priority UInt64` — порядковый номер тома, согласно конфигурации.
- `disks Array(String)` — имена дисков, содержащихся в данной политике хранения.
- `max_data_part_size UInt64` — максимальный размер куска, который может храниться на дисках этого тома (0 — без ограничений).
- `move_factor Float64` — доля свободного места, при превышении которой данные начинают перемещаться на следующий том.

View File

@ -394,3 +394,128 @@ ALTER TABLE example_table
Если вы выполните запрос `SELECT` между слияниями вы можете получить устаревшие данные. Чтобы избежать этого используйте запрос [OPTIMIZE](../../query_language/misc.md#misc_operations-optimize) перед `SELECT`.
[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/mergetree/) <!--hide-->
## Хранение данных таблицы на нескольких блочных устройствах {#table_engine-mergetree-multiple-volumes}
### Общее
Данные таблиц семейства MergeTree могут храниться на нескольких блочных устройствах. Это может оказаться полезным, например, при неявном разделении данных одной таблицы на "горячие" и "холодные", когда наиболее свежая часть занимает малый объем и запрашивается регулярно, а большой хвост исторических данных запрашивается редко. При наличии в системе нескольких дисков, "горячая" часть данных может быть размещена на быстрых дисках (NVMe SSDs или даже в памяти), а холодная на более медленных (HDD).
Минимальной перемещаемой единицей для MergeTree является кусок (part). Данные одного куска могут находится только на одном диске. Куски могут перемещаться между дисками в фоне, согласно пользовательским настройкам, а также с помощью [ALTER](../../query_language/alter.md#alter_move-partition) запросов.
### Термины
* Диск — примонтированное в файловой системе блочное устройство.
* Диск по умолчанию — диск, на котором находится путь, указанный в корне `config.xml` в теге `<path>`.
* Том (Volume) — упорядоченный набор равноценных дисков (схоже с [JBOD](https://ru.wikipedia.org/wiki/JBOD))
* Политика хранения (StoragePolicy) — множество томов с правилами перемещения данных между ними.
У всех описанных сущностей, при создании, указываются имена, которые будут отражены в системных таблицах [system.storage_policies](../system_tables.md#system_tables-storage_policies) и [system.disks](../system_tables.md#system_tables-disks). Имя политики хранения используется как настройка у таблиц семейства MergeTree.
### Конфигурация {#table_engine-mergetree-multiple-volumes_configure}
Диски, тома и политики хранения задаются в корне конфигурации (внутри тега `<yandex>`) в основном файле `config.xml` или в отдельном файле в директории `config.d`. Правила составления данной секции конфигурации имеет следующую структуру:
```xml
<disks>
<fast_disk> <!-- disk name -->
<path>/mnt/fast_ssd/clickhouse</path>
</fast_disk>
<disk1>
<path>/mnt/hdd1/clickhouse</path>
<keep_free_space_bytes>10485760</keep_free_space_bytes>_
</disk1>
<disk2>
<path>/mnt/hdd2/clickhouse</path>
<keep_free_space_bytes>10485760</keep_free_space_bytes>_
</disk2>
...
</disks>
```
Где,
* имя диска задается внутри имени тега.
* `path` — путь по которому будут храниться данные сервера (каталоги `data` и `shadow`), должен быть терминирован `/`.
* `keep_free_space_bytes` — размер зарезервированного свободного места на диске.
Порядок задания дисков не имеет значения.
Конфигурация политик хранения:
```xml
<policies>
<hdd_in_order> <!-- policy name -->
<volumes>
<single> <!-- volume name -->
<disk>disk1</disk>
<disk>disk2</disk>
</single>
</volumes>
</hdd_in_order>
<moving_from_ssd_to_hdd>
<volumes>
<hot>
<disk>fast_ssd</disk>
<max_data_part_size_bytes>1073741824</max_data_part_size_bytes>
</hot>
<cold>
<disk>disk1</disk>
</cold>
<move_factor>0.2</move_factor>
</volumes>
</moving_from_ssd_to_hdd>
</policies>
```
Где,
* имя политики и тома задаются внутри имен тегов.
* `disk` — диск, находящийся внутри тома.
* `max_data_part_size_bytes` — максимальный размер куска, который может находится на любом из дисков этого тома.
* `move_factor` — доля свободного места, при превышении которого данные начинают перемещаться на следующий том, если он есть (по умолчанию 0.1).
В приведенном примере, политика `hdd_in_order` реализует прицип [round-robin](https://ru.wikipedia.org/wiki/Round-robin_(%D0%B0%D0%BB%D0%B3%D0%BE%D1%80%D0%B8%D1%82%D0%BC)). Так как в политике есть всего 1 том (`single`) все записи производятся на его диски по круговому циклу. Такая политика может быть полезна при наличии в системе нескольких похожих дисков. Политика `moving_from_ssd_to_hdd` полезна при наличии в разных типов дисков. В томе `hot` находится один SSD-диск (`fast_ssd`), а также задается ограничение на максимальный размер куска, который может храниться на этом томе (1GB). Все куски такой таблицы больше 1GB будут записываться сразу на том `cold`, в котором содержится один HDD-диск `disk1`. Также, при заполнении диска `fast_ssd` более чем на 80% данные будут переносится на диск `disk1` фоновым процессом.
Порядок томов в политиках хранения важен, при достижении условий на переполнение тома данные переносятся на следующий. Порядок дисков в томах так же важен, данные пишутся по очереди на каждый из них.
После задания конфигурации дисков и политик хранения их можно использовать, как настройку при создании таблиц:
```sql
CREATE TABLE table_with_non_default_policy (
EventDate Date,
OrderID UInt64,
BannerID UInt64,
SearchPhrase String
) ENGINE = MergeTree()
ORDER BY (OrderID, BannerID)
PARTITION BY toYYYYMM(EventDate)
SETTINGS storage_policy_name='moving_from_ssd_to_hdd'
```
По умолчанию используется политика хранения `default` в которой есть один том и один диск, указанный в `<path>`. В данный момент менять политику хранения после создания таблицы нельзя.
### Особенности работы
В таблицах MergeTree данные попадают на диск несколькими способами:
* В результате вставки (запрос `INSERT`).
* В фоновых операциях слияний и [мутаций](../../query_language/alter.md#alter-mutations).
* При скачивании данных с другой реплики.
* В результате заморозки партиций [ALTER TABLE ... FREEZE PARTITION](../../query_language/alter.md#alter_freeze-partition).
Во всех случаях, кроме мутаций и заморозки партиций, при записи куска выбирается том и диск в соответствии с указанной конфигурацией хранилища:
1. Выбирается первый по порядку том, на котором есть свободное место для записи куска (`unreserved_space > current_part_size`) и который позволяет записывать куски требуемого размера `max_data_part_size_bytes > current_part_size`.
2. Внутри тома выбирается следующий диск после того, на который была предыдущая запись и на котором свободного места больше чем размер куска (`unreserved_space - keep_free_space_bytes > current_part_size`)
Мутации и запросы заморозки партиций в реализации используют [жесткие ссылки](https://ru.wikipedia.org/wiki/%D0%96%D1%91%D1%81%D1%82%D0%BA%D0%B0%D1%8F_%D1%81%D1%81%D1%8B%D0%BB%D0%BA%D0%B0). Жесткие ссылки между различными дисками не поддерживаются, поэтому в случае таких операций куски размещаются на тех же дисках, что и исходные.
В фоне куски перемещаются между томами на основе информации о занятом месте (настройка `move_factor`) по порядку, в котором указаны тома в конфигурации. Данные никогда не перемещаются с последнего тома и на первый том. Следить за фоновыми перемещениями можно с помощью системных таблиц [system.part_log](../system_tables.md#system_tables-part-log) (поле `type = MOVE_PART`) и [system.parts](../system_tables.md#system_tables-parts) (поля `path` и `disk`). Также подробная информация о перемещениях доступна в логах сервера.
С помощью запроса [ALTER TABLE ... MOVE PART|PARTITION ... TO VOLUME|DISK ...](../../query_language/alter.md#alter_move-partition) пользователь может принудительно перенести кусок или партицию с одного раздела на другой. При этом учитываются все ограничения, указанные для фоновых операций. Запрос самостоятельно инициирует процесс перемещения не дожидаясь фоновых операций. В случае недостатка места или неудовлетворения ограничениям пользователь получит сообщение об ошибке.
Перемещения данных не взаимодействуют с репликацией данных, поэтому на разных репликах одной и той же таблицы могут быть указаны разные политики хранения.
После выполнения фоновых слияний или мутаций старые куски не удаляются сразу, а через некоторое время (табличная настройка `old_parts_lifetime`). Также они не перемещаются на другие тома или диски, поэтому до момента удаления они продолжают учитываться при подсчёте занятого дискового пространства.

View File

@ -192,7 +192,8 @@ ALTER TABLE [db].name DROP CONSTRAINT constraint_name;
- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) удалить все значения в столбце для заданной партиции;
- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - очистить построенные вторичные индексы для заданной партиции;
- [FREEZE PARTITION](#alter_freeze-partition) создать резервную копию партиции;
- [FETCH PARTITION](#alter_fetch-partition) скачать партицию с другого сервера.
- [FETCH PARTITION](#alter_fetch-partition) скачать партицию с другого сервера;
- [MOVE PARTITION|PART](#alter_move-partition) -- переместить партицию/кускок на другой диск или том.
#### DETACH PARTITION {#alter_detach-partition}
@ -309,6 +310,9 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr]
- `/var/lib/clickhouse/` — рабочая директория ClickHouse, заданная в конфигурационном файле;
- `N` — инкрементальный номер резервной копии.
!!! note "Примечание"
При использовании [нескольких дисков для хранения данных таблицы](../operations/table_engines/mergetree.md#table_engine-mergetree-multiple-volumes) директория `shadow/N` появляется на каждом из дисков, на которых были куски, попавшие под выражение `PARTITION`.
Структура директорий внутри резервной копии такая же, как внутри `/var/lib/clickhouse/`. Запрос выполнит 'chmod' для всех файлов, запрещая запись в них.
Обратите внимание, запрос `ALTER TABLE t FREEZE PARTITION` не реплицируется. Он создает резервную копию только на локальном сервере. После создания резервной копии данные из `/var/lib/clickhouse/shadow/` можно скопировать на удалённый сервер, а локальную копию удалить.
@ -355,6 +359,27 @@ ALTER TABLE users ATTACH PARTITION 201902;
Несмотря на то что запрос называется `ALTER TABLE`, он не изменяет структуру таблицы и не изменяет сразу доступные данные в таблице.
#### MOVE PARTITION|PART {#alter_move-partition}
```sql
ALTER TABLE table_name MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name'
```
Перемещает партицию или кусок на другой том или диск. Запрос работает только для движков семейства MergeTree. Подробнее о хранении данных на разных дисках читайте в разделе [Хранение данных таблицы на нескольких блочных устройствах](../operations/table_engines/mergetree.md#table_engine-mergetree-multiple-volumes).
Следует иметь ввиду:
- Запрос `ALTER TABLE t MOVE` не реплицируется, т.к. на разных репликах могут быть различные конфигурации политик хранения.
- Запрос `ALTER TABLE t MOVE` будет возвращать ошибку, если указан несуществующий том или диск, а также в случае невыполнения условий перемещения данных, которые указаны в конфигурации политики хранения.
- Запрос `ALTER TABLE t MOVE` может возвращать ошибку в случае, когда перемещаемые данные уже оказались перемещены в результате фонового процесса, конкурентного запроса `ALTER TABLE t MOVE` или как часть результата фоновой операции слияния. В данном случае никаких дополнительных действий от пользователя не требуется.
Примеры:
```sql
ALTER TABLE hits MOVE PART '20190301_14343_16206_438' TO VOLUME 'slow'
ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd'
```
#### Как задавать имя партиции в запросах ALTER {#alter-how-to-specify-part-expr}
Чтобы задать нужную партицию в запросах `ALTER ... PARTITION`, можно использовать: