mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-18 04:12:19 +00:00
Little better #2281
This commit is contained in:
parent
0947c7808d
commit
7dc1276afe
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
Loading…
Reference in New Issue
Block a user