diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index f66b1eb5b91..4a66d43606b 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -40,7 +40,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log split->addTextLog(log, text_log_max_priority); auto current_logger = config.getString("logger", ""); - if (config_logger == current_logger) + if (config_logger == current_logger) //-V1051 return; config_logger = current_logger; diff --git a/src/Access/ExternalAuthenticators.cpp b/src/Access/ExternalAuthenticators.cpp index 1cade973724..0c4d2f417c9 100644 --- a/src/Access/ExternalAuthenticators.cpp +++ b/src/Access/ExternalAuthenticators.cpp @@ -77,7 +77,7 @@ auto parseLDAPServer(const Poco::Util::AbstractConfiguration & config, const Str if (enable_tls_lc_str == "starttls") params.enable_tls = LDAPClient::Params::TLSEnable::YES_STARTTLS; else if (config.getBool(ldap_server_config + ".enable_tls")) - params.enable_tls = LDAPClient::Params::TLSEnable::YES; + params.enable_tls = LDAPClient::Params::TLSEnable::YES; //-V1048 else params.enable_tls = LDAPClient::Params::TLSEnable::NO; } @@ -96,7 +96,7 @@ auto parseLDAPServer(const Poco::Util::AbstractConfiguration & config, const Str else if (tls_minimum_protocol_version_lc_str == "tls1.1") params.tls_minimum_protocol_version = LDAPClient::Params::TLSProtocolVersion::TLS1_1; else if (tls_minimum_protocol_version_lc_str == "tls1.2") - params.tls_minimum_protocol_version = LDAPClient::Params::TLSProtocolVersion::TLS1_2; + params.tls_minimum_protocol_version = LDAPClient::Params::TLSProtocolVersion::TLS1_2; //-V1048 else throw Exception("Bad value for 'tls_minimum_protocol_version' entry, allowed values are: 'ssl2', 'ssl3', 'tls1.0', 'tls1.1', 'tls1.2'", ErrorCodes::BAD_ARGUMENTS); } @@ -113,7 +113,7 @@ auto parseLDAPServer(const Poco::Util::AbstractConfiguration & config, const Str else if (tls_require_cert_lc_str == "try") params.tls_require_cert = LDAPClient::Params::TLSRequireCert::TRY; else if (tls_require_cert_lc_str == "demand") - params.tls_require_cert = LDAPClient::Params::TLSRequireCert::DEMAND; + params.tls_require_cert = LDAPClient::Params::TLSRequireCert::DEMAND; //-V1048 else throw Exception("Bad value for 'tls_require_cert' entry, allowed values are: 'never', 'allow', 'try', 'demand'", ErrorCodes::BAD_ARGUMENTS); } diff --git a/src/Access/GrantedRoles.cpp b/src/Access/GrantedRoles.cpp index 7930b56e44d..2659f8a3ec9 100644 --- a/src/Access/GrantedRoles.cpp +++ b/src/Access/GrantedRoles.cpp @@ -136,7 +136,7 @@ GrantedRoles::Elements GrantedRoles::getElements() const boost::range::set_difference(roles, roles_with_admin_option, std::back_inserter(element.ids)); if (!element.empty()) { - element.admin_option = false; + element.admin_option = false; //-V1048 elements.emplace_back(std::move(element)); } diff --git a/src/AggregateFunctions/AggregateFunctionStatistics.h b/src/AggregateFunctions/AggregateFunctionStatistics.h index 473bce1d89a..2b778f85d99 100644 --- a/src/AggregateFunctions/AggregateFunctionStatistics.h +++ b/src/AggregateFunctions/AggregateFunctionStatistics.h @@ -13,7 +13,7 @@ namespace DB { -namespace +namespace detail { /// This function returns true if both values are large and comparable. @@ -72,7 +72,7 @@ public: Float64 factor = static_cast(count * source.count) / total_count; Float64 delta = mean - source.mean; - if (areComparable(count, source.count)) + if (detail::areComparable(count, source.count)) mean = (source.count * source.mean + count * mean) / total_count; else mean = source.mean + delta * (static_cast(count) / total_count); @@ -302,7 +302,7 @@ public: Float64 left_delta = left_mean - source.left_mean; Float64 right_delta = right_mean - source.right_mean; - if (areComparable(count, source.count)) + if (detail::areComparable(count, source.count)) { left_mean = (source.count * source.left_mean + count * left_mean) / total_count; right_mean = (source.count * source.right_mean + count * right_mean) / total_count; diff --git a/src/Common/HyperLogLogCounter.h b/src/Common/HyperLogLogCounter.h index 0acffaaaf33..4f44e0c3c55 100644 --- a/src/Common/HyperLogLogCounter.h +++ b/src/Common/HyperLogLogCounter.h @@ -82,17 +82,12 @@ template class __attribute__ ((packed)) Denominator; -namespace -{ - /// Returns true if rank storage is big. constexpr bool isBigRankStore(UInt8 precision) { return precision >= 12; } -} - /// Used to deduce denominator type depending on options provided. template struct IntermediateDenominator; @@ -120,7 +115,7 @@ struct IntermediateDenominator -class __attribute__ ((packed)) Denominator> { @@ -164,7 +159,7 @@ private: /// Used when rank storage is big. template -class __attribute__ ((packed)) Denominator> { @@ -252,6 +247,7 @@ struct RankWidth } + /// Sets behavior of HyperLogLog class. enum class HyperLogLogMode { diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 141ac7a7e49..78d45a93b13 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -388,7 +388,7 @@ void PoolWithFailoverBase::updateErrorCounts(PoolWithFailoverBase TestKeeperCreateRequest::process(TestKeeper::Contai else { TestKeeper::Node created_node; - created_node.seq_num = 0; + created_node.seq_num = 0; //-V1044 created_node.stat.czxid = zxid; created_node.stat.mzxid = zxid; created_node.stat.ctime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1); @@ -271,7 +271,7 @@ std::pair TestKeeperRemoveRequest::process(TestKeeper::Contai auto & parent = container.at(parentPath(path)); --parent.stat.numChildren; ++parent.stat.cversion; - response.error = Error::ZOK; + response.error = Error::ZOK; //-V1044 undo = [prev_node, &container, path = path] { @@ -293,7 +293,7 @@ std::pair TestKeeperExistsRequest::process(TestKeeper::Contai if (it != container.end()) { response.stat = it->second.stat; - response.error = Error::ZOK; + response.error = Error::ZOK; //-V1044 } else { @@ -316,7 +316,7 @@ std::pair TestKeeperGetRequest::process(TestKeeper::Container { response.stat = it->second.stat; response.data = it->second.data; - response.error = Error::ZOK; + response.error = Error::ZOK; //-V1044 } return { std::make_shared(response), {} }; @@ -343,7 +343,7 @@ std::pair TestKeeperSetRequest::process(TestKeeper::Container it->second.data = data; ++container.at(parentPath(path)).stat.cversion; response.stat = it->second.stat; - response.error = Error::ZOK; + response.error = Error::ZOK; //-V1044 undo = [prev_node, &container, path = path] { @@ -387,7 +387,7 @@ std::pair TestKeeperListRequest::process(TestKeeper::Containe } response.stat = it->second.stat; - response.error = Error::ZOK; + response.error = Error::ZOK; //-V1044 } return { std::make_shared(response), {} }; @@ -407,7 +407,7 @@ std::pair TestKeeperCheckRequest::process(TestKeeper::Contain } else { - response.error = Error::ZOK; + response.error = Error::ZOK; //-V1044 } return { std::make_shared(response), {} }; @@ -422,7 +422,7 @@ std::pair TestKeeperMultiRequest::process(TestKeeper::Contain try { auto request_it = requests.begin(); - response.error = Error::ZOK; + response.error = Error::ZOK; //-V1044 while (request_it != requests.end()) { const TestKeeperRequest & concrete_request = dynamic_cast(**request_it); diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 51a66fba3aa..3d8eec63471 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -79,7 +79,6 @@ std::filesystem::path getMountPoint(std::filesystem::path absolute_path) if (device_id != parent_device_id) return absolute_path; absolute_path = parent; - device_id = parent_device_id; } return absolute_path; diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index bfcd1d05607..20fe5476807 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -35,29 +35,6 @@ using namespace DB; -namespace std -{ -template -std::ostream & operator<<(std::ostream & ostr, const std::optional & opt) -{ - if (!opt) - { - return ostr << ""; - } - - return ostr << *opt; -} - -template -std::vector operator+(std::vector && left, std::vector && right) -{ - std::vector result(std::move(left)); - std::move(std::begin(right), std::end(right), std::back_inserter(result)); - - return result; -} - -} namespace { @@ -337,6 +314,14 @@ CodecTestSequence operator+(CodecTestSequence && left, const CodecTestSequence & return left.append(right); } +std::vector operator+(const std::vector & left, const std::vector & right) +{ + std::vector result(std::move(left)); + std::move(std::begin(right), std::end(right), std::back_inserter(result)); + + return result; +} + template CodecTestSequence operator*(CodecTestSequence && left, T times) { @@ -362,7 +347,7 @@ std::ostream & operator<<(std::ostream & ostr, const Codec & codec) { return ostr << "Codec{" << "name: " << codec.codec_statement - << ", expected_compression_ratio: " << codec.expected_compression_ratio + << ", expected_compression_ratio: " << *codec.expected_compression_ratio << "}"; } @@ -775,15 +760,13 @@ auto FFand0Generator = []() return [step = 0](auto i) mutable { decltype(i) result; - if (step++ % 2 == 0) - { - memset(&result, 0, sizeof(result)); - } - else - { - memset(&result, 0xFF, sizeof(result)); - } + if (step % 2 == 0) + memset(&result, 0, sizeof(result)); + else + memset(&result, 0xFF, sizeof(result)); + + ++step; return result; }; }; diff --git a/src/DataStreams/IBlockInputStream.h b/src/DataStreams/IBlockInputStream.h index b077f87e8a8..090ea394fd6 100644 --- a/src/DataStreams/IBlockInputStream.h +++ b/src/DataStreams/IBlockInputStream.h @@ -158,7 +158,7 @@ public: /** Set the approximate total number of rows to read. */ - virtual void addTotalRowsApprox(size_t value) { total_rows_approx += value; } + void addTotalRowsApprox(size_t value) { total_rows_approx += value; } /** Ask to abort the receipt of data as soon as possible. diff --git a/src/DataStreams/NativeBlockOutputStream.cpp b/src/DataStreams/NativeBlockOutputStream.cpp index 2a016c9a0c8..a5d8c15986b 100644 --- a/src/DataStreams/NativeBlockOutputStream.cpp +++ b/src/DataStreams/NativeBlockOutputStream.cpp @@ -51,7 +51,7 @@ static void writeData(const IDataType & type, const ColumnPtr & column, WriteBuf ISerialization::SerializeBinaryBulkSettings settings; settings.getter = [&ostr](ISerialization::SubstreamPath) -> WriteBuffer * { return &ostr; }; settings.position_independent_encoding = false; - settings.low_cardinality_max_dictionary_size = 0; + settings.low_cardinality_max_dictionary_size = 0; //-V1048 auto serialization = type.getDefaultSerialization(); diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 8e2c6dba6c1..63a8e9edb79 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -210,7 +210,8 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view { max_dimension = std::max(max_dimension, dimension); - if (--dimension == 0) + --dimension; + if (dimension == 0) break; dimensions[dimension].emplace_back(Array(dimensions[dimension + 1].begin(), dimensions[dimension + 1].end())); diff --git a/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp b/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp index 9d8c32c92b6..675ecfbe6e7 100644 --- a/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp +++ b/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp @@ -15,8 +15,6 @@ #include -namespace std -{ template inline std::ostream& operator<<(std::ostream & ostr, const std::vector & v) @@ -29,8 +27,6 @@ inline std::ostream& operator<<(std::ostream & ostr, const std::vector & v) return ostr << "] (" << v.size() << ") items"; } -} - using namespace DB; struct ParseDataTypeTestCase diff --git a/src/Dictionaries/Embedded/RegionsNames.h b/src/Dictionaries/Embedded/RegionsNames.h index 7216f238156..91830490325 100644 --- a/src/Dictionaries/Embedded/RegionsNames.h +++ b/src/Dictionaries/Embedded/RegionsNames.h @@ -84,7 +84,7 @@ public: { size_t language_id = static_cast(language); - if (region_id >= names_refs[language_id].size()) + if (region_id >= names_refs[language_id].size()) //-V1051 return StringRef("", 0); StringRef ref = names_refs[language_id][region_id]; diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 676863ae588..e440ed59fb0 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -164,7 +164,7 @@ bool MySQLDictionarySource::isModified() const if (!invalidate_query.empty()) { auto response = doInvalidateQuery(invalidate_query); - if (response == invalidate_query_response) + if (response == invalidate_query_response) //-V1051 return false; invalidate_query_response = response; return true; diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 051ee94ef3e..af7aef45acc 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -103,7 +103,7 @@ bool PostgreSQLDictionarySource::isModified() const if (!invalidate_query.empty()) { auto response = doInvalidateQuery(invalidate_query); - if (response == invalidate_query_response) + if (response == invalidate_query_response) //-V1051 return false; invalidate_query_response = response; } diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 5774641a90f..618e5282068 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -225,7 +225,7 @@ bool XDBCDictionarySource::isModified() const if (!invalidate_query.empty()) { auto response = doInvalidateQuery(invalidate_query); - if (invalidate_query_response == response) + if (invalidate_query_response == response) //-V1051 return false; invalidate_query_response = response; } diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index d060b86c54b..29aa5913b83 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -146,7 +146,7 @@ public: if (!outlen) { outlen = 0; - dst_pos = savepoint; + dst_pos = savepoint; //-V1048 // clean the symbol dst_pos[0] = 0; } diff --git a/src/Functions/isValidUTF8.cpp b/src/Functions/isValidUTF8.cpp index d8c5d3cc580..e3158bb709c 100644 --- a/src/Functions/isValidUTF8.cpp +++ b/src/Functions/isValidUTF8.cpp @@ -277,7 +277,7 @@ SOFTWARE. len -= 16; }; - while (len >= 16) // NOLINT + while (len >= 16) // NOLINT //-V1044 check_packed(_mm_loadu_si128(reinterpret_cast(data))); /// 0 <= len <= 15 for now. Reading data from data - 1 because of right padding of 15 and left padding diff --git a/src/Interpreters/ConvertStringsToEnumVisitor.cpp b/src/Interpreters/ConvertStringsToEnumVisitor.cpp new file mode 100644 index 00000000000..85971457c36 --- /dev/null +++ b/src/Interpreters/ConvertStringsToEnumVisitor.cpp @@ -0,0 +1,186 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + +/// @note We place strings in ascending order here under the assumption it colud speed up String to Enum conversion. +String makeStringsEnum(const std::set & values) +{ + String enum_string = "Enum8("; + if (values.size() >= 255) + enum_string = "Enum16("; + + size_t number = 1; + for (const auto & item : values) + { + enum_string += "\'" + item + "\' = " + std::to_string(number++); + + if (number <= values.size()) + enum_string += ", "; + } + + enum_string += ")"; + return enum_string; +} + +void changeIfArguments(ASTPtr & first, ASTPtr & second) +{ + String first_value = first->as()->value.get(); + String second_value = second->as()->value.get(); + + std::set values; + values.insert(first_value); + values.insert(second_value); + + String enum_string = makeStringsEnum(values); + auto enum_literal = std::make_shared(enum_string); + + auto first_cast = makeASTFunction("CAST"); + first_cast->arguments->children.push_back(first); + first_cast->arguments->children.push_back(enum_literal); + + auto second_cast = makeASTFunction("CAST"); + second_cast->arguments->children.push_back(second); + second_cast->arguments->children.push_back(enum_literal); + + first = first_cast; + second = second_cast; +} + +void changeTransformArguments(ASTPtr & array_to, ASTPtr & other) +{ + std::set values; + + for (const auto & item : array_to->as()->value.get()) + values.insert(item.get()); + values.insert(other->as()->value.get()); + + String enum_string = makeStringsEnum(values); + + auto array_cast = makeASTFunction("CAST"); + array_cast->arguments->children.push_back(array_to); + array_cast->arguments->children.push_back(std::make_shared("Array(" + enum_string + ")")); + array_to = array_cast; + + auto other_cast = makeASTFunction("CAST"); + other_cast->arguments->children.push_back(other); + other_cast->arguments->children.push_back(std::make_shared(enum_string)); + other = other_cast; +} + +bool checkSameType(const Array & array, const String & type) +{ + for (const auto & item : array) + if (item.getTypeName() != type) + return false; + return true; +} + +} + + +bool FindUsedFunctionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) +{ + return !(node->as()); +} + +void FindUsedFunctionsMatcher::visit(const ASTPtr & ast, Data & data) +{ + if (auto * func = ast->as()) + visit(*func, data); +} + +void FindUsedFunctionsMatcher::visit(const ASTFunction & func, Data & data) +{ + if (data.names.count(func.name) && !data.call_stack.empty()) + { + String alias = func.tryGetAlias(); + if (!alias.empty()) + { + data.used_functions.insert(alias); + } + } + + data.call_stack.push_back(func.name); + + /// Visit children with known call stack + Visitor(data).visit(func.arguments); + + data.call_stack.pop_back(); +} + + +bool ConvertStringsToEnumMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) +{ + return !(node->as()); +} + +void ConvertStringsToEnumMatcher::visit(ASTPtr & ast, Data & data) +{ + if (auto * func = ast->as()) + visit(*func, data); +} + +void ConvertStringsToEnumMatcher::visit(ASTFunction & function_node, Data & data) +{ + if (!function_node.arguments) + return; + + /// We are not sure we could change the type of function result + /// cause it is present in other function as argument + if (data.used_functions.count(function_node.tryGetAlias())) + return; + + if (function_node.name == "if") + { + if (function_node.arguments->children.size() != 2) + return; + + auto literal1 = function_node.arguments->children[1]->as(); + auto literal2 = function_node.arguments->children[2]->as(); + if (!literal1 || !literal2) + return; + + if (String(literal1->value.getTypeName()) != "String" || + String(literal2->value.getTypeName()) != "String") + return; + + changeIfArguments(function_node.arguments->children[1], + function_node.arguments->children[2]); + } + else if (function_node.name == "transform") + { + if (function_node.arguments->children.size() != 4) + return; + + auto literal_to = function_node.arguments->children[2]->as(); + auto literal_other = function_node.arguments->children[3]->as(); + if (!literal_to || !literal_other) + return; + + if (String(literal_to->value.getTypeName()) != "Array" || + String(literal_other->value.getTypeName()) != "String") + return; + + Array array_to = literal_to->value.get(); + if (array_to.size() == 0) + return; + + bool to_strings = checkSameType(array_to, "String"); + if (!to_strings) + return; + + changeTransformArguments(function_node.arguments->children[2], function_node.arguments->children[3]); + } +} + +} + diff --git a/src/Interpreters/ConvertStringsToEnumVisitor.h b/src/Interpreters/ConvertStringsToEnumVisitor.h index adf408f87bd..b1389f40654 100644 --- a/src/Interpreters/ConvertStringsToEnumVisitor.h +++ b/src/Interpreters/ConvertStringsToEnumVisitor.h @@ -1,98 +1,17 @@ #pragma once -#include -#include +#include +#include +#include + #include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include + namespace DB { -namespace -{ - -/// @note We place strings in ascending order here under the assumption it colud speed up String to Enum conversion. -String makeStringsEnum(const std::set & values) -{ - String enum_string = "Enum8("; - if (values.size() >= 255) - enum_string = "Enum16("; - - size_t number = 1; - for (const auto & item : values) - { - enum_string += "\'" + item + "\' = " + std::to_string(number++); - - if (number <= values.size()) - enum_string += ", "; - } - - enum_string += ")"; - return enum_string; -} - -void changeIfArguments(ASTPtr & first, ASTPtr & second) -{ - String first_value = first->as()->value.get(); - String second_value = second->as()->value.get(); - - std::set values; - values.insert(first_value); - values.insert(second_value); - - String enum_string = makeStringsEnum(values); - auto enum_literal = std::make_shared(enum_string); - - auto first_cast = makeASTFunction("CAST"); - first_cast->arguments->children.push_back(first); - first_cast->arguments->children.push_back(enum_literal); - - auto second_cast = makeASTFunction("CAST"); - second_cast->arguments->children.push_back(second); - second_cast->arguments->children.push_back(enum_literal); - - first = first_cast; - second = second_cast; -} - -void changeTransformArguments(ASTPtr & array_to, ASTPtr & other) -{ - std::set values; - - for (const auto & item : array_to->as()->value.get()) - values.insert(item.get()); - values.insert(other->as()->value.get()); - - String enum_string = makeStringsEnum(values); - - auto array_cast = makeASTFunction("CAST"); - array_cast->arguments->children.push_back(array_to); - array_cast->arguments->children.push_back(std::make_shared("Array(" + enum_string + ")")); - array_to = array_cast; - - auto other_cast = makeASTFunction("CAST"); - other_cast->arguments->children.push_back(other); - other_cast->arguments->children.push_back(std::make_shared(enum_string)); - other = other_cast; -} - -bool checkSameType(const Array & array, const String & type) -{ - for (const auto & item : array) - if (item.getTypeName() != type) - return false; - return true; -} - -} +class ASTFunction; struct FindUsedFunctionsMatcher { @@ -105,35 +24,9 @@ struct FindUsedFunctionsMatcher std::vector call_stack = {}; }; - static bool needChildVisit(const ASTPtr & node, const ASTPtr &) - { - return !(node->as()); - } - - static void visit(const ASTPtr & ast, Data & data) - { - if (auto * func = ast->as()) - visit(*func, data); - } - - static void visit(const ASTFunction & func, Data & data) - { - if (data.names.count(func.name) && !data.call_stack.empty()) - { - String alias = func.tryGetAlias(); - if (!alias.empty()) - { - data.used_functions.insert(alias); - } - } - - data.call_stack.push_back(func.name); - - /// Visit children with known call stack - Visitor(data).visit(func.arguments); - - data.call_stack.pop_back(); - } + static bool needChildVisit(const ASTPtr & node, const ASTPtr &); + static void visit(const ASTPtr & ast, Data & data); + static void visit(const ASTFunction & func, Data & data); }; using FindUsedFunctionsVisitor = FindUsedFunctionsMatcher::Visitor; @@ -145,70 +38,9 @@ struct ConvertStringsToEnumMatcher std::unordered_set & used_functions; }; - static bool needChildVisit(const ASTPtr & node, const ASTPtr &) - { - return !(node->as()); - } - - static void visit(ASTPtr & ast, Data & data) - { - if (auto * func = ast->as()) - visit(*func, data); - } - - static void visit(ASTFunction & function_node, Data & data) - { - if (!function_node.arguments) - return; - - /// We are not sure we could change the type of function result - /// cause it is present in other function as argument - if (data.used_functions.count(function_node.tryGetAlias())) - return; - - if (function_node.name == "if") - { - if (function_node.arguments->children.size() != 2) - return; - - auto literal1 = function_node.arguments->children[1]->as(); - auto literal2 = function_node.arguments->children[2]->as(); - if (!literal1 || !literal2) - return; - - if (String(literal1->value.getTypeName()) != "String" || - String(literal2->value.getTypeName()) != "String") - return; - - changeIfArguments(function_node.arguments->children[1], - function_node.arguments->children[2]); - } - else if (function_node.name == "transform") - { - if (function_node.arguments->children.size() != 4) - return; - - auto literal_to = function_node.arguments->children[2]->as(); - auto literal_other = function_node.arguments->children[3]->as(); - if (!literal_to || !literal_other) - return; - - if (String(literal_to->value.getTypeName()) != "Array" || - String(literal_other->value.getTypeName()) != "String") - return; - - Array array_to = literal_to->value.get(); - if (array_to.size() == 0) - return; - - bool to_strings = checkSameType(array_to, "String"); - if (!to_strings) - return; - - changeTransformArguments(function_node.arguments->children[2], - function_node.arguments->children[3]); - } - } + static bool needChildVisit(const ASTPtr & node, const ASTPtr &); + static void visit(ASTPtr & ast, Data & data); + static void visit(ASTFunction & function_node, Data & data); }; using ConvertStringsToEnumVisitor = InDepthNodeVisitor; diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index 84959f2f624..edf0f37c00e 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -77,7 +77,7 @@ BlockIO InterpreterWatchQuery::execute() if (IBlockInputStream * stream = dynamic_cast(streams[0].get())) { StreamLocalLimits limits; - limits.mode = LimitsMode::LIMITS_CURRENT; + limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 limits.size_limits.max_rows = settings.max_result_rows; limits.size_limits.max_bytes = settings.max_result_bytes; limits.size_limits.overflow_mode = settings.result_overflow_mode; diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index e4459399336..ad4fb60f00c 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -143,7 +143,7 @@ bool PartLog::addNewParts( if (query_id.data && query_id.size) elem.query_id.insert(0, query_id.data, query_id.size); - elem.event_type = PartLogElement::NEW_PART; + elem.event_type = PartLogElement::NEW_PART; //-V1048 // construct event_time and event_time_microseconds using the same time point // so that the two times will always be equal up to a precision of a second. diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 0abd34a88dd..90e39ae0af7 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -552,7 +552,7 @@ static std::tuple executeQueryImpl( StreamLocalLimits limits; if (!interpreter->ignoreLimits()) { - limits.mode = LimitsMode::LIMITS_CURRENT; + limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); } @@ -632,7 +632,7 @@ static std::tuple executeQueryImpl( { QueryLogElement elem; - elem.type = QueryLogElementType::QUERY_START; + elem.type = QueryLogElementType::QUERY_START; //-V1048 elem.event_time = time_in_seconds(current_time); elem.event_time_microseconds = time_in_microseconds(current_time); diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index cc51848b4a4..80299610a44 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -281,7 +281,7 @@ void createMissedColumns(Block & block) for (size_t i = 0; i < block.columns(); ++i) { auto & column = block.getByPosition(i); - if (!column.column) + if (!column.column) //-V1051 column.column = column.type->createColumn(); } } diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index 7a7abba94a9..c8d8dc10a7f 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -31,7 +31,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected else if (s_pipeline.ignore(pos, expected)) kind = ASTExplainQuery::ExplainKind::QueryPipeline; else if (s_plan.ignore(pos, expected)) - kind = ASTExplainQuery::ExplainKind::QueryPlan; + kind = ASTExplainQuery::ExplainKind::QueryPlan; //-V1048 } else return false; diff --git a/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp b/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp index 458fa0a0a90..736ad1ecefe 100644 --- a/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp +++ b/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp @@ -121,11 +121,4 @@ void PipelineExecutingBlockInputStream::setQuota(const std::shared_ptr & quota_) final; - void addTotalRowsApprox(size_t value) final; protected: void readPrefixImpl() override; diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 0c8caa93539..90c7e370880 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -135,7 +135,7 @@ public: { auto num_threads = pipe.maxParallelStreams(); - if (max_threads) + if (max_threads) //-V1051 num_threads = std::min(num_threads, max_threads); return std::max(1, num_threads); diff --git a/src/Processors/QueryPlan/PartialSortingStep.cpp b/src/Processors/QueryPlan/PartialSortingStep.cpp index f4abea440fe..d713a63028a 100644 --- a/src/Processors/QueryPlan/PartialSortingStep.cpp +++ b/src/Processors/QueryPlan/PartialSortingStep.cpp @@ -58,7 +58,7 @@ void PartialSortingStep::transformPipeline(QueryPipeline & pipeline, const Build }); StreamLocalLimits limits; - limits.mode = LimitsMode::LIMITS_CURRENT; + limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 limits.size_limits = size_limits; pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index d04d98c9d97..44c5c48975c 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -158,7 +158,7 @@ QueryPipelinePtr QueryPlan::buildQueryPipeline( if (last_pipeline) { frame.pipelines.emplace_back(std::move(last_pipeline)); - last_pipeline = nullptr; + last_pipeline = nullptr; //-V1048 } size_t next_child = frame.pipelines.size(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 916f29ba1d4..9ef0292ee06 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -745,7 +745,7 @@ void TCPHandler::processTablesStatusRequest() status.absolute_delay = replicated_table->getAbsoluteDelay(); } else - status.is_replicated = false; + status.is_replicated = false; //-V1048 response.table_states_by_id.emplace(table_name, std::move(status)); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index dad7f021491..854b64181cc 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1179,7 +1179,7 @@ void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & pa part_log_elem.event_type = PartLogElement::REMOVE_PART; part_log_elem.event_time = time(nullptr); - part_log_elem.duration_ms = 0; + part_log_elem.duration_ms = 0; //-V1048 part_log_elem.database_name = table_id.database_name; part_log_elem.table_name = table_id.table_name; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 2efda206cf9..5e85f88a487 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -114,8 +114,8 @@ void writeColumnSingleGranule( ISerialization::SerializeBinaryBulkSettings serialize_settings; serialize_settings.getter = stream_getter; - serialize_settings.position_independent_encoding = true; - serialize_settings.low_cardinality_max_dictionary_size = 0; + serialize_settings.position_independent_encoding = true; //-V1048 + serialize_settings.low_cardinality_max_dictionary_size = 0; //-V1048 serialization->serializeBinaryBulkStatePrefix(serialize_settings, state); serialization->serializeBinaryBulkWithMultipleStreams(*column.column, from_row, number_of_rows, serialize_settings, state); diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index ff875b185e9..6cee80983d6 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -69,7 +69,7 @@ void MergeTreeIndexGranuleSet::serializeBinary(WriteBuffer & ostr) const ISerialization::SerializeBinaryBulkSettings settings; settings.getter = [&ostr](ISerialization::SubstreamPath) -> WriteBuffer * { return &ostr; }; settings.position_independent_encoding = false; - settings.low_cardinality_max_dictionary_size = 0; + settings.low_cardinality_max_dictionary_size = 0; //-V1048 auto serialization = type->getDefaultSerialization(); ISerialization::SerializeBinaryBulkStatePtr state; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 525a08784be..39884022e07 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -401,7 +401,7 @@ void StorageRabbitMQ::bindExchange() } } - while (!binding_created) + while (!binding_created) //-V1044 { event_handler->iterateLoop(); } @@ -462,7 +462,7 @@ void StorageRabbitMQ::bindQueue(size_t queue_id) const String queue_name = !hash_exchange ? queue_base : std::to_string(queue_id) + "_" + queue_base; setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); - while (!binding_created) + while (!binding_created) //-V1044 { event_handler->iterateLoop(); }