QOL log improvements (#41947)

* Uniformize disk reservation logs

* Remove log about destroying stuff that appears all the time

* More tweaks on disk reservation logs

* Reorder logs in hash join

* Remove log that provides little information

* Collapse part removal logs

Co-authored-by: Sergei Trifonov <sergei@clickhouse.com>
This commit is contained in:
Raúl Marín 2022-10-06 14:22:44 +02:00 committed by GitHub
parent 54ec6a5047
commit adbaaca2f5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 52 additions and 28 deletions

View File

@ -230,19 +230,27 @@ std::optional<UInt64> DiskLocal::tryReserve(UInt64 bytes)
if (bytes == 0)
{
LOG_DEBUG(logger, "Reserving 0 bytes on disk {}", backQuote(name));
LOG_TRACE(logger, "Reserved 0 bytes on local disk {}", backQuote(name));
++reservation_count;
return {unreserved_space};
}
if (unreserved_space >= bytes)
{
LOG_DEBUG(logger, "Reserving {} on disk {}, having unreserved {}.",
ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space));
LOG_TRACE(
logger,
"Reserved {} on local disk {}, having unreserved {}.",
ReadableSize(bytes),
backQuote(name),
ReadableSize(unreserved_space));
++reservation_count;
reserved_bytes += bytes;
return {unreserved_space - bytes};
}
else
{
LOG_TRACE(logger, "Could not reserve {} on local disk {}. Not enough unreserved space", ReadableSize(bytes), backQuote(name));
}
return {};
}

View File

@ -459,19 +459,27 @@ std::optional<UInt64> DiskObjectStorage::tryReserve(UInt64 bytes)
if (bytes == 0)
{
LOG_TRACE(log, "Reserving 0 bytes on remote_fs disk {}", backQuote(name));
LOG_TRACE(log, "Reserved 0 bytes on remote disk {}", backQuote(name));
++reservation_count;
return {unreserved_space};
}
if (unreserved_space >= bytes)
{
LOG_TRACE(log, "Reserving {} on disk {}, having unreserved {}.",
ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space));
LOG_TRACE(
log,
"Reserved {} on remote disk {}, having unreserved {}.",
ReadableSize(bytes),
backQuote(name),
ReadableSize(unreserved_space));
++reservation_count;
reserved_bytes += bytes;
return {unreserved_space - bytes};
}
else
{
LOG_TRACE(log, "Could not reserve {} on remote disk {}. Not enough unreserved space", ReadableSize(bytes), backQuote(name));
}
return {};
}

View File

@ -5,6 +5,7 @@
#include <Interpreters/Context.h>
#include <Common/escapeForFileName.h>
#include <Common/formatReadable.h>
#include <Common/quoteString.h>
#include <set>
@ -212,17 +213,15 @@ UInt64 StoragePolicy::getMaxUnreservedFreeSpace() const
ReservationPtr StoragePolicy::reserve(UInt64 bytes, size_t min_volume_index) const
{
LOG_TRACE(log, "Reserving bytes {} from volume index {}, total volumes {}", bytes, min_volume_index, volumes.size());
for (size_t i = min_volume_index; i < volumes.size(); ++i)
{
const auto & volume = volumes[i];
auto reservation = volume->reserve(bytes);
if (reservation)
{
LOG_TRACE(log, "Successfully reserved {} bytes on volume index {}", bytes, i);
return reservation;
}
}
LOG_TRACE(log, "Could not reserve {} from volume index {}, total volumes {}", ReadableSize(bytes), min_volume_index, volumes.size());
return {};
}

View File

@ -3275,8 +3275,6 @@ void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result) cons
if (result.empty())
return;
LOG_TRACE(log, "Destroying aggregate states");
/// In what data structure is the data aggregated?
if (result.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
destroyWithoutKey(result);

View File

@ -249,7 +249,6 @@ FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader()
return nullptr;
}
LOG_TRACE(log, "Extracted reader from file segment");
return std::move(remote_file_reader);
}

View File

@ -224,6 +224,7 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
, right_sample_block(right_sample_block_)
, log(&Poco::Logger::get("HashJoin"))
{
LOG_DEBUG(log, "HashJoin. Datatype: {}, kind: {}, strictness: {}", data->type, kind, strictness);
LOG_DEBUG(log, "Right sample block: {}", right_sample_block.dumpStructure());
if (isCrossOrComma(kind))
@ -303,8 +304,6 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
for (auto & maps : data->maps)
dataMapInit(maps);
LOG_DEBUG(log, "Join type: {}, kind: {}, strictness: {}", data->type, kind, strictness);
}
HashJoin::Type HashJoin::chooseMethod(JoinKind kind, const ColumnRawPtrs & key_columns, Sizes & key_sizes)

