diff --git a/programs/library-bridge/LibraryBridgeHandlers.cpp b/programs/library-bridge/LibraryBridgeHandlers.cpp index 26d887cfc98..8d116e537aa 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.cpp +++ b/programs/library-bridge/LibraryBridgeHandlers.cpp @@ -284,7 +284,6 @@ void ExternalDictionaryLibraryBridgeRequestHandler::handleRequest(HTTPServerRequ else if (method == "extDict_loadIds") { LOG_DEBUG(log, "Getting diciontary ids for dictionary with id: {}", dictionary_id); - String ids_string; std::vector ids = parseIdsFromBinary(request.getStream()); auto library_handler = ExternalDictionaryLibraryHandlerFactory::instance().get(dictionary_id); diff --git a/programs/library-bridge/SharedLibrary.cpp b/programs/library-bridge/SharedLibrary.cpp index d70709474b5..7423f9b89f4 100644 --- a/programs/library-bridge/SharedLibrary.cpp +++ b/programs/library-bridge/SharedLibrary.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes SharedLibrary::SharedLibrary(std::string_view path, int flags) { - handle = dlopen(path.data(), flags); + handle = dlopen(path.data(), flags); // NOLINT if (!handle) throw Exception(ErrorCodes::CANNOT_DLOPEN, "Cannot dlopen: ({})", dlerror()); // NOLINT(concurrency-mt-unsafe) // MT-Safe on Linux, see man dlerror @@ -34,7 +34,7 @@ void * SharedLibrary::getImpl(std::string_view name, bool no_throw) { dlerror(); // NOLINT(concurrency-mt-unsafe) // MT-Safe on Linux, see man dlerror - auto * res = dlsym(handle, name.data()); + auto * res = dlsym(handle, name.data()); // NOLINT if (char * error = dlerror()) // NOLINT(concurrency-mt-unsafe) // MT-Safe on Linux, see man dlerror { diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index b2bf942af4e..8035f053b41 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -674,8 +674,7 @@ private: if (pos + length > end) length = end - pos; - if (length > sizeof(CodePoint)) - length = sizeof(CodePoint); + length = std::min(length, sizeof(CodePoint)); CodePoint res = 0; memcpy(&res, pos, length); @@ -883,9 +882,7 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error in markov model"); size_t offset_from_begin_of_string = pos - data; - size_t determinator_sliding_window_size = params.determinator_sliding_window_size; - if (determinator_sliding_window_size > determinator_size) - determinator_sliding_window_size = determinator_size; + size_t determinator_sliding_window_size = std::min(params.determinator_sliding_window_size, determinator_size); size_t determinator_sliding_window_overflow = offset_from_begin_of_string + determinator_sliding_window_size > determinator_size ? offset_from_begin_of_string + determinator_sliding_window_size - determinator_size : 0; diff --git a/programs/odbc-bridge/ODBCSource.cpp b/programs/odbc-bridge/ODBCSource.cpp index 7f0d47f7e2e..940970f36ab 100644 --- a/programs/odbc-bridge/ODBCSource.cpp +++ b/programs/odbc-bridge/ODBCSource.cpp @@ -119,8 +119,7 @@ void ODBCSource::insertValue( time_t time = 0; const DataTypeDateTime & datetime_type = assert_cast(*data_type); readDateTimeText(time, in, datetime_type.getTimeZone()); - if (time < 0) - time = 0; + time = std::max(time, 0); column.insert(static_cast(time)); break; } diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 835f414df37..8d54265ec39 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -245,7 +245,7 @@ bool AccessRightsElements::sameOptions() const void AccessRightsElements::eraseNonGrantable() { - boost::range::remove_erase_if(*this, [](AccessRightsElement & element) + boost::range::remove_erase_if(*this, [](AccessRightsElement & element) // NOLINT { element.eraseNonGrantable(); return element.empty(); diff --git a/src/Access/Common/AllowedClientHosts.cpp b/src/Access/Common/AllowedClientHosts.cpp index bee0cdd7264..d5e2200811e 100644 --- a/src/Access/Common/AllowedClientHosts.cpp +++ b/src/Access/Common/AllowedClientHosts.cpp @@ -308,7 +308,7 @@ void AllowedClientHosts::removeAddress(const IPAddress & address) if (address.isLoopback()) local_host = false; else - boost::range::remove_erase(addresses, address); + boost::range::remove_erase(addresses, address); // NOLINT } void AllowedClientHosts::addSubnet(const IPSubnet & subnet) @@ -328,7 +328,7 @@ void AllowedClientHosts::removeSubnet(const IPSubnet & subnet) else if (subnet.isMaskAllBitsOne()) removeAddress(subnet.getPrefix()); else - boost::range::remove_erase(subnets, subnet); + boost::range::remove_erase(subnets, subnet); // NOLINT } void AllowedClientHosts::addName(const String & name) @@ -344,7 +344,7 @@ void AllowedClientHosts::removeName(const String & name) if (boost::iequals(name, "localhost")) local_host = false; else - boost::range::remove_erase(names, name); + boost::range::remove_erase(names, name); // NOLINT } void AllowedClientHosts::addNameRegexp(const String & name_regexp) @@ -364,7 +364,7 @@ void AllowedClientHosts::removeNameRegexp(const String & name_regexp) else if (name_regexp == ".*") any_host = false; else - boost::range::remove_erase(name_regexps, name_regexp); + boost::range::remove_erase(name_regexps, name_regexp); // NOLINT } void AllowedClientHosts::addLikePattern(const String & pattern) @@ -384,7 +384,7 @@ void AllowedClientHosts::removeLikePattern(const String & pattern) else if ((pattern == "%") || (pattern == "0.0.0.0/0") || (pattern == "::/0")) any_host = false; else - boost::range::remove_erase(like_patterns, pattern); + boost::range::remove_erase(like_patterns, pattern); // NOLINT } void AllowedClientHosts::addLocalHost() diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index 979c3f2af15..9d84f079daa 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -234,8 +234,7 @@ public: BetterFloat qr = (sum + l_count + r->count * 0.5) / count; BetterFloat err2 = qr * (1 - qr); - if (err > err2) - err = err2; + err = std::min(err, err2); BetterFloat k = count_epsilon_4 * err; diff --git a/src/Common/ArenaWithFreeLists.h b/src/Common/ArenaWithFreeLists.h index ddfe2fa1ebd..29cf975efdf 100644 --- a/src/Common/ArenaWithFreeLists.h +++ b/src/Common/ArenaWithFreeLists.h @@ -132,7 +132,7 @@ public: void free(char * ptr, const size_t size) { std::lock_guard lock{mutex}; - return ArenaWithFreeLists::free(ptr, size); + ArenaWithFreeLists::free(ptr, size); } /// Size of the allocated pool in bytes diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index 341e571e4eb..392ee64dcbf 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -253,8 +253,7 @@ namespace cctz_extension size_t Read(void * buf, size_t bytes) override { - if (bytes > size) - bytes = size; + bytes = std::min(bytes, size); memcpy(buf, data, bytes); data += bytes; size -= bytes; diff --git a/src/Common/FST.h b/src/Common/FST.h index e2a1cbb20cb..e5dcea2c8d8 100644 --- a/src/Common/FST.h +++ b/src/Common/FST.h @@ -11,6 +11,7 @@ #include #include + namespace DB { /// Finite State Transducer is an efficient way to represent term dictionary. diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index f104fea72cb..a26797a687a 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -291,7 +291,7 @@ public: * the compiler can not guess about this, and generates the `load`, `increment`, `store` code. */ if (inserted) - new (&it->getMapped()) typename Cell::Mapped(); + new (reinterpret_cast(&it->getMapped())) typename Cell::Mapped(); return it->getMapped(); } diff --git a/src/Common/Jemalloc.cpp b/src/Common/Jemalloc.cpp index 6514639e700..fbe2f62c944 100644 --- a/src/Common/Jemalloc.cpp +++ b/src/Common/Jemalloc.cpp @@ -67,7 +67,7 @@ std::string flushJemallocProfile(const std::string & file_prefix) checkJemallocProfilingEnabled(); char * prefix_buffer; size_t prefix_size = sizeof(prefix_buffer); - int n = mallctl("opt.prof_prefix", &prefix_buffer, &prefix_size, nullptr, 0); + int n = mallctl("opt.prof_prefix", &prefix_buffer, &prefix_size, nullptr, 0); // NOLINT if (!n && std::string_view(prefix_buffer) != "jeprof") { LOG_TRACE(getLogger("SystemJemalloc"), "Flushing memory profile with prefix {}", prefix_buffer); @@ -80,7 +80,7 @@ std::string flushJemallocProfile(const std::string & file_prefix) const auto * profile_dump_path_str = profile_dump_path.c_str(); LOG_TRACE(getLogger("SystemJemalloc"), "Flushing memory profile to {}", profile_dump_path_str); - mallctl("prof.dump", nullptr, nullptr, &profile_dump_path_str, sizeof(profile_dump_path_str)); + mallctl("prof.dump", nullptr, nullptr, &profile_dump_path_str, sizeof(profile_dump_path_str)); // NOLINT return profile_dump_path; } diff --git a/src/Common/LoggingFormatStringHelpers.cpp b/src/Common/LoggingFormatStringHelpers.cpp index 3e90526f76d..4cf57807b0b 100644 --- a/src/Common/LoggingFormatStringHelpers.cpp +++ b/src/Common/LoggingFormatStringHelpers.cpp @@ -27,7 +27,7 @@ void LogFrequencyLimiterIml::log(Poco::Message & message) SipHash hash; hash.update(logger->name()); /// Format strings are compile-time constants, so they are uniquely identified by pointer and size - hash.update(pattern.data()); + hash.update(reinterpret_cast(pattern.data())); hash.update(pattern.size()); time_t now = time(nullptr); diff --git a/src/Common/NetlinkMetricsProvider.cpp b/src/Common/NetlinkMetricsProvider.cpp index 172fede525a..b12d81df45c 100644 --- a/src/Common/NetlinkMetricsProvider.cpp +++ b/src/Common/NetlinkMetricsProvider.cpp @@ -112,7 +112,7 @@ struct NetlinkMessage if (bytes_sent <= 0) { - if (errno == EAGAIN) + if (bytes_sent < 0 && errno == EAGAIN) continue; else throw ErrnoException(ErrorCodes::NETLINK_ERROR, "Can't send a Netlink command"); diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index b6852964efe..712cab80aff 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -40,8 +40,7 @@ size_t shortest_literal_length(const Literals & literals) if (literals.empty()) return 0; size_t shortest = std::numeric_limits::max(); for (const auto & lit : literals) - if (shortest > lit.literal.size()) - shortest = lit.literal.size(); + shortest = std::min(shortest, lit.literal.size()); return shortest; } @@ -451,7 +450,7 @@ try { Literals alternative_literals; Literal required_literal; - analyzeImpl(regexp_, regexp_.data(), required_literal, is_trivial, alternative_literals); + analyzeImpl(regexp_, regexp_.data(), required_literal, is_trivial, alternative_literals); // NOLINT required_substring = std::move(required_literal.literal); required_substring_is_prefix = required_literal.prefix; for (auto & lit : alternative_literals) @@ -649,8 +648,7 @@ unsigned OptimizedRegularExpression::match(const char * subject, size_t subject_ if (limit == 0) return 0; - if (limit > number_of_subpatterns + 1) - limit = number_of_subpatterns + 1; + limit = std::min(limit, number_of_subpatterns + 1); if (is_trivial) { diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 8365e818840..2359137012c 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -228,8 +228,7 @@ PoolWithFailoverBase::getMany( std::vector shuffled_pools = getShuffledPools(max_ignored_errors, get_priority); /// Limit `max_tries` value by `max_error_cap` to avoid unlimited number of retries - if (max_tries > max_error_cap) - max_tries = max_error_cap; + max_tries = std::min(max_tries, max_error_cap); /// We will try to get a connection from each pool until a connection is produced or max_tries is reached. std::vector try_results(shuffled_pools.size()); diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index e9e923c50d7..7b07c72824a 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -163,8 +163,7 @@ void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) WriteBufferFromOwnString profiling_msg_builder; /// We don't want -0. that can appear due to rounding errors. - if (cpu_usage <= 0) - cpu_usage = 0; + cpu_usage = std::max(cpu_usage, 0.); profiling_msg_builder << "(" << fmt::format("{:.1f}", cpu_usage) << " CPU"; diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index ca79b9433b5..4f72b4aba75 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -152,8 +152,7 @@ void Timer::createIfNecessary(UInt64 thread_id, int clock_type, int pause_signal void Timer::set(UInt32 period) { /// Too high frequency can introduce infinite busy loop of signal handlers. We will limit maximum frequency (with 1000 signals per second). - if (period < 1000000) - period = 1000000; + period = std::max(period, 1000000); /// Randomize offset as uniform random value from 0 to period - 1. /// It will allow to sample short queries even if timer period is large. /// (For example, with period of 1 second, query with 50 ms duration will be sampled with 1 / 20 probability). diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index 20c1f4332da..df8d86f379c 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -206,14 +206,25 @@ public: while (true) { if (!queue.empty()) - return processQueue(std::move(lock)); - if (postponed.empty()) + { + processQueue(std::move(lock)); + return; + } + else if (postponed.empty()) + { wait(lock); + } else { if (postponed.front().key <= now()) - return processPostponed(std::move(lock)); - waitUntil(lock, postponed.front().key); + { + processPostponed(std::move(lock)); + return; + } + else + { + waitUntil(lock, postponed.front().key); + } } } } diff --git a/src/Common/StudentTTest.cpp b/src/Common/StudentTTest.cpp index 03159dca2ca..6a3c7d7c014 100644 --- a/src/Common/StudentTTest.cpp +++ b/src/Common/StudentTTest.cpp @@ -21,11 +21,11 @@ namespace { 1.533, 2.132, 2.776, 3.747, 4.604, 7.173 }, { 1.476, 2.015, 2.571, 3.365, 4.032, 5.893 }, { 1.440, 1.943, 2.447, 3.143, 3.707, 5.208 }, - { 1.415, 1.895, 2.365, 2.998, 3.499, 4.782 }, + { 1.415, 1.895, 2.365, 2.998, 3.499, 4.782 }, // NOLINT { 1.397, 1.860, 2.306, 2.896, 3.355, 4.499 }, { 1.383, 1.833, 2.262, 2.821, 3.250, 4.296 }, { 1.372, 1.812, 2.228, 2.764, 3.169, 4.143 }, - { 1.363, 1.796, 2.201, 2.718, 3.106, 4.024 }, + { 1.363, 1.796, 2.201, 2.718, 3.106, 4.024 }, // NOLINT { 1.356, 1.782, 2.179, 2.681, 3.055, 3.929 }, { 1.350, 1.771, 2.160, 2.650, 3.012, 3.852 }, { 1.345, 1.761, 2.145, 2.624, 2.977, 3.787 }, @@ -50,7 +50,7 @@ namespace { 1.308, 1.692, 2.035, 2.445, 2.733, 3.356 }, { 1.307, 1.691, 2.032, 2.441, 2.728, 3.348 }, { 1.306, 1.690, 2.030, 2.438, 2.724, 3.340 }, - { 1.306, 1.688, 2.028, 2.434, 2.719, 3.333 }, + { 1.306, 1.688, 2.028, 2.434, 2.719, 3.333 }, // NOLINT { 1.305, 1.687, 2.026, 2.431, 2.715, 3.326 }, { 1.304, 1.686, 2.024, 2.429, 2.712, 3.319 }, { 1.304, 1.685, 2.023, 2.426, 2.708, 3.313 }, @@ -137,8 +137,7 @@ void StudentTTest::add(size_t distribution, double value) /// Confidence_level_index can be set in range [0, 5]. Corresponding values can be found above. std::pair StudentTTest::compareAndReport(size_t confidence_level_index) const { - if (confidence_level_index > 5) - confidence_level_index = 5; + confidence_level_index = std::min(confidence_level_index, 5); if (data[0].size == 0 || data[1].size == 0) return {true, ""}; diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index a2d58fc7e8c..f8ced95653b 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -141,8 +141,7 @@ void collectSymbolsFromProgramHeaders( __msan_unpoison(buckets, hash[0] * sizeof(buckets[0])); for (ElfW(Word) i = 0; i < hash[0]; ++i) - if (buckets[i] > sym_cnt) - sym_cnt = buckets[i]; + sym_cnt = std::max(sym_cnt, buckets[i]); if (sym_cnt) { diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 4dee6d905d9..15803db4929 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -86,8 +86,7 @@ void SystemLogQueue::push(LogElement&& element) // It is enough to only wake the flushing thread once, after the message // count increases past half available size. const uint64_t queue_end = queue_front_index + queue.size(); - if (requested_flush_up_to < queue_end) - requested_flush_up_to = queue_end; + requested_flush_up_to = std::max(requested_flush_up_to, queue_end); flush_event.notify_all(); } diff --git a/src/Common/ThreadProfileEvents.h b/src/Common/ThreadProfileEvents.h index a3f79bf4d78..26aeab08302 100644 --- a/src/Common/ThreadProfileEvents.h +++ b/src/Common/ThreadProfileEvents.h @@ -110,7 +110,8 @@ private: static inline UInt64 getClockMonotonic() { struct timespec ts; - clock_gettime(CLOCK_MONOTONIC, &ts); + if (0 != clock_gettime(CLOCK_MONOTONIC, &ts)) + throw std::system_error(std::error_code(errno, std::system_category())); return ts.tv_sec * 1000000000ULL + ts.tv_nsec; } }; diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 231ef7e94e9..2e0c73131d5 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -244,12 +244,12 @@ public: /// Yield leadership and become follower. void yieldLeadership() { - return server->yieldLeadership(); + server->yieldLeadership(); } void recalculateStorageStats() { - return server->recalculateStorageStats(); + server->recalculateStorageStats(); } static void cleanResources(); diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 69c8c8bc240..adf7a41193c 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -22,8 +22,8 @@ class WriteBuffer; enum class SettingsWriteFormat : uint8_t { - BINARY, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour. - STRINGS_WITH_FLAGS, /// All settings are serialized as strings. Before each value the flag `is_important` is serialized. + BINARY = 0, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour. + STRINGS_WITH_FLAGS = 1, /// All settings are serialized as strings. Before each value the flag `is_important` is serialized. DEFAULT = STRINGS_WITH_FLAGS, }; diff --git a/src/Core/BaseSettingsProgramOptions.h b/src/Core/BaseSettingsProgramOptions.h index 6c8166fc119..81f6c59a5e5 100644 --- a/src/Core/BaseSettingsProgramOptions.h +++ b/src/Core/BaseSettingsProgramOptions.h @@ -34,11 +34,11 @@ void addProgramOptionsAsMultitokens(T &cmd_settings, boost::program_options::opt /// Adds program options to set the settings from a command line. /// (Don't forget to call notify() on the `variables_map` after parsing it!) template -void addProgramOption(T &cmd_settings, boost::program_options::options_description & options, std::string_view name, const typename T::SettingFieldRef & field) +void addProgramOption(T & cmd_settings, boost::program_options::options_description & options, std::string_view name, const typename T::SettingFieldRef & field) { auto on_program_option = boost::function1([&cmd_settings, name](const std::string & value) { cmd_settings.set(name, value); }); options.add(boost::shared_ptr(new boost::program_options::option_description( - name.data(), boost::program_options::value()->composing()->notifier(on_program_option), field.getDescription()))); + name.data(), boost::program_options::value()->composing()->notifier(on_program_option), field.getDescription()))); // NOLINT } template diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index 7630fbb0b23..807e4a7187a 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -831,7 +831,7 @@ public: [[maybe_unused]] Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) override { - return setPassword(user_name, "", session, address); + setPassword(user_name, "", session, address); } AuthenticationType getType() const override @@ -855,7 +855,7 @@ public: if (type == Messaging::FrontMessageType::PASSWORD_MESSAGE) { std::unique_ptr password = mt.receive(); - return setPassword(user_name, password->password, session, address); + setPassword(user_name, password->password, session, address); } else throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, diff --git a/src/DataTypes/DataTypeDecimalBase.h b/src/DataTypes/DataTypeDecimalBase.h index 9887dfabcdb..642d2de833f 100644 --- a/src/DataTypes/DataTypeDecimalBase.h +++ b/src/DataTypes/DataTypeDecimalBase.h @@ -195,7 +195,7 @@ inline DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Wrong precision: it must be between {} and {}, got {}", DecimalUtils::min_precision, DecimalUtils::max_precision, precision_value); - if (static_cast(scale_value) > precision_value) + if (scale_value > precision_value) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Negative scales and scales larger than precision are not supported"); if (precision_value <= DecimalUtils::max_precision) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index b84d60c4591..614fe413503 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -12,7 +12,6 @@ #include #include #include -#include #include #include @@ -55,7 +54,6 @@ using DisksMap = std::map; class IReservation; using ReservationPtr = std::unique_ptr; -using Reservations = std::vector; class ReadBufferFromFileBase; class WriteBufferFromFileBase; diff --git a/src/Disks/IDiskTransaction.h b/src/Disks/IDiskTransaction.h index 7df1b71eb2b..49fcdde1a4f 100644 --- a/src/Disks/IDiskTransaction.h +++ b/src/Disks/IDiskTransaction.h @@ -3,12 +3,17 @@ #include #include #include -#include #include +#include + namespace DB { +struct ReadSettings; +struct WriteSettings; +class WriteBufferFromFileBase; + struct RemoveRequest { std::string path; /// Relative path. @@ -47,7 +52,7 @@ public: /// Move directory from `from_path` to `to_path`. virtual void moveDirectory(const std::string & from_path, const std::string & to_path) = 0; - virtual void moveFile(const String & from_path, const String & to_path) = 0; + virtual void moveFile(const std::string & from_path, const std::string & to_path) = 0; virtual void createFile(const String & path) = 0; @@ -73,10 +78,10 @@ public: const WriteSettings & settings = {}, bool autocommit = true) = 0; - using WriteBlobFunction = std::function & object_attributes)>; + using WriteBlobFunction = std::function & blob_path, WriteMode mode, const std::optional & object_attributes)>; /// Write a file using a custom function to write an object to the disk's object storage. - virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; + virtual void writeFileUsingBlobWritingFunction(const std::string & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; /// Remove file. Throws exception if file doesn't exists or it's a directory. virtual void removeFile(const std::string & path) = 0; diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index d5dfa18987a..e36365a8174 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -39,7 +39,7 @@ public: void setReadUntilPosition(size_t position) override; - void setReadUntilEnd() override { return setReadUntilPosition(getFileSize()); } + void setReadUntilEnd() override { setReadUntilPosition(getFileSize()); } size_t getFileSize() override { return getTotalSize(blobs_to_read); } diff --git a/src/IO/Archives/LibArchiveReader.h b/src/IO/Archives/LibArchiveReader.h index 148d5dd17f2..b991cedcee5 100644 --- a/src/IO/Archives/LibArchiveReader.h +++ b/src/IO/Archives/LibArchiveReader.h @@ -2,7 +2,6 @@ #include #include -#include #include "config.h" diff --git a/src/IO/MMappedFileDescriptor.cpp b/src/IO/MMappedFileDescriptor.cpp index ebc4e7a6bbb..a7eb8e4ede5 100644 --- a/src/IO/MMappedFileDescriptor.cpp +++ b/src/IO/MMappedFileDescriptor.cpp @@ -75,7 +75,7 @@ void MMappedFileDescriptor::set(int fd_, size_t offset_) { size_t file_size = getFileSize(fd_); - if (offset > static_cast(file_size)) + if (offset > file_size) throw Exception(ErrorCodes::BAD_ARGUMENTS, "MMappedFileDescriptor: requested offset is greater than file size"); set(fd_, offset_, file_size - offset); @@ -101,5 +101,3 @@ MMappedFileDescriptor::~MMappedFileDescriptor() } } - - diff --git a/src/IO/PeekableReadBuffer.cpp b/src/IO/PeekableReadBuffer.cpp index be650f2f3b4..d1838ce2c4f 100644 --- a/src/IO/PeekableReadBuffer.cpp +++ b/src/IO/PeekableReadBuffer.cpp @@ -283,9 +283,7 @@ void PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append) { size_t pos_offset = pos - memory.data(); - size_t new_size_amortized = memory.size() * 2; - if (new_size_amortized < new_size) - new_size_amortized = new_size; + size_t new_size_amortized = std::max(memory.size() * 2, new_size); memory.resize(new_size_amortized); if (need_update_checkpoint) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 80366510b53..fa9d018eaa6 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -202,7 +202,7 @@ static Aws::String getAWSMetadataEndpoint() if (ec2_metadata_service_endpoint.empty()) { Aws::String ec2_metadata_service_endpoint_mode = Aws::Environment::GetEnv("AWS_EC2_METADATA_SERVICE_ENDPOINT_MODE"); - if (ec2_metadata_service_endpoint_mode.length() == 0) + if (ec2_metadata_service_endpoint_mode.empty()) { ec2_metadata_service_endpoint = "http://169.254.169.254"; //default to IPv4 default endpoint } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 3ea372f75d8..ff18a77f09f 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -339,7 +339,10 @@ void WriteBufferFromS3::allocateBuffer() chassert(0 == hidden_size); if (buffer_allocation_policy->getBufferNumber() == 1) - return allocateFirstBuffer(); + { + allocateFirstBuffer(); + return; + } memory = Memory(buffer_allocation_policy->getBufferSize()); WriteBuffer::set(memory.data(), memory.size()); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 1df559b252c..fbfec3588fa 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -12,13 +12,13 @@ #include #include #include -#include #include #include #include #include + namespace DB { /** diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index c5311e39718..6f322355d36 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -6,6 +6,7 @@ #include #include "Interpreters/Cache/Guards.h" + namespace DB { diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index c746ce39b2d..ea801afc931 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -144,59 +144,59 @@ public: /// Loads a specified object. /// The function does nothing if it's already loaded. /// The function doesn't throw an exception if it's failed to load. - template , void>> + template , void>> // NOLINT ReturnType tryLoad(const String & name, Duration timeout = WAIT) const; /// Loads objects by filter. /// The function does nothing for already loaded objects, it just returns them. /// The function doesn't throw an exception if it's failed to load something. - template , void>> + template , void>> // NOLINT ReturnType tryLoad(const FilterByNameFunction & filter, Duration timeout = WAIT) const; /// Loads all objects. /// The function does nothing for already loaded objects, it just returns them. /// The function doesn't throw an exception if it's failed to load something. - template , void>> + template , void>> // NOLINT ReturnType tryLoadAll(Duration timeout = WAIT) const { return tryLoad(FilterByNameFunction{}, timeout); } /// Loads a specified object. /// The function does nothing if it's already loaded. /// The function throws an exception if it's failed to load. - template , void>> + template , void>> // NOLINT ReturnType load(const String & name) const; /// Loads objects by filter. /// The function does nothing for already loaded objects, it just returns them. /// The function throws an exception if it's failed to load something. - template , void>> + template , void>> // NOLINT ReturnType load(const FilterByNameFunction & filter) const; /// Loads all objects. Not recommended to use. /// The function does nothing for already loaded objects, it just returns them. /// The function throws an exception if it's failed to load something. - template , void>> + template , void>> // NOLINT ReturnType loadAll() const { return load(FilterByNameFunction{}); } /// Loads or reloads a specified object. /// The function reloads the object if it's already loaded. /// The function throws an exception if it's failed to load or reload. - template , void>> + template , void>> // NOLINT ReturnType loadOrReload(const String & name) const; /// Loads or reloads objects by filter. /// The function reloads the objects which are already loaded. /// The function throws an exception if it's failed to load or reload something. - template , void>> + template , void>> // NOLINT ReturnType loadOrReload(const FilterByNameFunction & filter) const; /// Load or reloads all objects. Not recommended to use. /// The function throws an exception if it's failed to load or reload something. - template , void>> + template , void>> // NOLINT ReturnType loadOrReloadAll() const { return loadOrReload(FilterByNameFunction{}); } /// Reloads objects by filter which were tried to load before (successfully or not). /// The function throws an exception if it's failed to load or reload something. - template , void>> + template , void>> // NOLINT ReturnType reloadAllTriedToLoad() const; /// Check if object with name exists in configuration diff --git a/src/Parsers/ASTBackupQuery.cpp b/src/Parsers/ASTBackupQuery.cpp index bdb78eaf971..bc1dbfc961d 100644 --- a/src/Parsers/ASTBackupQuery.cpp +++ b/src/Parsers/ASTBackupQuery.cpp @@ -180,7 +180,7 @@ namespace if (settings) changes = assert_cast(settings.get())->changes; - boost::remove_erase_if( + boost::remove_erase_if( // NOLINT changes, [](const SettingChange & change) { diff --git a/src/Parsers/ASTCheckQuery.h b/src/Parsers/ASTCheckQuery.h index eca08b2b094..9dc4155c39d 100644 --- a/src/Parsers/ASTCheckQuery.h +++ b/src/Parsers/ASTCheckQuery.h @@ -38,11 +38,7 @@ struct ASTCheckTableQuery : public ASTQueryWithTableAndOutput protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { - std::string nl_or_nothing = settings.one_line ? "" : "\n"; - std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); - std::string nl_or_ws = settings.one_line ? " " : "\n"; - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "CHECK TABLE " << (settings.hilite ? hilite_none : ""); if (table) @@ -83,11 +79,7 @@ struct ASTCheckAllTablesQuery : public ASTQueryWithOutput protected: void formatQueryImpl(const FormatSettings & settings, FormatState & /* state */, FormatStateStacked frame) const override { - std::string nl_or_nothing = settings.one_line ? "" : "\n"; - std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); - std::string nl_or_ws = settings.one_line ? " " : "\n"; - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "CHECK ALL TABLES" << (settings.hilite ? hilite_none : ""); } }; diff --git a/src/Parsers/IParser.cpp b/src/Parsers/IParser.cpp index ddd210b01ec..857009680b1 100644 --- a/src/Parsers/IParser.cpp +++ b/src/Parsers/IParser.cpp @@ -14,10 +14,7 @@ IParser::Pos & IParser::Pos::operator=(const IParser::Pos & rhs) { depth = rhs.depth; max_depth = rhs.max_depth; - - if (rhs.backtracks > backtracks) - backtracks = rhs.backtracks; - + backtracks = std::max(backtracks, rhs.backtracks); max_backtracks = rhs.max_backtracks; if (rhs < *this) diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index 15f0c83c074..9ac6e623803 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -484,7 +484,7 @@ Token Lexer::nextTokenImpl() if (heredoc_name_end_position != std::string::npos) { size_t heredoc_size = heredoc_name_end_position + 1; - std::string_view heredoc = {token_stream.data(), heredoc_size}; + std::string_view heredoc = {token_stream.data(), heredoc_size}; // NOLINT size_t heredoc_end_position = token_stream.find(heredoc, heredoc_size); if (heredoc_end_position != std::string::npos) diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index 5eab0cb288d..6a86edd7e2f 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -304,7 +304,7 @@ namespace changes = assert_cast(settings.get())->changes; } - boost::remove_erase_if(changes, [](const SettingChange & change) { return change.name == "async"; }); + boost::remove_erase_if(changes, [](const SettingChange & change) { return change.name == "async"; }); // NOLINT changes.emplace_back("async", async); auto new_settings = std::make_shared(); diff --git a/src/Parsers/makeASTForLogicalFunction.cpp b/src/Parsers/makeASTForLogicalFunction.cpp index fd9b78a5f52..416441f210b 100644 --- a/src/Parsers/makeASTForLogicalFunction.cpp +++ b/src/Parsers/makeASTForLogicalFunction.cpp @@ -12,7 +12,7 @@ namespace DB ASTPtr makeASTForLogicalAnd(ASTs && arguments) { bool partial_result = true; - boost::range::remove_erase_if(arguments, [&](const ASTPtr & argument) -> bool + boost::range::remove_erase_if(arguments, [&](const ASTPtr & argument) -> bool // NOLINT { bool b; if (!tryGetLiteralBool(argument.get(), b)) @@ -41,7 +41,7 @@ ASTPtr makeASTForLogicalAnd(ASTs && arguments) ASTPtr makeASTForLogicalOr(ASTs && arguments) { bool partial_result = false; - boost::range::remove_erase_if(arguments, [&](const ASTPtr & argument) -> bool + boost::range::remove_erase_if(arguments, [&](const ASTPtr & argument) -> bool // NOLINT { bool b; if (!tryGetLiteralBool(argument.get(), b))