Better names

This commit is contained in:
kssenii 2021-04-02 16:12:14 +00:00
parent bcc37cd587
commit 2de6b99f84
9 changed files with 15 additions and 15 deletions

View File

@ -370,7 +370,7 @@ class IColumn;
\
M(UInt64, postgresql_connection_pool_size, 16, "Connection pool size for PostgreSQL table engine and database engine.", 0) \
M(Int64, postgresql_connection_pool_wait_timeout, -1, "Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.", 0) \
M(UInt64, storage_external_distributed_max_addresses, 1000, "Maximum number of addresses for storage ExternalDistributed.", 0) \
M(UInt64, glob_expansion_max_elements, 1000, "Maximum number of allowed addresses (For external storages, table functions, etc).", 0) \
\
M(Seconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.", 0) \
M(UInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up an incredible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.", 0) \

View File

@ -145,7 +145,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
{
auto mysql_database_settings = std::make_unique<ConnectionMySQLSettings>();
/// Split into replicas if needed.
size_t max_addresses = context.getSettingsRef().storage_external_distributed_max_addresses;
size_t max_addresses = context.getSettingsRef().glob_expansion_max_elements;
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
auto mysql_pool = mysqlxx::PoolWithFailover(mysql_database_name, addresses, mysql_user_name, mysql_user_password);
@ -249,7 +249,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
use_table_cache = safeGetLiteralValue<UInt64>(engine_args[4], engine_name);
/// Split into replicas if needed.
size_t max_addresses = context.getSettingsRef().storage_external_distributed_max_addresses;
size_t max_addresses = context.getSettingsRef().glob_expansion_max_elements;
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
/// no connection is made here

View File

@ -119,7 +119,7 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate()
}
}
BlockInputStreamPtr MySQLDictionarySource::loadBase(const String & query)
BlockInputStreamPtr MySQLDictionarySource::loadFromQuery(const String & query)
{
return std::make_shared<MySQLWithFailoverBlockInputStream>(
pool, query, sample_block, max_block_size, close_connection, false, max_tries_for_mysql_block_input_stream);
@ -131,7 +131,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadAll()
last_modification = getLastModification(connection, false);
LOG_TRACE(log, load_all_query);
return loadBase(load_all_query);
return loadFromQuery(load_all_query);
}
BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll()
@ -141,21 +141,21 @@ BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll()
std::string load_update_query = getUpdateFieldAndDate();
LOG_TRACE(log, load_update_query);
return loadBase(load_update_query);
return loadFromQuery(load_update_query);
}
BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
const auto query = query_builder.composeLoadIdsQuery(ids);
return loadBase(query);
return loadFromQuery(query);
}
BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN);
return loadBase(query);
return loadFromQuery(query);
}
bool MySQLDictionarySource::isModified() const

View File

@ -49,8 +49,6 @@ public:
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
BlockInputStreamPtr loadBase(const String & query);
bool isModified() const override;
bool supportsSelectiveLoad() const override;
@ -62,6 +60,8 @@ public:
std::string toString() const override;
private:
BlockInputStreamPtr loadFromQuery(const String & query);
std::string getUpdateFieldAndDate();
static std::string quoteForLike(const std::string s);

View File

@ -44,7 +44,7 @@ StorageExternalDistributed::StorageExternalDistributed(
storage_metadata.setConstraints(constraints_);
setInMemoryMetadata(storage_metadata);
size_t max_addresses = context.getSettingsRef().storage_external_distributed_max_addresses;
size_t max_addresses = context.getSettingsRef().glob_expansion_max_elements;
std::vector<String> shards_descriptions = parseRemoteDescription(cluster_description, 0, cluster_description.size(), ',', max_addresses);
std::vector<std::pair<std::string, UInt16>> addresses;

View File

@ -236,7 +236,7 @@ void registerStorageMySQL(StorageFactory & factory)
const String & remote_table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
const String & username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
const String & password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
size_t max_addresses = args.context.getSettingsRef().storage_external_distributed_max_addresses;
size_t max_addresses = args.context.getSettingsRef().glob_expansion_max_elements;
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
mysqlxx::PoolWithFailover pool(remote_database, addresses, username, password);

View File

@ -308,7 +308,7 @@ void registerStoragePostgreSQL(StorageFactory & factory)
auto host_port = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
/// Split into replicas if needed.
size_t max_addresses = args.context.getSettingsRef().storage_external_distributed_max_addresses;
size_t max_addresses = args.context.getSettingsRef().glob_expansion_max_elements;
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
const String & remote_database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();

View File

@ -61,7 +61,7 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Conte
password = args[4]->as<ASTLiteral &>().value.safeGet<String>();
/// Split into replicas if needed. 3306 is the default MySQL port number
size_t max_addresses = context.getSettingsRef().storage_external_distributed_max_addresses;
size_t max_addresses = context.getSettingsRef().glob_expansion_max_elements;
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
pool.emplace(remote_database_name, addresses, user_name, password);

View File

@ -70,7 +70,7 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, const
/// Split into replicas if needed. 5432 is a default postgresql port.
const auto & host_port = args[0]->as<ASTLiteral &>().value.safeGet<String>();
size_t max_addresses = context.getSettingsRef().storage_external_distributed_max_addresses;
size_t max_addresses = context.getSettingsRef().glob_expansion_max_elements;
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
remote_table_name = args[2]->as<ASTLiteral &>().value.safeGet<String>();