From e5e84419aff0f559bc545737bfdc0518a732f7ff Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 10 Mar 2024 14:29:18 +0000 Subject: [PATCH] Fix clang-tidy-s --- contrib/libmetrohash/src/metrohash128.h | 3 +++ src/Access/AccessControl.h | 10 +++++----- src/Access/IAccessStorage.cpp | 2 +- src/Access/IAccessStorage.h | 2 +- src/Common/Arena.h | 4 +--- src/Common/DNSResolver.cpp | 2 +- src/Common/DNSResolver.h | 2 +- src/Common/DateLUTImpl.h | 2 +- src/Common/MultiVersion.h | 4 ++-- src/Common/PODArray.h | 6 +++--- src/Common/SipHash.h | 2 +- src/Common/TransactionID.h | 2 +- src/Common/ZooKeeper/IKeeper.cpp | 8 ++++---- src/Common/ZooKeeper/IKeeper.h | 16 ++++++++-------- src/Common/logger_useful.h | 16 ++++++++-------- src/Core/PostgreSQL/insertPostgreSQLValue.cpp | 4 ++-- src/Core/PostgreSQL/insertPostgreSQLValue.h | 4 ++-- src/Core/Settings.h | 2 ++ src/Dictionaries/CacheDictionary.cpp | 4 ++-- src/Dictionaries/CacheDictionary.h | 2 +- .../GeodataProviders/IHierarchiesProvider.h | 2 +- src/Dictionaries/RegExpTreeDictionary.cpp | 2 +- src/Dictionaries/RegExpTreeDictionary.h | 2 +- src/Functions/IFunction.h | 4 ---- src/IO/ReadSettings.h | 1 + src/Interpreters/AsynchronousInsertQueue.cpp | 2 +- src/Interpreters/AsynchronousInsertQueue.h | 2 +- src/Interpreters/Context.h | 4 ++-- src/Interpreters/IExternalLoadable.h | 2 +- src/Interpreters/ProcessList.h | 2 +- src/Processors/Chunk.h | 2 +- .../Algorithms/AggregatingSortedAlgorithm.cpp | 2 +- src/Processors/Port.h | 2 +- src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp | 4 ++-- src/Processors/TTL/TTLUpdateInfoAlgorithm.h | 4 ++-- src/Storages/StorageInMemoryMetadata.h | 4 ++-- 36 files changed, 69 insertions(+), 69 deletions(-) diff --git a/contrib/libmetrohash/src/metrohash128.h b/contrib/libmetrohash/src/metrohash128.h index 2dbb6ca5a8a..f507c917caf 100644 --- a/contrib/libmetrohash/src/metrohash128.h +++ b/contrib/libmetrohash/src/metrohash128.h @@ -17,6 +17,8 @@ #ifndef METROHASH_METROHASH_128_H #define METROHASH_METROHASH_128_H +// NOLINTBEGIN(readability-avoid-const-params-in-decls) + #include class MetroHash128 @@ -68,5 +70,6 @@ private: void metrohash128_1(const uint8_t * key, uint64_t len, uint32_t seed, uint8_t * out); void metrohash128_2(const uint8_t * key, uint64_t len, uint32_t seed, uint8_t * out); +// NOLINTEND(readability-avoid-const-params-in-decls) #endif // #ifndef METROHASH_METROHASH_128_H diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 55ea4e4f717..1af74e02fb7 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -133,20 +133,20 @@ public: /// This function also enables custom prefixes to be used. void setCustomSettingsPrefixes(const Strings & prefixes); void setCustomSettingsPrefixes(const String & comma_separated_prefixes); - bool isSettingNameAllowed(const std::string_view name) const; - void checkSettingNameIsAllowed(const std::string_view name) const; + bool isSettingNameAllowed(std::string_view name) const; + void checkSettingNameIsAllowed(std::string_view name) const; /// Allows implicit user creation without password (by default it's allowed). /// In other words, allow 'CREATE USER' queries without 'IDENTIFIED WITH' clause. - void setImplicitNoPasswordAllowed(const bool allow_implicit_no_password_); + void setImplicitNoPasswordAllowed(bool allow_implicit_no_password_); bool isImplicitNoPasswordAllowed() const; /// Allows users without password (by default it's allowed). - void setNoPasswordAllowed(const bool allow_no_password_); + void setNoPasswordAllowed(bool allow_no_password_); bool isNoPasswordAllowed() const; /// Allows users with plaintext password (by default it's allowed). - void setPlaintextPasswordAllowed(const bool allow_plaintext_password_); + void setPlaintextPasswordAllowed(bool allow_plaintext_password_); bool isPlaintextPasswordAllowed() const; /// Default password type when the user does not specify it. diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index fbe9e231002..1d6b8d99cd5 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -616,7 +616,7 @@ UUID IAccessStorage::generateRandomID() } -void IAccessStorage::clearConflictsInEntitiesList(std::vector> & entities, const LoggerPtr log_) +void IAccessStorage::clearConflictsInEntitiesList(std::vector> & entities, LoggerPtr log_) { std::unordered_map positions_by_id; std::unordered_map positions_by_type_and_name[static_cast(AccessEntityType::MAX)]; diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index ebb5a39cdf0..ad78bf92e02 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -228,7 +228,7 @@ protected: static UUID generateRandomID(); LoggerPtr getLogger() const; static String formatEntityTypeWithName(AccessEntityType type, const String & name) { return AccessEntityTypeInfo::get(type).formatEntityNameWithType(name); } - static void clearConflictsInEntitiesList(std::vector> & entities, const LoggerPtr log_); + static void clearConflictsInEntitiesList(std::vector> & entities, LoggerPtr log_); [[noreturn]] void throwNotFound(const UUID & id) const; [[noreturn]] void throwNotFound(AccessEntityType type, const String & name) const; [[noreturn]] static void throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type); diff --git a/src/Common/Arena.h b/src/Common/Arena.h index cb26397844b..ba5b9ea9205 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -47,9 +47,7 @@ private: std::unique_ptr prev; - MemoryChunk() - { - } + MemoryChunk() = default; void swap(MemoryChunk & other) { diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index e36e1483da8..4b577a251af 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -297,7 +297,7 @@ void DNSResolver::setDisableCacheFlag(bool is_disabled) impl->disable_cache = is_disabled; } -void DNSResolver::setCacheMaxEntries(const UInt64 cache_max_entries) +void DNSResolver::setCacheMaxEntries(UInt64 cache_max_entries) { impl->cache_address.setMaxSizeInBytes(cache_max_entries); impl->cache_host.setMaxSizeInBytes(cache_max_entries); diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index e3030e51a96..1ddd9d3b991 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -56,7 +56,7 @@ public: void setDisableCacheFlag(bool is_disabled = true); /// Set a limit of entries in cache - void setCacheMaxEntries(const UInt64 cache_max_entries); + void setCacheMaxEntries(UInt64 cache_max_entries); /// Drops all caches void dropCache(); diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 7bf66c0504a..4087e77d588 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -255,7 +255,7 @@ private: static LUTIndex toLUTIndex(ExtendedDayNum d) { - return normalizeLUTIndex(static_cast(d + daynum_offset_epoch)); + return normalizeLUTIndex(static_cast(d) + daynum_offset_epoch); } LUTIndex toLUTIndex(Time t) const diff --git a/src/Common/MultiVersion.h b/src/Common/MultiVersion.h index 8f488f9fcbc..680e224f869 100644 --- a/src/Common/MultiVersion.h +++ b/src/Common/MultiVersion.h @@ -41,9 +41,9 @@ public: } /// There is no copy constructor because only one MultiVersion should own the same object. - MultiVersion(MultiVersion && src) { *this = std::move(src); } + MultiVersion(MultiVersion && src) { *this = std::move(src); } /// NOLINT - MultiVersion & operator=(MultiVersion && src) + MultiVersion & operator=(MultiVersion && src) /// NOLINT { if (this != &src) { diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 1a4047a2588..af863e01fb2 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -25,7 +25,7 @@ */ template constexpr bool memcpy_can_be_used_for_assignment = std::is_same_v - || (std::is_integral_v && std::is_integral_v && sizeof(T) == sizeof(U)); + || (std::is_integral_v && std::is_integral_v && sizeof(T) == sizeof(U)); /// NOLINT(misc-redundant-expression) namespace DB { @@ -558,7 +558,7 @@ public: } template - void swap(PODArray & rhs, TAllocatorParams &&... allocator_params) + void swap(PODArray & rhs, TAllocatorParams &&... allocator_params) /// NOLINT(performance-noexcept-swap) { #ifndef NDEBUG this->unprotect(); @@ -756,7 +756,7 @@ public: }; template -void swap(PODArray & lhs, PODArray & rhs) +void swap(PODArray & lhs, PODArray & rhs) /// NOLINT { lhs.swap(rhs); } diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 729fb76a573..c89ee2c9d90 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -149,7 +149,7 @@ public: /// Pad the remainder, which is missing up to an 8-byte word. current_word = 0; - switch (end - data) + switch (end - data) /// NOLINT(bugprone-switch-missing-default-case) { case 7: current_bytes[CURRENT_BYTES_IDX(6)] = data[6]; [[fallthrough]]; case 6: current_bytes[CURRENT_BYTES_IDX(5)] = data[5]; [[fallthrough]]; diff --git a/src/Common/TransactionID.h b/src/Common/TransactionID.h index 3ab86f7589c..97d0072bc14 100644 --- a/src/Common/TransactionID.h +++ b/src/Common/TransactionID.h @@ -16,7 +16,7 @@ class MergeTreeTransaction; /// or transaction object is not needed and not passed intentionally. #ifndef NO_TRANSACTION_PTR #define NO_TRANSACTION_PTR std::shared_ptr(nullptr) -#define NO_TRANSACTION_RAW static_cast(nullptr) +#define NO_TRANSACTION_RAW static_cast(nullptr) /// NOLINT(bugprone-macro-parentheses) #endif /// Commit Sequence Number diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 6c47ea68b84..7d2602bde1e 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -23,7 +23,7 @@ namespace ProfileEvents namespace Coordination { -void Exception::incrementErrorMetrics(const Error code_) +void Exception::incrementErrorMetrics(Error code_) { if (Coordination::isUserError(code_)) ProfileEvents::increment(ProfileEvents::ZooKeeperUserExceptions); @@ -33,14 +33,14 @@ void Exception::incrementErrorMetrics(const Error code_) ProfileEvents::increment(ProfileEvents::ZooKeeperOtherExceptions); } -Exception::Exception(const std::string & msg, const Error code_, int) +Exception::Exception(const std::string & msg, Error code_, int) : DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION) , code(code_) { incrementErrorMetrics(code); } -Exception::Exception(PreformattedMessage && msg, const Error code_) +Exception::Exception(PreformattedMessage && msg, Error code_) : DB::Exception(std::move(msg), DB::ErrorCodes::KEEPER_EXCEPTION) , code(code_) { @@ -48,7 +48,7 @@ Exception::Exception(PreformattedMessage && msg, const Error code_) incrementErrorMetrics(code); } -Exception::Exception(const Error code_) +Exception::Exception(Error code_) : Exception(code_, "Coordination error: {}", errorMessage(code_)) { } diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index c7b902ea03a..ec49c94808e 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -466,13 +466,13 @@ class Exception : public DB::Exception { private: /// Delegate constructor, used to minimize repetition; last parameter used for overload resolution. - Exception(const std::string & msg, const Error code_, int); /// NOLINT - Exception(PreformattedMessage && msg, const Error code_); + Exception(const std::string & msg, Error code_, int); /// NOLINT + Exception(PreformattedMessage && msg, Error code_); /// Message must be a compile-time constant template requires std::is_convertible_v - Exception(T && message, const Error code_) : DB::Exception(std::forward(message), DB::ErrorCodes::KEEPER_EXCEPTION, /* remote_= */ false), code(code_) + Exception(T && message, Error code_) : DB::Exception(std::forward(message), DB::ErrorCodes::KEEPER_EXCEPTION, /* remote_= */ false), code(code_) { incrementErrorMetrics(code); } @@ -480,23 +480,23 @@ private: static void incrementErrorMetrics(Error code_); public: - explicit Exception(const Error code_); /// NOLINT + explicit Exception(Error code_); /// NOLINT Exception(const Exception & exc); template - Exception(const Error code_, FormatStringHelper fmt, Args &&... args) + Exception(Error code_, FormatStringHelper fmt, Args &&... args) : DB::Exception(DB::ErrorCodes::KEEPER_EXCEPTION, std::move(fmt), std::forward(args)...) , code(code_) { incrementErrorMetrics(code); } - inline static Exception createDeprecated(const std::string & msg, const Error code_) + inline static Exception createDeprecated(const std::string & msg, Error code_) { return Exception(msg, code_, 0); } - inline static Exception fromPath(const Error code_, const std::string & path) + inline static Exception fromPath(Error code_, const std::string & path) { return Exception(code_, "Coordination error: {}, path {}", errorMessage(code_), path); } @@ -504,7 +504,7 @@ public: /// Message must be a compile-time constant template requires std::is_convertible_v - inline static Exception fromMessage(const Error code_, T && message) + inline static Exception fromMessage(Error code_, T && message) { return Exception(std::forward(message), code_); } diff --git a/src/Common/logger_useful.h b/src/Common/logger_useful.h index 8e78e93e198..013b35e695e 100644 --- a/src/Common/logger_useful.h +++ b/src/Common/logger_useful.h @@ -19,14 +19,14 @@ namespace Poco { class Logger; } using LogSeriesLimiterPtr = std::shared_ptr; -namespace +namespace impl { - [[maybe_unused]] LoggerPtr getLoggerHelper(const LoggerPtr & logger) { return logger; } - [[maybe_unused]] LoggerPtr getLoggerHelper(const AtomicLogger & logger) { return logger.load(); } - [[maybe_unused]] const ::Poco::Logger * getLoggerHelper(const ::Poco::Logger * logger) { return logger; } - [[maybe_unused]] std::unique_ptr getLoggerHelper(std::unique_ptr && logger) { return logger; } - [[maybe_unused]] std::unique_ptr getLoggerHelper(std::unique_ptr && logger) { return logger; } - [[maybe_unused]] LogSeriesLimiterPtr getLoggerHelper(LogSeriesLimiterPtr & logger) { return logger; } + [[maybe_unused]] inline LoggerPtr getLoggerHelper(const LoggerPtr & logger) { return logger; } + [[maybe_unused]] inline LoggerPtr getLoggerHelper(const AtomicLogger & logger) { return logger.load(); } + [[maybe_unused]] inline const ::Poco::Logger * getLoggerHelper(const ::Poco::Logger * logger) { return logger; } + [[maybe_unused]] inline std::unique_ptr getLoggerHelper(std::unique_ptr && logger) { return logger; } + [[maybe_unused]] inline std::unique_ptr getLoggerHelper(std::unique_ptr && logger) { return logger; } + [[maybe_unused]] inline LogSeriesLimiterPtr getLoggerHelper(LogSeriesLimiterPtr & logger) { return logger; } } #define LOG_IMPL_FIRST_ARG(X, ...) X @@ -65,7 +65,7 @@ namespace #define LOG_IMPL(logger, priority, PRIORITY, ...) do \ { \ - auto _logger = ::getLoggerHelper(logger); \ + auto _logger = ::impl::getLoggerHelper(logger); \ const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ (DB::CurrentThread::get().getClientLogsLevel() >= (priority)); \ if (!_is_clients_log && !_logger->is((PRIORITY))) \ diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index aa60bdee28a..b507b300769 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -36,7 +36,7 @@ void insertDefaultPostgreSQLValue(IColumn & column, const IColumn & sample_colum void insertPostgreSQLValue( IColumn & column, std::string_view value, - const ExternalResultDescription::ValueType type, const DataTypePtr data_type, + ExternalResultDescription::ValueType type, DataTypePtr data_type, const std::unordered_map & array_info, size_t idx) { switch (type) @@ -170,7 +170,7 @@ void insertPostgreSQLValue( void preparePostgreSQLArrayInfo( - std::unordered_map & array_info, size_t column_idx, const DataTypePtr data_type) + std::unordered_map & array_info, size_t column_idx, DataTypePtr data_type) { const auto * array_type = typeid_cast(data_type.get()); auto nested = array_type->getNestedType(); diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.h b/src/Core/PostgreSQL/insertPostgreSQLValue.h index 3bc83292b96..bfb85422aa1 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.h +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.h @@ -22,11 +22,11 @@ struct PostgreSQLArrayInfo void insertPostgreSQLValue( IColumn & column, std::string_view value, - const ExternalResultDescription::ValueType type, const DataTypePtr data_type, + ExternalResultDescription::ValueType type, DataTypePtr data_type, const std::unordered_map & array_info, size_t idx); void preparePostgreSQLArrayInfo( - std::unordered_map & array_info, size_t column_idx, const DataTypePtr data_type); + std::unordered_map & array_info, size_t column_idx, DataTypePtr data_type); void insertDefaultPostgreSQLValue(IColumn & column, const IColumn & sample_column); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d70a6cf51c5..c41db9d2141 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1192,6 +1192,7 @@ class IColumn; FORMAT_FACTORY_SETTINGS(M, ALIAS) \ OBSOLETE_FORMAT_SETTINGS(M, ALIAS) \ +/// NOLINTNEXTLINE(clang-analyzer-optin.performance.Padding) DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(SettingsTraits, LIST_OF_SETTINGS) @@ -1236,6 +1237,7 @@ private: /* * User-specified file format settings for File and URL engines. */ +/// NOLINTNEXTLINE(clang-analyzer-optin.performance.Padding) DECLARE_SETTINGS_TRAITS(FormatFactorySettingsTraits, LIST_OF_ALL_FORMAT_SETTINGS) struct FormatFactorySettings : public BaseSettings diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 8444042db9e..6e9b09f8919 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -423,7 +423,7 @@ MutableColumns CacheDictionary::aggregateColumnsInOrderOfKe const DictionaryStorageFetchRequest & request, const MutableColumns & fetched_columns, const PaddedPODArray & key_index_to_state, - IColumn::Filter * const default_mask) const + IColumn::Filter * default_mask) const { MutableColumns aggregated_columns = request.makeAttributesResultColumns(); @@ -473,7 +473,7 @@ MutableColumns CacheDictionary::aggregateColumns( const PaddedPODArray & key_index_to_fetched_columns_from_storage_result, const MutableColumns & fetched_columns_during_update, const HashMap & found_keys_to_fetched_columns_during_update_index, - IColumn::Filter * const default_mask) const + IColumn::Filter * default_mask) const { /** * Aggregation of columns fetched from storage and from source during update. diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index 8897fb40fa9..c02fb91c60e 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -162,7 +162,7 @@ private: const DictionaryStorageFetchRequest & request, const MutableColumns & fetched_columns, const PaddedPODArray & key_index_to_state, - IColumn::Filter * const default_mask = nullptr) const; + IColumn::Filter * default_mask = nullptr) const; MutableColumns aggregateColumns( const PaddedPODArray & keys, diff --git a/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h b/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h index 68ab0fdca2d..a4b88127786 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h +++ b/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h @@ -14,7 +14,7 @@ class IRegionsHierarchyReader public: virtual bool readNext(RegionEntry & entry) = 0; - virtual ~IRegionsHierarchyReader() {} + virtual ~IRegionsHierarchyReader() = default; }; using IRegionsHierarchyReaderPtr = std::unique_ptr; diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 4d82aa9ca0e..1f5c2d6d2c7 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -568,7 +568,7 @@ bool RegExpTreeDictionary::setAttributesShortCircuit( const String & data, std::unordered_set & visited_nodes, const std::unordered_map & attributes, - std::unordered_set * const defaults) const + std::unordered_set * defaults) const { if (visited_nodes.contains(id)) return attributes_to_set.attributesFull() == attributes.size(); diff --git a/src/Dictionaries/RegExpTreeDictionary.h b/src/Dictionaries/RegExpTreeDictionary.h index 9e14abb49d0..d6bc90ef651 100644 --- a/src/Dictionaries/RegExpTreeDictionary.h +++ b/src/Dictionaries/RegExpTreeDictionary.h @@ -210,7 +210,7 @@ private: const String & data, std::unordered_set & visited_nodes, const std::unordered_map & attributes, - std::unordered_set * const defaults) const; + std::unordered_set * defaults) const; struct RegexTreeNode; using RegexTreeNodePtr = std::shared_ptr; diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 05aa08e2ad7..9b7cdf12d57 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -13,10 +13,6 @@ #include -#if USE_EMBEDDED_COMPILER -# include -#endif - /// This file contains user interface for functions. namespace llvm diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 38904df4403..5c401c0c8d9 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -63,6 +63,7 @@ enum class RemoteFSReadMethod class MMappedFileCache; class PageCache; +/// NOLINTNEXTLINE(clang-analyzer-optin.performance.Padding) struct ReadSettings { /// Method to use reading from local filesystem. diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 7d56dbabe3c..9327f31b6ff 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -905,7 +905,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( const InsertDataPtr & data, const Block & header, const ContextPtr & insert_context, - const LoggerPtr logger, + LoggerPtr logger, LogFunc && add_to_async_insert_log) { size_t total_rows = 0; diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index f60b3d343fb..5076701d0b0 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -265,7 +265,7 @@ private: const InsertDataPtr & data, const Block & header, const ContextPtr & insert_context, - const LoggerPtr logger, + LoggerPtr logger, LogFunc && add_to_async_insert_log); template diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c8aa3604a6f..43df8d6adf2 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -330,7 +330,7 @@ protected: return *this; } - void swap(QueryAccessInfo & rhs) + void swap(QueryAccessInfo & rhs) noexcept { std::swap(databases, rhs.databases); std::swap(tables, rhs.tables); @@ -680,7 +680,7 @@ public: void addSpecialScalar(const String & name, const Block & block); const QueryAccessInfo & getQueryAccessInfo() const { return *getQueryAccessInfoPtr(); } - const QueryAccessInfoPtr getQueryAccessInfoPtr() const { return query_access_info; } + QueryAccessInfoPtr getQueryAccessInfoPtr() const { return query_access_info; } void setQueryAccessInfo(QueryAccessInfoPtr other) { query_access_info = other; } void addQueryAccessInfo( diff --git a/src/Interpreters/IExternalLoadable.h b/src/Interpreters/IExternalLoadable.h index 3c004508b0a..47031778876 100644 --- a/src/Interpreters/IExternalLoadable.h +++ b/src/Interpreters/IExternalLoadable.h @@ -23,7 +23,7 @@ struct ExternalLoadableLifetime UInt64 max_sec = 0; ExternalLoadableLifetime(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); - ExternalLoadableLifetime() {} + ExternalLoadableLifetime() = default; }; /// Get delay before trying to load again after error. diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 1c253f562e8..ad47041c762 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -318,7 +318,7 @@ public: ~ProcessListEntry(); QueryStatusPtr getQueryStatus() { return *it; } - const QueryStatusPtr getQueryStatus() const { return *it; } + QueryStatusPtr getQueryStatus() const { return *it; } }; diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 9a7d6bc294d..4f753798eaa 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -59,7 +59,7 @@ public: Chunk clone() const; - void swap(Chunk & other) + void swap(Chunk & other) noexcept { columns.swap(other.columns); chunk_info.swap(other.chunk_info); diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp index d2d2434c477..3bd0b532d90 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp @@ -126,7 +126,7 @@ static void postprocessChunk(Chunk & chunk, const AggregatingSortedAlgorithm::Co AggregatingSortedAlgorithm::SimpleAggregateDescription::SimpleAggregateDescription( - AggregateFunctionPtr function_, const size_t column_number_, + AggregateFunctionPtr function_, size_t column_number_, DataTypePtr nested_type_, DataTypePtr real_type_) : function(std::move(function_)), column_number(column_number_) , nested_type(std::move(nested_type_)), real_type(std::move(real_type_)) diff --git a/src/Processors/Port.h b/src/Processors/Port.h index 67af2f041aa..f3c7bbb5fee 100644 --- a/src/Processors/Port.h +++ b/src/Processors/Port.h @@ -110,7 +110,7 @@ protected: return result; } - uintptr_t ALWAYS_INLINE swap(std::atomic & value, std::uintptr_t flags, std::uintptr_t mask) + uintptr_t ALWAYS_INLINE swap(std::atomic & value, std::uintptr_t flags, std::uintptr_t mask) /// NOLINT { Data * expected = nullptr; Data * desired = getPtr(flags | getUInt(data)); diff --git a/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp index b7cddf3c165..13d3030bbb8 100644 --- a/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp +++ b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp @@ -6,8 +6,8 @@ namespace DB TTLUpdateInfoAlgorithm::TTLUpdateInfoAlgorithm( const TTLExpressions & ttl_expressions_, const TTLDescription & description_, - const TTLUpdateField ttl_update_field_, - const String ttl_update_key_, + TTLUpdateField ttl_update_field_, + String ttl_update_key_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) diff --git a/src/Processors/TTL/TTLUpdateInfoAlgorithm.h b/src/Processors/TTL/TTLUpdateInfoAlgorithm.h index 0cf31765aef..b6aee6f7cb0 100644 --- a/src/Processors/TTL/TTLUpdateInfoAlgorithm.h +++ b/src/Processors/TTL/TTLUpdateInfoAlgorithm.h @@ -22,8 +22,8 @@ public: TTLUpdateInfoAlgorithm( const TTLExpressions & ttl_expressions_, const TTLDescription & description_, - const TTLUpdateField ttl_update_field_, - const String ttl_update_key_, + TTLUpdateField ttl_update_field_, + String ttl_update_key_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_ ); diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 2823aba1224..69cd3422a7d 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -72,8 +72,8 @@ struct StorageInMemoryMetadata StorageInMemoryMetadata(const StorageInMemoryMetadata & other); StorageInMemoryMetadata & operator=(const StorageInMemoryMetadata & other); - StorageInMemoryMetadata(StorageInMemoryMetadata && other) = default; - StorageInMemoryMetadata & operator=(StorageInMemoryMetadata && other) = default; + StorageInMemoryMetadata(StorageInMemoryMetadata && other) = default; /// NOLINT + StorageInMemoryMetadata & operator=(StorageInMemoryMetadata && other) = default; /// NOLINT /// NOTE: Thread unsafe part. You should not modify same StorageInMemoryMetadata /// structure from different threads. It should be used as MultiVersion