Little better #2281

This commit is contained in:
Alexey Milovidov 2018-07-08 08:26:51 +03:00
parent 0947c7808d
commit 7dc1276afe
3 changed files with 15 additions and 17 deletions

View File

@ -68,7 +68,7 @@ public:
} }
/// Change amount of reserved space. When new_size is greater than before, availability of free space is not checked. /// Change amount of reserved space. When new_size is greater than before, availability of free space is not checked.
void update(size_t new_size) void update(UInt64 new_size)
{ {
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex); std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
DiskSpaceMonitor::reserved_bytes -= size; DiskSpaceMonitor::reserved_bytes -= size;
@ -76,12 +76,12 @@ public:
DiskSpaceMonitor::reserved_bytes += size; DiskSpaceMonitor::reserved_bytes += size;
} }
size_t getSize() const UInt64 getSize() const
{ {
return size; return size;
} }
Reservation(size_t size_) Reservation(UInt64 size_)
: size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size) : size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size)
{ {
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex); std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
@ -90,23 +90,23 @@ public:
} }
private: private:
size_t size; UInt64 size;
CurrentMetrics::Increment metric_increment; CurrentMetrics::Increment metric_increment;
}; };
using ReservationPtr = std::unique_ptr<Reservation>; using ReservationPtr = std::unique_ptr<Reservation>;
static size_t getUnreservedFreeSpace(const std::string & path) static UInt64 getUnreservedFreeSpace(const std::string & path)
{ {
struct statvfs fs; struct statvfs fs;
if (statvfs(path.c_str(), &fs) != 0) if (statvfs(path.c_str(), &fs) != 0)
throwFromErrno("Could not calculate available disk space (statvfs)", ErrorCodes::CANNOT_STATVFS); throwFromErrno("Could not calculate available disk space (statvfs)", ErrorCodes::CANNOT_STATVFS);
size_t res = fs.f_bfree * fs.f_bsize; 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. /// Heuristic by Michael Kolupaev: reserve 30 MB more, because statvfs shows few megabytes more space than df.
res -= std::min(res, static_cast<size_t>(30 * (1ul << 20))); res -= std::min(res, static_cast<UInt64>(30 * (1ul << 20)));
std::lock_guard<std::mutex> lock(mutex); std::lock_guard<std::mutex> lock(mutex);
@ -118,22 +118,22 @@ public:
return res; return res;
} }
static size_t getReservedSpace() static UInt64 getReservedSpace()
{ {
std::lock_guard<std::mutex> lock(mutex); std::lock_guard<std::mutex> lock(mutex);
return reserved_bytes; return reserved_bytes;
} }
static size_t getReservationCount() static UInt64 getReservationCount()
{ {
std::lock_guard<std::mutex> lock(mutex); std::lock_guard<std::mutex> lock(mutex);
return reservation_count; return reservation_count;
} }
/// If not enough (approximately) space, throw an exception. /// If not enough (approximately) space, throw an exception.
static ReservationPtr reserve(const std::string & path, size_t size) static ReservationPtr reserve(const std::string & path, UInt64 size)
{ {
size_t free_bytes = getUnreservedFreeSpace(path); UInt64 free_bytes = getUnreservedFreeSpace(path);
if (free_bytes < size) if (free_bytes < size)
throw Exception("Not enough free disk space to reserve: " + formatReadableSizeWithBinarySuffix(free_bytes) + " available, " throw Exception("Not enough free disk space to reserve: " + formatReadableSizeWithBinarySuffix(free_bytes) + " available, "
+ formatReadableSizeWithBinarySuffix(size) + " requested", ErrorCodes::NOT_ENOUGH_SPACE); + formatReadableSizeWithBinarySuffix(size) + " requested", ErrorCodes::NOT_ENOUGH_SPACE);
@ -141,8 +141,8 @@ public:
} }
private: private:
static size_t reserved_bytes; static UInt64 reserved_bytes;
static size_t reservation_count; static UInt64 reservation_count;
static std::mutex mutex; static std::mutex mutex;
}; };

View File

@ -301,8 +301,6 @@ bool StorageMergeTree::merge(
auto structure_lock = lockStructure(true, __PRETTY_FUNCTION__); auto structure_lock = lockStructure(true, __PRETTY_FUNCTION__);
size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
MergeTreeDataMergerMutator::FuturePart future_part; MergeTreeDataMergerMutator::FuturePart future_part;
/// You must call destructor with unlocked `currently_merging_mutex`. /// You must call destructor with unlocked `currently_merging_mutex`.
@ -326,6 +324,7 @@ bool StorageMergeTree::merge(
} }
else else
{ {
UInt64 disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
selected = merger.selectAllPartsToMergeWithinPartition(future_part, disk_space, can_merge, partition_id, final, out_disable_reason); selected = merger.selectAllPartsToMergeWithinPartition(future_part, disk_space, can_merge, partition_id, final, out_disable_reason);
} }

View File

@ -2904,8 +2904,6 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
/// (merge_selecting_thread or OPTIMIZE queries) could assign new merges. /// (merge_selecting_thread or OPTIMIZE queries) could assign new merges.
std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex); std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
MergeTreeDataMergerMutator::FuturePart future_merged_part; MergeTreeDataMergerMutator::FuturePart future_merged_part;
String disable_reason; String disable_reason;
bool selected = false; bool selected = false;
@ -2923,6 +2921,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
} }
else else
{ {
UInt64 disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
String partition_id = data.getPartitionIDFromQuery(partition, context); String partition_id = data.getPartitionIDFromQuery(partition, context);
selected = merger_mutator.selectAllPartsToMergeWithinPartition( selected = merger_mutator.selectAllPartsToMergeWithinPartition(
future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason); future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason);