mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 18:42:26 +00:00
clang-tidy, part 12
This commit is contained in:
parent
6320f59abb
commit
cde492a784
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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)
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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";
|
||||||
}
|
}
|
||||||
|
@ -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());
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
@ -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)
|
||||||
|
@ -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
|
||||||
|
@ -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);
|
||||||
|
@ -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))
|
||||||
{
|
{
|
||||||
|
@ -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;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user