Skip to content

Commit

Permalink
clang-tidy check performance-move-const-arg fix
Browse files Browse the repository at this point in the history
  • Loading branch information
kitaisreal committed Mar 2, 2022
1 parent 1f58373 commit b1a956c
Show file tree
Hide file tree
Showing 85 changed files with 198 additions and 134 deletions.
2 changes: 1 addition & 1 deletion .clang-tidy
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
24 changes: 24 additions & 0 deletions base/base/insertAtEnd.h
Original file line number Diff line number Diff line change
Expand Up @@ -26,3 +26,27 @@ void insertAtEnd(std::vector<T> & dest, std::vector<T> && src)
dest.insert(dest.end(), std::make_move_iterator(src.begin()), std::make_move_iterator(src.end()));
src.clear();
}

template <typename Container>
void insertAtEnd(Container & dest, const Container & src)
{
if (src.empty())
return;

dest.insert(dest.end(), src.begin(), src.end());
}

template <typename Container>
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();
}
2 changes: 1 addition & 1 deletion src/Client/ConnectionEstablisher.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,7 @@ std::variant<int, ConnectionEstablisher::TryResult> ConnectionEstablisherAsync::
fiber = std::move(fiber).resume();

if (exception)
std::rethrow_exception(std::move(exception));
std::rethrow_exception(exception);

if (connection_establisher.isFinished())
{
Expand Down
2 changes: 1 addition & 1 deletion src/Client/HedgedConnections.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -363,7 +363,7 @@ bool HedgedConnections::resumePacketReceiver(const HedgedConnections::ReplicaLoc
else if (std::holds_alternative<std::exception_ptr>(res))
{
finishProcessReplica(replica_state, true);
std::rethrow_exception(std::move(std::get<std::exception_ptr>(res)));
std::rethrow_exception(std::get<std::exception_ptr>(res));
}

return false;
Expand Down
4 changes: 2 additions & 2 deletions src/Client/LocalConnection.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
}

Expand Down
2 changes: 1 addition & 1 deletion src/Columns/ColumnNullable.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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());
});
Expand Down
2 changes: 1 addition & 1 deletion src/Columns/ColumnSparse.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
5 changes: 5 additions & 0 deletions src/Common/ConcurrentBoundedQueue.h
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,11 @@ class ConcurrentBoundedQueue
return emplace(x);
}

[[nodiscard]] bool push(T && x)
{
return emplace(std::move(x));
}

/// Returns false if queue is finished
template <typename... Args>
[[nodiscard]] bool emplace(Args &&... args)
Expand Down
2 changes: 1 addition & 1 deletion src/Compression/tests/gtest_compressionCodec.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -316,7 +316,7 @@ CodecTestSequence operator+(CodecTestSequence && left, const CodecTestSequence &

std::vector<CodecTestSequence> operator+(const std::vector<CodecTestSequence> & left, const std::vector<CodecTestSequence> & right)
{
std::vector<CodecTestSequence> result(std::move(left));
std::vector<CodecTestSequence> result(left);
std::move(std::begin(right), std::end(right), std::back_inserter(result));

return result;
Expand Down
2 changes: 1 addition & 1 deletion src/Coordination/KeeperStorage.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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());

Expand Down
5 changes: 5 additions & 0 deletions src/Core/Block.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,11 @@ Block::Block(const ColumnsWithTypeAndName & data_) : data{data_}
initializeIndexByName();
}

Block::Block(ColumnsWithTypeAndName && data_) : data{std::move(data_)}
{
initializeIndexByName();
}


void Block::initializeIndexByName()
{
Expand Down
1 change: 1 addition & 0 deletions src/Core/Block.h
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ class Block
Block() = default;
Block(std::initializer_list<ColumnWithTypeAndName> il);
Block(const ColumnsWithTypeAndName & data_);
Block(ColumnsWithTypeAndName && data_);

/// insert the column at the specified position
void insert(size_t position, ColumnWithTypeAndName elem);
Expand Down
2 changes: 1 addition & 1 deletion src/DataTypes/Serializations/SerializationInfoTuple.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}

