Merge pull request #66528 from Algunenano/getSettingsCopy

Rename Context::getSettings()
This commit is contained in:
Raúl Marín 2024-07-31 09:55:21 +00:00 committed by GitHub
commit 850b7f71fb
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
30 changed files with 37 additions and 38 deletions

View File

@ -80,7 +80,7 @@ namespace ErrorCodes
void applySettingsOverridesForLocal(ContextMutablePtr context)
{
Settings settings = context->getSettings();
Settings settings = context->getSettingsCopy();
settings.allow_introspection_functions = true;
settings.storage_file_read_method = LocalFSReadMethod::mmap;

View File

@ -237,7 +237,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q
/// Remove global settings limit and offset
if (const auto & settings_ref = updated_context->getSettingsRef(); settings_ref.limit || settings_ref.offset)
{
Settings settings = updated_context->getSettings();
Settings settings = updated_context->getSettingsCopy();
limit = settings.limit;
offset = settings.offset;
settings.limit = 0;

View File

@ -503,7 +503,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
ProfileEvents::increment(ProfileEvents::ScalarSubqueriesCacheMiss);
auto subquery_context = Context::createCopy(context);
Settings subquery_settings = context->getSettings();
Settings subquery_settings = context->getSettingsCopy();
subquery_settings.max_result_rows = 1;
subquery_settings.extremes = false;
subquery_context->setSettings(subquery_settings);

View File

@ -867,7 +867,7 @@ void updateContextForSubqueryExecution(ContextMutablePtr & mutable_context)
* max_rows_in_join, max_bytes_in_join, join_overflow_mode,
* which are checked separately (in the Set, Join objects).
*/
Settings subquery_settings = mutable_context->getSettings();
Settings subquery_settings = mutable_context->getSettingsCopy();
subquery_settings.max_result_rows = 0;
subquery_settings.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).

View File

@ -232,7 +232,7 @@ int IBridge::main(const std::vector<std::string> & /*args*/)
auto context = Context::createGlobal(shared_context.get());
context->makeGlobalContext();
auto settings = context->getSettings();
auto settings = context->getSettingsCopy();
settings.set("http_max_field_value_size", http_max_field_value_size);
context->setSettings(settings);

View File

@ -656,7 +656,7 @@ void ClientBase::initLogsOutputStream()
void ClientBase::adjustSettings()
{
Settings settings = global_context->getSettings();
Settings settings = global_context->getSettingsCopy();
/// NOTE: Do not forget to set changed=false to avoid sending it to the server (to avoid breakage read only profiles)
@ -865,7 +865,7 @@ bool ClientBase::isSyncInsertWithData(const ASTInsertQuery & insert_query, const
if (!insert_query.data)
return false;
auto settings = context->getSettings();
auto settings = context->getSettingsCopy();
if (insert_query.settings_ast)
settings.applyChanges(insert_query.settings_ast->as<ASTSetQuery>()->changes);
@ -2671,7 +2671,7 @@ bool ClientBase::processMultiQueryFromFile(const String & file_name)
if (!getClientConfiguration().has("log_comment"))
{
Settings settings = client_context->getSettings();
Settings settings = client_context->getSettingsCopy();
/// NOTE: cannot use even weakly_canonical() since it fails for /dev/stdin due to resolving of "pipe:[X]"
settings.log_comment = fs::absolute(fs::path(file_name));
client_context->setSettings(settings);

View File

@ -111,7 +111,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, Co
buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")";
}
auto settings = getContext()->getSettingsRef();
const auto & settings = getContext()->getSettingsRef();
ParserCreateQuery parser;
const char * pos = query.data();
std::string error_message;
@ -133,7 +133,7 @@ ASTPtr DatabaseDictionary::getCreateDatabaseQuery() const
if (const auto comment_value = getDatabaseComment(); !comment_value.empty())
buffer << " COMMENT " << backQuote(comment_value);
}
auto settings = getContext()->getSettingsRef();
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);
}

View File

