Merge pull request #49342 from ClickHouse/disks-space-improvement

Slight improvement in Disks interface
This commit is contained in:
Alexey Milovidov 2023-07-08 02:24:57 +03:00 committed by GitHub
commit 03d2606eb9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 142 additions and 76 deletions

View File

@ -266,7 +266,7 @@ public:
}
UInt64 getSize() const override { return reservation->getSize(); }
UInt64 getUnreservedSpace() const override { return reservation->getUnreservedSpace(); }
std::optional<UInt64> getUnreservedSpace() const override { return reservation->getUnreservedSpace(); }
DiskPtr getDisk(size_t i) const override
{

View File

@ -312,17 +312,17 @@ public:
}
}
UInt64 getTotalSpace() const override
std::optional<UInt64> getTotalSpace() const override
{
return delegate->getTotalSpace();
}
UInt64 getAvailableSpace() const override
std::optional<UInt64> getAvailableSpace() const override
{
return delegate->getAvailableSpace();
}
UInt64 getUnreservedSpace() const override
std::optional<UInt64> getUnreservedSpace() const override
{
return delegate->getUnreservedSpace();
}

View File

@ -78,7 +78,7 @@ public:
{}
UInt64 getSize() const override { return size; }
UInt64 getUnreservedSpace() const override { return unreserved_space; }
std::optional<UInt64> getUnreservedSpace() const override { return unreserved_space; }
DiskPtr getDisk(size_t i) const override
{
@ -175,8 +175,11 @@ std::optional<UInt64> DiskLocal::tryReserve(UInt64 bytes)
{
std::lock_guard lock(DiskLocal::reservation_mutex);
UInt64 available_space = getAvailableSpace();
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
auto available_space = getAvailableSpace();
UInt64 unreserved_space = available_space
? *available_space - std::min(*available_space, reserved_bytes)
: std::numeric_limits<UInt64>::max();
if (bytes == 0)
{
@ -187,12 +190,24 @@ std::optional<UInt64> DiskLocal::tryReserve(UInt64 bytes)
if (unreserved_space >= bytes)
{
LOG_TRACE(
logger,
"Reserved {} on local disk {}, having unreserved {}.",
ReadableSize(bytes),
backQuote(name),
ReadableSize(unreserved_space));
if (available_space)
{
LOG_TRACE(
logger,
"Reserved {} on local disk {}, having unreserved {}.",
ReadableSize(bytes),
backQuote(name),
ReadableSize(unreserved_space));
}
else
{
LOG_TRACE(
logger,
"Reserved {} on local disk {}.",
ReadableSize(bytes),
backQuote(name));
}
++reservation_count;
reserved_bytes += bytes;
return {unreserved_space - bytes};
@ -218,14 +233,14 @@ static UInt64 getTotalSpaceByName(const String & name, const String & disk_path,
return total_size - keep_free_space_bytes;
}
UInt64 DiskLocal::getTotalSpace() const
std::optional<UInt64> DiskLocal::getTotalSpace() const
{
if (broken || readonly)
return 0;
return getTotalSpaceByName(name, disk_path, keep_free_space_bytes);
}
UInt64 DiskLocal::getAvailableSpace() const
std::optional<UInt64> DiskLocal::getAvailableSpace() const
{
if (broken || readonly)
return 0;
@ -242,10 +257,10 @@ UInt64 DiskLocal::getAvailableSpace() const
return total_size - keep_free_space_bytes;
}
UInt64 DiskLocal::getUnreservedSpace() const
std::optional<UInt64> DiskLocal::getUnreservedSpace() const
{
std::lock_guard lock(DiskLocal::reservation_mutex);
auto available_space = getAvailableSpace();
auto available_space = *getAvailableSpace();
available_space -= std::min(available_space, reserved_bytes);
return available_space;
}

View File

@ -35,11 +35,9 @@ public:
ReservationPtr reserve(UInt64 bytes) override;
UInt64 getTotalSpace() const override;
UInt64 getAvailableSpace() const override;
UInt64 getUnreservedSpace() const override;
std::optional<UInt64> getTotalSpace() const override;
std::optional<UInt64> getAvailableSpace() const override;
std::optional<UInt64> getUnreservedSpace() const override;
UInt64 getKeepingFreeSpace() const override { return keep_free_space_bytes; }

View File

@ -140,13 +140,13 @@ public:
const String & getName() const override { return name; }
/// Total available space on the disk.
virtual UInt64 getTotalSpace() const = 0;
virtual std::optional<UInt64> getTotalSpace() const = 0;
/// Space currently available on the disk.
virtual UInt64 getAvailableSpace() const = 0;
virtual std::optional<UInt64> getAvailableSpace() const = 0;
/// Space available for reservation (available space minus reserved space).
virtual UInt64 getUnreservedSpace() const = 0;
virtual std::optional<UInt64> getUnreservedSpace() const = 0;
/// Amount of bytes which should be kept free on the disk.
virtual UInt64 getKeepingFreeSpace() const { return 0; }
@ -495,7 +495,7 @@ public:
/// Space available for reservation
/// (with this reservation already take into account).
virtual UInt64 getUnreservedSpace() const = 0;
virtual std::optional<UInt64> getUnreservedSpace() const = 0;
/// Get i-th disk where reservation take place.
virtual DiskPtr getDisk(size_t i = 0) const = 0; /// NOLINT

View File

@ -49,11 +49,18 @@ IVolume::IVolume(
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Volume must contain at least one disk");
}
UInt64 IVolume::getMaxUnreservedFreeSpace() const
std::optional<UInt64> IVolume::getMaxUnreservedFreeSpace() const
{
UInt64 res = 0;
std::optional<UInt64> res;
for (const auto & disk : disks)
res = std::max(res, disk->getUnreservedSpace());
{
auto disk_unreserved_space = disk->getUnreservedSpace();
if (!disk_unreserved_space)
return std::nullopt; /// There is at least one unlimited disk.
if (!res || *disk_unreserved_space > *res)
res = disk_unreserved_space;
}
return res;
}

View File

@ -74,7 +74,7 @@ public:
virtual VolumeType getType() const = 0;
/// Return biggest unreserved space across all disks
UInt64 getMaxUnreservedFreeSpace() const;
std::optional<UInt64> getMaxUnreservedFreeSpace() const;
DiskPtr getDisk() const { return getDisk(0); }
virtual DiskPtr getDisk(size_t i) const { return disks[i]; }

View File

@ -410,18 +410,25 @@ void DiskObjectStorage::removeSharedRecursive(
transaction->commit();
}
std::optional<UInt64> DiskObjectStorage::tryReserve(UInt64 bytes)
bool DiskObjectStorage::tryReserve(UInt64 bytes)
{
std::lock_guard lock(reservation_mutex);
auto available_space = getAvailableSpace();
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
if (!available_space)
{
++reservation_count;
reserved_bytes += bytes;
return true;
}
UInt64 unreserved_space = *available_space - std::min(*available_space, reserved_bytes);
if (bytes == 0)
{
LOG_TRACE(log, "Reserved 0 bytes on remote disk {}", backQuote(name));
++reservation_count;
return {unreserved_space};
return true;
}
if (unreserved_space >= bytes)
@ -434,14 +441,14 @@ std::optional<UInt64> DiskObjectStorage::tryReserve(UInt64 bytes)
ReadableSize(unreserved_space));
++reservation_count;
reserved_bytes += bytes;
return {unreserved_space - bytes};
return true;
}
else
{
LOG_TRACE(log, "Could not reserve {} on remote disk {}. Not enough unreserved space", ReadableSize(bytes), backQuote(name));
}
return {};
return false;
}
bool DiskObjectStorage::supportsCache() const

View File

@ -53,11 +53,9 @@ public:
const std::string & getCacheName() const override { return object_storage->getCacheName(); }
UInt64 getTotalSpace() const override { return std::numeric_limits<UInt64>::max(); }
UInt64 getAvailableSpace() const override { return std::numeric_limits<UInt64>::max(); }
UInt64 getUnreservedSpace() const override { return std::numeric_limits<UInt64>::max(); }
std::optional<UInt64> getTotalSpace() const override { return {}; }
std::optional<UInt64> getAvailableSpace() const override { return {}; }
std::optional<UInt64> getUnreservedSpace() const override { return {}; }
UInt64 getKeepingFreeSpace() const override { return 0; }
@ -224,7 +222,7 @@ private:
UInt64 reservation_count = 0;
std::mutex reservation_mutex;
std::optional<UInt64> tryReserve(UInt64 bytes);
bool tryReserve(UInt64 bytes);
const bool send_metadata;
@ -244,7 +242,7 @@ public:
UInt64 getSize() const override { return size; }
UInt64 getUnreservedSpace() const override { return unreserved_space; }
std::optional<UInt64> getUnreservedSpace() const override { return unreserved_space; }
DiskPtr getDisk(size_t i) const override;

View File

@ -209,10 +209,17 @@ DiskPtr StoragePolicy::tryGetDiskByName(const String & disk_name) const
UInt64 StoragePolicy::getMaxUnreservedFreeSpace() const
{
UInt64 res = 0;
std::optional<UInt64> res;
for (const auto & volume : volumes)
res = std::max(res, volume->getMaxUnreservedFreeSpace());
return res;
{
auto volume_unreserved_space = volume->getMaxUnreservedFreeSpace();
if (!volume_unreserved_space)
return -1ULL; /// There is at least one unlimited disk.
if (!res || *volume_unreserved_space > *res)
res = volume_unreserved_space;
}
return res.value_or(-1ULL);
}
@ -248,22 +255,37 @@ ReservationPtr StoragePolicy::reserveAndCheck(UInt64 bytes) const
ReservationPtr StoragePolicy::makeEmptyReservationOnLargestDisk() const
{
UInt64 max_space = 0;
bool found_bottomless_disk = false;
DiskPtr max_disk;
for (const auto & volume : volumes)
{
for (const auto & disk : volume->getDisks())
{
auto avail_space = disk->getAvailableSpace();
if (avail_space > max_space)
auto available_space = disk->getAvailableSpace();
if (!available_space)
{
max_space = avail_space;
max_disk = disk;
found_bottomless_disk = true;
break;
}
if (*available_space > max_space)
{
max_space = *available_space;
max_disk = disk;
}
}
if (found_bottomless_disk)
break;
}
if (!max_disk)
throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "There is no space on any disk in storage policy: {}. "
"It's likely all disks are broken", name);
auto reservation = max_disk->reserve(0);
if (!reservation)
{

View File

@ -40,20 +40,28 @@ VolumeJBOD::VolumeJBOD(
auto ratio = config.getDouble(config_prefix + ".max_data_part_size_ratio");
if (ratio < 0)
throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "'max_data_part_size_ratio' have to be not less then 0.");
UInt64 sum_size = 0;
std::vector<UInt64> sizes;
for (const auto & disk : disks)
{
sizes.push_back(disk->getTotalSpace());
sum_size += sizes.back();
auto size = disk->getTotalSpace();
if (size)
sum_size += *size;
else
break;
sizes.push_back(*size);
}
max_data_part_size = static_cast<decltype(max_data_part_size)>(sum_size * ratio / disks.size());
for (size_t i = 0; i < disks.size(); ++i)
if (sizes.size() == disks.size())
{
if (sizes[i] < max_data_part_size)
max_data_part_size = static_cast<UInt64>(sum_size * ratio / disks.size());
for (size_t i = 0; i < disks.size(); ++i)
{
LOG_WARNING(logger, "Disk {} on volume {} have not enough space ({}) for containing part the size of max_data_part_size ({})",
backQuote(disks[i]->getName()), backQuote(config_prefix), ReadableSize(sizes[i]), ReadableSize(max_data_part_size));
if (sizes[i] < max_data_part_size)
{
LOG_WARNING(logger, "Disk {} on volume {} have not enough space ({}) for containing part the size of max_data_part_size ({})",
backQuote(disks[i]->getName()), backQuote(config_prefix), ReadableSize(sizes[i]), ReadableSize(max_data_part_size));
}
}
}
}

View File

@ -68,7 +68,7 @@ private:
struct DiskWithSize
{
DiskPtr disk;
uint64_t free_size = 0;
std::optional<UInt64> free_size = 0;
DiskWithSize(DiskPtr disk_)
: disk(disk_)
@ -80,7 +80,7 @@ private:
return free_size < rhs.free_size;
}
ReservationPtr reserve(uint64_t bytes)
ReservationPtr reserve(UInt64 bytes)
{
ReservationPtr reservation = disk->reserve(bytes);
if (!reservation)

View File

@ -56,7 +56,7 @@ void loadDiskLocalConfig(const String & name,
tmp_path = context->getPath();
// Create tmp disk for getting total disk space.
keep_free_space_bytes = static_cast<UInt64>(DiskLocal("tmp", tmp_path, 0, config, config_prefix).getTotalSpace() * ratio);
keep_free_space_bytes = static_cast<UInt64>(*DiskLocal("tmp", tmp_path, 0, config, config_prefix).getTotalSpace() * ratio);
}
}

View File

@ -22,19 +22,19 @@ namespace
struct FilesystemAvailable
{
static constexpr auto name = "filesystemAvailable";
static std::uintmax_t get(const DiskPtr & disk) { return disk->getAvailableSpace(); }
static UInt64 get(const DiskPtr & disk) { return disk->getAvailableSpace().value_or(std::numeric_limits<UInt64>::max()); }
};
struct FilesystemUnreserved
{
static constexpr auto name = "filesystemUnreserved";
static std::uintmax_t get(const DiskPtr & disk) { return disk->getUnreservedSpace(); }
static UInt64 get(const DiskPtr & disk) { return disk->getUnreservedSpace().value_or(std::numeric_limits<UInt64>::max()); }
};
struct FilesystemCapacity
{
static constexpr auto name = "filesystemCapacity";
static std::uintmax_t get(const DiskPtr & disk) { return disk->getTotalSpace(); }
static UInt64 get(const DiskPtr & disk) { return disk->getTotalSpace().value_or(std::numeric_limits<UInt64>::max()); }
};
template <typename Impl>

View File

@ -191,14 +191,21 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values
auto available = disk->getAvailableSpace();
auto unreserved = disk->getUnreservedSpace();
new_values[fmt::format("DiskTotal_{}", name)] = { total,
"The total size in bytes of the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." };
new_values[fmt::format("DiskUsed_{}", name)] = { total - available,
"Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." };
new_values[fmt::format("DiskAvailable_{}", name)] = { available,
"Available bytes on the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." };
new_values[fmt::format("DiskUnreserved_{}", name)] = { unreserved,
"Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems can show a large value like 16 EiB." };
new_values[fmt::format("DiskTotal_{}", name)] = { *total,
"The total size in bytes of the disk (virtual filesystem). Remote filesystems may not provide this information." };
if (available)
{
new_values[fmt::format("DiskUsed_{}", name)] = { *total - *available,
"Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." };
new_values[fmt::format("DiskAvailable_{}", name)] = { *available,
"Available bytes on the disk (virtual filesystem). Remote filesystems may not provide this information." };
}
if (unreserved)
new_values[fmt::format("DiskUnreserved_{}", name)] = { *unreserved,
"Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems may not provide this information." };
}
}

