clang-tidy, part 12

This commit is contained in:
Alexey Milovidov 2020-03-09 04:22:33 +03:00
parent 6320f59abb
commit cde492a784
12 changed files with 36 additions and 50 deletions

View File

@ -46,7 +46,7 @@ namespace ErrorCodes
namespace namespace
{ {
static constexpr const std::chrono::minutes decrease_error_count_period{5}; constexpr const std::chrono::minutes decrease_error_count_period{5};
template <typename PoolFactory> template <typename PoolFactory>
ConnectionPoolPtrs createPoolsForAddresses(const std::string & name, PoolFactory && factory) ConnectionPoolPtrs createPoolsForAddresses(const std::string & name, PoolFactory && factory)
@ -238,7 +238,7 @@ bool StorageDistributedDirectoryMonitor::processFiles()
const auto & file_path_str = it->path(); const auto & file_path_str = it->path();
Poco::Path file_path{file_path_str}; Poco::Path file_path{file_path_str};
if (!it->isDirectory() && startsWith(file_path.getExtension().data(), "bin")) if (!it->isDirectory() && startsWith(file_path.getExtension(), "bin"))
files[parse<UInt64>(file_path.getBaseName())] = file_path_str; files[parse<UInt64>(file_path.getBaseName())] = file_path_str;
} }

View File

@ -33,8 +33,8 @@ namespace DataPartsExchange
namespace namespace
{ {
static constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE = 1; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE = 1;
static constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS = 2; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS = 2;
std::string getEndpointId(const std::string & node_id) std::string getEndpointId(const std::string & node_id)

View File

@ -268,8 +268,8 @@ public:
{ {
} }
void load(const MergeTreeData & storage, const String & part_path); void load(const MergeTreeData & data, const String & part_path);
void store(const MergeTreeData & storage, const String & part_path, Checksums & checksums) const; void store(const MergeTreeData & data, const String & part_path, Checksums & checksums) const;
void store(const Names & column_names, const DataTypes & data_types, const String & part_path, Checksums & checksums) const; void store(const Names & column_names, const DataTypes & data_types, const String & part_path, Checksums & checksums) const;
void update(const Block & block, const Names & column_names); void update(const Block & block, const Names & column_names);

View File

@ -500,12 +500,12 @@ void StorageLog::loadMarks()
std::unique_ptr<ReadBuffer> marks_rb = disk->readFile(marks_file_path, 32768); std::unique_ptr<ReadBuffer> marks_rb = disk->readFile(marks_file_path, 32768);
while (!marks_rb->eof()) while (!marks_rb->eof())
{ {
for (size_t i = 0; i < files_by_index.size(); ++i) for (auto & file : files_by_index)
{ {
Mark mark; Mark mark;
readIntBinary(mark.rows, *marks_rb); readIntBinary(mark.rows, *marks_rb);
readIntBinary(mark.offset, *marks_rb); readIntBinary(mark.offset, *marks_rb);
files_by_index[i]->second.marks.push_back(mark); file->second.marks.push_back(mark);
} }
} }
} }

View File

@ -9,11 +9,9 @@ String getDiskName(ASTStorage & storage_def)
if (storage_def.settings) if (storage_def.settings)
{ {
SettingsChanges changes = storage_def.settings->changes; SettingsChanges changes = storage_def.settings->changes;
for (auto it = changes.begin(); it != changes.end(); ++it) for (const auto & change : changes)
{ if (change.name == "disk")
if (it->name == "disk") return change.value.safeGet<String>();
return it->value.safeGet<String>();
}
} }
return "default"; return "default";
} }

View File

