Hide Settings implementation

This commit is contained in:
Raúl Marín 2024-09-18 14:20:53 +02:00
parent 7fd2207626
commit f3df7bd171
481 changed files with 8642 additions and 4857 deletions

View File

@ -19,6 +19,7 @@
#include <Common/Exception.h>
#include <Common/randomSeed.h>
#include <Common/clearPasswordFromCommandLine.h>
#include <Core/Settings.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadHelpers.h>
@ -36,7 +37,6 @@
#include <Common/StudentTTest.h>
#include <Common/CurrentMetrics.h>
#include <Common/ErrorCodes.h>
#include <Core/BaseSettingsProgramOptions.h>
/** A tool for evaluating ClickHouse performance.
@ -58,8 +58,9 @@ static constexpr std::string_view DEFAULT_CLIENT_NAME = "benchmark";
namespace ErrorCodes
{
extern const int CANNOT_BLOCK_SIGNAL;
extern const int EMPTY_DATA_PASSED;
extern const int BAD_ARGUMENTS;
extern const int CANNOT_BLOCK_SIGNAL;
extern const int EMPTY_DATA_PASSED;
}
class Benchmark : public Poco::Util::Application
@ -637,7 +638,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
;
Settings settings;
addProgramOptions(settings, desc);
settings.addToProgramOptions(desc);
boost::program_options::variables_map options;
boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options);

View File

@ -56,6 +56,12 @@ using namespace std::literals;
namespace DB
{
namespace Setting
{
extern const SettingsDialect dialect;
extern const SettingsBool use_client_time_zone;
}
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
@ -307,9 +313,9 @@ void Client::initialize(Poco::Util::Application & self)
config().setString("password", env_password);
/// settings and limits could be specified in config file, but passed settings has higher priority
for (const auto & setting : global_context->getSettingsRef().allUnchanged())
for (const auto & setting : global_context->getSettingsRef().getUnchangedNames())
{
const auto & name = setting.getName();
String name{setting};
if (config().has(name))
global_context->setSetting(name, config().getString(name));
}
@ -525,7 +531,7 @@ void Client::connect()
}
}
if (!client_context->getSettingsRef().use_client_time_zone)
if (!client_context->getSettingsRef()[Setting::use_client_time_zone])
{
const auto & time_zone = connection->getServerTimezone(connection_parameters.timeouts);
if (!time_zone.empty())
@ -730,7 +736,7 @@ bool Client::processWithFuzzing(const String & full_query)
}
// Kusto is not a subject for fuzzing (yet)
if (client_context->getSettingsRef().dialect == DB::Dialect::kusto)
if (client_context->getSettingsRef()[Setting::dialect] == DB::Dialect::kusto)
{
return true;
}
@ -1073,17 +1079,7 @@ void Client::processOptions(const OptionsDescription & options_description,
/// Copy settings-related program options to config.
/// TODO: Is this code necessary?
for (const auto & setting : global_context->getSettingsRef().all())
{
const auto & name = setting.getName();
if (options.count(name))
{
if (allow_repeated_settings)
config().setString(name, options[name].as<Strings>().back());
else
config().setString(name, options[name].as<String>());
}
}
global_context->getSettingsRef().addToClientOptions(config(), options, allow_repeated_settings);
if (options.count("config-file") && options.count("config"))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Two or more configuration files referenced in arguments");

View File

@ -3,11 +3,12 @@
#include <string_view>
#include <boost/program_options.hpp>
#include <IO/copyData.h>
#include <Core/Settings.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/copyData.h>
#include <Interpreters/registerInterpreters.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ParserQuery.h>
@ -17,7 +18,6 @@
#include <Common/ErrorCodes.h>
#include <Common/StringUtils.h>
#include <Common/TerminalSize.h>
#include <Core/BaseSettingsProgramOptions.h>
#include <Interpreters/Context.h>
#include <Functions/FunctionFactory.h>
@ -35,6 +35,15 @@
#include <Formats/registerFormats.h>
#include <Processors/Transforms/getSourceFromASTInsertQuery.h>
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
extern const SettingsUInt64 max_query_size;
}
}
namespace DB::ErrorCodes
{
@ -99,12 +108,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
;
Settings cmd_settings;
for (const auto & field : cmd_settings.all())
{
std::string_view name = field.getName();
if (name == "max_parser_depth" || name == "max_query_size")
addProgramOption(cmd_settings, desc, name, field);
}
cmd_settings.addToProgramOptions("max_parser_depth", desc);
cmd_settings.addToProgramOptions("max_query_size", desc);
boost::program_options::variables_map options;
boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options);
@ -240,7 +245,14 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
size_t approx_query_length = multiple ? find_first_symbols<';'>(pos, end) - pos : end - pos;
ASTPtr res = parseQueryAndMovePosition(
parser, pos, end, "query", multiple, cmd_settings.max_query_size, cmd_settings.max_parser_depth, cmd_settings.max_parser_backtracks);
parser,
pos,
end,
"query",
multiple,
cmd_settings[Setting::max_query_size],
cmd_settings[Setting::max_parser_depth],
cmd_settings[Setting::max_parser_backtracks]);
std::unique_ptr<ReadBuffer> insert_query_payload;
/// If the query is INSERT ... VALUES, then we will try to parse the data.

View File

@ -71,6 +71,11 @@ namespace CurrentMetrics
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_introspection_functions;
extern const SettingsLocalFSReadMethod storage_file_read_method;
}
namespace ErrorCodes
{
@ -83,8 +88,8 @@ void applySettingsOverridesForLocal(ContextMutablePtr context)
{
Settings settings = context->getSettingsCopy();
settings.allow_introspection_functions = true;
settings.storage_file_read_method = LocalFSReadMethod::mmap;
settings[Setting::allow_introspection_functions] = true;
settings[Setting::storage_file_read_method] = LocalFSReadMethod::mmap;
context->setSettings(settings);
}

View File

@ -4,7 +4,6 @@
#include <Client/LocalConnection.h>
#include <Core/ServerSettings.h>
#include <Core/Settings.h>
#include <Interpreters/Context.h>
#include <Loggers/Loggers.h>
#include <Common/InterruptListener.h>

View File

@ -2,6 +2,7 @@
#if USE_ODBC
#include <Core/NamesAndTypes.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeNullable.h>
@ -27,6 +28,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 odbc_bridge_connection_pool_size;
}
namespace ErrorCodes
{
@ -129,8 +134,7 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ
const bool external_table_functions_use_nulls = Poco::NumberParser::parseBool(params.get("external_table_functions_use_nulls", "false"));
auto connection_holder = ODBCPooledConnectionFactory::instance().get(
validateODBCConnectionString(connection_string),
getContext()->getSettingsRef().odbc_bridge_connection_pool_size);
validateODBCConnectionString(connection_string), getContext()->getSettingsRef()[Setting::odbc_bridge_connection_pool_size]);
/// In XDBC tables it is allowed to pass either database_name or schema_name in table definion, but not both of them.
/// They both are passed as 'schema' parameter in request URL, so it is not clear whether it is database_name or schema_name passed.

View File

@ -19,6 +19,11 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 odbc_bridge_connection_pool_size;
}
void IdentifierQuoteHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & /*write_event*/)
{
HTMLForm params(getContext()->getSettingsRef(), request, request.getStream());
@ -68,7 +73,7 @@ void IdentifierQuoteHandler::handleRequest(HTTPServerRequest & request, HTTPServ
nanodbc::ConnectionHolderPtr connection;
if (use_connection_pooling)
connection = ODBCPooledConnectionFactory::instance().get(
validateODBCConnectionString(connection_string), getContext()->getSettingsRef().odbc_bridge_connection_pool_size);
validateODBCConnectionString(connection_string), getContext()->getSettingsRef()[Setting::odbc_bridge_connection_pool_size]);
else
connection = std::make_shared<nanodbc::ConnectionHolder>(validateODBCConnectionString(connection_string));

View File

@ -1,27 +1,28 @@
#include "MainHandler.h"
#include "validateODBCConnectionString.h"
#include "ODBCSource.h"
#include "ODBCSink.h"
#include "getIdentifierQuote.h"
#include <Core/Settings.h>
#include <DataTypes/DataTypeFactory.h>
#include <Formats/FormatFactory.h>
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <Core/Settings.h>
#include <IO/Operators.h>
#include <IO/ReadBufferFromIStream.h>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/Net/HTMLForm.h>
#include <Poco/ThreadPool.h>
#include <QueryPipeline/QueryPipeline.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Processors/Executors/CompletedPipelineExecutor.h>
#include <Processors/Formats/IInputFormat.h>
#include <QueryPipeline/QueryPipeline.h>
#include <Server/HTTP/HTMLForm.h>
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
#include <Poco/Net/HTMLForm.h>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/ThreadPool.h>
#include <Common/BridgeProtocolVersion.h>
#include <Common/logger_useful.h>
#include <Server/HTTP/HTMLForm.h>
#include "ODBCSink.h"
#include "ODBCSource.h"
#include "config.h"
#include "getIdentifierQuote.h"
#include "validateODBCConnectionString.h"
#include <mutex>
#include <memory>
@ -29,6 +30,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 odbc_bridge_connection_pool_size;
}
namespace
{
@ -139,7 +144,7 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
nanodbc::ConnectionHolderPtr connection_handler;
if (use_connection_pooling)
connection_handler = ODBCPooledConnectionFactory::instance().get(
validateODBCConnectionString(connection_string), getContext()->getSettingsRef().odbc_bridge_connection_pool_size);
validateODBCConnectionString(connection_string), getContext()->getSettingsRef()[Setting::odbc_bridge_connection_pool_size]);
else
connection_handler = std::make_shared<nanodbc::ConnectionHolder>(validateODBCConnectionString(connection_string));

View File