View File

@ -112,11 +112,15 @@ bool MergeTreePartsMover::selectPartsForMove(
{
for (const auto & disk : volumes[i]->getDisks())
{
UInt64 required_maximum_available_space = static_cast<UInt64>(disk->getTotalSpace() * policy->getMoveFactor());
UInt64 unreserved_space = disk->getUnreservedSpace();
auto total_space = disk->getTotalSpace();
auto unreserved_space = disk->getUnreservedSpace();
if (total_space && unreserved_space)
{
UInt64 required_maximum_available_space = static_cast<UInt64>(*total_space * policy->getMoveFactor());
if (unreserved_space < required_maximum_available_space && !disk->isBroken())
need_to_move.emplace(disk, required_maximum_available_space - unreserved_space);
if (*unreserved_space < required_maximum_available_space && !disk->isBroken())
need_to_move.emplace(disk, required_maximum_available_space - *unreserved_space);
}
}
}
}

View File

@ -64,9 +64,9 @@ Pipe StorageSystemDisks::read(
{
col_name->insert(disk_name);
col_path->insert(disk_ptr->getPath());
col_free->insert(disk_ptr->getAvailableSpace());
col_total->insert(disk_ptr->getTotalSpace());
col_unreserved->insert(disk_ptr->getUnreservedSpace());
col_free->insert(disk_ptr->getAvailableSpace().value_or(std::numeric_limits<UInt64>::max()));
col_total->insert(disk_ptr->getTotalSpace().value_or(std::numeric_limits<UInt64>::max()));
col_unreserved->insert(disk_ptr->getUnreservedSpace().value_or(std::numeric_limits<UInt64>::max()));
col_keep->insert(disk_ptr->getKeepingFreeSpace());
auto data_source_description = disk_ptr->getDataSourceDescription();
col_type->insert(toString(data_source_description.type));