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.
void update(size_t new_size)
void update(UInt64 new_size)
{
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
DiskSpaceMonitor::reserved_bytes -= size;
@ -76,12 +76,12 @@ public:
DiskSpaceMonitor::reserved_bytes += size;
}
size_t getSize() const
UInt64 getSize() const
{
return size;
}
Reservation(size_t size_)
Reservation(UInt64 size_)
: size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size)
{
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
@ -90,23 +90,23 @@ public:
}
private:
size_t size;
UInt64 size;
CurrentMetrics::Increment metric_increment;
};
using ReservationPtr = std::unique_ptr<Reservation>;
static size_t getUnreservedFreeSpace(const std::string & path)
static UInt64 getUnreservedFreeSpace(const std::string & path)
{
struct statvfs fs;
if (statvfs(path.c_str(), &fs) != 0)
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.
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);
@ -118,22 +118,22 @@ public:
return res;
}
static size_t getReservedSpace()
static UInt64 getReservedSpace()
{
std::lock_guard<std::mutex> lock(mutex);
return reserved_bytes;
}
static size_t getReservationCount()
static UInt64 getReservationCount()
{
std::lock_guard<std::mutex> lock(mutex);
return reservation_count;
}
/// 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)
throw Exception("Not enough free disk space to reserve: " + formatReadableSizeWithBinarySuffix(free_bytes) + " available, "
+ formatReadableSizeWithBinarySuffix(size) + " requested", ErrorCodes::NOT_ENOUGH_SPACE);
@ -141,8 +141,8 @@ public:
}
private:
static size_t reserved_bytes;
static size_t reservation_count;
static UInt64 reserved_bytes;
static UInt64 reservation_count;
static std::mutex mutex;
};

View File

@ -301,8 +301,6 @@ bool StorageMergeTree::merge(
auto structure_lock = lockStructure(true, __PRETTY_FUNCTION__);
size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
MergeTreeDataMergerMutator::FuturePart future_part;
/// You must call destructor with unlocked `currently_merging_mutex`.
@ -326,6 +324,7 @@ bool StorageMergeTree::merge(
}
else
{
UInt64 disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
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.
std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
MergeTreeDataMergerMutator::FuturePart future_merged_part;
String disable_reason;
bool selected = false;
@ -2923,6 +2921,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
}
else
{
UInt64 disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
String partition_id = data.getPartitionIDFromQuery(partition, context);
selected = merger_mutator.selectAllPartsToMergeWithinPartition(
future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason);