ClickHouse/dbms/src/Disks/DiskLocal.cpp

317 lines
9.1 KiB
C++
Raw Normal View History

2019-11-27 09:39:44 +00:00
#include "DiskLocal.h"
#include "DiskFactory.h"
#include <Interpreters/Context.h>
2019-11-27 09:39:44 +00:00
#include <Common/filesystemHelpers.h>
#include <Common/quoteString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
2020-01-10 21:42:26 +00:00
extern const int PATH_ACCESS_DENIED;
}
2020-01-18 23:42:46 +00:00
std::mutex DiskLocal::reservation_mutex;
using DiskLocalPtr = std::shared_ptr<DiskLocal>;
class DiskLocalReservation : public IReservation
{
public:
DiskLocalReservation(const DiskLocalPtr & disk_, UInt64 size_)
: disk(disk_), size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_)
{
}
UInt64 getSize() const override { return size; }
DiskPtr getDisk() const override { return disk; }
void update(UInt64 new_size) override;
~DiskLocalReservation() override;
private:
DiskLocalPtr disk;
UInt64 size;
CurrentMetrics::Increment metric_increment;
};
class DiskLocalDirectoryIterator : public IDiskDirectoryIterator
{
public:
explicit DiskLocalDirectoryIterator(const String & disk_path_, const String & dir_path_) :
dir_path(dir_path_), iter(disk_path_ + dir_path_) {}
void next() override { ++iter; }
bool isValid() const override { return iter != Poco::DirectoryIterator(); }
String path() const override
{
if (iter->isDirectory())
return dir_path + iter.name() + '/';
else
return dir_path + iter.name();
}
private:
String dir_path;
Poco::DirectoryIterator iter;
};
2019-12-03 13:37:40 +00:00
ReservationPtr DiskLocal::reserve(UInt64 bytes)
2019-11-27 09:39:44 +00:00
{
if (!tryReserve(bytes))
return {};
2019-12-03 13:37:40 +00:00
return std::make_unique<DiskLocalReservation>(std::static_pointer_cast<DiskLocal>(shared_from_this()), bytes);
2019-11-27 09:39:44 +00:00
}
2019-12-03 13:37:40 +00:00
bool DiskLocal::tryReserve(UInt64 bytes)
2019-11-27 09:39:44 +00:00
{
2020-01-18 23:42:46 +00:00
std::lock_guard lock(DiskLocal::reservation_mutex);
2019-11-27 09:39:44 +00:00
if (bytes == 0)
{
LOG_DEBUG(&Logger::get("DiskLocal"), "Reserving 0 bytes on disk " << backQuote(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("DiskLocal"),
"Reserving " << formatReadableSizeWithBinarySuffix(bytes) << " on disk " << backQuote(name) << ", having unreserved "
<< formatReadableSizeWithBinarySuffix(unreserved_space) << ".");
++reservation_count;
reserved_bytes += bytes;
return true;
}
return false;
}
UInt64 DiskLocal::getTotalSpace() const
{
2019-12-03 13:37:40 +00:00
auto fs = getStatVFS(disk_path);
2019-11-27 09:39:44 +00:00
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 DiskLocal::getAvailableSpace() const
{
/// we use f_bavail, because part of b_free space is
/// available for superuser only and for system purposes
2019-12-03 13:37:40 +00:00
auto fs = getStatVFS(disk_path);
2019-11-27 09:39:44 +00:00
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;
}
UInt64 DiskLocal::getUnreservedSpace() const
{
2020-01-18 23:42:46 +00:00
std::lock_guard lock(DiskLocal::reservation_mutex);
2019-11-27 09:39:44 +00:00
auto available_space = getAvailableSpace();
available_space -= std::min(available_space, reserved_bytes);
return available_space;
}
2019-12-03 13:37:40 +00:00
bool DiskLocal::exists(const String & path) const
2019-11-27 09:39:44 +00:00
{
2019-12-03 13:37:40 +00:00
return Poco::File(disk_path + path).exists();
2019-11-27 09:39:44 +00:00
}
2019-12-03 13:37:40 +00:00
bool DiskLocal::isFile(const String & path) const
{
2019-12-03 13:37:40 +00:00
return Poco::File(disk_path + path).isFile();
}
2019-12-03 13:37:40 +00:00
bool DiskLocal::isDirectory(const String & path) const
{
2019-12-03 13:37:40 +00:00
return Poco::File(disk_path + path).isDirectory();
}
size_t DiskLocal::getFileSize(const String & path) const
{
return Poco::File(disk_path + path).getSize();
}
2019-12-03 13:37:40 +00:00
void DiskLocal::createDirectory(const String & path)
{
2019-12-03 13:37:40 +00:00
Poco::File(disk_path + path).createDirectory();
}
2019-12-03 13:37:40 +00:00
void DiskLocal::createDirectories(const String & path)
{
2019-12-03 13:37:40 +00:00
Poco::File(disk_path + path).createDirectories();
}
void DiskLocal::clearDirectory(const String & path)
{
std::vector<Poco::File> files;
Poco::File(disk_path + path).list(files);
for (auto & file : files)
2020-01-02 14:37:31 +00:00
file.remove();
}
void DiskLocal::moveDirectory(const String & from_path, const String & to_path)
{
Poco::File(disk_path + from_path).renameTo(disk_path + to_path);
}
2019-12-03 13:37:40 +00:00
DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path)
{
2020-01-10 21:42:26 +00:00
return std::make_unique<DiskLocalDirectoryIterator>(disk_path, path);
}
2019-12-03 13:37:40 +00:00
void DiskLocal::moveFile(const String & from_path, const String & to_path)
{
2019-12-03 13:37:40 +00:00
Poco::File(disk_path + from_path).renameTo(disk_path + to_path);
}
2020-01-02 14:37:31 +00:00
void DiskLocal::replaceFile(const String & from_path, const String & to_path)
{
Poco::File from_file(disk_path + from_path);
Poco::File to_file(disk_path + to_path);
if (to_file.exists())
{
Poco::File tmp_file(disk_path + to_path + ".old");
to_file.renameTo(tmp_file.path());
from_file.renameTo(disk_path + to_path);
tmp_file.remove();
}
else
from_file.renameTo(to_file.path());
}
2019-12-03 13:37:40 +00:00
void DiskLocal::copyFile(const String & from_path, const String & to_path)
{
2019-12-03 13:37:40 +00:00
Poco::File(disk_path + from_path).copyTo(disk_path + to_path);
}
std::unique_ptr<SeekableReadBuffer> DiskLocal::readFile(const String & path, size_t buf_size) const
{
return std::make_unique<ReadBufferFromFile>(disk_path + path, buf_size);
}
2019-12-26 14:28:22 +00:00
std::unique_ptr<WriteBuffer> DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode)
{
2019-12-26 14:28:22 +00:00
int flags = (mode == WriteMode::Append) ? (O_APPEND | O_CREAT | O_WRONLY) : -1;
return std::make_unique<WriteBufferFromFile>(disk_path + path, buf_size, flags);
2019-11-27 09:39:44 +00:00
}
2020-01-18 23:18:23 +00:00
void DiskLocal::remove(const String & path)
{
2020-01-18 23:18:23 +00:00
Poco::File(disk_path + path).remove(false);
}
void DiskLocal::removeRecursive(const String & path)
{
Poco::File(disk_path + path).remove(true);
}
2020-01-18 23:42:46 +00:00
2019-11-27 09:39:44 +00:00
void DiskLocalReservation::update(UInt64 new_size)
{
2020-01-18 23:42:46 +00:00
std::lock_guard lock(DiskLocal::reservation_mutex);
2019-12-03 13:37:40 +00:00
disk->reserved_bytes -= size;
2019-11-27 09:39:44 +00:00
size = new_size;
2019-12-03 13:37:40 +00:00
disk->reserved_bytes += size;
2019-11-27 09:39:44 +00:00
}
2020-01-18 23:42:46 +00:00
2019-11-27 09:39:44 +00:00
DiskLocalReservation::~DiskLocalReservation()
{
try
{
2020-01-18 23:42:46 +00:00
std::lock_guard lock(DiskLocal::reservation_mutex);
2019-12-03 13:37:40 +00:00
if (disk->reserved_bytes < size)
2019-11-27 09:39:44 +00:00
{
2019-12-03 13:37:40 +00:00
disk->reserved_bytes = 0;
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '" + disk->getName() + "'.");
2019-11-27 09:39:44 +00:00
}
else
{
2019-12-03 13:37:40 +00:00
disk->reserved_bytes -= size;
2019-11-27 09:39:44 +00:00
}
2019-12-03 13:37:40 +00:00
if (disk->reservation_count == 0)
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '" + disk->getName() + "'.");
2019-11-27 09:39:44 +00:00
else
2019-12-03 13:37:40 +00:00
--disk->reservation_count;
2019-11-27 09:39:44 +00:00
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void registerDiskLocal(DiskFactory & factory)
{
auto creator = [](const String & name,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
const Context & context) -> DiskPtr {
String path = config.getString(config_prefix + ".path", "");
if (name == "default")
{
if (!path.empty())
throw Exception(
"\"default\" disk path should be provided in <path> not it <storage_configuration>",
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
path = context.getPath();
}
else
{
if (path.empty())
throw Exception("Disk path can not be empty. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
if (path.back() != '/')
throw Exception("Disk path must end with /. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
}
2020-01-10 21:42:26 +00:00
if (Poco::File disk{path}; !disk.canRead() || !disk.canWrite())
{
throw Exception("There is no RW access to disk " + name + " (" + path + ")", ErrorCodes::PATH_ACCESS_DENIED);
}
2019-11-27 09:39:44 +00:00
bool has_space_ratio = config.has(config_prefix + ".keep_free_space_ratio");
if (config.has(config_prefix + ".keep_free_space_bytes") && has_space_ratio)
throw Exception(
"Only one of 'keep_free_space_bytes' and 'keep_free_space_ratio' can be specified",
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
UInt64 keep_free_space_bytes = config.getUInt64(config_prefix + ".keep_free_space_bytes", 0);
if (has_space_ratio)
{
auto ratio = config.getDouble(config_prefix + ".keep_free_space_ratio");
if (ratio < 0 || ratio > 1)
throw Exception("'keep_free_space_ratio' have to be between 0 and 1", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
String tmp_path = path;
if (tmp_path.empty())
tmp_path = context.getPath();
// Create tmp disk for getting total disk space.
keep_free_space_bytes = static_cast<UInt64>(DiskLocal("tmp", tmp_path, 0).getTotalSpace() * ratio);
}
2019-12-03 13:37:40 +00:00
return std::make_shared<DiskLocal>(name, path, keep_free_space_bytes);
2019-11-27 09:39:44 +00:00
};
factory.registerDiskType("local", creator);
2019-11-27 09:39:44 +00:00
}
}