diff --git a/.clang-tidy b/.clang-tidy index cfb42ebd4c7..cbeac377079 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -5,6 +5,9 @@ # a) the new check is not controversial (this includes many checks in readability-* and google-*) or # b) too noisy (checks with > 100 new warnings are considered noisy, this includes e.g. cppcoreguidelines-*). +# TODO: Once clang(-tidy) 17 is the minimum, we can convert this list to YAML +# See https://releases.llvm.org/17.0.1/tools/clang/tools/extra/docs/ReleaseNotes.html#improvements-to-clang-tidy + # TODO Let clang-tidy check headers in further directories # --> HeaderFilterRegex: '^.*/(src|base|programs|utils)/.*(h|hpp)$' HeaderFilterRegex: '^.*/(base)/.*(h|hpp)$' @@ -25,6 +28,7 @@ Checks: '*, -bugprone-not-null-terminated-result, -bugprone-reserved-identifier, # useful but too slow, TODO retry when https://reviews.llvm.org/rG1c282052624f9d0bd273bde0b47b30c96699c6c7 is merged -bugprone-unchecked-optional-access, + -bugprone-*, -- category temporarily disabled because some check(s) in it are slow -cert-dcl16-c, -cert-dcl37-c, @@ -39,6 +43,7 @@ Checks: '*, -clang-analyzer-optin.portability.UnixAPI, -clang-analyzer-security.insecureAPI.bzero, -clang-analyzer-security.insecureAPI.strcpy, + -clang-analyzer-*, -- category temporarily disabled because some check(s) in it are slow -cppcoreguidelines-avoid-c-arrays, -cppcoreguidelines-avoid-const-or-ref-data-members, @@ -67,6 +72,7 @@ Checks: '*, -cppcoreguidelines-pro-type-vararg, -cppcoreguidelines-slicing, -cppcoreguidelines-special-member-functions, + -cppcoreguidelines-*, -- category temporarily disabled because some check(s) in it are slow -darwin-*, @@ -128,10 +134,12 @@ Checks: '*, -performance-inefficient-string-concatenation, -performance-no-int-to-ptr, + -performance-avoid-endl, -performance-unnecessary-value-param, -portability-simd-intrinsics, + -readability-avoid-unconditional-preprocessor-if, -readability-braces-around-statements, -readability-convert-member-functions-to-static, -readability-else-after-return, @@ -155,6 +163,12 @@ Checks: '*, WarningsAsErrors: '*' +ExtraArgs: +# clang-tidy 17 started to complain (for unknown reasons) that various pragmas are unknown ("clang-diagnostic-unknown-pragmas"). +# This is technically a compiler error, not a clang-tidy error. We could litter the code base with more pragmas that suppress +# this error but it is better to pass the following flag to the compiler: +- '-Wno-unknown-pragmas' + CheckOptions: readability-identifier-naming.ClassCase: CamelCase readability-identifier-naming.EnumCase: CamelCase diff --git a/base/pcg-random/pcg_extras.hpp b/base/pcg-random/pcg_extras.hpp index cc11d907006..32dc5c318c3 100644 --- a/base/pcg-random/pcg_extras.hpp +++ b/base/pcg-random/pcg_extras.hpp @@ -463,7 +463,7 @@ auto bounded_rand(RngType& rng, typename RngType::result_type upper_bound) } template -void shuffle(Iter from, Iter to, RandType&& rng) +void shuffle(Iter from, Iter to, RandType&& rng) // NOLINT(cppcoreguidelines-missing-std-forward) { typedef typename std::iterator_traits::difference_type delta_t; typedef typename std::remove_reference::type::result_type result_t; diff --git a/src/Bridge/IBridge.cpp b/src/Bridge/IBridge.cpp index c913a6e80c0..87a5a0c2d52 100644 --- a/src/Bridge/IBridge.cpp +++ b/src/Bridge/IBridge.cpp @@ -140,7 +140,7 @@ void IBridge::initialize(Application & self) throw Poco::OpenFileException("Cannot attach stdout to " + stdout_path); /// Disable buffering for stdout. - setbuf(stdout, nullptr); + setbuf(stdout, nullptr); // NOLINT(cert-msc24-c,cert-msc33-c) } const auto stderr_path = config().getString("logger.stderr", ""); if (!stderr_path.empty()) @@ -149,7 +149,7 @@ void IBridge::initialize(Application & self) throw Poco::OpenFileException("Cannot attach stderr to " + stderr_path); /// Disable buffering for stderr. - setbuf(stderr, nullptr); + setbuf(stderr, nullptr); // NOLINT(cert-msc24-c,cert-msc33-c) } buildLoggers(config(), logger(), self.commandName()); diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 6a685b602ae..9cb352da0ba 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -310,8 +310,8 @@ static String cacheElemToString(const Poco::Net::IPAddress & addr) { return addr template bool DNSResolver::updateCacheImpl( - UpdateF && update_func, - ElemsT && elems, + UpdateF && update_func, // NOLINT(cppcoreguidelines-missing-std-forward) + ElemsT && elems, // NOLINT(cppcoreguidelines-missing-std-forward) UInt32 max_consecutive_failures, FormatStringHelper notfound_log_msg, FormatStringHelper dropped_log_msg) diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index fa89e2da645..24f16a55c25 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -36,9 +36,7 @@ using namespace DB; namespace { -template using is_pod = std::is_trivial>; -template inline constexpr bool is_pod_v = is_pod::value; - +template inline constexpr bool is_pod_v = std::is_trivial_v>; template struct AsHexStringHelper diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 99f280d3641..d796818e9cb 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -429,7 +429,7 @@ constexpr auto getEnumValues() if (it != map.end()) \ return it->second; \ throw Exception(ERROR_CODE_FOR_UNEXPECTED_NAME, \ - "Unexpected value of " #NEW_NAME ":{}", std::to_string(std::underlying_type::type(value))); \ + "Unexpected value of " #NEW_NAME ":{}", std::to_string(std::underlying_type_t(value))); \ } \ \ typename SettingField##NEW_NAME::EnumType SettingField##NEW_NAME##Traits::fromString(std::string_view str) \ diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index be323dc6786..84e778ab185 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -843,7 +843,7 @@ void BaseDaemon::initialize(Application & self) throw Poco::OpenFileException("Cannot attach stderr to " + stderr_path); /// Disable buffering for stderr - setbuf(stderr, nullptr); + setbuf(stderr, nullptr); // NOLINT(cert-msc24-c,cert-msc33-c) } if ((!log_path.empty() && is_daemon) || config().has("logger.stdout")) diff --git a/src/DataTypes/DataTypeLowCardinality.cpp b/src/DataTypes/DataTypeLowCardinality.cpp index 8293455cabc..079d9d82cd7 100644 --- a/src/DataTypes/DataTypeLowCardinality.cpp +++ b/src/DataTypes/DataTypeLowCardinality.cpp @@ -111,7 +111,7 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUnique(const IDataTyp { auto creator = [&](auto x) { - using ColumnType = typename std::remove_pointer::type; + using ColumnType = typename std::remove_pointer_t; return ColumnUnique::create(keys_type); }; return createColumnUniqueImpl(keys_type, creator); @@ -121,7 +121,7 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUnique(const IDataTyp { auto creator = [&](auto x) { - using ColumnType = typename std::remove_pointer::type; + using ColumnType = typename std::remove_pointer_t; return ColumnUnique::create(std::move(keys), keys_type.isNullable()); }; return createColumnUniqueImpl(keys_type, creator); diff --git a/src/Functions/GatherUtils/ends_with.cpp b/src/Functions/GatherUtils/ends_with.cpp index 579d903005a..0d9537f88df 100644 --- a/src/Functions/GatherUtils/ends_with.cpp +++ b/src/Functions/GatherUtils/ends_with.cpp @@ -15,7 +15,7 @@ struct ArrayEndsWithSelectArraySourcePair : public ArraySourcePairSelector::type; + using SourceType = typename std::decay_t; if (is_second_nullable) { @@ -40,7 +40,7 @@ struct ArrayEndsWithSelectArraySourcePair : public ArraySourcePairSelector::type; + using SourceType = typename std::decay_t; if (is_first_nullable) { diff --git a/src/Functions/GatherUtils/has_all.cpp b/src/Functions/GatherUtils/has_all.cpp index 6e34a851c02..c0be9c9273a 100644 --- a/src/Functions/GatherUtils/has_all.cpp +++ b/src/Functions/GatherUtils/has_all.cpp @@ -15,7 +15,7 @@ struct ArrayHasAllSelectArraySourcePair : public ArraySourcePairSelector::type; + using SourceType = typename std::decay_t; if (is_second_nullable) { @@ -40,7 +40,7 @@ struct ArrayHasAllSelectArraySourcePair : public ArraySourcePairSelector::type; + using SourceType = typename std::decay_t; if (is_first_nullable) { diff --git a/src/Functions/GatherUtils/has_any.cpp b/src/Functions/GatherUtils/has_any.cpp index b7a8c9f620d..fa8df466096 100644 --- a/src/Functions/GatherUtils/has_any.cpp +++ b/src/Functions/GatherUtils/has_any.cpp @@ -15,7 +15,7 @@ struct ArrayHasAnySelectArraySourcePair : public ArraySourcePairSelector::type; + using SourceType = typename std::decay_t; if (is_second_nullable) { @@ -40,7 +40,7 @@ struct ArrayHasAnySelectArraySourcePair : public ArraySourcePairSelector::type; + using SourceType = typename std::decay_t; if (is_first_nullable) { diff --git a/src/Functions/GatherUtils/has_substr.cpp b/src/Functions/GatherUtils/has_substr.cpp index 244a1d21633..0d579a8460a 100644 --- a/src/Functions/GatherUtils/has_substr.cpp +++ b/src/Functions/GatherUtils/has_substr.cpp @@ -15,7 +15,7 @@ struct ArrayHasSubstrSelectArraySourcePair : public ArraySourcePairSelector::type; + using SourceType = typename std::decay_t; if (is_second_nullable) { @@ -40,7 +40,7 @@ struct ArrayHasSubstrSelectArraySourcePair : public ArraySourcePairSelector::type; + using SourceType = typename std::decay_t; if (is_first_nullable) { diff --git a/src/Functions/GatherUtils/starts_with.cpp b/src/Functions/GatherUtils/starts_with.cpp index 813294bc092..790e28f9528 100644 --- a/src/Functions/GatherUtils/starts_with.cpp +++ b/src/Functions/GatherUtils/starts_with.cpp @@ -15,7 +15,7 @@ struct ArrayStartsWithSelectArraySourcePair : public ArraySourcePairSelector::type; + using SourceType = typename std::decay_t; if (is_second_nullable) { @@ -40,7 +40,7 @@ struct ArrayStartsWithSelectArraySourcePair : public ArraySourcePairSelector::type; + using SourceType = typename std::decay_t; if (is_first_nullable) { diff --git a/src/Functions/UTCTimestampTransform.cpp b/src/Functions/UTCTimestampTransform.cpp index ff3c9c27ffc..77cc1826941 100644 --- a/src/Functions/UTCTimestampTransform.cpp +++ b/src/Functions/UTCTimestampTransform.cpp @@ -71,8 +71,8 @@ namespace { if (arguments.size() != 2) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {}'s arguments number must be 2.", name); - ColumnWithTypeAndName arg1 = arguments[0]; - ColumnWithTypeAndName arg2 = arguments[1]; + const ColumnWithTypeAndName & arg1 = arguments[0]; + const ColumnWithTypeAndName & arg2 = arguments[1]; const auto * time_zone_const_col = checkAndGetColumnConstData(arg2.column.get()); if (!time_zone_const_col) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of 2nd argument of function {}. Excepted const(String).", arg2.column->getName(), name); diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 6bdba251c36..83fde8e8830 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -434,7 +434,7 @@ ReturnType parseDateTimeBestEffortImpl( num_digits = readDigits(digits, sizeof(digits), in); if (fractional) { - using FractionalType = typename std::decayvalue)>::type; + using FractionalType = typename std::decay_tvalue)>; // Reading more decimal digits than fits into FractionalType would case an // overflow, so it is better to skip all digits from the right side that do not // fit into result type. To provide less precise value rather than bogus one. diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 4cac2f0e20c..f3a65154524 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -103,7 +103,7 @@ void addDefaultRequiredExpressionsRecursively( /// and this identifier will be in required columns. If such column is not in ColumnsDescription we ignore it. /// This column is required, but doesn't have default expression, so lets use "default default" - auto column = columns.get(required_column_name); + const auto & column = columns.get(required_column_name); auto default_value = column.type->getDefault(); ASTPtr expr = std::make_shared(default_value); if (is_column_in_query && convert_null_to_default) diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index d9cfd40e168..978787d07cd 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -229,10 +229,10 @@ struct StatisticsStringRef /// The Coverter* structs below are responsible for that. /// When conversion is not needed, getBatch() will just return pointer into original data. -template ::value, +template , To, - typename std::make_unsigned::type>::type> + typename std::make_unsigned_t>> struct ConverterNumeric { using Statistics = StatisticsNumeric; @@ -517,14 +517,14 @@ void writeColumnImpl( bool use_dictionary = options.use_dictionary_encoding && !s.is_bool; std::optional fixed_string_descr; - if constexpr (std::is_same::value) + if constexpr (std::is_same_v) { /// This just communicates one number to MakeTypedEncoder(): the fixed string length. fixed_string_descr.emplace(parquet::schema::PrimitiveNode::Make( "", parquet::Repetition::REQUIRED, parquet::Type::FIXED_LEN_BYTE_ARRAY, parquet::ConvertedType::NONE, static_cast(converter.fixedStringSize())), 0, 0); - if constexpr (std::is_same::value) + if constexpr (std::is_same_v) page_statistics.fixed_string_size = converter.fixedStringSize(); } diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index f38fc1f3734..b99a712c672 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1154,7 +1154,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const "Column {} doesn't have MATERIALIZED, cannot remove it", backQuote(column_name)); - auto column_from_table = all_columns.get(column_name); + const auto & column_from_table = all_columns.get(column_name); if (command.to_remove == AlterCommand::RemoveProperty::TTL && column_from_table.ttl == nullptr) throw Exception( ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Storages/Kafka/KafkaProducer.cpp b/src/Storages/Kafka/KafkaProducer.cpp index c36f9b48315..edbfc76ef93 100644 --- a/src/Storages/Kafka/KafkaProducer.cpp +++ b/src/Storages/Kafka/KafkaProducer.cpp @@ -27,7 +27,7 @@ KafkaProducer::KafkaProducer( if (header.has("_key")) { auto column_index = header.getPositionByName("_key"); - auto column_info = header.getByPosition(column_index); + const auto & column_info = header.getByPosition(column_index); if (isString(column_info.type)) key_column_index = column_index; // else ? (not sure it's a good place to report smth to user) @@ -36,7 +36,7 @@ KafkaProducer::KafkaProducer( if (header.has("_timestamp")) { auto column_index = header.getPositionByName("_timestamp"); - auto column_info = header.getByPosition(column_index); + const auto & column_info = header.getByPosition(column_index); if (isDateTime(column_info.type)) timestamp_column_index = column_index; } diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 75f2fd26600..3716ee08bed 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -95,7 +95,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( context_) , WithContext(context_) , prefetch_threadpool(getContext()->getPrefetchThreadpool()) - , log(&Poco::Logger::get("MergeTreePrefetchedReadPool(" + (parts_.empty() ? "" : parts_.front().data_part->storage.getStorageID().getNameForLogs()) + ")")) + , log(&Poco::Logger::get("MergeTreePrefetchedReadPool(" + (parts_ranges.empty() ? "" : parts_ranges.front().data_part->storage.getStorageID().getNameForLogs()) + ")")) { /// Tasks creation might also create a lost of readers - check they do not /// do any time consuming operations in ctor. diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 272f35303bd..c14abfc9ab2 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -490,7 +490,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu replaceAliasColumnsInQuery(column_expr, storage_metadata_snapshot->getColumns(), syntax_result->array_join_result_to_source, context); - auto column_description = storage_columns.get(column); + const auto & column_description = storage_columns.get(column); column_expr = addTypeConversionToAST(std::move(column_expr), column_description.type->getName(), storage_metadata_snapshot->getColumns().getAll(), context); column_expr = setAlias(column_expr, column); diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index f86d2b44dd7..12f4c87c5ae 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -336,7 +336,7 @@ void Runner::runBenchmark() for (size_t i = 0; i < concurrency; ++i) { auto thread_connections = connections; - pool->scheduleOrThrowOnError([this, connections = std::move(thread_connections)]() mutable { thread(connections); }); + pool->scheduleOrThrowOnError([this, connections_ = std::move(thread_connections)]() mutable { thread(connections_); }); } } catch (...)