@ -534,7 +534,7 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const
{
ASTPtr ast;
auto settings = getContext()->getSettingsRef();
const auto & settings = getContext()->getSettingsRef();
{
std::lock_guard lock(mutex);
auto database_metadata_path = getContext()->getPath() + "metadata/" + escapeForFileName(database_name) + ".sql";
@ -733,7 +733,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(
return nullptr;
}
auto settings = local_context->getSettingsRef();
const auto & settings = local_context->getSettingsRef();
ParserCreateQuery parser;
const char * pos = query.data();
std::string error_message;

View File

@ -89,7 +89,7 @@ static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync";
static ContextMutablePtr createQueryContext(ContextPtr context)
{
Settings new_query_settings = context->getSettings();
Settings new_query_settings = context->getSettingsCopy();
new_query_settings.insert_allow_materialized_columns = true;
/// To avoid call AST::format

View File

@ -39,7 +39,7 @@ public:
FunctionFormatQuery(ContextPtr context, String name_, OutputFormatting output_formatting_, ErrorHandling error_handling_)
: name(name_), output_formatting(output_formatting_), error_handling(error_handling_)
{
const Settings & settings = context->getSettings();
const Settings & settings = context->getSettingsRef();
max_query_size = settings.max_query_size;
max_parser_depth = settings.max_parser_depth;
max_parser_backtracks = settings.max_parser_backtracks;

View File

@ -143,7 +143,7 @@ ColumnPtr FunctionHasColumnInTable::executeImpl(const ColumnsWithTypeAndName & a
/* cluster_name= */ "",
/* password= */ ""
};
auto cluster = std::make_shared<Cluster>(getContext()->getSettings(), host_names, params);
auto cluster = std::make_shared<Cluster>(getContext()->getSettingsRef(), host_names, params);
// FIXME this (probably) needs a non-constant access to query context,
// because it might initialized a storage. Ideally, the tables required

View File

@ -2270,7 +2270,7 @@ bool Context::displaySecretsInShowAndSelect() const
return shared->server_settings.display_secrets_in_show_and_select;
}
Settings Context::getSettings() const
Settings Context::getSettingsCopy() const
{
SharedLockGuard lock(mutex);
return *settings;

View File

@ -829,7 +829,8 @@ public:
void setMacros(std::unique_ptr<Macros> && macros);
bool displaySecretsInShowAndSelect() const;
Settings getSettings() const;
Settings getSettingsCopy() const;
const Settings & getSettingsRef() const { return *settings; }
void setSettings(const Settings & settings_);
/// Set settings by name.
@ -954,8 +955,6 @@ public:
void makeSessionContext();
void makeGlobalContext();
const Settings & getSettingsRef() const { return *settings; }
void setProgressCallback(ProgressCallback callback);
/// Used in executeQuery() to pass it to the QueryPipeline.
ProgressCallback getProgressCallback() const;

View File

@ -74,7 +74,7 @@ void ExecuteScalarSubqueriesMatcher::visit(ASTPtr & ast, Data & data)
static auto getQueryInterpreter(const ASTSubquery & subquery, ExecuteScalarSubqueriesMatcher::Data & data)
{
auto subquery_context = Context::createCopy(data.getContext());
Settings subquery_settings = data.getContext()->getSettings();
Settings subquery_settings = data.getContext()->getSettingsCopy();
subquery_settings.max_result_rows = 1;
subquery_settings.extremes = false;
subquery_context->setSettings(subquery_settings);

View File

@ -171,7 +171,7 @@ ExpressionAnalyzer::ExpressionAnalyzer(
PreparedSetsPtr prepared_sets_,
bool is_create_parameterized_view_)
: WithContext(context_)
, query(query_), settings(getContext()->getSettings())
, query(query_), settings(getContext()->getSettingsRef())
, subquery_depth(subquery_depth_)
, syntax(syntax_analyzer_result_)
, is_create_parameterized_view(is_create_parameterized_view_)
@ -984,7 +984,7 @@ static std::shared_ptr<IJoin> tryCreateJoin(
algorithm == JoinAlgorithm::PARALLEL_HASH ||
algorithm == JoinAlgorithm::DEFAULT)
{
const auto & settings = context->getSettings();
const auto & settings = context->getSettingsRef();
if (analyzed_join->allowParallelHashJoin())
return std::make_shared<ConcurrentHashJoin>(

View File

@ -464,7 +464,7 @@ QueryPipeline InterpreterInsertQuery::buildInsertSelectPipeline(ASTInsertQuery &
* to avoid unnecessary squashing.
*/
Settings new_settings = select_context->getSettings();
Settings new_settings = select_context->getSettingsCopy();
new_settings.max_threads = std::max<UInt64>(1, settings.max_insert_threads);

View File

@ -249,7 +249,7 @@ namespace
ContextPtr getSubqueryContext(const ContextPtr & context)
{
auto subquery_context = Context::createCopy(context);
Settings subquery_settings = context->getSettings();
Settings subquery_settings = context->getSettingsCopy();
subquery_settings.max_result_rows = 0;
subquery_settings.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).

View File

@ -308,7 +308,7 @@ std::shared_ptr<TableJoin> JoinedTables::makeTableJoin(const ASTSelectQuery & se
if (tables_with_columns.size() < 2)
return {};
auto settings = context->getSettingsRef();
const auto & settings = context->getSettingsRef();
MultiEnum<JoinAlgorithm> join_algorithm = settings.join_algorithm;
bool try_use_direct_join = join_algorithm.isSet(JoinAlgorithm::DIRECT) || join_algorithm.isSet(JoinAlgorithm::DEFAULT);
auto table_join = std::make_shared<TableJoin>(settings, context->getGlobalTemporaryVolume(), context->getTempDataOnDisk());

View File

@ -657,7 +657,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even
{
if (auto ctx = context.lock())
{
res.query_settings = std::make_shared<Settings>(ctx->getSettings());
res.query_settings = std::make_shared<Settings>(ctx->getSettingsRef());
res.current_database = ctx->getCurrentDatabase();
}
}

View File

@ -62,7 +62,7 @@ std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
* which are checked separately (in the Set, Join objects).
*/
auto subquery_context = Context::createCopy(context);
Settings subquery_settings = context->getSettings();
Settings subquery_settings = context->getSettingsCopy();
subquery_settings.max_result_rows = 0;
subquery_settings.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 instead of the whole query).

View File

@ -406,7 +406,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
{
const Block & header = getPort().getHeader();
const IDataType & type = *header.getByPosition(column_idx).type;
auto settings = context->getSettingsRef();
const auto & settings = context->getSettingsRef();
/// Advance the token iterator until the start of the column expression
readUntilTheEndOfRowAndReTokenize(column_idx);

View File

@ -1055,7 +1055,7 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node,
}
auto context = read_from_merge_tree->getContext();
const auto & settings = context->getSettings();
const auto & settings = context->getSettingsRef();
if (!settings.optimize_read_in_window_order || (settings.optimize_read_in_order && settings.query_plan_read_in_order) || context->getSettingsRef().allow_experimental_analyzer)
{
return 0;

View File

@ -473,7 +473,7 @@ void MySQLHandler::comQuery(ReadBuffer & payload, bool binary_protocol)
query_context->setCurrentQueryId(fmt::format("mysql:{}:{}", connection_id, toString(UUIDHelpers::generateV4())));
/// --- Workaround for Bug 56173. Can be removed when the analyzer is on by default.
auto settings = query_context->getSettings();
auto settings = query_context->getSettingsCopy();
settings.prefer_column_name_to_alias = true;
query_context->setSettings(settings);

View File

@ -283,7 +283,7 @@ ConnectionPoolWithFailoverPtr DistributedAsyncInsertDirectoryQueue::createPool(c
auto pools = createPoolsForAddresses(addresses, pool_factory, storage.log);
const auto settings = storage.getContext()->getSettings();
const auto & settings = storage.getContext()->getSettingsRef();
return std::make_shared<ConnectionPoolWithFailover>(std::move(pools),
settings.load_balancing,
settings.distributed_replica_error_half_life.totalSeconds(),

View File

@ -34,7 +34,7 @@ MergedBlockOutputStream::MergedBlockOutputStream(
, write_settings(write_settings_)
{
MergeTreeWriterSettings writer_settings(
data_part->storage.getContext()->getSettings(),
data_part->storage.getContext()->getSettingsRef(),
write_settings,
storage_settings,
data_part->index_granularity_info.mark_type.adaptive,

View File

@ -23,7 +23,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream(
const MergeTreeIndexGranularityInfo * index_granularity_info)
: IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true)
{
const auto & global_settings = data_part->storage.getContext()->getSettings();
const auto & global_settings = data_part->storage.getContext()->getSettingsRef();
MergeTreeWriterSettings writer_settings(
global_settings,

View File

@ -464,7 +464,7 @@ std::pair<Poco::URI, std::unique_ptr<ReadWriteBufferFromHTTP>> StorageURLSource:
setCredentials(credentials, request_uri);
const auto settings = context_->getSettings();
const auto & settings = context_->getSettingsRef();
auto proxy_config = getProxyConfiguration(request_uri.getScheme());
@ -1328,7 +1328,7 @@ std::optional<time_t> IStorageURLBase::tryGetLastModificationTime(
const Poco::Net::HTTPBasicCredentials & credentials,
const ContextPtr & context)
{
auto settings = context->getSettingsRef();
const auto & settings = context->getSettingsRef();
auto uri = Poco::URI(url);

View File

@ -97,7 +97,7 @@ bool hasJoin(const ASTSelectWithUnionQuery & ast)
ContextPtr getViewContext(ContextPtr context, const StorageSnapshotPtr & storage_snapshot)
{
auto view_context = storage_snapshot->metadata->getSQLSecurityOverriddenContext(context);
Settings view_settings = view_context->getSettings();
Settings view_settings = view_context->getSettingsCopy();
view_settings.max_result_rows = 0;
view_settings.max_result_bytes = 0;
view_settings.extremes = false;

View File

@ -65,7 +65,7 @@ ColumnsDescription getStructureOfRemoteTableInShard(
/// Ignore limit for result number of rows (that could be set during handling CSE/CTE),
/// since this is a service query and should not lead to query failure.
{
Settings new_settings = new_context->getSettings();
Settings new_settings = new_context->getSettingsCopy();
new_settings.max_result_rows = 0;
new_settings.max_result_bytes = 0;
new_context->setSettings(new_settings);

View File

@ -93,7 +93,7 @@ StoragePtr TableFunctionHive::executeImpl(
ColumnsDescription /*cached_columns_*/,
bool /*is_insert_query*/) const
{
const Settings & settings = context_->getSettings();
const Settings & settings = context_->getSettingsRef();
ParserExpression partition_by_parser;
ASTPtr partition_by_ast = parseQuery(
partition_by_parser,