View File

@ -360,15 +360,15 @@ private:
friend class JoinSource;
std::shared_ptr<TableJoin> table_join;
JoinKind kind;
JoinStrictness strictness;
const JoinKind kind;
const JoinStrictness strictness;
/// This join was created from StorageJoin and it is already filled.
bool from_storage_join = false;
bool any_take_last_row; /// Overwrite existing values when encountering the same key again
std::optional<TypeIndex> asof_type;
ASOFJoinInequality asof_inequality;
const ASOFJoinInequality asof_inequality;
/// Right table data. StorageJoin shares it between many Join objects.
/// Flags that indicate that particular row already used in join.

View File

@ -91,6 +91,18 @@
#include <unordered_set>
#include <filesystem>
#include <fmt/format.h>
template <>
struct fmt::formatter<DB::DataPartPtr> : fmt::formatter<std::string>
{
template <typename FormatCtx>
auto format(const DB::DataPartPtr & part, FormatCtx & ctx) const
{
return fmt::formatter<std::string>::format(part->name, ctx);
}
};
namespace fs = std::filesystem;
@ -1905,6 +1917,7 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
ThreadPool pool(num_threads);
/// NOTE: Under heavy system load you may get "Cannot schedule a task" from ThreadPool.
LOG_DEBUG(log, "Removing {} parts from filesystem: {} (concurrently)", parts_to_remove.size(), fmt::join(parts_to_remove, ", "));
for (const DataPartPtr & part : parts_to_remove)
{
pool.scheduleOrThrowOnError([&, thread_group = CurrentThread::getGroup()]
@ -1912,7 +1925,6 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
LOG_DEBUG(log, "Removing part from filesystem {} (concurrently)", part->name);
part->remove();
if (part_names_succeed)
{
@ -1924,11 +1936,11 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
pool.wait();
}
else
else if (!parts_to_remove.empty())
{
LOG_DEBUG(log, "Removing {} parts from filesystem: {}", parts_to_remove.size(), fmt::join(parts_to_remove, ", "));
for (const DataPartPtr & part : parts_to_remove)
{
LOG_DEBUG(log, "Removing part from filesystem {}", part->name);
part->remove();
if (part_names_succeed)
part_names_succeed->insert(part->name);
@ -4911,15 +4923,13 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(
DiskPtr selected_disk) const
{
expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size);
LOG_TRACE(log, "Trying reserve {} bytes preffering TTL rules", expected_size);
ReservationPtr reservation;
auto move_ttl_entry = selectTTLDescriptionForTTLInfos(metadata_snapshot->getMoveTTLs(), ttl_infos.moves_ttl, time_of_move, true);
if (move_ttl_entry)
{
LOG_TRACE(log, "Got move TTL entry, will try to reserver destination for move");
LOG_TRACE(log, "Trying to reserve {} to apply a TTL rule. Will try to reserve in the destination", ReadableSize(expected_size));
SpacePtr destination_ptr = getDestinationForMoveTTL(*move_ttl_entry);
bool perform_ttl_move_on_insert = is_insert && destination_ptr && shouldPerformTTLMoveOnInsert(destination_ptr);
@ -4949,11 +4959,9 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(
}
else
{
LOG_TRACE(log, "Reserving bytes on selected destination");
reservation = destination_ptr->reserve(expected_size);
if (reservation)
{
LOG_TRACE(log, "Reservation successful");
return reservation;
}
else
@ -4977,13 +4985,18 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(
// Prefer selected_disk
if (selected_disk)
{
LOG_DEBUG(log, "Disk for reservation provided: {} (with type {})", selected_disk->getName(), toString(selected_disk->getDataSourceDescription().type));
LOG_TRACE(
log,
"Trying to reserve {} on the selected disk: {} (with type {})",
ReadableSize(expected_size),
selected_disk->getName(),
toString(selected_disk->getDataSourceDescription().type));
reservation = selected_disk->reserve(expected_size);
}
if (!reservation)
{
LOG_DEBUG(log, "No reservation, reserving using storage policy from min volume index {}", min_volume_index);
LOG_TRACE(log, "Trying to reserve {} using storage policy from min volume index {}", ReadableSize(expected_size), min_volume_index);
reservation = getStoragePolicy()->reserve(expected_size, min_volume_index);
}