diff --git a/base/base/ReplxxLineReader.cpp b/base/base/ReplxxLineReader.cpp index a014fa4b8f2..e0dc81af5b0 100644 --- a/base/base/ReplxxLineReader.cpp +++ b/base/base/ReplxxLineReader.cpp @@ -151,7 +151,7 @@ public: { size_t dot_pos = path.rfind('.'); if (dot_pos != std::string::npos) - fd = ::mkstemps(path.data(), path.size() - dot_pos); + fd = ::mkstemps(path.data(), static_cast(path.size() - dot_pos)); else fd = ::mkstemp(path.data()); @@ -408,7 +408,7 @@ ReplxxLineReader::ReplxxLineReader( // In a simplest case use simple comment. commented_line = fmt::format("-- {}", state.text()); } - rx.set_state(replxx::Replxx::State(commented_line.c_str(), commented_line.size())); + rx.set_state(replxx::Replxx::State(commented_line.c_str(), static_cast(commented_line.size()))); return rx.invoke(Replxx::ACTION::COMMIT_LINE, code); }; @@ -480,7 +480,7 @@ void ReplxxLineReader::openEditor() if (executeCommand(argv) == 0) { const std::string & new_query = readFile(editor_file.getPath()); - rx.set_state(replxx::Replxx::State(new_query.c_str(), new_query.size())); + rx.set_state(replxx::Replxx::State(new_query.c_str(), static_cast(new_query.size()))); } } catch (const std::runtime_error & e) @@ -526,7 +526,7 @@ void ReplxxLineReader::openInteractiveHistorySearch() { std::string new_query = readFile(output_file.getPath()); rightTrim(new_query); - rx.set_state(replxx::Replxx::State(new_query.c_str(), new_query.size())); + rx.set_state(replxx::Replxx::State(new_query.c_str(), static_cast(new_query.size()))); } } catch (const std::runtime_error & e) diff --git a/base/base/StringRef.h b/base/base/StringRef.h index 5ee197021ca..779099a1573 100644 --- a/base/base/StringRef.h +++ b/base/base/StringRef.h @@ -265,7 +265,7 @@ inline size_t hashLessThan16(const char * data, size_t size) struct CRC32Hash { - size_t operator() (StringRef x) const + unsigned operator() (StringRef x) const { const char * pos = x.data; size_t size = x.size; @@ -275,22 +275,22 @@ struct CRC32Hash if (size < 8) { - return hashLessThan8(x.data, x.size); + return static_cast(hashLessThan8(x.data, x.size)); } const char * end = pos + size; - size_t res = -1ULL; + unsigned res = -1U; do { UInt64 word = unalignedLoad(pos); - res = CRC_INT(res, word); + res = static_cast(CRC_INT(res, word)); pos += 8; } while (pos + 8 < end); UInt64 word = unalignedLoad(end - 8); /// I'm not sure if this is normal. - res = CRC_INT(res, word); + res = static_cast(CRC_INT(res, word)); return res; } diff --git a/base/base/itoa.h b/base/base/itoa.h index 5e0b18d50c0..809b7c86c42 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -122,7 +122,7 @@ QuotientAndRemainder static inline split(UnsignedOfSize value) constexpr DivisionBy10PowN division; UnsignedOfSize quotient = (division.multiplier * (UnsignedOfSize<2 * N>(value) + division.add)) >> division.shift; - UnsignedOfSize remainder = value - quotient * pow10>(N); + UnsignedOfSize remainder = static_cast>(value - quotient * pow10>(N)); return {quotient, remainder}; } diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e616cb8cf72..58569a32619 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1108,15 +1108,21 @@ void Client::processConfig() else format = config().getString("format", is_interactive ? "PrettyCompact" : "TabSeparated"); - format_max_block_size = config().getInt("format_max_block_size", global_context->getSettingsRef().max_block_size); + format_max_block_size = config().getUInt64("format_max_block_size", + global_context->getSettingsRef().max_block_size); insert_format = "Values"; /// Setting value from cmd arg overrides one from config if (global_context->getSettingsRef().max_insert_block_size.changed) + { insert_format_max_block_size = global_context->getSettingsRef().max_insert_block_size; + } else - insert_format_max_block_size = config().getInt("insert_format_max_block_size", global_context->getSettingsRef().max_insert_block_size); + { + insert_format_max_block_size = config().getUInt64("insert_format_max_block_size", + global_context->getSettingsRef().max_insert_block_size); + } ClientInfo & client_info = global_context->getClientInfo(); client_info.setInitialQuery(); diff --git a/programs/copier/ZooKeeperStaff.h b/programs/copier/ZooKeeperStaff.h index a9e04578607..3d4a11186e3 100644 --- a/programs/copier/ZooKeeperStaff.h +++ b/programs/copier/ZooKeeperStaff.h @@ -47,8 +47,8 @@ public: WrappingUInt32 epoch; WrappingUInt32 counter; explicit Zxid(UInt64 _zxid) - : epoch(_zxid >> 32) - , counter(_zxid) + : epoch(static_cast(_zxid >> 32)) + , counter(static_cast(_zxid)) {} bool operator<=(const Zxid & other) const diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 00c86571265..8028ccde72d 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -893,7 +893,7 @@ namespace if (fs::exists(pid_file)) { ReadBufferFromFile in(pid_file.string()); - UInt64 pid; + Int32 pid; if (tryReadIntText(pid, in)) { fmt::print("{} file exists and contains pid = {}.\n", pid_file.string(), pid); @@ -982,9 +982,9 @@ namespace return 0; } - UInt64 isRunning(const fs::path & pid_file) + int isRunning(const fs::path & pid_file) { - UInt64 pid = 0; + int pid = 0; if (fs::exists(pid_file)) { @@ -1057,7 +1057,7 @@ namespace if (force && do_not_kill) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Specified flags are incompatible"); - UInt64 pid = isRunning(pid_file); + int pid = isRunning(pid_file); if (!pid) return 0; diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 5077f59b7dd..e1d03b40b66 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -68,12 +68,12 @@ namespace ErrorCodes namespace { -int waitServersToFinish(std::vector & servers, size_t seconds_to_wait) +size_t waitServersToFinish(std::vector & servers, size_t seconds_to_wait) { - const int sleep_max_ms = 1000 * seconds_to_wait; - const int sleep_one_ms = 100; - int sleep_current_ms = 0; - int current_connections = 0; + const size_t sleep_max_ms = 1000 * seconds_to_wait; + const size_t sleep_one_ms = 100; + size_t sleep_current_ms = 0; + size_t current_connections = 0; for (;;) { current_connections = 0; @@ -441,7 +441,7 @@ int Keeper::main(const std::vector & /*args*/) main_config_reloader.reset(); LOG_DEBUG(log, "Waiting for current connections to Keeper to finish."); - int current_connections = 0; + size_t current_connections = 0; for (auto & server : *servers) { server.stop(); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e7bc019f597..070f86aaad2 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -546,9 +546,14 @@ void LocalServer::processConfig() /// Setting value from cmd arg overrides one from config if (global_context->getSettingsRef().max_insert_block_size.changed) + { insert_format_max_block_size = global_context->getSettingsRef().max_insert_block_size; + } else - insert_format_max_block_size = config().getInt("insert_format_max_block_size", global_context->getSettingsRef().max_insert_block_size); + { + insert_format_max_block_size = config().getUInt64("insert_format_max_block_size", + global_context->getSettingsRef().max_insert_block_size); + } /// Sets external authenticators config (LDAP, Kerberos). global_context->setExternalAuthenticatorsConfig(config()); diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index bdf26c9e730..7fdc5a54d8a 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -279,7 +279,7 @@ Float transformFloatMantissa(Float x, UInt64 seed) constexpr size_t mantissa_num_bits = std::is_same_v ? 23 : 52; UInt x_uint = bit_cast(x); - x_uint = feistelNetwork(x_uint, mantissa_num_bits, seed); + x_uint = static_cast(feistelNetwork(x_uint, mantissa_num_bits, seed)); return bit_cast(x_uint); } @@ -511,13 +511,13 @@ public: for (size_t i = 0; i < size; ++i) { UInt32 src_datetime = src_data[i]; - UInt32 src_date = date_lut.toDate(src_datetime); + UInt32 src_date = static_cast(date_lut.toDate(src_datetime)); Int32 src_diff = src_datetime - src_prev_value; - Int32 res_diff = transformSigned(src_diff, seed); + Int32 res_diff = static_cast(transformSigned(src_diff, seed)); UInt32 new_datetime = res_prev_value + res_diff; - UInt32 new_time = new_datetime - date_lut.toDate(new_datetime); + UInt32 new_time = new_datetime - static_cast(date_lut.toDate(new_datetime)); res_data[i] = src_date + new_time; src_prev_value = src_datetime; diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 0ea2495af78..7fa51fc8fcd 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -183,7 +183,10 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ if (columns.empty()) throw Exception("Columns definition was not returned", ErrorCodes::LOGICAL_ERROR); - WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + WriteBufferFromHTTPServerResponse out( + response, + request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, + static_cast(keep_alive_timeout)); try { writeStringBinary(columns.toString(), out); diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index dec4c249b4b..5bbc39dc559 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -139,7 +139,7 @@ void ODBCSource::insertValue( readDateTimeText(time, in, assert_cast(data_type.get())->getTimeZone()); if (time < 0) time = 0; - assert_cast(column).insertValue(time); + assert_cast(column).insertValue(static_cast(time)); break; } case ValueType::vtDateTime64: diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 7c3193ceac6..b412b579539 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -228,12 +228,12 @@ catch (...) path)); } -int waitServersToFinish(std::vector & servers, size_t seconds_to_wait) +size_t waitServersToFinish(std::vector & servers, size_t seconds_to_wait) { - const int sleep_max_ms = 1000 * seconds_to_wait; - const int sleep_one_ms = 100; - int sleep_current_ms = 0; - int current_connections = 0; + const size_t sleep_max_ms = 1000 * seconds_to_wait; + const size_t sleep_one_ms = 100; + size_t sleep_current_ms = 0; + size_t current_connections = 0; for (;;) { current_connections = 0; @@ -933,7 +933,7 @@ int Server::main(const std::vector & /*args*/) else { rlim_t old = rlim.rlim_cur; - rlim.rlim_cur = config().getUInt("max_open_files", rlim.rlim_max); + rlim.rlim_cur = config().getUInt("max_open_files", static_cast(rlim.rlim_max)); int rc = setrlimit(RLIMIT_NOFILE, &rlim); if (rc != 0) LOG_WARNING(log, "Cannot set max number of file descriptors to {}. Try to specify max_open_files according to your system limits. error: {}", rlim.rlim_cur, errnoToString()); @@ -1507,7 +1507,7 @@ int Server::main(const std::vector & /*args*/) if (!servers_to_start_before_tables.empty()) { LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish."); - int current_connections = 0; + size_t current_connections = 0; for (auto & server : servers_to_start_before_tables) { server.stop(); @@ -1793,7 +1793,7 @@ int Server::main(const std::vector & /*args*/) is_cancelled = true; - int current_connections = 0; + size_t current_connections = 0; { std::lock_guard lock(servers_lock); for (auto & server : servers) diff --git a/src/Access/Common/AllowedClientHosts.cpp b/src/Access/Common/AllowedClientHosts.cpp index 2f8151bf757..905f7ba08b5 100644 --- a/src/Access/Common/AllowedClientHosts.cpp +++ b/src/Access/Common/AllowedClientHosts.cpp @@ -236,7 +236,7 @@ void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, const IPAddres void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, size_t num_prefix_bits) { - set(prefix_, IPAddress(num_prefix_bits, prefix_.family())); + set(prefix_, IPAddress(static_cast(num_prefix_bits), prefix_.family())); } void AllowedClientHosts::IPSubnet::set(const IPAddress & address) diff --git a/src/Access/LDAPClient.cpp b/src/Access/LDAPClient.cpp index ff1ee6f3609..2affbc293ec 100644 --- a/src/Access/LDAPClient.cpp +++ b/src/Access/LDAPClient.cpp @@ -254,7 +254,7 @@ bool LDAPClient::openConnection() #endif { - const int search_timeout = params.search_timeout.count(); + const int search_timeout = static_cast(params.search_timeout.count()); diag(ldap_set_option(handle, LDAP_OPT_TIMELIMIT, &search_timeout)); } diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h index fd8357e3ba8..dacde67f3ca 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h @@ -45,7 +45,8 @@ public: void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { - assert_cast &>(to).getData().push_back(this->data(place).rbs.size()); + assert_cast &>(to).getData().push_back( + static_cast(this->data(place).rbs.size())); } }; @@ -142,7 +143,8 @@ public: void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { - assert_cast &>(to).getData().push_back(this->data(place).rbs.size()); + assert_cast &>(to).getData().push_back( + static_cast(this->data(place).rbs.size())); } }; diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 21ba7cd7301..801526432ae 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -426,7 +426,7 @@ public: return 0; if (isSmall()) - return small.find(x) != small.end(); + return small.find(static_cast(x)) != small.end(); else return rb->contains(static_cast(x)); } diff --git a/src/AggregateFunctions/AggregateFunctionHistogram.h b/src/AggregateFunctions/AggregateFunctionHistogram.h index 9031eb73c09..18bfc085ba3 100644 --- a/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -136,8 +136,8 @@ private: for (size_t i = 0; i <= size; ++i) { - previous[i] = i - 1; - next[i] = i + 1; + previous[i] = static_cast(i - 1); + next[i] = static_cast(i + 1); } next[size] = 0; @@ -157,7 +157,7 @@ private: auto quality = [&](UInt32 i) { return points[next[i]].mean - points[i].mean; }; for (size_t i = 0; i + 1 < size; ++i) - queue.push({quality(i), i}); + queue.push({quality(static_cast(i)), i}); while (new_size > max_bins && !queue.empty()) { @@ -217,7 +217,7 @@ private: points[left] = points[right]; } } - size = left + 1; + size = static_cast(left + 1); } public: diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index 783fa0606b5..f8d252cf8e9 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -540,7 +540,7 @@ public: /// Assuming to.has() void changeImpl(StringRef value, Arena * arena) { - Int32 value_size = value.size; + Int32 value_size = static_cast(value.size); if (value_size <= MAX_SMALL_STRING_SIZE) { @@ -555,7 +555,7 @@ public: if (capacity < value_size) { /// Don't free large_data here. - capacity = roundUpToPowerOfTwoOrZero(value_size); + capacity = static_cast(roundUpToPowerOfTwoOrZero(value_size)); large_data = arena->alloc(capacity); } diff --git a/src/AggregateFunctions/AggregateFunctionRetention.h b/src/AggregateFunctions/AggregateFunctionRetention.h index a949b5e93f6..18d04fb1ea4 100644 --- a/src/AggregateFunctions/AggregateFunctionRetention.h +++ b/src/AggregateFunctions/AggregateFunctionRetention.h @@ -44,7 +44,7 @@ struct AggregateFunctionRetentionData void serialize(WriteBuffer & buf) const { - UInt32 event_value = events.to_ulong(); + UInt32 event_value = static_cast(events.to_ulong()); writeBinary(event_value, buf); } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceMatch.h b/src/AggregateFunctions/AggregateFunctionSequenceMatch.h index 5c1ab803f19..bcea408d26b 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -272,7 +272,7 @@ private: actions.emplace_back(PatternActionType::SpecificEvent, event_number - 1); dfa_states.back().transition = DFATransition::SpecificEvent; - dfa_states.back().event = event_number - 1; + dfa_states.back().event = static_cast(event_number - 1); dfa_states.emplace_back(); conditions_in_pattern.set(event_number - 1); } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 423b53b03f3..c29055ae8db 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -226,7 +226,7 @@ public: for (UInt8 i = 0; i < events_size; ++i) if (assert_cast *>(columns[min_required_args + i])->getData()[row_num]) node->events_bitset.set(i); - node->event_time = timestamp; + node->event_time = static_cast(timestamp); node->can_be_base = assert_cast *>(columns[base_cond_column_idx])->getData()[row_num]; @@ -365,7 +365,7 @@ public: /// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on. UInt32 getNextNodeIndex(Data & data) const { - const UInt32 unmatched_idx = data.value.size(); + const UInt32 unmatched_idx = static_cast(data.value.size()); if (data.value.size() <= events_size) return unmatched_idx; diff --git a/src/AggregateFunctions/AggregateFunctionSparkbar.h b/src/AggregateFunctions/AggregateFunctionSparkbar.h index cb017053fd7..f0fbdd2f2e4 100644 --- a/src/AggregateFunctions/AggregateFunctionSparkbar.h +++ b/src/AggregateFunctions/AggregateFunctionSparkbar.h @@ -165,7 +165,7 @@ private: { for (size_t i = 0; i <= diff_x; ++i) { - auto it = data.points.find(min_x_local + i); + auto it = data.points.find(static_cast(min_x_local + i)); bool found = it != data.points.end(); value += getBar(found ? std::round(((it->getMapped() - min_y) / diff_y) * 7) + 1 : 0.0); } @@ -173,7 +173,7 @@ private: else { for (size_t i = 0; i <= diff_x; ++i) - value += getBar(data.points.has(min_x_local + i) ? 1 : 0); + value += getBar(data.points.has(min_x_local + static_cast(i)) ? 1 : 0); } } else @@ -202,7 +202,7 @@ private: if (i == bound.first) // is bound { Float64 proportion = bound.second - bound.first; - auto it = data.points.find(min_x_local + i); + auto it = data.points.find(min_x_local + static_cast(i)); bool found = (it != data.points.end()); if (found && proportion > 0) new_y = new_y.value_or(0) + it->getMapped() * proportion; @@ -229,7 +229,7 @@ private: } else { - auto it = data.points.find(min_x_local + i); + auto it = data.points.find(min_x_local + static_cast(i)); if (it != data.points.end()) new_y = new_y.value_or(0) + it->getMapped(); } @@ -267,8 +267,8 @@ public: if (params.size() == 3) { specified_min_max_x = true; - min_x = params.at(1).safeGet(); - max_x = params.at(2).safeGet(); + min_x = static_cast(params.at(1).safeGet()); + max_x = static_cast(params.at(2).safeGet()); } else { diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index d44937b3f9d..fe2530800cc 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -175,8 +175,9 @@ struct OneAdder { if constexpr (!std::is_same_v) { + using ValueType = typename decltype(data.set)::value_type; const auto & value = assert_cast &>(column).getElement(row_num); - data.set.insert(AggregateFunctionUniqTraits::hash(value)); + data.set.insert(static_cast(AggregateFunctionUniqTraits::hash(value))); } else { diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/src/AggregateFunctions/AggregateFunctionUniqCombined.h index 51020abe826..47b3081225b 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -43,7 +43,7 @@ namespace detail { static Ret hash(UInt128 x) { - return sipHash64(x); + return static_cast(sipHash64(x)); } }; diff --git a/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/src/AggregateFunctions/ReservoirSamplerDeterministic.h index a64c02e823b..557fd93a3a9 100644 --- a/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -84,7 +84,7 @@ public: if (isNaN(v)) return; - UInt32 hash = intHash64(determinator); + UInt32 hash = static_cast(intHash64(determinator)); insertImpl(v, hash); sorted = false; ++total_values; diff --git a/src/AggregateFunctions/UniquesHashSet.h b/src/AggregateFunctions/UniquesHashSet.h index 5c82ec10691..777ec0edc7e 100644 --- a/src/AggregateFunctions/UniquesHashSet.h +++ b/src/AggregateFunctions/UniquesHashSet.h @@ -118,7 +118,7 @@ private: HashValue hash(Value key) const { - return Hash()(key); + return static_cast(Hash()(key)); } /// Delete all values whose hashes do not divide by 2 ^ skip_degree diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index be509824261..12038a8a30c 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -44,12 +44,12 @@ namespace S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( settings.auth_settings.region, context->getRemoteHostFilter(), - context->getGlobalContext()->getSettingsRef().s3_max_redirects, + static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, /* for_disk_s3 = */ false); client_configuration.endpointOverride = s3_uri.endpoint; - client_configuration.maxConnections = context->getSettingsRef().s3_max_connections; + client_configuration.maxConnections = static_cast(context->getSettingsRef().s3_max_connections); /// Increase connect timeout client_configuration.connectTimeoutMs = 10 * 1000; /// Requests in backups can be extremely long, set to one hour @@ -221,7 +221,7 @@ void BackupWriterS3::copyObjectMultipartImpl( part_request.SetBucket(dst_bucket); part_request.SetKey(dst_key); part_request.SetUploadId(multipart_upload_id); - part_request.SetPartNumber(part_number); + part_request.SetPartNumber(static_cast(part_number)); part_request.SetCopySourceRange(fmt::format("bytes={}-{}", position, std::min(size, position + upload_part_size) - 1)); auto outcome = client->UploadPartCopy(part_request); @@ -251,7 +251,7 @@ void BackupWriterS3::copyObjectMultipartImpl( for (size_t i = 0; i < part_tags.size(); ++i) { Aws::S3::Model::CompletedPart part; - multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(i + 1)); + multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(static_cast(i) + 1)); } req.SetMultipartUpload(multipart_upload); diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index a4b20e0b863..295ab723326 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -62,7 +62,6 @@ namespace #define LIST_OF_BACKUP_SETTINGS(M) \ M(String, id) \ M(String, compression_method) \ - M(Int64, compression_level) \ M(String, password) \ M(Bool, structure_only) \ M(Bool, async) \ @@ -72,6 +71,7 @@ namespace M(String, host_id) \ M(String, coordination_zk_path) \ M(OptionalUUID, backup_uuid) + /// M(Int64, compression_level) BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) { @@ -82,6 +82,9 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) const auto & settings = query.settings->as().changes; for (const auto & setting : settings) { + if (setting.name == "compression_level") + res.compression_level = static_cast(SettingFieldInt64{setting.value}.value); + else #define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \ if (setting.name == #NAME) \ res.NAME = SettingField##TYPE{setting.value}.value; \ diff --git a/src/Bridge/IBridge.h b/src/Bridge/IBridge.h index 2f35c361cca..68af8860cb3 100644 --- a/src/Bridge/IBridge.h +++ b/src/Bridge/IBridge.h @@ -43,7 +43,7 @@ private: std::string hostname; size_t port; std::string log_level; - size_t max_server_connections; + unsigned max_server_connections; size_t http_timeout; Poco::Logger * log; diff --git a/src/BridgeHelper/IBridgeHelper.h b/src/BridgeHelper/IBridgeHelper.h index a3348c81b68..b7fac3f1303 100644 --- a/src/BridgeHelper/IBridgeHelper.h +++ b/src/BridgeHelper/IBridgeHelper.h @@ -43,7 +43,7 @@ protected: virtual String serviceFileName() const = 0; - virtual size_t getDefaultPort() const = 0; + virtual unsigned getDefaultPort() const = 0; virtual bool startBridgeManually() const = 0; diff --git a/src/BridgeHelper/LibraryBridgeHelper.h b/src/BridgeHelper/LibraryBridgeHelper.h index 447a4c713f4..1723d1f8fb4 100644 --- a/src/BridgeHelper/LibraryBridgeHelper.h +++ b/src/BridgeHelper/LibraryBridgeHelper.h @@ -23,7 +23,7 @@ protected: String serviceFileName() const override { return serviceAlias(); } - size_t getDefaultPort() const override { return DEFAULT_PORT; } + unsigned getDefaultPort() const override { return DEFAULT_PORT; } bool startBridgeManually() const override { return false; } diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index b62cb277ecb..139c1ab9726 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -109,7 +109,7 @@ protected: String getName() const override { return BridgeHelperMixin::getName(); } - size_t getDefaultPort() const override { return DEFAULT_PORT; } + unsigned getDefaultPort() const override { return DEFAULT_PORT; } String serviceAlias() const override { return BridgeHelperMixin::serviceAlias(); } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 0db7a9533db..ee5c17ee8f2 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -550,7 +550,7 @@ try out_file_buf = wrapWriteBufferWithCompressionMethod( std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), compression_method, - compression_level + static_cast(compression_level) ); if (query_with_output->is_into_outfile_with_stdout) @@ -1602,6 +1602,8 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( if (this_query_begin >= all_queries_end) return MultiQueryProcessingStage::QUERIES_END; + unsigned max_parser_depth = static_cast(global_context->getSettingsRef().max_parser_depth); + // If there are only comments left until the end of file, we just // stop. The parser can't handle this situation because it always // expects that there is some query that it can parse. @@ -1611,7 +1613,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( // and it makes more sense to treat them as such. { Tokens tokens(this_query_begin, all_queries_end); - IParser::Pos token_iterator(tokens, global_context->getSettingsRef().max_parser_depth); + IParser::Pos token_iterator(tokens, max_parser_depth); if (!token_iterator.isValid()) return MultiQueryProcessingStage::QUERIES_END; } @@ -1632,7 +1634,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( if (ignore_error) { Tokens tokens(this_query_begin, all_queries_end); - IParser::Pos token_iterator(tokens, global_context->getSettingsRef().max_parser_depth); + IParser::Pos token_iterator(tokens, max_parser_depth); while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid()) ++token_iterator; this_query_begin = token_iterator->end; @@ -1672,7 +1674,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( // after we have processed the query. But even this guess is // beneficial so that we see proper trailing comments in "echo" and // server log. - adjustQueryEnd(this_query_end, all_queries_end, global_context->getSettingsRef().max_parser_depth); + adjustQueryEnd(this_query_end, all_queries_end, max_parser_depth); return MultiQueryProcessingStage::EXECUTE_QUERY; } @@ -1866,7 +1868,9 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) if (insert_ast && isSyncInsertWithData(*insert_ast, global_context)) { this_query_end = insert_ast->end; - adjustQueryEnd(this_query_end, all_queries_end, global_context->getSettingsRef().max_parser_depth); + adjustQueryEnd( + this_query_end, all_queries_end, + static_cast(global_context->getSettingsRef().max_parser_depth)); } // Report error. @@ -2347,7 +2351,7 @@ void ClientBase::init(int argc, char ** argv) if (options.count("print-profile-events")) config().setBool("print-profile-events", true); if (options.count("profile-events-delay-ms")) - config().setInt("profile-events-delay-ms", options["profile-events-delay-ms"].as()); + config().setUInt64("profile-events-delay-ms", options["profile-events-delay-ms"].as()); if (options.count("progress")) config().setBool("progress", true); if (options.count("echo")) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index f1802467b57..c7392a86a7e 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -338,7 +338,7 @@ HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(As offset_states[location.offset].replicas[location.index].change_replica_timeout.reset(); offset_states[location.offset].replicas[location.index].is_change_replica_timeout_expired = true; offset_states[location.offset].next_replica_in_process = true; - offsets_queue.push(location.offset); + offsets_queue.push(static_cast(location.offset)); ProfileEvents::increment(ProfileEvents::HedgedRequestsChangeReplica); startNewReplica(); } diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 81067f51d29..bed73b1c200 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -362,7 +362,7 @@ void HedgedConnectionsFactory::removeReplicaFromEpoll(int index, int fd) timeout_fd_to_replica_index.erase(replicas[index].change_replica_timeout.getDescriptor()); } -int HedgedConnectionsFactory::numberOfProcessingReplicas() const +size_t HedgedConnectionsFactory::numberOfProcessingReplicas() const { if (epoll.empty()) return 0; @@ -381,7 +381,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(C && result.is_usable && !replicas[i].is_ready && (!skip_replicas_with_two_level_aggregation_incompatibility || !isTwoLevelAggregationIncompatible(&*result.entry))) - indexes.push_back(i); + indexes.push_back(static_cast(i)); } if (indexes.empty()) diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index c5e8d493efa..194e962d549 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -70,7 +70,7 @@ public: const ConnectionTimeouts & getConnectionTimeouts() const { return timeouts; } - int numberOfProcessingReplicas() const; + size_t numberOfProcessingReplicas() const; /// Tell Factory to not return connections with two level aggregation incompatibility. void skipReplicasWithTwoLevelAggregationIncompatibility() { skip_replicas_with_two_level_aggregation_incompatibility = true; } diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 6c5f5850b92..d5cd4ef1548 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -81,9 +81,9 @@ Field QueryFuzzer::getRandomField(int type) { static constexpr UInt64 scales[] = {0, 1, 2, 10}; return DecimalField( - bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) - / sizeof(*bad_int64_values))], - scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))]); + bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) / sizeof(*bad_int64_values))], + static_cast(scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))]) + ); } default: assert(false); diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index bb56baf9216..0346851ee34 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -569,8 +569,8 @@ void ColumnArray::expand(const IColumn::Filter & mask, bool inverted) if (mask.size() < offsets_data.size()) throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR); - int index = mask.size() - 1; - int from = offsets_data.size() - 1; + ssize_t index = mask.size() - 1; + ssize_t from = offsets_data.size() - 1; offsets_data.resize(mask.size()); UInt64 last_offset = offsets_data[from]; while (index >= 0) diff --git a/src/Columns/ColumnCompressed.cpp b/src/Columns/ColumnCompressed.cpp index 292c6968b86..3560e9d7e1c 100644 --- a/src/Columns/ColumnCompressed.cpp +++ b/src/Columns/ColumnCompressed.cpp @@ -27,8 +27,8 @@ std::shared_ptr> ColumnCompressed::compressBuffer(const void * data, si auto compressed_size = LZ4_compress_default( reinterpret_cast(data), compressed.data(), - data_size, - max_dest_size); + static_cast(data_size), + static_cast(max_dest_size)); if (compressed_size <= 0) throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress column"); @@ -51,8 +51,8 @@ void ColumnCompressed::decompressBuffer( auto processed_size = LZ4_decompress_safe( reinterpret_cast(compressed_data), reinterpret_cast(decompressed_data), - compressed_size, - decompressed_size); + static_cast(compressed_size), + static_cast(decompressed_size)); if (processed_size <= 0) throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress column"); diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 7038579d436..a45d4a0b5f1 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -277,8 +277,8 @@ void ColumnFixedString::expand(const IColumn::Filter & mask, bool inverted) if (mask.size() < size()) throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR); - int index = mask.size() - 1; - int from = size() - 1; + ssize_t index = mask.size() - 1; + ssize_t from = size() - 1; chars.resize_fill(mask.size() * n, 0); while (index >= 0) { diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 17e9bd97669..0ac5a2c31bb 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -46,7 +46,7 @@ namespace HashMap hash_map; for (auto val : index) - hash_map.insert({val, hash_map.size()}); + hash_map.insert({val, static_cast(hash_map.size())}); auto res_col = ColumnVector::create(); auto & data = res_col->getData(); @@ -632,7 +632,7 @@ void ColumnLowCardinality::Index::convertPositions() /// TODO: Optimize with SSE? for (size_t i = 0; i < size; ++i) - new_data[i] = data[i]; + new_data[i] = static_cast(data[i]); positions = std::move(new_positions); size_of_type = sizeof(IndexType); @@ -717,7 +717,7 @@ void ColumnLowCardinality::Index::insertPositionsRange(const IColumn & column, U positions_data.resize(size + limit); for (UInt64 i = 0; i < limit; ++i) - positions_data[size + i] = column_data[offset + i]; + positions_data[size + i] = static_cast(column_data[offset + i]); }; callForType(std::move(copy), size_of_type); diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 9c8082dcd22..982951f05b0 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -168,8 +168,8 @@ void ColumnString::expand(const IColumn::Filter & mask, bool inverted) /// We cannot change only offsets, because each string should end with terminating zero byte. /// So, we will insert one zero byte when mask value is zero. - int index = mask.size() - 1; - int from = offsets_data.size() - 1; + ssize_t index = mask.size() - 1; + ssize_t from = offsets_data.size() - 1; /// mask.size() - offsets_data.size() should be equal to the number of zeros in mask /// (if not, one of exceptions below will throw) and we can calculate the resulting chars size. UInt64 last_offset = offsets_data[from] + (mask.size() - offsets_data.size()); diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index bba10bfebf0..a1579a46ae0 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -550,7 +550,7 @@ MutableColumnPtr ColumnUnique::uniqueInsertRangeImpl( auto insert_key = [&](StringRef ref, ReverseIndex & cur_index) -> MutableColumnPtr { auto inserted_pos = cur_index.insert(ref); - positions[num_added_rows] = inserted_pos; + positions[num_added_rows] = static_cast(inserted_pos); if (inserted_pos == next_position) return update_position(next_position); @@ -562,9 +562,9 @@ MutableColumnPtr ColumnUnique::uniqueInsertRangeImpl( auto row = start + num_added_rows; if (null_map && (*null_map)[row]) - positions[num_added_rows] = getNullValueIndex(); + positions[num_added_rows] = static_cast(getNullValueIndex()); else if (column->compareAt(getNestedTypeDefaultValueIndex(), row, *src_column, 1) == 0) - positions[num_added_rows] = getNestedTypeDefaultValueIndex(); + positions[num_added_rows] = static_cast(getNestedTypeDefaultValueIndex()); else { auto ref = src_column->getDataAt(row); @@ -576,7 +576,7 @@ MutableColumnPtr ColumnUnique::uniqueInsertRangeImpl( if (insertion_point == reverse_index.lastInsertionPoint()) res = insert_key(ref, *secondary_index); else - positions[num_added_rows] = insertion_point; + positions[num_added_rows] = static_cast(insertion_point); } else res = insert_key(ref, reverse_index); diff --git a/src/Columns/MaskOperations.cpp b/src/Columns/MaskOperations.cpp index 3120828921f..e320e1d57a3 100644 --- a/src/Columns/MaskOperations.cpp +++ b/src/Columns/MaskOperations.cpp @@ -22,8 +22,8 @@ void expandDataByMask(PaddedPODArray & data, const PaddedPODArray & ma if (mask.size() < data.size()) throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR); - int from = data.size() - 1; - int index = mask.size() - 1; + ssize_t from = data.size() - 1; + ssize_t index = mask.size() - 1; data.resize(mask.size()); while (index >= 0) { @@ -317,7 +317,7 @@ int checkShortCircuitArguments(const ColumnsWithTypeAndName & arguments) for (size_t i = 0; i != arguments.size(); ++i) { if (checkAndGetShortCircuitArgument(arguments[i].column)) - last_short_circuit_argument_index = i; + last_short_circuit_argument_index = static_cast(i); } return last_short_circuit_argument_index; diff --git a/src/Columns/tests/gtest_weak_hash_32.cpp b/src/Columns/tests/gtest_weak_hash_32.cpp index 2fa6c0ea8ac..8027bd4d6cc 100644 --- a/src/Columns/tests/gtest_weak_hash_32.cpp +++ b/src/Columns/tests/gtest_weak_hash_32.cpp @@ -164,7 +164,7 @@ TEST(WeakHash32, ColumnVectorU32) for (int idx [[maybe_unused]] : {1, 2}) { - for (uint64_t i = 0; i < 65536; ++i) + for (uint32_t i = 0; i < 65536; ++i) data.push_back(i << 16u); } @@ -181,7 +181,7 @@ TEST(WeakHash32, ColumnVectorI32) for (int idx [[maybe_unused]] : {1, 2}) { - for (int64_t i = -32768; i < 32768; ++i) + for (int32_t i = -32768; i < 32768; ++i) data.push_back(i << 16); //-V610 } @@ -240,7 +240,7 @@ TEST(WeakHash32, ColumnVectorU128) val.items[0] = i << 32u; val.items[1] = i << 32u; data.push_back(val); - eq_data.push_back(i); + eq_data.push_back(static_cast(i)); } } @@ -274,7 +274,7 @@ TEST(WeakHash32, ColumnDecimal32) for (int idx [[maybe_unused]] : {1, 2}) { - for (int64_t i = -32768; i < 32768; ++i) + for (int32_t i = -32768; i < 32768; ++i) data.push_back(i << 16); //-V610 } @@ -326,7 +326,7 @@ TEST(WeakHash32, ColumnString1) for (int idx [[maybe_unused]] : {1, 2}) { - for (int64_t i = 0; i < 65536; ++i) + for (int32_t i = 0; i < 65536; ++i) { data.push_back(i); auto str = std::to_string(i); @@ -359,7 +359,7 @@ TEST(WeakHash32, ColumnString2) { size_t max_size = 3000; char letter = 'a'; - for (int64_t i = 0; i < 65536; ++i) + for (int32_t i = 0; i < 65536; ++i) { data.push_back(i); size_t s = (i % max_size) + 1; @@ -401,7 +401,7 @@ TEST(WeakHash32, ColumnString3) char letter = 'a'; for (int64_t i = 0; i < 65536; ++i) { - data.push_back(i); + data.push_back(static_cast(i)); size_t s = (i % max_size) + 1; std::string str(s,'\0'); str[0] = letter; @@ -430,7 +430,7 @@ TEST(WeakHash32, ColumnFixedString) char letter = 'a'; for (int64_t i = 0; i < 65536; ++i) { - data.push_back(i); + data.push_back(static_cast(i)); size_t s = (i % max_size) + 1; std::string str(s, letter); col->insertData(str.data(), str.size()); @@ -471,7 +471,7 @@ TEST(WeakHash32, ColumnArray) UInt32 cur = 0; for (int64_t i = 0; i < 65536; ++i) { - eq_data.push_back(i); + eq_data.push_back(static_cast(i)); size_t s = (i % max_size) + 1; cur_off += s; @@ -505,9 +505,9 @@ TEST(WeakHash32, ColumnArray2) UInt64 cur_off = 0; for (int idx [[maybe_unused]] : {1, 2}) { - for (int64_t i = 0; i < 1000; ++i) + for (int32_t i = 0; i < 1000; ++i) { - for (size_t j = 0; j < 1000; ++j) + for (uint32_t j = 0; j < 1000; ++j) { eq_data.push_back(i * 1000 + j); @@ -556,7 +556,7 @@ TEST(WeakHash32, ColumnArrayArray) UInt32 cur = 1; for (int64_t i = 0; i < 3000; ++i) { - eq_data.push_back(i); + eq_data.push_back(static_cast(i)); size_t s = (i % max_size) + 1; cur_off2 += s; @@ -667,7 +667,7 @@ TEST(WeakHash32, ColumnTupleUInt64UInt64) { data1.push_back(l); data2.push_back(i << 32u); - eq.push_back(l * 65536 + i); + eq.push_back(static_cast(l * 65536 + i)); } } @@ -695,7 +695,7 @@ TEST(WeakHash32, ColumnTupleUInt64String) size_t max_size = 3000; char letter = 'a'; - for (int64_t i = 0; i < 65536; ++i) + for (int32_t i = 0; i < 65536; ++i) { data1.push_back(l); eq.push_back(l * 65536 + i); @@ -737,7 +737,7 @@ TEST(WeakHash32, ColumnTupleUInt64FixedString) for (int64_t i = 0; i < 65536; ++i) { data1.push_back(l); - eq.push_back(l * 65536 + i); + eq.push_back(static_cast(l * 65536 + i)); size_t s = (i % max_size) + 1; std::string str(s, letter); @@ -778,7 +778,7 @@ TEST(WeakHash32, ColumnTupleUInt64Array) auto l = idx % 2; UInt32 cur = 0; - for (int64_t i = 0; i < 65536; ++i) + for (int32_t i = 0; i < 65536; ++i) { data1.push_back(l); eq_data.push_back(l * 65536 + i); diff --git a/src/Common/CombinedCardinalityEstimator.h b/src/Common/CombinedCardinalityEstimator.h index 3f4b481dce9..1911cafeaa2 100644 --- a/src/Common/CombinedCardinalityEstimator.h +++ b/src/Common/CombinedCardinalityEstimator.h @@ -65,7 +65,7 @@ public: private: using Small = SmallSet; using Medium = HashContainer; - using Large = HyperLogLogCounter; + using Large = HyperLogLogCounter; public: CombinedCardinalityEstimator() diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index c4b32a3466b..6eb8b47b114 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -149,9 +149,9 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) /// Fill lookup table for years and months. size_t year_months_lut_index = 0; - size_t first_day_of_last_month = 0; + unsigned first_day_of_last_month = 0; - for (size_t day = 0; day < DATE_LUT_SIZE; ++day) + for (unsigned day = 0; day < DATE_LUT_SIZE; ++day) { const Values & values = lut[day]; diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 2deb477ca23..3afbb6735dc 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -73,7 +73,7 @@ private: return LUTIndex(0); if (index >= DATE_LUT_SIZE) return LUTIndex(DATE_LUT_SIZE - 1); - return LUTIndex{index}; + return LUTIndex{static_cast(index)}; } template @@ -229,12 +229,12 @@ private: if (t >= lut[guess].date) { if (guess + 1 >= DATE_LUT_SIZE || t < lut[guess + 1].date) - return LUTIndex(guess); + return LUTIndex(static_cast(guess)); - return LUTIndex(guess + 1); + return LUTIndex(static_cast(guess) + 1); } - return LUTIndex(guess ? guess - 1 : 0); + return LUTIndex(guess ? static_cast(guess) - 1 : 0); } static inline LUTIndex toLUTIndex(DayNum d) @@ -272,11 +272,11 @@ private: if (likely(offset_is_whole_number_of_hours_during_epoch)) { if (likely(x >= 0)) - return x / divisor * divisor; + return static_cast(x / divisor * divisor); /// Integer division for negative numbers rounds them towards zero (up). /// We will shift the number so it will be rounded towards -inf (down). - return (x + 1 - divisor) / divisor * divisor; + return static_cast((x + 1 - divisor) / divisor * divisor); } Time date = find(x).date; @@ -285,7 +285,7 @@ private: { if (unlikely(res < 0)) return 0; - return res; + return static_cast(res); } else return res; @@ -509,7 +509,7 @@ public: if (time >= lut[index].time_at_offset_change()) time += lut[index].amount_of_offset_change(); - unsigned res = time / 3600; + unsigned res = static_cast(time / 3600); /// In case time was changed backwards at the start of next day, we will repeat the hour 23. return res <= 23 ? res : 23; @@ -548,8 +548,8 @@ public: { Time res = t % 60; if (likely(res >= 0)) - return res; - return res + 60; + return static_cast(res); + return static_cast(res) + 60; } LUTIndex index = findIndex(t); @@ -973,7 +973,7 @@ public: if constexpr (std::is_same_v) return DayNum(4 + (d - 4) / days * days); else - return ExtendedDayNum(4 + (d - 4) / days * days); + return ExtendedDayNum(static_cast(4 + (d - 4) / days * days)); } template @@ -983,9 +983,9 @@ public: if (days == 1) return toDate(d); if constexpr (std::is_same_v) - return lut_saturated[toLUTIndex(ExtendedDayNum(d / days * days))].date; + return lut_saturated[toLUTIndex(ExtendedDayNum(static_cast(d / days * days)))].date; else - return lut[toLUTIndex(ExtendedDayNum(d / days * days))].date; + return lut[toLUTIndex(ExtendedDayNum(static_cast(d / days * days)))].date; } template @@ -1034,7 +1034,7 @@ public: { if (unlikely(res < 0)) return 0; - return res; + return static_cast(res); } else return res; @@ -1047,8 +1047,8 @@ public: if (likely(offset_is_whole_number_of_minutes_during_epoch)) { if (likely(t >= 0)) - return t / divisor * divisor; - return (t + 1 - divisor) / divisor * divisor; + return static_cast(t / divisor * divisor); + return static_cast((t + 1 - divisor) / divisor * divisor); } Time date = find(t).date; @@ -1057,7 +1057,7 @@ public: { if (unlikely(res < 0)) return 0; - return res; + return static_cast(res); } else return res; @@ -1071,7 +1071,7 @@ public: if (seconds % 60 == 0) return toStartOfMinuteInterval(t, seconds / 60); - return roundDown(t, seconds); + return static_cast(roundDown(t, seconds)); } inline LUTIndex makeLUTIndex(Int16 year, UInt8 month, UInt8 day_of_month) const @@ -1249,9 +1249,9 @@ public: return lut[new_index].date + time; } - inline NO_SANITIZE_UNDEFINED Time addWeeks(Time t, Int32 delta) const + inline NO_SANITIZE_UNDEFINED Time addWeeks(Time t, Int64 delta) const { - return addDays(t, static_cast(delta) * 7); + return addDays(t, delta * 7); } inline UInt8 saturateDayOfMonth(Int16 year, UInt8 month, UInt8 day_of_month) const @@ -1331,9 +1331,9 @@ public: } template - inline auto addQuarters(DateOrTime d, Int32 delta) const + inline auto addQuarters(DateOrTime d, Int64 delta) const { - return addMonths(d, static_cast(delta) * 3); + return addMonths(d, delta * 3); } template diff --git a/src/Common/Dwarf.h b/src/Common/Dwarf.h index 09178c66d47..ef6364b6b18 100644 --- a/src/Common/Dwarf.h +++ b/src/Common/Dwarf.h @@ -218,7 +218,7 @@ private: // Offset from start to first attribute uint8_t attr_offset; // Offset within debug info. - uint32_t offset; + uint64_t offset; uint64_t code; DIEAbbreviation abbr; }; @@ -252,10 +252,10 @@ private: uint8_t unit_type = DW_UT_compile; // DW_UT_compile or DW_UT_skeleton uint8_t addr_size = 0; // Offset in .debug_info of this compilation unit. - uint32_t offset = 0; - uint32_t size = 0; + uint64_t offset = 0; + uint64_t size = 0; // Offset in .debug_info for the first DIE in this compilation unit. - uint32_t first_die = 0; + uint64_t first_die = 0; uint64_t abbrev_offset = 0; // The beginning of the CU's contribution to .debug_addr diff --git a/src/Common/HashTable/Hash.h b/src/Common/HashTable/Hash.h index 4d798173698..1e98b9e4102 100644 --- a/src/Common/HashTable/Hash.h +++ b/src/Common/HashTable/Hash.h @@ -48,33 +48,36 @@ inline DB::UInt64 intHash64(DB::UInt64 x) #include #endif -inline DB::UInt64 intHashCRC32(DB::UInt64 x) +/// NOTE: Intel intrinsic can be confusing. +/// - https://code.google.com/archive/p/sse-intrinsics/wikis/PmovIntrinsicBug.wiki +/// - https://stackoverflow.com/questions/15752770/mm-crc32-u64-poorly-defined +inline DB::UInt32 intHashCRC32(DB::UInt64 x) { #ifdef __SSE4_2__ - return _mm_crc32_u64(-1ULL, x); + return static_cast(_mm_crc32_u64(-1ULL, x)); #elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32) - return __crc32cd(-1U, x); + return static_cast(__crc32cd(-1U, x)); #else /// On other platforms we do not have CRC32. NOTE This can be confusing. - return intHash64(x); + /// NOTE: consider using intHash32() + return static_cast(intHash64(x)); #endif } - -inline DB::UInt64 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value) +inline DB::UInt32 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value) { #ifdef __SSE4_2__ - return _mm_crc32_u64(updated_value, x); + return static_cast(_mm_crc32_u64(updated_value, x)); #elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32) - return __crc32cd(updated_value, x); + return __crc32cd(static_cast(updated_value), x); #else /// On other platforms we do not have CRC32. NOTE This can be confusing. - return intHash64(x) ^ updated_value; + return static_cast(intHash64(x) ^ updated_value); #endif } template requires (sizeof(T) > sizeof(DB::UInt64)) -inline DB::UInt64 intHashCRC32(const T & x, DB::UInt64 updated_value) +inline DB::UInt32 intHashCRC32(const T & x, DB::UInt64 updated_value) { const auto * begin = reinterpret_cast(&x); for (size_t i = 0; i < sizeof(T); i += sizeof(UInt64)) @@ -83,7 +86,7 @@ inline DB::UInt64 intHashCRC32(const T & x, DB::UInt64 updated_value) begin += sizeof(DB::UInt64); } - return updated_value; + return static_cast(updated_value); } @@ -219,7 +222,7 @@ template struct HashCRC32; template requires (sizeof(T) <= sizeof(UInt64)) -inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1) +inline UInt32 hashCRC32(T key, DB::UInt64 updated_value = -1) { union { @@ -233,7 +236,7 @@ inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1) template requires (sizeof(T) > sizeof(UInt64)) -inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1) +inline UInt32 hashCRC32(T key, DB::UInt64 updated_value = -1) { return intHashCRC32(key, updated_value); } @@ -241,7 +244,7 @@ inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1) #define DEFINE_HASH(T) \ template <> struct HashCRC32\ {\ - size_t operator() (T key) const\ + UInt32 operator() (T key) const\ {\ return hashCRC32(key);\ }\ @@ -302,8 +305,8 @@ struct UInt128HashCRC32 size_t operator()(UInt128 x) const { UInt64 crc = -1ULL; - crc = __crc32cd(crc, x.items[0]); - crc = __crc32cd(crc, x.items[1]); + crc = __crc32cd(static_cast(crc), x.items[0]); + crc = __crc32cd(static_cast(crc), x.items[1]); return crc; } }; @@ -358,10 +361,10 @@ struct UInt256HashCRC32 size_t operator()(UInt256 x) const { UInt64 crc = -1ULL; - crc = __crc32cd(crc, x.items[0]); - crc = __crc32cd(crc, x.items[1]); - crc = __crc32cd(crc, x.items[2]); - crc = __crc32cd(crc, x.items[3]); + crc = __crc32cd(static_cast(crc), x.items[0]); + crc = __crc32cd(static_cast(crc), x.items[1]); + crc = __crc32cd(static_cast(crc), x.items[2]); + crc = __crc32cd(static_cast(crc), x.items[3]); return crc; } }; @@ -423,7 +426,7 @@ inline DB::UInt32 intHash32(DB::UInt64 key) key = key + (key << 6); key = key ^ ((key >> 22) | (key << 42)); - return key; + return static_cast(key); } diff --git a/src/Common/HashTable/TwoLevelHashTable.h b/src/Common/HashTable/TwoLevelHashTable.h index b8d5eedd430..5acc8b19195 100644 --- a/src/Common/HashTable/TwoLevelHashTable.h +++ b/src/Common/HashTable/TwoLevelHashTable.h @@ -44,8 +44,8 @@ protected: public: using Impl = ImplTable; - static constexpr size_t NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET; - static constexpr size_t MAX_BUCKET = NUM_BUCKETS - 1; + static constexpr UInt32 NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET; + static constexpr UInt32 MAX_BUCKET = NUM_BUCKETS - 1; size_t hash(const Key & x) const { return Hash::operator()(x); } @@ -286,13 +286,13 @@ public: void write(DB::WriteBuffer & wb) const { - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) impls[i].write(wb); } void writeText(DB::WriteBuffer & wb) const { - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) { if (i != 0) DB::writeChar(',', wb); @@ -302,13 +302,13 @@ public: void read(DB::ReadBuffer & rb) { - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) impls[i].read(rb); } void readText(DB::ReadBuffer & rb) { - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) { if (i != 0) DB::assertChar(',', rb); @@ -320,7 +320,7 @@ public: size_t size() const { size_t res = 0; - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) res += impls[i].size(); return res; @@ -328,7 +328,7 @@ public: bool empty() const { - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) if (!impls[i].empty()) return false; @@ -338,7 +338,7 @@ public: size_t getBufferSizeInBytes() const { size_t res = 0; - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) res += impls[i].getBufferSizeInBytes(); return res; diff --git a/src/Common/HashTable/TwoLevelStringHashTable.h b/src/Common/HashTable/TwoLevelStringHashTable.h index 77acca5e707..ea1914348b2 100644 --- a/src/Common/HashTable/TwoLevelStringHashTable.h +++ b/src/Common/HashTable/TwoLevelStringHashTable.h @@ -13,8 +13,8 @@ public: using Key = StringRef; using Impl = ImplTable; - static constexpr size_t NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET; - static constexpr size_t MAX_BUCKET = NUM_BUCKETS - 1; + static constexpr UInt32 NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET; + static constexpr UInt32 MAX_BUCKET = NUM_BUCKETS - 1; // TODO: currently hashing contains redundant computations when doing distributed or external aggregations size_t hash(const Key & x) const @@ -175,13 +175,13 @@ public: void write(DB::WriteBuffer & wb) const { - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) impls[i].write(wb); } void writeText(DB::WriteBuffer & wb) const { - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) { if (i != 0) DB::writeChar(',', wb); @@ -191,13 +191,13 @@ public: void read(DB::ReadBuffer & rb) { - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) impls[i].read(rb); } void readText(DB::ReadBuffer & rb) { - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) { if (i != 0) DB::assertChar(',', rb); @@ -208,7 +208,7 @@ public: size_t size() const { size_t res = 0; - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) res += impls[i].size(); return res; @@ -216,7 +216,7 @@ public: bool empty() const { - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) if (!impls[i].empty()) return false; @@ -226,7 +226,7 @@ public: size_t getBufferSizeInBytes() const { size_t res = 0; - for (size_t i = 0; i < NUM_BUCKETS; ++i) + for (UInt32 i = 0; i < NUM_BUCKETS; ++i) res += impls[i].getBufferSizeInBytes(); return res; diff --git a/src/Common/HyperLogLogCounter.h b/src/Common/HyperLogLogCounter.h index 36db00a5982..32c04d85d57 100644 --- a/src/Common/HyperLogLogCounter.h +++ b/src/Common/HyperLogLogCounter.h @@ -264,7 +264,8 @@ enum class HyperLogLogMode /// of Algorithms). template < UInt8 precision, - typename Hash = IntHash32, + typename Key = UInt64, + typename Hash = IntHash32, typename HashValueType = UInt32, typename DenominatorType = double, typename BiasEstimator = TrivialBiasEstimator, @@ -409,7 +410,9 @@ private: inline HashValueType getHash(Value key) const { - return Hash::operator()(key); + /// NOTE: this should be OK, since value is the same as key for HLL. + return static_cast( + Hash::operator()(static_cast(key))); } /// Update maximum rank for current bucket. @@ -532,6 +535,7 @@ private: template < UInt8 precision, + typename Key, typename Hash, typename HashValueType, typename DenominatorType, @@ -542,6 +546,7 @@ template details::LogLUT HyperLogLogCounter < precision, + Key, Hash, HashValueType, DenominatorType, @@ -555,6 +560,7 @@ details::LogLUT HyperLogLogCounter /// Serialization format must not be changed. using HLL12 = HyperLogLogCounter< 12, + UInt64, IntHash32, UInt32, double, diff --git a/src/Common/HyperLogLogWithSmallSetOptimization.h b/src/Common/HyperLogLogWithSmallSetOptimization.h index 39c00660ebe..1d2408186de 100644 --- a/src/Common/HyperLogLogWithSmallSetOptimization.h +++ b/src/Common/HyperLogLogWithSmallSetOptimization.h @@ -26,7 +26,7 @@ class HyperLogLogWithSmallSetOptimization : private boost::noncopyable { private: using Small = SmallSet; - using Large = HyperLogLogCounter; + using Large = HyperLogLogCounter; using LargeValueType = typename Large::value_type; Small small; diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 60efab69433..e95bc42a1ea 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -506,8 +506,16 @@ unsigned OptimizedRegularExpressionImpl::match(const char * subject DB::PODArrayWithStackMemory pieces(limit); - if (!re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, pieces.data(), pieces.size())) + if (!re2->Match( + StringPieceType(subject, subject_size), + 0, + subject_size, + RegexType::UNANCHORED, + pieces.data(), + static_cast(pieces.size()))) + { return 0; + } else { matches.resize(limit); diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 42b5b3d0990..f26648bd213 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -339,7 +339,7 @@ struct PoolWithFailoverBase::PoolState Int64 config_priority = 1; /// Priority from the GetPriorityFunc. Int64 priority = 0; - UInt32 random = 0; + UInt64 random = 0; void randomize() { diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 7266b9b9553..b50e0c0ab49 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -132,11 +132,11 @@ QueryProfilerBase::QueryProfilerBase(UInt64 thread_id, int clock_t sev.sigev_signo = pause_signal; #if defined(OS_FREEBSD) - sev._sigev_un._threadid = thread_id; + sev._sigev_un._threadid = static_cast(thread_id); #elif defined(USE_MUSL) - sev.sigev_notify_thread_id = thread_id; + sev.sigev_notify_thread_id = static_cast(thread_id); #else - sev._sigev_un._tid = thread_id; + sev._sigev_un._tid = static_cast(thread_id); #endif timer_t local_timer_id; if (timer_create(clock_type, &sev, &local_timer_id)) diff --git a/src/Common/RadixSort.h b/src/Common/RadixSort.h index 9ca43bee30c..739bec8d9dd 100644 --- a/src/Common/RadixSort.h +++ b/src/Common/RadixSort.h @@ -273,13 +273,13 @@ private: { /// Replace the histograms with the accumulated sums: the value in position i is the sum of the previous positions minus one. - size_t sums[NUM_PASSES] = {0}; + CountType sums[NUM_PASSES] = {0}; for (size_t i = 0; i < HISTOGRAM_SIZE; ++i) { for (size_t pass = 0; pass < NUM_PASSES; ++pass) { - size_t tmp = histograms[pass * HISTOGRAM_SIZE + i] + sums[pass]; + CountType tmp = histograms[pass * HISTOGRAM_SIZE + i] + sums[pass]; histograms[pass * HISTOGRAM_SIZE + i] = sums[pass] - 1; sums[pass] = tmp; } diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index b8f8a9d3a88..048e7a1f34c 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -40,7 +40,7 @@ public: #ifdef __SSE2__ protected: static constexpr auto n = sizeof(__m128i); - const int page_size = ::getPageSize(); + const Int64 page_size = ::getPageSize(); bool pageSafe(const void * const ptr) const { diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 0b89139fa53..b70b1fc5e60 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -87,7 +87,7 @@ void ThreadPoolImpl::setQueueSize(size_t value) template template -ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context) +ReturnType ThreadPoolImpl::scheduleImpl(Job job, ssize_t priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context) { auto on_error = [&](const std::string & reason) { @@ -163,19 +163,19 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::opti } template -void ThreadPoolImpl::scheduleOrThrowOnError(Job job, int priority) +void ThreadPoolImpl::scheduleOrThrowOnError(Job job, ssize_t priority) { scheduleImpl(std::move(job), priority, std::nullopt); } template -bool ThreadPoolImpl::trySchedule(Job job, int priority, uint64_t wait_microseconds) noexcept +bool ThreadPoolImpl::trySchedule(Job job, ssize_t priority, uint64_t wait_microseconds) noexcept { return scheduleImpl(std::move(job), priority, wait_microseconds); } template -void ThreadPoolImpl::scheduleOrThrow(Job job, int priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context) +void ThreadPoolImpl::scheduleOrThrow(Job job, ssize_t priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context) { scheduleImpl(std::move(job), priority, wait_microseconds, propagate_opentelemetry_tracing_context); } diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 76ada9e0d75..6742a554a85 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -50,13 +50,13 @@ public: /// NOTE: Probably you should call wait() if exception was thrown. If some previously scheduled jobs are using some objects, /// located on stack of current thread, the stack must not be unwinded until all jobs finished. However, /// if ThreadPool is a local object, it will wait for all scheduled jobs in own destructor. - void scheduleOrThrowOnError(Job job, int priority = 0); + void scheduleOrThrowOnError(Job job, ssize_t priority = 0); /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or return false. - bool trySchedule(Job job, int priority = 0, uint64_t wait_microseconds = 0) noexcept; + bool trySchedule(Job job, ssize_t priority = 0, uint64_t wait_microseconds = 0) noexcept; /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or throw an exception. - void scheduleOrThrow(Job job, int priority = 0, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true); + void scheduleOrThrow(Job job, ssize_t priority = 0, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true); /// Wait for all currently active jobs to be done. /// You may call schedule and wait many times in arbitrary order. @@ -96,10 +96,10 @@ private: struct JobWithPriority { Job job; - int priority; + ssize_t priority; DB::OpenTelemetry::TracingContextOnThread thread_trace_context; - JobWithPriority(Job job_, int priority_, const DB::OpenTelemetry::TracingContextOnThread& thread_trace_context_) + JobWithPriority(Job job_, ssize_t priority_, const DB::OpenTelemetry::TracingContextOnThread& thread_trace_context_) : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) {} bool operator< (const JobWithPriority & rhs) const @@ -113,7 +113,7 @@ private: std::exception_ptr first_exception; template - ReturnType scheduleImpl(Job job, int priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context = true); + ReturnType scheduleImpl(Job job, ssize_t priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context = true); void worker(typename std::list::iterator thread_it); diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index baa77468a13..76a4d8b1adf 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -121,7 +121,7 @@ TasksStatsCounters::TasksStatsCounters(const UInt64 tid, const MetricsProvider p stats_getter = [metrics_provider = std::make_shared(), tid]() { ::taskstats result{}; - metrics_provider->getStat(result, tid); + metrics_provider->getStat(result, static_cast(tid)); return result; }; break; @@ -526,7 +526,7 @@ void PerfEventsCounters::finalizeProfileEvents(ProfileEvents::Counters & profile continue; constexpr ssize_t bytes_to_read = sizeof(current_values[0]); - const int bytes_read = read(fd, ¤t_values[i], bytes_to_read); + const ssize_t bytes_read = read(fd, ¤t_values[i], bytes_to_read); if (bytes_read != bytes_to_read) { diff --git a/src/Common/UTF8Helpers.h b/src/Common/UTF8Helpers.h index ce90af3d5ce..623a62a6f79 100644 --- a/src/Common/UTF8Helpers.h +++ b/src/Common/UTF8Helpers.h @@ -99,7 +99,10 @@ requires (sizeof(CharT) == 1) size_t convertCodePointToUTF8(int code_point, CharT * out_bytes, size_t out_length) { static const Poco::UTF8Encoding utf8; - int res = utf8.convert(code_point, reinterpret_cast(out_bytes), out_length); + int res = utf8.convert( + code_point, + reinterpret_cast(out_bytes), + static_cast(out_length)); assert(res >= 0); return res; } @@ -109,7 +112,9 @@ requires (sizeof(CharT) == 1) std::optional convertUTF8ToCodePoint(const CharT * in_bytes, size_t in_length) { static const Poco::UTF8Encoding utf8; - int res = utf8.queryConvert(reinterpret_cast(in_bytes), in_length); + int res = utf8.queryConvert( + reinterpret_cast(in_bytes), + static_cast(in_length)); if (res >= 0) return res; diff --git a/src/Common/Volnitsky.h b/src/Common/Volnitsky.h index 6f5948b6564..a27fd36f704 100644 --- a/src/Common/Volnitsky.h +++ b/src/Common/Volnitsky.h @@ -404,7 +404,8 @@ public: /// And also adding from the end guarantees that we will find first occurrence because we will lookup bigger offsets first. for (auto i = static_cast(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i) { - bool ok = VolnitskyTraits::putNGram(needle + i, i + 1, needle, needle_size, callback); + bool ok = VolnitskyTraits::putNGram( + needle + i, static_cast(i + 1), needle, needle_size, callback); /** `putNGramUTF8CaseInsensitive` does not work if characters with lower and upper cases * are represented by different number of bytes or code points. diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 098dc522eeb..134374f98d0 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -218,7 +218,7 @@ std::pair TestKeeperCreateRequest::process(TestKeeper::Contai created_node.stat.ctime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1); created_node.stat.mtime = created_node.stat.ctime; created_node.stat.numChildren = 0; - created_node.stat.dataLength = data.length(); + created_node.stat.dataLength = static_cast(data.length()); created_node.data = data; created_node.is_ephemeral = is_ephemeral; created_node.is_sequental = is_sequential; diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 5098788fb2e..f7d5bccc0e0 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -45,7 +45,7 @@ struct ShuffleHost { String host; Int64 priority = 0; - UInt32 random = 0; + UInt64 random = 0; void randomize() { diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index a565a322790..f555ebb132e 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -724,7 +724,7 @@ void ZooKeeperRequest::createLogElements(LogElements & elems) const elem.has_watch = has_watch; elem.op_num = static_cast(getOpNum()); elem.path = getPath(); - elem.request_idx = elems.size() - 1; + elem.request_idx = static_cast(elems.size()) - 1; } @@ -762,7 +762,7 @@ void ZooKeeperCheckRequest::createLogElements(LogElements & elems) const void ZooKeeperMultiRequest::createLogElements(LogElements & elems) const { ZooKeeperRequest::createLogElements(elems); - elems.back().requests_size = requests.size(); + elems.back().requests_size = static_cast(requests.size()); for (const auto & request : requests) { auto & req = dynamic_cast(*request); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index f70dac74a6a..ebab18b5ed7 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -546,7 +546,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) if (read_xid != AUTH_XID) throw Exception(Error::ZMARSHALLINGERROR, "Unexpected event received in reply to auth request: {}", read_xid); - int32_t actual_length = in->count() - count_before_event; + int32_t actual_length = static_cast(in->count() - count_before_event); if (length != actual_length) throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", length, actual_length); @@ -821,7 +821,7 @@ void ZooKeeper::receiveEvent() } } - int32_t actual_length = in->count() - count_before_event; + int32_t actual_length = static_cast(in->count() - count_before_event); if (length != actual_length) throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", length, actual_length); diff --git a/src/Common/examples/parallel_aggregation.cpp b/src/Common/examples/parallel_aggregation.cpp index f54c4cee12c..bd252b330f3 100644 --- a/src/Common/examples/parallel_aggregation.cpp +++ b/src/Common/examples/parallel_aggregation.cpp @@ -492,7 +492,7 @@ int main(int argc, char ** argv) watch.restart(); - for (size_t i = 0; i < MapTwoLevel::NUM_BUCKETS; ++i) + for (unsigned i = 0; i < MapTwoLevel::NUM_BUCKETS; ++i) pool.scheduleOrThrowOnError([&] { merge2(maps.data(), num_threads, i); }); pool.wait(); @@ -545,7 +545,7 @@ int main(int argc, char ** argv) watch.restart(); - for (size_t i = 0; i < MapTwoLevel::NUM_BUCKETS; ++i) + for (unsigned i = 0; i < MapTwoLevel::NUM_BUCKETS; ++i) pool.scheduleOrThrowOnError([&] { merge2(maps.data(), num_threads, i); }); pool.wait(); diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 4c60a6ddac0..07a08dc7fbc 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -258,7 +258,7 @@ size_t getSizeFromFileDescriptor(int fd, const String & file_name) return buf.st_size; } -int getINodeNumberFromPath(const String & path) +Int64 getINodeNumberFromPath(const String & path) { struct stat file_stat; if (stat(path.data(), &file_stat)) diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 9faaabb42f2..0e6e16941bb 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -74,7 +74,7 @@ std::optional tryGetSizeFromFilePath(const String & path); /// Get inode number for a file path. /// Will not work correctly if filesystem does not support inodes. -int getINodeNumberFromPath(const String & path); +Int64 getINodeNumberFromPath(const String & path); } diff --git a/src/Common/formatIPv6.cpp b/src/Common/formatIPv6.cpp index 2e08828f724..bc1878c0bc6 100644 --- a/src/Common/formatIPv6.cpp +++ b/src/Common/formatIPv6.cpp @@ -80,7 +80,7 @@ static void printInteger(char *& out, T value) void formatIPv6(const unsigned char * src, char *& dst, uint8_t zeroed_tail_bytes_count) { - struct { int base, len; } best{-1, 0}, cur{-1, 0}; + struct { Int64 base, len; } best{-1, 0}, cur{-1, 0}; std::array words{}; /** Preprocess: @@ -122,14 +122,18 @@ void formatIPv6(const unsigned char * src, char *& dst, uint8_t zeroed_tail_byte best.base = -1; /// Format the result. - for (const int i : collections::range(0, words.size())) + for (const size_t i : collections::range(0, words.size())) { /// Are we inside the best run of 0x00's? - if (best.base != -1 && i >= best.base && i < (best.base + best.len)) + if (best.base != -1) { - if (i == best.base) - *dst++ = ':'; - continue; + size_t best_base = static_cast(best.base); + if (i >= best_base && i < (best_base + best.len)) + { + if (i == best_base) + *dst++ = ':'; + continue; + } } /// Are we following an initial run of 0x00s or any real hex? diff --git a/src/Common/getCurrentProcessFDCount.cpp b/src/Common/getCurrentProcessFDCount.cpp index 4c0f328c853..6217d92fbc1 100644 --- a/src/Common/getCurrentProcessFDCount.cpp +++ b/src/Common/getCurrentProcessFDCount.cpp @@ -7,10 +7,10 @@ #include -int getCurrentProcessFDCount() +Int64 getCurrentProcessFDCount() { namespace fs = std::filesystem; - int result = -1; + Int64 result = -1; #if defined(OS_LINUX) || defined(OS_DARWIN) using namespace DB; diff --git a/src/Common/getCurrentProcessFDCount.h b/src/Common/getCurrentProcessFDCount.h index 583b99f6c13..f6273580df7 100644 --- a/src/Common/getCurrentProcessFDCount.h +++ b/src/Common/getCurrentProcessFDCount.h @@ -1,5 +1,7 @@ #pragma once +#include + /// Get current process file descriptor count /// @return -1 os doesn't support "lsof" command or some error occurs. -int getCurrentProcessFDCount(); +Int64 getCurrentProcessFDCount(); diff --git a/src/Common/intExp.h b/src/Common/intExp.h index 3529990ef3b..4d39a56670d 100644 --- a/src/Common/intExp.h +++ b/src/Common/intExp.h @@ -232,7 +232,7 @@ template constexpr inline T intExp10OfSize(int x) { if constexpr (sizeof(T) <= 8) - return intExp10(x); + return static_cast(intExp10(x)); else if constexpr (sizeof(T) <= 16) return common::exp10_i128(x); else diff --git a/src/Common/mysqlxx/Value.cpp b/src/Common/mysqlxx/Value.cpp index 85b63b722a2..6954080f864 100644 --- a/src/Common/mysqlxx/Value.cpp +++ b/src/Common/mysqlxx/Value.cpp @@ -124,7 +124,7 @@ double Value::readFloatText(const char * buf, size_t length) const case 'E': { ++buf; - Int32 exponent = readIntText(buf, end - buf); + Int32 exponent = static_cast(readIntText(buf, end - buf)); x *= preciseExp10(exponent); if (negative) x = -x; diff --git a/src/Common/mysqlxx/mysqlxx/Row.h b/src/Common/mysqlxx/mysqlxx/Row.h index 6ef40ff2060..5690389d1cf 100644 --- a/src/Common/mysqlxx/mysqlxx/Row.h +++ b/src/Common/mysqlxx/mysqlxx/Row.h @@ -44,9 +44,9 @@ public: /** Получить значение по индексу. * Здесь используется int, а не unsigned, чтобы не было неоднозначности с тем же методом, принимающим const char *. */ - Value operator[] (int n) const + Value operator[] (size_t n) const { - if (unlikely(static_cast(n) >= res->getNumFields())) + if (unlikely(n >= res->getNumFields())) throw Exception("Index of column is out of range."); return Value(row[n], lengths[n], res); } diff --git a/src/Common/mysqlxx/mysqlxx/Value.h b/src/Common/mysqlxx/mysqlxx/Value.h index 46fcee0753f..892e5a19d93 100644 --- a/src/Common/mysqlxx/mysqlxx/Value.h +++ b/src/Common/mysqlxx/mysqlxx/Value.h @@ -242,8 +242,8 @@ template <> inline unsigned char Value::get() cons template <> inline char8_t Value::get() const { return getUInt(); } template <> inline short Value::get() const { return getInt(); } /// NOLINT template <> inline unsigned short Value::get() const { return getUInt(); } /// NOLINT -template <> inline int Value::get() const { return getInt(); } -template <> inline unsigned int Value::get() const { return getUInt(); } +template <> inline int Value::get() const { return static_cast(getInt()); } +template <> inline unsigned int Value::get() const { return static_cast(getUInt()); } template <> inline long Value::get() const { return getInt(); } /// NOLINT template <> inline unsigned long Value::get() const { return getUInt(); } /// NOLINT template <> inline long long Value::get() const { return getInt(); } /// NOLINT diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index 8e9195f9842..439852907a1 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -68,14 +68,14 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob output_width = std::max(range_begin_width, range_end_width); if (leading_zeros) - oss_for_replacing << std::setfill('0') << std::setw(output_width); + oss_for_replacing << std::setfill('0') << std::setw(static_cast(output_width)); oss_for_replacing << range_begin; for (size_t i = range_begin + 1; i <= range_end; ++i) { oss_for_replacing << '|'; if (leading_zeros) - oss_for_replacing << std::setfill('0') << std::setw(output_width); + oss_for_replacing << std::setfill('0') << std::setw(static_cast(output_width)); oss_for_replacing << i; } } diff --git a/src/Common/parseRemoteDescription.cpp b/src/Common/parseRemoteDescription.cpp index 1f614945491..c8434b0993e 100644 --- a/src/Common/parseRemoteDescription.cpp +++ b/src/Common/parseRemoteDescription.cpp @@ -82,8 +82,8 @@ std::vector parseRemoteDescription(const String & description, size_t l, /// Either the numeric interval (8..10) or equivalent expression in brackets if (description[i] == '{') { - int cnt = 1; - int last_dot = -1; /// The rightmost pair of points, remember the index of the right of the two + ssize_t cnt = 1; + ssize_t last_dot = -1; /// The rightmost pair of points, remember the index of the right of the two size_t m; std::vector buffer; bool have_splitter = false; diff --git a/src/Common/safe_cast.h b/src/Common/safe_cast.h new file mode 100644 index 00000000000..133808ca259 --- /dev/null +++ b/src/Common/safe_cast.h @@ -0,0 +1,22 @@ +#pragma once + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +template +To safe_cast(From from) +{ + constexpr auto max = std::numeric_limits::max(); + if (from > max) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Overflow ({} > {})", from, max); + return static_cast(from); +} + +} diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index aca17ae4f93..6ae934b2296 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -58,7 +58,7 @@ FailuresCount countFailures(const ::testing::TestResult & test_result) const size_t count = test_result.total_part_count(); for (size_t i = 0; i < count; ++i) { - const auto & part = test_result.GetTestPartResult(i); + const auto & part = test_result.GetTestPartResult(static_cast(i)); if (part.nonfatally_failed()) { ++failures.non_fatal; diff --git a/src/Common/tests/gtest_hash_table.cpp b/src/Common/tests/gtest_hash_table.cpp index b06ee5a666e..fd0b2495fde 100644 --- a/src/Common/tests/gtest_hash_table.cpp +++ b/src/Common/tests/gtest_hash_table.cpp @@ -216,27 +216,27 @@ TEST(HashTable, Erase) using Cont = HashSet, HashTableGrowerWithPrecalculation<1>>; Cont cont; - for (size_t i = 0; i < 5000; ++i) + for (int i = 0; i < 5000; ++i) { cont.insert(i); } - for (size_t i = 0; i < 2500; ++i) + for (int i = 0; i < 2500; ++i) { cont.erase(i); } - for (size_t i = 5000; i < 10000; ++i) + for (int i = 5000; i < 10000; ++i) { cont.insert(i); } - for (size_t i = 5000; i < 10000; ++i) + for (int i = 5000; i < 10000; ++i) { cont.erase(i); } - for (size_t i = 2500; i < 5000; ++i) + for (int i = 2500; i < 5000; ++i) { cont.erase(i); } diff --git a/src/Common/tests/gtest_lru_hash_map.cpp b/src/Common/tests/gtest_lru_hash_map.cpp index 562ee667b7b..f45a503be43 100644 --- a/src/Common/tests/gtest_lru_hash_map.cpp +++ b/src/Common/tests/gtest_lru_hash_map.cpp @@ -26,7 +26,7 @@ void testInsert(size_t elements_to_insert_size, size_t map_size) std::vector expected; for (size_t i = 0; i < elements_to_insert_size; ++i) - map.insert(i, i); + map.insert(static_cast(i), static_cast(i)); for (size_t i = elements_to_insert_size - map_size; i < elements_to_insert_size; ++i) expected.emplace_back(i); diff --git a/src/Common/waitForPid.cpp b/src/Common/waitForPid.cpp index 73d88c68adb..2cf80de644d 100644 --- a/src/Common/waitForPid.cpp +++ b/src/Common/waitForPid.cpp @@ -54,7 +54,7 @@ namespace DB static int syscall_pidfd_open(pid_t pid) { - return syscall(SYS_pidfd_open, pid, 0); + return static_cast(syscall(SYS_pidfd_open, pid, 0)); } static bool supportsPidFdOpen() @@ -170,7 +170,8 @@ bool waitForPid(pid_t pid, size_t timeout_in_seconds) /// If timeout is positive try waitpid without block in loop until /// process is normally terminated or waitpid return error - int timeout_in_ms = timeout_in_seconds * 1000; + /// NOTE: timeout casted to int, since poll() accept int for timeout + int timeout_in_ms = static_cast(timeout_in_seconds * 1000); while (timeout_in_ms > 0) { int waitpid_res = HANDLE_EINTR(waitpid(pid, &status, WNOHANG)); diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 9101caf568e..0492b7faec5 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -279,7 +279,7 @@ static void readHeaderAndGetCodec(const char * compressed_buffer, size_t size_de void CompressedReadBufferBase::decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum) { readHeaderAndGetCodec(compressed_buffer, size_decompressed, codec, allow_different_codecs); - codec->decompress(compressed_buffer, size_compressed_without_checksum, to); + codec->decompress(compressed_buffer, static_cast(size_compressed_without_checksum), to); } void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum) @@ -300,7 +300,7 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d to = BufferBase::Buffer(compressed_buffer + header_size, compressed_buffer + size_compressed_without_checksum); } else - codec->decompress(compressed_buffer, size_compressed_without_checksum, to.begin()); + codec->decompress(compressed_buffer, static_cast(size_compressed_without_checksum), to.begin()); } void CompressedReadBufferBase::flushAsynchronousDecompressRequests() const diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index 6c1dbd9e00c..82beeea37cd 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include "CompressedWriteBuffer.h" @@ -22,7 +23,8 @@ void CompressedWriteBuffer::nextImpl() if (!offset()) return; - size_t decompressed_size = offset(); + chassert(offset() <= INT_MAX); + UInt32 decompressed_size = static_cast(offset()); UInt32 compressed_reserve_size = codec->getCompressedReserveSize(decompressed_size); /** During compression we need buffer with capacity >= compressed_reserve_size + CHECKSUM_SIZE. diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 816f242672a..c1278cb88de 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -353,7 +353,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest) writer.flush(); - return (dest - dest_start) + (writer.count() + 7) / 8; + return static_cast((dest - dest_start) + (writer.count() + 7) / 8); } template @@ -414,7 +414,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, if (write_spec.data_bits != 0) { const UInt8 sign = reader.readBit(); - double_delta = reader.readBits(write_spec.data_bits - 1) + 1; + double_delta = static_cast(reader.readBits(write_spec.data_bits - 1) + 1); if (sign) { /// It's well defined for unsigned data types. diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index a9550c9e28d..02b640ae402 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -7,6 +7,7 @@ #include #include #include +#include // This depends on BoringSSL-specific API, notably . #if USE_SSL @@ -480,7 +481,8 @@ UInt32 CompressionCodecEncrypted::doCompressData(const char * source, UInt32 sou if (out_len != source_size + tag_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't encrypt data, length after encryption {} is wrong, expected {}", out_len, source_size + tag_size); - return out_len + keyid_size + nonce_size; + size_t out_size = out_len + keyid_size + nonce_size; + return safe_cast(out_size); } void CompressionCodecEncrypted::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const diff --git a/src/Compression/CompressionCodecFPC.cpp b/src/Compression/CompressionCodecFPC.cpp index 247eb73b65b..48eba210b60 100644 --- a/src/Compression/CompressionCodecFPC.cpp +++ b/src/Compression/CompressionCodecFPC.cpp @@ -453,9 +453,9 @@ UInt32 CompressionCodecFPC::doCompressData(const char * source, UInt32 source_si switch (float_width) { case sizeof(Float64): - return HEADER_SIZE + FPCOperation(destination, level).encode(src); + return static_cast(HEADER_SIZE + FPCOperation(destination, level).encode(src)); case sizeof(Float32): - return HEADER_SIZE + FPCOperation(destination, level).encode(src); + return static_cast(HEADER_SIZE + FPCOperation(destination, level).encode(src)); default: break; } diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index 0ca3e5660e0..0da6ff46dbc 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -259,7 +259,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest, writer.flush(); - return (dest - dest_start) + (writer.count() + 7) / 8; + return static_cast((dest - dest_start) + (writer.count() + 7) / 8); } template @@ -320,7 +320,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) ErrorCodes::CANNOT_DECOMPRESS); } - xored_data = reader.readBits(curr_xored_info.data_bits); + xored_data = static_cast(reader.readBits(curr_xored_info.data_bits)); xored_data <<= curr_xored_info.trailing_zero_bits; curr_value = prev_value ^ xored_data; } diff --git a/src/Compression/CompressionCodecLZ4.cpp b/src/Compression/CompressionCodecLZ4.cpp index 12f138dc95a..5b93e7ef60f 100644 --- a/src/Compression/CompressionCodecLZ4.cpp +++ b/src/Compression/CompressionCodecLZ4.cpp @@ -134,7 +134,7 @@ void registerCodecLZ4HC(CompressionCodecFactory & factory) if (!literal) throw Exception("LZ4HC codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); - level = literal->value.safeGet(); + level = static_cast(literal->value.safeGet()); } return std::make_shared(level); diff --git a/src/Compression/CompressionCodecMultiple.cpp b/src/Compression/CompressionCodecMultiple.cpp index 8ad054673e1..628c2d97d86 100644 --- a/src/Compression/CompressionCodecMultiple.cpp +++ b/src/Compression/CompressionCodecMultiple.cpp @@ -48,7 +48,7 @@ UInt32 CompressionCodecMultiple::getMaxCompressedDataSize(UInt32 uncompressed_si compressed_size = codec->getCompressedReserveSize(compressed_size); /// TotalCodecs ByteForEachCodec data - return sizeof(UInt8) + codecs.size() + compressed_size; + return static_cast(sizeof(UInt8) + codecs.size() + compressed_size); } UInt32 CompressionCodecMultiple::doCompressData(const char * source, UInt32 source_size, char * dest) const @@ -73,7 +73,7 @@ UInt32 CompressionCodecMultiple::doCompressData(const char * source, UInt32 sour memcpy(&dest[1 + codecs.size()], uncompressed_buf.data(), source_size); - return 1 + codecs.size() + source_size; + return static_cast(1 + codecs.size() + source_size); } void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 decompressed_size) const diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index bfcebad9676..cc8ce24476f 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -550,7 +550,7 @@ void decompressData(const char * src, UInt32 bytes_size, char * dst, UInt32 unco UInt32 num_bits = getValuableBitsNumber(min, max); if (!num_bits) { - T min_value = min; + T min_value = static_cast(min); for (UInt32 i = 0; i < num_elements; ++i, dst += sizeof(T)) unalignedStore(dst, min_value); return; @@ -572,14 +572,14 @@ void decompressData(const char * src, UInt32 bytes_size, char * dst, UInt32 unco T upper_max [[maybe_unused]] = 0; T sign_bit [[maybe_unused]] = 0; if (num_bits < 64) - upper_min = static_cast(min) >> num_bits << num_bits; + upper_min = static_cast(static_cast(min) >> num_bits << num_bits); if constexpr (is_signed_v) { if (min < 0 && max >= 0 && num_bits < 64) { - sign_bit = 1ull << (num_bits - 1); - upper_max = static_cast(max) >> num_bits << num_bits; + sign_bit = static_cast(1ull << (num_bits - 1)); + upper_max = static_cast(static_cast(max) >> num_bits << num_bits); } } diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index b47c8c4b080..f1c50840e54 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -65,7 +65,7 @@ void CompressionCodecZSTD::updateHash(SipHash & hash) const UInt32 CompressionCodecZSTD::getMaxCompressedDataSize(UInt32 uncompressed_size) const { - return ZSTD_compressBound(uncompressed_size); + return static_cast(ZSTD_compressBound(uncompressed_size)); } @@ -84,7 +84,7 @@ UInt32 CompressionCodecZSTD::doCompressData(const char * source, UInt32 source_s if (ZSTD_isError(compressed_size)) throw Exception("Cannot compress block with ZSTD: " + std::string(ZSTD_getErrorName(compressed_size)), ErrorCodes::CANNOT_COMPRESS); - return compressed_size; + return static_cast(compressed_size); } @@ -124,18 +124,20 @@ void registerCodecZSTD(CompressionCodecFactory & factory) if (!literal) throw Exception("ZSTD codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); - level = literal->value.safeGet(); + level = static_cast(literal->value.safeGet()); if (level > ZSTD_maxCLevel()) - throw Exception( - "ZSTD codec can't have level more than " + toString(ZSTD_maxCLevel()) + ", given " + toString(level), - ErrorCodes::ILLEGAL_CODEC_PARAMETER); + { + throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, + "ZSTD codec can't have level more than {}, given {}", + ZSTD_maxCLevel(), level); + } if (arguments->children.size() > 1) { const auto * window_literal = children[1]->as(); if (!window_literal) throw Exception("ZSTD codec second argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); - const int window_log = window_literal->value.safeGet(); + const int window_log = static_cast(window_literal->value.safeGet()); ZSTD_bounds window_log_bounds = ZSTD_cParam_getBounds(ZSTD_c_windowLog); if (ZSTD_isError(window_log_bounds.error)) diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index 2df3edb23ad..e5e50cd5320 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -391,7 +391,7 @@ CodecTestSequence generateSeq(Generator gen, const char* gen_name, B Begin = 0, for (auto i = Begin; i < End; i += direction) { - const T v = gen(static_cast(i)); + const T v = static_cast(gen(i)); unalignedStoreLE(write_pos, v); write_pos += sizeof(v); @@ -464,13 +464,15 @@ void testTranscoding(Timer & timer, ICompressionCodec & codec, const CodecTestSe { const auto & source_data = test_sequence.serialized_data; - const UInt32 encoded_max_size = codec.getCompressedReserveSize(source_data.size()); + const UInt32 encoded_max_size = codec.getCompressedReserveSize( + static_cast(source_data.size())); PODArray encoded(encoded_max_size); timer.start(); assert(source_data.data() != nullptr); // Codec assumes that source buffer is not null. - const UInt32 encoded_size = codec.compress(source_data.data(), source_data.size(), encoded.data()); + const UInt32 encoded_size = codec.compress( + source_data.data(), static_cast(source_data.size()), encoded.data()); timer.report("encoding"); encoded.resize(encoded_size); @@ -478,7 +480,8 @@ void testTranscoding(Timer & timer, ICompressionCodec & codec, const CodecTestSe PODArray decoded(source_data.size()); timer.start(); - const UInt32 decoded_size = codec.decompress(encoded.data(), encoded.size(), decoded.data()); + const UInt32 decoded_size = codec.decompress( + encoded.data(), static_cast(encoded.size()), decoded.data()); timer.report("decoding"); decoded.resize(decoded_size); @@ -542,10 +545,12 @@ TEST_P(CodecTestCompatibility, Encoding) const auto & source_data = data_sequence.serialized_data; // Just encode the data with codec - const UInt32 encoded_max_size = codec->getCompressedReserveSize(source_data.size()); + const UInt32 encoded_max_size = codec->getCompressedReserveSize( + static_cast(source_data.size())); PODArray encoded(encoded_max_size); - const UInt32 encoded_size = codec->compress(source_data.data(), source_data.size(), encoded.data()); + const UInt32 encoded_size = codec->compress( + source_data.data(), static_cast(source_data.size()), encoded.data()); encoded.resize(encoded_size); SCOPED_TRACE(::testing::Message("encoded: ") << AsHexString(encoded)); @@ -560,7 +565,8 @@ TEST_P(CodecTestCompatibility, Decoding) const auto codec = makeCodec(codec_spec.codec_statement, expected.data_type); PODArray decoded(expected.serialized_data.size()); - const UInt32 decoded_size = codec->decompress(encoded_data.c_str(), encoded_data.size(), decoded.data()); + const UInt32 decoded_size = codec->decompress( + encoded_data.c_str(), static_cast(encoded_data.size()), decoded.data()); decoded.resize(decoded_size); ASSERT_TRUE(EqualByteContainers(expected.data_type->getSizeOfValueInMemory(), expected.serialized_data, decoded)); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 1c8959379da..a097cb57bc6 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -284,8 +284,9 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co params.client_req_timeout_ = getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds(), "operation_timeout_ms", log); params.auto_forwarding_ = coordination_settings->auto_forwarding; - params.auto_forwarding_req_timeout_ - = std::max(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, std::numeric_limits::max()); + params.auto_forwarding_req_timeout_ = std::max( + static_cast(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2), + std::numeric_limits::max()); params.auto_forwarding_req_timeout_ = getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, "operation_timeout_ms", log); params.max_append_size_ diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index a30a32b5735..d6b75baa90e 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -922,7 +922,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr stat.version = 0; stat.aversion = 0; stat.cversion = 0; - stat.dataLength = request.data.length(); + stat.dataLength = static_cast(request.data.length()); stat.ephemeralOwner = request.is_ephemeral ? session_id : 0; new_deltas.emplace_back( @@ -1222,7 +1222,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce value.stat.version++; value.stat.mzxid = zxid; value.stat.mtime = time; - value.stat.dataLength = data.length(); + value.stat.dataLength = static_cast(data.length()); value.setData(data); }, request.version}); diff --git a/src/Coordination/SummingStateMachine.cpp b/src/Coordination/SummingStateMachine.cpp index ae3d2b06d75..4151b727744 100644 --- a/src/Coordination/SummingStateMachine.cpp +++ b/src/Coordination/SummingStateMachine.cpp @@ -71,10 +71,10 @@ void SummingStateMachine::createSnapshotInternal(nuraft::snapshot & s) snapshots[s.get_last_log_idx()] = ctx; // Maintain last 3 snapshots only. - int num = snapshots.size(); + ssize_t num = snapshots.size(); auto entry = snapshots.begin(); - for (int ii = 0; ii < num - MAX_SNAPSHOTS; ++ii) + for (ssize_t ii = 0; ii < num - MAX_SNAPSHOTS; ++ii) { if (entry == snapshots.end()) break; diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 6702c4cc718..82fce5297a1 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -119,7 +119,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L Coordination::read(node.stat.pzxid, in); if (!path.empty()) { - node.stat.dataLength = node.getData().length(); + node.stat.dataLength = static_cast(node.getData().length()); node.seq_num = node.stat.cversion; storage.container.insertOrReplace(path, node); diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index b1d27d4541d..fa4c42dd82a 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -941,7 +941,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot) EXPECT_EQ(itr->active_in_map, true); itr = std::next(itr); EXPECT_EQ(itr, map_snp.end()); - for (size_t i = 0; i < 5; ++i) + for (int i = 0; i < 5; ++i) { EXPECT_TRUE(map_snp.insert("/hello" + std::to_string(i), i).second); } @@ -1982,7 +1982,7 @@ TEST_P(CoordinationTest, TestListRequestTypes) KeeperStorage storage{500, "", keeper_context}; - int64_t zxid = 0; + int32_t zxid = 0; static constexpr std::string_view test_path = "/list_request_type/node"; diff --git a/src/Core/DecimalComparison.h b/src/Core/DecimalComparison.h index 530722a2519..953c98f80b4 100644 --- a/src/Core/DecimalComparison.h +++ b/src/Core/DecimalComparison.h @@ -230,7 +230,7 @@ private: if constexpr (is_decimal) y = b.value; else - y = b; + y = static_cast(b); if constexpr (_check_overflow) { diff --git a/src/Core/DecimalFunctions.h b/src/Core/DecimalFunctions.h index 0f2158fb83b..263f78af5cc 100644 --- a/src/Core/DecimalFunctions.h +++ b/src/Core/DecimalFunctions.h @@ -241,7 +241,7 @@ inline DecimalComponents split(const DecimalType & decimal, UInt32 * If scale is to big, result is undefined. */ template -inline typename DecimalType::NativeType getWholePart(const DecimalType & decimal, size_t scale) +inline typename DecimalType::NativeType getWholePart(const DecimalType & decimal, UInt32 scale) { if (scale == 0) return decimal.value; @@ -273,7 +273,7 @@ inline typename DecimalType::NativeType getFractionalPartWithScaleMultiplier( * If scale is to big, result is undefined. */ template -inline typename DecimalType::NativeType getFractionalPart(const DecimalType & decimal, size_t scale) +inline typename DecimalType::NativeType getFractionalPart(const DecimalType & decimal, UInt32 scale) { if (scale == 0) return 0; @@ -283,7 +283,7 @@ inline typename DecimalType::NativeType getFractionalPart(const DecimalType & de /// Decimal to integer/float conversion template -ReturnType convertToImpl(const DecimalType & decimal, size_t scale, To & result) +ReturnType convertToImpl(const DecimalType & decimal, UInt32 scale, To & result) { using DecimalNativeType = typename DecimalType::NativeType; static constexpr bool throw_exception = std::is_void_v; @@ -334,7 +334,7 @@ ReturnType convertToImpl(const DecimalType & decimal, size_t scale, To & result) template -To convertTo(const DecimalType & decimal, size_t scale) +To convertTo(const DecimalType & decimal, UInt32 scale) { To result; convertToImpl(decimal, scale, result); @@ -342,7 +342,7 @@ To convertTo(const DecimalType & decimal, size_t scale) } template -bool tryConvertTo(const DecimalType & decimal, size_t scale, To & result) +bool tryConvertTo(const DecimalType & decimal, UInt32 scale, To & result) { return convertToImpl(decimal, scale, result); } diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 87566eade54..6f3ac1b40e9 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -161,7 +161,7 @@ namespace MySQLReplication /// https://dev.mysql.com/doc/internals/en/table-map-event.html void TableMapEvent::parseImpl(ReadBuffer & payload) { - column_count = readLengthEncodedNumber(payload); + column_count = static_cast(readLengthEncodedNumber(payload)); for (auto i = 0U; i < column_count; ++i) { UInt8 v = 0x00; @@ -283,7 +283,7 @@ namespace MySQLReplication void RowsEvent::parseImpl(ReadBuffer & payload) { - number_columns = readLengthEncodedNumber(payload); + number_columns = static_cast(readLengthEncodedNumber(payload)); size_t columns_bitmap_size = (number_columns + 7) / 8; switch (header.type) { @@ -494,7 +494,7 @@ namespace MySQLReplication readBigEndianStrict(payload, reinterpret_cast(&uintpart), 6); intpart = uintpart - 0x800000000000L; ltime = intpart; - frac = std::abs(intpart % (1L << 24)); + frac = static_cast(std::abs(intpart % (1L << 24))); break; } default: @@ -536,7 +536,7 @@ namespace MySQLReplication readBigEndianStrict(payload, reinterpret_cast(&val), 5); readTimeFractionalPart(payload, fsp, meta); - UInt32 year_month = readBits(val, 1, 17, 40); + UInt32 year_month = static_cast(readBits(val, 1, 17, 40)); time_t date_time = DateLUT::instance().makeDateTime( year_month / 13, year_month % 13, readBits(val, 18, 5, 40) , readBits(val, 23, 5, 40), readBits(val, 28, 6, 40), readBits(val, 34, 6, 40) @@ -625,7 +625,7 @@ namespace MySQLReplication { UInt32 val = 0; readBigEndianStrict(payload, reinterpret_cast(&val), 4); - res *= intExp10OfSize(digits_per_integer); + res *= intExp10OfSize(static_cast(digits_per_integer)); res += (val ^ mask); } } @@ -638,7 +638,7 @@ namespace MySQLReplication { UInt32 val = 0; readBigEndianStrict(payload, reinterpret_cast(&val), 4); - res *= intExp10OfSize(digits_per_integer); + res *= intExp10OfSize(static_cast(digits_per_integer)); res += (val ^ mask); } @@ -651,7 +651,7 @@ namespace MySQLReplication if (to_read) //-V547 { readBigEndianStrict(payload, reinterpret_cast(&val), to_read); - res *= intExp10OfSize(compressed_decimals); + res *= intExp10OfSize(static_cast(compressed_decimals)); res += (val ^ (mask & compressed_integer_align_numbers[compressed_decimals])); } } diff --git a/src/Core/MySQL/PacketsReplication.cpp b/src/Core/MySQL/PacketsReplication.cpp index ec5e8868cc5..74c6ca2d81f 100644 --- a/src/Core/MySQL/PacketsReplication.cpp +++ b/src/Core/MySQL/PacketsReplication.cpp @@ -57,7 +57,7 @@ void BinlogDumpGTID::writePayloadImpl(WriteBuffer & buffer) const const UInt64 position = 4; buffer.write(reinterpret_cast(&position), 8); - UInt32 gtid_size = gtid_datas.size(); + UInt32 gtid_size = static_cast(gtid_datas.size()); buffer.write(reinterpret_cast(>id_size), 4); buffer.write(gtid_datas.data(), gtid_datas.size()); } diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index 61a0a91ab2e..e5f3a0f91c3 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -100,7 +100,7 @@ void insertPostgreSQLValue( readDateTimeText(time, in, assert_cast(data_type.get())->getTimeZone()); if (time < 0) time = 0; - assert_cast(column).insertValue(time); + assert_cast(column).insertValue(static_cast(time)); break; } case ExternalResultDescription::ValueType::vtDateTime64: diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index 908518eff5e..994494fc92f 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -336,7 +336,12 @@ public: Int32 size() const override { // message length part + (1 + sizes of other fields + 1) + null byte in the end of the message - return 4 + (1 + enum_to_string[severity].size() + 1) + (1 + sql_state.size() + 1) + (1 + message.size() + 1) + 1; + return static_cast( + 4 + + (1 + enum_to_string[severity].size() + 1) + + (1 + sql_state.size() + 1) + + (1 + message.size() + 1) + + 1); } MessageType getMessageType() const override @@ -518,7 +523,7 @@ public: Int32 size() const override { - return 4 + name.size() + 1 + value.size() + 1; + return static_cast(4 + name.size() + 1 + value.size() + 1); } MessageType getMessageType() const override @@ -633,7 +638,7 @@ public: // + object ID of the table (Int32 and always zero) + attribute number of the column (Int16 and always zero) // + type object id (Int32) + data type size (Int16) // + type modifier (Int32 and always -1) + format code (Int16) - return (name.size() + 1) + 4 + 2 + 4 + 2 + 4 + 2; + return static_cast((name.size() + 1) + 4 + 2 + 4 + 2 + 4 + 2); } }; @@ -682,7 +687,7 @@ public: Int32 size() const override { - return str.size(); + return static_cast(str.size()); } }; @@ -762,7 +767,7 @@ public: Int32 size() const override { - return 4 + value.size() + 1; + return static_cast(4 + value.size() + 1); } MessageType getMessageType() const override diff --git a/src/Core/tests/gtest_DecimalFunctions.cpp b/src/Core/tests/gtest_DecimalFunctions.cpp index 1712785488e..aeaf3a1fac9 100644 --- a/src/Core/tests/gtest_DecimalFunctions.cpp +++ b/src/Core/tests/gtest_DecimalFunctions.cpp @@ -28,7 +28,7 @@ class DecimalUtilsSplitAndCombineTest : public ::testing::TestWithParam void testSplit(const DecimalUtilsSplitAndCombineTestParam & param) { - const DecimalType decimal_value = param.decimal_value; + const DecimalType decimal_value(static_cast(param.decimal_value)); const auto & actual_components = DecimalUtils::split(decimal_value, param.scale); EXPECT_EQ(param.components.whole, actual_components.whole); @@ -39,21 +39,28 @@ template void testDecimalFromComponents(const DecimalUtilsSplitAndCombineTestParam & param) { EXPECT_EQ(param.decimal_value, - DecimalUtils::decimalFromComponents(param.components.whole, param.components.fractional, param.scale)); + DecimalUtils::decimalFromComponents( + static_cast(param.components.whole), + static_cast(param.components.fractional), + param.scale)); } template void testGetWhole(const DecimalUtilsSplitAndCombineTestParam & param) { EXPECT_EQ(param.components.whole, - DecimalUtils::getWholePart(DecimalType{param.decimal_value}, param.scale)); + DecimalUtils::getWholePart( + DecimalType{static_cast(param.decimal_value)}, + param.scale)); } template void testGetFractional(const DecimalUtilsSplitAndCombineTestParam & param) { EXPECT_EQ(param.components.fractional, - DecimalUtils::getFractionalPart(DecimalType{param.decimal_value}, param.scale)); + DecimalUtils::getFractionalPart( + DecimalType{static_cast(param.decimal_value)}, + param.scale)); } // Unfortunately typed parametrized tests () are not supported in this version of gtest, so I have to emulate by hand. diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 2dddfd6874a..758f85e688f 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -575,7 +575,8 @@ void BaseDaemon::closeFDs() { int max_fd = -1; #if defined(_SC_OPEN_MAX) - max_fd = sysconf(_SC_OPEN_MAX); + // fd cannot be > INT_MAX + max_fd = static_cast(sysconf(_SC_OPEN_MAX)); if (max_fd == -1) #endif max_fd = 256; /// bad fallback diff --git a/src/Daemon/SentryWriter.cpp b/src/Daemon/SentryWriter.cpp index a6033af6fe3..bb330162818 100644 --- a/src/Daemon/SentryWriter.cpp +++ b/src/Daemon/SentryWriter.cpp @@ -189,7 +189,7 @@ void SentryWriter::onFault(int sig, const std::string & error_message, const Sta sentry_value_set_by_key(sentry_frame, "filename", sentry_value_new_string(current_frame.file.value().c_str())); if (current_frame.line.has_value()) - sentry_value_set_by_key(sentry_frame, "lineno", sentry_value_new_int32(current_frame.line.value())); + sentry_value_set_by_key(sentry_frame, "lineno", sentry_value_new_int32(static_cast(current_frame.line.value()))); sentry_value_append(sentry_frames, sentry_frame); } diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index c58e186b980..aab328eaa33 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -102,7 +102,7 @@ bool DataTypeEnum::textCanContainOnlyValidUTF8() const if (pos + length > end) return false; - if (Poco::UTF8Encoding::isLegal(reinterpret_cast(pos), length)) + if (Poco::UTF8Encoding::isLegal(reinterpret_cast(pos), static_cast(length))) pos += length; else return false; diff --git a/src/DataTypes/Native.h b/src/DataTypes/Native.h index 40086b14a0c..9782c5d64e9 100644 --- a/src/DataTypes/Native.h +++ b/src/DataTypes/Native.h @@ -224,7 +224,7 @@ static inline std::pair nativeCastToCommon(llvm::I size_t rhs_bit_width = rhs->getType()->getIntegerBitWidth() + (!rhs_is_signed && lhs_is_signed); size_t max_bit_width = std::max(lhs_bit_width, rhs_bit_width); - common = b.getIntNTy(max_bit_width); + common = b.getIntNTy(static_cast(max_bit_width)); } else { diff --git a/src/DataTypes/Serializations/SerializationDateTime.cpp b/src/DataTypes/Serializations/SerializationDateTime.cpp index fd56c1baebd..7238d3ce190 100644 --- a/src/DataTypes/Serializations/SerializationDateTime.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime.cpp @@ -75,7 +75,7 @@ void SerializationDateTime::deserializeTextEscaped(IColumn & column, ReadBuffer readText(x, istr, settings, time_zone, utc_time_zone); if (x < 0) x = 0; - assert_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(static_cast(x)); } void SerializationDateTime::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -99,7 +99,9 @@ void SerializationDateTime::deserializeTextQuoted(IColumn & column, ReadBuffer & } if (x < 0) x = 0; - assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. + + /// It's important to do this at the end - for exception safety. + assert_cast(column).getData().push_back(static_cast(x)); } void SerializationDateTime::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -123,7 +125,7 @@ void SerializationDateTime::deserializeTextJSON(IColumn & column, ReadBuffer & i } if (x < 0) x = 0; - assert_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(static_cast(x)); } void SerializationDateTime::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -153,7 +155,7 @@ void SerializationDateTime::deserializeTextCSV(IColumn & column, ReadBuffer & is if (x < 0) x = 0; - assert_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(static_cast(x)); } } diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.cpp b/src/DataTypes/Serializations/SerializationInfoTuple.cpp index d0fa5572a48..6c326743e8a 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.cpp +++ b/src/DataTypes/Serializations/SerializationInfoTuple.cpp @@ -124,7 +124,7 @@ void SerializationInfoTuple::fromJSON(const Poco::JSON::Object & object) "Expected: {}, got: {}", elems.size(), subcolumns->size()); for (size_t i = 0; i < elems.size(); ++i) - elems[i]->fromJSON(*subcolumns->getObject(i)); + elems[i]->fromJSON(*subcolumns->getObject(static_cast(i))); } } diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index dfe0188c8e7..761adf3b765 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -385,13 +385,13 @@ namespace } else if (map[val] == 0 && val != zero_pos_value) { - map[val] = cur_pos; + map[val] = static_cast(cur_pos); ++cur_pos; } } else { - T shifted_val = val - dict_size; + T shifted_val = static_cast(val - dict_size); if (cur_overflowed_pos == 0) { zero_pos_overflowed_value = shifted_val; @@ -399,7 +399,7 @@ namespace } else if (overflow_map[shifted_val] == 0 && shifted_val != zero_pos_overflowed_value) { - overflow_map[shifted_val] = cur_overflowed_pos; + overflow_map[shifted_val] = static_cast(cur_overflowed_pos); ++cur_overflowed_pos; } } @@ -429,7 +429,7 @@ namespace if (val < dict_size) val = map[val]; else - val = overflow_map[val - dict_size] + cur_pos; + val = overflow_map[val - dict_size] + static_cast(cur_pos); } return {std::move(dictionary_map), std::move(additional_keys_map)}; diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index fee3cf1553e..87c6055c35a 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -447,8 +447,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types) /// For String and FixedString, or for different FixedStrings, the common type is String. /// No other types are compatible with Strings. TODO Enums? { - UInt32 have_string = type_ids.count(TypeIndex::String); - UInt32 have_fixed_string = type_ids.count(TypeIndex::FixedString); + size_t have_string = type_ids.count(TypeIndex::String); + size_t have_fixed_string = type_ids.count(TypeIndex::FixedString); if (have_string || have_fixed_string) { @@ -462,10 +462,10 @@ DataTypePtr getLeastSupertype(const DataTypes & types) /// For Date and DateTime/DateTime64, the common type is DateTime/DateTime64. No other types are compatible. { - UInt32 have_date = type_ids.count(TypeIndex::Date); - UInt32 have_date32 = type_ids.count(TypeIndex::Date32); - UInt32 have_datetime = type_ids.count(TypeIndex::DateTime); - UInt32 have_datetime64 = type_ids.count(TypeIndex::DateTime64); + size_t have_date = type_ids.count(TypeIndex::Date); + size_t have_date32 = type_ids.count(TypeIndex::Date32); + size_t have_datetime = type_ids.count(TypeIndex::DateTime); + size_t have_datetime64 = type_ids.count(TypeIndex::DateTime64); if (have_date || have_date32 || have_datetime || have_datetime64) { @@ -526,23 +526,21 @@ DataTypePtr getLeastSupertype(const DataTypes & types) /// Decimals { - UInt32 have_decimal32 = type_ids.count(TypeIndex::Decimal32); - UInt32 have_decimal64 = type_ids.count(TypeIndex::Decimal64); - UInt32 have_decimal128 = type_ids.count(TypeIndex::Decimal128); + size_t have_decimal32 = type_ids.count(TypeIndex::Decimal32); + size_t have_decimal64 = type_ids.count(TypeIndex::Decimal64); + size_t have_decimal128 = type_ids.count(TypeIndex::Decimal128); if (have_decimal32 || have_decimal64 || have_decimal128) { - UInt32 num_supported = have_decimal32 + have_decimal64 + have_decimal128; + size_t num_supported = have_decimal32 + have_decimal64 + have_decimal128; std::vector int_ids = {TypeIndex::Int8, TypeIndex::UInt8, TypeIndex::Int16, TypeIndex::UInt16, - TypeIndex::Int32, TypeIndex::UInt32, TypeIndex::Int64, TypeIndex::UInt64}; - std::vector num_ints(int_ids.size(), 0); + TypeIndex::Int32, TypeIndex::UInt32, TypeIndex::Int64, TypeIndex::UInt64}; TypeIndex max_int = TypeIndex::Nothing; for (size_t i = 0; i < int_ids.size(); ++i) { - UInt32 num = type_ids.count(int_ids[i]); - num_ints[i] = num; + size_t num = type_ids.count(int_ids[i]); num_supported += num; if (num) max_int = int_ids[i]; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 796142884a3..88aa086fe65 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -716,8 +716,12 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromStorage(const String & table_name, cons auto ast_storage = std::make_shared(); ast_storage->set(ast_storage->engine, ast_engine); - auto create_table_query = DB::getCreateQueryFromStorage(storage, ast_storage, false, - getContext()->getSettingsRef().max_parser_depth, throw_on_error); + unsigned max_parser_depth = static_cast(getContext()->getSettingsRef().max_parser_depth); + auto create_table_query = DB::getCreateQueryFromStorage(storage, + ast_storage, + false, + max_parser_depth, + throw_on_error); create_table_query->set(create_table_query->as()->comment, std::make_shared("SYSTEM TABLE is built on the fly.")); diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 80301732ff8..5f59f6497e2 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -164,8 +164,13 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, Context std::erase_if(storage_children, [&](const ASTPtr & element) { return element.get() == ast_storage->settings; }); ast_storage->settings = nullptr; } - auto create_table_query = DB::getCreateQueryFromStorage(storage, table_storage_define, true, - getContext()->getSettingsRef().max_parser_depth, throw_on_error); + + unsigned max_parser_depth = static_cast(getContext()->getSettingsRef().max_parser_depth); + auto create_table_query = DB::getCreateQueryFromStorage(storage, + table_storage_define, + true, + max_parser_depth, + throw_on_error); return create_table_query; } diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 604dc220fed..ed9199a359f 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -478,8 +478,9 @@ static inline UInt32 randomNumber() { std::mt19937 rng; rng.seed(std::random_device()()); - std::uniform_int_distribution dist6(std::numeric_limits::min(), std::numeric_limits::max()); - return dist6(rng); + std::uniform_int_distribution dist6( + std::numeric_limits::min(), std::numeric_limits::max()); + return static_cast(dist6(rng)); } bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & metadata) @@ -679,11 +680,11 @@ static void writeFieldsToColumn( if (write_data_to_null_map(value, index)) { if (value.getType() == Field::Types::UInt64) - casted_int32_column->insertValue(value.get()); + casted_int32_column->insertValue(static_cast(value.get())); else if (value.getType() == Field::Types::Int64) { /// For MYSQL_TYPE_INT24 - const Int32 & num = value.get(); + const Int32 & num = static_cast(value.get()); casted_int32_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num); } else diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index 44a392ce1f2..83c5ebe00d3 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -192,8 +192,10 @@ ASTPtr DatabaseSQLite::getCreateTableQueryImpl(const String & table_name, Contex /// Add table_name to engine arguments storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 1, std::make_shared(table_id.table_name)); + unsigned max_parser_depth = static_cast(getContext()->getSettingsRef().max_parser_depth); auto create_table_query = DB::getCreateQueryFromStorage(storage, table_storage_define, true, - getContext()->getSettingsRef().max_parser_depth, throw_on_error); + max_parser_depth, + throw_on_error); return create_table_query; } diff --git a/src/Dictionaries/CassandraSource.cpp b/src/Dictionaries/CassandraSource.cpp index fd5982443fa..878921c53d0 100644 --- a/src/Dictionaries/CassandraSource.cpp +++ b/src/Dictionaries/CassandraSource.cpp @@ -32,7 +32,7 @@ CassandraSource::CassandraSource( , has_more_pages(cass_true) { description.init(sample_block); - cassandraCheck(cass_statement_set_paging_size(statement, max_block_size)); + cassandraCheck(cass_statement_set_paging_size(statement, static_cast(max_block_size))); } void CassandraSource::insertValue(IColumn & column, ValueType type, const CassValue * cass_value) diff --git a/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp b/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp index f91bbaa12a6..68bd6142416 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp @@ -31,7 +31,7 @@ bool RegionsHierarchyFormatReader::readNext(RegionEntry & entry) UInt64 population_big = 0; DB::readIntText(population_big, *input); population = population_big > std::numeric_limits::max() ? std::numeric_limits::max() - : population_big; + : static_cast(population_big); } DB::assertChar('\n', *input); diff --git a/src/Dictionaries/RedisSource.cpp b/src/Dictionaries/RedisSource.cpp index 4208d5fa63b..50a73e52010 100644 --- a/src/Dictionaries/RedisSource.cpp +++ b/src/Dictionaries/RedisSource.cpp @@ -109,7 +109,7 @@ namespace DB readDateTimeText(time, in); if (time < 0) time = 0; - assert_cast(column).insertValue(time); + assert_cast(column).insertValue(static_cast(time)); break; } case ValueType::vtUUID: diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index 5f73352a4c9..428b4321ffd 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -675,7 +675,7 @@ public: pointers.push_back(&requests.back()); } - AIOContext aio_context(read_from_file_buffer_blocks_size); + AIOContext aio_context(static_cast(read_from_file_buffer_blocks_size)); PaddedPODArray processed(requests.size(), false); PaddedPODArray events; @@ -735,7 +735,8 @@ public: ++to_pop; /// add new io tasks - const int new_tasks_count = std::min(read_from_file_buffer_blocks_size - (to_push - to_pop), requests.size() - to_push); + const int new_tasks_count = static_cast(std::min( + read_from_file_buffer_blocks_size - (to_push - to_pop), requests.size() - to_push)); int pushed = 0; while (new_tasks_count > 0 && (pushed = io_submit(aio_context.ctx, new_tasks_count, &pointers[to_push])) <= 0) diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index ba041efe24a..899d06b4ed7 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -68,7 +68,7 @@ private: IAsynchronousReader & reader; - Int32 priority; + Int64 priority; std::shared_ptr impl; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index 56cc20098ba..dc4898559c0 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -67,14 +67,6 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf) } } -void DiskObjectStorageMetadata::createFromSingleObject(const std::string & relative_path, size_t bytes_size, size_t ref_count_, bool read_only_) -{ - storage_objects.emplace_back(relative_path, bytes_size); - total_size = bytes_size; - ref_count = ref_count_; - read_only = read_only_; -} - void DiskObjectStorageMetadata::deserializeFromString(const std::string & data) { ReadBufferFromString buf(data); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h index 09e0f4ee85b..d3ea5795dd3 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h @@ -50,7 +50,6 @@ public: void deserialize(ReadBuffer & buf); void deserializeFromString(const std::string & data); - void createFromSingleObject(const std::string & relative_path, size_t bytes_size, size_t ref_count_, bool is_read_only_); void serialize(WriteBuffer & buf, bool sync) const; std::string serializeToString() const; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 213f744d84f..e627a807bd8 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -482,7 +482,7 @@ void S3ObjectStorage::copyObjectMultipartImpl( part_request.SetBucket(dst_bucket); part_request.SetKey(dst_key); part_request.SetUploadId(multipart_upload_id); - part_request.SetPartNumber(part_number); + part_request.SetPartNumber(static_cast(part_number)); part_request.SetCopySourceRange(fmt::format("bytes={}-{}", position, std::min(size, position + upload_part_size) - 1)); auto outcome = client_ptr->UploadPartCopy(part_request); @@ -515,7 +515,7 @@ void S3ObjectStorage::copyObjectMultipartImpl( for (size_t i = 0; i < part_tags.size(); ++i) { Aws::S3::Model::CompletedPart part; - multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(i + 1)); + multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(static_cast(i) + 1)); } req.SetMultipartUpload(multipart_upload); diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 1635cb5c552..e61987163d2 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -116,7 +116,8 @@ std::unique_ptr getClient(const Poco::Util::AbstractConfigura { S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( config.getString(config_prefix + ".region", ""), - context->getRemoteHostFilter(), context->getGlobalContext()->getSettingsRef().s3_max_redirects, + context->getRemoteHostFilter(), + static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, /* for_disk_s3 = */ true); diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index b8702380aa7..a7ff065aca5 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -138,7 +138,7 @@ static String getCapnProtoFullTypeName(const capnp::Type & type) auto enum_schema = type.asEnum(); String enum_name = "Enum("; auto enumerants = enum_schema.getEnumerants(); - for (size_t i = 0; i != enumerants.size(); ++i) + for (unsigned i = 0; i != enumerants.size(); ++i) { enum_name += String(enumerants[i].getProto().getName()) + " = " + std::to_string(enumerants[i].getOrdinal()); if (i + 1 != enumerants.size()) diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 567a2a9ee98..2f56c4242e5 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -864,7 +864,7 @@ namespace case FieldTypeId::TYPE_ENUM: { write_function = [this](std::string_view str) { writeInt(stringToProtobufEnumValue(str)); }; - read_function = [this](PaddedPODArray & str) { protobufEnumValueToStringAppend(readInt(), str); }; + read_function = [this](PaddedPODArray & str) { protobufEnumValueToStringAppend(static_cast(readInt()), str); }; default_function = [this]() -> String { return field_descriptor.default_value_enum()->name(); }; break; } @@ -1029,7 +1029,7 @@ namespace case FieldTypeId::TYPE_ENUM: { this->write_function = [this](NumberType value) { writeInt(enumDataTypeValueToProtobufEnumValue(value)); }; - this->read_function = [this]() -> NumberType { return protobufEnumValueToEnumDataTypeValue(readInt()); }; + this->read_function = [this]() -> NumberType { return protobufEnumValueToEnumDataTypeValue(static_cast(readInt())); }; this->default_function = [this]() -> NumberType { return protobufEnumValueToEnumDataTypeValue(this->field_descriptor.default_value_enum()->number()); }; break; } @@ -1539,10 +1539,13 @@ namespace read_function = [this]() -> UInt32 { readStr(text_buffer); - return stringToDateTime(text_buffer, date_lut); + return static_cast(stringToDateTime(text_buffer, date_lut)); }; - default_function = [this]() -> UInt32 { return stringToDateTime(field_descriptor.default_value_string(), date_lut); }; + default_function = [this]() -> UInt32 + { + return static_cast(stringToDateTime(field_descriptor.default_value_string(), date_lut)); + }; break; } diff --git a/src/Functions/CRC.cpp b/src/Functions/CRC.cpp index 92f0130c19b..91b549873a3 100644 --- a/src/Functions/CRC.cpp +++ b/src/Functions/CRC.cpp @@ -15,7 +15,7 @@ struct CRCBase { for (size_t i = 0; i < 256; ++i) { - T c = i; + T c = static_cast(i); for (size_t j = 0; j < 8; ++j) c = c & 1 ? polynomial ^ (c >> 1) : c >> 1; tab[i] = c; @@ -58,7 +58,7 @@ struct CRC32ZLIBImpl static UInt32 makeCRC(const unsigned char *buf, size_t size) { - return crc32_z(0L, buf, size); + return static_cast(crc32_z(0L, buf, size)); } }; diff --git a/src/Functions/CustomWeekTransforms.h b/src/Functions/CustomWeekTransforms.h index b690463d456..d8745001c60 100644 --- a/src/Functions/CustomWeekTransforms.h +++ b/src/Functions/CustomWeekTransforms.h @@ -134,14 +134,17 @@ struct WeekTransformer void vector(const FromVectorType & vec_from, ToVectorType & vec_to, UInt8 week_mode, const DateLUTImpl & time_zone) const { + using ValueType = typename ToVectorType::value_type; size_t size = vec_from.size(); vec_to.resize(size); for (size_t i = 0; i < size; ++i) + { if constexpr (is_extended_result) - vec_to[i] = transform.executeExtendedResult(vec_from[i], week_mode, time_zone); + vec_to[i] = static_cast(transform.executeExtendedResult(vec_from[i], week_mode, time_zone)); else - vec_to[i] = transform.execute(vec_from[i], week_mode, time_zone); + vec_to[i] = static_cast(transform.execute(vec_from[i], week_mode, time_zone)); + } } private: diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index fa66cb2a891..b1d2979da85 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -107,11 +107,11 @@ struct ToStartOfDayImpl if (t.whole < 0 || (t.whole >= 0 && t.fractional < 0)) return 0; - return time_zone.toDate(std::min(t.whole, Int64(0xffffffff))); + return static_cast(time_zone.toDate(std::min(t.whole, Int64(0xffffffff)))); } static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { - return time_zone.toDate(t); + return static_cast(time_zone.toDate(t)); } static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone) { @@ -119,15 +119,12 @@ struct ToStartOfDayImpl return 0; auto date_time = time_zone.fromDayNum(ExtendedDayNum(d)); - if (date_time <= 0xffffffff) - return date_time; - else - return time_zone.toDate(0xffffffff); + return static_cast(date_time <= 0xffffffff ? date_time : time_zone.toDate(0xffffffff)); } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { auto date_time = time_zone.fromDayNum(ExtendedDayNum(d)); - return date_time < 0xffffffff ? date_time : time_zone.toDate(0xffffffff); + return static_cast(date_time < 0xffffffff ? date_time : time_zone.toDate(0xffffffff)); } static inline DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { @@ -251,7 +248,8 @@ struct ToStartOfQuarterImpl static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { - return t < 0 ? 0 : time_zone.toFirstDayNumOfQuarter(ExtendedDayNum(std::min(Int64(time_zone.toDayNum(t)), Int64(DATE_LUT_MAX_DAY_NUM)))); + return t < 0 ? 0 : time_zone.toFirstDayNumOfQuarter( + ExtendedDayNum(std::min(time_zone.toDayNum(t), DATE_LUT_MAX_DAY_NUM))); } static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { @@ -316,11 +314,11 @@ struct ToTimeImpl static UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { - return time_zone.toTime(t.whole) + 86400; + return static_cast(time_zone.toTime(t.whole) + 86400); } static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { - return time_zone.toTime(t) + 86400; + return static_cast(time_zone.toTime(t) + 86400); } static inline UInt32 execute(Int32, const DateLUTImpl &) { @@ -343,7 +341,7 @@ struct ToStartOfMinuteImpl if (t.whole < 0 || (t.whole >= 0 && t.fractional < 0)) return 0; - return time_zone.toStartOfMinute(std::min(t.whole, Int64(0xffffffff))); + return static_cast(time_zone.toStartOfMinute(std::min(t.whole, Int64(0xffffffff)))); } static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { @@ -541,7 +539,7 @@ struct ToStartOfFiveMinutesImpl static inline UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { - return time_zone.toStartOfFiveMinutes(t.whole); + return static_cast(time_zone.toStartOfFiveMinutes(t.whole)); } static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { @@ -573,7 +571,7 @@ struct ToStartOfTenMinutesImpl static inline UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { - return time_zone.toStartOfTenMinutes(t.whole); + return static_cast(time_zone.toStartOfTenMinutes(t.whole)); } static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { @@ -605,7 +603,7 @@ struct ToStartOfFifteenMinutesImpl static inline UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { - return time_zone.toStartOfFifteenMinutes(t.whole); + return static_cast(time_zone.toStartOfFifteenMinutes(t.whole)); } static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { @@ -638,7 +636,7 @@ struct TimeSlotImpl static inline UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl &) { - return t.whole / 1800 * 1800; + return static_cast(t.whole / 1800 * 1800); } static inline UInt32 execute(UInt32 t, const DateLUTImpl &) @@ -680,7 +678,7 @@ struct ToStartOfHourImpl if (t.whole < 0 || (t.whole >= 0 && t.fractional < 0)) return 0; - return time_zone.toStartOfHour(std::min(t.whole, Int64(0xffffffff))); + return static_cast(time_zone.toStartOfHour(std::min(t.whole, Int64(0xffffffff)))); } static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) @@ -1215,9 +1213,9 @@ struct ToRelativeHourNumImpl static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) - return time_zone.toStableRelativeHourNum(static_cast(t)); + return static_cast(time_zone.toStableRelativeHourNum(static_cast(t))); else - return time_zone.toRelativeHourNum(static_cast(t)); + return static_cast(time_zone.toRelativeHourNum(static_cast(t))); } static inline auto execute(Int32 d, const DateLUTImpl & time_zone) { @@ -1229,9 +1227,9 @@ struct ToRelativeHourNumImpl static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) - return time_zone.toStableRelativeHourNum(DayNum(d)); + return static_cast(time_zone.toStableRelativeHourNum(DayNum(d))); else - return time_zone.toRelativeHourNum(DayNum(d)); + return static_cast(time_zone.toRelativeHourNum(DayNum(d))); } using FactorTransform = ZeroTransform; @@ -1251,7 +1249,7 @@ struct ToRelativeMinuteNumImpl } static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { - return time_zone.toRelativeMinuteNum(static_cast(t)); + return static_cast(time_zone.toRelativeMinuteNum(static_cast(t))); } static inline auto execute(Int32 d, const DateLUTImpl & time_zone) { @@ -1262,7 +1260,7 @@ struct ToRelativeMinuteNumImpl } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toRelativeMinuteNum(DayNum(d)); + return static_cast(time_zone.toRelativeMinuteNum(DayNum(d))); } using FactorTransform = ZeroTransform; @@ -1290,7 +1288,7 @@ struct ToRelativeSecondNumImpl } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.fromDayNum(DayNum(d)); + return static_cast(time_zone.fromDayNum(DayNum(d))); } using FactorTransform = ZeroTransform; @@ -1375,14 +1373,17 @@ struct Transformer template static void vector(const FromTypeVector & vec_from, ToTypeVector & vec_to, const DateLUTImpl & time_zone, const Transform & transform) { + using ValueType = typename ToTypeVector::value_type; size_t size = vec_from.size(); vec_to.resize(size); for (size_t i = 0; i < size; ++i) + { if constexpr (is_extended_result) - vec_to[i] = transform.executeExtendedResult(vec_from[i], time_zone); + vec_to[i] = static_cast(transform.executeExtendedResult(vec_from[i], time_zone)); else - vec_to[i] = transform.execute(vec_from[i], time_zone); + vec_to[i] = static_cast(transform.execute(vec_from[i], time_zone)); + } } }; diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index e120595c4d9..98e5c690eb9 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -163,7 +163,7 @@ struct ModuloImpl return static_cast(int_a % static_cast(int_b)); } else - return IntegerAType(a) % IntegerBType(b); + return static_cast(IntegerAType(a) % IntegerBType(b)); } } diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index baa3c65537d..4e77d085248 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -415,8 +415,8 @@ public: { for (size_t i = 0; i < size; ++i) c[i] = applyScaled( - unwrap(a, i), - unwrap(b, i), + static_cast(unwrap(a, i)), + static_cast(unwrap(b, i)), scale_a); return; } @@ -424,8 +424,8 @@ public: { for (size_t i = 0; i < size; ++i) c[i] = applyScaled( - unwrap(a, i), - unwrap(b, i), + static_cast(unwrap(a, i)), + static_cast(unwrap(b, i)), scale_b); return; } @@ -436,8 +436,8 @@ public: { for (size_t i = 0; i < size; ++i) c[i] = applyScaled( - unwrap(a, i), - unwrap(b, i), + static_cast(unwrap(a, i)), + static_cast(unwrap(b, i)), scale_a); return; } @@ -445,8 +445,8 @@ public: { for (size_t i = 0; i < size; ++i) c[i] = applyScaled( - unwrap(a, i), - unwrap(b, i), + static_cast(unwrap(a, i)), + static_cast(unwrap(b, i)), scale_b); return; } @@ -456,12 +456,20 @@ public: { processWithRightNullmapImpl(a, b, c, size, right_nullmap, [&scale_a](const auto & left, const auto & right) { - return applyScaledDiv(left, right, scale_a); + return applyScaledDiv( + static_cast(left), right, scale_a); }); return; } - processWithRightNullmapImpl(a, b, c, size, right_nullmap, [](const auto & left, const auto & right){ return apply(left, right); }); + processWithRightNullmapImpl( + a, b, c, size, right_nullmap, + [](const auto & left, const auto & right) + { + return apply( + static_cast(left), + static_cast(right)); + }); } template @@ -995,8 +1003,10 @@ class FunctionBinaryArithmetic : public IFunction /// non-vector result if (col_left_const && col_right_const) { - const NativeResultType const_a = helperGetOrConvert(col_left_const, left); - const NativeResultType const_b = helperGetOrConvert(col_right_const, right); + const NativeResultType const_a = static_cast( + helperGetOrConvert(col_left_const, left)); + const NativeResultType const_b = static_cast( + helperGetOrConvert(col_right_const, right)); ResultType res = {}; if (!right_nullmap || !(*right_nullmap)[0]) @@ -1020,14 +1030,16 @@ class FunctionBinaryArithmetic : public IFunction } else if (col_left_const && col_right) { - const NativeResultType const_a = helperGetOrConvert(col_left_const, left); + const NativeResultType const_a = static_cast( + helperGetOrConvert(col_left_const, left)); helperInvokeEither( const_a, col_right->getData(), vec_res, scale_a, scale_b, right_nullmap); } else if (col_left && col_right_const) { - const NativeResultType const_b = helperGetOrConvert(col_right_const, right); + const NativeResultType const_b = static_cast( + helperGetOrConvert(col_right_const, right)); helperInvokeEither( col_left->getData(), const_b, vec_res, scale_a, scale_b, right_nullmap); diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index 29dcf87316d..2259cc71f07 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -63,7 +63,7 @@ struct AddNanosecondsImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) { Int64 multiplier = DecimalUtils::scaleMultiplier(9); - return t * multiplier + delta; + return static_cast(t * multiplier + delta); } static inline NO_SANITIZE_UNDEFINED DateTime64 execute(UInt16, Int64, const DateLUTImpl &, UInt16 = 0) @@ -107,7 +107,7 @@ struct AddMicrosecondsImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) { Int64 multiplier = DecimalUtils::scaleMultiplier(6); - return t * multiplier + delta; + return static_cast(t * multiplier + delta); } static inline NO_SANITIZE_UNDEFINED DateTime64 execute(UInt16, Int64, const DateLUTImpl &, UInt16 = 0) @@ -151,7 +151,7 @@ struct AddMillisecondsImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) { Int64 multiplier = DecimalUtils::scaleMultiplier(3); - return t * multiplier + delta; + return static_cast(t * multiplier + delta); } static inline NO_SANITIZE_UNDEFINED DateTime64 execute(UInt16, Int64, const DateLUTImpl &, UInt16 = 0) @@ -183,7 +183,7 @@ struct AddSecondsImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) { - return t + delta; + return static_cast(t + delta); } static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) @@ -194,7 +194,7 @@ struct AddSecondsImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { - return time_zone.fromDayNum(DayNum(d)) + delta; + return static_cast(time_zone.fromDayNum(DayNum(d)) + delta); } }; @@ -216,7 +216,7 @@ struct AddMinutesImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) { - return t + delta * 60; + return static_cast(t + delta * 60); } static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) @@ -227,7 +227,7 @@ struct AddMinutesImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { - return time_zone.fromDayNum(DayNum(d)) + delta * 60; + return static_cast(time_zone.fromDayNum(DayNum(d)) + delta * 60); } }; @@ -249,7 +249,7 @@ struct AddHoursImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) { - return t + delta * 3600; + return static_cast(t + delta * 3600); } static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) @@ -260,7 +260,7 @@ struct AddHoursImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { - return time_zone.fromDayNum(DayNum(d)) + delta * 3600; + return static_cast(time_zone.fromDayNum(DayNum(d)) + delta * 3600); } }; @@ -284,7 +284,7 @@ struct AddDaysImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { - return time_zone.addDays(t, delta); + return static_cast(time_zone.addDays(t, delta)); } static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, UInt16 = 0) @@ -294,7 +294,7 @@ struct AddDaysImpl static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, UInt16 = 0) { - return d + delta; + return static_cast(d + delta); } }; @@ -303,32 +303,32 @@ struct AddWeeksImpl static constexpr auto name = "addWeeks"; static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents - execute(DecimalUtils::DecimalComponents t, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0) + execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { return {time_zone.addWeeks(t.whole, delta), t.fractional}; } static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int32 delta, const DateLUTImpl & time_zone, UInt16 scale = 0) + execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale = 0) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addDays(d.quot, delta * 7) * multiplier + d.rem; } - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { - return time_zone.addWeeks(t, delta); + return static_cast(time_zone.addWeeks(t, delta)); } - static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int32 delta, const DateLUTImpl &, UInt16 = 0) + static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, UInt16 = 0) { - return d + delta * 7; + return static_cast(d + delta * 7); } - static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int32 delta, const DateLUTImpl &, UInt16 = 0) + static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, UInt16 = 0) { - return d + delta * 7; + return static_cast(d + delta * 7); } }; @@ -352,7 +352,7 @@ struct AddMonthsImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { - return time_zone.addMonths(t, delta); + return static_cast(time_zone.addMonths(t, delta)); } static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) @@ -371,30 +371,30 @@ struct AddQuartersImpl static constexpr auto name = "addQuarters"; static inline DecimalUtils::DecimalComponents - execute(DecimalUtils::DecimalComponents t, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0) + execute(DecimalUtils::DecimalComponents t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { return {time_zone.addQuarters(t.whole, delta), t.fractional}; } static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int32 delta, const DateLUTImpl & time_zone, UInt16 scale = 0) + execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale = 0) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addQuarters(d.quot, delta) * multiplier + d.rem; } - static inline UInt32 execute(UInt32 t, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { - return time_zone.addQuarters(t, delta); + return static_cast(time_zone.addQuarters(t, delta)); } - static inline UInt16 execute(UInt16 d, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { return time_zone.addQuarters(DayNum(d), delta); } - static inline Int32 execute(Int32 d, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { return time_zone.addQuarters(ExtendedDayNum(d), delta); } @@ -420,7 +420,7 @@ struct AddYearsImpl static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) { - return time_zone.addYears(t, delta); + return static_cast(time_zone.addYears(t, delta)); } static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) diff --git a/src/Functions/FunctionIfBase.h b/src/Functions/FunctionIfBase.h index 2b0f8289914..4c9ecf78a12 100644 --- a/src/Functions/FunctionIfBase.h +++ b/src/Functions/FunctionIfBase.h @@ -67,7 +67,7 @@ public: b.SetInsertPoint(join); - auto * phi = b.CreatePHI(toNativeType(b, return_type), returns.size()); + auto * phi = b.CreatePHI(toNativeType(b, return_type), static_cast(returns.size())); for (const auto & [block, value] : returns) phi->addIncoming(value, block); diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index dce953ddc6f..3a0f4f483b3 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -164,7 +164,7 @@ public: /// 2. Create ASTPtr /// 3. Parser(Tokens, ASTPtr) -> complete AST /// 4. Execute functions: call getNextItem on generator and handle each item - uint32_t parse_depth = getContext()->getSettingsRef().max_parser_depth; + unsigned parse_depth = static_cast(getContext()->getSettingsRef().max_parser_depth); #if USE_SIMDJSON if (getContext()->getSettingsRef().allow_simdjson) return FunctionSQLJSONHelpers::Executor::run(arguments, result_type, input_rows_count, parse_depth); diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index f4a62e509ed..998db98890a 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -109,7 +109,8 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { - result_data[i] = ((source_data[i] >> time_shift) + snowflake_epoch) / 1000; + result_data[i] = static_cast( + ((source_data[i] >> time_shift) + snowflake_epoch) / 1000); } return res_column; } diff --git a/src/Functions/FunctionUnixTimestamp64.h b/src/Functions/FunctionUnixTimestamp64.h index 8c248d79c4b..d869ccccca8 100644 --- a/src/Functions/FunctionUnixTimestamp64.h +++ b/src/Functions/FunctionUnixTimestamp64.h @@ -57,7 +57,7 @@ public: const auto & source_data = typeid_cast &>(col).getData(); - const Int32 scale_diff = typeid_cast(*src.type).getScale() - target_scale; + const Int32 scale_diff = static_cast(typeid_cast(*src.type).getScale() - target_scale); if (scale_diff == 0) { for (size_t i = 0; i < input_rows_count; ++i) @@ -140,7 +140,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - auto result_column = ColumnDecimal::create(input_rows_count, target_scale); + auto result_column = ColumnDecimal::create(input_rows_count, static_cast(target_scale)); if (!((executeType(result_column, arguments, input_rows_count)) || (executeType(result_column, arguments, input_rows_count)) diff --git a/src/Functions/FunctionsAES.h b/src/Functions/FunctionsAES.h index 94ff55d180b..ad129a315b3 100644 --- a/src/Functions/FunctionsAES.h +++ b/src/Functions/FunctionsAES.h @@ -2,6 +2,7 @@ #include "config.h" +#include #include #include #include @@ -336,7 +337,7 @@ private: if (EVP_EncryptInit_ex(evp_ctx, evp_cipher, nullptr, nullptr, nullptr) != 1) onError("Failed to initialize encryption context with cipher"); - if (EVP_CIPHER_CTX_ctrl(evp_ctx, EVP_CTRL_AEAD_SET_IVLEN, iv_value.size, nullptr) != 1) + if (EVP_CIPHER_CTX_ctrl(evp_ctx, EVP_CTRL_AEAD_SET_IVLEN, safe_cast(iv_value.size), nullptr) != 1) onError("Failed to set custom IV length to " + std::to_string(iv_value.size)); if (EVP_EncryptInit_ex(evp_ctx, nullptr, nullptr, @@ -350,7 +351,7 @@ private: const auto aad_data = aad_column->getDataAt(row_idx); int tmp_len = 0; if (aad_data.size != 0 && EVP_EncryptUpdate(evp_ctx, nullptr, &tmp_len, - reinterpret_cast(aad_data.data), aad_data.size) != 1) + reinterpret_cast(aad_data.data), safe_cast(aad_data.size)) != 1) onError("Failed to set AAD data"); } } @@ -636,7 +637,7 @@ private: onError("Failed to initialize cipher context 1"); // 1.a.1 : Set custom IV length - if (EVP_CIPHER_CTX_ctrl(evp_ctx, EVP_CTRL_AEAD_SET_IVLEN, iv_value.size, nullptr) != 1) + if (EVP_CIPHER_CTX_ctrl(evp_ctx, EVP_CTRL_AEAD_SET_IVLEN, safe_cast(iv_value.size), nullptr) != 1) onError("Failed to set custom IV length to " + std::to_string(iv_value.size)); // 1.a.1 : Init CTX with key and IV @@ -651,7 +652,7 @@ private: StringRef aad_data = aad_column->getDataAt(row_idx); int tmp_len = 0; if (aad_data.size != 0 && EVP_DecryptUpdate(evp_ctx, nullptr, &tmp_len, - reinterpret_cast(aad_data.data), aad_data.size) != 1) + reinterpret_cast(aad_data.data), safe_cast(aad_data.size)) != 1) onError("Failed to sed AAD data"); } } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 8cbe3b0e532..b2f2fb0e25f 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -101,7 +101,7 @@ inline UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column) Field field; named_column.column->get(0, field); - return field.get(); + return static_cast(field.get()); } /// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type. @@ -335,7 +335,7 @@ struct ToDateTimeImpl return 0; auto date_time = time_zone.fromDayNum(ExtendedDayNum(d)); - return date_time <= 0xffffffff ? date_time : 0xffffffff; + return date_time <= 0xffffffff ? static_cast(date_time) : 0xffffffff; } static UInt32 execute(UInt32 dt, const DateLUTImpl & /*time_zone*/) @@ -349,7 +349,7 @@ struct ToDateTimeImpl return 0; auto date_time = time_zone.toDate(d); - return date_time <= 0xffffffff ? date_time : 0xffffffff; + return date_time <= 0xffffffff ? static_cast(date_time) : 0xffffffff; } static UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & /*time_zone*/) @@ -357,7 +357,7 @@ struct ToDateTimeImpl if (t.whole < 0 || (t.whole >= 0 && t.fractional < 0)) return 0; - return std::min(t.whole, Int64(0xFFFFFFFF)); + return t.whole > 0xffffffff ? 0xffffffff : static_cast(t.whole); } }; @@ -434,7 +434,7 @@ struct ToDate32Transform32Or64 static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { return (from < DATE_LUT_MAX_EXTEND_DAY_NUM) - ? from + ? static_cast(from) : std::min(Int32(time_zone.toDayNum(from)), Int32(DATE_LUT_MAX_EXTEND_DAY_NUM)); } }; @@ -531,7 +531,7 @@ struct ToDateTimeTransform64 static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { - return std::min(Int64(from), Int64(0xFFFFFFFF)); + return from > 0xffffffff ? 0xffffffff : static_cast(from); } }; @@ -558,7 +558,7 @@ struct ToDateTimeTransform64Signed if (from < 0) return 0; - return std::min(Int64(from), Int64(0xFFFFFFFF)); + return from > 0xffffffff ? 0xffffffff : static_cast(from); } }; @@ -618,8 +618,8 @@ struct ToDateTime64TransformSigned NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const { - from = std::max(from, LUT_MIN_TIME); - from = std::min(from, LUT_MAX_TIME); + from = static_cast(std::max(from, LUT_MIN_TIME)); + from = static_cast(std::min(from, LUT_MAX_TIME)); return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); } }; @@ -979,7 +979,7 @@ inline void convertFromTime(DataTypeDate::FieldType & x, time_t & template <> inline void convertFromTime(DataTypeDate32::FieldType & x, time_t & time) { - x = time; + x = static_cast(time); } template <> @@ -990,7 +990,7 @@ inline void convertFromTime(DataTypeDateTime::FieldType & x, t else if (unlikely(time > 0xFFFFFFFF)) x = 0xFFFFFFFF; else - x = time; + x = static_cast(time); } /** Conversion of strings to numbers, dates, datetimes: through parsing. @@ -1070,7 +1070,7 @@ inline bool tryParseImpl(DataTypeDateTime::FieldType & x, Read time_t tmp = 0; if (!tryReadDateTimeText(tmp, rb, *time_zone)) return false; - x = tmp; + x = static_cast(tmp); return true; } diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index aefc82d2f5d..493fc36ca3c 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -231,7 +231,7 @@ private: { case MoveType::ConstIndex: { - if (!moveToElementByIndex(res_element, moves[j].index, key)) + if (!moveToElementByIndex(res_element, static_cast(moves[j].index), key)) return false; break; } @@ -245,7 +245,7 @@ private: case MoveType::Index: { Int64 index = (*arguments[j + 1].column)[row].get(); - if (!moveToElementByIndex(res_element, index, key)) + if (!moveToElementByIndex(res_element, static_cast(index), key)) return false; break; } diff --git a/src/Functions/FunctionsLanguageClassification.cpp b/src/Functions/FunctionsLanguageClassification.cpp index ecc958a0a0c..ebdb2f1afaa 100644 --- a/src/Functions/FunctionsLanguageClassification.cpp +++ b/src/Functions/FunctionsLanguageClassification.cpp @@ -83,7 +83,10 @@ struct FunctionDetectLanguageImpl if (UTF8::isValidUTF8(str, str_len)) { - auto lang = CLD2::DetectLanguage(reinterpret_cast(str), str_len, true, &is_reliable); + auto lang = CLD2::DetectLanguage( + reinterpret_cast(str), + static_cast(str_len), + true, &is_reliable); res = codeISO(LanguageCode(lang)); } else @@ -178,7 +181,10 @@ public: if (UTF8::isValidUTF8(str, str_len)) { - CLD2::DetectLanguageSummary(reinterpret_cast(str), str_len, true, result_lang_top3, pc, bytes, &is_reliable); + CLD2::DetectLanguageSummary( + reinterpret_cast(str), + static_cast(str_len), + true, result_lang_top3, pc, bytes, &is_reliable); for (size_t j = 0; j < top_N; ++j) { diff --git a/src/Functions/FunctionsLogical.h b/src/Functions/FunctionsLogical.h index d7e8ffb0c9f..d2b9a96b4ad 100644 --- a/src/Functions/FunctionsLogical.h +++ b/src/Functions/FunctionsLogical.h @@ -193,7 +193,7 @@ public: auto * next = b.GetInsertBlock(); auto * stop = llvm::BasicBlock::Create(next->getContext(), "", next->getParent()); b.SetInsertPoint(stop); - auto * phi = b.CreatePHI(b.getInt8Ty(), values.size()); + auto * phi = b.CreatePHI(b.getInt8Ty(), static_cast(values.size())); for (size_t i = 0; i < types.size(); ++i) { b.SetInsertPoint(next); diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 96c41225242..40522fa1a6e 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -37,7 +37,7 @@ namespace UInt32 next() { current = current * a + c; - return current >> 16; + return static_cast(current >> 16); } }; diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index ccab6e9feca..283f1ea5a43 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -178,7 +178,7 @@ struct IntegerRoundingComputation return; } } - *out = compute(*in, scale); + *out = compute(*in, static_cast(scale)); } static ALWAYS_INLINE void compute(const T * __restrict in, T scale, T * __restrict out) requires(!std::integral) @@ -436,7 +436,7 @@ public: scale_arg = in_scale - scale_arg; if (scale_arg > 0) { - auto scale = intExp10OfSize(scale_arg); + auto scale = intExp10OfSize(scale_arg); const NativeType * __restrict p_in = reinterpret_cast(in.data()); const NativeType * end_in = reinterpret_cast(in.data()) + in.size(); diff --git a/src/Functions/FunctionsStringHash.cpp b/src/Functions/FunctionsStringHash.cpp index e7dbe4087f2..949503e2367 100644 --- a/src/Functions/FunctionsStringHash.cpp +++ b/src/Functions/FunctionsStringHash.cpp @@ -35,13 +35,13 @@ struct Hash #ifdef __SSE4_2__ return _mm_crc32_u64(crc, val); #elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32) - return __crc32cd(crc, val); + return __crc32cd(static_cast(crc), val); #else throw Exception("String hash is not implemented without sse4.2 support", ErrorCodes::NOT_IMPLEMENTED); #endif } - static UInt64 crc32u32(UInt64 crc [[maybe_unused]], UInt32 val [[maybe_unused]]) + static UInt64 crc32u32(UInt32 crc [[maybe_unused]], UInt32 val [[maybe_unused]]) { #ifdef __SSE4_2__ return _mm_crc32_u32(crc, val); @@ -52,7 +52,7 @@ struct Hash #endif } - static UInt64 crc32u16(UInt64 crc [[maybe_unused]], UInt16 val [[maybe_unused]]) + static UInt64 crc32u16(UInt32 crc [[maybe_unused]], UInt16 val [[maybe_unused]]) { #ifdef __SSE4_2__ return _mm_crc32_u16(crc, val); @@ -63,7 +63,7 @@ struct Hash #endif } - static UInt64 crc32u8(UInt64 crc [[maybe_unused]], UInt8 val [[maybe_unused]]) + static UInt64 crc32u8(UInt32 crc [[maybe_unused]], UInt8 val [[maybe_unused]]) { #ifdef __SSE4_2__ return _mm_crc32_u8(crc, val); @@ -84,7 +84,7 @@ struct Hash if constexpr (CaseInsensitive) x |= 0x20u; /// see toLowerIfAlphaASCII from StringUtils.h - crc = crc32u8(crc, x); + crc = crc32u8(static_cast(crc), x); --size; ++start; } @@ -96,7 +96,7 @@ struct Hash if constexpr (CaseInsensitive) x |= 0x2020u; - crc = crc32u16(crc, x); + crc = crc32u16(static_cast(crc), x); size -= 2; start += 2; } @@ -108,7 +108,7 @@ struct Hash if constexpr (CaseInsensitive) x |= 0x20202020u; - crc = crc32u32(crc, x); + crc = crc32u32(static_cast(crc), x); size -= 4; start += 4; } diff --git a/src/Functions/FunctionsTimeWindow.h b/src/Functions/FunctionsTimeWindow.h index 3ea397e4c7d..4346e691046 100644 --- a/src/Functions/FunctionsTimeWindow.h +++ b/src/Functions/FunctionsTimeWindow.h @@ -64,7 +64,7 @@ struct ToStartOfTransform; { static UInt32 execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) { - return time_zone.toStartOfDayInterval(time_zone.toDayNum(t), delta); + return static_cast(time_zone.toStartOfDayInterval(time_zone.toDayNum(t), delta)); } }; @@ -74,7 +74,7 @@ struct ToStartOfTransform; { \ static UInt32 execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) \ { \ - return time_zone.toStartOf##INTERVAL_KIND##Interval(t, delta); \ + return static_cast(time_zone.toStartOf##INTERVAL_KIND##Interval(t, delta)); \ } \ }; TRANSFORM_TIME(Hour) @@ -114,7 +114,7 @@ template<> \ template <> \ struct AddTime \ { \ - static inline auto execute(UInt16 d, UInt64 delta, const DateLUTImpl & time_zone) \ + static inline auto execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) \ { \ return time_zone.add##INTERVAL_KIND##s(ExtendedDayNum(d), delta); \ } \ @@ -127,14 +127,18 @@ template<> \ template <> struct AddTime { - static inline NO_SANITIZE_UNDEFINED ExtendedDayNum execute(UInt16 d, UInt64 delta, const DateLUTImpl &) { return ExtendedDayNum(d + delta * 7);} + static inline NO_SANITIZE_UNDEFINED ExtendedDayNum execute(UInt16 d, UInt64 delta, const DateLUTImpl &) + { + return ExtendedDayNum(static_cast(d + delta * 7)); + } }; #define ADD_TIME(INTERVAL_KIND, INTERVAL) \ template <> \ struct AddTime \ { \ - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &) { return t + delta * INTERVAL; } \ + static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &) \ + { return static_cast(t + delta * INTERVAL); } \ }; ADD_TIME(Day, 86400) ADD_TIME(Hour, 3600) diff --git a/src/Functions/GregorianDate.h b/src/Functions/GregorianDate.h index ef2b9e6eede..d6b80ff3836 100644 --- a/src/Functions/GregorianDate.h +++ b/src/Functions/GregorianDate.h @@ -340,7 +340,7 @@ namespace DB const auto d = gd::mod(c, 1461); const auto y = gd::min(gd::div(d, 365), 3); day_of_year_ = d - y * 365 + 1; - year_ = quad_cent * 400 + cent * 100 + quad * 4 + y + 1; + year_ = static_cast(quad_cent * 400 + cent * 100 + quad * 4 + y + 1); } template diff --git a/src/Functions/JSONPath/Generator/GeneratorJSONPath.h b/src/Functions/JSONPath/Generator/GeneratorJSONPath.h index fe00f06bbbf..3d646a6ff31 100644 --- a/src/Functions/JSONPath/Generator/GeneratorJSONPath.h +++ b/src/Functions/JSONPath/Generator/GeneratorJSONPath.h @@ -89,7 +89,7 @@ public: for (size_t i = current_visitor; i < visitors.size(); ++i) { status = visitors[i]->visit(current); - current_visitor = i; + current_visitor = static_cast(i); if (status == VisitorStatus::Error || status == VisitorStatus::Ignore) { break; diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp b/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp index bc153b9d747..03c006774c0 100644 --- a/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp @@ -46,7 +46,7 @@ bool ParserJSONPathRange::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte { return false; } - range_indices.first = number_ptr->as()->value.get(); + range_indices.first = static_cast(number_ptr->as()->value.get()); if (pos->type == TokenType::Comma || pos->type == TokenType::ClosingSquareBracket) { @@ -63,7 +63,7 @@ bool ParserJSONPathRange::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte { return false; } - range_indices.second = number_ptr->as()->value.get(); + range_indices.second = static_cast(number_ptr->as()->value.get()); } else { diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index 3490c854f22..8e355405093 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -116,7 +116,7 @@ struct MultiMatchAllIndicesImpl err = hs_scan( regexps->getDB(), reinterpret_cast(haystack_data.data()) + offset, - length, + static_cast(length), 0, smart_scratch.get(), on_match, @@ -227,7 +227,7 @@ struct MultiMatchAllIndicesImpl err = hs_scan( regexps->getDB(), reinterpret_cast(haystack_data.data()) + prev_haystack_offset, - cur_haystack_length, + static_cast(cur_haystack_length), 0, smart_scratch.get(), on_match, diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 2d4db261bb4..4b02e78dc25 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -136,7 +136,7 @@ struct MultiMatchAnyImpl err = hs_scan( regexps->getDB(), reinterpret_cast(haystack_data.data()) + offset, - length, + static_cast(length), 0, smart_scratch.get(), on_match, @@ -260,7 +260,7 @@ struct MultiMatchAnyImpl err = hs_scan( regexps->getDB(), reinterpret_cast(haystack_data.data()) + prev_haystack_offset, - cur_haystack_length, + static_cast(cur_haystack_length), 0, smart_scratch.get(), on_match, diff --git a/src/Functions/PolygonUtils.h b/src/Functions/PolygonUtils.h index 0ffe05fbffc..37517313879 100644 --- a/src/Functions/PolygonUtils.h +++ b/src/Functions/PolygonUtils.h @@ -625,7 +625,7 @@ UInt128 sipHash128(Polygon && polygon) auto hash_ring = [&hash](const auto & ring) { - UInt32 size = ring.size(); + UInt32 size = static_cast(ring.size()); hash.update(size); hash.update(reinterpret_cast(ring.data()), size * sizeof(ring[0])); }; diff --git a/src/Functions/Regexps.h b/src/Functions/Regexps.h index 1e40c845788..c1ff83d04fe 100644 --- a/src/Functions/Regexps.h +++ b/src/Functions/Regexps.h @@ -207,7 +207,7 @@ inline Regexps constructRegexps(const std::vector & str_patterns, [[mayb { ids.reset(new unsigned int[patterns.size()]); for (size_t i = 0; i < patterns.size(); ++i) - ids[i] = i + 1; + ids[i] = static_cast(i + 1); } hs_error_t err; @@ -216,7 +216,7 @@ inline Regexps constructRegexps(const std::vector & str_patterns, [[mayb patterns.data(), flags.data(), ids.get(), - patterns.size(), + static_cast(patterns.size()), HS_MODE_BLOCK, nullptr, &db, @@ -227,7 +227,7 @@ inline Regexps constructRegexps(const std::vector & str_patterns, [[mayb flags.data(), ids.get(), ext_exprs_ptrs.data(), - patterns.size(), + static_cast(patterns.size()), HS_MODE_BLOCK, nullptr, &db, diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index 88bc48a6d8c..3325c7b8eb1 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -189,7 +189,7 @@ struct ReplaceRegexpImpl /// Cannot perform search for whole columns. Will process each string separately. for (size_t i = 0; i < size; ++i) { - int from = i > 0 ? offsets[i - 1] : 0; + ssize_t from = i > 0 ? offsets[i - 1] : 0; re2_st::StringPiece input(reinterpret_cast(data.data() + from), offsets[i] - from - 1); processString(input, res_data, res_offset, searcher, num_captures, instructions); @@ -220,7 +220,7 @@ struct ReplaceRegexpImpl for (size_t i = 0; i < size; ++i) { - int from = i * n; + size_t from = i * n; re2_st::StringPiece input(reinterpret_cast(data.data() + from), n); processString(input, res_data, res_offset, searcher, num_captures, instructions); diff --git a/src/Functions/URL/CMakeLists.txt b/src/Functions/URL/CMakeLists.txt index 6328476543d..0e148e87604 100644 --- a/src/Functions/URL/CMakeLists.txt +++ b/src/Functions/URL/CMakeLists.txt @@ -2,6 +2,7 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") add_headers_and_sources(clickhouse_functions_url .) add_library(clickhouse_functions_url OBJECT ${clickhouse_functions_url_sources} ${clickhouse_functions_url_headers}) target_link_libraries(clickhouse_functions_url PRIVATE dbms) +set_source_files_properties(tldLookup.generated.cpp PROPERTIES COMPILE_FLAGS -Wno-shorten-64-to-32) if (OMIT_HEAVY_DEBUG_SYMBOLS) target_compile_options(clickhouse_functions_url PRIVATE "-g0") diff --git a/src/Functions/array/arrayAggregation.cpp b/src/Functions/array/arrayAggregation.cpp index 7b72060f0c0..c8eae78dfaa 100644 --- a/src/Functions/array/arrayAggregation.cpp +++ b/src/Functions/array/arrayAggregation.cpp @@ -223,7 +223,7 @@ struct ArrayAggregateImpl if (unlikely(result_scale > DecimalUtils::max_precision)) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", result_scale); - res[i] = DecimalUtils::convertTo(product, result_scale); + res[i] = DecimalUtils::convertTo(product, static_cast(result_scale)); } else { @@ -332,7 +332,7 @@ struct ArrayAggregateImpl if (unlikely(result_scale > DecimalUtils::max_precision)) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", result_scale); - res[i] = DecimalUtils::convertTo(aggregate_value, result_scale); + res[i] = DecimalUtils::convertTo(aggregate_value, static_cast(result_scale)); } else { diff --git a/src/Functions/array/arrayCount.cpp b/src/Functions/array/arrayCount.cpp index cb902206e8b..f7ded051e5e 100644 --- a/src/Functions/array/arrayCount.cpp +++ b/src/Functions/array/arrayCount.cpp @@ -49,7 +49,7 @@ struct ArrayCountImpl size_t pos = 0; for (size_t i = 0; i < offsets.size(); ++i) { - out_counts[i] = offsets[i] - pos; + out_counts[i] = static_cast(offsets[i] - pos); pos = offsets[i]; } @@ -73,7 +73,7 @@ struct ArrayCountImpl if (filter[pos]) ++count; } - out_counts[i] = count; + out_counts[i] = static_cast(count); } return out_column; diff --git a/src/Functions/array/arrayEnumerate.cpp b/src/Functions/array/arrayEnumerate.cpp index b20f91fe2dd..666e01899bd 100644 --- a/src/Functions/array/arrayEnumerate.cpp +++ b/src/Functions/array/arrayEnumerate.cpp @@ -60,7 +60,7 @@ public: for (auto off : offsets) { for (ColumnArray::Offset j = prev_off; j < off; ++j) - res_values[j] = j - prev_off + 1; + res_values[j] = static_cast(j - prev_off + 1); prev_off = off; } diff --git a/src/Functions/array/arrayEnumerateRanked.cpp b/src/Functions/array/arrayEnumerateRanked.cpp index 7c4b755e020..d19781f97c3 100644 --- a/src/Functions/array/arrayEnumerateRanked.cpp +++ b/src/Functions/array/arrayEnumerateRanked.cpp @@ -38,7 +38,7 @@ ArraysDepths getArraysDepths(const ColumnsWithTypeAndName & arguments) if (depths.size() < array_num && prev_array_depth) depths.emplace_back(prev_array_depth); - prev_array_depth = type_array->getNumberOfDimensions(); + prev_array_depth = static_cast(type_array->getNumberOfDimensions()); ++array_num; } else @@ -55,7 +55,7 @@ ArraysDepths getArraysDepths(const ColumnsWithTypeAndName & arguments) if (i == 0) { - clear_depth = value; + clear_depth = static_cast(value); } else { diff --git a/src/Functions/array/arrayFirstLastIndex.cpp b/src/Functions/array/arrayFirstLastIndex.cpp index f7355eb2b38..effcb04ab48 100644 --- a/src/Functions/array/arrayFirstLastIndex.cpp +++ b/src/Functions/array/arrayFirstLastIndex.cpp @@ -61,7 +61,7 @@ struct ArrayFirstLastIndexImpl if constexpr (strategy == ArrayFirstLastIndexStrategy::First) out_index[offset_index] = 1; else - out_index[offset_index] = end_offset - start_offset; + out_index[offset_index] = static_cast(end_offset - start_offset); } else { @@ -113,7 +113,7 @@ struct ArrayFirstLastIndexImpl } } - out_index[offset_index] = result_index; + out_index[offset_index] = static_cast(result_index); } return out_column; diff --git a/src/Functions/array/arrayUniq.cpp b/src/Functions/array/arrayUniq.cpp index ff75efaae71..a43c21508d9 100644 --- a/src/Functions/array/arrayUniq.cpp +++ b/src/Functions/array/arrayUniq.cpp @@ -233,7 +233,7 @@ void FunctionArrayUniq::executeMethodImpl( method.emplaceKey(set, j, pool); } - res_values[i] = set.size() + found_null; + res_values[i] = static_cast(set.size() + found_null); prev_off = off; } } diff --git a/src/Functions/array/range.cpp b/src/Functions/array/range.cpp index 6b3d8ad1139..3b5bb686e60 100644 --- a/src/Functions/array/range.cpp +++ b/src/Functions/array/range.cpp @@ -97,7 +97,7 @@ private: for (size_t row_idx = 0, rows = in->size(); row_idx < rows; ++row_idx) { for (size_t elem_idx = 0, elems = in_data[row_idx]; elem_idx < elems; ++elem_idx) - out_data[offset + elem_idx] = elem_idx; + out_data[offset + elem_idx] = static_cast(elem_idx); offset += in_data[row_idx]; out_offsets[row_idx] = offset; @@ -153,7 +153,7 @@ private: { for (size_t st = start, ed = end_data[row_idx]; st < ed; st += step) { - out_data[offset++] = st; + out_data[offset++] = static_cast(st); if (st > st + step) throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", @@ -212,7 +212,7 @@ private: { for (size_t st = start_data[row_idx], ed = end_data[row_idx]; st < ed; st += step) { - out_data[offset++] = st; + out_data[offset++] = static_cast(st); if (st > st + step) throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", @@ -271,7 +271,7 @@ private: { for (size_t st = start, ed = end_data[row_idx]; st < ed; st += step_data[row_idx]) { - out_data[offset++] = st; + out_data[offset++] = static_cast(st); if (st > st + step_data[row_idx]) throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", @@ -333,7 +333,7 @@ private: { for (size_t st = start_data[row_idx], ed = end_start[row_idx]; st < ed; st += step_data[row_idx]) { - out_data[offset++] = st; + out_data[offset++] = static_cast(st); if (st > st + step_data[row_idx]) throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", @@ -407,7 +407,7 @@ private: if ((res = executeConstStartStep(column_ptrs[1], start, step, input_rows_count)) || (res = executeConstStartStep(column_ptrs[1], start, step, input_rows_count)) || - (res = executeConstStartStep(column_ptrs[1], start, step, input_rows_count)) || + (res = executeConstStartStep(column_ptrs[1], static_cast(start), static_cast(step), input_rows_count)) || (res = executeConstStartStep(column_ptrs[1], start, step, input_rows_count))) { } @@ -418,7 +418,7 @@ private: if ((res = executeConstStart(column_ptrs[1], column_ptrs[2], start, input_rows_count)) || (res = executeConstStart(column_ptrs[1], column_ptrs[2], start, input_rows_count)) || - (res = executeConstStart(column_ptrs[1], column_ptrs[2], start, input_rows_count)) || + (res = executeConstStart(column_ptrs[1], column_ptrs[2], static_cast(start), input_rows_count)) || (res = executeConstStart(column_ptrs[1], column_ptrs[2], start, input_rows_count))) { } @@ -429,7 +429,7 @@ private: if ((res = executeConstStep(column_ptrs[0], column_ptrs[1], step, input_rows_count)) || (res = executeConstStep(column_ptrs[0], column_ptrs[1], step, input_rows_count)) || - (res = executeConstStep(column_ptrs[0], column_ptrs[1], step, input_rows_count)) || + (res = executeConstStep(column_ptrs[0], column_ptrs[1], static_cast(step), input_rows_count)) || (res = executeConstStep(column_ptrs[0], column_ptrs[1], step, input_rows_count))) { } diff --git a/src/Functions/divide/divideImpl.cpp b/src/Functions/divide/divideImpl.cpp index 6c151dfefb5..940f4b35df9 100644 --- a/src/Functions/divide/divideImpl.cpp +++ b/src/Functions/divide/divideImpl.cpp @@ -18,7 +18,7 @@ namespace NAMESPACE template void divideImpl(const A * __restrict a_pos, B b, ResultType * __restrict c_pos, size_t size) { - libdivide::divider divider(b); + libdivide::divider divider(static_cast(b)); const A * a_end = a_pos + size; #if defined(__SSE2__) diff --git a/src/Functions/errorCodeToName.cpp b/src/Functions/errorCodeToName.cpp index 1736311c6cc..0025d38c8f2 100644 --- a/src/Functions/errorCodeToName.cpp +++ b/src/Functions/errorCodeToName.cpp @@ -45,7 +45,8 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { const Int64 error_code = input_column.getInt(i); - std::string_view error_name = ErrorCodes::getName(error_code); + std::string_view error_name = + ErrorCodes::getName(static_cast(error_code)); col_res->insertData(error_name.data(), error_name.size()); } diff --git a/src/Functions/extractAllGroups.h b/src/Functions/extractAllGroups.h index 06b16181c94..e077086a359 100644 --- a/src/Functions/extractAllGroups.h +++ b/src/Functions/extractAllGroups.h @@ -136,7 +136,8 @@ public: const auto * end = pos + current_row.size(); while (pos < end && regexp->Match({pos, static_cast(end - pos)}, - 0, end - pos, regexp->UNANCHORED, matched_groups.data(), matched_groups.size())) + 0, end - pos, regexp->UNANCHORED, + matched_groups.data(), static_cast(matched_groups.size()))) { // 1 is to exclude group #0 which is whole re match. for (size_t group = 1; group <= groups_count; ++group) @@ -179,7 +180,8 @@ public: const auto * end = pos + current_row.size; while (pos < end && regexp->Match({pos, static_cast(end - pos)}, - 0, end - pos, regexp->UNANCHORED, matched_groups.data(), matched_groups.size())) + 0, end - pos, regexp->UNANCHORED, matched_groups.data(), + static_cast(matched_groups.size()))) { // 1 is to exclude group #0 which is whole re match. for (size_t group = 1; group <= groups_count; ++group) diff --git a/src/Functions/extractGroups.cpp b/src/Functions/extractGroups.cpp index eb6e609a4be..8ec389827db 100644 --- a/src/Functions/extractGroups.cpp +++ b/src/Functions/extractGroups.cpp @@ -90,7 +90,8 @@ public: std::string_view current_row = column_haystack->getDataAt(i).toView(); if (re2->Match(re2_st::StringPiece(current_row.data(), current_row.size()), - 0, current_row.size(), re2_st::RE2::UNANCHORED, matched_groups.data(), matched_groups.size())) + 0, current_row.size(), re2_st::RE2::UNANCHORED, matched_groups.data(), + static_cast(matched_groups.size()))) { // 1 is to exclude group #0 which is whole re match. for (size_t group = 1; group <= groups_count; ++group) diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 09071c5c1a0..a10c059b342 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -499,7 +499,7 @@ public: else { for (auto & instruction : instructions) - instruction.perform(pos, vec[i], time_zone); + instruction.perform(pos, static_cast(vec[i]), time_zone); } dst_offsets[i] = pos - begin; diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index e2d93c0fdc9..c9571a7333d 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -294,7 +294,7 @@ public: else if (unlikely(date_time > 0x0ffffffffll)) date_time = 0x0ffffffffll; - result_data[i] = date_time; + result_data[i] = static_cast(date_time); } return res_column; @@ -365,7 +365,7 @@ public: fraction_data = &typeid_cast(*converted_arguments[6]).getData(); } - auto res_column = ColumnDecimal::create(input_rows_count, precision); + auto res_column = ColumnDecimal::create(input_rows_count, static_cast(precision)); auto & result_data = res_column->getData(); const auto & year_data = typeid_cast(*converted_arguments[0]).getData(); @@ -411,7 +411,10 @@ public: fraction = max_fraction; } - result_data[i] = DecimalUtils::decimalFromComponents(date_time, static_cast(fraction), precision); + result_data[i] = DecimalUtils::decimalFromComponents( + date_time, + static_cast(fraction), + static_cast(precision)); } return res_column; diff --git a/src/Functions/minus.cpp b/src/Functions/minus.cpp index 3668e4afc18..04877a42b18 100644 --- a/src/Functions/minus.cpp +++ b/src/Functions/minus.cpp @@ -23,7 +23,7 @@ struct MinusImpl return static_cast(static_cast(a)) - static_cast(static_cast(b)); } else - return static_cast(a) - b; + return static_cast(a) - static_cast(b); } /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false otherwise. diff --git a/src/Functions/modulo.cpp b/src/Functions/modulo.cpp index 9a3aa12037f..b2411899160 100644 --- a/src/Functions/modulo.cpp +++ b/src/Functions/modulo.cpp @@ -80,7 +80,7 @@ struct ModuloByConstantImpl || (std::is_signed_v && std::is_signed_v && b < std::numeric_limits::lowest()))) { for (size_t i = 0; i < size; ++i) - dst[i] = src[i]; + dst[i] = static_cast(src[i]); return; } @@ -101,16 +101,19 @@ struct ModuloByConstantImpl if (b & (b - 1)) { - libdivide::divider divider(b); + libdivide::divider divider(static_cast(b)); for (size_t i = 0; i < size; ++i) - dst[i] = src[i] - (src[i] / divider) * b; /// NOTE: perhaps, the division semantics with the remainder of negative numbers is not preserved. + { + /// NOTE: perhaps, the division semantics with the remainder of negative numbers is not preserved. + dst[i] = static_cast(src[i] - (src[i] / divider) * b); + } } else { // gcc libdivide doesn't work well for pow2 division auto mask = b - 1; for (size_t i = 0; i < size; ++i) - dst[i] = src[i] & mask; + dst[i] = static_cast(src[i] & mask); } } diff --git a/src/Functions/now64.cpp b/src/Functions/now64.cpp index c5225d3317f..a6df4235d60 100644 --- a/src/Functions/now64.cpp +++ b/src/Functions/now64.cpp @@ -130,7 +130,7 @@ public: ". Expected const integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - scale = argument.column->get64(0); + scale = static_cast(argument.column->get64(0)); } if (arguments.size() == 2) { diff --git a/src/Functions/nowInBlock.cpp b/src/Functions/nowInBlock.cpp index db72e791587..c771d83225a 100644 --- a/src/Functions/nowInBlock.cpp +++ b/src/Functions/nowInBlock.cpp @@ -74,7 +74,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override { - return ColumnUInt32::create(input_rows_count, time(nullptr)); + return ColumnUInt32::create(input_rows_count, static_cast(time(nullptr))); } }; diff --git a/src/Functions/plus.cpp b/src/Functions/plus.cpp index 4b81c23584c..cd9cf6cec5c 100644 --- a/src/Functions/plus.cpp +++ b/src/Functions/plus.cpp @@ -25,7 +25,7 @@ struct PlusImpl return static_cast(static_cast(a)) + static_cast(static_cast(b)); } else - return static_cast(a) + b; + return static_cast(a) + static_cast(b); } /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false otherwise. diff --git a/src/Functions/pointInEllipses.cpp b/src/Functions/pointInEllipses.cpp index f69886ad71f..07b7f013cac 100644 --- a/src/Functions/pointInEllipses.cpp +++ b/src/Functions/pointInEllipses.cpp @@ -102,7 +102,7 @@ private: Float64 ellipse_data[4]; for (const auto idx : collections::range(0, 4)) { - int arg_idx = 2 + 4 * ellipse_idx + idx; + size_t arg_idx = 2 + 4 * ellipse_idx + idx; const auto * column = arguments[arg_idx].column.get(); if (const auto * col = checkAndGetColumnConst>(column)) { diff --git a/src/Functions/randomStringUTF8.cpp b/src/Functions/randomStringUTF8.cpp index 043db179d71..bcaa603b85d 100644 --- a/src/Functions/randomStringUTF8.cpp +++ b/src/Functions/randomStringUTF8.cpp @@ -116,8 +116,8 @@ public: { UInt64 rand = rng(); - UInt32 code_point1 = generate_code_point(rand); - UInt32 code_point2 = generate_code_point(rand >> 32); + UInt32 code_point1 = generate_code_point(static_cast(rand)); + UInt32 code_point2 = generate_code_point(static_cast(rand >> 32u)); /// We have padding in column buffers that we can overwrite. size_t length1 = UTF8::convertCodePointToUTF8(code_point1, pos, sizeof(int)); diff --git a/src/Functions/runningConcurrency.cpp b/src/Functions/runningConcurrency.cpp index 37fa11bce8f..c759476006f 100644 --- a/src/Functions/runningConcurrency.cpp +++ b/src/Functions/runningConcurrency.cpp @@ -43,6 +43,7 @@ namespace DB const typename ColVecArg::Container & vec_end = col_end->getData(); using ColVecConc = typename ConcurrencyDataType::ColumnType; + using FieldType = typename ConcurrencyDataType::FieldType; typename ColVecConc::MutablePtr col_concurrency = ColVecConc::create(input_rows_count); typename ColVecConc::Container & vec_concurrency = col_concurrency->getData(); @@ -74,7 +75,7 @@ namespace DB ongoing_until.erase( ongoing_until.begin(), ongoing_until.upper_bound(begin)); - vec_concurrency[i] = ongoing_until.size(); + vec_concurrency[i] = static_cast(ongoing_until.size()); } return col_concurrency; diff --git a/src/Functions/stem.cpp b/src/Functions/stem.cpp index 9c7ce895fce..91c98ec9b82 100644 --- a/src/Functions/stem.cpp +++ b/src/Functions/stem.cpp @@ -51,8 +51,8 @@ struct StemImpl /// Note that accessing -1th element is valid for PaddedPODArray. size_t original_size = offsets[i] - offsets[i - 1]; const sb_symbol * result = sb_stemmer_stem(stemmer, - reinterpret_cast(data.data() + offsets[i - 1]), - original_size - 1); + reinterpret_cast(data.data() + offsets[i - 1]), + static_cast(original_size - 1)); size_t new_size = sb_stemmer_length(stemmer) + 1; memcpy(res_data.data() + data_size, result, new_size); diff --git a/src/Functions/tests/gtest_has_all.cpp b/src/Functions/tests/gtest_has_all.cpp index ca7bc80b4aa..1776a461580 100644 --- a/src/Functions/tests/gtest_has_all.cpp +++ b/src/Functions/tests/gtest_has_all.cpp @@ -18,9 +18,9 @@ void arrayInit(T* elements_to_have, size_t nb_elements_to_have, T* array_element { for (size_t i = 0; i < array_size; ++i) { - array_elements[i] = i; + array_elements[i] = static_cast(i); } - auto [dist, gen] = uni_int_dist(0, array_size - 1); + auto [dist, gen] = uni_int_dist(0, static_cast(array_size - 1)); for (size_t i = 0; i < nb_elements_to_have; ++i) { elements_to_have[i] = array_elements[dist(gen)]; @@ -28,14 +28,14 @@ void arrayInit(T* elements_to_have, size_t nb_elements_to_have, T* array_element if (!all_elements_present) { /// make one element to be searched for missing from the target array - elements_to_have[nb_elements_to_have - 1] = array_size + 1; + elements_to_have[nb_elements_to_have - 1] = static_cast(array_size + 1); } } void nullMapInit(UInt8 * null_map, size_t null_map_size, size_t nb_null_elements) { /// -2 to keep the last element of the array non-null - auto [dist, gen] = uni_int_dist(0, null_map_size - 2); + auto [dist, gen] = uni_int_dist(0, static_cast(null_map_size - 2)); for (size_t i = 0; i < null_map_size; ++i) { null_map[i] = 0; diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index ac648b87448..32fe574f56a 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -133,17 +133,17 @@ namespace { static UInt32 execute(UInt16 d, Int64 days, const DateLUTImpl & time_zone, Int64) { - return time_zone.toStartOfDayInterval(ExtendedDayNum(d), days); + return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days)); } static UInt32 execute(Int32 d, Int64 days, const DateLUTImpl & time_zone, Int64) { - return time_zone.toStartOfDayInterval(ExtendedDayNum(d), days); + return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days)); } static UInt32 execute(UInt32 t, Int64 days, const DateLUTImpl & time_zone, Int64) { - return time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days); + return static_cast(time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days)); } static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier) @@ -517,6 +517,7 @@ private: ColumnPtr execute(const FromDataType &, const ColumnType & time_column_type, Int64 num_units, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const { using ToColumnType = typename ToDataType::ColumnType; + using ToFieldType = typename ToDataType::FieldType; const auto & time_data = time_column_type.getData(); size_t size = time_data.size(); @@ -529,7 +530,8 @@ private: Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); for (size_t i = 0; i != size; ++i) - result_data[i] = Transform::execute(time_data[i], num_units, time_zone, scale_multiplier); + result_data[i] = static_cast( + Transform::execute(time_data[i], num_units, time_zone, scale_multiplier)); return result_col; } diff --git a/src/Functions/toValidUTF8.cpp b/src/Functions/toValidUTF8.cpp index 9874e39baa4..4b79bc0bbda 100644 --- a/src/Functions/toValidUTF8.cpp +++ b/src/Functions/toValidUTF8.cpp @@ -106,7 +106,7 @@ struct ToValidUTF8Impl /// Sequence was not fully written to this buffer. break; } - else if (Poco::UTF8Encoding::isLegal(reinterpret_cast(p), len)) + else if (Poco::UTF8Encoding::isLegal(reinterpret_cast(p), static_cast(len))) { /// Valid sequence. p += len; diff --git a/src/IO/AIO.cpp b/src/IO/AIO.cpp index fb762271e4d..494ed3bae38 100644 --- a/src/IO/AIO.cpp +++ b/src/IO/AIO.cpp @@ -23,22 +23,22 @@ namespace DB int io_setup(unsigned nr, aio_context_t * ctxp) { - return syscall(__NR_io_setup, nr, ctxp); + return static_cast(syscall(__NR_io_setup, nr, ctxp)); } int io_destroy(aio_context_t ctx) { - return syscall(__NR_io_destroy, ctx); + return static_cast(syscall(__NR_io_destroy, ctx)); } int io_submit(aio_context_t ctx, long nr, struct iocb * iocbpp[]) // NOLINT { - return syscall(__NR_io_submit, ctx, nr, iocbpp); + return static_cast(syscall(__NR_io_submit, ctx, nr, iocbpp)); } int io_getevents(aio_context_t ctx, long min_nr, long max_nr, io_event * events, struct timespec * timeout) // NOLINT { - return syscall(__NR_io_getevents, ctx, min_nr, max_nr, events, timeout); + return static_cast(syscall(__NR_io_getevents, ctx, min_nr, max_nr, events, timeout)); } diff --git a/src/IO/Archives/ZipArchiveReader.cpp b/src/IO/Archives/ZipArchiveReader.cpp index 3127f299f5c..f48699a8408 100644 --- a/src/IO/Archives/ZipArchiveReader.cpp +++ b/src/IO/Archives/ZipArchiveReader.cpp @@ -281,7 +281,7 @@ private: bool nextImpl() override { RawHandle raw_handle = handle.getRawHandle(); - auto bytes_read = unzReadCurrentFile(raw_handle, internal_buffer.begin(), internal_buffer.size()); + auto bytes_read = unzReadCurrentFile(raw_handle, internal_buffer.begin(), static_cast(internal_buffer.size())); if (bytes_read < 0) checkResult(bytes_read); diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index 817e8132b64..d413783356d 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -134,7 +134,8 @@ private: if (!offset()) return; RawHandle raw_handle = handle.getRawHandle(); - checkResult(zipWriteInFileInZip(raw_handle, working_buffer.begin(), offset())); + int code = zipWriteInFileInZip(raw_handle, working_buffer.begin(), static_cast(offset())); + checkResult(code); } void checkResult(int code) const { handle.checkResult(code); } diff --git a/src/IO/Bzip2ReadBuffer.cpp b/src/IO/Bzip2ReadBuffer.cpp index 9d183393159..9970edcbcf3 100644 --- a/src/IO/Bzip2ReadBuffer.cpp +++ b/src/IO/Bzip2ReadBuffer.cpp @@ -85,11 +85,11 @@ bool Bzip2ReadBuffer::nextImpl() if (!bz->stream.avail_in) { in->nextIfAtEnd(); - bz->stream.avail_in = in->buffer().end() - in->position(); + bz->stream.avail_in = static_cast(in->buffer().end() - in->position()); bz->stream.next_in = in->position(); } - bz->stream.avail_out = internal_buffer.size(); + bz->stream.avail_out = static_cast(internal_buffer.size()); bz->stream.next_out = internal_buffer.begin(); ret = BZ2_bzDecompress(&bz->stream); @@ -99,7 +99,7 @@ bool Bzip2ReadBuffer::nextImpl() if (ret == BZ_STREAM_END && !in->eof()) { bz->reinitialize(); - bz->stream.avail_in = in->buffer().end() - in->position(); + bz->stream.avail_in = static_cast(in->buffer().end() - in->position()); bz->stream.next_in = in->position(); ret = BZ_OK; diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index 10a1803fec8..4b6bed70d35 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -58,7 +58,7 @@ void Bzip2WriteBuffer::nextImpl() } bz->stream.next_in = working_buffer.begin(); - bz->stream.avail_in = offset(); + bz->stream.avail_in = static_cast(offset()); try { @@ -66,7 +66,7 @@ void Bzip2WriteBuffer::nextImpl() { out->nextIfAtEnd(); bz->stream.next_out = out->position(); - bz->stream.avail_out = out->buffer().end() - out->position(); + bz->stream.avail_out = static_cast(out->buffer().end() - out->position()); int ret = BZ2_bzCompress(&bz->stream, BZ_RUN); @@ -95,7 +95,7 @@ void Bzip2WriteBuffer::finalizeBefore() out->nextIfAtEnd(); bz->stream.next_out = out->position(); - bz->stream.avail_out = out->buffer().end() - out->position(); + bz->stream.avail_out = static_cast(out->buffer().end() - out->position()); int ret = BZ2_bzCompress(&bz->stream, BZ_FINISH); diff --git a/src/IO/FileEncryptionCommon.cpp b/src/IO/FileEncryptionCommon.cpp index 13d8acb8c7b..5592da8721c 100644 --- a/src/IO/FileEncryptionCommon.cpp +++ b/src/IO/FileEncryptionCommon.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -84,10 +85,13 @@ namespace while (in_size < size) { out.nextIfAtEnd(); + size_t part_size = std::min(size - in_size, out.available()); + part_size = std::min(part_size, INT_MAX); + uint8_t * ciphertext = reinterpret_cast(out.position()); int ciphertext_size = 0; - if (!EVP_EncryptUpdate(evp_ctx, ciphertext, &ciphertext_size, &in[in_size], part_size)) + if (!EVP_EncryptUpdate(evp_ctx, ciphertext, &ciphertext_size, &in[in_size], static_cast(part_size))) throw Exception("Failed to encrypt", ErrorCodes::DATA_ENCRYPTION_ERROR); in_size += part_size; @@ -110,7 +114,7 @@ namespace uint8_t ciphertext[kBlockSize]; int ciphertext_size = 0; - if (!EVP_EncryptUpdate(evp_ctx, ciphertext, &ciphertext_size, padded_data, padded_data_size)) + if (!EVP_EncryptUpdate(evp_ctx, ciphertext, &ciphertext_size, padded_data, safe_cast(padded_data_size))) throw Exception("Failed to encrypt", ErrorCodes::DATA_ENCRYPTION_ERROR); if (!ciphertext_size) @@ -142,7 +146,7 @@ namespace const uint8_t * in = reinterpret_cast(data); uint8_t * plaintext = reinterpret_cast(out); int plaintext_size = 0; - if (!EVP_DecryptUpdate(evp_ctx, plaintext, &plaintext_size, in, size)) + if (!EVP_DecryptUpdate(evp_ctx, plaintext, &plaintext_size, in, safe_cast(size))) throw Exception("Failed to decrypt", ErrorCodes::DATA_ENCRYPTION_ERROR); return plaintext_size; } @@ -153,10 +157,9 @@ namespace uint8_t padded_data[kBlockSize] = {}; memcpy(&padded_data[pad_left], data, size); size_t padded_data_size = pad_left + size; - uint8_t plaintext[kBlockSize]; int plaintext_size = 0; - if (!EVP_DecryptUpdate(evp_ctx, plaintext, &plaintext_size, padded_data, padded_data_size)) + if (!EVP_DecryptUpdate(evp_ctx, plaintext, &plaintext_size, padded_data, safe_cast(padded_data_size))) throw Exception("Failed to decrypt", ErrorCodes::DATA_ENCRYPTION_ERROR); if (!plaintext_size) diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 9fd48914f64..f33b2399492 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -142,7 +142,7 @@ namespace bool proxy_https_, size_t max_pool_size_, bool resolve_host_ = true) - : Base(max_pool_size_, &Poco::Logger::get("HTTPSessionPool")) + : Base(static_cast(max_pool_size_), &Poco::Logger::get("HTTPSessionPool")) , host(host_) , port(port_) , https(https_) @@ -271,7 +271,7 @@ namespace }; } -void setResponseDefaultHeaders(HTTPServerResponse & response, unsigned keep_alive_timeout) +void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout) { if (!response.getKeepAlive()) return; diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index 18e83abb83b..51da17d4ca7 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -38,7 +38,7 @@ public: using PooledHTTPSessionPtr = SingleEndpointHTTPSessionPool::Entry; using HTTPSessionPtr = std::shared_ptr; -void setResponseDefaultHeaders(HTTPServerResponse & response, unsigned keep_alive_timeout); +void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout); /// Create session object to perform requests and set required parameters. HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, bool resolve_host = true); diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 988ad75cdf4..7ba23dd1588 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes extern const int NETWORK_ERROR; extern const int SOCKET_TIMEOUT; extern const int CANNOT_READ_FROM_SOCKET; + extern const int LOGICAL_ERROR; } @@ -54,7 +55,10 @@ bool ReadBufferFromPocoSocket::nextImpl() while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_READ)) async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), socket_description); - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), internal_buffer.size()); + if (internal_buffer.size() > INT_MAX) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); + + bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); } catch (const Poco::Net::NetException & e) { diff --git a/src/IO/VarInt.h b/src/IO/VarInt.h index 3161ca6d8a8..816aa8fd057 100644 --- a/src/IO/VarInt.h +++ b/src/IO/VarInt.h @@ -83,14 +83,14 @@ inline void readVarUInt(UInt32 & x, ReadBuffer & istr) { UInt64 tmp; readVarUInt(tmp, istr); - x = tmp; + x = static_cast(tmp); } inline void readVarInt(Int32 & x, ReadBuffer & istr) { Int64 tmp; readVarInt(tmp, istr); - x = tmp; + x = static_cast(tmp); } inline void readVarUInt(UInt16 & x, ReadBuffer & istr) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index fb4e5df9b59..95d532e9bd4 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -31,6 +31,7 @@ namespace ErrorCodes extern const int NETWORK_ERROR; extern const int SOCKET_TIMEOUT; extern const int CANNOT_WRITE_TO_SOCKET; + extern const int LOGICAL_ERROR; } @@ -55,7 +56,11 @@ void WriteBufferFromPocoSocket::nextImpl() try { CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkSend); - res = socket.impl()->sendBytes(working_buffer.begin() + bytes_written, offset() - bytes_written); + char * pos = working_buffer.begin() + bytes_written; + size_t size = offset() - bytes_written; + if (size > INT_MAX) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); + res = socket.impl()->sendBytes(pos, static_cast(size)); } catch (const Poco::Net::NetException & e) { diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 14118c3c04e..f823015bd7d 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -305,7 +305,7 @@ void WriteBufferFromS3::writePart() UploadPartTask task; auto & tags = TSA_SUPPRESS_WARNING_FOR_WRITE(part_tags); /// Suppress warning because schedule == false. - fillUploadRequest(task.req, tags.size() + 1); + fillUploadRequest(task.req, static_cast(tags.size() + 1)); processUploadRequest(task); tags.push_back(task.tag); } @@ -362,7 +362,7 @@ void WriteBufferFromS3::completeMultipartUpload() for (size_t i = 0; i < tags.size(); ++i) { Aws::S3::Model::CompletedPart part; - multipart_upload.AddParts(part.WithETag(tags[i]).WithPartNumber(i + 1)); + multipart_upload.AddParts(part.WithETag(tags[i]).WithPartNumber(static_cast(i + 1))); } req.SetMultipartUpload(multipart_upload); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 782e580d8be..28f831856d7 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -110,8 +110,8 @@ private: std::unique_ptr put_object_task; /// Does not need protection by mutex because of the logic around is_finished field. std::list TSA_GUARDED_BY(bg_tasks_mutex) upload_object_tasks; - size_t num_added_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0; - size_t num_finished_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0; + int num_added_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0; + int num_finished_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0; std::mutex bg_tasks_mutex; std::condition_variable bg_tasks_condvar; diff --git a/src/IO/WriteBufferValidUTF8.cpp b/src/IO/WriteBufferValidUTF8.cpp index 10e86f01343..4c8e172f43c 100644 --- a/src/IO/WriteBufferValidUTF8.cpp +++ b/src/IO/WriteBufferValidUTF8.cpp @@ -102,7 +102,7 @@ void WriteBufferValidUTF8::nextImpl() break; #endif - size_t len = length_of_utf8_sequence[static_cast(*p)]; + UInt8 len = length_of_utf8_sequence[static_cast(*p)]; if (len > 4) { // NOLINT diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index c3e1e59218f..42d84e080af 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -139,7 +139,7 @@ inline void writeBoolText(bool x, WriteBuffer & buf) template inline size_t writeFloatTextFastPath(T x, char * buffer) { - int result = 0; + Int64 result = 0; if constexpr (std::is_same_v) { diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp index c265791e38a..43014096e2a 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -49,7 +49,7 @@ void ZlibDeflatingWriteBuffer::nextImpl() return; zstr.next_in = reinterpret_cast(working_buffer.begin()); - zstr.avail_in = offset(); + zstr.avail_in = static_cast(offset()); try { @@ -57,7 +57,7 @@ void ZlibDeflatingWriteBuffer::nextImpl() { out->nextIfAtEnd(); zstr.next_out = reinterpret_cast(out->position()); - zstr.avail_out = out->buffer().end() - out->position(); + zstr.avail_out = static_cast(out->buffer().end() - out->position()); int rc = deflate(&zstr, Z_NO_FLUSH); out->position() = out->buffer().end() - zstr.avail_out; @@ -96,7 +96,7 @@ void ZlibDeflatingWriteBuffer::finalizeBefore() { out->nextIfAtEnd(); zstr.next_out = reinterpret_cast(out->position()); - zstr.avail_out = out->buffer().end() - out->position(); + zstr.avail_out = static_cast(out->buffer().end() - out->position()); int rc = deflate(&zstr, Z_FULL_FLUSH); out->position() = out->buffer().end() - zstr.avail_out; @@ -110,7 +110,7 @@ void ZlibDeflatingWriteBuffer::finalizeBefore() { out->nextIfAtEnd(); zstr.next_out = reinterpret_cast(out->position()); - zstr.avail_out = out->buffer().end() - out->position(); + zstr.avail_out = static_cast(out->buffer().end() - out->position()); int rc = deflate(&zstr, Z_FINISH); out->position() = out->buffer().end() - zstr.avail_out; diff --git a/src/IO/ZlibInflatingReadBuffer.cpp b/src/IO/ZlibInflatingReadBuffer.cpp index 4cb56bef6b1..9c2ee640cbe 100644 --- a/src/IO/ZlibInflatingReadBuffer.cpp +++ b/src/IO/ZlibInflatingReadBuffer.cpp @@ -61,11 +61,11 @@ bool ZlibInflatingReadBuffer::nextImpl() { in->nextIfAtEnd(); zstr.next_in = reinterpret_cast(in->position()); - zstr.avail_in = in->buffer().end() - in->position(); + zstr.avail_in = static_cast(in->buffer().end() - in->position()); } /// init output bytes (place, where decompressed data will be) zstr.next_out = reinterpret_cast(internal_buffer.begin()); - zstr.avail_out = internal_buffer.size(); + zstr.avail_out = static_cast(internal_buffer.size()); int rc = inflate(&zstr, Z_NO_FLUSH); diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index 459f486af18..79fb4ccead5 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -149,7 +149,7 @@ void ZstdDeflatingAppendableWriteBuffer::finalizeZstd() { try { - int err = ZSTD_freeCCtx(cctx); + size_t err = ZSTD_freeCCtx(cctx); /// This is just in case, since it is impossible to get an error by using this wrapper. if (unlikely(err)) throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, "ZSTD_freeCCtx failed: error: '{}'; zstd version: {}", ZSTD_getErrorName(err), ZSTD_VERSION_STRING); diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index 238645b16df..c7f9b0d718b 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -100,7 +100,7 @@ void ZstdDeflatingWriteBuffer::finalizeAfter() { try { - int err = ZSTD_freeCCtx(cctx); + size_t err = ZSTD_freeCCtx(cctx); /// This is just in case, since it is impossible to get an error by using this wrapper. if (unlikely(err)) throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, "ZSTD_freeCCtx failed: error: '{}'; zstd version: {}", ZSTD_getErrorName(err), ZSTD_VERSION_STRING); diff --git a/src/IO/readDecimalText.h b/src/IO/readDecimalText.h index 2e06acb2f3e..64374a20574 100644 --- a/src/IO/readDecimalText.h +++ b/src/IO/readDecimalText.h @@ -106,7 +106,7 @@ inline bool readDigits(ReadBuffer & buf, T & x, uint32_t & digits, int32_t & exp exponent -= places; // TODO: accurate shift10 for big integers - x *= intExp10OfSize(places); + x *= intExp10OfSize(places); places = 0; x += (byte - '0'); diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index b8d0c1ba3c0..a72ff82008e 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -349,11 +349,11 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) constexpr int significant_digits = std::numeric_limits::digits10; readUIntTextUpToNSignificantDigits(before_point, in); - int read_digits = in.count() - count_after_sign; + size_t read_digits = in.count() - count_after_sign; if (unlikely(read_digits > significant_digits)) { - int before_point_additional_exponent = read_digits - significant_digits; + int before_point_additional_exponent = static_cast(read_digits) - significant_digits; x = static_cast(shift10(before_point, before_point_additional_exponent)); } else @@ -377,11 +377,11 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) ++in.position(); auto after_leading_zeros_count = in.count(); - auto after_point_num_leading_zeros = after_leading_zeros_count - after_point_count; + int after_point_num_leading_zeros = static_cast(after_leading_zeros_count - after_point_count); readUIntTextUpToNSignificantDigits(after_point, in); read_digits = in.count() - after_leading_zeros_count; - after_point_exponent = (read_digits > significant_digits ? -significant_digits : -read_digits) - after_point_num_leading_zeros; + after_point_exponent = (read_digits > significant_digits ? -significant_digits : static_cast(-read_digits)) - after_point_num_leading_zeros; } if (checkChar('e', in) || checkChar('E', in)) diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index a27745d2cfa..fea013fd075 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -140,7 +140,7 @@ public: * when we add lots of column with same prefix. One counter for all * prefixes is good enough. */ - int next_unique_suffix; + size_t next_unique_suffix; Data( ContextPtr context_, diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index e9a72ce0156..182f2292b28 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1638,14 +1638,14 @@ Block Aggregator::convertOneBucketToBlock( Method & method, Arena * arena, bool final, - size_t bucket) const + Int32 bucket) const { // Used in ConvertingAggregatedToChunksSource -> ConvertingAggregatedToChunksTransform (expects single chunk for each bucket_id). constexpr bool return_single_block = true; Block block = convertToBlockImpl( method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size()); - block.info.bucket_num = bucket; + block.info.bucket_num = static_cast(bucket); return block; } @@ -1653,7 +1653,7 @@ Block Aggregator::mergeAndConvertOneBucketToBlock( ManyAggregatedDataVariants & variants, Arena * arena, bool final, - size_t bucket, + Int32 bucket, std::atomic * is_cancelled) const { auto & merged_data = *variants[0]; @@ -1697,7 +1697,7 @@ void Aggregator::writeToTemporaryFileImpl( max_temporary_block_size_bytes = block_size_bytes; }; - for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket) + for (UInt32 bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket) { Block block = convertOneBucketToBlock(data_variants, method, data_variants.aggregates_pool, false, bucket); out.write(block); @@ -3159,19 +3159,19 @@ void NO_INLINE Aggregator::convertBlockToTwoLevelImpl( selector[i] = bucket; } - size_t num_buckets = destinations.size(); + UInt32 num_buckets = static_cast(destinations.size()); for (size_t column_idx = 0; column_idx < columns; ++column_idx) { const ColumnWithTypeAndName & src_col = source.getByPosition(column_idx); MutableColumns scattered_columns = src_col.column->scatter(num_buckets, selector); - for (size_t bucket = 0, size = num_buckets; bucket < size; ++bucket) + for (UInt32 bucket = 0, size = num_buckets; bucket < size; ++bucket) { if (!scattered_columns[bucket]->empty()) { Block & dst = destinations[bucket]; - dst.info.bucket_num = bucket; + dst.info.bucket_num = static_cast(bucket); dst.insert({std::move(scattered_columns[bucket]), src_col.type, src_col.name}); } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index b8aab7a3343..c81cfa2c0a2 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1301,13 +1301,13 @@ private: Method & method, Arena * arena, bool final, - size_t bucket) const; + Int32 bucket) const; Block mergeAndConvertOneBucketToBlock( ManyAggregatedDataVariants & variants, Arena * arena, bool final, - size_t bucket, + Int32 bucket, std::atomic * is_cancelled = nullptr) const; Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const; diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 6877c0ece06..b76434b23e7 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -423,7 +423,7 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, info.all_addresses.push_back(address); auto pool = ConnectionPoolFactory::instance().get( - settings.distributed_connections_pool_size, + static_cast(settings.distributed_connections_pool_size), address.host_name, address.port, address.default_database, address.user, address.password, address.quota_key, address.cluster, address.cluster_secret, @@ -497,7 +497,7 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, for (const auto & replica : replica_addresses) { auto replica_pool = ConnectionPoolFactory::instance().get( - settings.distributed_connections_pool_size, + static_cast(settings.distributed_connections_pool_size), replica.host_name, replica.port, replica.default_database, replica.user, replica.password, replica.quota_key, replica.cluster, replica.cluster_secret, @@ -585,11 +585,11 @@ Cluster::Cluster( for (const auto & replica : current) { auto replica_pool = ConnectionPoolFactory::instance().get( - settings.distributed_connections_pool_size, - replica.host_name, replica.port, - replica.default_database, replica.user, replica.password, replica.quota_key, - replica.cluster, replica.cluster_secret, - "server", replica.compression, replica.secure, replica.priority); + static_cast(settings.distributed_connections_pool_size), + replica.host_name, replica.port, + replica.default_database, replica.user, replica.password, replica.quota_key, + replica.cluster, replica.cluster_secret, + "server", replica.compression, replica.secure, replica.priority); all_replicas.emplace_back(replica_pool); if (replica.is_local && !treat_local_as_remote) shard_local_addresses.push_back(replica); @@ -693,7 +693,7 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti info.all_addresses.push_back(address); auto pool = ConnectionPoolFactory::instance().get( - settings.distributed_connections_pool_size, + static_cast(settings.distributed_connections_pool_size), address.host_name, address.port, address.default_database, diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index fce2e9b2f08..4653491aac9 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -69,7 +69,7 @@ void SelectStreamFactory::createForShard( query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); }; - auto emplace_remote_stream = [&](bool lazy = false, UInt32 local_delay = 0) + auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0) { remote_shards.emplace_back(Shard{ .query = query_ast, @@ -131,7 +131,7 @@ void SelectStreamFactory::createForShard( return; } - UInt32 local_delay = replicated_storage->getAbsoluteDelay(); + UInt64 local_delay = replicated_storage->getAbsoluteDelay(); if (local_delay < max_allowed_delay) { @@ -205,7 +205,7 @@ SelectStreamFactory::ShardPlans SelectStreamFactory::createForShardWithParallelR if (!max_allowed_delay) return false; - UInt32 local_delay = replicated_storage->getAbsoluteDelay(); + UInt64 local_delay = replicated_storage->getAbsoluteDelay(); return local_delay >= max_allowed_delay; }; diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 440017a8e80..8ebddea4988 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -46,7 +46,7 @@ public: /// If we connect to replicas lazily. /// (When there is a local replica with big delay). bool lazy = false; - UInt32 local_delay = 0; + time_t local_delay = 0; }; using Shards = std::vector; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 923b4a767b7..0a62e04e090 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -180,7 +180,7 @@ void executeQuery( stream_factory.createForShard(shard_info, query_ast_for_shard, main_table, table_func_ptr, - new_context, plans, remote_shards, shards); + new_context, plans, remote_shards, static_cast(shards)); } if (!remote_shards.empty()) @@ -284,7 +284,8 @@ void executeQueryWithParallelReplicas( query_ast_for_shard = query_ast; auto shard_plans = stream_factory.createForShardWithParallelReplicas(shard_info, - query_ast_for_shard, main_table, table_func_ptr, throttler, context, shards, query_info.storage_limits); + query_ast_for_shard, main_table, table_func_ptr, throttler, context, + static_cast(shards), query_info.storage_limits); if (!shard_plans.local_plan && !shard_plans.remote_plan) throw Exception(ErrorCodes::LOGICAL_ERROR, "No plans were generated for reading from shard. This is a bug"); diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index b7ad56dca91..cc79a71245b 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -38,7 +38,7 @@ static UInt32 toPowerOfTwo(UInt32 x) ConcurrentHashJoin::ConcurrentHashJoin(ContextPtr context_, std::shared_ptr table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_) : context(context_) , table_join(table_join_) - , slots(toPowerOfTwo(std::min(slots_, 256))) + , slots(toPowerOfTwo(std::min(static_cast(slots_), 256))) { for (size_t i = 0; i < slots; ++i) { diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 0e4c658a1ee..f1fa0955c34 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -498,7 +498,7 @@ bool DDLWorker::tryExecuteQuery(const String & query, DDLTaskBase & task, const void DDLWorker::updateMaxDDLEntryID(const String & entry_name) { - UInt64 id = DDLTaskBase::getLogEntryNumber(entry_name); + UInt32 id = DDLTaskBase::getLogEntryNumber(entry_name); auto prev_id = max_id.load(std::memory_order_relaxed); while (prev_id < id) { diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index e3c1fa4c271..5aea460ad2e 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -159,7 +159,7 @@ protected: /// How many tasks could be in the queue size_t max_tasks_in_queue = 1000; - std::atomic max_id = 0; + std::atomic max_id = 0; const CurrentMetrics::Metric * max_entry_metric; const CurrentMetrics::Metric * max_pushed_entry_metric; }; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index ab6f088ac69..7ceb0bf3a00 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -144,9 +144,9 @@ StoragePtr TemporaryTableHolder::getTable() const void DatabaseCatalog::initializeAndLoadTemporaryDatabase() { drop_delay_sec = getContext()->getConfigRef().getInt("database_atomic_delay_before_drop_table_sec", default_drop_delay_sec); - unused_dir_hide_timeout_sec = getContext()->getConfigRef().getInt("database_catalog_unused_dir_hide_timeout_sec", unused_dir_hide_timeout_sec); - unused_dir_rm_timeout_sec = getContext()->getConfigRef().getInt("database_catalog_unused_dir_rm_timeout_sec", unused_dir_rm_timeout_sec); - unused_dir_cleanup_period_sec = getContext()->getConfigRef().getInt("database_catalog_unused_dir_cleanup_period_sec", unused_dir_cleanup_period_sec); + unused_dir_hide_timeout_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_hide_timeout_sec", unused_dir_hide_timeout_sec); + unused_dir_rm_timeout_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_rm_timeout_sec", unused_dir_rm_timeout_sec); + unused_dir_cleanup_period_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_cleanup_period_sec", unused_dir_cleanup_period_sec); auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE, getContext()); attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 74ff055efd4..f41ae31b239 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1698,8 +1698,12 @@ void InterpreterCreateQuery::addColumnsDescriptionToCreateQueryIfNecessary(ASTCr return; auto ast_storage = std::make_shared(); - auto query_from_storage = DB::getCreateQueryFromStorage(storage, ast_storage, false, - getContext()->getSettingsRef().max_parser_depth, true); + unsigned max_parser_depth = static_cast(getContext()->getSettingsRef().max_parser_depth); + auto query_from_storage = DB::getCreateQueryFromStorage(storage, + ast_storage, + false, + max_parser_depth, + true); auto & create_query_from_storage = query_from_storage->as(); if (!create.columns_list) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 51a3dde261a..107740c3b96 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -292,7 +292,7 @@ Chain InterpreterInsertQuery::buildChainImpl( out.addSource(std::make_shared( out.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0)); + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); } auto counting = std::make_shared(out.getInputHeader(), thread_status, getContext()->getQuota()); diff --git a/src/Interpreters/JIT/compileFunction.cpp b/src/Interpreters/JIT/compileFunction.cpp index d3a7eb0cfaa..e12b4894eb0 100644 --- a/src/Interpreters/JIT/compileFunction.cpp +++ b/src/Interpreters/JIT/compileFunction.cpp @@ -614,7 +614,7 @@ static void compileSortDescription(llvm::Module & module, /** Join results from all comparator steps. * Result of columns comparison equals to first compare block where lhs is not equal to lhs or last compare block. */ - auto * compare_result_phi = b.CreatePHI(b.getInt8Ty(), comparator_steps_and_results.size()); + auto * compare_result_phi = b.CreatePHI(b.getInt8Ty(), static_cast(comparator_steps_and_results.size())); for (const auto & [block, result_value] : comparator_steps_and_results) compare_result_phi->addIncoming(result_value, block); diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp index 40c42f7728e..a0fdafc976c 100644 --- a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp @@ -45,8 +45,7 @@ void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, SelectUnionModesSet current_set_of_modes; bool distinct_found = false; - int i; - for (i = union_modes.size() - 1; i >= 0; --i) + for (ssize_t i = union_modes.size() - 1; i >= 0; --i) { current_set_of_modes.insert(union_modes[i]); if (const auto * union_ast = typeid_cast(select_list[i + 1].get())) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index d5194a02513..d7e0ad23643 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -243,7 +243,12 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as } auto process_it = processes.emplace(processes.end(), - query_context, query_, client_info, priorities.insert(settings.priority), std::move(thread_group), query_kind); + query_context, + query_, + client_info, + priorities.insert(static_cast(settings.priority)), + std::move(thread_group), + query_kind); increaseQueryKindAmount(query_kind); diff --git a/src/Interpreters/RowRefs.cpp b/src/Interpreters/RowRefs.cpp index 68076e1fec2..0e553ef145e 100644 --- a/src/Interpreters/RowRefs.cpp +++ b/src/Interpreters/RowRefs.cpp @@ -89,7 +89,7 @@ public: assert(!sorted.load(std::memory_order_acquire)); - entries.emplace_back(key, row_refs.size()); + entries.emplace_back(key, static_cast(row_refs.size())); row_refs.emplace_back(RowRef(block, row_num)); } diff --git a/src/Interpreters/RowRefs.h b/src/Interpreters/RowRefs.h index 2c9f2062a82..197ac3ca00f 100644 --- a/src/Interpreters/RowRefs.h +++ b/src/Interpreters/RowRefs.h @@ -29,7 +29,10 @@ struct RowRef SizeT row_num = 0; RowRef() = default; - RowRef(const Block * block_, size_t row_num_) : block(block_), row_num(row_num_) {} + RowRef(const Block * block_, size_t row_num_) + : block(block_) + , row_num(static_cast(row_num_)) + {} }; /// Single linked list of references to rows. Used for ALL JOINs (non-unique JOINs) diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 3eb93f1f20e..c5ae6f6c885 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes extern const int NOT_ENOUGH_SPACE; } -void TemporaryDataOnDiskScope::deltaAllocAndCheck(int compressed_delta, int uncompressed_delta) +void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta) { if (parent) parent->deltaAllocAndCheck(compressed_delta, uncompressed_delta); diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index 81bd2067650..11edc8700d2 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -53,7 +53,7 @@ public: VolumePtr getVolume() const { return volume; } protected: - void deltaAllocAndCheck(int compressed_delta, int uncompressed_delta); + void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta); TemporaryDataOnDiskScopePtr parent = nullptr; VolumePtr volume; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 4810174e395..ee126f2da11 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -55,12 +55,12 @@ void ThreadStatus::applyQuerySettings() #if defined(OS_LINUX) /// Set "nice" value if required. - Int32 new_os_thread_priority = settings.os_thread_priority; + Int32 new_os_thread_priority = static_cast(settings.os_thread_priority); if (new_os_thread_priority && hasLinuxCapability(CAP_SYS_NICE)) { LOG_TRACE(log, "Setting nice to {}", new_os_thread_priority); - if (0 != setpriority(PRIO_PROCESS, thread_id, new_os_thread_priority)) + if (0 != setpriority(PRIO_PROCESS, static_cast(thread_id), new_os_thread_priority)) throwFromErrno("Cannot 'setpriority'", ErrorCodes::CANNOT_SET_THREAD_PRIORITY); os_thread_priority = new_os_thread_priority; @@ -349,7 +349,7 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) { LOG_TRACE(log, "Resetting nice"); - if (0 != setpriority(PRIO_PROCESS, thread_id, 0)) + if (0 != setpriority(PRIO_PROCESS, static_cast(thread_id), 0)) LOG_ERROR(log, "Cannot 'setpriority' back to zero: {}", errnoToString()); os_thread_priority = 0; diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 55156cde7be..9ba171d2665 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -88,7 +88,7 @@ Field convertIntToDecimalType(const Field & from, const DataTypeDecimal & typ if (!type.canStoreWhole(value)) throw Exception("Number is too big to place in " + type.getName(), ErrorCodes::ARGUMENT_OUT_OF_BOUND); - T scaled_value = type.getScaleMultiplier() * static_cast(value); + T scaled_value = type.getScaleMultiplier() * T(static_cast(value)); return DecimalField(scaled_value, type.getScale()); } diff --git a/src/Interpreters/createBlockSelector.cpp b/src/Interpreters/createBlockSelector.cpp index b1a9a4e9e35..fce9833ddfb 100644 --- a/src/Interpreters/createBlockSelector.cpp +++ b/src/Interpreters/createBlockSelector.cpp @@ -50,7 +50,7 @@ IColumn::Selector createBlockSelector( /// libdivide support only UInt32 and UInt64. using TUInt32Or64 = std::conditional_t; - libdivide::divider divider(total_weight); + libdivide::divider divider(static_cast(total_weight)); const auto & data = typeid_cast &>(column).getData(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index abca563de55..1a9940307d3 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -944,9 +944,10 @@ static std::tuple executeQueryImpl( processor_elem.processor_name = processor->getName(); - processor_elem.elapsed_us = processor->getElapsedUs(); - processor_elem.input_wait_elapsed_us = processor->getInputWaitElapsedUs(); - processor_elem.output_wait_elapsed_us = processor->getOutputWaitElapsedUs(); + /// NOTE: convert this to UInt64 + processor_elem.elapsed_us = static_cast(processor->getElapsedUs()); + processor_elem.input_wait_elapsed_us = static_cast(processor->getInputWaitElapsedUs()); + processor_elem.output_wait_elapsed_us = static_cast(processor->getOutputWaitElapsedUs()); auto stats = processor->getProcessorDataStats(); processor_elem.input_rows = stats.input_rows; diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index 90278e27c0c..86dd85e0eb8 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -93,7 +93,7 @@ void ASTTTLElement::setExpression(int & pos, ASTPtr && ast) { if (pos == -1) { - pos = children.size(); + pos = static_cast(children.size()); children.emplace_back(ast); } else diff --git a/src/Parsers/ParserSampleRatio.cpp b/src/Parsers/ParserSampleRatio.cpp index 2f444bcf9e8..b6be04cbcc0 100644 --- a/src/Parsers/ParserSampleRatio.cpp +++ b/src/Parsers/ParserSampleRatio.cpp @@ -14,7 +14,7 @@ static bool parseDecimal(const char * pos, const char * end, ASTSampleRatio::Rat { UInt64 num_before = 0; UInt64 num_after = 0; - Int64 exponent = 0; + Int32 exponent = 0; const char * pos_after_first_num = tryReadIntText(num_before, pos, end); @@ -28,12 +28,12 @@ static bool parseDecimal(const char * pos, const char * end, ASTSampleRatio::Rat if (!has_num_before_point && !has_point) return false; - size_t number_of_digits_after_point = 0; + int number_of_digits_after_point = 0; if (has_point) { const char * pos_after_second_num = tryReadIntText(num_after, pos, end); - number_of_digits_after_point = pos_after_second_num - pos; + number_of_digits_after_point = static_cast(pos_after_second_num - pos); pos = pos_after_second_num; } diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index af8c9dc58a6..4a0c60da48d 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -236,7 +236,8 @@ ASTPtr tryParseQuery( { const char * query_begin = _out_query_end; Tokens tokens(query_begin, all_queries_end, max_query_size); - IParser::Pos token_iterator(tokens, max_parser_depth); + /// NOTE: consider use UInt32 for max_parser_depth setting. + IParser::Pos token_iterator(tokens, static_cast(max_parser_depth)); if (token_iterator->isEnd() || token_iterator->type == TokenType::Semicolon) diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 52395338279..abd91ddcf35 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -188,7 +188,7 @@ Chunk IRowInputFormat::generate() } e.setFileName(getFileNameFromReadBuffer(getReadBuffer())); - e.setLineNumber(total_rows); + e.setLineNumber(static_cast(total_rows)); e.addMessage(verbose_diagnostic); throw; } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 70510a165e6..e9b01ec7dda 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -82,7 +82,7 @@ static ColumnWithTypeAndName readColumnWithNumericData(std::shared_ptr(*internal_column).getData(); column_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { std::shared_ptr chunk = arrow_column->chunk(chunk_i); if (chunk->length() == 0) @@ -108,7 +108,7 @@ static ColumnWithTypeAndName readColumnWithStringData(std::shared_ptr & column_offsets = assert_cast(*internal_column).getOffsets(); size_t chars_t_size = 0; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { ArrowArray & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); const size_t chunk_length = chunk.length(); @@ -123,7 +123,7 @@ static ColumnWithTypeAndName readColumnWithStringData(std::shared_ptrlength()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { ArrowArray & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); std::shared_ptr buffer = chunk.value_data(); @@ -151,7 +151,7 @@ static ColumnWithTypeAndName readColumnWithBooleanData(std::shared_ptr &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { arrow::BooleanArray & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); if (chunk.length() == 0) @@ -173,7 +173,7 @@ static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr & column_data = assert_cast &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { arrow::Date32Array & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); @@ -198,7 +198,7 @@ static ColumnWithTypeAndName readColumnWithDate64Data(std::shared_ptr &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) @@ -219,7 +219,7 @@ static ColumnWithTypeAndName readColumnWithTimestampData(std::shared_ptr &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { const auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) @@ -239,7 +239,7 @@ static ColumnWithTypeAndName readColumnWithTimeData(std::shared_ptrcreateColumn(); internal_column->reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); if (chunk.length() == 0) @@ -272,7 +272,7 @@ static ColumnWithTypeAndName readColumnWithDecimalDataImpl(std::shared_ptrlength()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) @@ -308,7 +308,7 @@ static ColumnPtr readByteMapFromArrowColumn(std::shared_ptr PaddedPODArray & bytemap_data = assert_cast &>(*nullmap_column).getData(); bytemap_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->num_chunks()); ++chunk_i) + for (int chunk_i = 0; chunk_i != arrow_column->num_chunks(); ++chunk_i) { std::shared_ptr chunk = arrow_column->chunk(chunk_i); @@ -324,7 +324,7 @@ static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr &>(*offsets_column).getData(); offsets_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { arrow::ListArray & list_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); auto arrow_offsets_array = list_chunk.offsets(); @@ -356,7 +356,7 @@ static ColumnWithTypeAndName readColumnWithIndexesDataImpl(std::shared_ptrlength()); NumericType shift = is_nullable ? 2 : 1; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { std::shared_ptr chunk = arrow_column->chunk(chunk_i); if (chunk->length() == 0) @@ -450,7 +450,8 @@ static ColumnPtr readColumnWithIndexesData(std::shared_ptr # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ case ARROW_NUMERIC_TYPE: \ { \ - return readColumnWithIndexesDataImpl(arrow_column, "", default_value_index, dict_size, is_nullable).column; \ + return readColumnWithIndexesDataImpl(\ + arrow_column, "", default_value_index, static_cast(dict_size), is_nullable).column; \ } FOR_ARROW_INDEXES_TYPES(DISPATCH) # undef DISPATCH @@ -463,7 +464,7 @@ static std::shared_ptr getNestedArrowColumn(std::shared_ptr { arrow::ArrayVector array_vector; array_vector.reserve(arrow_column->num_chunks()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { arrow::ListArray & list_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); std::shared_ptr chunk = list_chunk.values(); @@ -582,7 +583,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( auto arrow_type = arrow_column->type(); auto * arrow_struct_type = assert_cast(arrow_type.get()); std::vector nested_arrow_columns(arrow_struct_type->num_fields()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { arrow::StructArray & struct_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); for (int i = 0; i < arrow_struct_type->num_fields(); ++i) @@ -631,7 +632,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( if (!dict_info.values) { arrow::ArrayVector dict_array; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { arrow::DictionaryArray & dict_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); dict_array.emplace_back(dict_chunk.dictionary()); @@ -656,7 +657,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( } arrow::ArrayVector indexes_array; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { arrow::DictionaryArray & dict_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); indexes_array.emplace_back(dict_chunk.indices()); diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 0ec04c61321..abb23d68334 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -283,14 +283,15 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node if (root_node->leaves() == 2 && (root_node->leafAt(0)->type() == avro::AVRO_NULL || root_node->leafAt(1)->type() == avro::AVRO_NULL)) { - size_t non_null_union_index = root_node->leafAt(0)->type() == avro::AVRO_NULL ? 1 : 0; + int non_null_union_index = root_node->leafAt(0)->type() == avro::AVRO_NULL ? 1 : 0; if (target.isNullable()) { - auto nested_deserialize = this->createDeserializeFn(root_node->leafAt(non_null_union_index), removeNullable(target_type)); + auto nested_deserialize = this->createDeserializeFn( + root_node->leafAt(non_null_union_index), removeNullable(target_type)); return [non_null_union_index, nested_deserialize](IColumn & column, avro::Decoder & decoder) { ColumnNullable & col = assert_cast(column); - size_t union_index = decoder.decodeUnionIndex(); + int union_index = static_cast(decoder.decodeUnionIndex()); if (union_index == non_null_union_index) { nested_deserialize(col.getNestedColumn(), decoder); @@ -308,7 +309,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node auto nested_deserialize = this->createDeserializeFn(root_node->leafAt(non_null_union_index), target_type); return [non_null_union_index, nested_deserialize](IColumn & column, avro::Decoder & decoder) { - size_t union_index = decoder.decodeUnionIndex(); + int union_index = static_cast(decoder.decodeUnionIndex()); if (union_index == non_null_union_index) nested_deserialize(column, decoder); else @@ -345,7 +346,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node if (target.isString()) { std::vector symbols; - for (size_t i = 0; i < root_node->names(); ++i) + for (int i = 0; i < static_cast(root_node->names()); ++i) { symbols.push_back(root_node->nameAt(i)); } @@ -360,7 +361,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node { const auto & enum_type = dynamic_cast(*target_type); Row symbol_mapping; - for (size_t i = 0; i < root_node->names(); ++i) + for (int i = 0; i < static_cast(root_node->names()); ++i) { symbol_mapping.push_back(enum_type.castToValue(root_node->nameAt(i))); } @@ -397,7 +398,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node if (root_node->leaves() != nested_types.size()) throw Exception(ErrorCodes::INCORRECT_DATA, "The number of leaves in record doesn't match the number of elements in tuple"); - for (size_t i = 0; i != root_node->leaves(); ++i) + for (int i = 0; i != static_cast(root_node->leaves()); ++i) { const auto & name = root_node->nameAt(i); size_t pos = tuple_type.getPositionByName(name); @@ -505,7 +506,7 @@ AvroDeserializer::SkipFn AvroDeserializer::createSkipFn(avro::NodePtr root_node) case avro::AVRO_UNION: { std::vector union_skip_fns; - for (size_t i = 0; i < root_node->leaves(); ++i) + for (int i = 0; i < static_cast(root_node->leaves()); ++i) { union_skip_fns.push_back(createSkipFn(root_node->leafAt(i))); } @@ -546,7 +547,7 @@ AvroDeserializer::SkipFn AvroDeserializer::createSkipFn(avro::NodePtr root_node) case avro::AVRO_RECORD: { std::vector field_skip_fns; - for (size_t i = 0; i < root_node->leaves(); ++i) + for (int i = 0; i < static_cast(root_node->leaves()); ++i) { field_skip_fns.push_back(createSkipFn(root_node->leafAt(i))); } @@ -633,7 +634,7 @@ AvroDeserializer::Action AvroDeserializer::createAction(const Block & header, co const auto & column = header.getByPosition(target_column_idx); try { - AvroDeserializer::Action action(target_column_idx, createDeserializeFn(node, column.type)); + AvroDeserializer::Action action(static_cast(target_column_idx), createDeserializeFn(node, column.type)); column_found[target_column_idx] = true; return action; } @@ -646,7 +647,7 @@ AvroDeserializer::Action AvroDeserializer::createAction(const Block & header, co else if (node->type() == avro::AVRO_RECORD) { std::vector field_actions(node->leaves()); - for (size_t i = 0; i < node->leaves(); ++i) + for (int i = 0; i < static_cast(node->leaves()); ++i) { const auto & field_node = node->leafAt(i); const auto & field_name = node->nameAt(i); @@ -657,7 +658,7 @@ AvroDeserializer::Action AvroDeserializer::createAction(const Block & header, co else if (node->type() == avro::AVRO_UNION) { std::vector branch_actions(node->leaves()); - for (size_t i = 0; i < node->leaves(); ++i) + for (int i = 0; i < static_cast(node->leaves()); ++i) { const auto & branch_node = node->leafAt(i); const auto & branch_name = nodeName(branch_node); @@ -687,7 +688,7 @@ AvroDeserializer::Action AvroDeserializer::createAction(const Block & header, co /// Create nested deserializer for each nested column. std::vector nested_deserializers; std::vector nested_indexes; - for (size_t i = 0; i != nested_avro_node->leaves(); ++i) + for (int i = 0; i != static_cast(nested_avro_node->leaves()); ++i) { const auto & name = nested_avro_node->nameAt(i); if (!nested_types.contains(name)) @@ -970,7 +971,7 @@ NamesAndTypesList AvroSchemaReader::readSchema() throw Exception("Root schema must be a record", ErrorCodes::TYPE_MISMATCH); NamesAndTypesList names_and_types; - for (size_t i = 0; i != root_node->leaves(); ++i) + for (int i = 0; i != static_cast(root_node->leaves()); ++i) names_and_types.emplace_back(root_node->nameAt(i), avroNodeToDataType(root_node->leafAt(i))); return names_and_types; @@ -999,14 +1000,14 @@ DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node) if (node->names() < 128) { EnumValues::Values values; - for (size_t i = 0; i != node->names(); ++i) + for (int i = 0; i != static_cast(node->names()); ++i) values.emplace_back(node->nameAt(i), i); return std::make_shared(std::move(values)); } else if (node->names() < 32768) { EnumValues::Values values; - for (size_t i = 0; i != node->names(); ++i) + for (int i = 0; i != static_cast(node->names()); ++i) values.emplace_back(node->nameAt(i), i); return std::make_shared(std::move(values)); } @@ -1022,7 +1023,7 @@ DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node) case avro::Type::AVRO_UNION: if (node->leaves() == 2 && (node->leafAt(0)->type() == avro::Type::AVRO_NULL || node->leafAt(1)->type() == avro::Type::AVRO_NULL)) { - size_t nested_leaf_index = node->leafAt(0)->type() == avro::Type::AVRO_NULL ? 1 : 0; + int nested_leaf_index = node->leafAt(0)->type() == avro::Type::AVRO_NULL ? 1 : 0; auto nested_type = avroNodeToDataType(node->leafAt(nested_leaf_index)); return nested_type->canBeInsideNullable() ? makeNullable(nested_type) : nested_type; } @@ -1035,7 +1036,7 @@ DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node) nested_types.reserve(node->leaves()); Names nested_names; nested_names.reserve(node->leaves()); - for (size_t i = 0; i != node->leaves(); ++i) + for (int i = 0; i != static_cast(node->leaves()); ++i) { nested_types.push_back(avroNodeToDataType(node->leafAt(i))); nested_names.push_back(node->nameAt(i)); diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index b63b1e7b9b1..e3d570d1876 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -193,7 +193,7 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF case TypeIndex::FixedString: { auto size = data_type->getSizeOfValueInMemory(); - auto schema = avro::FixedSchema(size, "fixed_" + toString(type_name_increment)); + auto schema = avro::FixedSchema(static_cast(size), "fixed_" + toString(type_name_increment)); return {schema, [](const IColumn & column, size_t row_num, avro::Encoder & encoder) { const std::string_view & s = assert_cast(column).getDataAt(row_num).toView(); diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 9e423f8a96b..87fff16c107 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -224,7 +224,14 @@ namespace DB for (size_t i = 0; i != column_tuple->tupleSize(); ++i) { ColumnPtr nested_column = column_tuple->getColumnPtr(i); - fillArrowArray(column_name + "." + nested_names[i], nested_column, nested_types[i], null_bytemap, builder.field_builder(i), format_name, start, end, output_string_as_string, dictionary_values); + fillArrowArray( + column_name + "." + nested_names[i], + nested_column, nested_types[i], null_bytemap, + builder.field_builder(static_cast(i)), + format_name, + start, end, + output_string_as_string, + dictionary_values); } for (size_t i = start; i != end; ++i) @@ -370,7 +377,7 @@ namespace DB else { std::string_view string_ref = internal_column.getDataAt(string_i).toView(); - status = builder.Append(string_ref.data(), string_ref.size()); + status = builder.Append(string_ref.data(), static_cast(string_ref.size())); } checkStatus(status, write_column->getName(), format_name); } diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index c6f8742455e..08d2cac743a 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -91,7 +91,7 @@ static void insertSignedInteger(IColumn & column, const DataTypePtr & column_typ assert_cast(column).insertValue(value); break; case TypeIndex::Int32: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::Int64: assert_cast(column).insertValue(value); @@ -117,7 +117,7 @@ static void insertUnsignedInteger(IColumn & column, const DataTypePtr & column_t break; case TypeIndex::DateTime: [[fallthrough]]; case TypeIndex::UInt32: - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; case TypeIndex::UInt64: assert_cast(column).insertValue(value); diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index fd33abfb587..654917b6357 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -79,7 +79,7 @@ static capnp::DynamicValue::Builder initStructFieldBuilder(const ColumnPtr & col if (const auto * array_column = checkAndGetColumn(*column)) { size_t size = array_column->getOffsets()[row_num] - array_column->getOffsets()[row_num - 1]; - return struct_builder.init(field, size); + return struct_builder.init(field, static_cast(size)); } if (field.getType().isStruct()) @@ -200,7 +200,7 @@ static std::optional convertToDynamicValue( size_t size = offsets[row_num] - offset; const auto * nested_array_column = checkAndGetColumn(*nested_column); - for (size_t i = 0; i != size; ++i) + for (unsigned i = 0; i != static_cast(size); ++i) { capnp::DynamicValue::Builder value_builder; /// For nested arrays we need to initialize nested list builder. @@ -208,7 +208,7 @@ static std::optional convertToDynamicValue( { const auto & nested_offset = nested_array_column->getOffsets(); size_t nested_array_size = nested_offset[offset + i] - nested_offset[offset + i - 1]; - value_builder = list_builder.init(i, nested_array_size); + value_builder = list_builder.init(i, static_cast(nested_array_size)); } else value_builder = list_builder[i]; diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 739fa8735b2..bc363e5aa98 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -502,7 +502,7 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co ParserTupleOfLiterals parser_tuple; Tokens tokens_number(istr.position(), istr.buffer().end()); - IParser::Pos iterator(tokens_number, settings.max_parser_depth); + IParser::Pos iterator(tokens_number, static_cast(settings.max_parser_depth)); Expected expected; ASTPtr ast; if (!parser_array.parse(iterator, ast, expected) && !parser_tuple.parse(iterator, ast, expected)) diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 931a7587903..80fdda687e2 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -128,7 +128,7 @@ static void insertInteger(IColumn & column, DataTypePtr type, UInt64 value) case TypeIndex::DateTime: [[fallthrough]]; case TypeIndex::UInt32: { - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; } case TypeIndex::UInt64: @@ -148,7 +148,7 @@ static void insertInteger(IColumn & column, DataTypePtr type, UInt64 value) } case TypeIndex::Int32: { - assert_cast(column).insertValue(value); + assert_cast(column).insertValue(static_cast(value)); break; } case TypeIndex::Int64: diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index a470e193300..da683913d4d 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -99,15 +99,15 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr case TypeIndex::String: { const std::string_view & string = assert_cast(column).getDataAt(row_num).toView(); - packer.pack_bin(string.size()); - packer.pack_bin_body(string.data(), string.size()); + packer.pack_bin(static_cast(string.size())); + packer.pack_bin_body(string.data(), static_cast(string.size())); return; } case TypeIndex::FixedString: { const std::string_view & string = assert_cast(column).getDataAt(row_num).toView(); - packer.pack_bin(string.size()); - packer.pack_bin_body(string.data(), string.size()); + packer.pack_bin(static_cast(string.size())); + packer.pack_bin_body(string.data(), static_cast(string.size())); return; } case TypeIndex::Array: @@ -118,7 +118,7 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr const ColumnArray::Offsets & offsets = column_array.getOffsets(); size_t offset = offsets[row_num - 1]; size_t size = offsets[row_num] - offset; - packer.pack_array(size); + packer.pack_array(static_cast(size)); for (size_t i = 0; i < size; ++i) { serializeField(nested_column, nested_type, offset + i); @@ -152,7 +152,7 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr const auto & offsets = nested_column.getOffsets(); size_t offset = offsets[row_num - 1]; size_t size = offsets[row_num] - offset; - packer.pack_map(size); + packer.pack_map(static_cast(size)); for (size_t i = 0; i < size; ++i) { serializeField(*key_column, map_type.getKeyType(), offset + i); @@ -179,8 +179,8 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr WriteBufferFromOwnString buf; writeBinary(uuid_column.getElement(row_num), buf); std::string_view uuid_bin = buf.stringView(); - packer.pack_bin(uuid_bin.size()); - packer.pack_bin_body(uuid_bin.data(), uuid_bin.size()); + packer.pack_bin(static_cast(uuid_bin.size())); + packer.pack_bin_body(uuid_bin.data(), static_cast(uuid_bin.size())); return; } case FormatSettings::MsgPackUUIDRepresentation::STR: @@ -188,8 +188,8 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr WriteBufferFromOwnString buf; writeText(uuid_column.getElement(row_num), buf); std::string_view uuid_text = buf.stringView(); - packer.pack_str(uuid_text.size()); - packer.pack_bin_body(uuid_text.data(), uuid_text.size()); + packer.pack_str(static_cast(uuid_text.size())); + packer.pack_bin_body(uuid_text.data(), static_cast(uuid_text.size())); return; } case FormatSettings::MsgPackUUIDRepresentation::EXT: @@ -200,7 +200,7 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr writeBinaryBigEndian(value.toUnderType().items[1], buf); std::string_view uuid_ext = buf.stringView(); packer.pack_ext(sizeof(UUID), int8_t(MsgPackExtensionTypes::UUIDType)); - packer.pack_ext_body(uuid_ext.data(), uuid_ext.size()); + packer.pack_ext_body(uuid_ext.data(), static_cast(uuid_ext.size())); return; } } diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 344c5c179db..b6b1b875add 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -65,7 +65,7 @@ void MySQLOutputFormat::consume(Chunk chunk) { for (size_t i = 0; i < chunk.getNumRows(); ++i) { - ProtocolText::ResultSetRow row_packet(serializations, chunk.getColumns(), i); + ProtocolText::ResultSetRow row_packet(serializations, chunk.getColumns(), static_cast(i)); packet_endpoint->sendPacket(row_packet); } } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 36126c21bf1..d6dbd69135a 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -136,7 +136,7 @@ void ORCBlockInputFormat::prepareReader() if (is_stopped) return; - stripe_total = file_reader->NumberOfStripes(); + stripe_total = static_cast(file_reader->NumberOfStripes()); stripe_current = 0; arrow_column_to_ch_column = std::make_unique( @@ -159,7 +159,7 @@ void ORCBlockInputFormat::prepareReader() { /// LIST type require 2 indices, STRUCT - the number of elements + 1, /// so we should recursively count the number of indices we need for this type. - int indexes_count = countIndicesForType(schema->field(i)->type()); + int indexes_count = static_cast(countIndicesForType(schema->field(i)->type())); const auto & name = schema->field(i)->name(); if (getPort().getHeader().has(name, ignore_case) || nested_table_names.contains(ignore_case ? boost::to_lower_copy(name) : name)) { diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index 2f54cca466c..9172c79c890 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -128,8 +128,9 @@ void ParallelParsingInputFormat::onBackgroundException(size_t offset) background_exception = std::current_exception(); if (ParsingException * e = exception_cast(background_exception)) { + /// NOTE: it is not that safe to use line number hack here (may exceed INT_MAX) if (e->getLineNumber() != -1) - e->setLineNumber(e->getLineNumber() + offset); + e->setLineNumber(static_cast(e->getLineNumber() + offset)); auto file_name = getFileNameFromReadBuffer(getReadBuffer()); if (!file_name.empty()) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 427c159314b..dd2826287b2 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -161,7 +161,7 @@ void ParquetBlockInputFormat::prepareReader() /// STRUCT type require the number of indexes equal to the number of /// nested elements, so we should recursively /// count the number of indices we need for this type. - int indexes_count = countIndicesForType(schema->field(i)->type()); + int indexes_count = static_cast(countIndicesForType(schema->field(i)->type())); const auto & name = schema->field(i)->name(); if (getPort().getHeader().has(name, ignore_case) || nested_table_names.contains(ignore_case ? boost::to_lower_copy(name) : name)) diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index c26b6b39e0d..6eacfe621e1 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -50,7 +50,11 @@ bool RegexpFieldExtractor::parseRow(PeekableReadBuffer & buf) if (line_size > 0 && buf.position()[line_size - 1] == '\r') --line_to_match; - bool match = re2_st::RE2::FullMatchN(re2_st::StringPiece(buf.position(), line_to_match), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size()); + bool match = re2_st::RE2::FullMatchN( + re2_st::StringPiece(buf.position(), line_to_match), + regexp, + re2_arguments_ptrs.data(), + static_cast(re2_arguments_ptrs.size())); if (!match && !skip_unmatched) throw Exception("Line \"" + std::string(buf.position(), line_to_match) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA); diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index aff4557a4b7..72e89ea0013 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -350,7 +350,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx Expected expected; Tokens tokens(buf->position(), buf->buffer().end()); - IParser::Pos token_iterator(tokens, settings.max_parser_depth); + IParser::Pos token_iterator(tokens, static_cast(settings.max_parser_depth)); ASTPtr ast; bool parsed = parser.parse(token_iterator, ast, expected); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index b340073e73d..59f9668d3a8 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -853,7 +853,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, ContextPtr context, - unsigned num_streams, + size_t num_streams, std::shared_ptr max_block_numbers_to_read, const MergeTreeData & data, const Names & real_column_names, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 0a013748e91..15258eb6c40 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -140,7 +140,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, ContextPtr context, - unsigned num_streams, + size_t num_streams, std::shared_ptr max_block_numbers_to_read, const MergeTreeData & data, const Names & real_column_names, diff --git a/src/Processors/Sources/MySQLSource.cpp b/src/Processors/Sources/MySQLSource.cpp index e868182f49b..ecc80bef40b 100644 --- a/src/Processors/Sources/MySQLSource.cpp +++ b/src/Processors/Sources/MySQLSource.cpp @@ -141,7 +141,7 @@ namespace read_bytes_size += 2; break; case ValueType::vtUInt32: - assert_cast(column).insertValue(value.getUInt()); + assert_cast(column).insertValue(static_cast(value.getUInt())); read_bytes_size += 4; break; case ValueType::vtUInt64: @@ -171,7 +171,7 @@ namespace read_bytes_size += 2; break; case ValueType::vtInt32: - assert_cast(column).insertValue(value.getInt()); + assert_cast(column).insertValue(static_cast(value.getInt())); read_bytes_size += 4; break; case ValueType::vtInt64: @@ -236,7 +236,7 @@ namespace readDateTimeText(time, in, assert_cast(data_type).getTimeZone()); if (time < 0) time = 0; - assert_cast(column).insertValue(time); + assert_cast(column).insertValue(static_cast(time)); read_bytes_size += 4; break; } diff --git a/src/Processors/Sources/SQLiteSource.cpp b/src/Processors/Sources/SQLiteSource.cpp index d2c6f2ebb23..79c4be7f692 100644 --- a/src/Processors/Sources/SQLiteSource.cpp +++ b/src/Processors/Sources/SQLiteSource.cpp @@ -35,7 +35,11 @@ SQLiteSource::SQLiteSource( description.init(sample_block); sqlite3_stmt * compiled_stmt = nullptr; - int status = sqlite3_prepare_v2(sqlite_db.get(), query_str.c_str(), query_str.size() + 1, &compiled_stmt, nullptr); + int status = sqlite3_prepare_v2( + sqlite_db.get(), + query_str.c_str(), + static_cast(query_str.size() + 1), + &compiled_stmt, nullptr); if (status != SQLITE_OK) throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, @@ -109,7 +113,7 @@ Chunk SQLiteSource::generate() return Chunk(std::move(columns), num_rows); } -void SQLiteSource::insertValue(IColumn & column, ExternalResultDescription::ValueType type, size_t idx) +void SQLiteSource::insertValue(IColumn & column, ExternalResultDescription::ValueType type, int idx) { switch (type) { @@ -120,7 +124,7 @@ void SQLiteSource::insertValue(IColumn & column, ExternalResultDescription::Valu assert_cast(column).insertValue(sqlite3_column_int(compiled_statement.get(), idx)); break; case ValueType::vtUInt32: - assert_cast(column).insertValue(sqlite3_column_int64(compiled_statement.get(), idx)); + assert_cast(column).insertValue(static_cast(sqlite3_column_int64(compiled_statement.get(), idx))); break; case ValueType::vtUInt64: /// There is no uint64 in sqlite3, only int and int64 diff --git a/src/Processors/Sources/SQLiteSource.h b/src/Processors/Sources/SQLiteSource.h index d792483c70f..c1bae4d8a67 100644 --- a/src/Processors/Sources/SQLiteSource.h +++ b/src/Processors/Sources/SQLiteSource.h @@ -33,7 +33,7 @@ private: Chunk generate() override; - void insertValue(IColumn & column, ExternalResultDescription::ValueType type, size_t idx); + void insertValue(IColumn & column, ExternalResultDescription::ValueType type, int idx); String query_str; UInt64 max_block_size; diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 8598b0197fc..3f70abaea6d 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -77,7 +77,7 @@ static bool pollFd(int fd, size_t timeout_milliseconds, int events) while (true) { - res = poll(&pfd, 1, timeout_milliseconds); + res = poll(&pfd, 1, static_cast(timeout_milliseconds)); if (res < 0) { @@ -527,7 +527,7 @@ Pipe ShellCommandSourceCoordinator::createPipe( } else { - auto descriptor = i + 2; + int descriptor = static_cast(i) + 2; auto it = process->write_fds.find(descriptor); if (it == process->write_fds.end()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Process does not contain descriptor to write {}", descriptor); diff --git a/src/Processors/TTL/ITTLAlgorithm.cpp b/src/Processors/TTL/ITTLAlgorithm.cpp index 489941950b5..c71ad740719 100644 --- a/src/Processors/TTL/ITTLAlgorithm.cpp +++ b/src/Processors/TTL/ITTLAlgorithm.cpp @@ -48,13 +48,13 @@ ColumnPtr ITTLAlgorithm::executeExpressionAndGetColumn( UInt32 ITTLAlgorithm::getTimestampByIndex(const IColumn * column, size_t index) const { if (const ColumnUInt16 * column_date = typeid_cast(column)) - return date_lut.fromDayNum(DayNum(column_date->getData()[index])); + return static_cast(date_lut.fromDayNum(DayNum(column_date->getData()[index]))); else if (const ColumnUInt32 * column_date_time = typeid_cast(column)) return column_date_time->getData()[index]; else if (const ColumnConst * column_const = typeid_cast(column)) { if (typeid_cast(&column_const->getDataColumn())) - return date_lut.fromDayNum(DayNum(column_const->getValue())); + return static_cast(date_lut.fromDayNum(DayNum(column_const->getValue()))); else if (typeid_cast(&column_const->getDataColumn())) return column_const->getValue(); } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 16abb72cbd4..78ae6b8771f 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -40,7 +40,9 @@ static FillColumnDescription::StepFunction getStepFunction( { #define DECLARE_CASE(NAME) \ case IntervalKind::NAME: \ - return [step, scale, &date_lut](Field & field) { field = Add##NAME##sImpl::execute(static_cast(field.get()), step, date_lut, scale); }; + return [step, scale, &date_lut](Field & field) { \ + field = Add##NAME##sImpl::execute(static_cast(\ + field.get()), static_cast(step), date_lut, scale); }; FOR_EACH_INTERVAL_KIND(DECLARE_CASE) #undef DECLARE_CASE diff --git a/src/Processors/Transforms/MongoDBSource.cpp b/src/Processors/Transforms/MongoDBSource.cpp index b548e8c4184..9eef17cf40d 100644 --- a/src/Processors/Transforms/MongoDBSource.cpp +++ b/src/Processors/Transforms/MongoDBSource.cpp @@ -184,7 +184,7 @@ namespace break; case Poco::MongoDB::ElementTraits::TypeId: assert_cast &>(column).getData().push_back( - static_cast &>(value).value()); + static_cast(static_cast &>(value).value())); break; case Poco::MongoDB::ElementTraits::TypeId: assert_cast &>(column).getData().push_back(static_cast( @@ -282,7 +282,7 @@ namespace ErrorCodes::TYPE_MISMATCH}; assert_cast(column).getData().push_back( - static_cast &>(value).value().epochTime()); + static_cast(static_cast &>(value).value().epochTime())); break; } case ValueType::vtUUID: diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index d6c57f69376..83b0b202d74 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -62,20 +62,21 @@ static int compareValuesWithOffset(const IColumn * _compared_column, _compared_column); const auto * reference_column = assert_cast( _reference_column); + + using ValueType = typename ColumnType::ValueType; // Note that the storage type of offset returned by get<> is different, so // we need to specify the type explicitly. - const typename ColumnType::ValueType offset - = _offset.get(); + const ValueType offset = static_cast(_offset.get()); assert(offset >= 0); const auto compared_value_data = compared_column->getDataAt(compared_row); - assert(compared_value_data.size == sizeof(typename ColumnType::ValueType)); - auto compared_value = unalignedLoad( + assert(compared_value_data.size == sizeof(ValueType)); + auto compared_value = unalignedLoad( compared_value_data.data); const auto reference_value_data = reference_column->getDataAt(reference_row); - assert(reference_value_data.size == sizeof(typename ColumnType::ValueType)); - auto reference_value = unalignedLoad( + assert(reference_value_data.size == sizeof(ValueType)); + auto reference_value = unalignedLoad( reference_value_data.data); bool is_overflow; @@ -84,15 +85,6 @@ static int compareValuesWithOffset(const IColumn * _compared_column, else is_overflow = common::addOverflow(reference_value, offset, reference_value); -// fmt::print(stderr, -// "compared [{}] = {}, old ref {}, shifted ref [{}] = {}, offset {} preceding {} overflow {} to negative {}\n", -// compared_row, toString(compared_value), -// // fmt doesn't like char8_t. -// static_cast(unalignedLoad(reference_value_data.data)), -// reference_row, toString(reference_value), -// toString(offset), offset_is_preceding, -// is_overflow, offset_is_preceding); - if (is_overflow) { if (offset_is_preceding) diff --git a/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp b/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp index 43bb5fc7083..4596bbb8961 100644 --- a/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp +++ b/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp @@ -126,12 +126,12 @@ bool RemoteQueryExecutorReadContext::checkTimeoutImpl(bool blocking) epoll_event events[3]; events[0].data.fd = events[1].data.fd = events[2].data.fd = -1; - int num_events = epoll.getManyReady(3, events, blocking); + size_t num_events = epoll.getManyReady(3, events, blocking); bool is_socket_ready = false; bool is_pipe_alarmed = false; - for (int i = 0; i < num_events; ++i) + for (size_t i = 0; i < num_events; ++i) { if (events[i].data.fd == connection_fd) is_socket_ready = true; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index a2a2db75d68..a9373555af7 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1082,7 +1082,8 @@ namespace NamesAndTypesList columns; for (size_t column_idx : collections::range(external_table.columns_size())) { - const auto & name_and_type = external_table.columns(column_idx); + /// TODO: consider changing protocol + const auto & name_and_type = external_table.columns(static_cast(column_idx)); NameAndTypePair column; column.name = name_and_type.name(); if (column.name.empty()) diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index c8ae9c6e07c..c8015cfd185 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -136,7 +136,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl() WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( HTTPServerResponse & response_, bool is_http_method_head_, - unsigned keep_alive_timeout_, + size_t keep_alive_timeout_, bool compress_, CompressionMethod compression_method_) : BufferWithOwnMemory(DBMS_DEFAULT_BUFFER_SIZE) diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h index 6905d5df8b5..ce677616755 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h @@ -36,7 +36,7 @@ public: WriteBufferFromHTTPServerResponse( HTTPServerResponse & response_, bool is_http_method_head_, - unsigned keep_alive_timeout_, + size_t keep_alive_timeout_, bool compress_ = false, /// If true - set Content-Encoding header and compress the result. CompressionMethod compression_method_ = CompressionMethod::None); @@ -105,7 +105,7 @@ private: bool is_http_method_head; bool add_cors_header = false; - unsigned keep_alive_timeout = 0; + size_t keep_alive_timeout = 0; bool compress = false; CompressionMethod compression_method; int compression_level = 1; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index d02da92c613..2b63524fb79 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -622,8 +622,10 @@ void HTTPHandler::processQuery( /// Request body can be compressed using algorithm specified in the Content-Encoding header. String http_request_compression_method_str = request.get("Content-Encoding", ""); + int zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); auto in_post = wrapReadBufferWithCompressionMethod( - wrapReadBufferReference(request.getStream()), chooseCompressionMethod({}, http_request_compression_method_str), context->getSettingsRef().zstd_window_log_max); + wrapReadBufferReference(request.getStream()), + chooseCompressionMethod({}, http_request_compression_method_str), zstd_window_log_max); /// The data can also be compressed using incompatible internal algorithm. This is indicated by /// 'decompress' query parameter. @@ -749,7 +751,7 @@ void HTTPHandler::processQuery( /// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on. used_output.out->setCompression(client_supports_http_compression && settings.enable_http_compression); if (client_supports_http_compression) - used_output.out->setCompressionLevel(settings.http_zlib_compression_level); + used_output.out->setCompressionLevel(static_cast(settings.http_zlib_compression_level)); used_output.out->setSendProgress(settings.send_progress_in_http_headers); used_output.out->setSendProgressInterval(settings.http_headers_progress_interval_ms); diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 94e3597f88e..ea91dbc3450 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -126,7 +126,8 @@ struct SocketInterruptablePollWrapper do { Poco::Timestamp start; - rc = epoll_wait(epollfd, evout, 2, remaining_time.totalMilliseconds()); + /// TODO: use epoll_pwait() for more precise timers + rc = epoll_wait(epollfd, evout, 2, static_cast(remaining_time.totalMilliseconds())); if (rc < 0 && errno == EINTR) { Poco::Timestamp end; @@ -380,7 +381,7 @@ void KeeperTCPHandler::runImpl() response->zxid); UInt8 single_byte = 1; - [[maybe_unused]] int result = write(response_fd, &single_byte, sizeof(single_byte)); + [[maybe_unused]] ssize_t result = write(response_fd, &single_byte, sizeof(single_byte)); }; keeper_dispatcher->registerSession(session_id, response_callback); diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 8e701956d29..8e2d99e2909 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -63,8 +63,11 @@ static String showTableStatusReplacementQuery(const String & query); static String killConnectionIdReplacementQuery(const String & query); static String selectLimitReplacementQuery(const String & query); -MySQLHandler::MySQLHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, - bool ssl_enabled, size_t connection_id_) +MySQLHandler::MySQLHandler( + IServer & server_, + TCPServer & tcp_server_, + const Poco::Net::StreamSocket & socket_, + bool ssl_enabled, uint32_t connection_id_) : Poco::Net::TCPServerConnection(socket_) , server(server_) , tcp_server(tcp_server_) @@ -215,7 +218,7 @@ void MySQLHandler::finishHandshake(MySQLProtocol::ConnectionPhase::HandshakeResp auto read_bytes = [this, &buf, &pos, &packet_size](size_t count) -> void { while (pos < count) { - int ret = socket().receiveBytes(buf + pos, packet_size - pos); + int ret = socket().receiveBytes(buf + pos, static_cast(packet_size - pos)); if (ret == 0) { throw Exception("Cannot read all data. Bytes read: " + std::to_string(pos) + ". Bytes expected: 3", ErrorCodes::CANNOT_READ_ALL_DATA); @@ -376,7 +379,14 @@ void MySQLHandler::finishHandshakeSSL( } #if USE_SSL -MySQLHandlerSSL::MySQLHandlerSSL(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_, RSA & public_key_, RSA & private_key_) +MySQLHandlerSSL::MySQLHandlerSSL( + IServer & server_, + TCPServer & tcp_server_, + const Poco::Net::StreamSocket & socket_, + bool ssl_enabled, + uint32_t connection_id_, + RSA & public_key_, + RSA & private_key_) : MySQLHandler(server_, tcp_server_, socket_, ssl_enabled, connection_id_) , public_key(public_key_) , private_key(private_key_) diff --git a/src/Server/MySQLHandler.h b/src/Server/MySQLHandler.h index 2f43d471c40..3366e8792c9 100644 --- a/src/Server/MySQLHandler.h +++ b/src/Server/MySQLHandler.h @@ -31,7 +31,12 @@ class TCPServer; class MySQLHandler : public Poco::Net::TCPServerConnection { public: - MySQLHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_); + MySQLHandler( + IServer & server_, + TCPServer & tcp_server_, + const Poco::Net::StreamSocket & socket_, + bool ssl_enabled, + uint32_t connection_id_); void run() final; @@ -57,7 +62,7 @@ protected: IServer & server; TCPServer & tcp_server; Poco::Logger * log; - UInt64 connection_id = 0; + uint32_t connection_id = 0; uint32_t server_capabilities = 0; uint32_t client_capabilities = 0; @@ -81,7 +86,14 @@ protected: class MySQLHandlerSSL : public MySQLHandler { public: - MySQLHandlerSSL(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_, RSA & public_key_, RSA & private_key_); + MySQLHandlerSSL( + IServer & server_, + TCPServer & tcp_server_, + const Poco::Net::StreamSocket & socket_, + bool ssl_enabled, + uint32_t connection_id_, + RSA & public_key_, + RSA & private_key_); private: void authPluginSSL() override; diff --git a/src/Server/MySQLHandlerFactory.cpp b/src/Server/MySQLHandlerFactory.cpp index c02a3015945..cbcddbb444a 100644 --- a/src/Server/MySQLHandlerFactory.cpp +++ b/src/Server/MySQLHandlerFactory.cpp @@ -127,7 +127,7 @@ void MySQLHandlerFactory::generateRSAKeys() Poco::Net::TCPServerConnection * MySQLHandlerFactory::createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) { - size_t connection_id = last_connection_id++; + uint32_t connection_id = last_connection_id++; LOG_TRACE(log, "MySQL connection. Id: {}. Address: {}", connection_id, socket.peerAddress().toString()); #if USE_SSL return new MySQLHandlerSSL(server, tcp_server, socket, ssl_enabled, connection_id, *public_key, *private_key); diff --git a/src/Server/MySQLHandlerFactory.h b/src/Server/MySQLHandlerFactory.h index 38caae922ee..fa4ce93f765 100644 --- a/src/Server/MySQLHandlerFactory.h +++ b/src/Server/MySQLHandlerFactory.h @@ -36,7 +36,7 @@ private: bool ssl_enabled = false; #endif - std::atomic last_connection_id = 0; + std::atomic last_connection_id = 0; public: explicit MySQLHandlerFactory(IServer & server_); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 25a832ab7e3..2b8c49dafa8 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -841,7 +841,7 @@ void TCPHandler::processTablesStatusRequest() if (auto * replicated_table = dynamic_cast(table.get())) { status.is_replicated = true; - status.absolute_delay = replicated_table->getAbsoluteDelay(); + status.absolute_delay = static_cast(replicated_table->getAbsoluteDelay()); } else status.is_replicated = false; //-V1048 diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index c36ce1e9378..9c8d3ca60f3 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -159,7 +159,7 @@ private: UInt64 client_version_major = 0; UInt64 client_version_minor = 0; UInt64 client_version_patch = 0; - UInt64 client_tcp_protocol_version = 0; + UInt32 client_tcp_protocol_version = 0; String quota_key; /// Connection settings, which are extracted from a context. diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 9dc3d773e01..4b9667aa95d 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -106,7 +106,7 @@ namespace for (size_t replica_index = 1; replica_index <= replicas; ++replica_index) { - address.replica_index = replica_index; + address.replica_index = static_cast(replica_index); make_connection(address); } } diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 005e1e5fd1b..efcd70d6742 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -70,10 +70,10 @@ void DirectoryWatcherBase::watchFunc() while (!stopped) { const auto & settings = owner.storage.getFileLogSettings(); - if (poll(&pfd, 1, milliseconds_to_wait) > 0 && pfd.revents & POLLIN) + if (poll(&pfd, 1, static_cast(milliseconds_to_wait)) > 0 && pfd.revents & POLLIN) { milliseconds_to_wait = settings->poll_directory_watch_events_backoff_init.totalMilliseconds(); - int n = read(fd, buffer.data(), buffer.size()); + ssize_t n = read(fd, buffer.data(), buffer.size()); int i = 0; if (n > 0) { diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 7848b75deec..722843a7ab6 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -315,7 +315,7 @@ Pipe StorageFileLog::read( ContextPtr local_context, QueryProcessingStage::Enum /* processed_stage */, size_t /* max_block_size */, - unsigned /* num_streams */) + size_t /* num_streams */) { /// If there are MVs depended on this table, we just forbid reading if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 4295a8a764a..56f2d40ef5a 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -54,7 +54,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; void drop() override; diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h index dd77fc70358..3726d3aae96 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h @@ -52,7 +52,7 @@ private: std::future asyncReadInto(char * data, size_t size); IAsynchronousReader & reader; - Int32 priority; + size_t priority; std::shared_ptr impl; std::future prefetch_future; Memory<> prefetch_buffer; diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 4aebcd6f6ab..3f5c81dc01b 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -3,6 +3,7 @@ #if USE_HDFS #include #include +#include #include #include @@ -90,7 +91,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(num_bytes_to_read)); if (bytes_read < 0) throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to read from HDFS: {}, file path: {}. Error: {}", diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 138c92ea62d..bbabd523c45 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -214,8 +214,8 @@ ColumnsDescription StorageHDFS::getTableStructureFromData( return nullptr; auto compression = chooseCompressionMethod(*it, compression_method); auto impl = std::make_unique(uri_without_path, *it++, ctx->getGlobalContext()->getConfigRef(), ctx->getReadSettings()); - const auto zstd_window_log_max = ctx->getSettingsRef().zstd_window_log_max; - return wrapReadBufferWithCompressionMethod(std::move(impl), compression, zstd_window_log_max); + const Int64 zstd_window_log_max = ctx->getSettingsRef().zstd_window_log_max; + return wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); }; ColumnsDescription columns; @@ -356,8 +356,8 @@ bool HDFSSource::initialize() auto compression = chooseCompressionMethod(path_from_uri, storage->compression_method); auto impl = std::make_unique( uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); - const auto zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - read_buf = wrapReadBufferWithCompressionMethod(std::move(impl), compression, zstd_window_log_max); + const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; + read_buf = wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); auto input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size); @@ -550,7 +550,7 @@ Pipe StorageHDFS::read( ContextPtr context_, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { std::shared_ptr iterator_wrapper{nullptr}; if (distributed_processing) diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 90a42d0c692..b641f5bfb43 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -40,7 +40,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 467203c58f6..64fdf77dbc2 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -72,7 +72,7 @@ Pipe StorageHDFSCluster::read( ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t /*max_block_size*/, - unsigned /*num_streams*/) + size_t /*num_streams*/) { auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef()); diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h index 3239a1e4076..0d6f2bbe1b8 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ b/src/Storages/HDFS/StorageHDFSCluster.h @@ -32,7 +32,7 @@ public: std::string getName() const override { return "HDFSCluster"; } Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, - ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, unsigned /*num_streams*/) override; + ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, size_t /*num_streams*/) override; QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index a179f484652..1f952ec2bd9 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include namespace DB @@ -57,7 +58,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl int write(const char * start, size_t size) const { - int bytes_written = hdfsWrite(fs.get(), fout, start, size); + int bytes_written = hdfsWrite(fs.get(), fout, start, safe_cast(size)); if (write_settings.remote_throttler) write_settings.remote_throttler->add(bytes_written); diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index fc08c046f93..8f5b1b5f5fd 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -210,7 +210,7 @@ std::unique_ptr HiveORCFile::buildMinMaxIndex(c { size_t pos = it->second; /// Attention: column statistics start from 1. 0 has special purpose. - const orc::ColumnStatistics * col_stats = statistics->getColumnStatistics(pos + 1); + const orc::ColumnStatistics * col_stats = statistics->getColumnStatistics(static_cast(pos + 1)); idx->hyperrectangle[i] = buildRange(col_stats); } ++i; @@ -297,7 +297,7 @@ void HiveParquetFile::loadSplitMinMaxIndexesImpl() const auto * schema = meta->schema(); for (size_t pos = 0; pos < num_cols; ++pos) { - String column{schema->Column(pos)->name()}; + String column{schema->Column(static_cast(pos))->name()}; boost::to_lower(column); parquet_column_positions[column] = pos; } @@ -306,7 +306,7 @@ void HiveParquetFile::loadSplitMinMaxIndexesImpl() split_minmax_idxes.resize(num_row_groups); for (size_t i = 0; i < num_row_groups; ++i) { - auto row_group_meta = meta->RowGroup(i); + auto row_group_meta = meta->RowGroup(static_cast(i)); split_minmax_idxes[i] = std::make_shared(); split_minmax_idxes[i]->hyperrectangle.resize(num_cols); @@ -321,7 +321,7 @@ void HiveParquetFile::loadSplitMinMaxIndexesImpl() continue; size_t pos = mit->second; - auto col_chunk = row_group_meta->ColumnChunk(pos); + auto col_chunk = row_group_meta->ColumnChunk(static_cast(pos)); if (!col_chunk->is_stats_set()) continue; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 01ee5a8c3c5..5bbc71364c6 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -727,7 +727,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( hive_file->getPath(), hive_file->describeMinMaxIndex(sub_minmax_idxes[i])); - skip_splits.insert(i); + skip_splits.insert(static_cast(i)); } } hive_file->setSkipSplits(skip_splits); @@ -749,7 +749,7 @@ Pipe StorageHive::read( ContextPtr context_, QueryProcessingStage::Enum /* processed_stage */, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { lazyInitialize(); @@ -809,7 +809,7 @@ Pipe StorageHive::read( } if (num_streams > sources_info->hive_files.size()) - num_streams = sources_info->hive_files.size(); + num_streams = static_cast(sources_info->hive_files.size()); Pipes pipes; for (size_t i = 0; i < num_streams; ++i) @@ -829,7 +829,7 @@ Pipe StorageHive::read( } HiveFiles StorageHive::collectHiveFiles( - unsigned max_threads, + size_t max_threads, const SelectQueryInfo & query_info, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, @@ -937,7 +937,13 @@ StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & qu auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, getContext()->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - HiveFiles hive_files = collectHiveFiles(settings.max_threads, query_info, hive_table_metadata, fs, context_, prune_level); + HiveFiles hive_files = collectHiveFiles( + settings.max_threads, + query_info, + hive_table_metadata, + fs, + context_, + prune_level); UInt64 total_rows = 0; for (const auto & hive_file : hive_files) diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 9c02d228f97..363042621c7 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -60,7 +60,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) override; @@ -98,7 +98,7 @@ private: void initMinMaxIndexExpression(); HiveFiles collectHiveFiles( - unsigned max_threads, + size_t max_threads, const SelectQueryInfo & query_info, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 5b12b720f1c..7a704a17f4d 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -108,7 +108,7 @@ Pipe IStorage::watch( ContextPtr /*context*/, QueryProcessingStage::Enum & /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/) + size_t /*num_streams*/) { throw Exception("Method watch is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } @@ -120,7 +120,7 @@ Pipe IStorage::read( ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/) + size_t /*num_streams*/) { throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } @@ -133,7 +133,7 @@ void IStorage::read( ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { auto pipe = read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, context, getName()); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 242f17d6f20..fd48d22b12b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -323,7 +323,7 @@ public: ContextPtr /*context*/, QueryProcessingStage::Enum & /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/); + size_t /*num_streams*/); /// Returns true if FINAL modifier must be added to SELECT query depending on required columns. /// It's needed for ReplacingMergeTree wrappers such as MaterializedMySQL and MaterializedPostrgeSQL @@ -357,7 +357,7 @@ private: ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/); + size_t /*num_streams*/); public: /// Other version of read which adds reading step to query plan. @@ -370,7 +370,7 @@ public: ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/); + size_t /*num_streams*/); /** Writes the data to a table. * Receives a description of the query, which can contain information about the data write method. diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index fa52850fb39..8e4dd78379e 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -212,7 +212,7 @@ StorageKafka::StorageKafka( , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value)) , num_consumers(kafka_settings->kafka_num_consumers.value) , log(&Poco::Logger::get("StorageKafka (" + table_id_.table_name + ")")) - , semaphore(0, num_consumers) + , semaphore(0, static_cast(num_consumers)) , intermediate_commit(kafka_settings->kafka_commit_every_batch.value) , settings_adjustments(createSettingsAdjustments()) , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) @@ -291,7 +291,7 @@ Pipe StorageKafka::read( ContextPtr local_context, QueryProcessingStage::Enum /* processed_stage */, size_t /* max_block_size */, - unsigned /* num_streams */) + size_t /* num_streams */) { if (num_created_consumers == 0) return {}; diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 77bad6e17a9..c1c67b19c51 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -53,7 +53,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write( const ASTPtr & query, diff --git a/src/Storages/LiveView/StorageBlocks.h b/src/Storages/LiveView/StorageBlocks.h index bc860a1fa3c..a732ada1da2 100644 --- a/src/Storages/LiveView/StorageBlocks.h +++ b/src/Storages/LiveView/StorageBlocks.h @@ -46,7 +46,7 @@ public: ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/) override + size_t /*num_streams*/) override { return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index e3d19d0a433..3d27205d638 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -531,7 +531,7 @@ Pipe StorageLiveView::read( ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { std::lock_guard lock(mutex); @@ -556,7 +556,7 @@ Pipe StorageLiveView::watch( ContextPtr local_context, QueryProcessingStage::Enum & processed_stage, size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { ASTWatchQuery & query = typeid_cast(*query_info.query); diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index c6a0379e2ab..31b1c425709 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -143,7 +143,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; Pipe watch( const Names & column_names, @@ -151,7 +151,7 @@ public: ContextPtr context, QueryProcessingStage::Enum & processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; std::shared_ptr getBlocksPtr() { return blocks_ptr; } MergeableBlocksPtr getMergeableBlocks() { return mergeable_blocks; } diff --git a/src/Storages/MeiliSearch/SourceMeiliSearch.cpp b/src/Storages/MeiliSearch/SourceMeiliSearch.cpp index 8e37e469e96..b516ad8d0cf 100644 --- a/src/Storages/MeiliSearch/SourceMeiliSearch.cpp +++ b/src/Storages/MeiliSearch/SourceMeiliSearch.cpp @@ -174,7 +174,7 @@ size_t MeiliSearchSource::parseJSON(MutableColumns & columns, const JSON & jres) { ++cnt_fields; const auto & name = kv_pair.getName(); - int pos = description.sample_block.getPositionByName(name); + size_t pos = description.sample_block.getPositionByName(name); MutableColumnPtr & col = columns[pos]; DataTypePtr type_ptr = description.sample_block.getByPosition(pos).type; insertWithTypeId(col, kv_pair.getValue(), type_ptr); diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp index c5966d9e322..30d49edbb10 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp @@ -80,7 +80,7 @@ Pipe StorageMeiliSearch::read( ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned) + size_t /*num_streams*/) { storage_snapshot->check(column_names); diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.h b/src/Storages/MeiliSearch/StorageMeiliSearch.h index d7a2697730c..5fa7ac2c0e3 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.h +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.h @@ -25,7 +25,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 91ecb3a37a0..5b3497bf926 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -140,7 +140,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( const SelectQueryInfo & query_info, ContextPtr context, const UInt64 max_block_size, - const unsigned num_streams, + const size_t num_streams, QueryProcessingStage::Enum processed_stage, std::shared_ptr max_block_numbers_to_read, bool enable_parallel_reading) const @@ -930,7 +930,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd Strings forced_indices; { Tokens tokens(indices.data(), &indices[indices.size()], settings.max_query_size); - IParser::Pos pos(tokens, settings.max_parser_depth); + IParser::Pos pos(tokens, static_cast(settings.max_parser_depth)); Expected expected; if (!parseIdentifiersOrStringLiterals(pos, expected, forced_indices)) throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse force_data_skipping_indices ('{}')", indices); @@ -1279,7 +1279,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar const SelectQueryInfo & query_info, const ActionDAGNodes & added_filter_nodes, ContextPtr context, - unsigned num_streams, + size_t num_streams, std::shared_ptr max_block_numbers_to_read) const { size_t total_parts = parts.size(); @@ -1318,7 +1318,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( const SelectQueryInfo & query_info, ContextPtr context, const UInt64 max_block_size, - const unsigned num_streams, + const size_t num_streams, std::shared_ptr max_block_numbers_to_read, MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr, bool enable_parallel_reading) const diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index bb44f260eec..66dd7f7e5db 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -33,7 +33,7 @@ public: const SelectQueryInfo & query_info, ContextPtr context, UInt64 max_block_size, - unsigned num_streams, + size_t num_streams, QueryProcessingStage::Enum processed_stage, std::shared_ptr max_block_numbers_to_read = nullptr, bool enable_parallel_reading = false) const; @@ -46,7 +46,7 @@ public: const SelectQueryInfo & query_info, ContextPtr context, UInt64 max_block_size, - unsigned num_streams, + size_t num_streams, std::shared_ptr max_block_numbers_to_read = nullptr, MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr = nullptr, bool enable_parallel_reading = false) const; @@ -62,7 +62,7 @@ public: const SelectQueryInfo & query_info, const ActionDAGNodes & added_filter_nodes, ContextPtr context, - unsigned num_streams, + size_t num_streams, std::shared_ptr max_block_numbers_to_read = nullptr) const; private: diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 595e790ea3b..052834358bb 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -116,7 +116,7 @@ MergeTreeIndexAggregatorAnnoy::MergeTreeIndexAggregatorAnnoy( MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy::getGranuleAndReset() { // NOLINTNEXTLINE(*) - index->build(number_of_trees, /*number_of_threads=*/1); + index->build(static_cast(number_of_trees), /*number_of_threads=*/1); auto granule = std::make_shared(index_name, index_sample_block, index); index = nullptr; return granule; diff --git a/src/Storages/MergeTree/MergeType.cpp b/src/Storages/MergeTree/MergeType.cpp index 4b03f5ab57c..045114578d0 100644 --- a/src/Storages/MergeTree/MergeType.cpp +++ b/src/Storages/MergeTree/MergeType.cpp @@ -10,7 +10,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -MergeType checkAndGetMergeType(UInt64 merge_type) +MergeType checkAndGetMergeType(UInt32 merge_type) { if (auto maybe_merge_type = magic_enum::enum_cast(merge_type)) return *maybe_merge_type; diff --git a/src/Storages/MergeTree/MergeType.h b/src/Storages/MergeTree/MergeType.h index fad1ba33e3e..ce9a40c5931 100644 --- a/src/Storages/MergeTree/MergeType.h +++ b/src/Storages/MergeTree/MergeType.h @@ -22,7 +22,7 @@ enum class MergeType }; /// Check parsed merge_type from raw int and get enum value. -MergeType checkAndGetMergeType(UInt64 merge_type); +MergeType checkAndGetMergeType(UInt32 merge_type); /// Check this merge assigned with TTL bool isTTLMergeType(MergeType merge_type); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index e7882ce4952..d7e3c3b1955 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -240,7 +240,7 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) if (checkString("merge_type: ", in)) { - UInt64 value; + UInt32 value; in >> value; merge_type = checkAndGetMergeType(value); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp index ce33ac8c467..626295d7255 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp @@ -91,8 +91,8 @@ std::optional ReplicatedMergeTreeMergeStrategyPicker::pickReplicaToExecu void ReplicatedMergeTreeMergeStrategyPicker::refreshState() { const auto settings = storage.getSettings(); - auto threshold = settings->execute_merges_on_single_replica_time_threshold.totalSeconds(); - auto threshold_init = 0; + time_t threshold = settings->execute_merges_on_single_replica_time_threshold.totalSeconds(); + time_t threshold_init = 0; if (settings->allow_remote_fs_zero_copy_replication) threshold_init = settings->remote_fs_execute_merges_on_single_replica_time_threshold.totalSeconds(); @@ -127,7 +127,7 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() active_replicas_tmp.push_back(replica); if (replica == storage.replica_name) { - current_replica_index_tmp = active_replicas_tmp.size() - 1; + current_replica_index_tmp = static_cast(active_replicas_tmp.size() - 1); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 6ffcde161da..d6d937ce66f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1809,9 +1809,9 @@ ReplicatedMergeTreeQueue::Status ReplicatedMergeTreeQueue::getStatus() const Status res; - res.future_parts = future_parts.size(); - res.queue_size = queue.size(); - res.last_queue_update = last_queue_update; + res.future_parts = static_cast(future_parts.size()); + res.queue_size = static_cast(queue.size()); + res.last_queue_update = static_cast(last_queue_update); res.inserts_in_queue = 0; res.merges_in_queue = 0; @@ -1824,7 +1824,7 @@ ReplicatedMergeTreeQueue::Status ReplicatedMergeTreeQueue::getStatus() const for (const LogEntryPtr & entry : queue) { if (entry->create_time && (!res.queue_oldest_time || entry->create_time < res.queue_oldest_time)) - res.queue_oldest_time = entry->create_time; + res.queue_oldest_time = static_cast(entry->create_time); if (entry->type == LogEntry::GET_PART || entry->type == LogEntry::ATTACH_PART) { @@ -1832,7 +1832,7 @@ ReplicatedMergeTreeQueue::Status ReplicatedMergeTreeQueue::getStatus() const if (entry->create_time && (!res.inserts_oldest_time || entry->create_time < res.inserts_oldest_time)) { - res.inserts_oldest_time = entry->create_time; + res.inserts_oldest_time = static_cast(entry->create_time); res.oldest_part_to_get = entry->new_part_name; } } @@ -1843,7 +1843,7 @@ ReplicatedMergeTreeQueue::Status ReplicatedMergeTreeQueue::getStatus() const if (entry->create_time && (!res.merges_oldest_time || entry->create_time < res.merges_oldest_time)) { - res.merges_oldest_time = entry->create_time; + res.merges_oldest_time = static_cast(entry->create_time); res.oldest_part_to_merge_to = entry->new_part_name; } } @@ -1854,7 +1854,7 @@ ReplicatedMergeTreeQueue::Status ReplicatedMergeTreeQueue::getStatus() const if (entry->create_time && (!res.part_mutations_oldest_time || entry->create_time < res.part_mutations_oldest_time)) { - res.part_mutations_oldest_time = entry->create_time; + res.part_mutations_oldest_time = static_cast(entry->create_time); res.oldest_part_to_mutate_to = entry->new_part_name; } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 6d1a3efb01d..2ebdd604af2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -425,6 +425,7 @@ public: struct Status { + /// TODO: consider using UInt64 here UInt32 future_parts; UInt32 queue_size; UInt32 inserts_in_queue; diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index fd313a10bc8..a3d578cf5f2 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -65,7 +65,7 @@ public: ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned num_streams) override + size_t num_streams) override { query_plan = std::move(*MergeTreeDataSelectExecutor(storage) .readFromParts( diff --git a/src/Storages/MySQL/MySQLHelpers.cpp b/src/Storages/MySQL/MySQLHelpers.cpp index 94c07d2670f..127bdb96eaf 100644 --- a/src/Storages/MySQL/MySQLHelpers.cpp +++ b/src/Storages/MySQL/MySQLHelpers.cpp @@ -23,7 +23,7 @@ createMySQLPoolWithFailover(const StorageMySQLConfiguration & configuration, con return mysqlxx::PoolWithFailover( configuration.database, configuration.addresses, configuration.username, configuration.password, MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, - mysql_settings.connection_pool_size, + static_cast(mysql_settings.connection_pool_size), mysql_settings.connection_max_tries, mysql_settings.connection_wait_timeout, mysql_settings.connect_timeout, diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index d33138419e2..70b3599aa09 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -111,7 +111,7 @@ void NATSConnectionManager::connectImpl() { servers[i] = configuration.servers[i].c_str(); } - natsOptions_SetServers(options, servers, configuration.servers.size()); + natsOptions_SetServers(options, servers, static_cast(configuration.servers.size())); } natsOptions_SetMaxReconnect(options, configuration.max_reconnect); natsOptions_SetReconnectWait(options, configuration.reconnect_wait); diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 4a3ba973e67..dea2553700b 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -60,7 +60,7 @@ StorageNATS::StorageNATS( , schema_name(getContext()->getMacros()->expand(nats_settings->nats_schema)) , num_consumers(nats_settings->nats_num_consumers.value) , log(&Poco::Logger::get("StorageNATS (" + table_id_.table_name + ")")) - , semaphore(0, num_consumers) + , semaphore(0, static_cast(num_consumers)) , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) , is_attach(is_attach_) { @@ -289,7 +289,7 @@ void StorageNATS::read( ContextPtr local_context, QueryProcessingStage::Enum /* processed_stage */, size_t /* max_block_size */, - unsigned /* num_streams */) + size_t /* num_streams */) { if (!consumers_ready) throw Exception("NATS consumers setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_NATS); diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 185b39250c8..a5a050d566f 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -47,7 +47,7 @@ public: ContextPtr local_context, QueryProcessingStage::Enum /* processed_stage */, size_t /* max_block_size */, - unsigned /* num_streams */) override; + size_t /* num_streams */) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index cc80d567d1d..6d12960824a 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -279,7 +279,7 @@ void StorageMaterializedPostgreSQL::read( ContextPtr context_, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { auto nested_table = getNested(); diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index d8e9e98c662..af0adb10f9f 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -98,7 +98,7 @@ public: ContextPtr context_, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; /// This method is called only from MateriaizePostgreSQL database engine, because it needs to maintain /// an invariant: a table exists only if its nested table exists. This atomic variable is set to _true_ diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 70838daec24..57f5ddd86e6 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -87,7 +87,7 @@ StorageRabbitMQ::StorageRabbitMQ( , use_user_setup(rabbitmq_settings->rabbitmq_queue_consume.value) , hash_exchange(num_consumers > 1 || num_queues > 1) , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")")) - , semaphore(0, num_consumers) + , semaphore(0, static_cast(num_consumers)) , unique_strbase(getRandomName()) , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) , milliseconds_to_wait(RESCHEDULE_MS) @@ -674,7 +674,7 @@ void StorageRabbitMQ::read( ContextPtr local_context, QueryProcessingStage::Enum /* processed_stage */, size_t /* max_block_size */, - unsigned /* num_streams */) + size_t /* num_streams */) { if (!rabbit_is_ready) throw Exception("RabbitMQ setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_RABBITMQ); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 455b2fe8f09..a1250f50829 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -50,7 +50,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write( const ASTPtr & query, diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.cpp b/src/Storages/ReadFinalForExternalReplicaStorage.cpp index 3ec7a074fd4..28053c84e20 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.cpp +++ b/src/Storages/ReadFinalForExternalReplicaStorage.cpp @@ -35,7 +35,7 @@ void readFinalFromNestedStorage( ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned int num_streams) + size_t num_streams) { NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); auto lock = nested_storage->lockForShare(context->getCurrentQueryId(), context->getSettingsRef().lock_acquire_timeout); @@ -59,7 +59,8 @@ void readFinalFromNestedStorage( } auto nested_snapshot = nested_storage->getStorageSnapshot(nested_metadata, context); - nested_storage->read(query_plan, require_columns_name, nested_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + nested_storage->read( + query_plan, require_columns_name, nested_snapshot, query_info, context, processed_stage, max_block_size, num_streams); if (!query_plan.isInitialized()) { diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.h b/src/Storages/ReadFinalForExternalReplicaStorage.h index 178164b6643..f8d1264ccb3 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.h +++ b/src/Storages/ReadFinalForExternalReplicaStorage.h @@ -21,7 +21,7 @@ void readFinalFromNestedStorage( ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned int num_streams); + size_t num_streams); } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 20b1de51a30..46ddb650eee 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -403,7 +403,7 @@ Pipe StorageEmbeddedRocksDB::read( ContextPtr context_, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { storage_snapshot->check(column_names); @@ -467,7 +467,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) String rocksdb_dir; bool read_only{false}; if (!engine_args.empty()) - ttl = checkAndGetLiteralArgument(engine_args[0], "ttl"); + ttl = static_cast(checkAndGetLiteralArgument(engine_args[0], "ttl")); if (engine_args.size() > 1) rocksdb_dir = checkAndGetLiteralArgument(engine_args[1], "rocksdb_dir"); if (engine_args.size() > 2) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 03848510e66..ca0ab7a1840 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -46,7 +46,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index f6b397950ed..65b4dce3ad2 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -228,7 +228,7 @@ void StorageBuffer::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { const auto & metadata_snapshot = storage_snapshot->metadata; diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 580742c0c84..387165171b9 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -82,7 +82,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool supportsParallelInsert() const override { return true; } diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 2839ac03a5b..f9995cceda3 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -169,7 +169,7 @@ Pipe StorageDictionary::read( ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, const size_t max_block_size, - const unsigned threads) + const size_t threads) { auto registered_dictionary_name = location == Location::SameDatabaseAndNameAsDictionary ? getStorageID().getInternalDictionaryName() : dictionary_name; auto dictionary = getContext()->getExternalDictionariesLoader().getDictionary(registered_dictionary_name, local_context); diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index f81503910ca..6eadd1b2c21 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -69,7 +69,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned threads) override; + size_t threads) override; static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure); static String generateNamesAndTypesDescription(const NamesAndTypesList & list); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4eb6697dd6e..bced552915b 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -653,7 +653,7 @@ void StorageDistributed::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { const auto * select_query = query_info.query->as(); if (select_query->final() && local_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas) diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 7cb25ae46ab..334f44a90f9 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -112,7 +112,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t /*max_block_size*/, - unsigned /*num_streams*/) override; + size_t /*num_streams*/) override; bool supportsParallelInsert() const override { return true; } std::optional totalBytes(const Settings &) const override; diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 2931e62b7ef..cd3cc4d48ac 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -111,7 +111,7 @@ void StorageExecutable::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned /*threads*/) + size_t /*threads*/) { auto & script_name = settings.script_name; diff --git a/src/Storages/StorageExecutable.h b/src/Storages/StorageExecutable.h index 2638474082a..2393920fa3c 100644 --- a/src/Storages/StorageExecutable.h +++ b/src/Storages/StorageExecutable.h @@ -41,7 +41,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned threads) override; + size_t threads) override; private: ExecutableSettings settings; diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index dcb7a90b2f6..7d1eef1e47c 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -181,7 +181,7 @@ void StorageExternalDistributed::read( ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { std::vector> plans; for (const auto & shard : shards) diff --git a/src/Storages/StorageExternalDistributed.h b/src/Storages/StorageExternalDistributed.h index 52a2a7a4106..a1bdb41dded 100644 --- a/src/Storages/StorageExternalDistributed.h +++ b/src/Storages/StorageExternalDistributed.h @@ -55,7 +55,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; private: using Shards = std::unordered_set; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 29f2d0667d9..eb3ed3f8ef3 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -209,7 +209,7 @@ std::unique_ptr createReadBuffer( in.setProgressCallback(context); } - auto zstd_window_log_max = context->getSettingsRef().zstd_window_log_max; + int zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); return wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method, zstd_window_log_max); } @@ -645,7 +645,7 @@ Pipe StorageFile::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { if (use_table_fd) { diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index e60e5f6b371..03b3aacb67f 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -48,7 +48,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write( const ASTPtr & query, diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 9cac1e57297..c00e82598b2 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -494,7 +494,7 @@ Pipe StorageGenerateRandom::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { storage_snapshot->check(column_names); diff --git a/src/Storages/StorageGenerateRandom.h b/src/Storages/StorageGenerateRandom.h index 8dc3e490ae7..6b050c07e52 100644 --- a/src/Storages/StorageGenerateRandom.h +++ b/src/Storages/StorageGenerateRandom.h @@ -28,7 +28,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool supportsTransactions() const override { return true; } private: diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index 4729d0a5bf8..18e8442c1b5 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -57,7 +57,7 @@ Pipe StorageInput::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/) + size_t /*num_streams*/) { Pipes pipes; auto query_context = context->getQueryContext(); diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index 991a4f35b7b..da4669aaf37 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -25,7 +25,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; private: Pipe pipe; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 2e3e1d443ae..e4f786cd23b 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -585,7 +585,7 @@ Pipe StorageJoin::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned /*num_streams*/) + size_t /*num_streams*/) { storage_snapshot->check(column_names); diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 390af09422c..43515f800d9 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -68,7 +68,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; std::optional totalRows(const Settings & settings) const override; std::optional totalBytes(const Settings & settings) const override; diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index f0bf4e431ae..21be205c0f6 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -408,7 +408,7 @@ Pipe StorageKeeperMap::read( ContextPtr context_, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { checkTable(); storage_snapshot->check(column_names); diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 87861362e42..45b32434f15 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -39,7 +39,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index c6bc55fd620..9909489d901 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -782,7 +782,7 @@ Pipe StorageLog::read( ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { storage_snapshot->check(column_names); diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 2e677dd3161..a2b1356f240 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -53,7 +53,7 @@ public: ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; diff --git a/src/Storages/StorageMaterializedMySQL.cpp b/src/Storages/StorageMaterializedMySQL.cpp index bb69f211a9e..0dc0b1bff0b 100644 --- a/src/Storages/StorageMaterializedMySQL.cpp +++ b/src/Storages/StorageMaterializedMySQL.cpp @@ -40,7 +40,7 @@ void StorageMaterializedMySQL::read( ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned int num_streams) + size_t num_streams) { if (const auto * db = typeid_cast(database)) db->rethrowExceptionIfNeeded(); diff --git a/src/Storages/StorageMaterializedMySQL.h b/src/Storages/StorageMaterializedMySQL.h index a66b7eba804..cbb59e508e8 100644 --- a/src/Storages/StorageMaterializedMySQL.h +++ b/src/Storages/StorageMaterializedMySQL.h @@ -24,8 +24,13 @@ public: bool needRewriteQueryWithFinal(const Names & column_names) const override; void read( - QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & metadata_snapshot, SelectQueryInfo & query_info, - ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & metadata_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, size_t num_streams) override; SinkToStoragePtr write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr) override { throwNotAllowed(); } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index b01415f9590..e256e087728 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -150,7 +150,7 @@ void StorageMaterializedView::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, const size_t max_block_size, - const unsigned num_streams) + const size_t num_streams) { auto storage = getTargetTable(); auto lock = storage->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 1d8808b302e..af2dedf8164 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -91,7 +91,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; Strings getDataPaths() const override; diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index e4dbfe15095..957aae450c8 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -231,7 +231,7 @@ Pipe StorageMemory::read( ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned num_streams) + size_t num_streams) { storage_snapshot->check(column_names); diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 3889acb952b..c739088dbe4 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -51,7 +51,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool supportsParallelInsert() const override { return true; } bool supportsSubcolumns() const override { return true; } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 9891340a0d0..c9067148739 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -249,7 +249,7 @@ void StorageMerge::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, const size_t max_block_size, - unsigned num_streams) + size_t num_streams) { /** Just in case, turn off optimization "transfer to PREWHERE", * since there is no certainty that it works when one of table is MergeTree and other is not. diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 6bf68660803..33406321100 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -61,7 +61,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; void checkAlterIsPossible(const AlterCommands & commands, ContextPtr context) const override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7cfce882e7a..aac5372a83e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -220,7 +220,7 @@ void StorageMergeTree::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { /// If true, then we will ask initiator if we can read chosen ranges bool enable_parallel_reading = local_context->getClientInfo().collaborate_with_initiator; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index ea2527e44a7..745546b96f6 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -66,7 +66,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; std::optional totalRows(const Settings &) const override; std::optional totalRowsByPartitionPredicate(const SelectQueryInfo &, ContextPtr) const override; diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index dce45b2431a..3ae9c974770 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -150,7 +150,7 @@ Pipe StorageMongoDB::read( ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned) + size_t /*num_streams*/) { connectIfNotConnected(); diff --git a/src/Storages/StorageMongoDB.h b/src/Storages/StorageMongoDB.h index 0e00b80432b..04fb759133a 100644 --- a/src/Storages/StorageMongoDB.h +++ b/src/Storages/StorageMongoDB.h @@ -37,7 +37,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write( const ASTPtr & query, diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 4ccd31ab981..20eb59c7262 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -78,7 +78,7 @@ Pipe StorageMySQL::read( ContextPtr context_, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned) + size_t /*num_streams*/) { storage_snapshot->check(column_names_); String query = transformQueryForExternalDatabase( diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index e3c0712c179..bf9a24c9bfe 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -46,7 +46,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index 0011b5c94ad..2270731c0e3 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -35,8 +35,8 @@ public: SelectQueryInfo &, ContextPtr /*context*/, QueryProcessingStage::Enum /*processing_stage*/, - size_t, - unsigned) override + size_t /*max_block_size*/, + size_t /*num_streams*/) override { return Pipe( std::make_shared(storage_snapshot->getSampleBlockForColumns(column_names))); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index e0c6dbf5463..6cf4e458438 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -81,7 +81,7 @@ Pipe StoragePostgreSQL::read( ContextPtr context_, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size_, - unsigned) + size_t /*num_streams*/) { storage_snapshot->check(column_names_); diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 0755e33269e..97c62daa50f 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -38,7 +38,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 0fabff59db4..2afd9e8a63b 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -50,7 +50,7 @@ public: ContextPtr context, QueryProcessingStage::Enum & processed_stage, size_t max_block_size, - unsigned num_streams) override + size_t num_streams) override { return getNested()->watch(column_names, query_info, context, processed_stage, max_block_size, num_streams); } @@ -63,7 +63,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override + size_t num_streams) override { return getNested()->read(query_plan, column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ef2d91fabfc..a8863b53b54 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4369,7 +4369,7 @@ void StorageReplicatedMergeTree::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, const size_t max_block_size, - const unsigned num_streams) + const size_t num_streams) { /// If true, then we will ask initiator if we can read chosen ranges const bool enable_parallel_reading = local_context->getClientInfo().collaborate_with_initiator; @@ -5574,7 +5574,8 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) res.queue = queue.getStatus(); res.absolute_delay = getAbsoluteDelay(); /// NOTE: may be slightly inconsistent with queue status. - res.parts_to_check = part_check_thread.size(); + /// NOTE: consider convert to UInt64 + res.parts_to_check = static_cast(part_check_thread.size()); res.zookeeper_path = zookeeper_path; res.replica_name = replica_name; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e10ffcce22c..4cd2ee0a621 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -131,7 +131,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; std::optional totalRows(const Settings & settings) const override; std::optional totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context) const override; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 675dd548088..9a4e37cde1b 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -457,8 +457,9 @@ bool StorageS3Source::initialize() file_path = fs::path(bucket) / current_key; - auto zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - read_buf = wrapReadBufferWithCompressionMethod(createS3ReadBuffer(current_key), chooseCompressionMethod(current_key, compression_hint), zstd_window_log_max); + int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); + read_buf = wrapReadBufferWithCompressionMethod( + createS3ReadBuffer(current_key), chooseCompressionMethod(current_key, compression_hint), zstd_window_log_max); auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size, format_settings); QueryPipelineBuilder builder; @@ -875,7 +876,7 @@ Pipe StorageS3::read( ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { bool has_wildcards = s3_configuration.uri.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; @@ -1079,12 +1080,12 @@ void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( upd.auth_settings.region, ctx->getRemoteHostFilter(), - ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, + static_cast(ctx->getGlobalContext()->getSettingsRef().s3_max_redirects), ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, /* for_disk_s3 = */ false); client_configuration.endpointOverride = upd.uri.endpoint; - client_configuration.maxConnections = upd.rw_settings.max_connections; + client_configuration.maxConnections = static_cast(upd.rw_settings.max_connections); auto credentials = Aws::Auth::AWSCredentials(upd.auth_settings.access_key_id, upd.auth_settings.secret_access_key); auto headers = upd.auth_settings.headers; @@ -1254,7 +1255,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( } first = false; - const auto zstd_window_log_max = ctx->getSettingsRef().zstd_window_log_max; + int zstd_window_log_max = static_cast(ctx->getSettingsRef().zstd_window_log_max); return wrapReadBufferWithCompressionMethod( std::make_unique( s3_configuration.client, s3_configuration.uri.bucket, key, s3_configuration.uri.version_id, s3_configuration.rw_settings.max_single_read_retries, ctx->getReadSettings()), diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index c74a8501964..8e79516ba4c 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -171,7 +171,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index df927069bb0..0e4e51f7926 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -97,7 +97,7 @@ Pipe StorageS3Cluster::read( ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t /*max_block_size*/, - unsigned /*num_streams*/) + size_t /*num_streams*/) { StorageS3::updateS3Configuration(context, s3_configuration); diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index d2cf1b917a1..194c2ed0103 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -30,7 +30,7 @@ public: std::string getName() const override { return "S3Cluster"; } Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, - ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, unsigned /*num_streams*/) override; + ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, size_t /*num_streams*/) override; QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index a86ed7646b3..92f954ebb9d 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -57,7 +57,7 @@ Pipe StorageSQLite::read( ContextPtr context_, QueryProcessingStage::Enum, size_t max_block_size, - unsigned int) + size_t /*num_streams*/) { if (!sqlite_db) sqlite_db = openSQLiteDB(database_path, getContext(), /* throw_on_error */true); diff --git a/src/Storages/StorageSQLite.h b/src/Storages/StorageSQLite.h index b0f209b5bc3..a021c00f627 100644 --- a/src/Storages/StorageSQLite.h +++ b/src/Storages/StorageSQLite.h @@ -38,7 +38,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 0ecbdb0db10..92d53ffc1ac 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -349,7 +349,7 @@ Pipe StorageStripeLog::read( ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, - unsigned num_streams) + size_t num_streams) { storage_snapshot->check(column_names); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index efdf18c0f7b..3f1b4ed0ad5 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -47,7 +47,7 @@ public: ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 2a4bfdf304b..b105e50a54f 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -101,7 +101,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override + size_t num_streams) override { String cnames; for (const auto & c : column_names) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index f1c924a3448..c38b4313359 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -266,6 +266,7 @@ namespace setCredentials(credentials, request_uri); const auto settings = context->getSettings(); + int zstd_window_log_max = static_cast(settings.zstd_window_log_max); try { if (download_threads > 1) @@ -354,7 +355,7 @@ namespace threadPoolCallbackRunner(IOThreadPool::get(), "URLParallelRead"), download_threads), compression_method, - settings.zstd_window_log_max); + zstd_window_log_max); } } catch (const Poco::Exception & e) @@ -386,7 +387,7 @@ namespace /* use_external_buffer */ false, /* skip_url_not_found_error */ skip_url_not_found_error), compression_method, - settings.zstd_window_log_max); + zstd_window_log_max); } catch (...) { @@ -641,7 +642,7 @@ Pipe IStorageURLBase::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size); @@ -730,7 +731,7 @@ Pipe StorageURLWithFailover::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned /*num_streams*/) + size_t /*num_streams*/) { ColumnsDescription columns_description; Block block_for_format; diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 63c803f2d26..bf8858b8b66 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -35,7 +35,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; @@ -206,7 +206,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; struct Configuration { diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index 2a3e1743983..300b11b7346 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -27,7 +27,7 @@ Pipe StorageValues::read( ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/) + size_t /*num_streams*/) { storage_snapshot->check(column_names); diff --git a/src/Storages/StorageValues.h b/src/Storages/StorageValues.h index bf7bf0466e4..55222903797 100644 --- a/src/Storages/StorageValues.h +++ b/src/Storages/StorageValues.h @@ -23,7 +23,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; /// Why we may have virtual columns in the storage from a single block? /// Because it used as tmp storage for pushing blocks into views, and some diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index adaf1c4e404..a55d7ad3c09 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -111,7 +111,7 @@ void StorageView::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { ASTPtr current_inner_query = storage_snapshot->metadata->getSelectQuery().inner_query; diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index 31c96addd08..593ac820ad4 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -32,7 +32,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; static void replaceWithSubquery(ASTSelectQuery & select_query, ASTPtr & view_name, const StorageMetadataPtr & metadata_snapshot) { diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index aacbb5fa302..5f57d37278b 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -106,7 +106,7 @@ Pipe StorageXDBC::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { storage_snapshot->check(column_names); diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index a2bb9c15baf..aa313e024ca 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -26,7 +26,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; StorageXDBC( const StorageID & table_id_, diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 2cfe2de05db..63b9a443f95 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -45,7 +45,7 @@ public: ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/) override + size_t /*num_streams*/) override { storage_snapshot->check(column_names); diff --git a/src/Storages/System/StorageSystemBackups.cpp b/src/Storages/System/StorageSystemBackups.cpp index e7146711c4a..52a26fe0cd6 100644 --- a/src/Storages/System/StorageSystemBackups.cpp +++ b/src/Storages/System/StorageSystemBackups.cpp @@ -51,8 +51,8 @@ void StorageSystemBackups::fillData(MutableColumns & res_columns, ContextPtr con column_uncompressed_size.insertValue(info.uncompressed_size); column_compressed_size.insertValue(info.compressed_size); column_error.insertData(info.error_message.data(), info.error_message.size()); - column_start_time.insertValue(std::chrono::system_clock::to_time_t(info.start_time)); - column_end_time.insertValue(std::chrono::system_clock::to_time_t(info.end_time)); + column_start_time.insertValue(static_cast(std::chrono::system_clock::to_time_t(info.start_time))); + column_end_time.insertValue(static_cast(std::chrono::system_clock::to_time_t(info.end_time))); }; for (const auto & entry : context->getBackupsWorker().getAllInfos()) diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 20cab9fdc47..18e7d269795 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -296,7 +296,7 @@ Pipe StorageSystemColumns::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, const size_t max_block_size, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { storage_snapshot->check(column_names); diff --git a/src/Storages/System/StorageSystemColumns.h b/src/Storages/System/StorageSystemColumns.h index 542e4ce9661..7b4b5dd8fb3 100644 --- a/src/Storages/System/StorageSystemColumns.h +++ b/src/Storages/System/StorageSystemColumns.h @@ -24,7 +24,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool isSystemStorage() const override { return true; } }; diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index e725f8a03c6..be04261cc4e 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -171,7 +171,7 @@ Pipe StorageSystemDataSkippingIndices::read( ContextPtr context, QueryProcessingStage::Enum /* processed_stage */, size_t max_block_size, - unsigned int /* num_streams */) + size_t /* num_streams */) { storage_snapshot->check(column_names); diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.h b/src/Storages/System/StorageSystemDataSkippingIndices.h index 046855edd5e..8a1e8c159b4 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.h +++ b/src/Storages/System/StorageSystemDataSkippingIndices.h @@ -21,7 +21,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool isSystemStorage() const override { return true; } }; diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 574ce4f44c2..d094fefddcb 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -36,7 +36,7 @@ Pipe StorageSystemDetachedParts::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { StoragesInfoStream stream(query_info, context); diff --git a/src/Storages/System/StorageSystemDetachedParts.h b/src/Storages/System/StorageSystemDetachedParts.h index 23f27816138..20ac69f0eea 100644 --- a/src/Storages/System/StorageSystemDetachedParts.h +++ b/src/Storages/System/StorageSystemDetachedParts.h @@ -27,7 +27,7 @@ protected: ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/) override; + size_t /*num_streams*/) override; }; } diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 6b50b00dc30..86b5eafdf72 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -37,7 +37,7 @@ Pipe StorageSystemDisks::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { storage_snapshot->check(column_names); diff --git a/src/Storages/System/StorageSystemDisks.h b/src/Storages/System/StorageSystemDisks.h index cd1dc1a8bbf..06cc7e8d4e2 100644 --- a/src/Storages/System/StorageSystemDisks.h +++ b/src/Storages/System/StorageSystemDisks.h @@ -27,7 +27,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool isSystemStorage() const override { return true; } }; diff --git a/src/Storages/System/StorageSystemErrors.cpp b/src/Storages/System/StorageSystemErrors.cpp index 4c8c8e60d69..bbe67bc0d21 100644 --- a/src/Storages/System/StorageSystemErrors.cpp +++ b/src/Storages/System/StorageSystemErrors.cpp @@ -51,7 +51,7 @@ void StorageSystemErrors::fillData(MutableColumns & res_columns, ContextPtr cont for (size_t i = 0, end = ErrorCodes::end(); i < end; ++i) { const auto & error = ErrorCodes::values[i].get(); - std::string_view name = ErrorCodes::getName(i); + std::string_view name = ErrorCodes::getName(static_cast(i)); if (name.empty()) continue; diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index 523ec25b89c..a4823d7e5e1 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -129,7 +129,7 @@ Pipe StorageSystemNumbers::read( ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { storage_snapshot->check(column_names); diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index 6bb89c0525e..acddac681ef 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -38,7 +38,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool hasEvenlyDistributedRead() const override { return true; } bool isSystemStorage() const override { return true; } diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index f262c981b83..3091ffdb51a 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -27,7 +27,7 @@ Pipe StorageSystemOne::read( ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { storage_snapshot->check(column_names); diff --git a/src/Storages/System/StorageSystemOne.h b/src/Storages/System/StorageSystemOne.h index 35dba59a99e..d8a26f1def4 100644 --- a/src/Storages/System/StorageSystemOne.h +++ b/src/Storages/System/StorageSystemOne.h @@ -28,7 +28,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool isSystemStorage() const override { return true; } diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index bcfd670ece9..a0c022f5540 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -247,7 +247,7 @@ Pipe StorageSystemPartsBase::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { bool has_state_column = hasStateColumn(column_names, storage_snapshot); diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 8db96700e1a..cb6265d82df 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -63,7 +63,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; NamesAndTypesList getVirtuals() const override; diff --git a/src/Storages/System/StorageSystemQuotaLimits.cpp b/src/Storages/System/StorageSystemQuotaLimits.cpp index 0261d3d2cd9..6cc269130a0 100644 --- a/src/Storages/System/StorageSystemQuotaLimits.cpp +++ b/src/Storages/System/StorageSystemQuotaLimits.cpp @@ -90,7 +90,7 @@ void StorageSystemQuotaLimits::fillData(MutableColumns & res_columns, ContextPtr auto add_row = [&](const String & quota_name, const Quota::Limits & limits) { column_quota_name.insertData(quota_name.data(), quota_name.length()); - column_duration.push_back(limits.duration.count()); + column_duration.push_back(static_cast(limits.duration.count())); column_is_randomized_interval.push_back(limits.randomize_interval); for (auto quota_type : collections::range(QuotaType::MAX)) diff --git a/src/Storages/System/StorageSystemQuotaUsage.cpp b/src/Storages/System/StorageSystemQuotaUsage.cpp index 6ba47a86dbf..5d047dc0359 100644 --- a/src/Storages/System/StorageSystemQuotaUsage.cpp +++ b/src/Storages/System/StorageSystemQuotaUsage.cpp @@ -162,8 +162,8 @@ void StorageSystemQuotaUsage::fillDataImpl( time_t end_time = std::chrono::system_clock::to_time_t(interval->end_of_interval); UInt32 duration = static_cast(std::chrono::duration_cast(interval->duration).count()); time_t start_time = end_time - duration; - column_start_time.getData().push_back(start_time); - column_end_time.getData().push_back(end_time); + column_start_time.getData().push_back(static_cast(start_time)); + column_end_time.getData().push_back(static_cast(end_time)); column_duration.getData().push_back(duration); column_start_time_null_map.push_back(false); column_end_time_null_map.push_back(false); diff --git a/src/Storages/System/StorageSystemQuotas.cpp b/src/Storages/System/StorageSystemQuotas.cpp index 17863fa7326..439883e038a 100644 --- a/src/Storages/System/StorageSystemQuotas.cpp +++ b/src/Storages/System/StorageSystemQuotas.cpp @@ -96,7 +96,10 @@ void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr cont column_key_types_offsets.push_back(column_key_types.size()); for (const auto & limits : all_limits) - column_durations.push_back(std::chrono::duration_cast(limits.duration).count()); + { + column_durations.push_back( + static_cast(std::chrono::duration_cast(limits.duration).count())); + } column_durations_offsets.push_back(column_durations.size()); auto apply_to_ast = apply_to.toASTWithNames(access_control); diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index de7e1911e44..20076603522 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -38,7 +38,7 @@ Pipe StorageSystemRemoteDataPaths::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { storage_snapshot->check(column_names); diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.h b/src/Storages/System/StorageSystemRemoteDataPaths.h index f868ae60795..7e883d144ef 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.h +++ b/src/Storages/System/StorageSystemRemoteDataPaths.h @@ -21,7 +21,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; }; } diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index e018ccc0733..0f7877a6e41 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -66,7 +66,7 @@ Pipe StorageSystemReplicas::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { storage_snapshot->check(column_names); diff --git a/src/Storages/System/StorageSystemReplicas.h b/src/Storages/System/StorageSystemReplicas.h index fc7f8f15861..e9c29dec0fd 100644 --- a/src/Storages/System/StorageSystemReplicas.h +++ b/src/Storages/System/StorageSystemReplicas.h @@ -25,7 +25,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool isSystemStorage() const override { return true; } }; diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index 549ce193137..df3d8b74e6e 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -258,7 +258,7 @@ Pipe StorageSystemStackTrace::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { storage_snapshot->check(column_names); @@ -324,7 +324,7 @@ Pipe StorageSystemStackTrace::read( sigval sig_value{}; sig_value.sival_int = sequence_num.load(std::memory_order_acquire); - if (0 != ::sigqueue(tid, sig, sig_value)) + if (0 != ::sigqueue(static_cast(tid), sig, sig_value)) { /// The thread may has been already finished. if (ESRCH == errno) diff --git a/src/Storages/System/StorageSystemStackTrace.h b/src/Storages/System/StorageSystemStackTrace.h index dd613882e49..9133a86aa55 100644 --- a/src/Storages/System/StorageSystemStackTrace.h +++ b/src/Storages/System/StorageSystemStackTrace.h @@ -33,7 +33,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool isSystemStorage() const override { return true; } diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index 832c430e2be..b42bd7859dd 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -44,7 +44,7 @@ Pipe StorageSystemStoragePolicies::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { storage_snapshot->check(column_names); diff --git a/src/Storages/System/StorageSystemStoragePolicies.h b/src/Storages/System/StorageSystemStoragePolicies.h index 3340a4b5e62..afc729c8368 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.h +++ b/src/Storages/System/StorageSystemStoragePolicies.h @@ -27,7 +27,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool isSystemStorage() const override { return true; } }; diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index e36b22a979e..83f922850a3 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -574,7 +574,7 @@ Pipe StorageSystemTables::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, const size_t max_block_size, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { storage_snapshot->check(column_names); diff --git a/src/Storages/System/StorageSystemTables.h b/src/Storages/System/StorageSystemTables.h index 11ac75aab08..60b6144f122 100644 --- a/src/Storages/System/StorageSystemTables.h +++ b/src/Storages/System/StorageSystemTables.h @@ -25,7 +25,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool isSystemStorage() const override { return true; } }; diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index 9e5836fa358..6c2ddd8d3dd 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -97,7 +97,7 @@ Pipe StorageSystemZeros::read( ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { storage_snapshot->check(column_names); diff --git a/src/Storages/System/StorageSystemZeros.h b/src/Storages/System/StorageSystemZeros.h index 5461feacb6b..64443a3cfd6 100644 --- a/src/Storages/System/StorageSystemZeros.h +++ b/src/Storages/System/StorageSystemZeros.h @@ -29,7 +29,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; bool hasEvenlyDistributedRead() const override { return true; } bool isSystemStorage() const override { return true; } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index d34066de769..c0bc5ad8da9 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1018,7 +1018,8 @@ void StorageWindowView::threadFuncFireProc() return; std::lock_guard lock(fire_signal_mutex); - UInt32 timestamp_now = std::time(nullptr); + /// TODO: consider using time_t instead (for every timestamp in this class) + UInt32 timestamp_now = static_cast(std::time(nullptr)); while (next_fire_signal <= timestamp_now) { @@ -1078,7 +1079,7 @@ void StorageWindowView::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, const size_t max_block_size, - const unsigned num_streams) + const size_t num_streams) { if (target_table_id.empty()) return; @@ -1118,7 +1119,7 @@ Pipe StorageWindowView::watch( ContextPtr local_context, QueryProcessingStage::Enum & processed_stage, size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const size_t /*num_streams*/) { ASTWatchQuery & query = typeid_cast(*query_info.query); @@ -1189,7 +1190,7 @@ StorageWindowView::StorageWindowView( target_table_id = has_inner_target_table ? StorageID(table_id_.database_name, generateTargetTableName(table_id_)) : query.to_table_id; if (is_proctime) - next_fire_signal = getWindowUpperBound(std::time(nullptr)); + next_fire_signal = getWindowUpperBound(static_cast(std::time(nullptr))); std::exchange(has_inner_table, true); if (!attach_) diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 96c034b9590..6da34389e4d 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -150,7 +150,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; Pipe watch( const Names & column_names, @@ -158,7 +158,7 @@ public: ContextPtr context, QueryProcessingStage::Enum & processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; std::pair getNewBlocks(UInt32 watermark); diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index fd474f037b3..4ecf29a05bd 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -43,7 +43,8 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr } else if (type == Field::Types::Int64 || type == Field::Types::UInt64) { - fd = (type == Field::Types::Int64) ? literal->value.get() : literal->value.get(); + fd = static_cast( + (type == Field::Types::Int64) ? literal->value.get() : literal->value.get()); if (fd < 0) throw Exception("File descriptor must be non-negative", ErrorCodes::BAD_ARGUMENTS); } diff --git a/utils/compressor/decompress_perf.cpp b/utils/compressor/decompress_perf.cpp index e3210164d79..891a6d3d1dd 100644 --- a/utils/compressor/decompress_perf.cpp +++ b/utils/compressor/decompress_perf.cpp @@ -107,8 +107,12 @@ protected: if (variant == LZ4_REFERENCE) { - if (LZ4_decompress_fast(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, to, size_decompressed) < 0) + if (LZ4_decompress_fast( + compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, to, + static_cast(size_decompressed)) < 0) + { throw Exception("Cannot LZ4_decompress_fast", ErrorCodes::CANNOT_DECOMPRESS); + } } else LZ4::decompress(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, to, size_compressed_without_checksum, size_decompressed, perf_stat);