@ -20,6 +20,11 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 odbc_bridge_connection_pool_size;
}
namespace
{
bool isSchemaAllowed(nanodbc::ConnectionHolderPtr connection_holder)
@ -82,7 +87,7 @@ void SchemaAllowedHandler::handleRequest(HTTPServerRequest & request, HTTPServer
if (use_connection_pooling)
connection = ODBCPooledConnectionFactory::instance().get(
validateODBCConnectionString(connection_string), getContext()->getSettingsRef().odbc_bridge_connection_pool_size);
validateODBCConnectionString(connection_string), getContext()->getSettingsRef()[Setting::odbc_bridge_connection_pool_size]);
else
connection = std::make_shared<nanodbc::ConnectionHolder>(validateODBCConnectionString(connection_string));

View File

@ -148,6 +148,18 @@
/// A minimal file used when the server is run without installation
INCBIN(resource_embedded_xml, SOURCE_DIR "/programs/server/embedded.xml");
namespace DB
{
namespace Setting
{
extern const SettingsSeconds http_receive_timeout;
extern const SettingsSeconds http_send_timeout;
extern const SettingsSeconds receive_timeout;
extern const SettingsSeconds send_timeout;
}
}
namespace CurrentMetrics
{
extern const Metric Revision;
@ -1831,10 +1843,13 @@ try
"Keeper (tcp): " + address.toString(),
std::make_unique<TCPServer>(
new KeeperTCPHandlerFactory(
config_getter, global_context->getKeeperDispatcher(),
global_context->getSettingsRef().receive_timeout.totalSeconds(),
global_context->getSettingsRef().send_timeout.totalSeconds(),
false), server_pool, socket));
config_getter,
global_context->getKeeperDispatcher(),
global_context->getSettingsRef()[Setting::receive_timeout].totalSeconds(),
global_context->getSettingsRef()[Setting::send_timeout].totalSeconds(),
false),
server_pool,
socket));
});
const char * secure_port_name = "keeper_server.tcp_port_secure";
@ -1854,9 +1869,13 @@ try
"Keeper with secure protocol (tcp_secure): " + address.toString(),
std::make_unique<TCPServer>(
new KeeperTCPHandlerFactory(
config_getter, global_context->getKeeperDispatcher(),
global_context->getSettingsRef().receive_timeout.totalSeconds(),
global_context->getSettingsRef().send_timeout.totalSeconds(), true), server_pool, socket));
config_getter,
global_context->getKeeperDispatcher(),
global_context->getSettingsRef()[Setting::receive_timeout].totalSeconds(),
global_context->getSettingsRef()[Setting::send_timeout].totalSeconds(),
true),
server_pool,
socket));
#else
UNUSED(port);
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.");
@ -2430,7 +2449,7 @@ void Server::createServers(
const Settings & settings = global_context->getSettingsRef();
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
http_params->setTimeout(settings.http_receive_timeout);
http_params->setTimeout(settings[Setting::http_receive_timeout]);
http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout);
http_params->setMaxKeepAliveRequests(static_cast<int>(global_context->getServerSettings().max_keep_alive_requests));
@ -2469,8 +2488,8 @@ void Server::createServers(
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(config, socket, host, port, is_secure);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
socket.setReceiveTimeout(settings[Setting::receive_timeout]);
socket.setSendTimeout(settings[Setting::send_timeout]);
return ProtocolServerAdapter(
host,
@ -2497,8 +2516,8 @@ void Server::createServers(
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(config, socket, listen_host, port);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
socket.setReceiveTimeout(settings[Setting::http_receive_timeout]);
socket.setSendTimeout(settings[Setting::http_send_timeout]);
return ProtocolServerAdapter(
listen_host,
@ -2518,8 +2537,8 @@ void Server::createServers(
#if USE_SSL
Poco::Net::SecureServerSocket socket;
auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
socket.setReceiveTimeout(settings[Setting::http_receive_timeout]);
socket.setSendTimeout(settings[Setting::http_send_timeout]);
return ProtocolServerAdapter(
listen_host,
port_name,
@ -2541,8 +2560,8 @@ void Server::createServers(
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(config, socket, listen_host, port);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
socket.setReceiveTimeout(settings[Setting::receive_timeout]);
socket.setSendTimeout(settings[Setting::send_timeout]);
return ProtocolServerAdapter(
listen_host,
port_name,
@ -2563,8 +2582,8 @@ void Server::createServers(
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(config, socket, listen_host, port);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
socket.setReceiveTimeout(settings[Setting::receive_timeout]);
socket.setSendTimeout(settings[Setting::send_timeout]);
return ProtocolServerAdapter(
listen_host,
port_name,
@ -2586,8 +2605,8 @@ void Server::createServers(
#if USE_SSL
Poco::Net::SecureServerSocket socket;
auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
socket.setReceiveTimeout(settings[Setting::receive_timeout]);
socket.setSendTimeout(settings[Setting::send_timeout]);
return ProtocolServerAdapter(
listen_host,
port_name,
@ -2612,7 +2631,7 @@ void Server::createServers(
Poco::Net::ServerSocket socket;
auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(Poco::Timespan());
socket.setSendTimeout(settings.send_timeout);
socket.setSendTimeout(settings[Setting::send_timeout]);
return ProtocolServerAdapter(
listen_host,
port_name,
@ -2629,7 +2648,7 @@ void Server::createServers(
Poco::Net::ServerSocket socket;
auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(Poco::Timespan());
socket.setSendTimeout(settings.send_timeout);
socket.setSendTimeout(settings[Setting::send_timeout]);
return ProtocolServerAdapter(
listen_host,
port_name,
@ -2661,8 +2680,8 @@ void Server::createServers(
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(config, socket, listen_host, port);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
socket.setReceiveTimeout(settings[Setting::http_receive_timeout]);
socket.setSendTimeout(settings[Setting::http_send_timeout]);
return ProtocolServerAdapter(
listen_host,
port_name,
@ -2687,7 +2706,7 @@ void Server::createInterserverServers(
const Settings & settings = global_context->getSettingsRef();
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
http_params->setTimeout(settings.http_receive_timeout);
http_params->setTimeout(settings[Setting::http_receive_timeout]);
http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout);
/// Now iterate over interserver_listen_hosts
@ -2703,8 +2722,8 @@ void Server::createInterserverServers(
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(config, socket, interserver_listen_host, port);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
socket.setReceiveTimeout(settings[Setting::http_receive_timeout]);
socket.setSendTimeout(settings[Setting::http_send_timeout]);
return ProtocolServerAdapter(
interserver_listen_host,
port_name,
@ -2728,8 +2747,8 @@ void Server::createInterserverServers(
#if USE_SSL
Poco::Net::SecureServerSocket socket;
auto address = socketBindListen(config, socket, interserver_listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
socket.setReceiveTimeout(settings[Setting::http_receive_timeout]);
socket.setSendTimeout(settings[Setting::http_send_timeout]);
return ProtocolServerAdapter(
interserver_listen_host,
port_name,

View File

@ -1,10 +1,17 @@
#include <Access/ContextAccessParams.h>
#include <Core/Settings.h>
#include <IO/Operators.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_ddl;
extern const SettingsBool allow_introspection_functions;
extern const SettingsUInt64 readonly;
}
ContextAccessParams::ContextAccessParams(
std::optional<UUID> user_id_,
@ -18,9 +25,9 @@ ContextAccessParams::ContextAccessParams(
, full_access(full_access_)
, use_default_roles(use_default_roles_)
, current_roles(current_roles_)
, readonly(settings_.readonly)
, allow_ddl(settings_.allow_ddl)
, allow_introspection(settings_.allow_introspection_functions)
, readonly(settings_[Setting::readonly])
, allow_ddl(settings_[Setting::allow_ddl])
, allow_introspection(settings_[Setting::allow_introspection_functions])
, current_database(current_database_)
, interface(client_info_.interface)
, http_method(client_info_.http_method)

View File

@ -10,10 +10,15 @@
#include <Common/SettingSource.h>
#include <IO/WriteHelpers.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <boost/range/algorithm_ext/erase.hpp>
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_ddl;
extern const SettingsUInt64 readonly;
}
namespace ErrorCodes
{
extern const int READONLY;
@ -164,7 +169,7 @@ void SettingsConstraints::check(const Settings & current_settings, const Setting
if (element.writability)
new_value = *element.writability;
auto setting_name = Settings::Traits::resolveName(element.setting_name);
auto setting_name = Settings::resolveName(element.setting_name);
auto it = constraints.find(setting_name);
if (it != constraints.end())
old_value = it->second.writability;
@ -255,7 +260,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings,
ReactionOnViolation reaction,
SettingSource source) const
{
std::string_view setting_name = Settings::Traits::resolveName(change.name);
std::string_view setting_name = Settings::resolveName(change.name);
if (setting_name == "profile")
return true;
@ -393,7 +398,7 @@ std::string_view SettingsConstraints::resolveSettingNameWithCache(std::string_vi
SettingsConstraints::Checker SettingsConstraints::getChecker(const Settings & current_settings, std::string_view setting_name) const
{
auto resolved_name = resolveSettingNameWithCache(setting_name);
if (!current_settings.allow_ddl && resolved_name == "allow_ddl")
if (!current_settings[Setting::allow_ddl] && resolved_name == "allow_ddl")
return Checker(PreformattedMessage::create("Cannot modify 'allow_ddl' setting when DDL queries are prohibited for the user"),
ErrorCodes::QUERY_IS_PROHIBITED);
@ -403,11 +408,11 @@ SettingsConstraints::Checker SettingsConstraints::getChecker(const Settings & cu
* 2 - only read requests, as well as changing settings, except for the `readonly` setting.
*/
if (current_settings.readonly > 1 && resolved_name == "readonly")
if (current_settings[Setting::readonly] > 1 && resolved_name == "readonly")
return Checker(PreformattedMessage::create("Cannot modify 'readonly' setting in readonly mode"), ErrorCodes::READONLY);
auto it = constraints.find(resolved_name);
if (current_settings.readonly == 1)
if (current_settings[Setting::readonly] == 1)
{
if (it == constraints.end() || it->second.writability != SettingConstraintWritability::CHANGEABLE_IN_READONLY)
return Checker(PreformattedMessage::create("Cannot modify '{}' setting in readonly mode", setting_name),
@ -416,9 +421,9 @@ SettingsConstraints::Checker SettingsConstraints::getChecker(const Settings & cu
else // For both readonly=0 and readonly=2
{
if (it == constraints.end())
return Checker(Settings::Traits::resolveName); // Allowed
return Checker(Settings::resolveName); // Allowed
}
return Checker(it->second, Settings::Traits::resolveName);
return Checker(it->second, Settings::resolveName);
}
SettingsConstraints::Checker SettingsConstraints::getMergeTreeChecker(std::string_view short_name) const

View File

@ -249,7 +249,7 @@ bool SettingsProfileElements::isBackupAllowed() const
bool SettingsProfileElements::isAllowBackupSetting(const String & setting_name)
{
static constexpr std::string_view ALLOW_BACKUP_SETTING_NAME = "allow_backup";
return Settings::Traits::resolveName(setting_name) == ALLOW_BACKUP_SETTING_NAME;
return Settings::resolveName(setting_name) == ALLOW_BACKUP_SETTING_NAME;
}
}

View File

@ -46,14 +46,6 @@ inline Field settingCastValueUtil(std::string_view full_name, const Field & valu
});
}
inline String settingValueToStringUtil(std::string_view full_name, const Field & value)
{
return resolveSetting(full_name, [&] <typename T> (std::string_view short_name, SettingsType<T>)
{
return T::valueToStringUtil(short_name, value);
});
}
inline Field settingStringToValueUtil(std::string_view full_name, const String & str)
{
return resolveSetting(full_name, [&] <typename T> (std::string_view short_name, SettingsType<T>)
@ -89,10 +81,9 @@ inline String settingFullName<MergeTreeSettings>(std::string_view short_name)
inline std::string resolveSettingName(std::string_view full_name)
{
return resolveSetting(full_name, [&] <typename T> (std::string_view short_name, SettingsType<T>)
{
return settingFullName<T>(T::Traits::resolveName(short_name));
});
return resolveSetting(
full_name,
[&]<typename T>(std::string_view short_name, SettingsType<T>) { return settingFullName<T>(T::resolveName(short_name)); });
}
}

View File

@ -14,6 +14,10 @@ static constexpr size_t MAX_AGGREGATE_FUNCTION_NAME_LENGTH = 1000;
namespace DB
{
struct Settings;
namespace Setting
{
extern const SettingsBool log_queries;
}
namespace ErrorCodes
{
@ -199,7 +203,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(
found = *opt;
out_properties = found.properties;
if (query_context && query_context->getSettingsRef().log_queries)
if (query_context && query_context->getSettingsRef()[Setting::log_queries])
query_context->addQueryFactoriesInfo(
Context::QueryLogFactories::AggregateFunction, is_case_insensitive ? case_insensitive_name : name);
@ -224,7 +228,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(
"Aggregate function combinator '{}' is only for internal usage",
combinator_name);
if (query_context && query_context->getSettingsRef().log_queries)
if (query_context && query_context->getSettingsRef()[Setting::log_queries])
query_context->addQueryFactoriesInfo(Context::QueryLogFactories::AggregateFunctionCombinator, combinator_name);
String nested_name = name.substr(0, name.size() - combinator_name.size());

View File

@ -17,6 +17,12 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_introspection_functions;
}
namespace ErrorCodes
{
extern const int FUNCTION_NOT_ALLOWED;
@ -628,7 +634,7 @@ static void check(const std::string & name, const DataTypes & argument_types, co
AggregateFunctionPtr createAggregateFunctionFlameGraph(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings * settings)
{
if (!settings->allow_introspection_functions)
if (!(*settings)[Setting::allow_introspection_functions])
throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED,
"Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0");

View File

@ -18,15 +18,19 @@
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <base/range.h>
#include <bitset>
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_experimental_funnel_functions;
}
constexpr size_t max_events_size = 64;
constexpr size_t min_required_args = 3;
namespace ErrorCodes
@ -448,7 +452,7 @@ inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(
AggregateFunctionPtr
createAggregateFunctionSequenceNode(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
if (settings == nullptr || !settings->allow_experimental_funnel_functions)
if (settings == nullptr || !(*settings)[Setting::allow_experimental_funnel_functions])
{
throw Exception(ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION, "Aggregate function {} is experimental. "
"Set `allow_experimental_funnel_functions` setting to enable it", name);

View File

@ -15,7 +15,10 @@
namespace DB
{
struct Settings;
namespace Setting
{
extern const SettingsMaxThreads max_threads;
}
namespace ErrorCodes
{
@ -149,7 +152,7 @@ void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory)
auto assign_bool_param = [](const std::string & name, const DataTypes & argument_types, const Array & params, const Settings * settings)
{
/// Using two level hash set if we wouldn't be able to merge in parallel can cause ~10% slowdown.
if (settings && settings->max_threads > 1)
if (settings && (*settings)[Setting::max_threads] > 1)
return createAggregateFunctionUniq<
true, AggregateFunctionUniqExactData, AggregateFunctionUniqExactDataForVariadic, true /* is_able_to_parallelize_merge */>(name, argument_types, params, settings);
else

View File

@ -3,6 +3,7 @@
#include <Core/Settings.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <base/range.h>
#include <unordered_set>
#include <Columns/ColumnsNumber.h>

View File

@ -14,6 +14,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_aggregators_of_group_by_keys;
}
namespace ErrorCodes
{
@ -34,7 +38,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_aggregators_of_group_by_keys)
if (!getSettings()[Setting::optimize_aggregators_of_group_by_keys])
return;
/// Collect group by keys.
@ -79,7 +83,7 @@ public:
/// Now we visit all nodes in QueryNode, we should remove group_by_keys from stack.
void leaveImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_aggregators_of_group_by_keys)
if (!getSettings()[Setting::optimize_aggregators_of_group_by_keys])
return;
if (node->getNodeType() == QueryTreeNodeType::FUNCTION)

View File

@ -15,6 +15,11 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_arithmetic_operations_in_aggregate_functions;
}
namespace ErrorCodes
{
@ -56,7 +61,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_arithmetic_operations_in_aggregate_functions)
if (!getSettings()[Setting::optimize_arithmetic_operations_in_aggregate_functions])
return;
auto * aggregate_function_node = node->as<FunctionNode>();

View File

@ -15,6 +15,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_rewrite_array_exists_to_has;
}
namespace
{
@ -27,7 +31,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_rewrite_array_exists_to_has)
if (!getSettings()[Setting::optimize_rewrite_array_exists_to_has])
return;
auto * array_exists_function_node = node->as<FunctionNode>();

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool final;
}
namespace
{
@ -24,7 +28,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().final)
if (!getSettings()[Setting::final])
return;
const auto * query_node = node->as<QueryNode>();

View File

@ -26,6 +26,14 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_hyperscan;
extern const SettingsUInt64 max_hyperscan_regexp_length;
extern const SettingsUInt64 max_hyperscan_regexp_total_length;
extern const SettingsBool reject_expensive_hyperscan_regexps;
extern const SettingsBool optimize_or_like_chain;
}
namespace
{
@ -48,10 +56,8 @@ public:
{
const auto & settings = getSettings();
return settings.optimize_or_like_chain
&& settings.allow_hyperscan
&& settings.max_hyperscan_regexp_length == 0
&& settings.max_hyperscan_regexp_total_length == 0;
return settings[Setting::optimize_or_like_chain] && settings[Setting::allow_hyperscan] && settings[Setting::max_hyperscan_regexp_length] == 0
&& settings[Setting::max_hyperscan_regexp_total_length] == 0;
}
void enterImpl(QueryTreeNodePtr & node)
@ -139,7 +145,11 @@ private:
void ConvertOrLikeChainPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context)
{
const auto & settings = context->getSettingsRef();
auto match_function_resolver = createInternalMultiMatchAnyOverloadResolver(settings.allow_hyperscan, settings.max_hyperscan_regexp_length, settings.max_hyperscan_regexp_total_length, settings.reject_expensive_hyperscan_regexps);
auto match_function_resolver = createInternalMultiMatchAnyOverloadResolver(
settings[Setting::allow_hyperscan],
settings[Setting::max_hyperscan_regexp_length],
settings[Setting::max_hyperscan_regexp_total_length],
settings[Setting::reject_expensive_hyperscan_regexps]);
auto or_function_resolver = createInternalFunctionOrOverloadResolver();
ConvertOrLikeChainVisitor visitor(std::move(or_function_resolver), std::move(match_function_resolver), std::move(context));

View File

@ -19,6 +19,13 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool convert_query_to_cnf;
extern const SettingsBool optimize_append_index;
extern const SettingsBool optimize_substitute_columns;
extern const SettingsBool optimize_using_constraints;
}
namespace
{
@ -681,7 +688,7 @@ void optimizeWithConstraints(Analyzer::CNF & cnf, const QueryTreeNodes & table_e
cnf.pushNotIntoFunctions(context);
const auto & settings = context->getSettingsRef();
if (settings.optimize_append_index)
if (settings[Setting::optimize_append_index])
addIndexConstraint(cnf, table_expressions, context);
}
@ -693,7 +700,7 @@ void optimizeNode(QueryTreeNodePtr & node, const QueryTreeNodes & table_expressi
if (!cnf)
return;
if (settings.optimize_using_constraints)
if (settings[Setting::optimize_using_constraints])
optimizeWithConstraints(*cnf, table_expressions, context);
auto new_node = cnf->toQueryTree();
@ -731,7 +738,7 @@ public:
optimize_filter(query_node->getPrewhere());
optimize_filter(query_node->getHaving());
if (has_filter && settings.optimize_substitute_columns)
if (has_filter && settings[Setting::optimize_substitute_columns])
substituteColumns(*query_node, table_expressions, context);
}
};
@ -741,7 +748,7 @@ public:
void ConvertLogicalExpressionToCNFPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context)
{
const auto & settings = context->getSettingsRef();
if (!settings.convert_query_to_cnf)
if (!settings[Setting::convert_query_to_cnf])
return;
ConvertQueryToCNFVisitor visitor(std::move(context));

View File

@ -15,6 +15,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool count_distinct_optimization;
}
namespace
{
@ -27,7 +31,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().count_distinct_optimization)
if (!getSettings()[Setting::count_distinct_optimization])
return;
auto * query_node = node->as<QueryNode>();

View File

@ -21,6 +21,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 cross_to_inner_join_rewrite;
}
namespace ErrorCodes
{
@ -193,17 +197,14 @@ public:
}
private:
bool isEnabled() const
{
return getSettings().cross_to_inner_join_rewrite;
}
bool isEnabled() const { return getSettings()[Setting::cross_to_inner_join_rewrite]; }
bool forceRewrite(JoinKind kind) const
{
if (kind == JoinKind::Cross)
return false;
/// Comma join can be forced to rewrite
return getSettings().cross_to_inner_join_rewrite >= 2;
return getSettings()[Setting::cross_to_inner_join_rewrite] >= 2;
}
QueryTreeNodePtr makeConjunction(const QueryTreeNodes & nodes)

View File

@ -25,6 +25,12 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool group_by_use_nulls;
extern const SettingsBool join_use_nulls;
extern const SettingsBool optimize_functions_to_subcolumns;
}
namespace
{
@ -257,7 +263,7 @@ public:
void enterImpl(const QueryTreeNodePtr & node)
{
if (!getSettings().optimize_functions_to_subcolumns)
if (!getSettings()[Setting::optimize_functions_to_subcolumns])
return;
if (auto * table_node = node->as<TableNode>())
@ -281,14 +287,14 @@ public:
if (const auto * join_node = node->as<JoinNode>())
{
can_wrap_result_columns_with_nullable |= getContext()->getSettingsRef().join_use_nulls;
can_wrap_result_columns_with_nullable |= getContext()->getSettingsRef()[Setting::join_use_nulls];
return;
}
if (const auto * query_node = node->as<QueryNode>())
{
if (query_node->isGroupByWithCube() || query_node->isGroupByWithRollup() || query_node->isGroupByWithGroupingSets())
can_wrap_result_columns_with_nullable |= getContext()->getSettingsRef().group_by_use_nulls;
can_wrap_result_columns_with_nullable |= getContext()->getSettingsRef()[Setting::group_by_use_nulls];
return;
}
}
@ -419,7 +425,7 @@ public:
void enterImpl(QueryTreeNodePtr & node) const
{
if (!getSettings().optimize_functions_to_subcolumns)
if (!getSettings()[Setting::optimize_functions_to_subcolumns])
return;
auto [function_node, first_argument_column_node, table_node] = getTypedNodesForOptimization(node, getContext());

View File

@ -21,6 +21,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_syntax_fuse_functions;
}
namespace ErrorCodes
{
@ -44,7 +48,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_syntax_fuse_functions)
if (!getSettings()[Setting::optimize_syntax_fuse_functions])
return;
auto * function_node = node->as<FunctionNode>();

View File

@ -15,6 +15,11 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool force_grouping_standard_compatibility;
extern const SettingsBool group_by_use_nulls;
}
namespace ErrorCodes
{
@ -72,41 +77,38 @@ public:
FunctionOverloadResolverPtr grouping_function_resolver;
bool add_grouping_set_column = false;
bool force_grouping_standard_compatibility = getSettings().force_grouping_standard_compatibility;
bool force_grouping_standard_compatibility = getSettings()[Setting::force_grouping_standard_compatibility];
size_t aggregation_keys_size = aggregation_key_to_index.size();
switch (group_by_kind)
{
case GroupByKind::ORDINARY:
{
auto grouping_ordinary_function = std::make_shared<FunctionGroupingOrdinary>(arguments_indexes,
force_grouping_standard_compatibility);
auto grouping_ordinary_function
= std::make_shared<FunctionGroupingOrdinary>(arguments_indexes, force_grouping_standard_compatibility);
grouping_function_resolver = std::make_shared<FunctionToOverloadResolverAdaptor>(std::move(grouping_ordinary_function));
break;
}
case GroupByKind::ROLLUP:
{
auto grouping_rollup_function = std::make_shared<FunctionGroupingForRollup>(arguments_indexes,
aggregation_keys_size,
force_grouping_standard_compatibility);
auto grouping_rollup_function = std::make_shared<FunctionGroupingForRollup>(
arguments_indexes, aggregation_keys_size, force_grouping_standard_compatibility);
grouping_function_resolver = std::make_shared<FunctionToOverloadResolverAdaptor>(std::move(grouping_rollup_function));
add_grouping_set_column = true;
break;
}
case GroupByKind::CUBE:
{
auto grouping_cube_function = std::make_shared<FunctionGroupingForCube>(arguments_indexes,
aggregation_keys_size,
force_grouping_standard_compatibility);
auto grouping_cube_function = std::make_shared<FunctionGroupingForCube>(
arguments_indexes, aggregation_keys_size, force_grouping_standard_compatibility);
grouping_function_resolver = std::make_shared<FunctionToOverloadResolverAdaptor>(std::move(grouping_cube_function));
add_grouping_set_column = true;
break;
}
case GroupByKind::GROUPING_SETS:
{
auto grouping_grouping_sets_function = std::make_shared<FunctionGroupingForGroupingSets>(arguments_indexes,
grouping_sets_keys_indexes,
force_grouping_standard_compatibility);
auto grouping_grouping_sets_function = std::make_shared<FunctionGroupingForGroupingSets>(
arguments_indexes, grouping_sets_keys_indexes, force_grouping_standard_compatibility);
grouping_function_resolver = std::make_shared<FunctionToOverloadResolverAdaptor>(std::move(grouping_grouping_sets_function));
add_grouping_set_column = true;
break;
@ -147,7 +149,8 @@ void resolveGroupingFunctions(QueryTreeNodePtr & query_node, ContextPtr context)
if (query_node_typed.hasGroupBy())
{
/// It is expected by execution layer that if there are only 1 grouping set it will be removed
if (query_node_typed.isGroupByWithGroupingSets() && query_node_typed.getGroupBy().getNodes().size() == 1 && !context->getSettingsRef().group_by_use_nulls)
if (query_node_typed.isGroupByWithGroupingSets() && query_node_typed.getGroupBy().getNodes().size() == 1
&& !context->getSettingsRef()[Setting::group_by_use_nulls])
{
auto grouping_set_list_node = query_node_typed.getGroupBy().getNodes().front();
auto & grouping_set_list_node_typed = grouping_set_list_node->as<ListNode &>();

View File

@ -10,6 +10,13 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_execute_multiif_columnar;
extern const SettingsBool allow_experimental_variant_type;
extern const SettingsBool optimize_if_chain_to_multiif;
extern const SettingsBool use_variant_as_common_type;
}
namespace
{
@ -27,7 +34,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_if_chain_to_multiif)
if (!getSettings()[Setting::optimize_if_chain_to_multiif])
return;
auto * function_node = node->as<FunctionNode>();
@ -84,7 +91,8 @@ private:
void IfChainToMultiIfPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context)
{
const auto & settings = context->getSettingsRef();
auto multi_if_function_ptr = createInternalMultiIfOverloadResolver(settings.allow_execute_multiif_columnar, settings.allow_experimental_variant_type, settings.use_variant_as_common_type);
auto multi_if_function_ptr = createInternalMultiIfOverloadResolver(
settings[Setting::allow_execute_multiif_columnar], settings[Setting::allow_experimental_variant_type], settings[Setting::use_variant_as_common_type]);
IfChainToMultiIfPassVisitor visitor(std::move(multi_if_function_ptr), std::move(context));
visitor.visit(query_tree_node);
}

View File

@ -16,6 +16,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_if_transform_strings_to_enum;
}
namespace
{
@ -101,7 +105,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_if_transform_strings_to_enum)
if (!getSettings()[Setting::optimize_if_transform_strings_to_enum])
return;
auto * function_node = node->as<FunctionNode>();

View File

@ -16,6 +16,11 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 optimize_min_equality_disjunction_chain_length;
extern const SettingsUInt64 optimize_min_inequality_conjunction_chain_length;
}
namespace ErrorCodes
{
@ -531,7 +536,8 @@ private:
for (auto & [expression, not_equals_functions] : node_to_not_equals_functions)
{
const auto & settings = getSettings();
if (not_equals_functions.size() < settings.optimize_min_inequality_conjunction_chain_length && !expression.node->getResultType()->lowCardinality())
if (not_equals_functions.size() < settings[Setting::optimize_min_inequality_conjunction_chain_length]
&& !expression.node->getResultType()->lowCardinality())
{
std::move(not_equals_functions.begin(), not_equals_functions.end(), std::back_inserter(and_operands));
continue;
@ -653,7 +659,8 @@ private:
for (auto & [expression, equals_functions] : node_to_equals_functions)
{
const auto & settings = getSettings();
if (equals_functions.size() < settings.optimize_min_equality_disjunction_chain_length && !expression.node->getResultType()->lowCardinality())
if (equals_functions.size() < settings[Setting::optimize_min_equality_disjunction_chain_length]
&& !expression.node->getResultType()->lowCardinality())
{
std::move(equals_functions.begin(), equals_functions.end(), std::back_inserter(or_operands));
continue;

View File

@ -8,6 +8,12 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_experimental_variant_type;
extern const SettingsBool optimize_multiif_to_if;
extern const SettingsBool use_variant_as_common_type;
}
namespace
{
@ -25,7 +31,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_multiif_to_if)
if (!getSettings()[Setting::optimize_multiif_to_if])
return;
auto * function_node = node->as<FunctionNode>();
@ -57,7 +63,8 @@ private:
void MultiIfToIfPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context)
{
const auto & settings = context->getSettingsRef();
auto if_function_ptr = createInternalFunctionIfOverloadResolver(settings.allow_experimental_variant_type, settings.use_variant_as_common_type);
auto if_function_ptr
= createInternalFunctionIfOverloadResolver(settings[Setting::allow_experimental_variant_type], settings[Setting::use_variant_as_common_type]);
MultiIfToIfVisitor visitor(std::move(if_function_ptr), std::move(context));
visitor.visit(query_tree_node);
}

View File

@ -13,6 +13,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_normalize_count_variants;
}
namespace
{
@ -25,7 +29,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_normalize_count_variants)
if (!getSettings()[Setting::optimize_normalize_count_variants])
return;
auto * function_node = node->as<FunctionNode>();

View File

@ -13,6 +13,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_time_filter_with_preimage;
}
namespace ErrorCodes
{
@ -58,7 +62,7 @@ public:
{"greaterOrEquals", "lessOrEquals"},
};
if (!getSettings().optimize_time_filter_with_preimage)
if (!getSettings()[Setting::optimize_time_filter_with_preimage])
return;
const auto * function = node->as<FunctionNode>();

View File

@ -12,6 +12,11 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool group_by_use_nulls;
extern const SettingsBool optimize_group_by_function_keys;
}
class OptimizeGroupByFunctionKeysVisitor : public InDepthQueryTreeVisitorWithContext<OptimizeGroupByFunctionKeysVisitor>
{
@ -29,13 +34,13 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_group_by_function_keys)
if (!getSettings()[Setting::optimize_group_by_function_keys])
return;
/// When group_by_use_nulls = 1 removing keys from GROUP BY can lead
/// to unexpected types in some functions.
/// See example in https://github.com/ClickHouse/ClickHouse/pull/61567#issuecomment-2018007887
if (getSettings().group_by_use_nulls)
if (getSettings()[Setting::group_by_use_nulls])
return;
auto * query = node->as<QueryNode>();

View File

@ -9,6 +9,11 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool group_by_use_nulls;
extern const SettingsBool optimize_injective_functions_in_group_by;
}
namespace
{
@ -23,14 +28,14 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_injective_functions_in_group_by)
if (!getSettings()[Setting::optimize_injective_functions_in_group_by])
return;
/// Don't optimize injective functions when group_by_use_nulls=true,
/// because in this case we make initial group by keys Nullable
/// and eliminating some functions can cause issues with arguments Nullability
/// during their execution. See examples in https://github.com/ClickHouse/ClickHouse/pull/61567#issuecomment-2008181143
if (getSettings().group_by_use_nulls)
if (getSettings()[Setting::group_by_use_nulls])
return;
auto * query = node->as<QueryNode>();

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_redundant_functions_in_order_by;
}
namespace
{
@ -31,7 +35,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_redundant_functions_in_order_by)
if (!getSettings()[Setting::optimize_redundant_functions_in_order_by])
return;
auto * query = node->as<QueryNode>();

View File

@ -20,6 +20,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_rewrite_aggregate_function_with_if;
}
namespace
{
@ -32,7 +36,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_rewrite_aggregate_function_with_if)
if (!getSettings()[Setting::optimize_rewrite_aggregate_function_with_if])
return;
auto * function_node = node->as<FunctionNode>();

View File

@ -11,6 +11,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_arithmetic_operations_in_aggregate_functions;
}
namespace
{
@ -23,7 +27,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_arithmetic_operations_in_aggregate_functions)
if (!getSettings()[Setting::optimize_arithmetic_operations_in_aggregate_functions])
return;
static const std::unordered_set<String> func_supported = {

View File

@ -14,6 +14,11 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool aggregate_functions_null_for_empty;
extern const SettingsBool optimize_rewrite_sum_if_to_count_if;
}
namespace
{
@ -26,7 +31,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_rewrite_sum_if_to_count_if)
if (!getSettings()[Setting::optimize_rewrite_sum_if_to_count_if])
return;
auto * function_node = node->as<FunctionNode>();
@ -56,7 +61,7 @@ public:
return;
const auto & constant_value_literal = constant_node->getValue();
if (getSettings().aggregate_functions_null_for_empty)
if (getSettings()[Setting::aggregate_functions_null_for_empty])
return;
/// Rewrite `sumIf(1, cond)` into `countIf(cond)`

View File

@ -14,6 +14,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_injective_functions_inside_uniq;
}
namespace
{
@ -36,7 +40,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_injective_functions_inside_uniq)
if (!getSettings()[Setting::optimize_injective_functions_inside_uniq])
return;
auto * function_node = node->as<FunctionNode>();

View File

@ -13,6 +13,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool optimize_uniq_to_count;
}
namespace
{
@ -123,7 +127,7 @@ public:
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_uniq_to_count)
if (!getSettings()[Setting::optimize_uniq_to_count])
return;
auto * query_node = node->as<QueryNode>();

View File

@ -50,6 +50,17 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_experimental_variant_type;
extern const SettingsBool any_join_distinct_right_table_keys;
extern const SettingsJoinStrictness join_default_strictness;
extern const SettingsBool enable_order_by_all;
extern const SettingsUInt64 limit;
extern const SettingsUInt64 offset;
extern const SettingsBool use_variant_as_common_type;
}
namespace ErrorCodes
{
@ -235,13 +246,13 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q
UInt64 offset = 0;
/// Remove global settings limit and offset
if (const auto & settings_ref = updated_context->getSettingsRef(); settings_ref.limit || settings_ref.offset)
if (const auto & settings_ref = updated_context->getSettingsRef(); settings_ref[Setting::limit] || settings_ref[Setting::offset])
{
Settings settings = updated_context->getSettingsCopy();
limit = settings.limit;
offset = settings.offset;
settings.limit = 0;
settings.offset = 0;
limit = settings[Setting::limit];
offset = settings[Setting::offset];
settings[Setting::limit] = 0;
settings[Setting::offset] = 0;
updated_context->setSettings(settings);
}
@ -268,7 +279,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q
}
}
const auto enable_order_by_all = updated_context->getSettingsRef().enable_order_by_all;
const auto enable_order_by_all = updated_context->getSettingsRef()[Setting::enable_order_by_all];
auto current_query_tree = std::make_shared<QueryNode>(std::move(updated_context), std::move(settings_changes));
@ -577,7 +588,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
}
else if (const auto * ast_literal = expression->as<ASTLiteral>())
{
if (context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_as_common_type)
if (context->getSettingsRef()[Setting::allow_experimental_variant_type] && context->getSettingsRef()[Setting::use_variant_as_common_type])
result = std::make_shared<ConstantNode>(ast_literal->value, applyVisitor(FieldToDataType<LeastSupertypeOnError::Variant>(), ast_literal->value));
else
result = std::make_shared<ConstantNode>(ast_literal->value);
@ -908,8 +919,8 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select
join_expression = buildExpression(table_join.on_expression, context);
const auto & settings = context->getSettingsRef();
auto join_default_strictness = settings.join_default_strictness;
auto any_join_distinct_right_table_keys = settings.any_join_distinct_right_table_keys;
auto join_default_strictness = settings[Setting::join_default_strictness];
auto any_join_distinct_right_table_keys = settings[Setting::any_join_distinct_right_table_keys];
JoinStrictness result_join_strictness = table_join.strictness;
JoinKind result_join_kind = table_join.kind;

View File

@ -7,6 +7,12 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool group_by_use_nulls;
extern const SettingsBool join_use_nulls;
}
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
@ -32,12 +38,12 @@ IdentifierResolveScope::IdentifierResolveScope(QueryTreeNodePtr scope_node_, Ide
else if (auto * query_node = scope_node->as<QueryNode>())
{
context = query_node->getContext();
group_by_use_nulls = context->getSettingsRef().group_by_use_nulls &&
(query_node->isGroupByWithGroupingSets() || query_node->isGroupByWithRollup() || query_node->isGroupByWithCube());
group_by_use_nulls = context->getSettingsRef()[Setting::group_by_use_nulls]
&& (query_node->isGroupByWithGroupingSets() || query_node->isGroupByWithRollup() || query_node->isGroupByWithCube());
}
if (context)
join_use_nulls = context->getSettingsRef().join_use_nulls;
join_use_nulls = context->getSettingsRef()[Setting::join_use_nulls];
else if (parent_scope)
join_use_nulls = parent_scope->join_use_nulls;

View File

@ -30,6 +30,12 @@
namespace DB
{
namespace Setting
{
extern const SettingsSeconds lock_acquire_timeout;
extern const SettingsBool single_join_prefer_left_table;
}
namespace ErrorCodes
{
extern const int UNKNOWN_IDENTIFIER;
@ -420,7 +426,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveTableIdentifierFromDatabaseCatalo
if (!storage)
return {};
auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef()[Setting::lock_acquire_timeout]);
auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context);
auto result = std::make_shared<TableNode>(std::move(storage), std::move(storage_lock), std::move(storage_snapshot));
if (is_temporary_table)
@ -1155,7 +1161,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromJoin(const Identifi
resolved_identifier = left_resolved_identifier;
}
}
else if (scope.joins_count == 1 && scope.context->getSettingsRef().single_join_prefer_left_table)
else if (scope.joins_count == 1 && scope.context->getSettingsRef()[Setting::single_join_prefer_left_table])
{
resolved_side = JoinTableSide::Left;
resolved_identifier = left_resolved_identifier;

View File

@ -77,6 +77,35 @@ namespace ProfileEvents
namespace DB
{
namespace Setting
{
extern const SettingsBool aggregate_functions_null_for_empty;
extern const SettingsBool analyzer_compatibility_join_using_top_level_identifier;
extern const SettingsBool asterisk_include_alias_columns;
extern const SettingsBool asterisk_include_materialized_columns;
extern const SettingsString count_distinct_implementation;
extern const SettingsBool enable_global_with_statement;
extern const SettingsBool enable_order_by_all;
extern const SettingsBool enable_positional_arguments;
extern const SettingsBool enable_scalar_subquery_optimization;
extern const SettingsBool extremes;
extern const SettingsBool force_grouping_standard_compatibility;
extern const SettingsBool format_display_secrets_in_show_and_select;
extern const SettingsBool joined_subquery_requires_alias;
extern const SettingsUInt64 max_bytes_in_set;
extern const SettingsUInt64 max_expanded_ast_elements;
extern const SettingsUInt64 max_result_rows;
extern const SettingsUInt64 max_rows_in_set;
extern const SettingsUInt64 max_subquery_depth;
extern const SettingsBool prefer_column_name_to_alias;
extern const SettingsBool rewrite_count_distinct_if_with_count_distinct_implementation;
extern const SettingsOverflowMode set_overflow_mode;
extern const SettingsBool single_join_prefer_left_table;
extern const SettingsBool transform_null_in;
extern const SettingsUInt64 use_structure_from_insertion_table_in_table_functions;
}
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
@ -506,8 +535,8 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
auto subquery_context = Context::createCopy(context);
Settings subquery_settings = context->getSettingsCopy();
subquery_settings.max_result_rows = 1;
subquery_settings.extremes = false;
subquery_settings[Setting::max_result_rows] = 1;
subquery_settings[Setting::extremes] = false;
subquery_context->setSettings(subquery_settings);
/// When execute `INSERT INTO t WITH ... SELECT ...`, it may lead to `Unknown columns`
/// exception with this settings enabled(https://github.com/ClickHouse/ClickHouse/issues/52494).
@ -627,10 +656,8 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
auto * nearest_query_scope = scope.getNearestQueryScope();
/// Always convert to literals when there is no query context
if (!context->getSettingsRef().enable_scalar_subquery_optimization ||
!useless_literal_types.contains(scalar_type_name) ||
!context->hasQueryContext() ||
!nearest_query_scope)
if (!context->getSettingsRef()[Setting::enable_scalar_subquery_optimization] || !useless_literal_types.contains(scalar_type_name)
|| !context->hasQueryContext() || !nearest_query_scope)
{
auto constant_value = std::make_shared<ConstantValue>(std::move(scalar_value), scalar_type);
auto constant_node = std::make_shared<ConstantNode>(constant_value, node);
@ -726,7 +753,7 @@ void QueryAnalyzer::mergeWindowWithParentWindow(const QueryTreeNodePtr & window_
void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope)
{
const auto & settings = scope.context->getSettingsRef();
if (!settings.enable_positional_arguments || scope.context->getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY)
if (!settings[Setting::enable_positional_arguments] || scope.context->getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY)
return;
auto & node_list_typed = node_list->as<ListNode &>();
@ -843,7 +870,7 @@ void QueryAnalyzer::validateTableExpressionModifiers(const QueryTreeNodePtr & ta
void QueryAnalyzer::validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope)
{
if (!scope.context->getSettingsRef().joined_subquery_requires_alias)
if (!scope.context->getSettingsRef()[Setting::joined_subquery_requires_alias])
return;
bool table_expression_has_alias = table_expression_node->hasAlias();
@ -938,7 +965,7 @@ void QueryAnalyzer::expandGroupByAll(QueryNode & query_tree_node_typed)
void QueryAnalyzer::expandOrderByAll(QueryNode & query_tree_node_typed, const Settings & settings)
{
if (!settings.enable_order_by_all || !query_tree_node_typed.isOrderByAll())
if (!settings[Setting::enable_order_by_all] || !query_tree_node_typed.isOrderByAll())
return;
auto * all_node = query_tree_node_typed.getOrderBy().getNodes()[0]->as<SortNode>();
@ -989,12 +1016,14 @@ std::string QueryAnalyzer::rewriteAggregateFunctionNameIfNeeded(
if (aggregate_function_name_lowercase == "countdistinct")
{
result_aggregate_function_name = settings.count_distinct_implementation;
result_aggregate_function_name = settings[Setting::count_distinct_implementation];
}
else if (aggregate_function_name_lowercase == "countifdistinct" ||
(settings.rewrite_count_distinct_if_with_count_distinct_implementation && aggregate_function_name_lowercase == "countdistinctif"))
else if (
aggregate_function_name_lowercase == "countifdistinct"
|| (settings[Setting::rewrite_count_distinct_if_with_count_distinct_implementation]
&& aggregate_function_name_lowercase == "countdistinctif"))
{
result_aggregate_function_name = settings.count_distinct_implementation;
result_aggregate_function_name = settings[Setting::count_distinct_implementation];
result_aggregate_function_name += "If";
}
else if (aggregate_function_name_lowercase.ends_with("ifdistinct"))
@ -1004,7 +1033,7 @@ std::string QueryAnalyzer::rewriteAggregateFunctionNameIfNeeded(
result_aggregate_function_name = result_aggregate_function_name.substr(0, prefix_length) + "DistinctIf";
}
bool need_add_or_null = settings.aggregate_functions_null_for_empty && !result_aggregate_function_name.ends_with("OrNull");
bool need_add_or_null = settings[Setting::aggregate_functions_null_for_empty] && !result_aggregate_function_name.ends_with("OrNull");
if (need_add_or_null)
{
auto properties = AggregateFunctionFactory::instance().tryGetProperties(result_aggregate_function_name, action);
@ -1215,7 +1244,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const
}
}
if (!scope.context->getSettingsRef().enable_global_with_statement)
if (!scope.context->getSettingsRef()[Setting::enable_global_with_statement])
return {};
/** Nested subqueries cannot access outer subqueries table expressions from JOIN tree because
@ -1347,7 +1376,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook
if (!resolve_result.resolved_identifier)
{
bool prefer_column_name_to_alias = scope.context->getSettingsRef().prefer_column_name_to_alias;
bool prefer_column_name_to_alias = scope.context->getSettingsRef()[Setting::prefer_column_name_to_alias];
if (identifier_lookup.isExpressionLookup())
{
@ -1558,10 +1587,10 @@ GetColumnsOptions QueryAnalyzer::buildGetColumnsOptions(QueryTreeNodePtr & match
const auto & settings = context->getSettingsRef();
if (settings.asterisk_include_alias_columns)
if (settings[Setting::asterisk_include_alias_columns])
get_columns_options_kind |= GetColumnsOptions::Kind::Aliases;
if (settings.asterisk_include_materialized_columns)
if (settings[Setting::asterisk_include_materialized_columns])
get_columns_options_kind |= GetColumnsOptions::Kind::Materialized;
}
@ -2810,7 +2839,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
allow_table_expressions /*allow_table_expression*/);
/// Mask arguments if needed
if (!scope.context->getSettingsRef().format_display_secrets_in_show_and_select)
if (!scope.context->getSettingsRef()[Setting::format_display_secrets_in_show_and_select])
{
if (FunctionSecretArgumentsFinder::Result secret_arguments = FunctionSecretArgumentsFinderTreeNode(*function_node_ptr).getResult(); secret_arguments.count)
{
@ -2834,7 +2863,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
if (is_special_function_in)
{
checkFunctionNodeHasEmptyNullsAction(function_node);
if (scope.context->getSettingsRef().transform_null_in)
if (scope.context->getSettingsRef()[Setting::transform_null_in])
{
static constexpr std::array<std::pair<std::string_view, std::string_view>, 4> in_function_to_replace_null_in_function_map =
{{
@ -3134,7 +3163,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
function_arguments_size);
checkFunctionNodeHasEmptyNullsAction(function_node);
bool force_grouping_standard_compatibility = scope.context->getSettingsRef().force_grouping_standard_compatibility;
bool force_grouping_standard_compatibility = scope.context->getSettingsRef()[Setting::force_grouping_standard_compatibility];
auto grouping_function = std::make_shared<FunctionGrouping>(force_grouping_standard_compatibility);
auto grouping_function_adaptor = std::make_shared<FunctionToOverloadResolverAdaptor>(std::move(grouping_function));
function_node.resolveAsFunction(grouping_function_adaptor->build(argument_columns));
@ -3388,14 +3417,12 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
const auto & settings = scope.context->getSettingsRef();
auto result_block = getSetElementsForConstantValue(first_argument_constant_type,
second_argument_constant_literal,
second_argument_constant_type,
settings.transform_null_in);
auto result_block = getSetElementsForConstantValue(
first_argument_constant_type, second_argument_constant_literal, second_argument_constant_type, settings[Setting::transform_null_in]);
SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode};
SizeLimits size_limits_for_set = {settings[Setting::max_rows_in_set], settings[Setting::max_bytes_in_set], settings[Setting::set_overflow_mode]};
auto set = std::make_shared<Set>(size_limits_for_set, 0, settings.transform_null_in);
auto set = std::make_shared<Set>(size_limits_for_set, 0, settings[Setting::transform_null_in]);
set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName());
set->insertFromBlock(result_block.getColumnsWithTypeAndName());
@ -3826,10 +3853,10 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(
}
}
validateTreeSize(node, scope.context->getSettingsRef().max_expanded_ast_elements, node_to_tree_size);
validateTreeSize(node, scope.context->getSettingsRef()[Setting::max_expanded_ast_elements], node_to_tree_size);
/// Lambda can be inside the aggregate function, so we should check parent scopes.
/// Most likely only the root scope can have an arrgegate function, but let's check all just in case.
/// Most likely only the root scope can have an aggregate function, but let's check all just in case.
bool in_aggregate_function_scope = false;
for (const auto * scope_ptr = &scope; scope_ptr; scope_ptr = scope_ptr->parent_scope)
in_aggregate_function_scope = in_aggregate_function_scope || scope_ptr->expressions_in_resolve_process_stack.hasAggregateFunction();
@ -4473,9 +4500,9 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table
if (auto * scope_query_node = scope.scope_node->as<QueryNode>())
{
auto left_table_expression = extractLeftTableExpression(scope_query_node->getJoinTree());
if (table_expression_node.get() == left_table_expression.get() &&
scope.joins_count == 1 && scope.context->getSettingsRef().single_join_prefer_left_table)
table_expression_data.should_qualify_columns = false;
if (table_expression_node.get() == left_table_expression.get() && scope.joins_count == 1
&& scope.context->getSettingsRef()[Setting::single_join_prefer_left_table])
table_expression_data.should_qualify_columns = false;
}
scope.table_expression_node_to_data.emplace(table_expression_node, std::move(table_expression_data));
@ -4672,11 +4699,10 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node,
table_function_ptr->parseArguments(table_function_ast, scope_context);
uint64_t use_structure_from_insertion_table_in_table_functions = scope_context->getSettingsRef().use_structure_from_insertion_table_in_table_functions;
if (!nested_table_function &&
use_structure_from_insertion_table_in_table_functions &&
scope_context->hasInsertionTable() &&
table_function_ptr->needStructureHint())
uint64_t use_structure_from_insertion_table_in_table_functions
= scope_context->getSettingsRef()[Setting::use_structure_from_insertion_table_in_table_functions];
if (!nested_table_function && use_structure_from_insertion_table_in_table_functions && scope_context->hasInsertionTable()
&& table_function_ptr->needStructureHint())
{
const auto & insertion_table = scope_context->getInsertionTable();
if (!insertion_table.empty())
@ -4806,8 +4832,8 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node,
if (!structure_hint.empty())
table_function_ptr->setStructureHint(structure_hint);
} else if (use_structure_from_insertion_table_in_table_functions == 1)
}
else if (use_structure_from_insertion_table_in_table_functions == 1)
throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, "Number of columns in insert table less than required by SELECT expression.");
}
}
@ -4931,7 +4957,7 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif
void QueryAnalyzer::checkDuplicateTableNamesOrAlias(const QueryTreeNodePtr & join_node, QueryTreeNodePtr & left_table_expr, QueryTreeNodePtr & right_table_expr, IdentifierResolveScope & scope)
{
Names column_names;
if (!scope.context->getSettingsRef().joined_subquery_requires_alias)
if (!scope.context->getSettingsRef()[Setting::joined_subquery_requires_alias])
return;
if (join_node->as<JoinNode &>().getKind() != JoinKind::Paste)
@ -5051,7 +5077,7 @@ void QueryAnalyzer::resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveS
* despite the fact that column from USING could be resolved from left table.
* It's compatibility with a default behavior for old analyzer.
*/
if (settings.analyzer_compatibility_join_using_top_level_identifier)
if (settings[Setting::analyzer_compatibility_join_using_top_level_identifier])
result_left_table_expression = try_resolve_identifier_from_query_projection(identifier_full_name, join_node_typed.getLeftTableExpression(), scope);
IdentifierLookup identifier_lookup{identifier_node->getIdentifier(), IdentifierLookupContext::EXPRESSION};
@ -5070,7 +5096,7 @@ void QueryAnalyzer::resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveS
{
String extra_message;
const QueryNode * query_node = scope.scope_node ? scope.scope_node->as<QueryNode>() : nullptr;
if (settings.analyzer_compatibility_join_using_top_level_identifier && query_node)
if (settings[Setting::analyzer_compatibility_join_using_top_level_identifier] && query_node)
{
for (const auto & projection_node : query_node->getProjection().getNodes())
{
@ -5250,11 +5276,9 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node,
*/
void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope)
{
size_t max_subquery_depth = scope.context->getSettingsRef().max_subquery_depth;
size_t max_subquery_depth = scope.context->getSettingsRef()[Setting::max_subquery_depth];
if (max_subquery_depth && scope.subquery_depth > max_subquery_depth)
throw Exception(ErrorCodes::TOO_DEEP_SUBQUERIES,
"Too deep subqueries. Maximum: {}",
max_subquery_depth);
throw Exception(ErrorCodes::TOO_DEEP_SUBQUERIES, "Too deep subqueries. Maximum: {}", max_subquery_depth);
auto & query_node_typed = query_node->as<QueryNode &>();
@ -5588,7 +5612,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
expandGroupByAll(query_node_typed);
validateFilters(query_node);
validateAggregates(query_node, { .group_by_use_nulls = scope.group_by_use_nulls });
validateAggregates(query_node, {.group_by_use_nulls = scope.group_by_use_nulls});
for (const auto & column : projection_columns)
{

View File

@ -14,6 +14,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsSeconds lock_acquire_timeout;
}
TableNode::TableNode(StoragePtr storage_, StorageID storage_id_, TableLockHolder storage_lock_, StorageSnapshotPtr storage_snapshot_)
: IQueryTreeNode(children_size)
@ -29,9 +33,10 @@ TableNode::TableNode(StoragePtr storage_, TableLockHolder storage_lock_, Storage
}
TableNode::TableNode(StoragePtr storage_, const ContextPtr & context)
: TableNode(storage_,
storage_->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout),
storage_->getStorageSnapshot(storage_->getInMemoryMetadataPtr(), context))
: TableNode(
storage_,
storage_->lockForShare(context->getInitialQueryId(), context->getSettingsRef()[Setting::lock_acquire_timeout]),
storage_->getStorageSnapshot(storage_->getInMemoryMetadataPtr(), context))
{
}
@ -39,7 +44,7 @@ void TableNode::updateStorage(StoragePtr storage_value, const ContextPtr & conte
{
storage = std::move(storage_value);
storage_id = storage->getStorageID();
storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef()[Setting::lock_acquire_timeout]);
storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context);
}

View File

@ -36,6 +36,12 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool extremes;
extern const SettingsUInt64 max_result_bytes;
extern const SettingsUInt64 max_result_rows;
}
namespace ErrorCodes
{
@ -868,10 +874,10 @@ void updateContextForSubqueryExecution(ContextMutablePtr & mutable_context)
* which are checked separately (in the Set, Join objects).
*/
Settings subquery_settings = mutable_context->getSettingsCopy();
subquery_settings.max_result_rows = 0;
subquery_settings.max_result_bytes = 0;
subquery_settings[Setting::max_result_rows] = 0;
subquery_settings[Setting::max_result_bytes] = 0;
/// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query).
subquery_settings.extremes = false;
subquery_settings[Setting::extremes] = false;
mutable_context->setSettings(subquery_settings);
}

View File

@ -35,6 +35,13 @@ namespace ProfileEvents
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 backup_restore_keeper_retry_initial_backoff_ms;
extern const SettingsUInt64 backup_restore_keeper_retry_max_backoff_ms;
extern const SettingsUInt64 backup_restore_keeper_max_retries;
extern const SettingsSeconds lock_acquire_timeout;
}
namespace ErrorCodes
{
@ -105,9 +112,9 @@ BackupEntriesCollector::BackupEntriesCollector(
, compare_collected_metadata(context->getConfigRef().getBool("backups.compare_collected_metadata", true))
, log(getLogger("BackupEntriesCollector"))
, global_zookeeper_retries_info(
context->getSettingsRef().backup_restore_keeper_max_retries,
context->getSettingsRef().backup_restore_keeper_retry_initial_backoff_ms,
context->getSettingsRef().backup_restore_keeper_retry_max_backoff_ms)
context->getSettingsRef()[Setting::backup_restore_keeper_max_retries],
context->getSettingsRef()[Setting::backup_restore_keeper_retry_initial_backoff_ms],
context->getSettingsRef()[Setting::backup_restore_keeper_retry_max_backoff_ms])
, threadpool(threadpool_)
{
}
@ -653,7 +660,7 @@ void BackupEntriesCollector::lockTablesForReading()
checkIsQueryCancelled();
table_info.table_lock = storage->tryLockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
table_info.table_lock = storage->tryLockForShare(context->getInitialQueryId(), context->getSettingsRef()[Setting::lock_acquire_timeout]);
}
std::erase_if(

View File

@ -25,6 +25,16 @@ namespace fs = std::filesystem;
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 backup_restore_s3_retry_attempts;
extern const SettingsBool enable_s3_requests_logging;
extern const SettingsBool s3_disable_checksum;
extern const SettingsUInt64 s3_max_connections;
extern const SettingsUInt64 s3_max_redirects;
}
namespace ErrorCodes
{
extern const int S3_ERROR;
@ -55,16 +65,16 @@ namespace
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
settings.auth_settings.region,
context->getRemoteHostFilter(),
static_cast<unsigned>(local_settings.s3_max_redirects),
static_cast<unsigned>(local_settings.backup_restore_s3_retry_attempts),
local_settings.enable_s3_requests_logging,
static_cast<unsigned>(local_settings[Setting::s3_max_redirects]),
static_cast<unsigned>(local_settings[Setting::backup_restore_s3_retry_attempts]),
local_settings[Setting::enable_s3_requests_logging],
/* for_disk_s3 = */ false,
request_settings.get_request_throttler,
request_settings.put_request_throttler,
s3_uri.uri.getScheme());
client_configuration.endpointOverride = s3_uri.endpoint;
client_configuration.maxConnections = static_cast<unsigned>(global_settings.s3_max_connections);
client_configuration.maxConnections = static_cast<unsigned>(global_settings[Setting::s3_max_connections]);
/// Increase connect timeout
client_configuration.connectTimeoutMs = 10 * 1000;
/// Requests in backups can be extremely long, set to one hour
@ -74,7 +84,7 @@ namespace
S3::ClientSettings client_settings{
.use_virtual_addressing = s3_uri.is_virtual_hosted_style,
.disable_checksum = local_settings.s3_disable_checksum,
.disable_checksum = local_settings[Setting::s3_disable_checksum],
.gcs_issue_compose_request = context->getConfigRef().getBool("s3.gcs_issue_compose_request", false),
.is_s3express_bucket = S3::isS3ExpressEndpoint(s3_uri.endpoint),
};

View File

@ -43,6 +43,15 @@ namespace CurrentMetrics
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 backup_restore_batch_size_for_keeper_multiread;
extern const SettingsUInt64 backup_restore_keeper_max_retries;
extern const SettingsUInt64 backup_restore_keeper_retry_initial_backoff_ms;
extern const SettingsUInt64 backup_restore_keeper_retry_max_backoff_ms;
extern const SettingsUInt64 backup_restore_keeper_fault_injection_seed;
extern const SettingsFloat backup_restore_keeper_fault_injection_probability;
}
namespace ErrorCodes
{
@ -98,12 +107,12 @@ namespace
RestoreCoordinationRemote::RestoreKeeperSettings keeper_settings
{
.keeper_max_retries = context->getSettingsRef().backup_restore_keeper_max_retries,
.keeper_retry_initial_backoff_ms = context->getSettingsRef().backup_restore_keeper_retry_initial_backoff_ms,
.keeper_retry_max_backoff_ms = context->getSettingsRef().backup_restore_keeper_retry_max_backoff_ms,
.batch_size_for_keeper_multiread = context->getSettingsRef().backup_restore_batch_size_for_keeper_multiread,
.keeper_fault_injection_probability = context->getSettingsRef().backup_restore_keeper_fault_injection_probability,
.keeper_fault_injection_seed = context->getSettingsRef().backup_restore_keeper_fault_injection_seed
.keeper_max_retries = context->getSettingsRef()[Setting::backup_restore_keeper_max_retries],
.keeper_retry_initial_backoff_ms = context->getSettingsRef()[Setting::backup_restore_keeper_retry_initial_backoff_ms],
.keeper_retry_max_backoff_ms = context->getSettingsRef()[Setting::backup_restore_keeper_retry_max_backoff_ms],
.batch_size_for_keeper_multiread = context->getSettingsRef()[Setting::backup_restore_batch_size_for_keeper_multiread],
.keeper_fault_injection_probability = context->getSettingsRef()[Setting::backup_restore_keeper_fault_injection_probability],
.keeper_fault_injection_seed = context->getSettingsRef()[Setting::backup_restore_keeper_fault_injection_seed]
};
auto all_hosts = BackupSettings::Util::filterHostIDs(

View File

@ -36,6 +36,11 @@ namespace fs = std::filesystem;
namespace DB
{
namespace Setting
{
extern const SettingsSeconds lock_acquire_timeout;
}
namespace ErrorCodes
{
extern const int BACKUP_ENTRY_NOT_FOUND;
@ -949,7 +954,7 @@ void RestorerFromBackup::checkTable(const QualifiedTableName & table_name)
StoragePtr storage = database->getTable(resolved_id.table_name, context);
table_info.storage = storage;
table_info.table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
table_info.table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef()[Setting::lock_acquire_timeout]);
if (!restore_settings.allow_different_table_def && !table_info.is_predefined_table)
{

View File

@ -5,20 +5,30 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 backup_restore_keeper_max_retries;
extern const SettingsUInt64 backup_restore_keeper_retry_initial_backoff_ms;
extern const SettingsUInt64 backup_restore_keeper_retry_max_backoff_ms;
extern const SettingsUInt64 backup_restore_batch_size_for_keeper_multiread;
extern const SettingsFloat backup_restore_keeper_fault_injection_probability;
extern const SettingsUInt64 backup_restore_keeper_fault_injection_seed;
extern const SettingsUInt64 backup_restore_keeper_value_max_size;
extern const SettingsUInt64 backup_restore_batch_size_for_keeper_multi;
}
WithRetries::KeeperSettings WithRetries::KeeperSettings::fromContext(ContextPtr context)
{
return
{
.keeper_max_retries = context->getSettingsRef().backup_restore_keeper_max_retries,
.keeper_retry_initial_backoff_ms = context->getSettingsRef().backup_restore_keeper_retry_initial_backoff_ms,
.keeper_retry_max_backoff_ms = context->getSettingsRef().backup_restore_keeper_retry_max_backoff_ms,
.batch_size_for_keeper_multiread = context->getSettingsRef().backup_restore_batch_size_for_keeper_multiread,
.keeper_fault_injection_probability = context->getSettingsRef().backup_restore_keeper_fault_injection_probability,
.keeper_fault_injection_seed = context->getSettingsRef().backup_restore_keeper_fault_injection_seed,
.keeper_value_max_size = context->getSettingsRef().backup_restore_keeper_value_max_size,
.batch_size_for_keeper_multi = context->getSettingsRef().backup_restore_batch_size_for_keeper_multi,
.keeper_max_retries = context->getSettingsRef()[Setting::backup_restore_keeper_max_retries],
.keeper_retry_initial_backoff_ms = context->getSettingsRef()[Setting::backup_restore_keeper_retry_initial_backoff_ms],
.keeper_retry_max_backoff_ms = context->getSettingsRef()[Setting::backup_restore_keeper_retry_max_backoff_ms],
.batch_size_for_keeper_multiread = context->getSettingsRef()[Setting::backup_restore_batch_size_for_keeper_multiread],
.keeper_fault_injection_probability = context->getSettingsRef()[Setting::backup_restore_keeper_fault_injection_probability],
.keeper_fault_injection_seed = context->getSettingsRef()[Setting::backup_restore_keeper_fault_injection_seed],
.keeper_value_max_size = context->getSettingsRef()[Setting::backup_restore_keeper_value_max_size],
.batch_size_for_keeper_multi = context->getSettingsRef()[Setting::backup_restore_batch_size_for_keeper_multi],
};
}

View File

@ -10,6 +10,8 @@
namespace DB
{
struct ColumnWithTypeAndName;
using ColumnsWithTypeAndName = std::vector<ColumnWithTypeAndName>;
class CatBoostLibraryBridgeHelper final : public LibraryBridgeHelper
{

View File

@ -1,11 +1,12 @@
#pragma once
#include <Interpreters/Context.h>
#include <Poco/Logger.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Net/HTTPRequest.h>
#include <Common/ShellCommand.h>
#include <Poco/Logger.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/URI.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{

View File

@ -6,12 +6,16 @@
namespace DB
{
namespace Setting
{
extern const SettingsSeconds http_receive_timeout;
}
LibraryBridgeHelper::LibraryBridgeHelper(ContextPtr context_)
: IBridgeHelper(context_)
, config(context_->getConfigRef())
, log(getLogger("LibraryBridgeHelper"))
, http_timeout(context_->getGlobalContext()->getSettingsRef().http_receive_timeout.value)
, http_timeout(context_->getGlobalContext()->getSettingsRef()[Setting::http_receive_timeout].value)
, bridge_host(config.getString("library_bridge.host", DEFAULT_HOST))
, bridge_port(config.getUInt("library_bridge.port", DEFAULT_PORT))
, http_timeouts(ConnectionTimeouts::getHTTPTimeouts(context_->getSettingsRef(), context_->getServerSettings().keep_alive_timeout))

View File

@ -2,7 +2,6 @@
#include <base/argsToConfig.h>
#include <base/safeExit.h>
#include <Core/BaseSettingsProgramOptions.h>
#include <Common/clearPasswordFromCommandLine.h>
#include <Common/TerminalSize.h>
#include <Common/Exception.h>

View File

@ -7,7 +7,6 @@
#include <base/safeExit.h>
#include <Core/Block.h>
#include <Core/BaseSettingsProgramOptions.h>
#include <Core/Protocol.h>
#include <Common/DateLUT.h>
#include <Common/MemoryTracker.h>
@ -93,6 +92,21 @@ using namespace std::literals;
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_settings_after_format_in_insert;
extern const SettingsBool async_insert;
extern const SettingsDialect dialect;
extern const SettingsUInt64 max_block_size;
extern const SettingsUInt64 max_insert_block_size;
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
extern const SettingsUInt64 max_query_size;
extern const SettingsUInt64 output_format_pretty_max_rows;
extern const SettingsUInt64 output_format_pretty_max_value_width;
extern const SettingsBool partial_result_on_first_cancel;
extern const SettingsBool throw_if_no_data_to_insert;
}
namespace ErrorCodes
{
@ -295,24 +309,24 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, const Setting
size_t max_length = 0;
if (!allow_multi_statements)
max_length = settings.max_query_size;
max_length = settings[Setting::max_query_size];
const Dialect & dialect = settings.dialect;
const Dialect dialect = settings[Setting::dialect];
if (dialect == Dialect::kusto)
parser = std::make_unique<ParserKQLStatement>(end, settings.allow_settings_after_format_in_insert);
parser = std::make_unique<ParserKQLStatement>(end, settings[Setting::allow_settings_after_format_in_insert]);
else if (dialect == Dialect::prql)
parser = std::make_unique<ParserPRQLQuery>(max_length, settings.max_parser_depth, settings.max_parser_backtracks);
parser = std::make_unique<ParserPRQLQuery>(max_length, settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks]);
else
parser = std::make_unique<ParserQuery>(end, settings.allow_settings_after_format_in_insert);
parser = std::make_unique<ParserQuery>(end, settings[Setting::allow_settings_after_format_in_insert]);
if (is_interactive || ignore_error)
{
String message;
if (dialect == Dialect::kusto)
res = tryParseKQLQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth, settings.max_parser_backtracks, true);
res = tryParseKQLQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks], true);
else
res = tryParseQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth, settings.max_parser_backtracks, true);
res = tryParseQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks], true);
if (!res)
{
@ -323,9 +337,9 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, const Setting
else
{
if (dialect == Dialect::kusto)
res = parseKQLQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth, settings.max_parser_backtracks);
res = parseKQLQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks]);
else
res = parseQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth, settings.max_parser_backtracks);
res = parseQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks]);
}
if (is_interactive)
@ -346,7 +360,8 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, const Setting
/// Consumes trailing semicolons and tries to consume the same-line trailing comment.
void ClientBase::adjustQueryEnd(const char *& this_query_end, const char * all_queries_end, uint32_t max_parser_depth, uint32_t max_parser_backtracks)
void ClientBase::adjustQueryEnd(
const char *& this_query_end, const char * all_queries_end, uint32_t max_parser_depth, uint32_t max_parser_backtracks)
{
// We have to skip the trailing semicolon that might be left
// after VALUES parsing or just after a normal semicolon-terminated query.
@ -672,16 +687,16 @@ void ClientBase::adjustSettings()
/// Do not limit pretty format output in case of --pager specified or in case of stdout is not a tty.
if (!pager.empty() || !stdout_is_a_tty)
{
if (!global_context->getSettingsRef().output_format_pretty_max_rows.changed)
if (!global_context->getSettingsRef()[Setting::output_format_pretty_max_rows].changed)
{
settings.output_format_pretty_max_rows = std::numeric_limits<UInt64>::max();
settings.output_format_pretty_max_rows.changed = false;
settings[Setting::output_format_pretty_max_rows] = std::numeric_limits<UInt64>::max();
settings[Setting::output_format_pretty_max_rows].changed = false;
}
if (!global_context->getSettingsRef().output_format_pretty_max_value_width.changed)
if (!global_context->getSettingsRef()[Setting::output_format_pretty_max_value_width].changed)
{
settings.output_format_pretty_max_value_width = std::numeric_limits<UInt64>::max();
settings.output_format_pretty_max_value_width.changed = false;
settings[Setting::output_format_pretty_max_value_width] = std::numeric_limits<UInt64>::max();
settings[Setting::output_format_pretty_max_value_width].changed = false;
}
}
@ -767,18 +782,17 @@ void ClientBase::setDefaultFormatsAndCompressionFromConfiguration()
default_input_format = "TSV";
}
format_max_block_size = getClientConfiguration().getUInt64("format_max_block_size",
global_context->getSettingsRef().max_block_size);
format_max_block_size = getClientConfiguration().getUInt64("format_max_block_size", global_context->getSettingsRef()[Setting::max_block_size]);
/// Setting value from cmd arg overrides one from config
if (global_context->getSettingsRef().max_insert_block_size.changed)
if (global_context->getSettingsRef()[Setting::max_insert_block_size].changed)
{
insert_format_max_block_size = global_context->getSettingsRef().max_insert_block_size;
insert_format_max_block_size = global_context->getSettingsRef()[Setting::max_insert_block_size];
}
else
{
insert_format_max_block_size = getClientConfiguration().getUInt64("insert_format_max_block_size",
global_context->getSettingsRef().max_insert_block_size);
insert_format_max_block_size
= getClientConfiguration().getUInt64("insert_format_max_block_size", global_context->getSettingsRef()[Setting::max_insert_block_size]);
}
}
@ -878,7 +892,7 @@ bool ClientBase::isSyncInsertWithData(const ASTInsertQuery & insert_query, const
if (insert_query.settings_ast)
settings.applyChanges(insert_query.settings_ast->as<ASTSetQuery>()->changes);
return !settings.async_insert;
return !settings[Setting::async_insert];
}
void ClientBase::processTextAsSingleQuery(const String & full_query)
@ -1031,7 +1045,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa
}
const auto & settings = client_context->getSettingsRef();
const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1;
const Int32 signals_before_stop = settings[Setting::partial_result_on_first_cancel] ? 2 : 1;
int retries_left = 10;
while (retries_left)
@ -1059,11 +1073,11 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa
catch (const NetException &)
{
// We still want to attempt to process whatever we already received or can receive (socket receive buffer can be not empty)
receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel);
receiveResult(parsed_query, signals_before_stop, settings[Setting::partial_result_on_first_cancel]);
throw;
}
receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel);
receiveResult(parsed_query, signals_before_stop, settings[Setting::partial_result_on_first_cancel]);
break;
}
@ -1491,7 +1505,7 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars
if ((!parsed_insert_query.data && !parsed_insert_query.infile) && (is_interactive || (!stdin_is_a_tty && !isStdinNotEmptyAndValid(std_in))))
{
const auto & settings = client_context->getSettingsRef();
if (settings.throw_if_no_data_to_insert)
if (settings[Setting::throw_if_no_data_to_insert])
throw Exception(ErrorCodes::NO_DATA_TO_INSERT, "No data to insert");
else
return;
@ -1609,14 +1623,14 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des
auto metadata = storage->getInMemoryMetadataPtr();
QueryPlan plan;
storage->read(
plan,
sample.getNames(),
storage->getStorageSnapshot(metadata, client_context),
query_info,
client_context,
{},
client_context->getSettingsRef().max_block_size,
getNumberOfPhysicalCPUCores());
plan,
sample.getNames(),
storage->getStorageSnapshot(metadata, client_context),
query_info,
client_context,
{},
client_context->getSettingsRef()[Setting::max_block_size],
getNumberOfPhysicalCPUCores());
auto builder = plan.buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(client_context),
@ -1953,7 +1967,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
if (insert && insert->select)
insert->tryFindInputFunction(input_function);
bool is_async_insert_with_inlined_data = client_context->getSettingsRef().async_insert && insert && insert->hasInlinedData();
bool is_async_insert_with_inlined_data = client_context->getSettingsRef()[Setting::async_insert] && insert && insert->hasInlinedData();
if (is_async_insert_with_inlined_data)
{
@ -2038,11 +2052,11 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
error_stream << progress_indication.elapsedSeconds() << "\n";
const auto & print_memory_mode = config.getString("print-memory-to-stderr", "");
auto peak_memeory_usage = std::max<Int64>(progress_indication.getMemoryUsage().peak, 0);
auto peak_memory_usage = std::max<Int64>(progress_indication.getMemoryUsage().peak, 0);
if (print_memory_mode == "default")
error_stream << peak_memeory_usage << "\n";
error_stream << peak_memory_usage << "\n";
else if (print_memory_mode == "readable")
error_stream << formatReadableSizeWithBinarySuffix(peak_memeory_usage) << "\n";
error_stream << formatReadableSizeWithBinarySuffix(peak_memory_usage) << "\n";
}
if (!is_interactive && getClientConfiguration().getBool("print-num-processed-rows", false))
@ -2075,8 +2089,8 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
if (this_query_begin >= all_queries_end)
return MultiQueryProcessingStage::QUERIES_END;
unsigned max_parser_depth = static_cast<unsigned>(client_context->getSettingsRef().max_parser_depth);
unsigned max_parser_backtracks = static_cast<unsigned>(client_context->getSettingsRef().max_parser_backtracks);
unsigned max_parser_depth = static_cast<unsigned>(client_context->getSettingsRef()[Setting::max_parser_depth]);
unsigned max_parser_backtracks = static_cast<unsigned>(client_context->getSettingsRef()[Setting::max_parser_backtracks]);
// If there are only comments left until the end of file, we just
// stop. The parser can't handle this situation because it always
@ -2400,9 +2414,10 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
{
this_query_end = insert_ast->end;
adjustQueryEnd(
this_query_end, all_queries_end,
static_cast<unsigned>(client_context->getSettingsRef().max_parser_depth),
static_cast<unsigned>(client_context->getSettingsRef().max_parser_backtracks));
this_query_end,
all_queries_end,
static_cast<unsigned>(client_context->getSettingsRef()[Setting::max_parser_depth]),
static_cast<unsigned>(client_context->getSettingsRef()[Setting::max_parser_backtracks]));
}
// Report error.

View File

@ -83,9 +83,9 @@ private:
void ClientApplicationBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments)
{
if (allow_repeated_settings)
addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value());
cmd_settings.addToProgramOptionsAsMultitokens(options_description.main_description.value());
else
addProgramOptions(cmd_settings, options_description.main_description.value());
cmd_settings.addToProgramOptions(options_description.main_description.value());
if (allow_merge_tree_settings)
{

View File

@ -51,6 +51,15 @@ namespace CurrentMetrics
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_experimental_codecs;
extern const SettingsBool allow_suspicious_codecs;
extern const SettingsBool enable_deflate_qpl_codec;
extern const SettingsBool enable_zstd_qat_codec;
extern const SettingsString network_compression_method;
extern const SettingsInt64 network_zstd_compression_level;
}
namespace FailPoints
{
@ -791,19 +800,19 @@ void Connection::sendQuery(
if (settings)
{
std::optional<int> level;
std::string method = Poco::toUpper(settings->network_compression_method.toString());
std::string method = Poco::toUpper((*settings)[Setting::network_compression_method].toString());
/// Bad custom logic
if (method == "ZSTD")
level = settings->network_zstd_compression_level;
level = (*settings)[Setting::network_zstd_compression_level];
CompressionCodecFactory::instance().validateCodec(
method,
level,
!settings->allow_suspicious_codecs,
settings->allow_experimental_codecs,
settings->enable_deflate_qpl_codec,
settings->enable_zstd_qat_codec);
!(*settings)[Setting::allow_suspicious_codecs],
(*settings)[Setting::allow_experimental_codecs],
(*settings)[Setting::enable_deflate_qpl_codec],
(*settings)[Setting::enable_zstd_qat_codec]);
compression_codec = CompressionCodecFactory::instance().get(method, level);
}
else

View File

@ -14,6 +14,10 @@ namespace ProfileEvents
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 max_replica_delay_for_distributed_queries;
}
namespace ErrorCodes
{
@ -78,7 +82,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std::
LOG_TRACE(log, "Table {}.{} is readonly on server {}", table_to_check->database, table_to_check->table, result.entry->getDescription());
}
const UInt64 max_allowed_delay = settings.max_replica_delay_for_distributed_queries;
const UInt64 max_allowed_delay = settings[Setting::max_replica_delay_for_distributed_queries];
if (!max_allowed_delay)
{
result.is_up_to_date = true;

View File

@ -5,6 +5,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsMilliseconds connection_pool_max_wait_ms;
}
ConnectionPoolPtr ConnectionPoolFactory::get(
unsigned max_connections,
@ -93,7 +97,7 @@ ConnectionPoolFactory & ConnectionPoolFactory::instance()
IConnectionPool::Entry ConnectionPool::get(const DB::ConnectionTimeouts& timeouts, const DB::Settings& settings,
bool force_connected)
{
Entry entry = Base::get(settings.connection_pool_max_wait_ms.totalMilliseconds());
Entry entry = Base::get(settings[Setting::connection_pool_max_wait_ms].totalMilliseconds());
if (force_connected)
entry->forceConnected(timeouts);

View File

@ -16,6 +16,17 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 connections_with_failover_max_tries;
extern const SettingsBool distributed_insert_skip_read_only_replicas;
extern const SettingsUInt64 distributed_replica_max_ignored_errors;
extern const SettingsBool fallback_to_stale_replicas_for_distributed_queries;
extern const SettingsLoadBalancing load_balancing;
extern const SettingsUInt64 load_balancing_first_offset;
extern const SettingsNonZeroUInt64 max_parallel_replicas;
extern const SettingsBool skip_unavailable_shards;
}
namespace ErrorCodes
{
@ -47,10 +58,10 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover(
IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts & timeouts)
{
Settings settings;
settings.load_balancing = get_priority_load_balancing.load_balancing;
settings.load_balancing_first_offset = 0;
settings.distributed_replica_max_ignored_errors = 0;
settings.fallback_to_stale_replicas_for_distributed_queries = true;
settings[Setting::load_balancing] = get_priority_load_balancing.load_balancing;
settings[Setting::load_balancing_first_offset] = 0;
settings[Setting::distributed_replica_max_ignored_errors] = 0;
settings[Setting::fallback_to_stale_replicas_for_distributed_queries] = true;
return get(timeouts, settings, /* force_connected= */ true);
}
@ -68,13 +79,12 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts
return tryGetEntry(pool, timeouts, fail_message, settings);
};
const size_t offset = settings.load_balancing_first_offset % nested_pools.size();
const LoadBalancing load_balancing = settings.load_balancing;
const size_t offset = settings[Setting::load_balancing_first_offset] % nested_pools.size();
GetPriorityFunc get_priority = get_priority_load_balancing.getPriorityFunc(load_balancing, offset, nested_pools.size());
GetPriorityFunc get_priority = get_priority_load_balancing.getPriorityFunc(settings[Setting::load_balancing], offset, nested_pools.size());
const UInt64 max_ignored_errors = settings.distributed_replica_max_ignored_errors;
const bool fallback_to_stale_replicas = settings.fallback_to_stale_replicas_for_distributed_queries;
const UInt64 max_ignored_errors = settings[Setting::distributed_replica_max_ignored_errors];
const bool fallback_to_stale_replicas = settings[Setting::fallback_to_stale_replicas_for_distributed_queries];
return Base::get(max_ignored_errors, fallback_to_stale_replicas, try_get_entry, get_priority);
}
@ -170,15 +180,13 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
return getManyImpl(settings, pool_mode, try_get_entry,
/*skip_unavailable_endpoints=*/ false, /// skip_unavailable_endpoints is used to get the min number of entries, and we need at least one
/*priority_func=*/ {},
settings.distributed_insert_skip_read_only_replicas);
settings[Setting::distributed_insert_skip_read_only_replicas]);
}
ConnectionPoolWithFailover::Base::GetPriorityFunc ConnectionPoolWithFailover::makeGetPriorityFunc(const Settings & settings)
{
const size_t offset = settings.load_balancing_first_offset % nested_pools.size();
const LoadBalancing load_balancing = LoadBalancing(settings.load_balancing);
return get_priority_load_balancing.getPriorityFunc(load_balancing, offset, nested_pools.size());
const size_t offset = settings[Setting::load_balancing_first_offset] % nested_pools.size();
return get_priority_load_balancing.getPriorityFunc(settings[Setting::load_balancing], offset, nested_pools.size());
}
std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::getManyImpl(
@ -195,11 +203,11 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
"Cannot get connection from ConnectionPoolWithFailover cause nested pools are empty");
if (!skip_unavailable_endpoints.has_value())
skip_unavailable_endpoints = settings.skip_unavailable_shards;
skip_unavailable_endpoints = settings[Setting::skip_unavailable_shards];
size_t min_entries = skip_unavailable_endpoints.value() ? 0 : 1;
size_t max_tries = settings.connections_with_failover_max_tries;
size_t max_tries = settings[Setting::connections_with_failover_max_tries];
size_t max_entries;
if (pool_mode == PoolMode::GET_ALL)
{
@ -212,7 +220,7 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
}
else if (pool_mode == PoolMode::GET_MANY)
{
max_entries = settings.max_parallel_replicas;
max_entries = settings[Setting::max_parallel_replicas];
}
else
{
@ -222,8 +230,8 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
if (!priority_func)
priority_func = makeGetPriorityFunc(settings);
UInt64 max_ignored_errors = settings.distributed_replica_max_ignored_errors.value;
bool fallback_to_stale_replicas = settings.fallback_to_stale_replicas_for_distributed_queries.value;
UInt64 max_ignored_errors = settings[Setting::distributed_replica_max_ignored_errors].value;
bool fallback_to_stale_replicas = settings[Setting::fallback_to_stale_replicas_for_distributed_queries].value;
return Base::getMany(min_entries, max_entries, max_tries, max_ignored_errors, fallback_to_stale_replicas, skip_read_only_replicas, try_get_entry, priority_func);
}
@ -272,7 +280,7 @@ ConnectionPoolWithFailover::getShuffledPools(const Settings & settings, GetPrior
if (!priority_func)
priority_func = makeGetPriorityFunc(settings);
UInt64 max_ignored_errors = settings.distributed_replica_max_ignored_errors.value;
UInt64 max_ignored_errors = settings[Setting::distributed_replica_max_ignored_errors].value;
return Base::getShuffledPools(max_ignored_errors, priority_func, use_slowdown_count);
}

View File

@ -14,6 +14,21 @@ namespace ProfileEvents
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_changing_replica_until_first_data_packet;
extern const SettingsBool allow_experimental_analyzer;
extern const SettingsUInt64 connections_with_failover_max_tries;
extern const SettingsDialect dialect;
extern const SettingsBool fallback_to_stale_replicas_for_distributed_queries;
extern const SettingsUInt64 group_by_two_level_threshold;
extern const SettingsUInt64 group_by_two_level_threshold_bytes;
extern const SettingsNonZeroUInt64 max_parallel_replicas;
extern const SettingsUInt64 parallel_replicas_count;
extern const SettingsUInt64 parallel_replica_offset;
extern const SettingsBool skip_unavailable_shards;
}
namespace ErrorCodes
{
extern const int MISMATCH_REPLICAS_DATA_SOURCES;
@ -32,15 +47,15 @@ HedgedConnections::HedgedConnections(
AsyncCallback async_callback,
GetPriorityForLoadBalancing::Func priority_func)
: hedged_connections_factory(
pool_,
context_->getSettingsRef(),
timeouts_,
context_->getSettingsRef().connections_with_failover_max_tries.value,
context_->getSettingsRef().fallback_to_stale_replicas_for_distributed_queries.value,
context_->getSettingsRef().max_parallel_replicas.value,
context_->getSettingsRef().skip_unavailable_shards.value,
table_to_check_,
priority_func)
pool_,
context_->getSettingsRef(),
timeouts_,
context_->getSettingsRef()[Setting::connections_with_failover_max_tries].value,
context_->getSettingsRef()[Setting::fallback_to_stale_replicas_for_distributed_queries].value,
context_->getSettingsRef()[Setting::max_parallel_replicas].value,
context_->getSettingsRef()[Setting::skip_unavailable_shards].value,
table_to_check_,
priority_func)
, context(std::move(context_))
, settings(context->getSettingsRef())
, throttler(throttler_)
@ -178,29 +193,29 @@ void HedgedConnections::sendQuery(
Settings modified_settings = settings;
/// Queries in foreign languages are transformed to ClickHouse-SQL. Ensure the setting before sending.
modified_settings.dialect = Dialect::clickhouse;
modified_settings.dialect.changed = false;
modified_settings[Setting::dialect] = Dialect::clickhouse;
modified_settings[Setting::dialect].changed = false;
if (disable_two_level_aggregation)
{
/// Disable two-level aggregation due to version incompatibility.
modified_settings.group_by_two_level_threshold = 0;
modified_settings.group_by_two_level_threshold_bytes = 0;
modified_settings[Setting::group_by_two_level_threshold] = 0;
modified_settings[Setting::group_by_two_level_threshold_bytes] = 0;
}
const bool enable_offset_parallel_processing = context->canUseOffsetParallelReplicas();
if (offset_states.size() > 1 && enable_offset_parallel_processing)
{
modified_settings.parallel_replicas_count = offset_states.size();
modified_settings.parallel_replica_offset = fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset;
modified_settings[Setting::parallel_replicas_count] = offset_states.size();
modified_settings[Setting::parallel_replica_offset] = fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset;
}
/// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting.
/// Make the analyzer being set, so it will be effectively applied on the remote server.
/// In other words, the initiator always controls whether the analyzer enabled or not for
/// all servers involved in the distributed query processing.
modified_settings.set("allow_experimental_analyzer", static_cast<bool>(modified_settings.allow_experimental_analyzer));
modified_settings.set("allow_experimental_analyzer", static_cast<bool>(modified_settings[Setting::allow_experimental_analyzer]));
replica.connection->sendQuery(
timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {});
@ -446,7 +461,7 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli
{
/// If we are allowed to change replica until the first data packet,
/// just restart timeout (if it hasn't expired yet). Otherwise disable changing replica with this offset.
if (settings.allow_changing_replica_until_first_data_packet && !replica.is_change_replica_timeout_expired)
if (settings[Setting::allow_changing_replica_until_first_data_packet] && !replica.is_change_replica_timeout_expired)
replica.change_replica_timeout.setRelative(hedged_connections_factory.getConnectionTimeouts().receive_data_timeout);
else
disableChangingReplica(replica_location);

View File

@ -23,6 +23,17 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_settings_after_format_in_insert;
extern const SettingsDialect dialect;
extern const SettingsBool input_format_defaults_for_omitted_fields;
extern const SettingsUInt64 interactive_delay;
extern const SettingsUInt64 max_insert_block_size;
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
extern const SettingsUInt64 max_query_size;
}
namespace ErrorCodes
{
@ -158,21 +169,38 @@ void LocalConnection::sendQuery(
const auto & settings = context->getSettingsRef();
const char * begin = state->query.data();
const char * end = begin + state->query.size();
const Dialect & dialect = settings.dialect;
const Dialect & dialect = settings[Setting::dialect];
std::unique_ptr<IParserBase> parser;
if (dialect == Dialect::kusto)
parser = std::make_unique<ParserKQLStatement>(end, settings.allow_settings_after_format_in_insert);
parser = std::make_unique<ParserKQLStatement>(end, settings[Setting::allow_settings_after_format_in_insert]);
else if (dialect == Dialect::prql)
parser = std::make_unique<ParserPRQLQuery>(settings.max_query_size, settings.max_parser_depth, settings.max_parser_backtracks);
parser
= std::make_unique<ParserPRQLQuery>(settings[Setting::max_query_size], settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks]);
else
parser = std::make_unique<ParserQuery>(end, settings.allow_settings_after_format_in_insert);
parser = std::make_unique<ParserQuery>(end, settings[Setting::allow_settings_after_format_in_insert]);
ASTPtr parsed_query;
if (dialect == Dialect::kusto)
parsed_query = parseKQLQueryAndMovePosition(*parser, begin, end, "", /*allow_multi_statements*/false, settings.max_query_size, settings.max_parser_depth, settings.max_parser_backtracks);
parsed_query = parseKQLQueryAndMovePosition(
*parser,
begin,
end,
"",
/*allow_multi_statements*/ false,
settings[Setting::max_query_size],
settings[Setting::max_parser_depth],
settings[Setting::max_parser_backtracks]);
else
parsed_query = parseQueryAndMovePosition(*parser, begin, end, "", /*allow_multi_statements*/false, settings.max_query_size, settings.max_parser_depth, settings.max_parser_backtracks);
parsed_query = parseQueryAndMovePosition(
*parser,
begin,
end,
"",
/*allow_multi_statements*/ false,
settings[Setting::max_query_size],
settings[Setting::max_parser_depth],
settings[Setting::max_parser_backtracks]);
if (const auto * insert = parsed_query->as<ASTInsertQuery>())
{
@ -180,7 +208,7 @@ void LocalConnection::sendQuery(
current_format = insert->format;
}
auto source = context->getInputFormat(current_format, *in, sample, context->getSettingsRef().max_insert_block_size);
auto source = context->getInputFormat(current_format, *in, sample, context->getSettingsRef()[Setting::max_insert_block_size]);
Pipe pipe(source);
auto columns_description = metadata_snapshot->getColumns();
@ -227,7 +255,7 @@ void LocalConnection::sendQuery(
}
const auto & table_id = query_context->getInsertionTable();
if (query_context->getSettingsRef().input_format_defaults_for_omitted_fields)
if (query_context->getSettingsRef()[Setting::input_format_defaults_for_omitted_fields])
{
if (!table_id.empty())
{
@ -255,7 +283,7 @@ void LocalConnection::sendQuery(
return false;
};
executor.setCancelCallback(callback, query_context->getSettingsRef().interactive_delay / 1000);
executor.setCancelCallback(callback, query_context->getSettingsRef()[Setting::interactive_delay] / 1000);
}
executor.execute();
}
@ -312,7 +340,7 @@ void LocalConnection::sendCancel()
bool LocalConnection::pullBlock(Block & block)
{
if (state->executor)
return state->executor->pull(block, query_context->getSettingsRef().interactive_delay / 1000);
return state->executor->pull(block, query_context->getSettingsRef()[Setting::interactive_delay] / 1000);
return false;
}
@ -466,14 +494,15 @@ bool LocalConnection::poll(size_t)
bool LocalConnection::needSendProgressOrMetrics()
{
if (send_progress && (state->after_send_progress.elapsedMicroseconds() >= query_context->getSettingsRef().interactive_delay))
if (send_progress && (state->after_send_progress.elapsedMicroseconds() >= query_context->getSettingsRef()[Setting::interactive_delay]))
{
state->after_send_progress.restart();
next_packet_type = Protocol::Server::Progress;
return true;
}
if (send_profile_events && (state->after_send_profile_events.elapsedMicroseconds() >= query_context->getSettingsRef().interactive_delay))
if (send_profile_events
&& (state->after_send_profile_events.elapsedMicroseconds() >= query_context->getSettingsRef()[Setting::interactive_delay]))
{
sendProfileEvents();
return true;

View File

@ -12,6 +12,16 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_experimental_analyzer;
extern const SettingsDialect dialect;
extern const SettingsUInt64 group_by_two_level_threshold;
extern const SettingsUInt64 group_by_two_level_threshold_bytes;
extern const SettingsUInt64 parallel_replicas_count;
extern const SettingsUInt64 parallel_replica_offset;
extern const SettingsSeconds receive_timeout;
}
// NOLINTBEGIN(bugprone-undefined-memory-manipulation)
@ -128,8 +138,8 @@ void MultiplexedConnections::sendQuery(
Settings modified_settings = settings;
/// Queries in foreign languages are transformed to ClickHouse-SQL. Ensure the setting before sending.
modified_settings.dialect = Dialect::clickhouse;
modified_settings.dialect.changed = false;
modified_settings[Setting::dialect] = Dialect::clickhouse;
modified_settings[Setting::dialect].changed = false;
for (auto & replica : replica_states)
{
@ -139,8 +149,8 @@ void MultiplexedConnections::sendQuery(
if (replica.connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD)
{
/// Disable two-level aggregation due to version incompatibility.
modified_settings.group_by_two_level_threshold = 0;
modified_settings.group_by_two_level_threshold_bytes = 0;
modified_settings[Setting::group_by_two_level_threshold] = 0;
modified_settings[Setting::group_by_two_level_threshold_bytes] = 0;
}
}
@ -154,7 +164,7 @@ void MultiplexedConnections::sendQuery(
/// Make the analyzer being set, so it will be effectively applied on the remote server.
/// In other words, the initiator always controls whether the analyzer enabled or not for
/// all servers involved in the distributed query processing.
modified_settings.set("allow_experimental_analyzer", static_cast<bool>(modified_settings.allow_experimental_analyzer));
modified_settings.set("allow_experimental_analyzer", static_cast<bool>(modified_settings[Setting::allow_experimental_analyzer]));
const bool enable_offset_parallel_processing = context->canUseOffsetParallelReplicas();
@ -163,12 +173,12 @@ void MultiplexedConnections::sendQuery(
{
if (enable_offset_parallel_processing)
/// Use multiple replicas for parallel query processing.
modified_settings.parallel_replicas_count = num_replicas;
modified_settings[Setting::parallel_replicas_count] = num_replicas;
for (size_t i = 0; i < num_replicas; ++i)
{
if (enable_offset_parallel_processing)
modified_settings.parallel_replica_offset = i;
modified_settings[Setting::parallel_replica_offset] = i;
replica_states[i].connection->sendQuery(
timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {});
@ -403,7 +413,7 @@ MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForRead
Poco::Net::Socket::SocketList write_list;
Poco::Net::Socket::SocketList except_list;
auto timeout = settings.receive_timeout;
auto timeout = settings[Setting::receive_timeout];
int n = 0;
/// EINTR loop

View File

@ -13,6 +13,13 @@
#include <filesystem>
#include <fstream>
namespace DB
{
namespace Setting
{
extern const SettingsTimezone session_timezone;
}
}
namespace
{
@ -203,5 +210,5 @@ DateLUT & DateLUT::getInstance()
std::string DateLUT::extractTimezoneFromContext(DB::ContextPtr query_context)
{
return query_context->getSettingsRef().session_timezone.value;
return query_context->getSettingsRef()[DB::Setting::session_timezone].value;
}

View File

@ -1,8 +1,9 @@
#include <Common/NamedCollections/NamedCollectionsFactory.h>
#include <Common/NamedCollections/NamedCollectionConfiguration.h>
#include <Common/NamedCollections/NamedCollectionsMetadataStorage.h>
#include <Core/Settings.h>
#include <base/sleep.h>
#include <Common/FieldVisitorToString.h>
#include <Common/NamedCollections/NamedCollectionConfiguration.h>
#include <Common/NamedCollections/NamedCollectionsFactory.h>
#include <Common/NamedCollections/NamedCollectionsMetadataStorage.h>
namespace DB
{

View File

@ -1,28 +1,36 @@
#include <Common/NamedCollections/NamedCollectionsMetadataStorage.h>
#include <Common/NamedCollections/NamedCollectionConfiguration.h>
#include <Common/escapeForFileName.h>
#include <Common/logger_useful.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <filesystem>
#include <Core/Settings.h>
#include <IO/FileEncryptionCommon.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/Context.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/formatAST.h>
#include <Interpreters/Context.h>
#include <filesystem>
#include <Parsers/parseQuery.h>
#include <boost/algorithm/hex.hpp>
#include <Common/NamedCollections/NamedCollectionConfiguration.h>
#include <Common/NamedCollections/NamedCollectionsMetadataStorage.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/escapeForFileName.h>
#include <Common/logger_useful.h>
namespace fs = std::filesystem;
namespace DB
{
namespace Setting
{
extern const SettingsBool fsync_metadata;
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
}
namespace ErrorCodes
{
extern const int NAMED_COLLECTION_ALREADY_EXISTS;
@ -157,7 +165,7 @@ public:
writeString(write_data, out);
out.next();
if (getContext()->getSettingsRef().fsync_metadata)
if (getContext()->getSettingsRef()[Setting::fsync_metadata])
out.sync();
out.close();
@ -573,7 +581,7 @@ ASTCreateNamedCollectionQuery NamedCollectionsMetadataStorage::readCreateQuery(c
const auto & settings = getContext()->getSettingsRef();
ParserCreateNamedCollectionQuery parser;
auto ast = parseQuery(parser, query, "in file " + path, 0, settings.max_parser_depth, settings.max_parser_backtracks);
auto ast = parseQuery(parser, query, "in file " + path, 0, settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks]);
const auto & create_query = ast->as<const ASTCreateNamedCollectionQuery &>();
return create_query;
}

View File

@ -14,6 +14,12 @@
namespace DB
{
namespace Setting
{
extern const SettingsFloat opentelemetry_start_trace_probability;
}
namespace OpenTelemetry
{
@ -329,7 +335,7 @@ TracingContextHolder::TracingContextHolder(
return;
// Start the trace with some configurable probability.
std::bernoulli_distribution should_start_trace{settings_ptr->opentelemetry_start_trace_probability};
std::bernoulli_distribution should_start_trace{(*settings_ptr)[Setting::opentelemetry_start_trace_probability]};
if (!should_start_trace(thread_local_rng))
/// skip tracing context initialization on current thread
return;

View File

@ -2,6 +2,7 @@
#include <Core/Field.h>
#include <vector>
namespace DB
{

View File

@ -1,12 +1,13 @@
#pragma once
#include <unordered_map>
#include <Core/SettingsFields.h>
#include <Common/SettingsChanges.h>
#include <Common/FieldVisitorToString.h>
#include <Core/SettingsWriteFormat.h>
#include <IO/Operators.h>
#include <base/range.h>
#include <boost/blank.hpp>
#include <unordered_map>
#include <Common/FieldVisitorToString.h>
#include <Common/SettingsChanges.h>
namespace boost::program_options
@ -20,13 +21,6 @@ namespace DB
class ReadBuffer;
class WriteBuffer;
enum class SettingsWriteFormat : uint8_t
{
BINARY = 0, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour.
STRINGS_WITH_FLAGS = 1, /// All settings are serialized as strings. Before each value the flag `is_important` is serialized.
DEFAULT = STRINGS_WITH_FLAGS,
};
/** Template class to define collections of settings.
* Example of usage:
*
@ -96,6 +90,8 @@ public:
static String valueToStringUtil(std::string_view name, const Field & value);
static Field stringToValueUtil(std::string_view name, const String & str);
static std::string_view resolveName(std::string_view name);
void write(WriteBuffer & out, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT) const;
void read(ReadBuffer & in, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT);
@ -191,8 +187,6 @@ public:
MutableRange allMutable(SkipFlags skip_flags = SKIP_NONE) { return MutableRange{*this, skip_flags}; }
Range allChanged() const { return all(SKIP_UNCHANGED); }
Range allUnchanged() const { return all(SKIP_CHANGED); }
Range allBuiltin() const { return all(SKIP_CUSTOM); }
Range allCustom() const { return all(SKIP_BUILTIN); }
Iterator begin() const { return allChanged().begin(); }
Iterator end() const { return allChanged().end(); }

View File

@ -0,0 +1,5 @@
#pragma once
#define DECLARE_SETTING_TRAIT(CLASS_NAME, TYPE) using CLASS_NAME##TYPE = SettingField##TYPE CLASS_NAME##Impl::*;
#define DECLARE_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) SettingField##TYPE & operator[](CLASS_NAME##TYPE t) const;

View File

@ -27,6 +27,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 http_max_multipart_form_data_size;
}
namespace ErrorCodes
{
@ -182,10 +186,12 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header,
const Settings & settings = getContext()->getSettingsRef();
if (settings.http_max_multipart_form_data_size)
if (settings[Setting::http_max_multipart_form_data_size])
read_buffer = std::make_unique<LimitReadBuffer>(
stream, settings.http_max_multipart_form_data_size,
/* trow_exception */ true, /* exact_limit */ std::optional<size_t>(),
stream,
settings[Setting::http_max_multipart_form_data_size],
/* trow_exception */ true,
/* exact_limit */ std::optional<size_t>(),
"the maximum size of multipart/form-data. "
"This limit can be tuned by 'http_max_multipart_form_data_size' setting");
else

View File

@ -0,0 +1,18 @@
#include <Core/BaseSettings.h>
#include <Core/FormatFactorySettingsDeclaration.h>
#include <Core/SettingsEnums.h>
namespace DB
{
/*
* User-specified file format settings for File and URL engines.
*/
DECLARE_SETTINGS_TRAITS(FormatFactorySettingsTraits, LIST_OF_ALL_FORMAT_SETTINGS)
struct FormatFactorySettingsImpl : public BaseSettings<FormatFactorySettingsTraits>
{
};
IMPLEMENT_SETTINGS_TRAITS(FormatFactorySettingsTraits, LIST_OF_ALL_FORMAT_SETTINGS)
}

View File

@ -0,0 +1,55 @@
#pragma once
#include <Core/BaseSettingsFwdMacros.h>
#include <Core/SettingsEnums.h>
#include <Core/SettingsFields.h>
#include <base/types.h>
namespace DB
{
struct FormatFactorySettingsImpl;
struct SettingChange;
class SettingsChanges;
#define FORMAT_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
M(CLASS_NAME, Bool) \
M(CLASS_NAME, Char) \
M(CLASS_NAME, Int64) \
M(CLASS_NAME, UInt64) \
M(CLASS_NAME, MsgPackUUIDRepresentation) \
M(CLASS_NAME, SchemaInferenceMode) \
M(CLASS_NAME, UInt64Auto) \
M(CLASS_NAME, DateTimeInputFormat) \
M(CLASS_NAME, DateTimeOutputFormat) \
M(CLASS_NAME, IntervalOutputFormat) \
M(CLASS_NAME, String) \
M(CLASS_NAME, ParquetVersion) \
M(CLASS_NAME, ParquetCompression) \
M(CLASS_NAME, EscapingRule) \
M(CLASS_NAME, ArrowCompression) \
M(CLASS_NAME, CapnProtoEnumComparingMode) \
M(CLASS_NAME, DateTimeOverflowBehavior) \
M(CLASS_NAME, IdentifierQuotingStyle)
FORMAT_SETTINGS_SUPPORTED_TYPES(FormatFactorySettings, DECLARE_SETTING_TRAIT)
struct FormatFactorySettings
{
FormatFactorySettings();
~FormatFactorySettings();
FORMAT_SETTINGS_SUPPORTED_TYPES(FormatFactorySettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
/// General API as needed
bool tryGet(std::string_view name, Field & value) const;
Field get(std::string_view name) const;
void set(std::string_view name, const Field & value);
bool has(std::string_view name) const;
void applyChange(const SettingChange & change);
void applyChanges(const SettingsChanges & changes);
private:
std::unique_ptr<FormatFactorySettingsImpl> impl;
};
}

View File

@ -0,0 +1,275 @@
#pragma once
#include <Core/SettingsObsoleteMacros.h>
/// This header exists so we can share it between Settings.cpp, FormatFactorySettings.cpp and other storage settings
// clang-format off
#if defined(__CLION_IDE__)
/// CLion freezes for a minute every time is processes this
#define FORMAT_FACTORY_SETTINGS(M, ALIAS)
#define OBSOLETE_FORMAT_SETTINGS(M, ALIAS)
#else
#define FORMAT_FACTORY_SETTINGS(M, ALIAS) \
M(Char, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \
M(Bool, format_csv_allow_single_quotes, false, "If it is set to true, allow strings in single quotes.", 0) \
M(Bool, format_csv_allow_double_quotes, true, "If it is set to true, allow strings in double quotes.", 0) \
M(Bool, output_format_csv_serialize_tuple_into_separate_columns, true, "If it set to true, then Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost)", 0) \
M(Bool, input_format_csv_deserialize_separate_columns_into_tuple, true, "If it set to true, then separate columns written in CSV format can be deserialized to Tuple column.", 0) \
M(Bool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \
M(Bool, input_format_csv_allow_cr_end_of_line, false, "If it is set true, \\r will be allowed at end of line not followed by \\n", 0) \
M(Bool, input_format_csv_enum_as_number, false, "Treat inserted enum values in CSV formats as enum indices", 0) \
M(Bool, input_format_csv_arrays_as_nested_csv, false, R"(When reading Array from CSV, expect that its elements were serialized in nested CSV and then put into string. Example: "[""Hello"", ""world"", ""42"""" TV""]". Braces around array can be omitted.)", 0) \
M(Bool, input_format_skip_unknown_fields, true, "Skip columns with unknown names from input data (it works for JSONEachRow, -WithNames, -WithNamesAndTypes and TSKV formats).", 0) \
M(Bool, input_format_with_names_use_header, true, "For -WithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \
M(Bool, input_format_with_types_use_header, true, "For -WithNamesAndTypes input formats this controls whether format parser should check if data types from the input match data types from the header.", 0) \
M(Bool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \
M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, -WithNames, -WithNamesAndTypes formats).", IMPORTANT) \
M(Bool, input_format_csv_empty_as_default, true, "Treat empty fields in CSV input as default values.", 0) \
M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \
M(Bool, input_format_tsv_enum_as_number, false, "Treat inserted enum values in TSV formats as enum indices.", 0) \
M(Bool, input_format_null_as_default, true, "Initialize null fields with default values if the data type of this field is not nullable and it is supported by the input format", 0) \
M(Bool, input_format_force_null_for_omitted_fields, false, "Force initialize omitted fields with null values", 0) \
M(Bool, input_format_arrow_case_insensitive_column_matching, false, "Ignore case when matching Arrow columns with CH columns.", 0) \
M(Int64, input_format_orc_row_batch_size, 100'000, "Batch size when reading ORC stripes.", 0) \
M(Bool, input_format_orc_case_insensitive_column_matching, false, "Ignore case when matching ORC columns with CH columns.", 0) \
M(Bool, input_format_parquet_case_insensitive_column_matching, false, "Ignore case when matching Parquet columns with CH columns.", 0) \
M(Bool, input_format_parquet_preserve_order, false, "Avoid reordering rows when reading from Parquet files. Usually makes it much slower.", 0) \
M(Bool, input_format_parquet_filter_push_down, true, "When reading Parquet files, skip whole row groups based on the WHERE/PREWHERE expressions and min/max statistics in the Parquet metadata.", 0) \
M(Bool, input_format_parquet_use_native_reader, false, "When reading Parquet files, to use native reader instead of arrow reader.", 0) \
M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \
M(Bool, input_format_orc_allow_missing_columns, true, "Allow missing columns while reading ORC input formats", 0) \
M(Bool, input_format_orc_use_fast_decoder, true, "Use a faster ORC decoder implementation.", 0) \
M(Bool, input_format_orc_filter_push_down, true, "When reading ORC files, skip whole stripes or row groups based on the WHERE/PREWHERE expressions, min/max statistics or bloom filter in the ORC metadata.", 0) \
M(String, input_format_orc_reader_time_zone_name, "GMT", "The time zone name for ORC row reader, the default ORC row reader's time zone is GMT.", 0) \
M(Bool, input_format_parquet_allow_missing_columns, true, "Allow missing columns while reading Parquet input formats", 0) \
M(UInt64, input_format_parquet_local_file_min_bytes_for_seek, 8192, "Min bytes required for local read (file) to do seek, instead of read with ignore in Parquet input format", 0) \
M(Bool, input_format_arrow_allow_missing_columns, true, "Allow missing columns while reading Arrow input formats", 0) \
M(Char, input_format_hive_text_fields_delimiter, '\x01', "Delimiter between fields in Hive Text File", 0) \
M(Char, input_format_hive_text_collection_items_delimiter, '\x02', "Delimiter between collection(array or map) items in Hive Text File", 0) \
M(Char, input_format_hive_text_map_keys_delimiter, '\x03', "Delimiter between a pair of map key/values in Hive Text File", 0) \
M(Bool, input_format_hive_text_allow_variable_number_of_columns, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values", 0) \
M(UInt64, input_format_msgpack_number_of_columns, 0, "The number of columns in inserted MsgPack data. Used for automatic schema inference from data.", 0) \
M(MsgPackUUIDRepresentation, output_format_msgpack_uuid_representation, FormatSettings::MsgPackUUIDRepresentation::EXT, "The way how to output UUID in MsgPack format.", 0) \
M(UInt64, input_format_max_rows_to_read_for_schema_inference, 25000, "The maximum rows of data to read for automatic schema inference", 0) \
M(UInt64, input_format_max_bytes_to_read_for_schema_inference, 32 * 1024 * 1024, "The maximum bytes of data to read for automatic schema inference", 0) \
M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \
M(Bool, input_format_csv_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference in CSV format", 0) \
M(Bool, input_format_csv_try_infer_strings_from_quoted_tuples, true, "Interpret quoted tuples in the input data as a value of type String.", 0) \
M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \
M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \
M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \
M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \
M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialization failed on bad value", 0) \
M(Bool, input_format_csv_allow_variable_number_of_columns, false, "Ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values", 0) \
M(Bool, input_format_tsv_allow_variable_number_of_columns, false, "Ignore extra columns in TSV input (if file has more columns than expected) and treat missing fields in TSV input as default values", 0) \
M(Bool, input_format_custom_allow_variable_number_of_columns, false, "Ignore extra columns in CustomSeparated input (if file has more columns than expected) and treat missing fields in CustomSeparated input as default values", 0) \
M(Bool, input_format_json_compact_allow_variable_number_of_columns, false, "Ignore extra columns in JSONCompact(EachRow) input (if file has more columns than expected) and treat missing fields in JSONCompact(EachRow) input as default values", 0) \
M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \
M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \
M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \
M(UInt64, input_format_parquet_max_block_size, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader.", 0) \
M(UInt64, input_format_parquet_prefer_block_bytes, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader", 0) \
M(Bool, input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format Protobuf", 0) \
M(Bool, input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format CapnProto", 0) \
M(Bool, input_format_orc_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format ORC", 0) \
M(Bool, input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Arrow", 0) \
M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \
M(String, schema_inference_hints, "", "The list of column names and types to use in schema inference for formats without column names. The format: 'column_name1 column_type1, column_name2 column_type2, ...'", 0) \
M(SchemaInferenceMode, schema_inference_mode, "default", "Mode of schema inference. 'default' - assume that all files have the same schema and schema can be inferred from any file, 'union' - files can have different schemas and the resulting schema should be the a union of schemas of all files", 0) \
M(UInt64Auto, schema_inference_make_columns_nullable, 1, "If set to true, all inferred types will be Nullable in schema inference. When set to false, no columns will be converted to Nullable. When set to 'auto', ClickHouse will use information about nullability from the data.", 0) \
M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \
M(Bool, input_format_json_read_bools_as_strings, true, "Allow to parse bools as strings in JSON input formats", 0) \
M(Bool, input_format_json_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference", 0) \
M(Bool, input_format_json_validate_types_from_metadata, true, "For JSON/JSONCompact/JSONColumnsWithMetadata input formats this controls whether format parser should check if data types from input metadata match data types of the corresponding columns from the table", 0) \
M(Bool, input_format_json_read_numbers_as_strings, true, "Allow to parse numbers as strings in JSON input formats", 0) \
M(Bool, input_format_json_read_objects_as_strings, true, "Allow to parse JSON objects as strings in JSON input formats", 0) \
M(Bool, input_format_json_read_arrays_as_strings, true, "Allow to parse JSON arrays as strings in JSON input formats", 0) \
M(Bool, input_format_json_try_infer_named_tuples_from_objects, true, "Try to infer named tuples from JSON objects in JSON input formats", 0) \
M(Bool, input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects, false, "Use String type instead of an exception in case of ambiguous paths in JSON objects during named tuples inference", 0) \
M(Bool, input_format_json_infer_incomplete_types_as_strings, true, "Use type String for keys that contains only Nulls or empty objects/arrays during schema inference in JSON input formats", 0) \
M(Bool, input_format_json_named_tuples_as_objects, true, "Deserialize named tuple columns as JSON objects", 0) \
M(Bool, input_format_json_ignore_unknown_keys_in_named_tuple, true, "Ignore unknown keys in json object for named tuples", 0) \
M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \
M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \
M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \
M(Bool, input_format_try_infer_variants, false, "Try to infer the Variant type in text formats when there is more than one possible type for column/array elements", 0) \
M(Bool, type_json_skip_duplicated_paths, false, "When enabled, during parsing JSON object into JSON type duplicated paths will be ignored and only the first one will be inserted instead of an exception", 0) \
M(UInt64, input_format_json_max_depth, 1000, "Maximum depth of a field in JSON. This is not a strict limit, it does not have to be applied precisely.", 0) \
M(Bool, input_format_json_empty_as_default, false, "Treat empty fields in JSON input as default values.", 0) \
M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_datetimes_only_datetime64, false, "When input_format_try_infer_datetimes is enabled, infer only DateTime64 but not DateTime types", 0) \
M(Bool, input_format_try_infer_exponent_floats, false, "Try to infer floats in exponential notation while schema inference in text formats (except JSON, where exponent numbers are always inferred)", 0) \
M(Bool, output_format_markdown_escape_special_characters, false, "Escape special characters in Markdown", 0) \
M(Bool, input_format_protobuf_flatten_google_wrappers, false, "Enable Google wrappers for regular non-nested columns, e.g. google.protobuf.StringValue 'str' for String column 'str'. For Nullable columns empty wrappers are recognized as defaults, and missing as nulls", 0) \
M(Bool, output_format_protobuf_nullables_with_google_wrappers, false, "When serializing Nullable columns with Google wrappers, serialize default values as empty wrappers. If turned off, default and null values are not serialized", 0) \
M(UInt64, input_format_csv_skip_first_lines, 0, "Skip specified number of lines at the beginning of data in CSV format", 0) \
M(UInt64, input_format_tsv_skip_first_lines, 0, "Skip specified number of lines at the beginning of data in TSV format", 0) \
M(Bool, input_format_csv_skip_trailing_empty_lines, false, "Skip trailing empty lines in CSV format", 0) \
M(Bool, input_format_tsv_skip_trailing_empty_lines, false, "Skip trailing empty lines in TSV format", 0) \
M(Bool, input_format_custom_skip_trailing_empty_lines, false, "Skip trailing empty lines in CustomSeparated format", 0) \
M(Bool, input_format_tsv_crlf_end_of_line, false, "If it is set true, file function will read TSV format with \\r\\n instead of \\n.", 0) \
\
M(Bool, input_format_native_allow_types_conversion, true, "Allow data types conversion in Native input format", 0) \
M(Bool, input_format_native_decode_types_in_binary_format, false, "Read data types in binary format instead of type names in Native input format", 0) \
M(Bool, output_format_native_encode_types_in_binary_format, false, "Write data types in binary format instead of type names in Native output format", 0) \
\
M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \
M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \
M(IntervalOutputFormat, interval_output_format, FormatSettings::IntervalOutputFormat::Numeric, "Textual representation of Interval. Possible values: 'kusto', 'numeric'.", 0) \
\
M(Bool, input_format_ipv4_default_on_conversion_error, false, "Deserialization of IPv4 will use default values instead of throwing exception on conversion error.", 0) \
M(Bool, input_format_ipv6_default_on_conversion_error, false, "Deserialization of IPV6 will use default values instead of throwing exception on conversion error.", 0) \
M(String, bool_true_representation, "true", "Text to represent bool value in TSV/CSV formats.", 0) \
M(String, bool_false_representation, "false", "Text to represent bool value in TSV/CSV formats.", 0) \
\
M(Bool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \
M(Bool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \
M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \
M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \
/** This setting is obsolete and do nothing, left for compatibility reasons. */ \
M(Bool, input_format_avro_null_as_default, false, "For Avro/AvroConfluent format: insert default in case of null and non Nullable column", 0) \
M(UInt64, format_binary_max_string_size, 1_GiB, "The maximum allowed size for String in RowBinary format. It prevents allocating large amount of memory in case of corrupted data. 0 means there is no limit", 0) \
M(UInt64, format_binary_max_array_size, 1_GiB, "The maximum allowed size for Array in RowBinary format. It prevents allocating large amount of memory in case of corrupted data. 0 means there is no limit", 0) \
M(Bool, input_format_binary_decode_types_in_binary_format, false, "Read data types in binary format instead of type names in RowBinaryWithNamesAndTypes input format", 0) \
M(Bool, output_format_binary_encode_types_in_binary_format, false, "Write data types in binary format instead of type names in RowBinaryWithNamesAndTypes output format ", 0) \
M(URI, format_avro_schema_registry_url, "", "For AvroConfluent format: Confluent Schema Registry URL.", 0) \
\
M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \
M(Bool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \
M(Bool, output_format_json_quote_decimals, false, "Controls quoting of decimals in JSON output format.", 0) \
M(Bool, output_format_json_quote_64bit_floats, false, "Controls quoting of 64-bit float numbers in JSON output format.", 0) \
\
M(Bool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.", 0) \
M(Bool, output_format_json_named_tuples_as_objects, true, "Serialize named tuple columns as JSON objects.", 0) \
M(Bool, output_format_json_skip_null_value_in_named_tuples, false, "Skip key value pairs with null value when serialize named tuple columns as JSON objects. It is only valid when output_format_json_named_tuples_as_objects is true.", 0) \
M(Bool, output_format_json_array_of_rows, false, "Output a JSON array of all rows in JSONEachRow(Compact) format.", 0) \
M(Bool, output_format_json_validate_utf8, false, "Validate UTF-8 sequences in JSON output formats, doesn't impact formats JSON/JSONCompact/JSONColumnsWithMetadata, they always validate utf8", 0) \
\
M(String, format_json_object_each_row_column_for_object_name, "", "The name of column that will be used as object names in JSONObjectEachRow format. Column type should be String", 0) \
\
M(UInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \
M(UInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \
M(UInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \
M(UInt64, output_format_pretty_max_value_width_apply_for_single_value, false, "Only cut values (see the `output_format_pretty_max_value_width` setting) when it is not a single value in a block. Otherwise output it entirely, which is useful for the `SHOW CREATE TABLE` query.", 0) \
M(UInt64Auto, output_format_pretty_color, "auto", "Use ANSI escape sequences in Pretty formats. 0 - disabled, 1 - enabled, 'auto' - enabled if a terminal.", 0) \
M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \
M(UInt64, output_format_pretty_display_footer_column_names, true, "Display column names in the footer if there are 999 or more rows.", 0) \
M(UInt64, output_format_pretty_display_footer_column_names_min_rows, 50, "Sets the minimum threshold value of rows for which to enable displaying column names in the footer. 50 (default)", 0) \
M(UInt64, output_format_parquet_row_group_size, 1000000, "Target row group size in rows.", 0) \
M(UInt64, output_format_parquet_row_group_size_bytes, 512 * 1024 * 1024, "Target row group size in bytes, before compression.", 0) \
M(Bool, output_format_parquet_string_as_string, true, "Use Parquet String type instead of Binary for String columns.", 0) \
M(Bool, output_format_parquet_fixed_string_as_fixed_byte_array, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary for FixedString columns.", 0) \
M(ParquetVersion, output_format_parquet_version, "2.latest", "Parquet format version for output format. Supported versions: 1.0, 2.4, 2.6 and 2.latest (default)", 0) \
M(ParquetCompression, output_format_parquet_compression_method, "zstd", "Compression method for Parquet output format. Supported codecs: snappy, lz4, brotli, zstd, gzip, none (uncompressed)", 0) \
M(Bool, output_format_parquet_compliant_nested_types, true, "In parquet file schema, use name 'element' instead of 'item' for list elements. This is a historical artifact of Arrow library implementation. Generally increases compatibility, except perhaps with some old versions of Arrow.", 0) \
M(Bool, output_format_parquet_use_custom_encoder, true, "Use a faster Parquet encoder implementation.", 0) \
M(Bool, output_format_parquet_parallel_encoding, true, "Do Parquet encoding in multiple threads. Requires output_format_parquet_use_custom_encoder.", 0) \
M(UInt64, output_format_parquet_data_page_size, 1024 * 1024, "Target page size in bytes, before compression.", 0) \
M(UInt64, output_format_parquet_batch_size, 1024, "Check page size every this many rows. Consider decreasing if you have columns with average values size above a few KBs.", 0) \
M(Bool, output_format_parquet_write_page_index, true, "Add a possibility to write page index into parquet files.", 0) \
M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy', 'zstd'.", 0) \
M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \
M(String, output_format_avro_string_column_pattern, "", "For Avro format: regexp of String columns to select as AVRO string.", 0) \
M(UInt64, output_format_avro_rows_in_file, 1, "Max rows in a file (if permitted by storage)", 0) \
M(Bool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \
M(String, format_csv_null_representation, "\\N", "Custom NULL representation in CSV format", 0) \
M(String, format_tsv_null_representation, "\\N", "Custom NULL representation in TSV format", 0) \
M(Bool, output_format_decimal_trailing_zeros, false, "Output trailing zeros when printing Decimal values. E.g. 1.230000 instead of 1.23.", 0) \
\
M(UInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
M(Float, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
M(String, input_format_record_errors_file_path, "", "Path of the file used to record errors while reading text formats (CSV, TSV).", 0) \
M(String, errors_output_format, "CSV", "Method to write Errors to text output.", 0) \
\
M(String, format_schema, "", "Schema identifier (used by schema-based formats)", 0) \
M(String, format_template_resultset, "", "Path to file which contains format string for result set (for Template format)", 0) \
M(String, format_template_row, "", "Path to file which contains format string for rows (for Template format)", 0) \
M(String, format_template_row_format, "", "Format string for rows (for Template format)", 0) \
M(String, format_template_resultset_format, "", "Format string for result set (for Template format)", 0) \
M(String, format_template_rows_between_delimiter, "\n", "Delimiter between rows (for Template format)", 0) \
\
M(EscapingRule, format_custom_escaping_rule, "Escaped", "Field escaping rule (for CustomSeparated format)", 0) \
M(String, format_custom_field_delimiter, "\t", "Delimiter between fields (for CustomSeparated format)", 0) \
M(String, format_custom_row_before_delimiter, "", "Delimiter before field of the first column (for CustomSeparated format)", 0) \
M(String, format_custom_row_after_delimiter, "\n", "Delimiter after field of the last column (for CustomSeparated format)", 0) \
M(String, format_custom_row_between_delimiter, "", "Delimiter between rows (for CustomSeparated format)", 0) \
M(String, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)", 0) \
M(String, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \
\
M(String, format_regexp, "", "Regular expression (for Regexp format)", 0) \
M(EscapingRule, format_regexp_escaping_rule, "Raw", "Field escaping rule (for Regexp format)", 0) \
M(Bool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format)", 0) \
\
M(Bool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \
M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \
M(Bool, output_format_pretty_row_numbers, true, "Add row numbers before each row for pretty output format", 0) \
M(Bool, output_format_pretty_highlight_digit_groups, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline.", 0) \
M(UInt64, output_format_pretty_single_large_number_tip_threshold, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)", 0) \
M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \
\
M(Bool, exact_rows_before_limit, false, "When enabled, ClickHouse will provide exact value for rows_before_limit_at_least statistic, but with the cost that the data before limit will have to be read completely", 0) \
M(Bool, rows_before_aggregation, false, "When enabled, ClickHouse will provide exact value for rows_before_aggregation statistic, represents the number of rows read before aggregation", 0) \
M(UInt64, cross_to_inner_join_rewrite, 1, "Use inner join instead of comma/cross join if there are joining expressions in the WHERE section. Values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible", 0) \
\
M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \
M(Bool, output_format_arrow_use_signed_indexes_for_dictionary, true, "Use signed integers for dictionary indexes in Arrow format", 0) \
M(Bool, output_format_arrow_use_64_bit_indexes_for_dictionary, false, "Always use 64 bit integers for dictionary indexes in Arrow format", 0) \
M(Bool, output_format_arrow_string_as_string, true, "Use Arrow String type instead of Binary for String columns", 0) \
M(Bool, output_format_arrow_fixed_string_as_fixed_byte_array, true, "Use Arrow FIXED_SIZE_BINARY type instead of Binary for FixedString columns.", 0) \
M(ArrowCompression, output_format_arrow_compression_method, "lz4_frame", "Compression method for Arrow output format. Supported codecs: lz4_frame, zstd, none (uncompressed)", 0) \
\
M(Bool, output_format_orc_string_as_string, true, "Use ORC String type instead of Binary for String columns", 0) \
M(ORCCompression, output_format_orc_compression_method, "zstd", "Compression method for ORC output format. Supported codecs: lz4, snappy, zlib, zstd, none (uncompressed)", 0) \
M(UInt64, output_format_orc_row_index_stride, 10'000, "Target row index stride in ORC output format", 0) \
M(Double, output_format_orc_dictionary_key_size_threshold, 0.0, "For a string column in ORC output format, if the number of distinct values is greater than this fraction of the total number of non-null rows, turn off dictionary encoding. Otherwise dictionary encoding is enabled", 0) \
\
M(CapnProtoEnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::CapnProtoEnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \
\
M(Bool, format_capn_proto_use_autogenerated_schema, true, "Use autogenerated CapnProto schema when format_schema is not set", 0) \
M(Bool, format_protobuf_use_autogenerated_schema, true, "Use autogenerated Protobuf when format_schema is not set", 0) \
M(String, output_format_schema, "", "The path to the file where the automatically generated schema will be saved", 0) \
\
M(String, input_format_mysql_dump_table_name, "", "Name of the table in MySQL dump from which to read data", 0) \
M(Bool, input_format_mysql_dump_map_column_names, true, "Match columns from table in MySQL dump and columns from ClickHouse table by names", 0) \
\
M(UInt64, output_format_sql_insert_max_batch_size, DEFAULT_BLOCK_SIZE, "The maximum number of rows in one INSERT statement.", 0) \
M(String, output_format_sql_insert_table_name, "table", "The name of table in the output INSERT query", 0) \
M(Bool, output_format_sql_insert_include_column_names, true, "Include column names in INSERT query", 0) \
M(Bool, output_format_sql_insert_use_replace, false, "Use REPLACE statement instead of INSERT", 0) \
M(Bool, output_format_sql_insert_quote_names, true, "Quote column names with '`' characters", 0) \
\
M(Bool, output_format_values_escape_quote_with_quote, false, "If true escape ' with '', otherwise quoted with \\'", 0) \
\
M(Bool, output_format_bson_string_as_string, false, "Use BSON String type instead of Binary for String columns.", 0) \
M(Bool, input_format_bson_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format BSON.", 0) \
\
M(Bool, format_display_secrets_in_show_and_select, false, "Do not hide secrets in SHOW and SELECT queries.", IMPORTANT) \
M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \
M(Bool, regexp_dict_flag_case_insensitive, false, "Use case-insensitive matching for a regexp_tree dictionary. Can be overridden in individual expressions with (?i) and (?-i).", 0) \
M(Bool, regexp_dict_flag_dotall, false, "Allow '.' to match newline characters for a regexp_tree dictionary.", 0) \
\
M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading dictionary source in several threads. It's supported only by dictionaries with local CLICKHOUSE source.", 0) \
M(Bool, precise_float_parsing, false, "Prefer more precise (but slower) float parsing algorithm", 0) \
M(DateTimeOverflowBehavior, date_time_overflow_behavior, "ignore", "Overflow mode for Date, Date32, DateTime, DateTime64 types. Possible values: 'ignore', 'throw', 'saturate'.", 0) \
M(Bool, validate_experimental_and_suspicious_types_inside_nested_types, true, "Validate usage of experimental and suspicious types inside nested types like Array/Map/Tuple", 0) \
\
M(Bool, output_format_always_quote_identifiers, false, "Always quote identifiers", 0) \
M(IdentifierQuotingStyle, output_format_identifier_quoting_style, IdentifierQuotingStyle::Backticks, "Set the quoting style for identifiers", 0) \
// End of FORMAT_FACTORY_SETTINGS
#define OBSOLETE_FORMAT_SETTINGS(M, ALIAS) \
/** Obsolete format settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
MAKE_OBSOLETE(M, Bool, input_format_arrow_import_nested, false) \
MAKE_OBSOLETE(M, Bool, input_format_parquet_import_nested, false) \
MAKE_OBSOLETE(M, Bool, input_format_orc_import_nested, false) \
#endif // __CLION_IDE__
#define LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS) \
FORMAT_FACTORY_SETTINGS(M, ALIAS) \
OBSOLETE_FORMAT_SETTINGS(M, ALIAS)

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,9 @@
#pragma once
// clang-format off
#define MAKE_OBSOLETE(M, TYPE, NAME, DEFAULT) \
M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE)
/// NOTE: ServerSettings::loadSettingsFromConfig() should be updated to include this settings
#define MAKE_DEPRECATED_BY_SERVER_CONFIG(M, TYPE, NAME, DEFAULT) \
M(TYPE, NAME, DEFAULT, "User-level setting is deprecated, and it must be defined in the server configuration instead.", BaseSettingsHelpers::Flags::OBSOLETE)

View File

@ -1,6 +1,6 @@
#include <Core/Defines.h>
#include <Core/Settings.h>
#include <Core/SettingsQuirks.h>
#include <base/defines.h>
#include <Poco/Environment.h>
#include <Poco/Platform.h>
#include <Common/VersionNumber.h>
@ -48,26 +48,35 @@ bool queryProfilerWorks() { return false; }
namespace DB
{
namespace Setting
{
extern const SettingsBool async_query_sending_for_remote;
extern const SettingsBool async_socket_for_remote;
extern const SettingsUInt64 query_profiler_cpu_time_period_ns;
extern const SettingsUInt64 query_profiler_real_time_period_ns;
extern const SettingsBool use_hedged_requests;
}
/// Update some settings defaults to avoid some known issues.
void applySettingsQuirks(Settings & settings, LoggerPtr log)
{
if (!nestedEpollWorks(log))
{
if (!settings.async_socket_for_remote.changed && settings.async_socket_for_remote)
if (!settings[Setting::async_socket_for_remote].changed && settings[Setting::async_socket_for_remote])
{
settings.async_socket_for_remote = false;
settings[Setting::async_socket_for_remote] = false;
if (log)
LOG_WARNING(log, "async_socket_for_remote has been disabled (you can explicitly enable it still)");
}
if (!settings.async_query_sending_for_remote.changed && settings.async_query_sending_for_remote)
if (!settings[Setting::async_query_sending_for_remote].changed && settings[Setting::async_query_sending_for_remote])
{
settings.async_query_sending_for_remote = false;
settings[Setting::async_query_sending_for_remote] = false;
if (log)
LOG_WARNING(log, "async_query_sending_for_remote has been disabled (you can explicitly enable it still)");
}
if (!settings.use_hedged_requests.changed && settings.use_hedged_requests)
if (!settings[Setting::use_hedged_requests].changed && settings[Setting::use_hedged_requests])
{
settings.use_hedged_requests = false;
settings[Setting::use_hedged_requests] = false;
if (log)
LOG_WARNING(log, "use_hedged_requests has been disabled (you can explicitly enable it still)");
}
@ -75,15 +84,15 @@ void applySettingsQuirks(Settings & settings, LoggerPtr log)
if (!queryProfilerWorks())
{
if (settings.query_profiler_real_time_period_ns)
if (settings[Setting::query_profiler_real_time_period_ns])
{
settings.query_profiler_real_time_period_ns = 0;
settings[Setting::query_profiler_real_time_period_ns] = 0;
if (log)
LOG_WARNING(log, "query_profiler_real_time_period_ns has been disabled (due to server had been compiled with sanitizers)");
}
if (settings.query_profiler_cpu_time_period_ns)
if (settings[Setting::query_profiler_cpu_time_period_ns])
{
settings.query_profiler_cpu_time_period_ns = 0;
settings[Setting::query_profiler_cpu_time_period_ns] = 0;
if (log)
LOG_WARNING(log, "query_profiler_cpu_time_period_ns has been disabled (due to server had been compiled with sanitizers)");
}

View File

@ -0,0 +1,10 @@
#pragma once
#include <cstdint>
enum class SettingsWriteFormat : uint8_t
{
BINARY = 0, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour.
STRINGS_WITH_FLAGS = 1, /// All settings are serialized as strings. Before each value the flag `is_important` is serialized.
DEFAULT = STRINGS_WITH_FLAGS,
};

View File

@ -18,12 +18,13 @@
#if defined(OS_LINUX)
#include <sys/prctl.h>
#endif
#include <algorithm>
#include <cerrno>
#include <cstring>
#include <unistd.h>
#include <algorithm>
#include <iostream>
#include <memory>
#include <sstream>
#include <unistd.h>
#include <Poco/Message.h>
#include <Poco/Util/Application.h>

View File

@ -5,6 +5,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool decimal_check_overflow;
}
namespace ErrorCodes
{
@ -12,11 +16,11 @@ namespace ErrorCodes
bool decimalCheckComparisonOverflow(ContextPtr context)
{
return context->getSettingsRef().decimal_check_overflow;
return context->getSettingsRef()[Setting::decimal_check_overflow];
}
bool decimalCheckArithmeticOverflow(ContextPtr context)
{
return context->getSettingsRef().decimal_check_overflow;
return context->getSettingsRef()[Setting::decimal_check_overflow];
}
template <is_decimal T>

View File

@ -17,6 +17,10 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool log_queries;
}
namespace ErrorCodes
{
@ -221,7 +225,7 @@ const DataTypeFactory::Value * DataTypeFactory::findCreatorByName(const String &
DataTypesDictionary::const_iterator it = data_types.find(family_name);
if (data_types.end() != it)
{
if (query_context && query_context->getSettingsRef().log_queries)
if (query_context && query_context->getSettingsRef()[Setting::log_queries])
query_context->addQueryFactoriesInfo(Context::QueryLogFactories::DataType, family_name);
return &it->second;
}
@ -233,7 +237,7 @@ const DataTypeFactory::Value * DataTypeFactory::findCreatorByName(const String &
DataTypesDictionary::const_iterator it = case_insensitive_data_types.find(family_name_lowercase);
if (case_insensitive_data_types.end() != it)
{
if (query_context && query_context->getSettingsRef().log_queries)
if (query_context && query_context->getSettingsRef()[Setting::log_queries])
query_context->addQueryFactoriesInfo(Context::QueryLogFactories::DataType, family_name_lowercase);
return &it->second;
}

View File

@ -32,6 +32,11 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_experimental_object_type;
extern const SettingsBool use_json_alias_for_old_object_type;
}
namespace ErrorCodes
{
@ -519,7 +524,7 @@ static DataTypePtr createJSON(const ASTPtr & arguments)
if (!context)
context = Context::getGlobalContextInstance();
if (context->getSettingsRef().allow_experimental_object_type && context->getSettingsRef().use_json_alias_for_old_object_type)
if (context->getSettingsRef()[Setting::allow_experimental_object_type] && context->getSettingsRef()[Setting::use_json_alias_for_old_object_type])
{
if (arguments && !arguments->children.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Experimental Object type doesn't support any arguments. If you want to use new JSON type, set settings allow_experimental_json_type = 1 and use_json_alias_for_old_object_type = 0");

View File

@ -22,6 +22,12 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
extern const SettingsUInt64 max_query_size;
}
namespace
{
@ -469,8 +475,8 @@ namespace
String description = fmt::format("Query for ClickHouse dictionary {}", data.table_name);
String fixed_query = removeWhereConditionPlaceholder(query);
const Settings & settings = data.global_context->getSettingsRef();
ASTPtr select = parseQuery(parser, fixed_query, description,
settings.max_query_size, settings.max_parser_depth, settings.max_parser_backtracks);
ASTPtr select = parseQuery(
parser, fixed_query, description, settings[Setting::max_query_size], settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks]);
DDLDependencyVisitor::Visitor visitor{data};
visitor.visit(select);

View File

@ -23,6 +23,12 @@ namespace fs = std::filesystem;
namespace DB
{
namespace Setting
{
extern const SettingsBool check_referential_table_dependencies;
extern const SettingsBool check_table_dependencies;
}
namespace ErrorCodes
{
extern const int UNKNOWN_TABLE;
@ -591,8 +597,8 @@ void DatabaseAtomic::renameDatabase(ContextPtr query_context, const String & new
waitDatabaseStarted();
bool check_ref_deps = query_context->getSettingsRef().check_referential_table_dependencies;
bool check_loading_deps = !check_ref_deps && query_context->getSettingsRef().check_table_dependencies;
bool check_ref_deps = query_context->getSettingsRef()[Setting::check_referential_table_dependencies];
bool check_loading_deps = !check_ref_deps && query_context->getSettingsRef()[Setting::check_table_dependencies];
if (check_ref_deps || check_loading_deps)
{
std::lock_guard lock(mutex);

View File

@ -15,6 +15,12 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
}
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
@ -115,8 +121,18 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, Co
ParserCreateQuery parser;
const char * pos = query.data();
std::string error_message;
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message,
/* hilite = */ false, "", /* allow_multi_statements = */ false, 0, settings.max_parser_depth, settings.max_parser_backtracks, true);
auto ast = tryParseQuery(
parser,
pos,
pos + query.size(),
error_message,
/* hilite = */ false,
"",
/* allow_multi_statements = */ false,
0,
settings[Setting::max_parser_depth],
settings[Setting::max_parser_backtracks],
true);
if (!ast && throw_on_error)
throw Exception::createDeprecated(error_message, ErrorCodes::SYNTAX_ERROR);
@ -135,7 +151,8 @@ ASTPtr DatabaseDictionary::getCreateDatabaseQuery() const
}
const auto & settings = getContext()->getSettingsRef();
ParserCreateQuery parser;
return parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth, settings.max_parser_backtracks);
return parseQuery(
parser, query.data(), query.data() + query.size(), "", 0, settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks]);
}
void DatabaseDictionary::shutdown()

View File

@ -16,6 +16,10 @@ namespace fs = std::filesystem;
namespace DB
{
namespace Setting
{
extern const SettingsBool log_queries;
}
namespace ErrorCodes
{
@ -103,7 +107,7 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m
DatabasePtr impl = getImpl(create, metadata_path, context);
if (impl && context->hasQueryContext() && context->getSettingsRef().log_queries)
if (impl && context->hasQueryContext() && context->getSettingsRef()[Setting::log_queries])
context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Database, impl->getEngineName());
/// Attach database metadata

View File

@ -22,6 +22,11 @@ namespace fs = std::filesystem;
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
}
namespace ErrorCodes
{
@ -188,7 +193,8 @@ ASTPtr DatabaseFilesystem::getCreateDatabaseQuery() const
const String query = fmt::format("CREATE DATABASE {} ENGINE = Filesystem('{}')", backQuoteIfNeed(getDatabaseName()), path);
ParserCreateQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth, settings.max_parser_backtracks);
ASTPtr ast
= parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks]);
if (const auto database_comment = getDatabaseComment(); !database_comment.empty())
{

View File

@ -15,6 +15,7 @@
#include <Storages/IStorage.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/re2.h>
#include <Common/RemoteHostFilter.h>
#include <Core/Settings.h>
#include <Poco/URI.h>
@ -25,6 +26,11 @@ namespace fs = std::filesystem;
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
}
namespace ErrorCodes
{
@ -184,7 +190,8 @@ ASTPtr DatabaseHDFS::getCreateDatabaseQuery() const
ParserCreateQuery parser;
const String query = fmt::format("CREATE DATABASE {} ENGINE = HDFS('{}')", backQuoteIfNeed(getDatabaseName()), source);
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth, settings.max_parser_backtracks);
ASTPtr ast
= parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks]);
if (const auto database_comment = getDatabaseComment(); !database_comment.empty())
{

View File

@ -43,6 +43,14 @@ namespace CurrentMetrics
namespace DB
{
namespace Setting
{
extern const SettingsBool force_remove_data_recursively_on_drop;
extern const SettingsBool fsync_metadata;
extern const SettingsSeconds lock_acquire_timeout;
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
}
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
@ -251,7 +259,7 @@ void DatabaseOnDisk::createTable(
WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
writeString(statement, out);
out.next();
if (settings.fsync_metadata)
if (settings[Setting::fsync_metadata])
out.sync();
out.close();
}
@ -434,7 +442,7 @@ void DatabaseOnDisk::renameTable(
/// We have to lock the table before detaching, because otherwise lockExclusively will throw. But the table may not exist.
bool need_lock = table != nullptr;
if (need_lock)
table_lock = table->lockExclusively(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout);
table_lock = table->lockExclusively(local_context->getCurrentQueryId(), local_context->getSettingsRef()[Setting::lock_acquire_timeout]);
detachTable(local_context, table_name);
if (!need_lock)
@ -549,7 +557,8 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const
/// If database.sql doesn't exist, then engine is Ordinary
String query = "CREATE DATABASE " + backQuoteIfNeed(getDatabaseName()) + " ENGINE = Ordinary";
ParserCreateQuery parser;
ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth, settings.max_parser_backtracks);
ast = parseQuery(
parser, query.data(), query.data() + query.size(), "", 0, settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks]);
}
if (const auto database_comment = getDatabaseComment(); !database_comment.empty())
@ -566,7 +575,7 @@ void DatabaseOnDisk::drop(ContextPtr local_context)
waitDatabaseStarted();
assert(TSA_SUPPRESS_WARNING_FOR_READ(tables).empty());
if (local_context->getSettingsRef().force_remove_data_recursively_on_drop)
if (local_context->getSettingsRef()[Setting::force_remove_data_recursively_on_drop])
{
(void)fs::remove_all(std::filesystem::path(getContext()->getPath()) / data_path);
(void)fs::remove_all(getMetadataPath());
@ -691,7 +700,9 @@ void DatabaseOnDisk::iterateMetadataFiles(const IteratingFunction & process_meta
process_metadata_file(file.first);
else
process_tmp_drop_metadata_file(file.first);
}, Priority{}, getContext()->getSettingsRef().lock_acquire_timeout.totalMicroseconds());
},
Priority{},
getContext()->getSettingsRef()[Setting::lock_acquire_timeout].totalMicroseconds());
}
pool.wait();
}
@ -737,8 +748,18 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(
ParserCreateQuery parser;
const char * pos = query.data();
std::string error_message;
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message, /* hilite = */ false,
"in file " + metadata_file_path, /* allow_multi_statements = */ false, 0, settings.max_parser_depth, settings.max_parser_backtracks, true);
auto ast = tryParseQuery(
parser,
pos,
pos + query.size(),
error_message,
/* hilite = */ false,
"in file " + metadata_file_path,
/* allow_multi_statements = */ false,
0,
settings[Setting::max_parser_depth],
settings[Setting::max_parser_backtracks],
true);
if (!ast && throw_on_error)
throw Exception::createDeprecated(error_message, ErrorCodes::SYNTAX_ERROR);
@ -801,8 +822,8 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromStorage(const String & table_name, cons
storage,
ast_storage,
false,
static_cast<unsigned>(settings.max_parser_depth),
static_cast<unsigned>(settings.max_parser_backtracks),
static_cast<unsigned>(settings[Setting::max_parser_depth]),
static_cast<unsigned>(settings[Setting::max_parser_backtracks]),
throw_on_error);
create_table_query->set(create_table_query->as<ASTCreateQuery>()->comment,
@ -854,7 +875,7 @@ void DatabaseOnDisk::modifySettingsMetadata(const SettingsChanges & settings_cha
writeString(statement, out);
out.next();
if (getContext()->getSettingsRef().fsync_metadata)
if (getContext()->getSettingsRef()[Setting::fsync_metadata])
out.sync();
out.close();

View File

@ -41,6 +41,15 @@ namespace fs = std::filesystem;
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_deprecated_database_ordinary;
extern const SettingsBool fsync_metadata;
extern const SettingsSeconds lock_acquire_timeout;
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
extern const SettingsSetOperationMode union_default_mode;
}
namespace ErrorCodes
{
@ -171,7 +180,7 @@ void DatabaseOrdinary::convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const Qu
WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
writeString(statement, out);
out.next();
if (getContext()->getSettingsRef().fsync_metadata)
if (getContext()->getSettingsRef()[Setting::fsync_metadata])
out.sync();
out.close();
}
@ -251,7 +260,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
convertMergeTreeToReplicatedIfNeeded(ast, qualified_name, file_name);
NormalizeSelectWithUnionQueryVisitor::Data data{local_context->getSettingsRef().union_default_mode};
NormalizeSelectWithUnionQueryVisitor::Data data{local_context->getSettingsRef()[Setting::union_default_mode]};
NormalizeSelectWithUnionQueryVisitor{data}.visit(ast);
std::lock_guard lock{metadata.mutex};
metadata.parsed_tables[qualified_name] = ParsedTableMetadata{full_path.string(), ast};
@ -403,7 +412,7 @@ LoadTaskPtr DatabaseOrdinary::startupTableAsync(
{
/// Since startup() method can use physical paths on disk we don't allow any exclusive actions (rename, drop so on)
/// until startup finished.
auto table_lock_holder = table->lockForShare(RWLockImpl::NO_QUERY, getContext()->getSettingsRef().lock_acquire_timeout);
auto table_lock_holder = table->lockForShare(RWLockImpl::NO_QUERY, getContext()->getSettingsRef()[Setting::lock_acquire_timeout]);
table->startup();
/// If table is ReplicatedMergeTree after conversion from MergeTree,
@ -550,7 +559,8 @@ void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & ta
statement.data() + statement.size(),
"in file " + table_metadata_path,
0,
local_context->getSettingsRef().max_parser_depth, local_context->getSettingsRef().max_parser_backtracks);
local_context->getSettingsRef()[Setting::max_parser_depth],
local_context->getSettingsRef()[Setting::max_parser_backtracks]);
applyMetadataChangesToCreateQuery(ast, metadata);
@ -559,7 +569,7 @@ void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & ta
WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
writeString(statement, out);
out.next();
if (local_context->getSettingsRef().fsync_metadata)
if (local_context->getSettingsRef()[Setting::fsync_metadata])
out.sync();
out.close();
}
@ -590,7 +600,7 @@ void registerDatabaseOrdinary(DatabaseFactory & factory)
{
auto create_fn = [](const DatabaseFactory::Arguments & args)
{
if (!args.create_query.attach && !args.context->getSettingsRef().allow_deprecated_database_ordinary)
if (!args.create_query.attach && !args.context->getSettingsRef()[Setting::allow_deprecated_database_ordinary])
throw Exception(
ErrorCodes::UNKNOWN_DATABASE_ENGINE,
"Ordinary database engine is deprecated (see also allow_deprecated_database_ordinary setting)");

View File

@ -48,6 +48,17 @@
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 database_replicated_allow_replicated_engine_arguments;
extern const SettingsBool database_replicated_always_detach_permanently;
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
extern const SettingsUInt64 max_query_size;
extern const SettingsBool throw_on_unsupported_query_inside_transaction;
}
namespace ErrorCodes
{
extern const int NO_ZOOKEEPER;
@ -921,7 +932,7 @@ void DatabaseReplicated::checkTableEngine(const ASTCreateQuery & query, ASTStora
}
/// We will replace it with default arguments if the setting is 2
if (query_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments != 2)
if (query_context->getSettingsRef()[Setting::database_replicated_allow_replicated_engine_arguments] != 2)
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Explicit zookeeper_path and replica_name are specified in ReplicatedMergeTree arguments. "
"If you really want to specify it explicitly, then you should use some macros "
@ -1020,7 +1031,8 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_
if (auto * query_drop = query->as<ASTDropQuery>())
{
if (query_drop->kind == ASTDropQuery::Kind::Detach && query_context->getSettingsRef().database_replicated_always_detach_permanently)
if (query_drop->kind == ASTDropQuery::Kind::Detach
&& query_context->getSettingsRef()[Setting::database_replicated_always_detach_permanently])
query_drop->permanently = true;
if (query_drop->kind == ASTDropQuery::Kind::Detach && !query_drop->permanently)
throw Exception(ErrorCodes::INCORRECT_QUERY, "DETACH TABLE is not allowed for Replicated databases. "
@ -1033,7 +1045,7 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex
{
waitDatabaseStarted();
if (query_context->getCurrentTransaction() && query_context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
if (query_context->getCurrentTransaction() && query_context->getSettingsRef()[Setting::throw_on_unsupported_query_inside_transaction])
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Distributed DDL queries inside transactions are not supported");
if (is_readonly)
@ -1080,9 +1092,9 @@ static UUID getTableUUIDIfReplicated(const String & metadata, ContextPtr context
return UUIDHelpers::Nil;
ParserCreateQuery parser;
auto size = context->getSettingsRef().max_query_size;
auto depth = context->getSettingsRef().max_parser_depth;
auto backtracks = context->getSettingsRef().max_parser_backtracks;
auto size = context->getSettingsRef()[Setting::max_query_size];
auto depth = context->getSettingsRef()[Setting::max_parser_depth];
auto backtracks = context->getSettingsRef()[Setting::max_parser_backtracks];
ASTPtr query = parseQuery(parser, metadata, size, depth, backtracks);
const ASTCreateQuery & create = query->as<const ASTCreateQuery &>();
if (!create.storage || !create.storage->engine)
@ -1479,7 +1491,13 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node
{
ParserCreateQuery parser;
String description = "in ZooKeeper " + zookeeper_path + "/metadata/" + node_name;
auto ast = parseQuery(parser, query, description, 0, getContext()->getSettingsRef().max_parser_depth, getContext()->getSettingsRef().max_parser_backtracks);
auto ast = parseQuery(
parser,
query,
description,
0,
getContext()->getSettingsRef()[Setting::max_parser_depth],
getContext()->getSettingsRef()[Setting::max_parser_backtracks]);
auto & create = ast->as<ASTCreateQuery &>();
if (create.uuid == UUIDHelpers::Nil || create.getTable() != TABLE_WITH_UUID_NAME_PLACEHOLDER || create.database)
@ -1818,7 +1836,8 @@ DatabaseReplicated::getTablesForBackup(const FilterByNameFunction & filter, cons
for (const auto & [table_name, metadata] : snapshot)
{
ParserCreateQuery parser;
auto create_table_query = parseQuery(parser, metadata, 0, getContext()->getSettingsRef().max_parser_depth, getContext()->getSettingsRef().max_parser_backtracks);
auto create_table_query = parseQuery(
parser, metadata, 0, getContext()->getSettingsRef()[Setting::max_parser_depth], getContext()->getSettingsRef()[Setting::max_parser_backtracks]);
auto & create = create_table_query->as<ASTCreateQuery &>();
create.attach = false;

View File

@ -11,6 +11,10 @@ namespace fs = std::filesystem;
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 database_replicated_initial_query_timeout_sec;
}
namespace ErrorCodes
{
@ -312,7 +316,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
task->is_initial_query = true;
LOG_DEBUG(log, "Waiting for worker thread to process all entries before {}", entry_name);
UInt64 timeout = query_context->getSettingsRef().database_replicated_initial_query_timeout_sec;
UInt64 timeout = query_context->getSettingsRef()[Setting::database_replicated_initial_query_timeout_sec];
{
std::unique_lock lock{mutex};
bool processed = wait_current_task_change.wait_for(lock, std::chrono::seconds(timeout), [&]()

View File

@ -5,6 +5,7 @@
#include <Databases/DatabaseFactory.h>
#include <Databases/DatabaseS3.h>
#include <Common/RemoteHostFilter.h>
#include <Core/Settings.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
@ -26,6 +27,11 @@ namespace fs = std::filesystem;
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 max_parser_backtracks;
extern const SettingsUInt64 max_parser_depth;
}
static const std::unordered_set<std::string_view> optional_configuration_keys = {
"url",
@ -192,7 +198,8 @@ ASTPtr DatabaseS3::getCreateDatabaseQuery() const
creation_args += fmt::format(", '{}', '{}'", config.access_key_id.value(), config.secret_access_key.value());
const String query = fmt::format("CREATE DATABASE {} ENGINE = S3({})", backQuoteIfNeed(getDatabaseName()), creation_args);
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth, settings.max_parser_backtracks);
ASTPtr ast
= parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings[Setting::max_parser_depth], settings[Setting::max_parser_backtracks]);
if (const auto database_comment = getDatabaseComment(); !database_comment.empty())
{

View File

@ -2,9 +2,10 @@
#if USE_MYSQL
# include <Core/Settings.h>
# include <Databases/MySQL/DatabaseMaterializedMySQL.h>
# include <Common/parseAddress.h>
# include <Common/parseRemoteDescription.h>
# include <Databases/MySQL/DatabaseMaterializedMySQL.h>
# include <Interpreters/evaluateConstantExpression.h>
# include <Databases/DatabaseFactory.h>
@ -25,6 +26,10 @@ namespace fs = std::filesystem;
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 glob_expansion_max_elements;
}
namespace ErrorCodes
{
@ -238,7 +243,7 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory)
if (engine_name == "MySQL")
{
size_t max_addresses = args.context->getSettingsRef().glob_expansion_max_elements;
size_t max_addresses = args.context->getSettingsRef()[Setting::glob_expansion_max_elements];
configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
}
else

Some files were not shown because too many files have changed in this diff Show More