@ -253,7 +253,7 @@ std::vector<MergeTreeData::AlterDataPartTransactionPtr> StorageMergeTree::prepar
} }
void StorageMergeTree::alter( void StorageMergeTree::alter(
const AlterCommands & params, const AlterCommands & commands,
const Context & context, const Context & context,
TableStructureWriteLockHolder & table_lock_holder) TableStructureWriteLockHolder & table_lock_holder)
{ {
@ -263,7 +263,7 @@ void StorageMergeTree::alter(
StorageInMemoryMetadata metadata = getInMemoryMetadata(); StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata); commands.apply(metadata);
/// Update metdata in memory /// Update metdata in memory
auto update_metadata = [&metadata, &table_lock_holder, this]() auto update_metadata = [&metadata, &table_lock_holder, this]()
@ -277,7 +277,7 @@ void StorageMergeTree::alter(
}; };
/// This alter can be performed at metadata level only /// This alter can be performed at metadata level only
if (!params.isModifyingData()) if (!commands.isModifyingData())
{ {
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());

View File

@ -214,8 +214,8 @@ void registerStorageMySQL(StorageFactory & factory)
"Storage MySQL requires 5-7 parameters: MySQL('host:port', database, table, 'user', 'password'[, replace_query, 'on_duplicate_clause']).", "Storage MySQL requires 5-7 parameters: MySQL('host:port', database, table, 'user', 'password'[, replace_query, 'on_duplicate_clause']).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (size_t i = 0; i < engine_args.size(); ++i) for (auto & engine_arg : engine_args)
engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[i], args.local_context); engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context);
/// 3306 is the default MySQL port. /// 3306 is the default MySQL port.
auto parsed_host_port = parseAddress(engine_args[0]->as<ASTLiteral &>().value.safeGet<String>(), 3306); auto parsed_host_port = parseAddress(engine_args[0]->as<ASTLiteral &>().value.safeGet<String>(), 3306);

View File

@ -1308,10 +1308,10 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
Coordination::Requests ops; Coordination::Requests ops;
for (size_t i = 0, size = replicas.size(); i < size; ++i) for (const auto & path_part : replicas)
{ {
Coordination::Stat stat; Coordination::Stat stat;
String path = zookeeper_path + "/replicas/" + replicas[i] + "/host"; String path = zookeeper_path + "/replicas/" + path_part + "/host";
zookeeper->get(path, &stat); zookeeper->get(path, &stat);
ops.emplace_back(zkutil::makeCheckRequest(path, stat.version)); ops.emplace_back(zkutil::makeCheckRequest(path, stat.version));
} }
@ -4079,7 +4079,6 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query
NullBlockOutputStream output({}); NullBlockOutputStream output({});
copyData(stream, output); copyData(stream, output);
return;
} }
@ -4089,14 +4088,13 @@ std::optional<Cluster::Address> StorageReplicatedMergeTree::findClusterAddress(c
{ {
const auto & shards = iter.second->getShardsAddresses(); const auto & shards = iter.second->getShardsAddresses();
for (size_t shard_num = 0; shard_num < shards.size(); ++shard_num) for (const auto & shard : shards)
{ {
for (size_t replica_num = 0; replica_num < shards[shard_num].size(); ++replica_num) for (const auto & replica : shard)
{ {
const Cluster::Address & address = shards[shard_num][replica_num];
/// user is actually specified, not default /// user is actually specified, not default
if (address.host_name == leader_address.host && address.port == leader_address.queries_port && address.user_specified) if (replica.host_name == leader_address.host && replica.port == leader_address.queries_port && replica.user_specified)
return address; return replica;
} }
} }
} }
@ -4873,14 +4871,12 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
} }
} }
for (size_t i = 0; i < src_all_parts.size(); ++i) for (const auto & src_part : src_all_parts)
{ {
/// We also make some kind of deduplication to avoid duplicated parts in case of ATTACH PARTITION /// We also make some kind of deduplication to avoid duplicated parts in case of ATTACH PARTITION
/// Assume that merges in the partition are quite rare /// Assume that merges in the partition are quite rare
/// Save deduplication block ids with special prefix replace_partition /// Save deduplication block ids with special prefix replace_partition
auto & src_part = src_all_parts[i];
if (!canReplacePartition(src_part)) if (!canReplacePartition(src_part))
throw Exception( throw Exception(
"Cannot replace partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table", "Cannot replace partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table",
@ -5054,17 +5050,15 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
/// Clone parts into destination table. /// Clone parts into destination table.
for (size_t i = 0; i < src_all_parts.size(); ++i) for (const auto & src_part : src_all_parts)
{ {
auto & src_part = src_all_parts[i];
if (!dest_table_storage->canReplacePartition(src_part)) if (!dest_table_storage->canReplacePartition(src_part))
throw Exception( throw Exception(
"Cannot move partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table", "Cannot move partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table",
ErrorCodes::LOGICAL_ERROR); ErrorCodes::LOGICAL_ERROR);
String hash_hex = src_part->checksums.getTotalChecksumHex(); String hash_hex = src_part->checksums.getTotalChecksumHex();
String block_id_path = ""; String block_id_path;
auto lock = dest_table_storage->allocateBlockNumber(partition_id, zookeeper, block_id_path); auto lock = dest_table_storage->allocateBlockNumber(partition_id, zookeeper, block_id_path);
if (!lock) if (!lock)

View File

@ -106,7 +106,7 @@ public:
void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & query_context) override; void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & query_context) override;
void mutate(const MutationCommands & commands, const Context & context) override; void mutate(const MutationCommands & commands, const Context & context) override;
void waitMutation(const String & znode_name, size_t mutation_sync) const; void waitMutation(const String & znode_name, size_t mutations_sync) const;
std::vector<MergeTreeMutationStatus> getMutationsStatus() const override; std::vector<MergeTreeMutationStatus> getMutationsStatus() const override;
CancellationCode killMutation(const String & mutation_id) override; CancellationCode killMutation(const String & mutation_id) override;
@ -522,7 +522,7 @@ private:
void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & query_context); void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & query_context);
void attachPartition(const ASTPtr & partition, bool part, const Context & query_context); void attachPartition(const ASTPtr & partition, bool part, const Context & query_context);
void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context);
void movePartitionToTable(const StoragePtr & source_table, const ASTPtr & partition, const Context & query_context); void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & query_context);
void fetchPartition(const ASTPtr & partition, const String & from, const Context & query_context); void fetchPartition(const ASTPtr & partition, const String & from, const Context & query_context);
/// Check granularity of already existing replicated table in zookeeper if it exists /// Check granularity of already existing replicated table in zookeeper if it exists

