diff --git a/.clang-tidy b/.clang-tidy index 80b669de0cb..8c5632e5b01 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -130,7 +130,6 @@ Checks: '*, -readability-implicit-bool-conversion, -readability-isolate-declaration, -readability-magic-numbers, - -readability-misleading-indentation, -readability-named-parameter, -readability-qualified-auto, -readability-redundant-declaration, diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index a55d588f2b5..8abf0b24782 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -627,27 +627,27 @@ bool DiskLocal::setup() /// Try to create a new checker file. The disk status can be either broken or readonly. if (disk_checker_magic_number == -1) - try - { - pcg32_fast rng(randomSeed()); - UInt32 magic_number = rng(); + try { - auto buf = writeFile(disk_checker_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); - writeIntBinary(magic_number, *buf); + pcg32_fast rng(randomSeed()); + UInt32 magic_number = rng(); + { + auto buf = writeFile(disk_checker_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); + writeIntBinary(magic_number, *buf); + } + disk_checker_magic_number = magic_number; + } + catch (...) + { + LOG_WARNING( + logger, + "Cannot create/write to {0}. Disk {1} is either readonly or broken. Without setting up disk checker file, DiskLocalCheckThread " + "will not be started. Disk is assumed to be RW. Try manually fix the disk and do `SYSTEM RESTART DISK {1}`", + disk_checker_path, + name); + disk_checker_can_check_read = false; + return true; } - disk_checker_magic_number = magic_number; - } - catch (...) - { - LOG_WARNING( - logger, - "Cannot create/write to {0}. Disk {1} is either readonly or broken. Without setting up disk checker file, DiskLocalCheckThread " - "will not be started. Disk is assumed to be RW. Try manually fix the disk and do `SYSTEM RESTART DISK {1}`", - disk_checker_path, - name); - disk_checker_can_check_read = false; - return true; - } if (disk_checker_magic_number == -1) throw Exception("disk_checker_magic_number is not initialized. It's a bug", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index dedde89a59e..d8e44dc297d 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -700,8 +700,8 @@ ColumnPtr FunctionArrayElement::executeArgument( || (res = executeConst(arguments, result_type, index_data, builder, input_rows_count)) || (res = executeString(arguments, index_data, builder)) || (res = executeGeneric(arguments, index_data, builder)))) - throw Exception("Illegal column " + arguments[0].column->getName() - + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Illegal column " + arguments[0].column->getName() + + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); return res; } diff --git a/src/Functions/pointInEllipses.cpp b/src/Functions/pointInEllipses.cpp index ea08b276670..1de6e1d5ce6 100644 --- a/src/Functions/pointInEllipses.cpp +++ b/src/Functions/pointInEllipses.cpp @@ -151,21 +151,21 @@ private: } return dst; - } - else if (const_cnt == 2) - { - const auto * col_const_x = assert_cast (col_x); - const auto * col_const_y = assert_cast (col_y); - size_t start_index = 0; - UInt8 res = isPointInEllipses(col_const_x->getValue(), col_const_y->getValue(), ellipses.data(), ellipses_count, start_index); - return DataTypeUInt8().createColumnConst(size, res); - } - else - { - throw Exception( - "Illegal types " + col_x->getName() + ", " + col_y->getName() + " of arguments 1, 2 of function " + getName() + ". Both must be either const or vector", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } + } + else if (const_cnt == 2) + { + const auto * col_const_x = assert_cast (col_x); + const auto * col_const_y = assert_cast (col_y); + size_t start_index = 0; + UInt8 res = isPointInEllipses(col_const_x->getValue(), col_const_y->getValue(), ellipses.data(), ellipses_count, start_index); + return DataTypeUInt8().createColumnConst(size, res); + } + else + { + throw Exception( + "Illegal types " + col_x->getName() + ", " + col_y->getName() + " of arguments 1, 2 of function " + getName() + ". Both must be either const or vector", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } } static bool isPointInEllipses(Float64 x, Float64 y, const Ellipse * ellipses, size_t ellipses_count, size_t & start_index) diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 1744c582b28..7772c8cdb3b 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -230,7 +230,7 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional ses log_entry.roles = roles_info->getCurrentRolesNames(); if (const auto profile_info = access->getDefaultProfileInfo()) - log_entry.profiles = profile_info->getProfileNames(); + log_entry.profiles = profile_info->getProfileNames(); for (const auto & s : settings.allChanged()) log_entry.settings.emplace_back(s.getName(), s.getValueString()); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5e2b7444b1b..96191a1c3d3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6490,153 +6490,153 @@ ReservationPtr MergeTreeData::balancedReservation( ReservationPtr reserved_space; auto min_bytes_to_rebalance_partition_over_jbod = getSettings()->min_bytes_to_rebalance_partition_over_jbod; if (tagger_ptr && min_bytes_to_rebalance_partition_over_jbod > 0 && part_size >= min_bytes_to_rebalance_partition_over_jbod) - try - { - const auto & disks = getStoragePolicy()->getVolume(max_volume_index)->getDisks(); - std::map disk_occupation; - std::map> disk_parts_for_logging; - for (const auto & disk : disks) - disk_occupation.emplace(disk->getName(), 0); - - std::set committed_big_parts_from_partition; - std::set submerging_big_parts_from_partition; - std::lock_guard lock(currently_submerging_emerging_mutex); - - for (const auto & part : currently_submerging_big_parts) + try { - if (part_info.partition_id == part->info.partition_id) - submerging_big_parts_from_partition.insert(part->name); - } + const auto & disks = getStoragePolicy()->getVolume(max_volume_index)->getDisks(); + std::map disk_occupation; + std::map> disk_parts_for_logging; + for (const auto & disk : disks) + disk_occupation.emplace(disk->getName(), 0); - { - auto lock_parts = lockParts(); - if (covered_parts.empty()) + std::set committed_big_parts_from_partition; + std::set submerging_big_parts_from_partition; + std::lock_guard lock(currently_submerging_emerging_mutex); + + for (const auto & part : currently_submerging_big_parts) { - // It's a part fetch. Calculate `covered_parts` here. - MergeTreeData::DataPartPtr covering_part; - covered_parts = getActivePartsToReplace(part_info, part_name, covering_part, lock_parts); + if (part_info.partition_id == part->info.partition_id) + submerging_big_parts_from_partition.insert(part->name); } - // Remove irrelevant parts. - std::erase_if(covered_parts, - [min_bytes_to_rebalance_partition_over_jbod](const auto & part) - { - return !(part->isStoredOnDisk() && part->getBytesOnDisk() >= min_bytes_to_rebalance_partition_over_jbod); - }); - - // Include current submerging big parts which are not yet in `currently_submerging_big_parts` - for (const auto & part : covered_parts) - submerging_big_parts_from_partition.insert(part->name); - - for (const auto & part : getDataPartsStateRange(MergeTreeData::DataPartState::Active)) { - if (part->isStoredOnDisk() && part->getBytesOnDisk() >= min_bytes_to_rebalance_partition_over_jbod - && part_info.partition_id == part->info.partition_id) + auto lock_parts = lockParts(); + if (covered_parts.empty()) { - auto name = part->volume->getDisk()->getName(); - auto it = disk_occupation.find(name); - if (it != disk_occupation.end()) - { - if (submerging_big_parts_from_partition.find(part->name) == submerging_big_parts_from_partition.end()) + // It's a part fetch. Calculate `covered_parts` here. + MergeTreeData::DataPartPtr covering_part; + covered_parts = getActivePartsToReplace(part_info, part_name, covering_part, lock_parts); + } + + // Remove irrelevant parts. + std::erase_if(covered_parts, + [min_bytes_to_rebalance_partition_over_jbod](const auto & part) { - it->second += part->getBytesOnDisk(); - disk_parts_for_logging[name].push_back(formatReadableSizeWithBinarySuffix(part->getBytesOnDisk())); - committed_big_parts_from_partition.insert(part->name); + return !(part->isStoredOnDisk() && part->getBytesOnDisk() >= min_bytes_to_rebalance_partition_over_jbod); + }); + + // Include current submerging big parts which are not yet in `currently_submerging_big_parts` + for (const auto & part : covered_parts) + submerging_big_parts_from_partition.insert(part->name); + + for (const auto & part : getDataPartsStateRange(MergeTreeData::DataPartState::Active)) + { + if (part->isStoredOnDisk() && part->getBytesOnDisk() >= min_bytes_to_rebalance_partition_over_jbod + && part_info.partition_id == part->info.partition_id) + { + auto name = part->volume->getDisk()->getName(); + auto it = disk_occupation.find(name); + if (it != disk_occupation.end()) + { + if (submerging_big_parts_from_partition.find(part->name) == submerging_big_parts_from_partition.end()) + { + it->second += part->getBytesOnDisk(); + disk_parts_for_logging[name].push_back(formatReadableSizeWithBinarySuffix(part->getBytesOnDisk())); + committed_big_parts_from_partition.insert(part->name); + } + else + { + disk_parts_for_logging[name].push_back(formatReadableSizeWithBinarySuffix(part->getBytesOnDisk()) + " (submerging)"); + } } else { - disk_parts_for_logging[name].push_back(formatReadableSizeWithBinarySuffix(part->getBytesOnDisk()) + " (submerging)"); + // Part is on different volume. Ignore it. } } + } + } + + for (const auto & [name, emerging_part] : currently_emerging_big_parts) + { + // It's possible that the emerging big parts are committed and get added twice. Thus a set is used to deduplicate. + if (committed_big_parts_from_partition.find(name) == committed_big_parts_from_partition.end() + && part_info.partition_id == emerging_part.partition_id) + { + auto it = disk_occupation.find(emerging_part.disk_name); + if (it != disk_occupation.end()) + { + it->second += emerging_part.estimate_bytes; + disk_parts_for_logging[emerging_part.disk_name].push_back( + formatReadableSizeWithBinarySuffix(emerging_part.estimate_bytes) + " (emerging)"); + } else { // Part is on different volume. Ignore it. } } } - } - for (const auto & [name, emerging_part] : currently_emerging_big_parts) - { - // It's possible that the emerging big parts are committed and get added twice. Thus a set is used to deduplicate. - if (committed_big_parts_from_partition.find(name) == committed_big_parts_from_partition.end() - && part_info.partition_id == emerging_part.partition_id) + size_t min_occupation_size = std::numeric_limits::max(); + std::vector candidates; + for (const auto & [disk_name, size] : disk_occupation) { - auto it = disk_occupation.find(emerging_part.disk_name); - if (it != disk_occupation.end()) + if (size < min_occupation_size) { - it->second += emerging_part.estimate_bytes; - disk_parts_for_logging[emerging_part.disk_name].push_back( - formatReadableSizeWithBinarySuffix(emerging_part.estimate_bytes) + " (emerging)"); + min_occupation_size = size; + candidates = {disk_name}; } + else if (size == min_occupation_size) + { + candidates.push_back(disk_name); + } + } + + if (!candidates.empty()) + { + // Random pick one disk from best candidates + std::shuffle(candidates.begin(), candidates.end(), thread_local_rng); + String selected_disk_name = candidates.front(); + WriteBufferFromOwnString log_str; + writeCString("\nbalancer: \n", log_str); + for (const auto & [disk_name, per_disk_parts] : disk_parts_for_logging) + writeString(fmt::format(" {}: [{}]\n", disk_name, fmt::join(per_disk_parts, ", ")), log_str); + LOG_DEBUG(log, fmt::runtime(log_str.str())); + + if (ttl_infos) + reserved_space = tryReserveSpacePreferringTTLRules( + metadata_snapshot, + part_size, + *ttl_infos, + time(nullptr), + max_volume_index, + is_insert, + getStoragePolicy()->getDiskByName(selected_disk_name)); else + reserved_space = tryReserveSpace(part_size, getStoragePolicy()->getDiskByName(selected_disk_name)); + + if (reserved_space) { - // Part is on different volume. Ignore it. + currently_emerging_big_parts.emplace( + part_name, EmergingPartInfo{reserved_space->getDisk(0)->getName(), part_info.partition_id, part_size}); + + for (const auto & part : covered_parts) + { + if (currently_submerging_big_parts.contains(part)) + LOG_WARNING(log, "currently_submerging_big_parts contains duplicates. JBOD might lose balance"); + else + currently_submerging_big_parts.insert(part); + } + + // Record submerging big parts in the tagger to clean them up. + tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log); } } } - - size_t min_occupation_size = std::numeric_limits::max(); - std::vector candidates; - for (const auto & [disk_name, size] : disk_occupation) + catch (...) { - if (size < min_occupation_size) - { - min_occupation_size = size; - candidates = {disk_name}; - } - else if (size == min_occupation_size) - { - candidates.push_back(disk_name); - } + LOG_DEBUG(log, "JBOD balancer encounters an error. Fallback to random disk selection"); + tryLogCurrentException(log); } - - if (!candidates.empty()) - { - // Random pick one disk from best candidates - std::shuffle(candidates.begin(), candidates.end(), thread_local_rng); - String selected_disk_name = candidates.front(); - WriteBufferFromOwnString log_str; - writeCString("\nbalancer: \n", log_str); - for (const auto & [disk_name, per_disk_parts] : disk_parts_for_logging) - writeString(fmt::format(" {}: [{}]\n", disk_name, fmt::join(per_disk_parts, ", ")), log_str); - LOG_DEBUG(log, fmt::runtime(log_str.str())); - - if (ttl_infos) - reserved_space = tryReserveSpacePreferringTTLRules( - metadata_snapshot, - part_size, - *ttl_infos, - time(nullptr), - max_volume_index, - is_insert, - getStoragePolicy()->getDiskByName(selected_disk_name)); - else - reserved_space = tryReserveSpace(part_size, getStoragePolicy()->getDiskByName(selected_disk_name)); - - if (reserved_space) - { - currently_emerging_big_parts.emplace( - part_name, EmergingPartInfo{reserved_space->getDisk(0)->getName(), part_info.partition_id, part_size}); - - for (const auto & part : covered_parts) - { - if (currently_submerging_big_parts.contains(part)) - LOG_WARNING(log, "currently_submerging_big_parts contains duplicates. JBOD might lose balance"); - else - currently_submerging_big_parts.insert(part); - } - - // Record submerging big parts in the tagger to clean them up. - tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log); - } - } - } - catch (...) - { - LOG_DEBUG(log, "JBOD balancer encounters an error. Fallback to random disk selection"); - tryLogCurrentException(log); - } return reserved_space; } diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 915d4d92b7e..11a1f8ba4d6 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -70,16 +70,16 @@ void StorageMongoDB::connectIfNotConnected() auto auth_db = database_name; if (auth_source != query_params.end()) auth_db = auth_source->second; -# if POCO_VERSION >= 0x01070800 - if (!username.empty() && !password.empty()) - { - Poco::MongoDB::Database poco_db(auth_db); - if (!poco_db.authenticate(*connection, username, password, Poco::MongoDB::Database::AUTH_SCRAM_SHA1)) - throw Exception("Cannot authenticate in MongoDB, incorrect user or password", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE); - } -# else - authenticate(*connection, database_name, username, password); -# endif +#if POCO_VERSION >= 0x01070800 + if (!username.empty() && !password.empty()) + { + Poco::MongoDB::Database poco_db(auth_db); + if (!poco_db.authenticate(*connection, username, password, Poco::MongoDB::Database::AUTH_SCRAM_SHA1)) + throw Exception("Cannot authenticate in MongoDB, incorrect user or password", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE); + } +#else + authenticate(*connection, database_name, username, password); +#endif authenticated = true; } }