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.
|
/// 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;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
Loading…
Reference in New Issue
Block a user