ClickHouse/dbms/Disks/DiskSpaceMonitor.h

195 lines
5.8 KiB
C++
Raw Normal View History

2019-11-27 09:39:44 +00:00
#pragma once
#include <Disks/IDisk.h>
#include <IO/WriteHelpers.h>
2019-11-27 09:39:44 +00:00
#include <Common/CurrentMetrics.h>
#include <Common/Exception.h>
#include <Common/formatReadable.h>
#include <common/logger_useful.h>
2019-11-27 09:39:44 +00:00
#include <memory>
#include <mutex>
#include <unistd.h>
#include <boost/noncopyable.hpp>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
class DiskSelector;
using DiskSelectorPtr = std::shared_ptr<const DiskSelector>;
2019-11-27 09:39:44 +00:00
/// Parse .xml configuration and store information about disks
/// Mostly used for introspection.
class DiskSelector
{
public:
DiskSelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Context & context);
DiskSelector(const DiskSelector & from): disks(from.disks) {}
DiskSelectorPtr updateFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Context & context) const;
2019-11-27 09:39:44 +00:00
/// Get disk by name
DiskPtr get(const String & name) const;
2019-11-27 09:39:44 +00:00
/// Get all disks with names
2019-11-27 09:39:44 +00:00
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 String & config_prefix,
DiskSelectorPtr disk_selector);
2019-11-27 09:39:44 +00:00
/// Next disk (round-robin)
///
/// - Used with policy for temporary data
/// - Ignores all limitations
/// - Shares last access with reserve()
DiskPtr getNextDisk();
2019-11-27 09:39:44 +00:00
/// Uses Round-robin to choose disk for reservation.
/// Returns valid reservation or nullptr if there is no space left on any disk.
2019-12-03 13:37:40 +00:00
ReservationPtr reserve(UInt64 bytes) override;
2019-11-27 09:39:44 +00:00
/// 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;
};
2019-12-03 13:37:40 +00:00
using VolumePtr = std::shared_ptr<Volume>;
2019-11-27 09:39:44 +00:00
using Volumes = std::vector<VolumePtr>;
class StoragePolicy;
using StoragePolicyPtr = std::shared_ptr<const StoragePolicy>;
2019-11-27 09:39:44 +00:00
/**
* Contains all information about volumes configuration for Storage.
* Can determine appropriate Volume and Disk for each reservation.
*/
2019-12-03 13:37:40 +00:00
class StoragePolicy
2019-11-27 09:39:44 +00:00
{
public:
StoragePolicy(String name_, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks);
2019-11-27 09:39:44 +00:00
StoragePolicy(String name_, Volumes volumes_, double move_factor_);
bool isDefaultPolicy() const;
2019-11-27 09:39:44 +00:00
/// 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;
2019-12-03 13:37:40 +00:00
const String & getName() const { return name; }
2019-11-27 09:39:44 +00:00
/// Returns valid reservation or null
2019-12-03 13:37:40 +00:00
ReservationPtr reserve(UInt64 bytes) const;
2019-11-27 09:39:44 +00:00
/// 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);
}
/// Checks if storage policy can be replaced by another one.
void checkCompatibleWith(const StoragePolicyPtr & new_storage_policy) const;
2019-11-27 09:39:44 +00:00
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%
};
class StoragePolicySelector;
using StoragePolicySelectorPtr = std::shared_ptr<const StoragePolicySelector>;
2019-11-27 09:39:44 +00:00
/// 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, DiskSelectorPtr disks);
StoragePolicySelectorPtr updateFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks) const;
2019-11-27 09:39:44 +00:00
/// Policy by name
StoragePolicyPtr get(const String & name) const;
2019-11-27 09:39:44 +00:00
/// All policies
const std::map<String, StoragePolicyPtr> & getPoliciesMap() const { return policies; }
private:
std::map<String, StoragePolicyPtr> policies;
};
}