diff --git a/.clang-tidy b/.clang-tidy index 5e931d24c91..0400b500e5c 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -34,7 +34,7 @@ Checks: '-*, performance-trivially-destructible, performance-unnecessary-copy-initialization, performance-noexcept-move-constructor, - # performance-move-const-arg, + performance-move-const-arg, readability-avoid-const-params-in-decls, readability-const-return-type, diff --git a/base/base/insertAtEnd.h b/base/base/insertAtEnd.h index c4fef664511..abb2aa7d563 100644 --- a/base/base/insertAtEnd.h +++ b/base/base/insertAtEnd.h @@ -26,3 +26,27 @@ void insertAtEnd(std::vector & dest, std::vector && src) dest.insert(dest.end(), std::make_move_iterator(src.begin()), std::make_move_iterator(src.end())); src.clear(); } + +template +void insertAtEnd(Container & dest, const Container & src) +{ + if (src.empty()) + return; + + dest.insert(dest.end(), src.begin(), src.end()); +} + +template +void insertAtEnd(Container & dest, Container && src) +{ + if (src.empty()) + return; + if (dest.empty()) + { + dest.swap(src); + return; + } + + dest.insert(dest.end(), std::make_move_iterator(src.begin()), std::make_move_iterator(src.end())); + src.clear(); +} diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 3385834e386..60cd24eddfb 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -165,7 +165,7 @@ std::variant ConnectionEstablisherAsync:: fiber = std::move(fiber).resume(); if (exception) - std::rethrow_exception(std::move(exception)); + std::rethrow_exception(exception); if (connection_establisher.isFinished()) { diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 791ac4c1ef1..9c3c20b4d02 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -363,7 +363,7 @@ bool HedgedConnections::resumePacketReceiver(const HedgedConnections::ReplicaLoc else if (std::holds_alternative(res)) { finishProcessReplica(replica_state, true); - std::rethrow_exception(std::move(std::get(res))); + std::rethrow_exception(std::get(res)); } return false; diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 8ee4b9e1c1f..021c68271a0 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -161,11 +161,11 @@ void LocalConnection::sendData(const Block & block, const String &, bool) if (state->pushing_async_executor) { - state->pushing_async_executor->push(std::move(block)); + state->pushing_async_executor->push(block); } else if (state->pushing_executor) { - state->pushing_executor->push(std::move(block)); + state->pushing_executor->push(block); } } diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 7a4ce8288a3..bc117b065b7 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -541,7 +541,7 @@ ColumnPtr ColumnNullable::compress() const size_t byte_size = nested_column->byteSize() + null_map->byteSize(); return ColumnCompressed::create(size(), byte_size, - [nested_column = std::move(nested_column), null_map = std::move(null_map)] + [nested_column = std::move(nested_compressed), null_map = std::move(null_map_compressed)] { return ColumnNullable::create(nested_column->decompress(), null_map->decompress()); }); diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index e9bdc3971c0..611c6fd7186 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -330,7 +330,7 @@ ColumnPtr ColumnSparse::filter(const Filter & filt, ssize_t) const } auto res_values = values->filter(values_filter, values_result_size_hint); - return this->create(std::move(res_values), std::move(res_offsets), res_offset); + return this->create(res_values, std::move(res_offsets), res_offset); } void ColumnSparse::expand(const Filter & mask, bool inverted) diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index df81d939267..6d7ef9cd34d 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -100,6 +100,11 @@ public: return emplace(x); } + [[nodiscard]] bool push(T && x) + { + return emplace(std::move(x)); + } + /// Returns false if queue is finished template [[nodiscard]] bool emplace(Args &&... args) diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index f0d234e5cc5..d90f6064292 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -316,7 +316,7 @@ CodecTestSequence operator+(CodecTestSequence && left, const CodecTestSequence & std::vector operator+(const std::vector & left, const std::vector & right) { - std::vector result(std::move(left)); + std::vector result(left); std::move(std::begin(right), std::end(right), std::back_inserter(result)); return result; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 1e6081a628f..e13b43d056a 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -317,7 +317,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr created_node.data = request.data; created_node.is_sequental = request.is_sequential; - auto [map_key, _] = container.insert(path_created, std::move(created_node)); + auto [map_key, _] = container.insert(path_created, created_node); /// Take child path from key owned by map. auto child_path = getBaseName(map_key->getKey()); diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 26c883b308d..f7d6761124f 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -122,6 +122,11 @@ Block::Block(const ColumnsWithTypeAndName & data_) : data{data_} initializeIndexByName(); } +Block::Block(ColumnsWithTypeAndName && data_) : data{std::move(data_)} +{ + initializeIndexByName(); +} + void Block::initializeIndexByName() { diff --git a/src/Core/Block.h b/src/Core/Block.h index 2624b57880c..2a1e6113b73 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -37,6 +37,7 @@ public: Block() = default; Block(std::initializer_list il); Block(const ColumnsWithTypeAndName & data_); + Block(ColumnsWithTypeAndName && data_); /// insert the column at the specified position void insert(size_t position, ColumnWithTypeAndName elem); diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.cpp b/src/DataTypes/Serializations/SerializationInfoTuple.cpp index 378bed2af53..803302f9642 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.cpp +++ b/src/DataTypes/Serializations/SerializationInfoTuple.cpp @@ -89,7 +89,7 @@ Poco::JSON::Object SerializationInfoTuple::toJSON() const for (const auto & elem : elems) subcolumns.add(elem->toJSON()); - object.set("subcolumns", std::move(subcolumns)); + object.set("subcolumns", subcolumns); return object; } diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 324f4808b1f..adfcd83f5a7 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -37,7 +37,7 @@ public: }; DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, ContextPtr context_) - : DatabaseOrdinary(name_, std::move(metadata_path_), "store/", logger_name, context_) + : DatabaseOrdinary(name_, metadata_path_, "store/", logger_name, context_) , path_to_table_symlinks(fs::path(getContext()->getPath()) / "data" / escapeForFileName(name_) / "") , path_to_metadata_symlink(fs::path(getContext()->getPath()) / "metadata" / escapeForFileName(name_)) , db_uuid(uuid) diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index cc6d808a564..5f4027a26b3 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -61,7 +61,7 @@ DatabaseMySQL::DatabaseMySQL( , database_engine_define(database_engine_define_->clone()) , database_name_in_mysql(database_name_in_mysql_) , database_settings(std::move(settings_)) - , mysql_pool(std::move(pool)) + , mysql_pool(std::move(pool)) /// NOLINT { try { diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 0facdfc20be..6a8184f6b66 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -253,7 +253,7 @@ void MaterializeMetadata::transaction(const MySQLReplication::Position & positio out.close(); } - commitMetadata(std::move(fun), persistent_tmp_path, persistent_path); + commitMetadata(fun, persistent_tmp_path, persistent_path); } MaterializeMetadata::MaterializeMetadata(const String & path_, const Settings & settings_) : persistent_path(path_), settings(settings_) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 8033d65c549..980565a0035 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -159,7 +159,7 @@ MaterializedMySQLSyncThread::MaterializedMySQLSyncThread( , log(&Poco::Logger::get("MaterializedMySQLSyncThread")) , database_name(database_name_) , mysql_database_name(mysql_database_name_) - , pool(std::move(pool_)) + , pool(std::move(pool_)) /// NOLINT , client(std::move(client_)) , settings(settings_) { diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index cad3e3b8799..8b8d0a57cc7 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -494,7 +494,7 @@ Pipe CacheDictionary::read(const Names & column_names, size { auto keys = cache_storage_ptr->getCachedSimpleKeys(); auto keys_column = getColumnFromPODArray(std::move(keys)); - key_columns = {ColumnWithTypeAndName(std::move(keys_column), std::make_shared(), dict_struct.id->name)}; + key_columns = {ColumnWithTypeAndName(keys_column, std::make_shared(), dict_struct.id->name)}; } else { diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index cd87cf831a2..fcad8398c0b 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -52,7 +52,7 @@ Block blockForKeys( auto filtered_column = source_column->filter(filter, requested_rows.size()); - block.insert({std::move(filtered_column), (*dict_struct.key)[i].type, (*dict_struct.key)[i].name}); + block.insert({filtered_column, (*dict_struct.key)[i].type, (*dict_struct.key)[i].name}); } return block; diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 0c82da7b73b..26144821a0e 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -147,7 +147,7 @@ ColumnPtr FlatDictionary::getColumn( callOnDictionaryAttributeType(attribute.type, type_call); if (attribute.is_nullable_set) - result = ColumnNullable::create(std::move(result), std::move(col_null_map_to)); + result = ColumnNullable::create(result, std::move(col_null_map_to)); return result; } @@ -572,7 +572,7 @@ Pipe FlatDictionary::read(const Names & column_names, size_t max_block_size, siz keys.push_back(key_index); auto keys_column = getColumnFromPODArray(std::move(keys)); - ColumnsWithTypeAndName key_columns = {ColumnWithTypeAndName(std::move(keys_column), std::make_shared(), dict_struct.id->name)}; + ColumnsWithTypeAndName key_columns = {ColumnWithTypeAndName(keys_column, std::make_shared(), dict_struct.id->name)}; std::shared_ptr dictionary = shared_from_this(); auto coordinator = DictionarySourceCoordinator::create(dictionary, column_names, std::move(key_columns), max_block_size); diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index ea041c63d73..65d9b3e7d42 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -578,7 +578,7 @@ ColumnPtr HashedArrayDictionary::getAttributeColumn( callOnDictionaryAttributeType(attribute.type, type_call); if (is_attribute_nullable) - result = ColumnNullable::create(std::move(result), std::move(col_null_map_to)); + result = ColumnNullable::create(result, std::move(col_null_map_to)); return result; } diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index b70f018df6b..178631d9c53 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -159,7 +159,7 @@ ColumnPtr HashedDictionary::getColumn( callOnDictionaryAttributeType(attribute.type, type_call); if (is_attribute_nullable) - result = ColumnNullable::create(std::move(result), std::move(col_null_map_to)); + result = ColumnNullable::create(result, std::move(col_null_map_to)); return result; } diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index deec1e6a588..1a4e01d4aa3 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -61,7 +61,7 @@ void IPolygonDictionary::convertKeyColumns(Columns & key_columns, DataTypes & ke auto & key_column_to_cast = key_columns[key_type_index]; ColumnWithTypeAndName column_to_cast = {key_column_to_cast, key_type, ""}; - auto casted_column = castColumnAccurate(std::move(column_to_cast), float_64_type); + auto casted_column = castColumnAccurate(column_to_cast, float_64_type); key_column_to_cast = std::move(casted_column); key_type = float_64_type; } diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 5330bc684c3..e82fcd580e2 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -198,7 +198,7 @@ ColumnPtr RangeHashedDictionary::getColumn( callOnDictionaryAttributeType(attribute.type, type_call); if (is_attribute_nullable) - result = ColumnNullable::create(std::move(result), std::move(col_null_map_to)); + result = ColumnNullable::create(result, std::move(col_null_map_to)); return result; } @@ -298,7 +298,7 @@ ColumnPtr RangeHashedDictionary::getColumnInternal( callOnDictionaryAttributeType(attribute.type, type_call); if (is_attribute_nullable) - result = ColumnNullable::create(std::move(result), std::move(col_null_map_to)); + result = ColumnNullable::create(result, std::move(col_null_map_to)); return result; } diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index a1b406b3424..fd381ab2921 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -136,9 +136,9 @@ namespace DB RedisArray keys; auto key_type = storageTypeToKeyType(configuration.storage_type); - for (const auto & key : all_keys) + for (auto && key : all_keys) if (key_type == connection->client->execute(RedisCommand("TYPE").addRedisType(key))) - keys.addRedisType(std::move(key)); + keys.addRedisType(key); if (configuration.storage_type == RedisStorageType::HASH_MAP) { @@ -165,10 +165,10 @@ namespace DB } if (primary_with_secondary.size() > 1) - hkeys.add(std::move(primary_with_secondary)); + hkeys.add(primary_with_secondary); } - keys = std::move(hkeys); + keys = hkeys; } return Pipe(std::make_shared( diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index be565a532bb..3fea8d3eb7b 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -278,9 +278,10 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( if (settings.output_format_parallel_formatting && getCreators(name).supports_parallel_formatting && !settings.output_format_json_array_of_rows) { - auto formatter_creator = [output_getter, sample, callback, format_settings] - (WriteBuffer & output) -> OutputFormatPtr - { return output_getter(output, sample, {std::move(callback)}, format_settings);}; + auto formatter_creator = [output_getter, sample, callback, format_settings] (WriteBuffer & output) -> OutputFormatPtr + { + return output_getter(output, sample, {callback}, format_settings); + }; ParallelFormattingOutputFormat::Params builder{buf, sample, formatter_creator, settings.max_threads}; diff --git a/src/Functions/array/mapPopulateSeries.cpp b/src/Functions/array/mapPopulateSeries.cpp index 17269f8dfe1..8b4a1dda197 100644 --- a/src/Functions/array/mapPopulateSeries.cpp +++ b/src/Functions/array/mapPopulateSeries.cpp @@ -379,8 +379,7 @@ private: if (!max_key_column_type->equals(*input.key_series_type)) { ColumnWithTypeAndName column_to_cast = {max_key_column, max_key_column_type, ""}; - auto casted_column = castColumnAccurate(std::move(column_to_cast), input.key_series_type); - max_key_column = std::move(casted_column); + max_key_column = castColumnAccurate(column_to_cast, input.key_series_type); } } diff --git a/src/Functions/castOrDefault.cpp b/src/Functions/castOrDefault.cpp index 95046d95176..628ac57f34d 100644 --- a/src/Functions/castOrDefault.cpp +++ b/src/Functions/castOrDefault.cpp @@ -99,7 +99,7 @@ public: { const ColumnWithTypeAndName & column_to_cast = arguments[0]; auto non_const_column_to_cast = column_to_cast.column->convertToFullColumnIfConst(); - ColumnWithTypeAndName column_to_cast_non_const { std::move(non_const_column_to_cast), column_to_cast.type, column_to_cast.name }; + ColumnWithTypeAndName column_to_cast_non_const { non_const_column_to_cast, column_to_cast.type, column_to_cast.name }; auto cast_result = castColumnAccurateOrNull(column_to_cast_non_const, return_type); diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 6841098ebcf..0b30f404f8e 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -1027,7 +1027,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count) const override { - ColumnsWithTypeAndName arguments = std::move(args); + ColumnsWithTypeAndName arguments = args; executeShortCircuitArguments(arguments); ColumnPtr res; if ( (res = executeForConstAndNullableCondition(arguments, result_type, input_rows_count)) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 070a7c2f05e..7ed0ee00954 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -117,7 +117,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count) const override { - ColumnsWithTypeAndName arguments = std::move(args); + ColumnsWithTypeAndName arguments = args; executeShortCircuitArguments(arguments); /** We will gather values from columns in branches to result column, * depending on values of conditions. diff --git a/src/Functions/now64.cpp b/src/Functions/now64.cpp index bd1038b1fc6..e7d9011db53 100644 --- a/src/Functions/now64.cpp +++ b/src/Functions/now64.cpp @@ -152,7 +152,7 @@ public: for (const auto & arg : arguments) arg_types.push_back(arg.type); - return std::make_unique(nowSubsecond(scale), std::move(arg_types), std::move(result_type)); + return std::make_unique(nowSubsecond(scale), std::move(arg_types), result_type); } }; diff --git a/src/Functions/nullIf.cpp b/src/Functions/nullIf.cpp index c54bbc08bcd..0b4d024c91c 100644 --- a/src/Functions/nullIf.cpp +++ b/src/Functions/nullIf.cpp @@ -61,7 +61,7 @@ public: auto func_if = FunctionFactory::instance().get("if", context)->build(if_columns); auto if_res = func_if->execute(if_columns, result_type, input_rows_count); - return makeNullable(std::move(if_res)); + return makeNullable(if_res); } }; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index f0524eed6b5..ec7b1772a78 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -386,7 +386,7 @@ void WriteBufferFromS3::waitForReadyBackGroundTasks() while (!upload_object_tasks.empty() && upload_object_tasks.front().is_finised) { auto & task = upload_object_tasks.front(); - auto exception = std::move(task.exception); + auto exception = task.exception; auto tag = std::move(task.tag); upload_object_tasks.pop_front(); diff --git a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp index e16ee03c711..d1d8ee63b8e 100644 --- a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include namespace DB @@ -76,11 +76,11 @@ ASTs InterpreterShowAccessQuery::getCreateAndGrantQueries() const { create_queries.push_back(InterpreterShowCreateAccessEntityQuery::getCreateQuery(*entity, access_control)); if (entity->isTypeOf(AccessEntityType::USER) || entity->isTypeOf(AccessEntityType::ROLE)) - boost::range::push_back(grant_queries, InterpreterShowGrantsQuery::getGrantQueries(*entity, access_control)); + insertAtEnd(grant_queries, InterpreterShowGrantsQuery::getGrantQueries(*entity, access_control)); } ASTs result = std::move(create_queries); - boost::range::push_back(result, std::move(grant_queries)); + insertAtEnd(result, std::move(grant_queries)); return result; } diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index a5122cd54c7..25116f5145a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -517,7 +517,7 @@ Block ActionsDAG::updateHeader(Block header) const { auto & list = it->second; pos_to_remove.insert(pos); - node_to_column[inputs[list.front()]] = std::move(col); + node_to_column[inputs[list.front()]] = col; list.pop_front(); } } @@ -590,7 +590,7 @@ Block ActionsDAG::updateHeader(Block header) const for (auto & col : result_columns) res.insert(std::move(col)); - for (const auto & item : header) + for (auto && item : header) res.insert(std::move(item)); return res; @@ -651,8 +651,8 @@ NameSet ActionsDAG::foldActionsByProjection( { /// Projection folding. node->type = ActionsDAG::ActionType::INPUT; - node->result_type = std::move(column_with_type_name->type); - node->result_name = std::move(column_with_type_name->name); + node->result_type = column_with_type_name->type; + node->result_name = column_with_type_name->name; node->children.clear(); inputs.push_back(node); } @@ -724,7 +724,7 @@ void ActionsDAG::addAliases(const NamesWithAliases & aliases) Node node; node.type = ActionType::ALIAS; node.result_type = child->result_type; - node.result_name = std::move(item.second); + node.result_name = item.second; node.column = child->column; node.children.emplace_back(child); @@ -771,7 +771,7 @@ void ActionsDAG::project(const NamesWithAliases & projection) Node node; node.type = ActionType::ALIAS; node.result_type = child->result_type; - node.result_name = std::move(item.second); + node.result_name = item.second; node.column = child->column; node.children.emplace_back(child); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 90ea5fdcbb2..99583c41b64 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -120,7 +120,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data if (i == tuple_size) for (i = 0; i < tuple_size; ++i) - columns[i]->insert(std::move(tuple_values[i])); + columns[i]->insert(tuple_values[i]); } } diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 9f300c36b81..c60ab0f6510 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -165,9 +165,9 @@ void AsynchronousInsertQueue::scheduleDataProcessingJob(const InsertQuery & key, { /// Wrap 'unique_ptr' with 'shared_ptr' to make this /// lambda copyable and allow to save it to the thread pool. - pool.scheduleOrThrowOnError([=, data = std::make_shared(std::move(data))] + pool.scheduleOrThrowOnError([key, global_context, data = std::make_shared(std::move(data))]() mutable { - processData(std::move(key), std::move(*data), std::move(global_context)); + processData(key, std::move(*data), std::move(global_context)); }); } diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 30c832e4917..83f8de78fa6 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -748,7 +748,7 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) if (execution_context.columns[pos].column) res.insert(execution_context.columns[pos]); - for (const auto & item : block) + for (auto && item : block) res.insert(std::move(item)); block.swap(res); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 2e464053131..7e150f59694 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1367,7 +1367,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain auto child_name = child->getColumnName(); if (!aggregated_names.count(child_name)) - step.addRequiredOutput(std::move(child_name)); + step.addRequiredOutput(child_name); } return true; diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index aab3a9e7437..a75cdce820c 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -56,7 +56,7 @@ namespace static_assert(std::is_same_v); ExternalLoader::Loadables objects; objects.reserve(results.size()); - for (const auto & result : results) + for (auto && result : results) { if (auto object = std::move(result.object)) objects.push_back(std::move(object)); diff --git a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp index 31b4d4a621c..e3d40033cff 100644 --- a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -139,7 +139,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create UserDefinedExecutableFunctionConfiguration function_configuration { - .name = std::move(name), + .name = name, .command = std::move(command_value), .command_arguments = std::move(command_arguments), .arguments = std::move(arguments), diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 9ccf4ce7356..df44814a96e 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -197,7 +197,7 @@ Chain InterpreterInsertQuery::buildChain( std::atomic_uint64_t * elapsed_counter_ms) { auto sample = getSampleBlock(columns, table, metadata_snapshot); - return buildChainImpl(table, metadata_snapshot, std::move(sample) , thread_status, elapsed_counter_ms); + return buildChainImpl(table, metadata_snapshot, sample, thread_status, elapsed_counter_ms); } Chain InterpreterInsertQuery::buildChainImpl( diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 2b949266c17..5ec6abb08a7 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -133,7 +133,7 @@ public: , process_list(process_list_) , processes_to_stop(std::move(processes_to_stop_)) , processes_block(std::move(processes_block_)) - , res_sample_block(std::move(res_sample_block_)) + , res_sample_block(res_sample_block_) { addTotalRowsApprox(processes_to_stop.size()); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5c096a7dbac..6271e8711c4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -549,7 +549,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Reuse already built sets for multiple passes of analysis subquery_for_sets = std::move(query_analyzer->getSubqueriesForSets()); - prepared_sets = query_info.sets.empty() ? std::move(query_analyzer->getPreparedSets()) : std::move(query_info.sets); + prepared_sets = query_info.sets.empty() ? query_analyzer->getPreparedSets() : query_info.sets; /// Do not try move conditions to PREWHERE for the second time. /// Otherwise, we won't be able to fallback from inefficient PREWHERE to WHERE later. diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index ed20b1b2048..d6a00ba89b4 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -576,7 +576,7 @@ std::shared_ptr subqueryExpressionList( needed_columns[table_pos].fillExpressionList(*expression_list); for (const auto & expr : alias_pushdown[table_pos]) - expression_list->children.emplace_back(std::move(expr)); + expression_list->children.emplace_back(expr); return expression_list; } diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index d9698be1a9b..a0c29c07d38 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -77,7 +77,7 @@ SessionLogElement::SessionLogElement(const UUID & auth_id_, Type type_) NamesAndTypesList SessionLogElement::getNamesAndTypes() { - const auto event_type = std::make_shared( + auto event_type = std::make_shared( DataTypeEnum8::Values { {"LoginFailure", static_cast(SESSION_LOGIN_FAILURE)}, @@ -86,7 +86,7 @@ NamesAndTypesList SessionLogElement::getNamesAndTypes() }); #define AUTH_TYPE_NAME_AND_VALUE(v) std::make_pair(AuthenticationTypeInfo::get(v).raw_name, static_cast(v)) - const auto identified_with_column = std::make_shared( + auto identified_with_column = std::make_shared( DataTypeEnum8::Values { AUTH_TYPE_NAME_AND_VALUE(AuthType::NO_PASSWORD), @@ -98,7 +98,7 @@ NamesAndTypesList SessionLogElement::getNamesAndTypes() }); #undef AUTH_TYPE_NAME_AND_VALUE - const auto interface_type_column = std::make_shared( + auto interface_type_column = std::make_shared( DataTypeEnum8::Values { {"TCP", static_cast(Interface::TCP)}, @@ -108,9 +108,9 @@ NamesAndTypesList SessionLogElement::getNamesAndTypes() {"PostgreSQL", static_cast(Interface::POSTGRESQL)} }); - const auto lc_string_datatype = std::make_shared(std::make_shared()); + auto lc_string_datatype = std::make_shared(std::make_shared()); - const auto settings_type_column = std::make_shared( + auto settings_type_column = std::make_shared( std::make_shared( DataTypes({ // setting name diff --git a/src/Interpreters/addMissingDefaults.cpp b/src/Interpreters/addMissingDefaults.cpp index 04e1e6856cc..d043fd16bb5 100644 --- a/src/Interpreters/addMissingDefaults.cpp +++ b/src/Interpreters/addMissingDefaults.cpp @@ -63,7 +63,7 @@ ActionsDAGPtr addMissingDefaults( { const auto & nested_type = array_type->getNestedType(); ColumnPtr nested_column = nested_type->createColumnConstWithDefaultValue(0); - const auto & constant = actions->addColumn({std::move(nested_column), nested_type, column.name}); + const auto & constant = actions->addColumn({nested_column, nested_type, column.name}); auto & group = nested_groups[offsets_name]; group[0] = &constant; @@ -76,7 +76,7 @@ ActionsDAGPtr addMissingDefaults( * it can be full (or the interpreter may decide that it is constant everywhere). */ auto new_column = column.type->createColumnConstWithDefaultValue(0); - const auto * col = &actions->addColumn({std::move(new_column), column.type, column.name}); + const auto * col = &actions->addColumn({new_column, column.type, column.name}); index.push_back(&actions->materializeNode(*col)); } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index c17e1173b20..c1606700540 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -781,8 +781,8 @@ static std::tuple executeQueryImpl( element.memory_usage = info.peak_memory_usage > 0 ? info.peak_memory_usage : 0; - element.thread_ids = std::move(info.thread_ids); - element.profile_counters = std::move(info.profile_counters); + element.thread_ids = info.thread_ids; + element.profile_counters = info.profile_counters; /// We need to refresh the access info since dependent views might have added extra information, either during /// creation of the view (PushingToViewsBlockOutputStream) or while executing its internal SELECT diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 69b7b7d833f..31913777902 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -57,7 +57,7 @@ bool removeJoin(ASTSelectQuery & select, TreeRewriterResult & rewriter_result, C const size_t left_table_pos = 0; /// Test each argument of `and` function and select ones related to only left table std::shared_ptr new_conj = makeASTFunction("and"); - for (const auto & node : collectConjunctions(where)) + for (auto && node : collectConjunctions(where)) { if (membership_collector.getIdentsMembership(node) == left_table_pos) new_conj->arguments->children.push_back(std::move(node)); diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 198d5ce5d8d..0f091e73743 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -35,7 +35,7 @@ struct PullingAsyncPipelineExecutor::Data if (has_exception) { has_exception = false; - std::rethrow_exception(std::move(exception)); + std::rethrow_exception(exception); } } }; diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 6c2e62b77dc..07cdb554aba 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -90,7 +90,7 @@ struct PushingAsyncPipelineExecutor::Data if (has_exception) { has_exception = false; - std::rethrow_exception(std::move(exception)); + std::rethrow_exception(exception); } } }; diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 558ba9bdd65..cf5cfa681a1 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -114,7 +114,7 @@ static std::shared_ptr createFileReader(ReadB if (is_stopped) return nullptr; - auto file_reader_status = arrow::ipc::RecordBatchFileReader::Open(std::move(arrow_file)); + auto file_reader_status = arrow::ipc::RecordBatchFileReader::Open(arrow_file); if (!file_reader_status.ok()) throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while opening a table: {}", file_reader_status.status().ToString()); diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 102c30088c9..ecaa485c3d6 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -241,7 +241,7 @@ static ColumnWithTypeAndName readColumnWithDecimalDataImpl(std::shared_ptr(chunk.Value(value_i))); // TODO: copy column } } - return {std::move(internal_column), std::move(internal_type), column_name}; + return {std::move(internal_column), internal_type, column_name}; } template @@ -337,7 +337,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values, read_ints_as_dates); auto nullmap_column = readByteMapFromArrowColumn(arrow_column); auto nullable_type = std::make_shared(std::move(nested_column.type)); - auto nullable_column = ColumnNullable::create(std::move(nested_column.column), std::move(nullmap_column)); + auto nullable_column = ColumnNullable::create(nested_column.column, nullmap_column); return {std::move(nullable_column), std::move(nullable_type), column_name}; } @@ -384,7 +384,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( const auto * tuple_column = assert_cast(nested_column.column.get()); const auto * tuple_type = assert_cast(nested_column.type.get()); - auto map_column = ColumnMap::create(std::move(tuple_column->getColumnPtr(0)), std::move(tuple_column->getColumnPtr(1)), std::move(offsets_column)); + auto map_column = ColumnMap::create(tuple_column->getColumnPtr(0), tuple_column->getColumnPtr(1), offsets_column); auto map_type = std::make_shared(tuple_type->getElements()[0], tuple_type->getElements()[1]); return {std::move(map_column), std::move(map_type), column_name}; } @@ -393,7 +393,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( auto arrow_nested_column = getNestedArrowColumn(arrow_column); auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates); auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); - auto array_column = ColumnArray::create(std::move(nested_column.column), std::move(offsets_column)); + auto array_column = ColumnArray::create(nested_column.column, offsets_column); auto array_type = std::make_shared(nested_column.type); return {std::move(array_column), std::move(array_type), column_name}; } @@ -458,7 +458,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( auto arrow_indexes_column = std::make_shared(indexes_array); auto indexes_column = readColumnWithIndexesData(arrow_indexes_column); - auto lc_column = ColumnLowCardinality::create(dict_values->column, std::move(indexes_column)); + auto lc_column = ColumnLowCardinality::create(dict_values->column, indexes_column); auto lc_type = std::make_shared(dict_values->type); return {std::move(lc_column), std::move(lc_type), column_name}; } diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index bb202a3e177..6918220feb4 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -15,9 +15,9 @@ namespace ErrorCodes BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) : RowInputFormatWithNamesAndTypes( - std::move(header), + header, in_, - std::move(params_), + params_, with_names_, with_types_, format_settings_, diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index b56a9c2729f..043e4f1e724 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -658,7 +658,7 @@ namespace DB auto nested_arrow_type = getArrowType(nested_types[i], tuple_column->getColumnPtr(i), name, format_name, out_is_column_nullable); nested_fields.push_back(std::make_shared(name, nested_arrow_type, *out_is_column_nullable)); } - return arrow::struct_(std::move(nested_fields)); + return arrow::struct_(nested_fields); } if (column_type->lowCardinality()) diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index 58f88c5c7cf..fd33abfb587 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -169,7 +169,7 @@ static std::optional convertToDynamicValue( auto value_builder = initStructFieldBuilder(nested_column, row_num, struct_builder, value_field); auto value = convertToDynamicValue(nested_column, nullable_type->getNestedType(), row_num, value_builder, enum_comparing_mode, temporary_text_data_storage); if (value) - struct_builder.set(value_field, std::move(*value)); + struct_builder.set(value_field, *value); } } else @@ -184,7 +184,7 @@ static std::optional convertToDynamicValue( = initStructFieldBuilder(nested_columns[pos], row_num, struct_builder, nested_struct_schema.getFieldByName(name)); auto value = convertToDynamicValue(nested_columns[pos], nested_types[pos], row_num, field_builder, enum_comparing_mode, temporary_text_data_storage); if (value) - struct_builder.set(name, std::move(*value)); + struct_builder.set(name, *value); } } return std::nullopt; @@ -215,7 +215,7 @@ static std::optional convertToDynamicValue( auto value = convertToDynamicValue(nested_column, nested_type, offset + i, value_builder, enum_comparing_mode, temporary_text_data_storage); if (value) - list_builder.set(i, std::move(*value)); + list_builder.set(i, *value); } return std::nullopt; } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index dcab55743cb..c087749d8d8 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -27,7 +27,7 @@ JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat( : RowInputFormatWithNamesAndTypes( header_, in_, - std::move(params_), + params_, with_names_, with_types_, format_settings_, diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 61511d634d3..4950e1fb952 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -111,7 +111,7 @@ static void getFileReaderAndSchema( if (is_stopped) return; - auto result = arrow::adapters::orc::ORCFileReader::Open(std::move(arrow_file), arrow::default_memory_pool()); + auto result = arrow::adapters::orc::ORCFileReader::Open(arrow_file, arrow::default_memory_pool()); if (!result.ok()) throw Exception(result.status().ToString(), ErrorCodes::BAD_ARGUMENTS); file_reader = std::move(result).ValueOrDie(); diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 72ad4616174..0247b8677af 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -457,7 +457,7 @@ static void postprocessChunk( { const auto & from_type = desc.nested_type; const auto & to_type = desc.real_type; - res_columns[desc.column_numbers[0]] = recursiveTypeConversion(std::move(column), from_type, to_type); + res_columns[desc.column_numbers[0]] = recursiveTypeConversion(column, from_type, to_type); } else res_columns[desc.column_numbers[0]] = std::move(column); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 45c3719ebca..6b6f9d361ef 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -138,7 +138,7 @@ void addCreatingSetsStep( auto creating_set = std::make_unique( plan->getCurrentDataStream(), - std::move(description), + description, std::move(set), limits, context); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 9a9a71f9688..ad4d1ea86d6 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -982,7 +982,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const { auto result_ptr = analyzed_result_ptr ? analyzed_result_ptr : selectRangesToRead(prepared_parts); if (std::holds_alternative(result_ptr->result)) - std::rethrow_exception(std::move(std::get(result_ptr->result))); + std::rethrow_exception(std::get(result_ptr->result)); return std::get(result_ptr->result); } @@ -1326,7 +1326,7 @@ bool MergeTreeDataSelectAnalysisResult::error() const size_t MergeTreeDataSelectAnalysisResult::marks() const { if (std::holds_alternative(result)) - std::rethrow_exception(std::move(std::get(result))); + std::rethrow_exception(std::get(result)); const auto & index_stats = std::get(result).index_stats; if (index_stats.empty()) diff --git a/src/Processors/Transforms/DistinctSortedTransform.cpp b/src/Processors/Transforms/DistinctSortedTransform.cpp index 01cef654388..5600476fd77 100644 --- a/src/Processors/Transforms/DistinctSortedTransform.cpp +++ b/src/Processors/Transforms/DistinctSortedTransform.cpp @@ -24,7 +24,7 @@ void DistinctSortedTransform::transform(Chunk & chunk) if (column_ptrs.empty()) return; - const ColumnRawPtrs clearing_hint_columns(getClearingColumns(chunk, column_ptrs)); + ColumnRawPtrs clearing_hint_columns(getClearingColumns(chunk, column_ptrs)); if (data.type == ClearableSetVariants::Type::EMPTY) data.init(ClearableSetVariants::chooseMethod(column_ptrs, key_sizes)); diff --git a/src/Processors/Transforms/ExceptionKeepingTransform.cpp b/src/Processors/Transforms/ExceptionKeepingTransform.cpp index f2b29a45f84..266407f21a5 100644 --- a/src/Processors/Transforms/ExceptionKeepingTransform.cpp +++ b/src/Processors/Transforms/ExceptionKeepingTransform.cpp @@ -138,7 +138,7 @@ void ExceptionKeepingTransform::work() { stage = Stage::Exception; ready_output = true; - data.exception = std::move(exception); + data.exception = exception; onException(); } } @@ -152,7 +152,7 @@ void ExceptionKeepingTransform::work() { stage = Stage::Exception; ready_output = true; - data.exception = std::move(exception); + data.exception = exception; onException(); } else @@ -166,7 +166,7 @@ void ExceptionKeepingTransform::work() { stage = Stage::Exception; ready_output = true; - data.exception = std::move(exception); + data.exception = exception; onException(); } else @@ -188,7 +188,7 @@ void ExceptionKeepingTransform::work() { stage = Stage::Exception; ready_output = true; - data.exception = std::move(exception); + data.exception = exception; onException(); } } diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 0da7541556b..df34c592819 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -206,7 +206,7 @@ WindowTransform::WindowTransform(const Block & input_header_, { column = std::move(column)->convertToFullColumnIfConst(); } - input_header.setColumns(std::move(input_columns)); + input_header.setColumns(input_columns); // Initialize window function workspaces. workspaces.reserve(functions.size()); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 19302afb5c9..a993b8acd7d 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -695,7 +695,7 @@ IProcessor::Status FinalizingViewsTransform::prepare() return Status::Ready; if (any_exception) - output.pushException(std::move(any_exception)); + output.pushException(any_exception); output.finish(); return Status::Finished; @@ -708,7 +708,7 @@ static std::exception_ptr addStorageToException(std::exception_ptr ptr, const St { try { - std::rethrow_exception(std::move(ptr)); + std::rethrow_exception(ptr); } catch (DB::Exception & exception) { @@ -736,7 +736,7 @@ void FinalizingViewsTransform::work() if (!any_exception) any_exception = status.exception; - view.setException(addStorageToException(std::move(status.exception), view.table_id)); + view.setException(addStorageToException(status.exception, view.table_id)); } else { diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index d0f92535844..9218c75c390 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1122,7 +1122,7 @@ std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLFo HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server, const std::string & config_prefix) { - const auto & query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query"); + auto query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query"); auto factory = std::make_shared>(server, std::move(query_param_name)); factory->addFiltersFromConfig(server.config(), config_prefix); diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 8b08f5f28dd..8ca3c44bac2 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -117,7 +117,7 @@ void ColumnDescription::readText(ReadBuffer & buf) ParserColumnDeclaration column_parser(/* require type */ true); ASTPtr ast = parseQuery(column_parser, "x T " + modifiers, "column parser", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - if (const auto * col_ast = ast->as()) + if (auto * col_ast = ast->as()) { if (col_ast->default_expression) { @@ -309,7 +309,7 @@ void ColumnsDescription::flattenNested() continue; } - ColumnDescription column = std::move(*it); + ColumnDescription column = *it; removeSubcolumns(column.name); it = columns.get<0>().erase(it); diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index 60202e2055e..7085c6e14c8 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -198,6 +198,20 @@ ConstraintsDescription & ConstraintsDescription::operator=(const ConstraintsDesc return *this; } +ConstraintsDescription::ConstraintsDescription(ConstraintsDescription && other) noexcept + : constraints(std::move(other.constraints)) +{ + update(); +} + +ConstraintsDescription & ConstraintsDescription::operator=(ConstraintsDescription && other) noexcept +{ + constraints = std::move(other.constraints); + update(); + + return *this; +} + void ConstraintsDescription::update() { if (constraints.empty()) diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index a5095a79ccb..eb1eb95d33d 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -18,6 +18,9 @@ public: ConstraintsDescription(const ConstraintsDescription & other); ConstraintsDescription & operator=(const ConstraintsDescription & other); + ConstraintsDescription(ConstraintsDescription && other) noexcept; + ConstraintsDescription & operator=(ConstraintsDescription && other) noexcept; + bool empty() const { return constraints.empty(); } String toString() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7bfff3b5088..8ea9d0a31d0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3663,7 +3663,7 @@ RestoreDataTasks MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & bac Strings part_names = backup->listFiles(data_path_in_backup); for (const String & part_name : part_names) { - const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, format_version); + auto part_info = MergeTreePartInfo::tryParsePartName(part_name, format_version); if (!part_info) continue; @@ -4745,7 +4745,7 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg query_options, /* prepared_sets_= */ query_info.sets); const auto & analysis_result = select.getAnalysisResult(); - query_info.sets = std::move(select.getQueryAnalyzer()->getPreparedSets()); + query_info.sets = select.getQueryAnalyzer()->getPreparedSets(); bool can_use_aggregate_projection = true; /// If the first stage of the query pipeline is more complex than Aggregating - Expression - Filter - ReadFromStorage, diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 1df97dc9241..737e89979a6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -231,8 +231,10 @@ void MergeTreeDataPartChecksums::addFile(const String & file_name, UInt64 file_s void MergeTreeDataPartChecksums::add(MergeTreeDataPartChecksums && rhs_checksums) { - for (auto & checksum : rhs_checksums.files) - files[std::move(checksum.first)] = std::move(checksum.second); + for (auto && checksum : rhs_checksums.files) + { + files[checksum.first] = std::move(checksum.second); + } rhs_checksums.files.clear(); } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index e22f662960c..d8dba458203 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -707,7 +707,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar { auto old_columns = block_before_prewhere.getColumns(); filterColumns(old_columns, read_result.getFilterOriginal()->getData()); - block_before_prewhere.setColumns(std::move(old_columns)); + block_before_prewhere.setColumns(old_columns); } for (auto & column : block_before_prewhere) diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 726b2141ffb..2e17611cd93 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -26,14 +26,14 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_) : IMergeTreeReader( - std::move(data_part_), - std::move(columns_), + data_part_, + columns_, metadata_snapshot_, uncompressed_cache_, mark_cache_, - std::move(mark_ranges_), - std::move(settings_), - std::move(avg_value_size_hints_)) + mark_ranges_, + settings_, + avg_value_size_hints_) , marks_loader( data_part->volume->getDisk(), mark_cache, diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 8a69183e858..9599e3ee82c 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -20,9 +20,15 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( const StorageMetadataPtr & metadata_snapshot_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_) - : IMergeTreeReader(data_part_, std::move(columns_), metadata_snapshot_, - nullptr, nullptr, std::move(mark_ranges_), - std::move(settings_), {}) + : IMergeTreeReader( + data_part_, + columns_, + metadata_snapshot_, + nullptr, + nullptr, + mark_ranges_, + settings_, + {}) , part_in_memory(std::move(data_part_)) { for (const auto & name_and_type : columns) diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 99a36a8e08a..7d7975e0bc0 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -36,14 +36,14 @@ MergeTreeReaderWide::MergeTreeReaderWide( const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_) : IMergeTreeReader( - std::move(data_part_), - std::move(columns_), + data_part_, + columns_, metadata_snapshot_, uncompressed_cache_, - std::move(mark_cache_), - std::move(mark_ranges_), - std::move(settings_), - std::move(avg_value_size_hints_)) + mark_cache_, + mark_ranges_, + settings_, + avg_value_size_hints_) { try { diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index da8c5f115b2..e6d856b80fc 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -295,7 +295,7 @@ void StorageDictionary::alter(const AlterCommands & params, ContextPtr alter_con } std::lock_guard lock(dictionary_config_mutex); - configuration->setString("dictionary.comment", std::move(new_comment)); + configuration->setString("dictionary.comment", new_comment); } void registerStorageDictionary(StorageFactory & factory) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 19e8f78d877..8934fd0ccbf 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -172,7 +172,7 @@ ColumnPtr fillColumnWithRandomData( auto data_column = fillColumnWithRandomData(nested_type, offset, max_array_length, max_string_length, rng, context); - return ColumnArray::create(std::move(data_column), std::move(offsets_column)); + return ColumnArray::create(data_column, std::move(offsets_column)); } case TypeIndex::Tuple: @@ -198,7 +198,7 @@ ColumnPtr fillColumnWithRandomData( for (UInt64 i = 0; i < limit; ++i) null_map[i] = rng() % 16 == 0; /// No real motivation for this. - return ColumnNullable::create(std::move(nested_column), std::move(null_map_column)); + return ColumnNullable::create(nested_column, std::move(null_map_column)); } case TypeIndex::UInt8: @@ -395,7 +395,7 @@ protected: for (const auto & elem : block_to_fill) columns.emplace_back(fillColumnWithRandomData(elem.type, block_size, max_array_length, max_string_length, rng, context)); - columns = Nested::flatten(block_to_fill.cloneWithColumns(std::move(columns))).getColumns(); + columns = Nested::flatten(block_to_fill.cloneWithColumns(columns)).getColumns(); return {std::move(columns), block_size}; } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 9d6935b609b..bdaed8b2624 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -55,6 +55,9 @@ struct StorageInMemoryMetadata StorageInMemoryMetadata(const StorageInMemoryMetadata & other); StorageInMemoryMetadata & operator=(const StorageInMemoryMetadata & other); + StorageInMemoryMetadata(StorageInMemoryMetadata && other) = default; + StorageInMemoryMetadata & operator=(StorageInMemoryMetadata && other) = default; + /// NOTE: Thread unsafe part. You should modify same StorageInMemoryMetadata /// structure from different threads. It should be used as MultiVersion /// object. See example in IStorage. diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 5ba1514877a..d5d1f312bec 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -171,7 +171,7 @@ Chunk LogSource::generate() } if (!column->empty()) - res.insert(ColumnWithTypeAndName(std::move(column), name_type.type, name_type.name)); + res.insert(ColumnWithTypeAndName(column, name_type.type, name_type.name)); } if (res) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 433fdb5b0b5..15e499c6e6c 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -730,7 +730,7 @@ void StorageMerge::convertingSourceStream( for (const auto & alias : aliases) { pipe_columns.emplace_back(NameAndTypePair(alias.name, alias.type)); - ASTPtr expr = std::move(alias.expression); + ASTPtr expr = alias.expression; auto syntax_result = TreeRewriter(local_context).analyze(expr, pipe_columns); auto expression_analyzer = ExpressionAnalyzer{alias.expression, syntax_result, local_context}; diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 83cf2b07b21..10b6c7ca447 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -178,7 +178,7 @@ public: { /// Avoid Excessive copy when block is small enough if (block.rows() <= max_rows) - return Blocks{std::move(block)}; + return {block}; const size_t split_block_size = ceil(block.rows() * 1.0 / max_rows); Blocks split_blocks(split_block_size); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index e62b14224ec..e06ca7a14d1 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -649,7 +649,7 @@ StorageURLWithFailover::StorageURLWithFailover( Poco::URI poco_uri(uri_option); context_->getRemoteHostFilter().checkURL(poco_uri); LOG_DEBUG(&Poco::Logger::get("StorageURLDistributed"), "Adding URL option: {}", uri_option); - uri_options.emplace_back(std::move(uri_option)); + uri_options.emplace_back(uri_option); } } diff --git a/src/Storages/System/StorageSystemRowPolicies.cpp b/src/Storages/System/StorageSystemRowPolicies.cpp index 455d715d5da..cd4f3dab109 100644 --- a/src/Storages/System/StorageSystemRowPolicies.cpp +++ b/src/Storages/System/StorageSystemRowPolicies.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include namespace DB @@ -43,7 +43,8 @@ NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes() {"apply_to_except", std::make_shared(std::make_shared())} }; - boost::range::push_back(names_and_types, std::move(extra_names_and_types)); + insertAtEnd(names_and_types, extra_names_and_types); + return names_and_types; } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 24e3fe4f7a9..9332bc6a004 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -509,8 +509,8 @@ protected: loading_dependencies_tables.reserve(info.dependencies.size()); for (auto && dependency : info.dependencies) { - loading_dependencies_databases.push_back(std::move(dependency.database)); - loading_dependencies_tables.push_back(std::move(dependency.table)); + loading_dependencies_databases.push_back(dependency.database); + loading_dependencies_tables.push_back(dependency.table); } Array loading_dependent_databases; @@ -519,8 +519,8 @@ protected: loading_dependent_tables.reserve(info.dependencies.size()); for (auto && dependent : info.dependent_database_objects) { - loading_dependent_databases.push_back(std::move(dependent.database)); - loading_dependent_tables.push_back(std::move(dependent.table)); + loading_dependent_databases.push_back(dependent.database); + loading_dependent_tables.push_back(dependent.table); } if (columns_mask[src_index++]) diff --git a/src/TableFunctions/TableFunctionExecutable.cpp b/src/TableFunctions/TableFunctionExecutable.cpp index 41ba2db5c33..18d7d8867e8 100644 --- a/src/TableFunctions/TableFunctionExecutable.cpp +++ b/src/TableFunctions/TableFunctionExecutable.cpp @@ -78,7 +78,7 @@ StoragePtr TableFunctionExecutable::executeImpl(const ASTPtr & /*ast_function*/, auto global_context = context->getGlobalContext(); ExecutableSettings settings; settings.script_name = script_name; - settings.script_arguments = std::move(arguments); + settings.script_arguments = arguments; auto storage = StorageExecutable::create(storage_id, format, settings, input_queries, getActualTableStructure(context), ConstraintsDescription{}); storage->startup();