Expand Down
2 changes: 1 addition & 1 deletion src/Databases/DatabaseAtomic.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshot
};

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)
Expand Down
2 changes: 1 addition & 1 deletion src/Databases/MySQL/DatabaseMySQL.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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
{
Expand Down
2 changes: 1 addition & 1 deletion src/Databases/MySQL/MaterializeMetadata.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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_)
Expand Down
2 changes: 1 addition & 1 deletion src/Databases/MySQL/MaterializedMySQLSyncThread.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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_)
{
Expand Down
2 changes: 1 addition & 1 deletion src/Dictionaries/CacheDictionary.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -494,7 +494,7 @@ Pipe CacheDictionary<dictionary_key_type>::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<DataTypeUInt64>(), dict_struct.id->name)};
key_columns = {ColumnWithTypeAndName(keys_column, std::make_shared<DataTypeUInt64>(), dict_struct.id->name)};
}
else
{
Expand Down
2 changes: 1 addition & 1 deletion src/Dictionaries/DictionarySourceHelpers.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
4 changes: 2 additions & 2 deletions src/Dictionaries/FlatDictionary.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
Expand Down Expand Up @@ -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<DataTypeUInt64>(), dict_struct.id->name)};
ColumnsWithTypeAndName key_columns = {ColumnWithTypeAndName(keys_column, std::make_shared<DataTypeUInt64>(), dict_struct.id->name)};

std::shared_ptr<const IDictionary> dictionary = shared_from_this();
auto coordinator = DictionarySourceCoordinator::create(dictionary, column_names, std::move(key_columns), max_block_size);
Expand Down
2 changes: 1 addition & 1 deletion src/Dictionaries/HashedArrayDictionary.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -578,7 +578,7 @@ ColumnPtr HashedArrayDictionary<dictionary_key_type>::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;
}
Expand Down
2 changes: 1 addition & 1 deletion src/Dictionaries/HashedDictionary.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::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;
}
Expand Down
2 changes: 1 addition & 1 deletion src/Dictionaries/PolygonDictionary.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
Expand Down
4 changes: 2 additions & 2 deletions src/Dictionaries/RangeHashedDictionary.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -198,7 +198,7 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::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;
}
Expand Down Expand Up @@ -298,7 +298,7 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::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;
}
Expand Down
8 changes: 4 additions & 4 deletions src/Dictionaries/RedisDictionarySource.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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<String>(RedisCommand("TYPE").addRedisType(key)))
keys.addRedisType(std::move(key));
keys.addRedisType(key);

if (configuration.storage_type == RedisStorageType::HASH_MAP)
{
Expand All @@ -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<RedisSource>(
Expand Down
7 changes: 4 additions & 3 deletions src/Formats/FormatFactory.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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};

Expand Down
3 changes: 1 addition & 2 deletions src/Functions/array/mapPopulateSeries.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -379,8 +379,7 @@ class FunctionMapPopulateSeries : public IFunction
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);
}
}

Expand Down
2 changes: 1 addition & 1 deletion src/Functions/castOrDefault.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ class FunctionCastOrDefault final : public IFunction
{
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);

Expand Down
2 changes: 1 addition & 1 deletion src/Functions/if.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1027,7 +1027,7 @@ class FunctionIf : public FunctionIfBase

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))
Expand Down
2 changes: 1 addition & 1 deletion src/Functions/multiIf.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ class FunctionMultiIf final : public FunctionIfBase

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.
Expand Down
2 changes: 1 addition & 1 deletion src/Functions/now64.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ class Now64OverloadResolver : public IFunctionOverloadResolver
for (const auto & arg : arguments)
arg_types.push_back(arg.type);

return std::make_unique<FunctionBaseNow64>(nowSubsecond(scale), std::move(arg_types), std::move(result_type));
return std::make_unique<FunctionBaseNow64>(nowSubsecond(scale), std::move(arg_types), result_type);
}
};

Expand Down
2 changes: 1 addition & 1 deletion src/Functions/nullIf.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ class FunctionNullIf : public IFunction
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);
}
};

Expand Down
2 changes: 1 addition & 1 deletion src/IO/WriteBufferFromS3.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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();

Expand Down
6 changes: 3 additions & 3 deletions src/Interpreters/Access/InterpreterShowAccessQuery.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@
#include <Access/AccessControl.h>
#include <base/range.h>
#include <base/sort.h>
#include <boost/range/algorithm_ext/push_back.hpp>
#include <base/insertAtEnd.h>


namespace DB
Expand Down Expand Up @@ -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;
}

Expand Down
Loading

0 comments on commit b1a956c

Please sign in to comment.