From b24ca8de522598640c8ad4aafa3fe568af2ea4dc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Apr 2022 10:18:31 +0200 Subject: [PATCH] Fix various clang-tidy warnings When I tried to add cool new clang-tidy 14 warnings, I noticed that the current clang-tidy settings already produce a ton of warnings. This commit addresses many of these. Almost all of them were non-critical, i.e. C vs. C++ style casts. --- base/base/LineReader.cpp | 2 +- base/daemon/BaseDaemon.cpp | 5 ++- base/loggers/OwnSplitChannel.cpp | 2 +- programs/install/Install.cpp | 4 +- programs/obfuscator/Obfuscator.cpp | 4 +- .../AggregateFunctionIf.cpp | 4 +- .../AggregateFunctionMLMethod.cpp | 4 +- src/Columns/ColumnAggregateFunction.cpp | 2 +- src/Columns/ColumnLowCardinality.cpp | 4 +- src/Columns/ColumnVector.cpp | 2 +- src/Common/FieldVisitorWriteBinary.cpp | 2 +- src/Common/ShellCommand.cpp | 24 +++++------ src/Common/ThreadPool.cpp | 2 +- src/Common/TraceSender.cpp | 2 +- src/Common/ZooKeeper/TestKeeper.cpp | 2 +- src/Common/ZooKeeper/ZooKeeper.cpp | 8 ++-- src/Common/ZooKeeper/ZooKeeperIO.cpp | 2 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 6 +-- src/Common/formatIPv6.cpp | 2 +- src/Compression/CompressionCodecDelta.cpp | 4 +- .../CompressionCodecDoubleDelta.cpp | 4 +- src/Compression/CompressionCodecEncrypted.cpp | 4 +- src/Compression/CompressionCodecGorilla.cpp | 4 +- src/Compression/CompressionCodecT64.cpp | 30 +++++++------- src/Compression/CompressionCodecZSTD.cpp | 2 +- src/Compression/LZ4_decompress_faster.cpp | 12 +++--- src/Coordination/CoordinationSettings.cpp | 18 ++++----- src/Core/MySQL/MySQLReplication.cpp | 18 ++++----- src/Dictionaries/IPAddressDictionary.cpp | 6 +-- src/Dictionaries/MongoDBDictionarySource.cpp | 4 +- src/Functions/FunctionsBitToArray.cpp | 2 +- src/Functions/FunctionsCodingIP.cpp | 40 +++++++++---------- src/Functions/FunctionsLogical.cpp | 2 +- src/Functions/FunctionsStringSimilarity.cpp | 14 +++---- .../FunctionsTransactionCounters.cpp | 2 +- src/Functions/GeoHash.cpp | 2 +- src/Functions/URL/port.cpp | 2 +- src/Functions/appendTrailingCharIfAbsent.cpp | 2 +- src/Functions/array/arrayAUC.cpp | 2 +- src/Functions/array/arrayAggregation.cpp | 2 +- src/Functions/array/arrayElement.cpp | 6 +-- src/Functions/array/arrayReverse.cpp | 2 +- src/Functions/bitShiftLeft.cpp | 4 +- src/Functions/bitShiftRight.cpp | 2 +- src/Functions/dateDiff.cpp | 4 +- src/Functions/hasColumnInTable.cpp | 2 +- src/Functions/jumpConsistentHash.cpp | 2 +- src/Functions/neighbor.cpp | 10 ++--- src/Functions/sigmoid.cpp | 2 +- src/IO/HadoopSnappyReadBuffer.cpp | 2 +- src/IO/PeekableReadBuffer.cpp | 2 +- src/IO/ReadBufferFromMemory.cpp | 8 ++-- src/IO/ReadBufferFromS3.cpp | 2 +- src/IO/WriteBufferFromS3.cpp | 4 +- .../ArithmeticOperationsInAgrFuncOptimize.cpp | 14 +++---- src/Interpreters/ClientInfo.cpp | 8 ++-- src/Interpreters/CrashLog.cpp | 2 +- src/Interpreters/DNSCacheUpdater.cpp | 2 +- src/Interpreters/InterpreterCheckQuery.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- .../InterpreterSelectWithUnionQuery.cpp | 8 ++-- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- .../MySQL/InterpretersMySQLDDLQuery.cpp | 4 +- src/Interpreters/ProfileEventsExt.cpp | 6 +-- src/Interpreters/QueryLog.cpp | 4 +- src/Interpreters/ThreadStatusExt.cpp | 2 +- src/Interpreters/TraceCollector.cpp | 6 +-- src/Interpreters/TreeOptimizer.cpp | 2 +- src/Interpreters/TreeRewriter.cpp | 4 +- src/Interpreters/executeQuery.cpp | 4 +- src/Parsers/MySQL/ASTDeclareOption.cpp | 4 +- src/Parsers/ParserSelectQuery.cpp | 2 +- src/Parsers/ParserSetQuery.cpp | 4 +- src/Parsers/makeASTForLogicalFunction.cpp | 8 ++-- .../Impl/TabSeparatedRowInputFormat.cpp | 2 +- src/Processors/Transforms/CubeTransform.cpp | 2 +- .../Transforms/PostgreSQLSource.cpp | 2 +- src/QueryPipeline/ExecutionSpeedLimits.cpp | 2 +- src/Storages/Distributed/DirectoryMonitor.cpp | 2 +- .../MergeTreeBaseSelectProcessor.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- .../MergeTreeDataPartWriterCompact.cpp | 6 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MaterializedPostgreSQLConsumer.cpp | 10 ++--- .../StorageMaterializedPostgreSQL.cpp | 2 +- .../ReadFinalForExternalReplicaStorage.cpp | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageMerge.cpp | 4 +- 88 files changed, 223 insertions(+), 222 deletions(-) diff --git a/base/base/LineReader.cpp b/base/base/LineReader.cpp index f4e741a54e7..4ac6215350f 100644 --- a/base/base/LineReader.cpp +++ b/base/base/LineReader.cpp @@ -73,7 +73,7 @@ replxx::Replxx::completions_t LineReader::Suggest::getCompletions(const String & if (std::string::npos == last_word_pos) last_word = prefix; else - last_word = std::string_view(prefix).substr(last_word_pos + 1, std::string::npos); + last_word = std::string_view{prefix}.substr(last_word_pos + 1, std::string::npos); /// last_word can be empty. std::pair range; diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index b27a904b31a..b0b90923c05 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -61,6 +61,7 @@ #if defined(OS_DARWIN) # pragma GCC diagnostic ignored "-Wunused-macros" +// NOLINTNEXTLINE(bugprone-reserved-identifier) # define _XOPEN_SOURCE 700 // ucontext is not available without _XOPEN_SOURCE #endif #include @@ -132,7 +133,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context) DB::writePODBinary(*info, out); DB::writePODBinary(signal_context, out); DB::writePODBinary(stack_trace, out); - DB::writeBinary(UInt32(getThreadId()), out); + DB::writeBinary(static_cast(getThreadId()), out); DB::writePODBinary(DB::current_thread, out); out.next(); @@ -435,7 +436,7 @@ static void sanitizerDeathCallback() DB::WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], buf_size, buf); DB::writeBinary(static_cast(SignalListener::StdTerminate), out); - DB::writeBinary(UInt32(getThreadId()), out); + DB::writeBinary(static_cast(getThreadId()), out); DB::writeBinary(log_message, out); out.next(); diff --git a/base/loggers/OwnSplitChannel.cpp b/base/loggers/OwnSplitChannel.cpp index b255d89f124..71be8007d85 100644 --- a/base/loggers/OwnSplitChannel.cpp +++ b/base/loggers/OwnSplitChannel.cpp @@ -103,7 +103,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) columns[i++]->insert(DNSResolver::instance().getHostName()); columns[i++]->insert(msg_ext.query_id); columns[i++]->insert(msg_ext.thread_id); - columns[i++]->insert(Int64(msg.getPriority())); + columns[i++]->insert(static_cast(msg.getPriority())); columns[i++]->insert(msg.getSource()); columns[i++]->insert(msg.getText()); diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 5dec09ea901..fe52e3a825e 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -239,12 +239,12 @@ int mainEntryClickHouseInstall(int argc, char ** argv) uint32_t path_length = 0; _NSGetExecutablePath(nullptr, &path_length); if (path_length <= 1) - Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot obtain path to the binary"); + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot obtain path to the binary"); std::string path(path_length, std::string::value_type()); auto res = _NSGetExecutablePath(&path[0], &path_length); if (res != 0) - Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot obtain path to the binary"); + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot obtain path to the binary"); if (path.back() == '\0') path.pop_back(); diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 1ffb0b437a6..a194718db58 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -549,7 +549,7 @@ private: CodePoint sample(UInt64 random, double end_multiplier) const { - UInt64 range = total + UInt64(count_end * end_multiplier); + UInt64 range = total + static_cast(count_end * end_multiplier); if (range == 0) return END; @@ -728,7 +728,7 @@ public: if (!histogram.total) continue; - double average = double(histogram.total) / histogram.buckets.size(); + double average = static_cast(histogram.total) / histogram.buckets.size(); UInt64 new_total = 0; for (auto & bucket : histogram.buckets) diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index dbf627ac6b5..84216ece975 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -225,7 +225,7 @@ public: throw Exception("Logical error: single argument is passed to AggregateFunctionIfNullVariadic", ErrorCodes::LOGICAL_ERROR); if (number_of_arguments > MAX_ARGS) - throw Exception("Maximum number of arguments for aggregate function with Nullable types is " + toString(size_t(MAX_ARGS)), + throw Exception("Maximum number of arguments for aggregate function with Nullable types is " + toString(MAX_ARGS), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (size_t i = 0; i < number_of_arguments; ++i) @@ -359,7 +359,7 @@ private: using Base = AggregateFunctionNullBase>; - enum { MAX_ARGS = 8 }; + static constexpr size_t MAX_ARGS = 8; size_t number_of_arguments = 0; std::array is_nullable; /// Plain array is better than std::vector due to one indirection less. }; diff --git a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp index 6f81461044a..851323e92b1 100644 --- a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp +++ b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp @@ -56,8 +56,8 @@ namespace /// Such default parameters were picked because they did good on some tests, /// though it still requires to fit parameters to achieve better result - auto learning_rate = Float64(1.0); - auto l2_reg_coef = Float64(0.5); + auto learning_rate = static_cast(1.0); + auto l2_reg_coef = static_cast(0.5); UInt64 batch_size = 15; std::string weights_updater_name = "Adam"; diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 59d56c6e437..55f851cb7c6 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -607,7 +607,7 @@ MutableColumns ColumnAggregateFunction::scatter(IColumn::ColumnIndex num_columns size_t num_rows = size(); { - size_t reserve_size = double(num_rows) / num_columns * 1.1; /// 1.1 is just a guess. Better to use n-sigma rule. + size_t reserve_size = static_cast(num_rows) / num_columns * 1.1; /// 1.1 is just a guess. Better to use n-sigma rule. if (reserve_size > 1) for (auto & column : columns) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 4d4488beaa1..62fb69a47e1 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -81,7 +81,7 @@ namespace if (max_val > size) return mapUniqueIndexImplRef(index); - auto map_size = UInt64(max_val) + 1; + auto map_size = static_cast(max_val) + 1; PaddedPODArray map(map_size, 0); T zero_pos_value = index[0]; index[0] = 0; @@ -98,7 +98,7 @@ namespace index[i] = map[val]; } - auto res_col = ColumnVector::create(UInt64(cur_pos) + 1); + auto res_col = ColumnVector::create(static_cast(cur_pos) + 1); auto & data = res_col->getData(); data[0] = zero_pos_value; for (size_t i = 0; i < map_size; ++i) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index dded5ff6c99..90b821bfe06 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -228,7 +228,7 @@ void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction if (s >= 256 && s <= std::numeric_limits::max() && use_radix_sort) { PaddedPODArray> pairs(s); - for (UInt32 i = 0; i < UInt32(s); ++i) + for (UInt32 i = 0; i < static_cast(s); ++i) pairs[i] = {data[i], i}; RadixSort>::executeLSD(pairs.data(), s, reverse, res.data()); diff --git a/src/Common/FieldVisitorWriteBinary.cpp b/src/Common/FieldVisitorWriteBinary.cpp index edabd26fd3a..85c32cee3c0 100644 --- a/src/Common/FieldVisitorWriteBinary.cpp +++ b/src/Common/FieldVisitorWriteBinary.cpp @@ -82,7 +82,7 @@ void FieldVisitorWriteBinary::operator() (const Object & x, WriteBuffer & buf) c void FieldVisitorWriteBinary::operator()(const bool & x, WriteBuffer & buf) const { - writeBinary(UInt8(x), buf); + writeBinary(static_cast(x), buf); } } diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index f24add7acf0..9f41f100bac 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -207,15 +207,15 @@ std::unique_ptr ShellCommand::executeImpl( /// Replace the file descriptors with the ends of our pipes. if (STDIN_FILENO != dup2(pipe_stdin.fds_rw[0], STDIN_FILENO)) - _exit(int(ReturnCodes::CANNOT_DUP_STDIN)); + _exit(static_cast(ReturnCodes::CANNOT_DUP_STDIN)); if (!config.pipe_stdin_only) { if (STDOUT_FILENO != dup2(pipe_stdout.fds_rw[1], STDOUT_FILENO)) - _exit(int(ReturnCodes::CANNOT_DUP_STDOUT)); + _exit(static_cast(ReturnCodes::CANNOT_DUP_STDOUT)); if (STDERR_FILENO != dup2(pipe_stderr.fds_rw[1], STDERR_FILENO)) - _exit(int(ReturnCodes::CANNOT_DUP_STDERR)); + _exit(static_cast(ReturnCodes::CANNOT_DUP_STDERR)); } for (size_t i = 0; i < config.read_fds.size(); ++i) @@ -224,7 +224,7 @@ std::unique_ptr ShellCommand::executeImpl( auto fd = config.read_fds[i]; if (fd != dup2(fds.fds_rw[1], fd)) - _exit(int(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR)); + _exit(static_cast(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR)); } for (size_t i = 0; i < config.write_fds.size(); ++i) @@ -233,7 +233,7 @@ std::unique_ptr ShellCommand::executeImpl( auto fd = config.write_fds[i]; if (fd != dup2(fds.fds_rw[0], fd)) - _exit(int(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR)); + _exit(static_cast(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR)); } // Reset the signal mask: it may be non-empty and will be inherited @@ -246,7 +246,7 @@ std::unique_ptr ShellCommand::executeImpl( execv(filename, argv); /// If the process is running, then `execv` does not return here. - _exit(int(ReturnCodes::CANNOT_EXEC)); + _exit(static_cast(ReturnCodes::CANNOT_EXEC)); } std::unique_ptr res(new ShellCommand( @@ -356,17 +356,17 @@ void ShellCommand::wait() { switch (retcode) { - case int(ReturnCodes::CANNOT_DUP_STDIN): + case static_cast(ReturnCodes::CANNOT_DUP_STDIN): throw Exception("Cannot dup2 stdin of child process", ErrorCodes::CANNOT_CREATE_CHILD_PROCESS); - case int(ReturnCodes::CANNOT_DUP_STDOUT): + case static_cast(ReturnCodes::CANNOT_DUP_STDOUT): throw Exception("Cannot dup2 stdout of child process", ErrorCodes::CANNOT_CREATE_CHILD_PROCESS); - case int(ReturnCodes::CANNOT_DUP_STDERR): + case static_cast(ReturnCodes::CANNOT_DUP_STDERR): throw Exception("Cannot dup2 stderr of child process", ErrorCodes::CANNOT_CREATE_CHILD_PROCESS); - case int(ReturnCodes::CANNOT_EXEC): + case static_cast(ReturnCodes::CANNOT_EXEC): throw Exception("Cannot execv in child process", ErrorCodes::CANNOT_CREATE_CHILD_PROCESS); - case int(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR): + case static_cast(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR): throw Exception("Cannot dup2 read descriptor of child process", ErrorCodes::CANNOT_CREATE_CHILD_PROCESS); - case int(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR): + case static_cast(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR): throw Exception("Cannot dup2 write descriptor of child process", ErrorCodes::CANNOT_CREATE_CHILD_PROCESS); default: throw Exception("Child process was exited with return code " + toString(retcode), ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY); diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index c5f806c8a89..a76037ae5cf 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -154,7 +154,7 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::opti new_job_or_shutdown.notify_one(); } - return ReturnType(true); + return static_cast(true); } template diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index 57ab3df8f96..6f0904bd50f 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -64,7 +64,7 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int size_t stack_trace_size = stack_trace.getSize(); size_t stack_trace_offset = stack_trace.getOffset(); - writeIntBinary(UInt8(stack_trace_size - stack_trace_offset), out); + writeIntBinary(static_cast(stack_trace_size - stack_trace_offset), out); for (size_t i = stack_trace_offset; i < stack_trace_size; ++i) writePODBinary(stack_trace.getFramePointers()[i], out); diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 065b1cf65ba..4756ebdf291 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -514,7 +514,7 @@ void TestKeeper::processingThread() { RequestInfo info; - UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds()); + UInt64 max_wait = static_cast(operation_timeout.totalMilliseconds()); if (requests_queue.tryPop(info, max_wait)) { if (expired) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 0f4b141d058..bef9f883138 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -76,7 +76,7 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ auto & host_string = host.host; try { - bool secure = bool(startsWith(host_string, "secure://")); + bool secure = startsWith(host_string, "secure://"); if (secure) host_string.erase(0, strlen("secure://")); @@ -801,7 +801,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & auto callback = [state](const Coordination::GetResponse & response) { - state->code = int32_t(response.error); + state->code = static_cast(response.error); if (state->code) state->event.set(); }; @@ -810,7 +810,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & { if (!state->code) { - state->code = int32_t(response.error); + state->code = static_cast(response.error); if (!state->code) state->event_type = response.type; state->event.set(); @@ -828,7 +828,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & if (!state->event.tryWait(1000)) continue; - if (state->code == int32_t(Coordination::Error::ZNONODE)) + if (state->code == static_cast(Coordination::Error::ZNONODE)) return true; if (state->code) diff --git a/src/Common/ZooKeeper/ZooKeeperIO.cpp b/src/Common/ZooKeeper/ZooKeeperIO.cpp index 0e0a034c633..12ef9f17b3d 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.cpp +++ b/src/Common/ZooKeeper/ZooKeeperIO.cpp @@ -40,7 +40,7 @@ void write(bool x, WriteBuffer & out) void write(const std::string & s, WriteBuffer & out) { - write(int32_t(s.size()), out); + write(static_cast(s.size()), out); out.write(s.data(), s.size()); } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index b8a6e040bc8..57317e16206 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -539,7 +539,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) Error::ZMARSHALLINGERROR); if (err != Error::ZOK) - throw Exception("Error received in reply to auth request. Code: " + DB::toString(int32_t(err)) + ". Message: " + String(errorMessage(err)), + throw Exception("Error received in reply to auth request. Code: " + DB::toString(static_cast(err)) + ". Message: " + String(errorMessage(err)), Error::ZMARSHALLINGERROR); } @@ -563,8 +563,8 @@ void ZooKeeper::sendThread() { /// Wait for the next request in queue. No more than operation timeout. No more than until next heartbeat time. UInt64 max_wait = std::min( - UInt64(std::chrono::duration_cast(next_heartbeat_time - now).count()), - UInt64(operation_timeout.totalMilliseconds())); + static_cast(std::chrono::duration_cast(next_heartbeat_time - now).count()), + static_cast(operation_timeout.totalMilliseconds())); RequestInfo info; if (requests_queue.tryPop(info, max_wait)) diff --git a/src/Common/formatIPv6.cpp b/src/Common/formatIPv6.cpp index f23e3f1ba99..2e08828f724 100644 --- a/src/Common/formatIPv6.cpp +++ b/src/Common/formatIPv6.cpp @@ -153,7 +153,7 @@ void formatIPv6(const unsigned char * src, char *& dst, uint8_t zeroed_tail_byte } /// Was it a trailing run of 0x00's? - if (best.base != -1 && size_t(best.base) + size_t(best.len) == words.size()) + if (best.base != -1 && static_cast(best.base) + static_cast(best.len) == words.size()) *dst++ = ':'; *dst++ = '\0'; diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index 07dd8219fae..7c22dec3777 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -143,7 +143,7 @@ void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_ UInt8 bytes_to_skip = uncompressed_size % bytes_size; UInt32 output_size = uncompressed_size - bytes_to_skip; - if (UInt32(2 + bytes_to_skip) > source_size) + if (static_cast(2 + bytes_to_skip) > source_size) throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); @@ -186,7 +186,7 @@ UInt8 getDeltaBytesSize(const IDataType * column_type) void registerCodecDelta(CompressionCodecFactory & factory) { - UInt8 method_code = UInt8(CompressionMethodByte::Delta); + UInt8 method_code = static_cast(CompressionMethodByte::Delta); factory.registerCompressionCodecWithType("Delta", method_code, [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr { UInt8 delta_bytes_size = 0; diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index c4561b1fbc8..be70b42c73c 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -520,7 +520,7 @@ void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 s UInt8 bytes_to_skip = uncompressed_size % bytes_size; UInt32 output_size = uncompressed_size - bytes_to_skip; - if (UInt32(2 + bytes_to_skip) > source_size) + if (static_cast(2 + bytes_to_skip) > source_size) throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); @@ -544,7 +544,7 @@ void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 s void registerCodecDoubleDelta(CompressionCodecFactory & factory) { - UInt8 method_code = UInt8(CompressionMethodByte::DoubleDelta); + UInt8 method_code = static_cast(CompressionMethodByte::DoubleDelta); factory.registerCompressionCodecWithType("DoubleDelta", method_code, [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr { diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index ddf2fb26712..646dd0cabe5 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -50,11 +50,11 @@ uint8_t getMethodCode(EncryptionMethod Method) { if (Method == AES_128_GCM_SIV) { - return uint8_t(CompressionMethodByte::AES_128_GCM_SIV); + return static_cast(CompressionMethodByte::AES_128_GCM_SIV); } else if (Method == AES_256_GCM_SIV) { - return uint8_t(CompressionMethodByte::AES_256_GCM_SIV); + return static_cast(CompressionMethodByte::AES_256_GCM_SIV); } else { diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index cf3947cc9b5..a8036f7490f 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -419,7 +419,7 @@ void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 sourc UInt8 bytes_to_skip = uncompressed_size % bytes_size; - if (UInt32(2 + bytes_to_skip) > source_size) + if (static_cast(2 + bytes_to_skip) > source_size) throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); @@ -443,7 +443,7 @@ void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 sourc void registerCodecGorilla(CompressionCodecFactory & factory) { - UInt8 method_code = UInt8(CompressionMethodByte::Gorilla); + UInt8 method_code = static_cast(CompressionMethodByte::Gorilla); factory.registerCompressionCodecWithType("Gorilla", method_code, [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr { diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index 3bff8474c2e..0efa98fbd82 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -112,7 +112,7 @@ MagicNumber serializeTypeId(TypeIndex type_id) break; } - throw Exception("Type is not supported by T64 codec: " + toString(UInt32(type_id)), ErrorCodes::LOGICAL_ERROR); + throw Exception("Type is not supported by T64 codec: " + toString(static_cast(type_id)), ErrorCodes::LOGICAL_ERROR); } TypeIndex deserializeTypeId(uint8_t serialized_type_id) @@ -137,7 +137,7 @@ TypeIndex deserializeTypeId(uint8_t serialized_type_id) case MagicNumber::Decimal64: return TypeIndex::Decimal64; } - throw Exception("Bad magic number in T64 codec: " + toString(UInt32(serialized_type_id)), ErrorCodes::LOGICAL_ERROR); + throw Exception("Bad magic number in T64 codec: " + toString(static_cast(serialized_type_id)), ErrorCodes::LOGICAL_ERROR); } @@ -284,22 +284,22 @@ void reverseTransposeBytes(const UInt64 * matrix, UInt32 col, T & value) if constexpr (sizeof(T) > 4) { - value |= UInt64(matrix8[64 * 7 + col]) << (8 * 7); - value |= UInt64(matrix8[64 * 6 + col]) << (8 * 6); - value |= UInt64(matrix8[64 * 5 + col]) << (8 * 5); - value |= UInt64(matrix8[64 * 4 + col]) << (8 * 4); + value |= static_cast(matrix8[64 * 7 + col]) << (8 * 7); + value |= static_cast(matrix8[64 * 6 + col]) << (8 * 6); + value |= static_cast(matrix8[64 * 5 + col]) << (8 * 5); + value |= static_cast(matrix8[64 * 4 + col]) << (8 * 4); } if constexpr (sizeof(T) > 2) { - value |= UInt32(matrix8[64 * 3 + col]) << (8 * 3); - value |= UInt32(matrix8[64 * 2 + col]) << (8 * 2); + value |= static_cast(matrix8[64 * 3 + col]) << (8 * 3); + value |= static_cast(matrix8[64 * 2 + col]) << (8 * 2); } if constexpr (sizeof(T) > 1) - value |= UInt32(matrix8[64 * 1 + col]) << (8 * 1); + value |= static_cast(matrix8[64 * 1 + col]) << (8 * 1); - value |= UInt32(matrix8[col]); + value |= static_cast(matrix8[col]); } @@ -422,12 +422,12 @@ UInt32 getValuableBitsNumber(Int64 min, Int64 max) if (min < 0 && max >= 0) { if (min + max >= 0) - return getValuableBitsNumber(0ull, UInt64(max)) + 1; + return getValuableBitsNumber(0ull, static_cast(max)) + 1; else - return getValuableBitsNumber(0ull, UInt64(~min)) + 1; + return getValuableBitsNumber(0ull, static_cast(~min)) + 1; } else - return getValuableBitsNumber(UInt64(min), UInt64(max)); + return getValuableBitsNumber(static_cast(min), static_cast(max)); } @@ -559,14 +559,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 = UInt64(min) >> num_bits << num_bits; + upper_min = 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 = UInt64(max) >> num_bits << num_bits; + upper_max = static_cast(max) >> num_bits << num_bits; } } diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index def2188d90a..b47c8c4b080 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -109,7 +109,7 @@ CompressionCodecZSTD::CompressionCodecZSTD(int level_) : level(level_), enable_l void registerCodecZSTD(CompressionCodecFactory & factory) { - UInt8 method_code = UInt8(CompressionMethodByte::ZSTD); + UInt8 method_code = static_cast(CompressionMethodByte::ZSTD); factory.registerCompressionCodec("ZSTD", method_code, [&](const ASTPtr & arguments) -> CompressionCodecPtr { int level = CompressionCodecZSTD::ZSTD_DEFAULT_LEVEL; if (arguments && !arguments->children.empty()) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 5018d392e3c..95435cdd911 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -628,12 +628,12 @@ void StreamStatistics::print() const { std::cerr << "Num tokens: " << num_tokens - << ", Avg literal length: " << double(sum_literal_lengths) / num_tokens - << ", Avg match length: " << double(sum_match_lengths) / num_tokens - << ", Avg match offset: " << double(sum_match_offsets) / num_tokens - << ", Offset < 8 ratio: " << double(count_match_offset_less_8) / num_tokens - << ", Offset < 16 ratio: " << double(count_match_offset_less_16) / num_tokens - << ", Match replicate itself: " << double(count_match_replicate_itself) / num_tokens + << ", Avg literal length: " << static_cast(sum_literal_lengths) / num_tokens + << ", Avg match length: " << static_cast(sum_match_lengths) / num_tokens + << ", Avg match offset: " << static_cast(sum_match_offsets) / num_tokens + << ", Offset < 8 ratio: " << static_cast(count_match_offset_less_8) / num_tokens + << ", Offset < 16 ratio: " << static_cast(count_match_offset_less_16) / num_tokens + << ", Match replicate itself: " << static_cast(count_match_replicate_itself) / num_tokens << "\n"; } diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index dd7cf1d2c8c..9f30fa566ee 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -99,20 +99,20 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const writeText("max_requests_batch_size=", buf); write_int(coordination_settings->max_requests_batch_size); writeText("min_session_timeout_ms=", buf); - write_int(uint64_t(coordination_settings->min_session_timeout_ms)); + write_int(static_cast(coordination_settings->min_session_timeout_ms)); writeText("session_timeout_ms=", buf); - write_int(uint64_t(coordination_settings->session_timeout_ms)); + write_int(static_cast(coordination_settings->session_timeout_ms)); writeText("operation_timeout_ms=", buf); - write_int(uint64_t(coordination_settings->operation_timeout_ms)); + write_int(static_cast(coordination_settings->operation_timeout_ms)); writeText("dead_session_check_period_ms=", buf); - write_int(uint64_t(coordination_settings->dead_session_check_period_ms)); + write_int(static_cast(coordination_settings->dead_session_check_period_ms)); writeText("heart_beat_interval_ms=", buf); - write_int(uint64_t(coordination_settings->heart_beat_interval_ms)); + write_int(static_cast(coordination_settings->heart_beat_interval_ms)); writeText("election_timeout_lower_bound_ms=", buf); - write_int(uint64_t(coordination_settings->election_timeout_lower_bound_ms)); + write_int(static_cast(coordination_settings->election_timeout_lower_bound_ms)); writeText("election_timeout_upper_bound_ms=", buf); - write_int(uint64_t(coordination_settings->election_timeout_upper_bound_ms)); + write_int(static_cast(coordination_settings->election_timeout_upper_bound_ms)); writeText("reserved_log_items=", buf); write_int(coordination_settings->reserved_log_items); @@ -122,9 +122,9 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const writeText("auto_forwarding=", buf); write_bool(coordination_settings->auto_forwarding); writeText("shutdown_timeout=", buf); - write_int(uint64_t(coordination_settings->shutdown_timeout)); + write_int(static_cast(coordination_settings->shutdown_timeout)); writeText("startup_timeout=", buf); - write_int(uint64_t(coordination_settings->startup_timeout)); + write_int(static_cast(coordination_settings->startup_timeout)); writeText("raft_logs_level=", buf); writeText(coordination_settings->raft_logs_level.toString(), buf); diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 1c1f6535550..87566eade54 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -221,7 +221,7 @@ namespace MySQLReplication case MYSQL_TYPE_BLOB: case MYSQL_TYPE_GEOMETRY: { - column_meta.emplace_back(UInt16(meta[pos])); + column_meta.emplace_back(static_cast(meta[pos])); pos += 1; break; } @@ -229,9 +229,9 @@ namespace MySQLReplication case MYSQL_TYPE_STRING: { /// Big-Endian - auto b0 = UInt16(meta[pos] << 8); - auto b1 = UInt8(meta[pos + 1]); - column_meta.emplace_back(UInt16(b0 + b1)); + auto b0 = static_cast(meta[pos] << 8); + auto b1 = static_cast(meta[pos + 1]); + column_meta.emplace_back(static_cast(b0 + b1)); pos += 2; break; } @@ -239,9 +239,9 @@ namespace MySQLReplication case MYSQL_TYPE_VARCHAR: case MYSQL_TYPE_VAR_STRING: { /// Little-Endian - auto b0 = UInt8(meta[pos]); - auto b1 = UInt16(meta[pos + 1] << 8); - column_meta.emplace_back(UInt16(b0 + b1)); + auto b0 = static_cast(meta[pos]); + auto b1 = static_cast(meta[pos + 1] << 8); + column_meta.emplace_back(static_cast(b0 + b1)); pos += 2; break; } @@ -543,7 +543,7 @@ namespace MySQLReplication ); if (!meta) - row.push_back(Field{UInt32(date_time)}); + row.push_back(Field{static_cast(date_time)}); else { DB::DecimalUtils::DecimalComponents components{ @@ -603,7 +603,7 @@ namespace MySQLReplication throw Exception("Attempt to read after EOF.", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF); if ((*payload.position() & 0x80) == 0) - mask = UInt32(-1); + mask = static_cast(-1); *payload.position() ^= 0x80; diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index e0bd8ebcbb9..0babecab5a5 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -301,7 +301,7 @@ ColumnUInt8::Ptr IPAddressDictionary::hasKeys(const Columns & key_columns, const uint8_t addrv6_buf[IPV6_BINARY_LENGTH]; for (const auto i : collections::range(0, rows)) { - auto addrv4 = UInt32(first_column->get64(i)); + auto addrv4 = static_cast(first_column->get64(i)); auto found = tryLookupIPv4(addrv4, addrv6_buf); out[i] = (found != ipNotFound()); keys_found += out[i]; @@ -387,7 +387,7 @@ void IPAddressDictionary::loadData() setAttributeValue(attribute, attribute_column[row]); } - const auto [addr, prefix] = parseIPFromString(std::string_view(key_column_ptr->getDataAt(row))); + const auto [addr, prefix] = parseIPFromString(std::string_view{key_column_ptr->getDataAt(row)}); has_ipv6 = has_ipv6 || (addr.family() == Poco::Net::IPAddress::IPv6); size_t row_number = ip_records.size(); @@ -716,7 +716,7 @@ void IPAddressDictionary::getItemsImpl( for (const auto i : collections::range(0, rows)) { // addrv4 has native endianness - auto addrv4 = UInt32(first_column->get64(i)); + auto addrv4 = static_cast(first_column->get64(i)); auto found = tryLookupIPv4(addrv4, addrv6_buf); if (found != ipNotFound()) { diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index fb637263cf4..98bc2b1b0c4 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -181,7 +181,7 @@ Pipe MongoDBDictionarySource::loadIds(const std::vector & ids) Poco::MongoDB::Array::Ptr ids_array(new Poco::MongoDB::Array); for (const UInt64 id : ids) - ids_array->add(DB::toString(id), Int32(id)); + ids_array->add(DB::toString(id), static_cast(id)); cursor->query().selector().addNewDocument(dict_struct.id->name).add("$in", ids_array); @@ -218,7 +218,7 @@ Pipe MongoDBDictionarySource::loadKeys(const Columns & key_columns, const std::v case AttributeUnderlyingType::Int32: case AttributeUnderlyingType::Int64: { - key.add(key_attribute.name, Int32(key_columns[attribute_index]->get64(row_idx))); + key.add(key_attribute.name, static_cast(key_columns[attribute_index]->get64(row_idx))); break; } case AttributeUnderlyingType::Float32: diff --git a/src/Functions/FunctionsBitToArray.cpp b/src/Functions/FunctionsBitToArray.cpp index 309fbafaf91..76496b102cd 100644 --- a/src/Functions/FunctionsBitToArray.cpp +++ b/src/Functions/FunctionsBitToArray.cpp @@ -93,7 +93,7 @@ private: if (!first) writeChar(',', out); first = false; - writeIntText(T(bit), out); + writeIntText(static_cast(bit), out); } } diff --git a/src/Functions/FunctionsCodingIP.cpp b/src/Functions/FunctionsCodingIP.cpp index de814529d03..c141d1bf102 100644 --- a/src/Functions/FunctionsCodingIP.cpp +++ b/src/Functions/FunctionsCodingIP.cpp @@ -651,18 +651,18 @@ struct ParseMACImpl */ static UInt64 parse(const char * pos) { - return (UInt64(unhex(pos[0])) << 44) - | (UInt64(unhex(pos[1])) << 40) - | (UInt64(unhex(pos[3])) << 36) - | (UInt64(unhex(pos[4])) << 32) - | (UInt64(unhex(pos[6])) << 28) - | (UInt64(unhex(pos[7])) << 24) - | (UInt64(unhex(pos[9])) << 20) - | (UInt64(unhex(pos[10])) << 16) - | (UInt64(unhex(pos[12])) << 12) - | (UInt64(unhex(pos[13])) << 8) - | (UInt64(unhex(pos[15])) << 4) - | (UInt64(unhex(pos[16]))); + return (static_cast(unhex(pos[0])) << 44) + | (static_cast(unhex(pos[1])) << 40) + | (static_cast(unhex(pos[3])) << 36) + | (static_cast(unhex(pos[4])) << 32) + | (static_cast(unhex(pos[6])) << 28) + | (static_cast(unhex(pos[7])) << 24) + | (static_cast(unhex(pos[9])) << 20) + | (static_cast(unhex(pos[10])) << 16) + | (static_cast(unhex(pos[12])) << 12) + | (static_cast(unhex(pos[13])) << 8) + | (static_cast(unhex(pos[15])) << 4) + | (static_cast(unhex(pos[16]))); } static constexpr auto name = "MACStringToNum"; @@ -678,12 +678,12 @@ struct ParseOUIImpl */ static UInt64 parse(const char * pos) { - return (UInt64(unhex(pos[0])) << 20) - | (UInt64(unhex(pos[1])) << 16) - | (UInt64(unhex(pos[3])) << 12) - | (UInt64(unhex(pos[4])) << 8) - | (UInt64(unhex(pos[6])) << 4) - | (UInt64(unhex(pos[7]))); + return (static_cast(unhex(pos[0])) << 20) + | (static_cast(unhex(pos[1])) << 16) + | (static_cast(unhex(pos[3])) << 12) + | (static_cast(unhex(pos[4])) << 8) + | (static_cast(unhex(pos[6])) << 4) + | (static_cast(unhex(pos[7]))); } static constexpr auto name = "MACStringToOUI"; @@ -895,9 +895,9 @@ private: if (bits_to_keep >= 8 * sizeof(UInt32)) return { src, src }; if (bits_to_keep == 0) - return { UInt32(0), UInt32(-1) }; + return { static_cast(0), static_cast(-1) }; - UInt32 mask = UInt32(-1) << (8 * sizeof(UInt32) - bits_to_keep); + UInt32 mask = static_cast(-1) << (8 * sizeof(UInt32) - bits_to_keep); UInt32 lower = src & mask; UInt32 upper = lower | ~mask; diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index c478446e744..b615f52652c 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -565,7 +565,7 @@ ColumnPtr FunctionAnyArityLogical::executeShortCircuit(ColumnsWithTy /// The result is !mask_n. bool inverted = Name::name != NameAnd::name; - UInt8 null_value = UInt8(Name::name == NameAnd::name); + UInt8 null_value = static_cast(Name::name == NameAnd::name); IColumn::Filter mask(arguments[0].column->size(), 1); /// If result is nullable, we need to create null bytemap of the resulting column. diff --git a/src/Functions/FunctionsStringSimilarity.cpp b/src/Functions/FunctionsStringSimilarity.cpp index a8cf6a5cf38..6b2d34a182c 100644 --- a/src/Functions/FunctionsStringSimilarity.cpp +++ b/src/Functions/FunctionsStringSimilarity.cpp @@ -271,9 +271,9 @@ struct NgramDistanceImpl size_t first_size = dispatchSearcher(calculateHaystackStatsAndMetric, data.data(), data_size, common_stats.get(), distance, nullptr); /// For !symmetric version we should not use first_size. if constexpr (symmetric) - res = distance * 1.f / std::max(first_size + second_size, size_t(1)); + res = distance * 1.f / std::max(first_size + second_size, static_cast(1)); else - res = 1.f - distance * 1.f / std::max(second_size, size_t(1)); + res = 1.f - distance * 1.f / std::max(second_size, static_cast(1)); } else { @@ -339,9 +339,9 @@ struct NgramDistanceImpl /// For !symmetric version we should not use haystack_stats_size. if constexpr (symmetric) - res[i] = distance * 1.f / std::max(haystack_stats_size + needle_stats_size, size_t(1)); + res[i] = distance * 1.f / std::max(haystack_stats_size + needle_stats_size, static_cast(1)); else - res[i] = 1.f - distance * 1.f / std::max(needle_stats_size, size_t(1)); + res[i] = 1.f - distance * 1.f / std::max(needle_stats_size, static_cast(1)); } else { @@ -410,7 +410,7 @@ struct NgramDistanceImpl for (size_t j = 0; j < needle_stats_size; ++j) --common_stats[needle_ngram_storage[j]]; - res[i] = 1.f - distance * 1.f / std::max(needle_stats_size, size_t(1)); + res[i] = 1.f - distance * 1.f / std::max(needle_stats_size, static_cast(1)); } else { @@ -457,9 +457,9 @@ struct NgramDistanceImpl ngram_storage.get()); /// For !symmetric version we should not use haystack_stats_size. if constexpr (symmetric) - res[i] = distance * 1.f / std::max(haystack_stats_size + needle_stats_size, size_t(1)); + res[i] = distance * 1.f / std::max(haystack_stats_size + needle_stats_size, static_cast(1)); else - res[i] = 1.f - distance * 1.f / std::max(needle_stats_size, size_t(1)); + res[i] = 1.f - distance * 1.f / std::max(needle_stats_size, static_cast(1)); } else { diff --git a/src/Functions/FunctionsTransactionCounters.cpp b/src/Functions/FunctionsTransactionCounters.cpp index f2e9d3aa84b..f28cfb60dfa 100644 --- a/src/Functions/FunctionsTransactionCounters.cpp +++ b/src/Functions/FunctionsTransactionCounters.cpp @@ -23,7 +23,7 @@ public: if (txn) res = {txn->tid.start_csn, txn->tid.local_tid, txn->tid.host_id}; else - res = {UInt64(0), UInt64(0), UUIDHelpers::Nil}; + res = {static_cast(0), static_cast(0), UUIDHelpers::Nil}; return res; } diff --git a/src/Functions/GeoHash.cpp b/src/Functions/GeoHash.cpp index 595bcacd41a..64dfe80ae93 100644 --- a/src/Functions/GeoHash.cpp +++ b/src/Functions/GeoHash.cpp @@ -306,7 +306,7 @@ GeohashesInBoxPreparedArgs geohashesInBoxPrepare( return GeohashesInBoxPreparedArgs { - std::max(1, UInt64(lon_items) * lat_items), + std::max(1, static_cast(lon_items) * lat_items), lon_items, lat_items, lon_min, diff --git a/src/Functions/URL/port.cpp b/src/Functions/URL/port.cpp index 8cc5ce711d5..afe27c9240c 100644 --- a/src/Functions/URL/port.cpp +++ b/src/Functions/URL/port.cpp @@ -113,7 +113,7 @@ private: return default_port; port = (port * 10) + (*p - '0'); - if (port < 0 || port > UInt16(-1)) + if (port < 0 || port > static_cast(-1)) return default_port; ++p; } diff --git a/src/Functions/appendTrailingCharIfAbsent.cpp b/src/Functions/appendTrailingCharIfAbsent.cpp index 698eeeed5f6..1661ec8fb0e 100644 --- a/src/Functions/appendTrailingCharIfAbsent.cpp +++ b/src/Functions/appendTrailingCharIfAbsent.cpp @@ -94,7 +94,7 @@ private: src_offset = src_offsets[i]; dst_offset += src_length; - if (src_length > 1 && dst_data[dst_offset - 2] != UInt8(trailing_char_str.front())) + if (src_length > 1 && dst_data[dst_offset - 2] != static_cast(trailing_char_str.front())) { dst_data[dst_offset - 1] = trailing_char_str.front(); dst_data[dst_offset] = 0; diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 9bebcf7fd8c..1af0ca331a7 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -132,7 +132,7 @@ public: if (count_positive == 0 || count_positive == size) return std::numeric_limits::quiet_NaN(); - return ResultType(area) / count_positive / (size - count_positive); + return static_cast(area) / count_positive / (size - count_positive); } }; diff --git a/src/Functions/array/arrayAggregation.cpp b/src/Functions/array/arrayAggregation.cpp index 97a2f9c4c17..9c18c6213de 100644 --- a/src/Functions/array/arrayAggregation.cpp +++ b/src/Functions/array/arrayAggregation.cpp @@ -183,7 +183,7 @@ struct ArrayAggregateImpl { size_t array_size = offsets[i] - pos; /// Just multiply the value by array size. - res[i] = x * ResultType(array_size); + res[i] = x * static_cast(array_size); } else if constexpr (aggregate_operation == AggregateOperation::min || aggregate_operation == AggregateOperation::max) diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index 04195a7eef9..dedde89a59e 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -152,7 +152,7 @@ public: void update() { - sink_null_map[index] = bool(src_null_map); + sink_null_map[index] = static_cast(src_null_map); ++index; } @@ -492,7 +492,7 @@ ColumnPtr FunctionArrayElement::executeNumberConst( /// arr[-2] is the element at offset 1 from the last and so on. ArrayElementNumImpl::template vectorConst( - col_nested->getData(), col_array->getOffsets(), -(UInt64(safeGet(index)) + 1), col_res->getData(), builder); + col_nested->getData(), col_array->getOffsets(), -(static_cast(safeGet(index)) + 1), col_res->getData(), builder); } else throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); @@ -605,7 +605,7 @@ ColumnPtr FunctionArrayElement::executeGenericConst( col_nested, col_array->getOffsets(), safeGet(index) - 1, *col_res, builder); else if (index.getType() == Field::Types::Int64) ArrayElementGenericImpl::vectorConst( - col_nested, col_array->getOffsets(), -(UInt64(safeGet(index) + 1)), *col_res, builder); + col_nested, col_array->getOffsets(), -(static_cast(safeGet(index) + 1)), *col_res, builder); else throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Functions/array/arrayReverse.cpp b/src/Functions/array/arrayReverse.cpp index 1a6b1aa6c1f..a517080ff2c 100644 --- a/src/Functions/array/arrayReverse.cpp +++ b/src/Functions/array/arrayReverse.cpp @@ -112,7 +112,7 @@ bool FunctionArrayReverse::executeGeneric(const IColumn & src_data, const Column { ssize_t src_index = src_array_offsets[i] - 1; - while (src_index >= ssize_t(src_prev_offset)) + while (src_index >= static_cast(src_prev_offset)) { res_data.insertFrom(src_data, src_index); --src_index; diff --git a/src/Functions/bitShiftLeft.cpp b/src/Functions/bitShiftLeft.cpp index 60e61753d53..83a31ddc122 100644 --- a/src/Functions/bitShiftLeft.cpp +++ b/src/Functions/bitShiftLeft.cpp @@ -72,7 +72,7 @@ struct BitShiftLeftImpl if (shift_left_bits) { /// The left b bit of the right byte is moved to the right b bit of this byte - *out = UInt8(UInt8(*(op_pointer) >> (8 - shift_left_bits)) | previous); + *out = static_cast(static_cast(*(op_pointer) >> (8 - shift_left_bits)) | previous); previous = *op_pointer << shift_left_bits; } else @@ -131,7 +131,7 @@ struct BitShiftLeftImpl if (op_pointer + 1 < end) { /// The left b bit of the right byte is moved to the right b bit of this byte - *out = UInt8(UInt8(*(op_pointer + 1) >> (8 - shift_left_bits)) | *out); + *out = static_cast(static_cast(*(op_pointer + 1) >> (8 - shift_left_bits)) | *out); } op_pointer++; out++; diff --git a/src/Functions/bitShiftRight.cpp b/src/Functions/bitShiftRight.cpp index f3e16e4ed62..d12847c5df8 100644 --- a/src/Functions/bitShiftRight.cpp +++ b/src/Functions/bitShiftRight.cpp @@ -41,7 +41,7 @@ struct BitShiftRightImpl if (op_pointer - 1 >= begin) { /// The right b bit of the left byte is moved to the left b bit of this byte - *out = UInt8(UInt8(*(op_pointer - 1) << (8 - shift_right_bits)) | *out); + *out = static_cast(static_cast(*(op_pointer - 1) << (8 - shift_right_bits)) | *out); } } } diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index b632627fea9..e778e4dc6a9 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -235,8 +235,8 @@ private: template Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const { - return Int64(transform_y.execute(y, timezone_y)) - - Int64(transform_x.execute(x, timezone_x)); + return static_cast(transform_y.execute(y, timezone_y)) + - static_cast(transform_x.execute(x, timezone_x)); } template diff --git a/src/Functions/hasColumnInTable.cpp b/src/Functions/hasColumnInTable.cpp index 1062fbed600..a70bf57f7c2 100644 --- a/src/Functions/hasColumnInTable.cpp +++ b/src/Functions/hasColumnInTable.cpp @@ -150,7 +150,7 @@ ColumnPtr FunctionHasColumnInTable::executeImpl(const ColumnsWithTypeAndName & a has_column = remote_columns.hasPhysical(column_name); } - return DataTypeUInt8().createColumnConst(input_rows_count, Field{UInt64(has_column)}); + return DataTypeUInt8().createColumnConst(input_rows_count, Field{static_cast(has_column)}); } } diff --git a/src/Functions/jumpConsistentHash.cpp b/src/Functions/jumpConsistentHash.cpp index a82f10cba16..bc41913f7e7 100644 --- a/src/Functions/jumpConsistentHash.cpp +++ b/src/Functions/jumpConsistentHash.cpp @@ -15,7 +15,7 @@ inline int32_t JumpConsistentHash(uint64_t key, int32_t num_buckets) { b = j; key = key * 2862933555777941757ULL + 1; - j = static_cast((b + 1) * (double(1LL << 31) / double((key >> 33) + 1))); + j = static_cast((b + 1) * (static_cast(1LL << 31) / static_cast((key >> 33) + 1))); } return static_cast(b); } diff --git a/src/Functions/neighbor.cpp b/src/Functions/neighbor.cpp index ab447e61aed..296eaf67d9c 100644 --- a/src/Functions/neighbor.cpp +++ b/src/Functions/neighbor.cpp @@ -135,7 +135,7 @@ public: } if (size <= 0) return; - if (size > Int64(input_rows_count)) + if (size > static_cast(input_rows_count)) size = input_rows_count; if (!src) @@ -163,14 +163,14 @@ public: } else if (offset > 0) { - insert_range_from(source_is_constant, source_column_casted, offset, Int64(input_rows_count) - offset); - insert_range_from(default_is_constant, default_column_casted, Int64(input_rows_count) - offset, offset); + insert_range_from(source_is_constant, source_column_casted, offset, static_cast(input_rows_count) - offset); + insert_range_from(default_is_constant, default_column_casted, static_cast(input_rows_count) - offset, offset); return result_column; } else { insert_range_from(default_is_constant, default_column_casted, 0, -offset); - insert_range_from(source_is_constant, source_column_casted, 0, Int64(input_rows_count) + offset); + insert_range_from(source_is_constant, source_column_casted, 0, static_cast(input_rows_count) + offset); return result_column; } } @@ -188,7 +188,7 @@ public: Int64 src_idx = row + offset; - if (src_idx >= 0 && src_idx < Int64(input_rows_count)) + if (src_idx >= 0 && src_idx < static_cast(input_rows_count)) result_column->insertFrom(*source_column_casted, source_is_constant ? 0 : src_idx); else if (has_defaults) result_column->insertFrom(*default_column_casted, default_is_constant ? 0 : row); diff --git a/src/Functions/sigmoid.cpp b/src/Functions/sigmoid.cpp index c4e00db6d5c..d675413ae67 100644 --- a/src/Functions/sigmoid.cpp +++ b/src/Functions/sigmoid.cpp @@ -30,7 +30,7 @@ using FunctionSigmoid = FunctionMathUnary; #else -static double sigmoid(double x) +double sigmoid(double x) { return 1.0 / (1.0 + exp(-x)); } diff --git a/src/IO/HadoopSnappyReadBuffer.cpp b/src/IO/HadoopSnappyReadBuffer.cpp index cac05b4827b..ee2cdc92556 100644 --- a/src/IO/HadoopSnappyReadBuffer.cpp +++ b/src/IO/HadoopSnappyReadBuffer.cpp @@ -26,7 +26,7 @@ inline bool HadoopSnappyDecoder::checkBufferLength(int max) const inline bool HadoopSnappyDecoder::checkAvailIn(size_t avail_in, int min) { - return avail_in >= size_t(min); + return avail_in >= static_cast(min); } inline void HadoopSnappyDecoder::copyToBuffer(size_t * avail_in, const char ** next_in) diff --git a/src/IO/PeekableReadBuffer.cpp b/src/IO/PeekableReadBuffer.cpp index 40929acd848..d9de3a5e76a 100644 --- a/src/IO/PeekableReadBuffer.cpp +++ b/src/IO/PeekableReadBuffer.cpp @@ -245,7 +245,7 @@ void PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append) /// Stack memory is not enough, allocate larger buffer. use_stack_memory = false; - memory.resize(std::max(size_t(DBMS_DEFAULT_BUFFER_SIZE), new_size)); + memory.resize(std::max(static_cast(DBMS_DEFAULT_BUFFER_SIZE), new_size)); memcpy(memory.data(), stack_memory, sizeof(stack_memory)); if (need_update_checkpoint) checkpoint.emplace(memory.data() + offset); diff --git a/src/IO/ReadBufferFromMemory.cpp b/src/IO/ReadBufferFromMemory.cpp index d0863878797..3ed5603ce2d 100644 --- a/src/IO/ReadBufferFromMemory.cpp +++ b/src/IO/ReadBufferFromMemory.cpp @@ -16,13 +16,13 @@ off_t ReadBufferFromMemory::seek(off_t offset, int whence) { pos = internal_buffer.begin() + offset; working_buffer = internal_buffer; /// We need to restore `working_buffer` in case the position was at EOF before this seek(). - return size_t(pos - internal_buffer.begin()); + return static_cast(pos - internal_buffer.begin()); } else throw Exception( "Seek position is out of bounds. " "Offset: " - + std::to_string(offset) + ", Max: " + std::to_string(size_t(internal_buffer.end() - internal_buffer.begin())), + + std::to_string(offset) + ", Max: " + std::to_string(static_cast(internal_buffer.end() - internal_buffer.begin())), ErrorCodes::SEEK_POSITION_OUT_OF_BOUND); } else if (whence == SEEK_CUR) @@ -32,13 +32,13 @@ off_t ReadBufferFromMemory::seek(off_t offset, int whence) { pos = new_pos; working_buffer = internal_buffer; /// We need to restore `working_buffer` in case the position was at EOF before this seek(). - return size_t(pos - internal_buffer.begin()); + return static_cast(pos - internal_buffer.begin()); } else throw Exception( "Seek position is out of bounds. " "Offset: " - + std::to_string(offset) + ", Max: " + std::to_string(size_t(internal_buffer.end() - internal_buffer.begin())), + + std::to_string(offset) + ", Max: " + std::to_string(static_cast(internal_buffer.end() - internal_buffer.begin())), ErrorCodes::SEEK_POSITION_OUT_OF_BOUND); } else diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 728893e912d..d19fbd28265 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -175,7 +175,7 @@ off_t ReadBufferFromS3::seek(off_t offset_, int whence) if (!restricted_seek) { if (!working_buffer.empty() - && size_t(offset_) >= offset - working_buffer.size() + && static_cast(offset_) >= offset - working_buffer.size() && offset_ < offset) { pos = working_buffer.end() - (offset - offset_); diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index c85f3989531..a44eb86fb3e 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -379,7 +379,7 @@ void WriteBufferFromS3::completeMultipartUpload() void WriteBufferFromS3::makeSinglepartUpload() { auto size = temporary_buffer->tellp(); - bool with_pool = bool(schedule); + bool with_pool = static_cast(schedule); LOG_TRACE(log, "Making single part upload. Bucket: {}, Key: {}, Size: {}, WithPool: {}", bucket, key, size, with_pool); @@ -467,7 +467,7 @@ void WriteBufferFromS3::fillPutRequest(Aws::S3::Model::PutObjectRequest & req) void WriteBufferFromS3::processPutRequest(PutObjectTask & task) { auto outcome = client_ptr->PutObject(task.req); - bool with_pool = bool(schedule); + bool with_pool = static_cast(schedule); if (outcome.IsSuccess()) LOG_TRACE(log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}, WithPool: {}", bucket, key, task.req.GetContentLength(), with_pool); diff --git a/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp b/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp index 383ca3db6f4..20ea48d5354 100644 --- a/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp +++ b/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp @@ -59,13 +59,13 @@ Field zeroField(const Field & value) { switch (value.getType()) { - case Field::Types::UInt64: return UInt64(0); - case Field::Types::Int64: return Int64(0); - case Field::Types::Float64: return Float64(0); - case Field::Types::UInt128: return UInt128(0); - case Field::Types::Int128: return Int128(0); - case Field::Types::UInt256: return UInt256(0); - case Field::Types::Int256: return Int256(0); + case Field::Types::UInt64: return static_cast(0); + case Field::Types::Int64: return static_cast(0); + case Field::Types::Float64: return static_cast(0); + case Field::Types::UInt128: return static_cast(0); + case Field::Types::Int128: return static_cast(0); + case Field::Types::UInt256: return static_cast(0); + case Field::Types::Int256: return static_cast(0); default: break; } diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index 614a966593f..8136a2dde67 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -24,7 +24,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const if (server_protocol_revision < DBMS_MIN_REVISION_WITH_CLIENT_INFO) throw Exception("Logical error: method ClientInfo::write is called for unsupported server revision", ErrorCodes::LOGICAL_ERROR); - writeBinary(UInt8(query_kind), out); + writeBinary(static_cast(query_kind), out); if (empty()) return; @@ -35,7 +35,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const if (server_protocol_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME) writeBinary(initial_query_start_time_microseconds, out); - writeBinary(UInt8(interface), out); + writeBinary(static_cast(interface), out); if (interface == Interface::TCP) { @@ -48,7 +48,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const } else if (interface == Interface::HTTP) { - writeBinary(UInt8(http_method), out); + writeBinary(static_cast(http_method), out); writeBinary(http_user_agent, out); if (server_protocol_revision >= DBMS_MIN_REVISION_WITH_X_FORWARDED_FOR_IN_CLIENT_INFO) @@ -86,7 +86,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const else { // Don't have OpenTelemetry header. - writeBinary(uint8_t(0), out); + writeBinary(static_cast(0), out); } } diff --git a/src/Interpreters/CrashLog.cpp b/src/Interpreters/CrashLog.cpp index 1abaec4a17f..e85958f1e94 100644 --- a/src/Interpreters/CrashLog.cpp +++ b/src/Interpreters/CrashLog.cpp @@ -82,7 +82,7 @@ void collectCrashLog(Int32 signal, UInt64 thread_id, const String & query_id, co stack_trace.toStringEveryLine([&trace_full](const std::string & line) { trace_full.push_back(line); }); - CrashLogElement element{time_t(time / 1000000000), time, signal, thread_id, query_id, trace, trace_full}; + CrashLogElement element{static_cast(time / 1000000000), time, signal, thread_id, query_id, trace, trace_full}; crash_log_owned->add(element); } } diff --git a/src/Interpreters/DNSCacheUpdater.cpp b/src/Interpreters/DNSCacheUpdater.cpp index e7918451a2f..bf88d19b7ef 100644 --- a/src/Interpreters/DNSCacheUpdater.cpp +++ b/src/Interpreters/DNSCacheUpdater.cpp @@ -39,7 +39,7 @@ void DNSCacheUpdater::run() * - automatically throttle when DNS requests take longer time; * - add natural randomization on huge clusters - avoid sending all requests at the same moment of time from different servers. */ - task_handle->scheduleAfter(size_t(update_period_seconds) * 1000); + task_handle->scheduleAfter(static_cast(update_period_seconds) * 1000); } void DNSCacheUpdater::start() diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index b52b91f47eb..333aed84873 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -48,7 +48,7 @@ BlockIO InterpreterCheckQuery::execute() { bool result = std::all_of(check_results.begin(), check_results.end(), [] (const CheckResult & res) { return res.success; }); auto column = ColumnUInt8::create(); - column->insertValue(UInt64(result)); + column->insertValue(static_cast(result)); block = Block{{std::move(column), std::make_shared(), "result"}}; } else diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 58ca9ccc978..b0aaac6e745 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -380,7 +380,7 @@ BlockIO InterpreterInsertQuery::execute() pipeline.dropTotalsAndExtremes(); if (table->supportsParallelInsert() && settings.max_insert_threads > 1) - out_streams_size = std::min(size_t(settings.max_insert_threads), pipeline.getNumStreams()); + out_streams_size = std::min(static_cast(settings.max_insert_threads), pipeline.getNumStreams()); pipeline.resize(out_streams_size); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 130b3aae58d..7506c3013cb 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -104,7 +104,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( } else if (settings.offset) { - ASTPtr new_limit_offset_ast = std::make_shared(Field(UInt64(settings.offset))); + ASTPtr new_limit_offset_ast = std::make_shared(Field(static_cast(settings.offset))); select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, std::move(new_limit_offset_ast)); } @@ -115,15 +115,15 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( UInt64 new_limit_length = 0; if (settings.offset == 0) - new_limit_length = std::min(limit_length, UInt64(settings.limit)); + new_limit_length = std::min(limit_length, static_cast(settings.limit)); else if (settings.offset < limit_length) - new_limit_length = settings.limit ? std::min(UInt64(settings.limit), limit_length - settings.offset) : (limit_length - settings.offset); + new_limit_length = settings.limit ? std::min(static_cast(settings.limit), limit_length - settings.offset) : (limit_length - settings.offset); limit_length_ast->as().value = Field(new_limit_length); } else if (settings.limit) { - ASTPtr new_limit_length_ast = std::make_shared(Field(UInt64(settings.limit))); + ASTPtr new_limit_length_ast = std::make_shared(Field(static_cast(settings.limit))); select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::move(new_limit_length_ast)); } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 30f0f892ca4..28a2082d233 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -588,7 +588,7 @@ void InterpreterSystemQuery::restartReplicas(ContextMutablePtr system_context) for (auto & guard : guards) guard.second = catalog.getDDLGuard(guard.first.database_name, guard.first.table_name); - ThreadPool pool(std::min(size_t(getNumberOfPhysicalCPUCores()), replica_names.size())); + ThreadPool pool(std::min(static_cast(getNumberOfPhysicalCPUCores()), replica_names.size())); for (auto & replica : replica_names) { diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 4c0aa20d795..d62ce9760e9 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -325,7 +325,7 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys) return std::make_shared(column_name); return makeASTFunction("intDiv", std::make_shared(column_name), - std::make_shared(UInt64(type_max_size / 1000))); + std::make_shared(static_cast(type_max_size / 1000))); }; ASTPtr best_partition; @@ -493,7 +493,7 @@ ASTs InterpreterCreateImpl::getRewrittenQueries( String sign_column_name = getUniqueColumnName(columns_name_and_type, "_sign"); String version_column_name = getUniqueColumnName(columns_name_and_type, "_version"); columns->set(columns->columns, InterpreterCreateQuery::formatColumns(columns_description)); - columns->columns->children.emplace_back(create_materialized_column_declaration(sign_column_name, "Int8", UInt64(1))); + columns->columns->children.emplace_back(create_materialized_column_declaration(sign_column_name, "Int8", static_cast(1))); columns->columns->children.emplace_back(create_materialized_column_declaration(version_column_name, "UInt64", UInt64(1))); /// Add minmax skipping index for _version column. diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index d7add73dbf0..6a3f4327de9 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -71,7 +71,7 @@ static void dumpProfileEvents(ProfileEventsSnapshot const & snapshot, DB::Mutabl { size_t i = 0; columns[i++]->insertData(host_name.data(), host_name.size()); - columns[i++]->insert(UInt64(snapshot.current_time)); + columns[i++]->insert(static_cast(snapshot.current_time)); columns[i++]->insert(UInt64{snapshot.thread_id}); columns[i++]->insert(Type::INCREMENT); } @@ -81,8 +81,8 @@ static void dumpMemoryTracker(ProfileEventsSnapshot const & snapshot, DB::Mutabl { size_t i = 0; columns[i++]->insertData(host_name.data(), host_name.size()); - columns[i++]->insert(UInt64(snapshot.current_time)); - columns[i++]->insert(UInt64{snapshot.thread_id}); + columns[i++]->insert(static_cast(snapshot.current_time)); + columns[i++]->insert(static_cast{snapshot.thread_id}); columns[i++]->insert(Type::GAUGE); columns[i++]->insertData(MemoryTracker::USAGE_EVENT_NAME, strlen(MemoryTracker::USAGE_EVENT_NAME)); diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 03eeb81f14d..72d9ce44102 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -279,7 +279,7 @@ void QueryLogElement::appendClientInfo(const ClientInfo & client_info, MutableCo columns[i++]->insert(client_info.initial_query_start_time); columns[i++]->insert(client_info.initial_query_start_time_microseconds); - columns[i++]->insert(UInt64(client_info.interface)); + columns[i++]->insert(static_cast(client_info.interface)); columns[i++]->insert(static_cast(client_info.is_secure)); columns[i++]->insert(client_info.os_user); @@ -290,7 +290,7 @@ void QueryLogElement::appendClientInfo(const ClientInfo & client_info, MutableCo columns[i++]->insert(client_info.client_version_minor); columns[i++]->insert(client_info.client_version_patch); - columns[i++]->insert(UInt64(client_info.http_method)); + columns[i++]->insert(static_cast(client_info.http_method)); columns[i++]->insert(client_info.http_user_agent); columns[i++]->insert(client_info.http_referer); columns[i++]->insert(client_info.forwarded_for); diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 8fbbdb44c99..8394f8050b8 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -49,7 +49,7 @@ void ThreadStatus::applyQuerySettings() initQueryProfiler(); untracked_memory_limit = settings.max_untracked_memory; - if (settings.memory_profiler_step && settings.memory_profiler_step < UInt64(untracked_memory_limit)) + if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast(untracked_memory_limit)) untracked_memory_limit = settings.memory_profiler_step; #if defined(OS_LINUX) diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 84d2e25f70a..1124bac6080 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -84,7 +84,7 @@ void TraceCollector::run() { uintptr_t addr = 0; readPODBinary(addr, in); - trace.emplace_back(UInt64(addr)); + trace.emplace_back(static_cast(addr)); } TraceType trace_type; @@ -103,8 +103,8 @@ void TraceCollector::run() struct timespec ts; clock_gettime(CLOCK_REALTIME, &ts); - UInt64 time = UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec); - UInt64 time_in_microseconds = UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); + UInt64 time = static_cast(ts.tv_sec * 1000000000LL + ts.tv_nsec); + UInt64 time_in_microseconds = static_cast((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size}; trace_log->add(element); } diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 8885db1ad78..76896e2fe60 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -76,7 +76,7 @@ void appendUnusedGroupByColumn(ASTSelectQuery * select_query) /// Also start unused_column integer must not intersect with ([1, source_columns.size()]) /// might be in positional GROUP BY. select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, std::make_shared()); - select_query->groupBy()->children.emplace_back(std::make_shared(Int64(-1))); + select_query->groupBy()->children.emplace_back(std::make_shared(static_cast(-1))); } /// Eliminates injective function calls and constant expressions from group by statement. diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index d0e54f9f598..4b8dc5befb9 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -294,7 +294,7 @@ struct ExistsExpressionData select_query->setExpression(ASTSelectQuery::Expression::SELECT, select_expr_list); select_query->setExpression(ASTSelectQuery::Expression::TABLES, tables_in_select); - ASTPtr limit_length_ast = std::make_shared(Field(UInt64(1))); + ASTPtr limit_length_ast = std::make_shared(Field(static_cast(1))); select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::move(limit_length_ast)); auto select_with_union_query = std::make_shared(); @@ -347,7 +347,7 @@ void replaceWithSumCount(String column_name, ASTFunction & func) /// Rewrite "avg" to sumCount().1 / sumCount().2 auto new_arg1 = makeASTFunction("tupleElement", func_base, std::make_shared(UInt8(1))); auto new_arg2 = makeASTFunction("CAST", - makeASTFunction("tupleElement", func_base, std::make_shared(UInt8(2))), + makeASTFunction("tupleElement", func_base, std::make_shared(static_cast(2))), std::make_shared("Float64")); func.name = "divide"; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index a3a5c2946a3..dbad5fcb60e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -892,7 +892,7 @@ static std::tuple executeQueryImpl( ReadableSize(elem.read_bytes / elapsed_seconds)); } - if (log_queries && elem.type >= log_queries_min_type && Int64(elem.query_duration_ms) >= log_queries_min_query_duration_ms) + if (log_queries && elem.type >= log_queries_min_type && static_cast(elem.query_duration_ms) >= log_queries_min_query_duration_ms) { if (auto query_log = context->getQueryLog()) query_log->add(elem); @@ -1009,7 +1009,7 @@ static std::tuple executeQueryImpl( logException(context, elem); /// In case of exception we log internal queries also - if (log_queries && elem.type >= log_queries_min_type && Int64(elem.query_duration_ms) >= log_queries_min_query_duration_ms) + if (log_queries && elem.type >= log_queries_min_type && static_cast(elem.query_duration_ms) >= log_queries_min_query_duration_ms) { if (auto query_log = context->getQueryLog()) query_log->add(elem); diff --git a/src/Parsers/MySQL/ASTDeclareOption.cpp b/src/Parsers/MySQL/ASTDeclareOption.cpp index 17be639b630..740d049bafd 100644 --- a/src/Parsers/MySQL/ASTDeclareOption.cpp +++ b/src/Parsers/MySQL/ASTDeclareOption.cpp @@ -85,13 +85,13 @@ ASTPtr ASTDeclareOptions::clone() const bool ParserAlwaysTrue::parseImpl(IParser::Pos & /*pos*/, ASTPtr & node, Expected & /*expected*/) { - node = std::make_shared(Field(UInt64(1))); + node = std::make_shared(Field(static_cast(1))); return true; } bool ParserAlwaysFalse::parseImpl(IParser::Pos & /*pos*/, ASTPtr & node, Expected & /*expected*/) { - node = std::make_shared(Field(UInt64(0))); + node = std::make_shared(Field(static_cast(0))); return true; } diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 572edd93e39..4e9d5c1d57d 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -391,7 +391,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// Transform `DISTINCT ON expr` to `LIMIT 1 BY expr` limit_by_expression_list = distinct_on_expression_list; - limit_by_length = std::make_shared(Field{UInt8(1)}); + limit_by_length = std::make_shared(Field{static_cast(1)}); distinct_on_expression_list = nullptr; } diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 76d6a299857..5f69db633ac 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -30,9 +30,9 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p return false; if (ParserKeyword("TRUE").ignore(pos, expected)) - value = std::make_shared(Field(UInt64(1))); + value = std::make_shared(Field(static_cast(1))); else if (ParserKeyword("FALSE").ignore(pos, expected)) - value = std::make_shared(Field(UInt64(0))); + value = std::make_shared(Field(static_cast(0))); else if (!value_p.parse(pos, value, expected)) return false; diff --git a/src/Parsers/makeASTForLogicalFunction.cpp b/src/Parsers/makeASTForLogicalFunction.cpp index a1816dc9d17..fd9b78a5f52 100644 --- a/src/Parsers/makeASTForLogicalFunction.cpp +++ b/src/Parsers/makeASTForLogicalFunction.cpp @@ -22,9 +22,9 @@ ASTPtr makeASTForLogicalAnd(ASTs && arguments) }); if (!partial_result) - return std::make_shared(Field{UInt8(0)}); + return std::make_shared(Field{static_cast(0)}); if (arguments.empty()) - return std::make_shared(Field{UInt8(1)}); + return std::make_shared(Field{static_cast(1)}); if (arguments.size() == 1) return arguments[0]; @@ -51,9 +51,9 @@ ASTPtr makeASTForLogicalOr(ASTs && arguments) }); if (partial_result) - return std::make_shared(Field{UInt8(1)}); + return std::make_shared(Field{static_cast(1)}); if (arguments.empty()) - return std::make_shared(Field{UInt8(0)}); + return std::make_shared(Field{static_cast(0)}); if (arguments.size() == 1) return arguments[0]; diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index b6c9438a57c..5f39c7bd646 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -338,7 +338,7 @@ void registerFileSegmentationEngineTabSeparated(FormatFactory & factory) { auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - size_t min_rows = 1 + int(with_names) + int(with_types); + size_t min_rows = 1 + static_cast(with_names) + static_cast(with_types); factory.registerFileSegmentationEngine(format_name, [is_raw, min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { return fileSegmentationEngineTabSeparatedImpl(in, memory, min_chunk_size, is_raw, min_rows); diff --git a/src/Processors/Transforms/CubeTransform.cpp b/src/Processors/Transforms/CubeTransform.cpp index c64f39af5de..456eccc732f 100644 --- a/src/Processors/Transforms/CubeTransform.cpp +++ b/src/Processors/Transforms/CubeTransform.cpp @@ -45,7 +45,7 @@ Chunk CubeTransform::generate() consumed_chunks.clear(); auto num_rows = cube_chunk.getNumRows(); - mask = (UInt64(1) << keys.size()) - 1; + mask = (static_cast(1) << keys.size()) - 1; current_columns = cube_chunk.getColumns(); current_zero_columns.clear(); diff --git a/src/Processors/Transforms/PostgreSQLSource.cpp b/src/Processors/Transforms/PostgreSQLSource.cpp index a31cd879257..d8b0364ae45 100644 --- a/src/Processors/Transforms/PostgreSQLSource.cpp +++ b/src/Processors/Transforms/PostgreSQLSource.cpp @@ -86,7 +86,7 @@ void PostgreSQLSource::onStart() } } - stream = std::make_unique(*tx, pqxx::from_query, std::string_view(query_str)); + stream = std::make_unique(*tx, pqxx::from_query, std::string_view{query_str}); } template diff --git a/src/QueryPipeline/ExecutionSpeedLimits.cpp b/src/QueryPipeline/ExecutionSpeedLimits.cpp index 2738903cedb..551f4976df8 100644 --- a/src/QueryPipeline/ExecutionSpeedLimits.cpp +++ b/src/QueryPipeline/ExecutionSpeedLimits.cpp @@ -33,7 +33,7 @@ static void limitProgressingSpeed(size_t total_progress_size, size_t max_speed_i /// Never sleep more than one second (it should be enough to limit speed for a reasonable amount, /// and otherwise it's too easy to make query hang). - sleep_microseconds = std::min(UInt64(1000000), sleep_microseconds); + sleep_microseconds = std::min(static_cast(1000000), sleep_microseconds); sleepForMicroseconds(sleep_microseconds); diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index d833371a742..a512319413d 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -340,7 +340,7 @@ namespace uint64_t doubleToUInt64(double d) { - if (d >= double(std::numeric_limits::max())) + if (d >= static_cast(std::numeric_limits::max())) return std::numeric_limits::max(); return static_cast(d); } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index c656de61bfd..b22b453618b 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -267,7 +267,7 @@ Chunk MergeTreeBaseSelectProcessor::readFromPartImpl() }; UInt64 recommended_rows = estimate_num_rows(*task, task->range_reader); - UInt64 rows_to_read = std::max(UInt64(1), std::min(current_max_block_size_rows, recommended_rows)); + UInt64 rows_to_read = std::max(static_cast(1), std::min(current_max_block_size_rows, recommended_rows)); auto read_result = task->range_reader.read(rows_to_read, task->mark_ranges); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 29b3083c38f..9098f40793b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -943,7 +943,7 @@ void MergeTreeData::loadDataPartsFromDisk( const MergeTreeSettingsPtr & settings) { /// Parallel loading of data parts. - pool.setMaxThreads(std::min(size_t(settings->max_part_loading_threads), num_parts)); + pool.setMaxThreads(std::min(static_cast(settings->max_part_loading_threads), num_parts)); size_t num_threads = pool.getMaxThreads(); std::vector parts_per_thread(num_threads, num_parts / num_threads); for (size_t i = 0ul; i < num_parts % num_threads; ++i) @@ -3250,7 +3250,7 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const "Too many inactive parts ({}). Parts cleaning are processing significantly slower than inserts", inactive_parts_count_in_partition); } - k_inactive = ssize_t(inactive_parts_count_in_partition) - ssize_t(settings->inactive_parts_to_delay_insert); + k_inactive = static_cast(inactive_parts_count_in_partition) - static_cast(settings->inactive_parts_to_delay_insert); } if (parts_count_in_partition >= settings->parts_to_throw_insert) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 6caff7c683f..cc62d955d5f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -206,7 +206,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G writeIntBinary(plain_hashing.count(), marks); - writeIntBinary(UInt64(0), marks); + writeIntBinary(static_cast(0), marks); writeColumnSingleGranule( block.getByName(name_and_type->name), data_part->getSerialization(*name_and_type), @@ -246,9 +246,9 @@ void MergeTreeDataPartWriterCompact::fillDataChecksums(IMergeTreeDataPart::Check for (size_t i = 0; i < columns_list.size(); ++i) { writeIntBinary(plain_hashing.count(), marks); - writeIntBinary(UInt64(0), marks); + writeIntBinary(static_cast(0), marks); } - writeIntBinary(UInt64(0), marks); + writeIntBinary(static_cast(0), marks); } plain_file->next(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a6c78d3107d..61533f764d6 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1095,7 +1095,7 @@ std::shared_ptr MergeTreeDataSelectExecutor::checkLimits( std::set partitions; for (const auto & part_with_ranges : result.parts_with_ranges) partitions.insert(part_with_ranges.data_part->info.partition_id); - if (partitions.size() > size_t(max_partitions_to_read)) + if (partitions.size() > static_cast(max_partitions_to_read)) throw Exception( ErrorCodes::TOO_MANY_PARTITIONS, "Too many partitions to read. Current {}, max {}", diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index e3aa4ff82a5..58f775daf20 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -178,7 +178,7 @@ T MaterializedPostgreSQLConsumer::unhexN(const char * message, size_t pos, size_ for (size_t i = 0; i < n; ++i) { if (i) result <<= 8; - result |= UInt32(unhex2(message + pos + 2 * i)); + result |= static_cast(unhex2(message + pos + 2 * i)); } return result; } @@ -276,14 +276,14 @@ void MaterializedPostgreSQLConsumer::readTupleData( { case PostgreSQLQuery::INSERT: { - buffer.columns[num_columns]->insert(Int8(1)); + buffer.columns[num_columns]->insert(static_cast(1)); buffer.columns[num_columns + 1]->insert(lsn_value); break; } case PostgreSQLQuery::DELETE: { - buffer.columns[num_columns]->insert(Int8(-1)); + buffer.columns[num_columns]->insert(static_cast(-1)); buffer.columns[num_columns + 1]->insert(lsn_value); break; @@ -292,9 +292,9 @@ void MaterializedPostgreSQLConsumer::readTupleData( { /// Process old value in case changed value is a primary key. if (old_value) - buffer.columns[num_columns]->insert(Int8(-1)); + buffer.columns[num_columns]->insert(static_cast(-1)); else - buffer.columns[num_columns]->insert(Int8(1)); + buffer.columns[num_columns]->insert(static_cast(1)); buffer.columns[num_columns + 1]->insert(lsn_value); diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index a126478857b..93fb5cf2384 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -352,7 +352,7 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d ast_expression->name = "DateTime64"; ast_expression->arguments = std::make_shared(); - ast_expression->arguments->children.emplace_back(std::make_shared(UInt32(6))); + ast_expression->arguments->children.emplace_back(std::make_shared(static_cast(6))); return ast_expression; } diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.cpp b/src/Storages/ReadFinalForExternalReplicaStorage.cpp index a03ccb5cf43..6befde7609f 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.cpp +++ b/src/Storages/ReadFinalForExternalReplicaStorage.cpp @@ -48,7 +48,7 @@ Pipe readFinalFromNestedStorage( require_columns_name.emplace_back(sign_column.name); const auto & sign_column_name = std::make_shared(sign_column.name); - const auto & fetch_sign_value = std::make_shared(Field(Int8(1))); + const auto & fetch_sign_value = std::make_shared(Field(static_cast(1))); expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value)); filter_column_name = expressions->children.back()->getColumnName(); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 999a1d1bb10..ae67a67af7f 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1376,7 +1376,7 @@ void StorageDistributed::delayInsertOrThrowIfNeeded() const { /// Step is 5% of the delay and minimal one second. /// NOTE: max_delay_to_insert is in seconds, and step is in ms. - const size_t step_ms = std::min(1., double(distributed_settings.max_delay_to_insert) * 1'000 * 0.05); + const size_t step_ms = std::min(1., static_cast(distributed_settings.max_delay_to_insert) * 1'000 * 0.05); UInt64 delayed_ms = 0; do { diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index a2f25a9db7e..72f93882194 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -296,7 +296,7 @@ Pipe StorageMerge::read( size_t tables_count = selected_tables.size(); Float64 num_streams_multiplier - = std::min(unsigned(tables_count), std::max(1U, unsigned(local_context->getSettingsRef().max_streams_multiplier_for_merge_tables))); + = std::min(static_cast(tables_count), std::max(1U, static_cast(local_context->getSettingsRef().max_streams_multiplier_for_merge_tables))); num_streams *= num_streams_multiplier; size_t remaining_streams = num_streams; @@ -327,7 +327,7 @@ Pipe StorageMerge::read( size_t current_need_streams = tables_count >= num_streams ? 1 : (num_streams / tables_count); size_t current_streams = std::min(current_need_streams, remaining_streams); remaining_streams -= current_streams; - current_streams = std::max(size_t(1), current_streams); + current_streams = std::max(static_cast(1), current_streams); const auto & storage = std::get<1>(table);