View File

@ -321,8 +321,8 @@ void registerStorageS3(StorageFactory & factory)
throw Exception( throw Exception(
"Storage S3 requires 2 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); "Storage S3 requires 2 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (size_t i = 0; i < engine_args.size(); ++i) for (auto & engine_arg : engine_args)
engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[i], args.local_context); engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context);
String url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>(); String url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
Poco::URI uri (url); Poco::URI uri (url);

View File

@ -29,8 +29,8 @@ namespace
/// Verifying that the function depends only on the specified columns /// Verifying that the function depends only on the specified columns
bool isValidFunction(const ASTPtr & expression, const NameSet & columns) bool isValidFunction(const ASTPtr & expression, const NameSet & columns)
{ {
for (size_t i = 0; i < expression->children.size(); ++i) for (const auto & child : expression->children)
if (!isValidFunction(expression->children[i], columns)) if (!isValidFunction(child, columns))
return false; return false;
if (auto opt_name = IdentifierSemantic::getColumnName(expression)) if (auto opt_name = IdentifierSemantic::getColumnName(expression))
@ -45,8 +45,8 @@ void extractFunctions(const ASTPtr & expression, const NameSet & columns, std::v
const auto * function = expression->as<ASTFunction>(); const auto * function = expression->as<ASTFunction>();
if (function && function->name == "and") if (function && function->name == "and")
{ {
for (size_t i = 0; i < function->arguments->children.size(); ++i) for (const auto & child : function->arguments->children)
extractFunctions(function->arguments->children[i], columns, result); extractFunctions(child, columns, result);
} }
else if (isValidFunction(expression, columns)) else if (isValidFunction(expression, columns))
{ {

View File

@ -114,7 +114,7 @@ bool isCompatible(const IAST & node)
return false; return false;
for (const auto & expr : function->arguments->children) for (const auto & expr : function->arguments->children)
if (!isCompatible(*expr.get())) if (!isCompatible(*expr))
return false; return false;
return true; return true;
@ -123,16 +123,10 @@ bool isCompatible(const IAST & node)
if (const auto * literal = node.as<ASTLiteral>()) if (const auto * literal = node.as<ASTLiteral>())
{ {
/// Foreign databases often have no support for Array. But Tuple literals are passed to support IN clause. /// Foreign databases often have no support for Array. But Tuple literals are passed to support IN clause.
if (literal->value.getType() == Field::Types::Array) return literal->value.getType() != Field::Types::Array;
return false;
return true;
} }
if (node.as<ASTIdentifier>()) return node.as<ASTIdentifier>();
return true;
return false;
} }
} }