mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 09:02:00 +00:00
Remove -Wshadow suppression which leaked into global namespace
This commit is contained in:
parent
f41354ccd6
commit
7a21d5888c
@ -1192,12 +1192,12 @@ try
|
||||
{
|
||||
Settings::checkNoSettingNamesAtTopLevel(*config, config_path);
|
||||
|
||||
ServerSettings server_settings;
|
||||
server_settings.loadSettingsFromConfig(*config);
|
||||
ServerSettings server_settings_;
|
||||
server_settings_.loadSettingsFromConfig(*config);
|
||||
|
||||
size_t max_server_memory_usage = server_settings.max_server_memory_usage;
|
||||
size_t max_server_memory_usage = server_settings_.max_server_memory_usage;
|
||||
|
||||
double max_server_memory_usage_to_ram_ratio = server_settings.max_server_memory_usage_to_ram_ratio;
|
||||
double max_server_memory_usage_to_ram_ratio = server_settings_.max_server_memory_usage_to_ram_ratio;
|
||||
size_t default_max_server_memory_usage = static_cast<size_t>(memory_amount * max_server_memory_usage_to_ram_ratio);
|
||||
|
||||
if (max_server_memory_usage == 0)
|
||||
@ -1225,7 +1225,7 @@ try
|
||||
total_memory_tracker.setDescription("(total)");
|
||||
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
|
||||
|
||||
total_memory_tracker.setAllowUseJemallocMemory(server_settings.allow_use_jemalloc_memory);
|
||||
total_memory_tracker.setAllowUseJemallocMemory(server_settings_.allow_use_jemalloc_memory);
|
||||
|
||||
auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker();
|
||||
total_memory_tracker.setOvercommitTracker(global_overcommit_tracker);
|
||||
@ -1243,23 +1243,23 @@ try
|
||||
|
||||
global_context->setRemoteHostFilter(*config);
|
||||
|
||||
global_context->setMaxTableSizeToDrop(server_settings.max_table_size_to_drop);
|
||||
global_context->setMaxPartitionSizeToDrop(server_settings.max_partition_size_to_drop);
|
||||
global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop);
|
||||
global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop);
|
||||
|
||||
ConcurrencyControl::SlotCount concurrent_threads_soft_limit = ConcurrencyControl::Unlimited;
|
||||
if (server_settings.concurrent_threads_soft_limit_num > 0 && server_settings.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit)
|
||||
concurrent_threads_soft_limit = server_settings.concurrent_threads_soft_limit_num;
|
||||
if (server_settings.concurrent_threads_soft_limit_ratio_to_cores > 0)
|
||||
if (server_settings_.concurrent_threads_soft_limit_num > 0 && server_settings_.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit)
|
||||
concurrent_threads_soft_limit = server_settings_.concurrent_threads_soft_limit_num;
|
||||
if (server_settings_.concurrent_threads_soft_limit_ratio_to_cores > 0)
|
||||
{
|
||||
auto value = server_settings.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency();
|
||||
auto value = server_settings_.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency();
|
||||
if (value > 0 && value < concurrent_threads_soft_limit)
|
||||
concurrent_threads_soft_limit = value;
|
||||
}
|
||||
ConcurrencyControl::instance().setMaxConcurrency(concurrent_threads_soft_limit);
|
||||
|
||||
global_context->getProcessList().setMaxSize(server_settings.max_concurrent_queries);
|
||||
global_context->getProcessList().setMaxInsertQueriesAmount(server_settings.max_concurrent_insert_queries);
|
||||
global_context->getProcessList().setMaxSelectQueriesAmount(server_settings.max_concurrent_select_queries);
|
||||
global_context->getProcessList().setMaxSize(server_settings_.max_concurrent_queries);
|
||||
global_context->getProcessList().setMaxInsertQueriesAmount(server_settings_.max_concurrent_insert_queries);
|
||||
global_context->getProcessList().setMaxSelectQueriesAmount(server_settings_.max_concurrent_select_queries);
|
||||
|
||||
if (config->has("keeper_server"))
|
||||
global_context->updateKeeperConfiguration(*config);
|
||||
@ -1270,34 +1270,34 @@ try
|
||||
/// This is done for backward compatibility.
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = server_settings.background_pool_size;
|
||||
auto new_ratio = server_settings.background_merges_mutations_concurrency_ratio;
|
||||
auto new_pool_size = server_settings_.background_pool_size;
|
||||
auto new_ratio = server_settings_.background_merges_mutations_concurrency_ratio;
|
||||
global_context->getMergeMutateExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, static_cast<size_t>(new_pool_size * new_ratio));
|
||||
global_context->getMergeMutateExecutor()->updateSchedulingPolicy(server_settings.background_merges_mutations_scheduling_policy.toString());
|
||||
global_context->getMergeMutateExecutor()->updateSchedulingPolicy(server_settings_.background_merges_mutations_scheduling_policy.toString());
|
||||
}
|
||||
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = server_settings.background_move_pool_size;
|
||||
auto new_pool_size = server_settings_.background_move_pool_size;
|
||||
global_context->getMovesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
|
||||
}
|
||||
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = server_settings.background_fetches_pool_size;
|
||||
auto new_pool_size = server_settings_.background_fetches_pool_size;
|
||||
global_context->getFetchesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
|
||||
}
|
||||
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = server_settings.background_common_pool_size;
|
||||
auto new_pool_size = server_settings_.background_common_pool_size;
|
||||
global_context->getCommonExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
|
||||
}
|
||||
|
||||
global_context->getBufferFlushSchedulePool().increaseThreadsCount(server_settings.background_buffer_flush_schedule_pool_size);
|
||||
global_context->getSchedulePool().increaseThreadsCount(server_settings.background_schedule_pool_size);
|
||||
global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings.background_message_broker_schedule_pool_size);
|
||||
global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings.background_distributed_schedule_pool_size);
|
||||
global_context->getBufferFlushSchedulePool().increaseThreadsCount(server_settings_.background_buffer_flush_schedule_pool_size);
|
||||
global_context->getSchedulePool().increaseThreadsCount(server_settings_.background_schedule_pool_size);
|
||||
global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size);
|
||||
global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size);
|
||||
|
||||
if (config->has("resources"))
|
||||
{
|
||||
|
@ -177,7 +177,7 @@ void TablesLoader::removeUnresolvableDependencies()
|
||||
}
|
||||
|
||||
|
||||
void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool)
|
||||
void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool_)
|
||||
{
|
||||
/// Compatibility setting which should be enabled by default on attach
|
||||
/// Otherwise server will be unable to start for some old-format of IPv6/IPv4 types of columns
|
||||
@ -189,12 +189,12 @@ void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool)
|
||||
|
||||
for (size_t level = 0; level != tables_to_load.size(); ++level)
|
||||
{
|
||||
startLoadingTables(pool, load_context, tables_to_load[level], level);
|
||||
pool.wait();
|
||||
startLoadingTables(pool_, load_context, tables_to_load[level], level);
|
||||
pool_.wait();
|
||||
}
|
||||
}
|
||||
|
||||
void TablesLoader::startLoadingTables(ThreadPool & pool, ContextMutablePtr load_context, const std::vector<StorageID> & tables_to_load, size_t level)
|
||||
void TablesLoader::startLoadingTables(ThreadPool & pool_, ContextMutablePtr load_context, const std::vector<StorageID> & tables_to_load, size_t level)
|
||||
{
|
||||
size_t total_tables = metadata.parsed_tables.size();
|
||||
|
||||
@ -202,7 +202,7 @@ void TablesLoader::startLoadingTables(ThreadPool & pool, ContextMutablePtr load_
|
||||
|
||||
for (const auto & table_id : tables_to_load)
|
||||
{
|
||||
pool.scheduleOrThrowOnError([this, load_context, total_tables, table_name = table_id.getQualifiedName()]()
|
||||
pool_.scheduleOrThrowOnError([this, load_context, total_tables, table_name = table_id.getQualifiedName()]()
|
||||
{
|
||||
const auto & path_and_query = metadata.parsed_tables[table_name];
|
||||
databases[table_name.database]->loadTableFromMetadata(load_context, path_and_query.path, table_name, path_and_query.ast, strictness_mode);
|
||||
|
@ -16,9 +16,6 @@
|
||||
#include <Interpreters/IExternalLoadable.h>
|
||||
|
||||
|
||||
/// Clang mistakenly warns about the names in enum class.
|
||||
#pragma clang diagnostic ignored "-Wshadow"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
using TypeIndexUnderlying = magic_enum::underlying_type_t<TypeIndex>;
|
||||
|
@ -139,13 +139,13 @@ private:
|
||||
void threadWorker(size_t shard)
|
||||
{
|
||||
Block block;
|
||||
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
|
||||
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder_;
|
||||
auto & shard_queue = *shards_queues[shard];
|
||||
|
||||
while (shard_queue.pop(block))
|
||||
{
|
||||
Stopwatch watch;
|
||||
dictionary.blockToAttributes(block, arena_holder, shard);
|
||||
dictionary.blockToAttributes(block, arena_holder_, shard);
|
||||
UInt64 elapsed_ms = watch.elapsedMilliseconds();
|
||||
if (elapsed_ms > 1'000)
|
||||
LOG_TRACE(dictionary.log, "Block processing for shard #{} is slow {}ms (rows {}).", shard, elapsed_ms, block.rows());
|
||||
|
@ -1227,7 +1227,7 @@ Pipe RangeHashedDictionary<dictionary_key_type>::read(const Names & column_names
|
||||
DictionarySourceCoordinator::ReadColumnsFunc read_keys_func = [dictionary_copy = dictionary](
|
||||
const Strings & attribute_names,
|
||||
const DataTypes & result_types,
|
||||
const Columns & key_columns,
|
||||
const Columns & key_columns_,
|
||||
const DataTypes,
|
||||
const Columns &)
|
||||
{
|
||||
@ -1238,15 +1238,15 @@ Pipe RangeHashedDictionary<dictionary_key_type>::read(const Names & column_names
|
||||
Columns result;
|
||||
result.reserve(attribute_names_size);
|
||||
|
||||
const ColumnPtr & key_column = key_columns.back();
|
||||
const ColumnPtr & key_column = key_columns_.back();
|
||||
|
||||
const auto * key_to_index_column = typeid_cast<const ColumnUInt64 *>(key_column.get());
|
||||
if (!key_to_index_column)
|
||||
const auto * key_to_index_column_ = typeid_cast<const ColumnUInt64 *>(key_column.get());
|
||||
if (!key_to_index_column_)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Dictionary {} read expect indexes column with type UInt64",
|
||||
range_dictionary_ptr->getFullName());
|
||||
|
||||
const auto & data = key_to_index_column->getData();
|
||||
const auto & data = key_to_index_column_->getData();
|
||||
|
||||
for (size_t i = 0; i < attribute_names_size; ++i)
|
||||
{
|
||||
|
@ -198,22 +198,22 @@ void RegExpTreeDictionary::initRegexNodes(Block & block)
|
||||
Array keys = (*keys_column)[i].safeGet<Array>();
|
||||
Array values = (*values_column)[i].safeGet<Array>();
|
||||
size_t keys_size = keys.size();
|
||||
for (size_t i = 0; i < keys_size; i++)
|
||||
for (size_t j = 0; j < keys_size; j++)
|
||||
{
|
||||
const String & name = keys[i].safeGet<String>();
|
||||
const String & value = values[i].safeGet<String>();
|
||||
if (structure.hasAttribute(name))
|
||||
const String & name_ = keys[j].safeGet<String>();
|
||||
const String & value = values[j].safeGet<String>();
|
||||
if (structure.hasAttribute(name_))
|
||||
{
|
||||
const auto & attr = structure.getAttribute(name);
|
||||
const auto & attr = structure.getAttribute(name_);
|
||||
auto string_pieces = createStringPieces(value, num_captures, regex, logger);
|
||||
if (!string_pieces.empty())
|
||||
{
|
||||
node->attributes[name] = RegexTreeNode::AttributeValue{.field = values[i], .pieces = std::move(string_pieces)};
|
||||
node->attributes[name_] = RegexTreeNode::AttributeValue{.field = values[j], .pieces = std::move(string_pieces)};
|
||||
}
|
||||
else
|
||||
{
|
||||
Field field = parseStringToField(values[i].safeGet<String>(), attr.type);
|
||||
node->attributes[name] = RegexTreeNode::AttributeValue{.field = std::move(field)};
|
||||
Field field = parseStringToField(values[j].safeGet<String>(), attr.type);
|
||||
node->attributes[name_] = RegexTreeNode::AttributeValue{.field = std::move(field)};
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -424,23 +424,23 @@ bool RegExpTreeDictionary::setAttributes(
|
||||
return attributes_to_set.size() == attributes.size();
|
||||
visited_nodes.emplace(id);
|
||||
const auto & node_attributes = regex_nodes.at(id)->attributes;
|
||||
for (const auto & [name, value] : node_attributes)
|
||||
for (const auto & [name_, value] : node_attributes)
|
||||
{
|
||||
if (!attributes.contains(name) || attributes_to_set.contains(name))
|
||||
if (!attributes.contains(name_) || attributes_to_set.contains(name_))
|
||||
continue;
|
||||
if (value.containsBackRefs())
|
||||
{
|
||||
auto [updated_str, use_default] = processBackRefs(data, regex_nodes.at(id)->searcher, value.pieces);
|
||||
if (use_default)
|
||||
{
|
||||
DefaultValueProvider default_value(attributes.at(name).null_value, defaults.at(name));
|
||||
attributes_to_set[name] = default_value.getDefaultValue(key_index);
|
||||
DefaultValueProvider default_value(attributes.at(name_).null_value, defaults.at(name_));
|
||||
attributes_to_set[name_] = default_value.getDefaultValue(key_index);
|
||||
}
|
||||
else
|
||||
attributes_to_set[name] = parseStringToField(updated_str, attributes.at(name).type);
|
||||
attributes_to_set[name_] = parseStringToField(updated_str, attributes.at(name_).type);
|
||||
}
|
||||
else
|
||||
attributes_to_set[name] = value.field;
|
||||
attributes_to_set[name_] = value.field;
|
||||
}
|
||||
|
||||
auto parent_id = regex_nodes.at(id)->parent_id;
|
||||
@ -541,11 +541,11 @@ std::unordered_map<String, ColumnPtr> RegExpTreeDictionary::match(
|
||||
std::unordered_map<String, MutableColumnPtr> columns;
|
||||
|
||||
/// initialize columns
|
||||
for (const auto & [name, attr] : attributes)
|
||||
for (const auto & [name_, attr] : attributes)
|
||||
{
|
||||
auto col_ptr = attr.type->createColumn();
|
||||
col_ptr->reserve(keys_offsets.size());
|
||||
columns[name] = std::move(col_ptr);
|
||||
columns[name_] = std::move(col_ptr);
|
||||
}
|
||||
|
||||
UInt64 offset = 0;
|
||||
@ -628,25 +628,25 @@ std::unordered_map<String, ColumnPtr> RegExpTreeDictionary::match(
|
||||
break;
|
||||
}
|
||||
|
||||
for (const auto & [name, attr] : attributes)
|
||||
for (const auto & [name_, attr] : attributes)
|
||||
{
|
||||
if (attributes_to_set.contains(name))
|
||||
if (attributes_to_set.contains(name_))
|
||||
continue;
|
||||
|
||||
DefaultValueProvider default_value(attr.null_value, defaults.at(name));
|
||||
columns[name]->insert(default_value.getDefaultValue(key_idx));
|
||||
DefaultValueProvider default_value(attr.null_value, defaults.at(name_));
|
||||
columns[name_]->insert(default_value.getDefaultValue(key_idx));
|
||||
}
|
||||
|
||||
/// insert to columns
|
||||
for (const auto & [name, value] : attributes_to_set)
|
||||
columns[name]->insert(value);
|
||||
for (const auto & [name_, value] : attributes_to_set)
|
||||
columns[name_]->insert(value);
|
||||
|
||||
offset = key_offset;
|
||||
}
|
||||
|
||||
std::unordered_map<String, ColumnPtr> result;
|
||||
for (auto & [name, mutable_ptr] : columns)
|
||||
result.emplace(name, std::move(mutable_ptr));
|
||||
for (auto & [name_, mutable_ptr] : columns)
|
||||
result.emplace(name_, std::move(mutable_ptr));
|
||||
|
||||
return result;
|
||||
}
|
||||
@ -684,8 +684,8 @@ Columns RegExpTreeDictionary::getColumns(
|
||||
defaults);
|
||||
|
||||
Columns result;
|
||||
for (const String & name : attribute_names)
|
||||
result.push_back(columns_map.at(name));
|
||||
for (const String & name_ : attribute_names)
|
||||
result.push_back(columns_map.at(name_));
|
||||
|
||||
return result;
|
||||
}
|
||||
|
@ -229,23 +229,23 @@ void parseMatchNode(UInt64 parent_id, UInt64 & id, const YAML::Node & node, Resu
|
||||
{
|
||||
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Yaml match rule must contain key {}", key_name);
|
||||
}
|
||||
for (const auto & [key, node] : match)
|
||||
for (const auto & [key, node_] : match)
|
||||
{
|
||||
if (key == key_name)
|
||||
{
|
||||
if (!node.IsScalar())
|
||||
if (!node_.IsScalar())
|
||||
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "`{}` should be a String", key_name);
|
||||
|
||||
attributes_to_insert.reg_exp = node.as<String>();
|
||||
attributes_to_insert.reg_exp = node_.as<String>();
|
||||
}
|
||||
else if (structure.hasAttribute(key))
|
||||
{
|
||||
attributes_to_insert.keys.push_back(key);
|
||||
attributes_to_insert.values.push_back(node.as<String>());
|
||||
attributes_to_insert.values.push_back(node_.as<String>());
|
||||
}
|
||||
else if (node.IsSequence())
|
||||
else if (node_.IsSequence())
|
||||
{
|
||||
parseMatchList(attributes_to_insert.id, id, node, result, key_name, structure);
|
||||
parseMatchList(attributes_to_insert.id, id, node_, result, key_name, structure);
|
||||
}
|
||||
/// unknown attributes.
|
||||
}
|
||||
|
@ -1068,11 +1068,11 @@ public:
|
||||
FunctionDictGetDescendantsExecutable(
|
||||
String name_,
|
||||
size_t level_,
|
||||
DictionaryHierarchicalParentToChildIndexPtr hierarchical_parent_to_child_index,
|
||||
DictionaryHierarchicalParentToChildIndexPtr hierarchical_parent_to_child_index_,
|
||||
std::shared_ptr<FunctionDictHelper> dictionary_helper_)
|
||||
: name(std::move(name_))
|
||||
, level(level_)
|
||||
, hierarchical_parent_to_child_index(std::move(hierarchical_parent_to_child_index))
|
||||
, hierarchical_parent_to_child_index(std::move(hierarchical_parent_to_child_index_))
|
||||
, dictionary_helper(std::move(dictionary_helper_))
|
||||
{}
|
||||
|
||||
@ -1110,13 +1110,13 @@ public:
|
||||
const DataTypes & argument_types_,
|
||||
const DataTypePtr & result_type_,
|
||||
size_t level_,
|
||||
DictionaryHierarchicalParentToChildIndexPtr hierarchical_parent_to_child_index,
|
||||
DictionaryHierarchicalParentToChildIndexPtr hierarchical_parent_to_child_index_,
|
||||
std::shared_ptr<FunctionDictHelper> helper_)
|
||||
: name(std::move(name_))
|
||||
, argument_types(argument_types_)
|
||||
, result_type(result_type_)
|
||||
, level(level_)
|
||||
, hierarchical_parent_to_child_index(std::move(hierarchical_parent_to_child_index))
|
||||
, hierarchical_parent_to_child_index(std::move(hierarchical_parent_to_child_index_))
|
||||
, helper(std::move(helper_))
|
||||
{}
|
||||
|
||||
|
@ -1525,9 +1525,9 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const
|
||||
|
||||
asterisk = true;
|
||||
}
|
||||
else if (auto * function = (*expression)->as<ASTFunction>())
|
||||
else if (auto * func = (*expression)->as<ASTFunction>())
|
||||
{
|
||||
if (use_structure_from_insertion_table_in_table_functions == 2 && findIdentifier(function))
|
||||
if (use_structure_from_insertion_table_in_table_functions == 2 && findIdentifier(func))
|
||||
{
|
||||
use_columns_from_insert_query = false;
|
||||
break;
|
||||
@ -2074,9 +2074,9 @@ BackupsWorker & Context::getBackupsWorker() const
|
||||
const bool allow_concurrent_restores = this->getConfigRef().getBool("backups.allow_concurrent_restores", true);
|
||||
|
||||
const auto & config = getConfigRef();
|
||||
const auto & settings = getSettingsRef();
|
||||
UInt64 backup_threads = config.getUInt64("backup_threads", settings.backup_threads);
|
||||
UInt64 restore_threads = config.getUInt64("restore_threads", settings.restore_threads);
|
||||
const auto & settings_ = getSettingsRef();
|
||||
UInt64 backup_threads = config.getUInt64("backup_threads", settings_.backup_threads);
|
||||
UInt64 restore_threads = config.getUInt64("restore_threads", settings_.restore_threads);
|
||||
|
||||
if (!shared->backups_worker)
|
||||
shared->backups_worker.emplace(backup_threads, restore_threads, allow_concurrent_backups, allow_concurrent_restores);
|
||||
@ -4285,10 +4285,10 @@ ReadSettings Context::getReadSettings() const
|
||||
|
||||
ReadSettings Context::getBackupReadSettings() const
|
||||
{
|
||||
ReadSettings settings = getReadSettings();
|
||||
settings.remote_throttler = getBackupsThrottler();
|
||||
settings.local_throttler = getBackupsThrottler();
|
||||
return settings;
|
||||
ReadSettings settings_ = getReadSettings();
|
||||
settings_.remote_throttler = getBackupsThrottler();
|
||||
settings_.local_throttler = getBackupsThrottler();
|
||||
return settings_;
|
||||
}
|
||||
|
||||
WriteSettings Context::getWriteSettings() const
|
||||
@ -4317,14 +4317,14 @@ std::shared_ptr<AsyncReadCounters> Context::getAsyncReadCounters() const
|
||||
|
||||
Context::ParallelReplicasMode Context::getParallelReplicasMode() const
|
||||
{
|
||||
const auto & settings = getSettingsRef();
|
||||
const auto & settings_ = getSettingsRef();
|
||||
|
||||
using enum Context::ParallelReplicasMode;
|
||||
if (!settings.parallel_replicas_custom_key.value.empty())
|
||||
if (!settings_.parallel_replicas_custom_key.value.empty())
|
||||
return CUSTOM_KEY;
|
||||
|
||||
if (settings.allow_experimental_parallel_reading_from_replicas
|
||||
&& !settings.use_hedged_requests)
|
||||
if (settings_.allow_experimental_parallel_reading_from_replicas
|
||||
&& !settings_.use_hedged_requests)
|
||||
return READ_TASKS;
|
||||
|
||||
return SAMPLE_KEY;
|
||||
@ -4332,17 +4332,17 @@ Context::ParallelReplicasMode Context::getParallelReplicasMode() const
|
||||
|
||||
bool Context::canUseParallelReplicasOnInitiator() const
|
||||
{
|
||||
const auto & settings = getSettingsRef();
|
||||
const auto & settings_ = getSettingsRef();
|
||||
return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS
|
||||
&& settings.max_parallel_replicas > 1
|
||||
&& settings_.max_parallel_replicas > 1
|
||||
&& !getClientInfo().collaborate_with_initiator;
|
||||
}
|
||||
|
||||
bool Context::canUseParallelReplicasOnFollower() const
|
||||
{
|
||||
const auto & settings = getSettingsRef();
|
||||
const auto & settings_ = getSettingsRef();
|
||||
return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS
|
||||
&& settings.max_parallel_replicas > 1
|
||||
&& settings_.max_parallel_replicas > 1
|
||||
&& getClientInfo().collaborate_with_initiator;
|
||||
}
|
||||
|
||||
|
@ -1866,8 +1866,8 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
{
|
||||
ExpressionActionsChain::Step & step = *chain.steps.at(prewhere_step_num);
|
||||
|
||||
auto required_columns = prewhere_info->prewhere_actions->getRequiredColumnsNames();
|
||||
NameSet required_source_columns(required_columns.begin(), required_columns.end());
|
||||
auto required_columns_ = prewhere_info->prewhere_actions->getRequiredColumnsNames();
|
||||
NameSet required_source_columns(required_columns_.begin(), required_columns_.end());
|
||||
/// Add required columns to required output in order not to remove them after prewhere execution.
|
||||
/// TODO: add sampling and final execution to common chain.
|
||||
for (const auto & column : additional_required_columns_after_prewhere)
|
||||
|
@ -825,11 +825,11 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query)
|
||||
if (query.replica.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica name is empty");
|
||||
|
||||
auto check_not_local_replica = [](const DatabaseReplicated * replicated, const ASTSystemQuery & query)
|
||||
auto check_not_local_replica = [](const DatabaseReplicated * replicated, const ASTSystemQuery & query_)
|
||||
{
|
||||
if (!query.replica_zk_path.empty() && fs::path(replicated->getZooKeeperPath()) != fs::path(query.replica_zk_path))
|
||||
if (!query_.replica_zk_path.empty() && fs::path(replicated->getZooKeeperPath()) != fs::path(query_.replica_zk_path))
|
||||
return;
|
||||
if (replicated->getFullReplicaName() != query.replica)
|
||||
if (replicated->getFullReplicaName() != query_.replica)
|
||||
return;
|
||||
|
||||
throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED, "There is a local database {}, which has the same path in ZooKeeper "
|
||||
|
@ -301,7 +301,7 @@ void JoinedTables::rewriteDistributedInAndJoins(ASTPtr & query)
|
||||
}
|
||||
}
|
||||
|
||||
std::shared_ptr<TableJoin> JoinedTables::makeTableJoin(const ASTSelectQuery & select_query)
|
||||
std::shared_ptr<TableJoin> JoinedTables::makeTableJoin(const ASTSelectQuery & select_query_)
|
||||
{
|
||||
if (tables_with_columns.size() < 2)
|
||||
return {};
|
||||
@ -310,7 +310,7 @@ std::shared_ptr<TableJoin> JoinedTables::makeTableJoin(const ASTSelectQuery & se
|
||||
MultiEnum<JoinAlgorithm> join_algorithm = settings.join_algorithm;
|
||||
auto table_join = std::make_shared<TableJoin>(settings, context->getGlobalTemporaryVolume());
|
||||
|
||||
const ASTTablesInSelectQueryElement * ast_join = select_query.join();
|
||||
const ASTTablesInSelectQueryElement * ast_join = select_query_.join();
|
||||
const auto & table_to_join = ast_join->table_expression->as<ASTTableExpression &>();
|
||||
|
||||
/// TODO This syntax does not support specifying a database name.
|
||||
@ -352,16 +352,16 @@ std::shared_ptr<TableJoin> JoinedTables::makeTableJoin(const ASTSelectQuery & se
|
||||
|
||||
if (!table_join->isSpecialStorage() &&
|
||||
settings.enable_optimize_predicate_expression)
|
||||
replaceJoinedTable(select_query);
|
||||
replaceJoinedTable(select_query_);
|
||||
|
||||
return table_join;
|
||||
}
|
||||
|
||||
void JoinedTables::reset(const ASTSelectQuery & select_query)
|
||||
void JoinedTables::reset(const ASTSelectQuery & select_query_)
|
||||
{
|
||||
table_expressions = getTableExpressions(select_query);
|
||||
left_table_expression = extractTableExpression(select_query, 0);
|
||||
left_db_and_table = getDatabaseAndTable(select_query, 0);
|
||||
table_expressions = getTableExpressions(select_query_);
|
||||
left_table_expression = extractTableExpression(select_query_, 0);
|
||||
left_db_and_table = getDatabaseAndTable(select_query_, 0);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -607,8 +607,8 @@ std::shared_ptr<DirectKeyValueJoin> tryDirectJoin(const std::shared_ptr<TableJoi
|
||||
|
||||
for (const auto & right_table_expression_column : right_table_expression_header)
|
||||
{
|
||||
const auto * table_column_name = right_table_expression_data.getColumnNameOrNull(right_table_expression_column.name);
|
||||
if (!table_column_name)
|
||||
const auto * table_column_name_ = right_table_expression_data.getColumnNameOrNull(right_table_expression_column.name);
|
||||
if (!table_column_name_)
|
||||
return {};
|
||||
|
||||
auto right_table_expression_column_with_storage_column_name = right_table_expression_column;
|
||||
|
@ -647,9 +647,9 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes)
|
||||
}
|
||||
else
|
||||
{
|
||||
auto inner_blocks_query = getInnerBlocksQuery();
|
||||
auto inner_blocks_query_ = getInnerBlocksQuery();
|
||||
block_context->addExternalTable(getBlocksTableName(), std::move(blocks_storage_table_holder));
|
||||
InterpreterSelectQuery interpreter(inner_blocks_query,
|
||||
InterpreterSelectQuery interpreter(inner_blocks_query_,
|
||||
block_context,
|
||||
StoragePtr(),
|
||||
nullptr,
|
||||
|
@ -725,12 +725,12 @@ ASTPtr StorageWindowView::getSourceTableSelectQuery()
|
||||
|
||||
if (!is_time_column_func_now)
|
||||
{
|
||||
auto query = select_query->clone();
|
||||
auto query_ = select_query->clone();
|
||||
DropTableIdentifierMatcher::Data drop_table_identifier_data;
|
||||
DropTableIdentifierMatcher::Visitor(drop_table_identifier_data).visit(query);
|
||||
DropTableIdentifierMatcher::Visitor(drop_table_identifier_data).visit(query_);
|
||||
|
||||
WindowFunctionMatcher::Data query_info_data;
|
||||
WindowFunctionMatcher::Visitor(query_info_data).visit(query);
|
||||
WindowFunctionMatcher::Visitor(query_info_data).visit(query_);
|
||||
|
||||
auto order_by = std::make_shared<ASTExpressionList>();
|
||||
auto order_by_elem = std::make_shared<ASTOrderByElement>();
|
||||
@ -749,12 +749,12 @@ ASTPtr StorageWindowView::getSourceTableSelectQuery()
|
||||
return select_with_union_query;
|
||||
}
|
||||
|
||||
ASTPtr StorageWindowView::getInnerTableCreateQuery(const ASTPtr & inner_query, const StorageID & inner_table_id)
|
||||
ASTPtr StorageWindowView::getInnerTableCreateQuery(const ASTPtr & inner_query, const StorageID & inner_table_id_)
|
||||
{
|
||||
/// We will create a query to create an internal table.
|
||||
auto inner_create_query = std::make_shared<ASTCreateQuery>();
|
||||
inner_create_query->setDatabase(inner_table_id.getDatabaseName());
|
||||
inner_create_query->setTable(inner_table_id.getTableName());
|
||||
inner_create_query->setDatabase(inner_table_id_.getDatabaseName());
|
||||
inner_create_query->setTable(inner_table_id_.getTableName());
|
||||
|
||||
Aliases aliases;
|
||||
QueryAliasesVisitor(aliases).visit(inner_query);
|
||||
@ -1208,7 +1208,7 @@ StorageWindowView::StorageWindowView(
|
||||
if (has_inner_target_table)
|
||||
{
|
||||
/// create inner target table
|
||||
auto create_context = Context::createCopy(context_);
|
||||
auto create_context_ = Context::createCopy(context_);
|
||||
auto target_create_query = std::make_shared<ASTCreateQuery>();
|
||||
target_create_query->setDatabase(table_id_.database_name);
|
||||
target_create_query->setTable(generateTargetTableName(table_id_));
|
||||
@ -1219,9 +1219,9 @@ StorageWindowView::StorageWindowView(
|
||||
target_create_query->set(target_create_query->columns_list, new_columns_list);
|
||||
target_create_query->set(target_create_query->storage, query.storage->ptr());
|
||||
|
||||
InterpreterCreateQuery create_interpreter(target_create_query, create_context);
|
||||
create_interpreter.setInternal(true);
|
||||
create_interpreter.execute();
|
||||
InterpreterCreateQuery create_interpreter_(target_create_query, create_context_);
|
||||
create_interpreter_.setInternal(true);
|
||||
create_interpreter_.execute();
|
||||
}
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user