This commit is contained in:
Alexey Milovidov 2024-08-27 17:28:55 -07:00 committed by GitHub
commit 9f7e4f6506
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
17 changed files with 143 additions and 199 deletions

View File

@ -1164,6 +1164,9 @@ void Client::processOptions(const OptionsDescription & options_description,
/// (There is no need to copy the context because clickhouse-client has no background tasks so it won't use that context in parallel.)
client_context = global_context;
initClientContext();
/// Allow to pass-through unknown settings to the server.
client_context->getAccessControl().allowAllSettings();
}

View File

@ -1923,7 +1923,7 @@ try
auto & access_control = global_context->getAccessControl();
try
{
access_control.setUpFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); });
access_control.setupFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); });
}
catch (...)
{

View File

@ -280,7 +280,7 @@ void AccessControl::shutdown()
}
void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
void AccessControl::setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_)
{
if (config_.has("custom_settings_prefixes"))
@ -868,4 +868,10 @@ const ExternalAuthenticators & AccessControl::getExternalAuthenticators() const
return *external_authenticators;
}
void AccessControl::allowAllSettings()
{
custom_settings_prefixes->registerPrefixes({""});
}
}

View File

@ -57,7 +57,7 @@ public:
void shutdown() override;
/// Initializes access storage (user directories).
void setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
void setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_);
/// Parses access entities from a configuration loaded from users.xml.
@ -238,6 +238,9 @@ public:
/// Gets manager of notifications.
AccessChangesNotifier & getChangesNotifier();
/// Allow all setting names - this can be used in clients to pass-through unknown settings to the server.
void allowAllSettings();
private:
class ContextAccessCache;
class CustomSettingsPrefixes;

View File

@ -219,8 +219,8 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang
});
}
template <class T>
bool getNewValueToCheck(const T & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure)
template <typename SettingsT>
bool getNewValueToCheck(const SettingsT & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure)
{
Field current_value;
bool has_current_value = current_settings.tryGet(change.name, current_value);
@ -230,12 +230,12 @@ bool getNewValueToCheck(const T & current_settings, SettingChange & change, Fiel
return false;
if (throw_on_failure)
new_value = T::castValueUtil(change.name, change.value);
new_value = SettingsT::castValueUtil(change.name, change.value);
else
{
try
{
new_value = T::castValueUtil(change.name, change.value);
new_value = SettingsT::castValueUtil(change.name, change.value);
}
catch (...)
{

View File

@ -58,6 +58,7 @@
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/ProfileEventsExt.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/CompressionMethod.h>
@ -1608,14 +1609,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().max_block_size,
getNumberOfPhysicalCPUCores());
auto builder = plan.buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(client_context),
@ -1892,48 +1893,19 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
profile_events.watch.restart();
{
/// Temporarily apply query settings to context.
std::optional<Settings> old_settings;
SCOPE_EXIT_SAFE({
if (old_settings)
client_context->setSettings(*old_settings);
/// Temporarily apply query settings to the context.
Settings old_settings = client_context->getSettingsCopy();
SCOPE_EXIT_SAFE(
{
client_context->setSettings(old_settings);
});
auto apply_query_settings = [&](const IAST & settings_ast)
{
if (!old_settings)
old_settings.emplace(client_context->getSettingsRef());
client_context->applySettingsChanges(settings_ast.as<ASTSetQuery>()->changes);
client_context->resetSettingsToDefaultValue(settings_ast.as<ASTSetQuery>()->default_settings);
};
const auto * insert = parsed_query->as<ASTInsertQuery>();
if (const auto * select = parsed_query->as<ASTSelectQuery>(); select && select->settings())
apply_query_settings(*select->settings());
else if (const auto * select_with_union = parsed_query->as<ASTSelectWithUnionQuery>())
{
const ASTs & children = select_with_union->list_of_selects->children;
if (!children.empty())
{
// On the client it is enough to apply settings only for the
// last SELECT, since the only thing that is important to apply
// on the client is format settings.
const auto * last_select = children.back()->as<ASTSelectQuery>();
if (last_select && last_select->settings())
{
apply_query_settings(*last_select->settings());
}
}
}
else if (const auto * query_with_output = parsed_query->as<ASTQueryWithOutput>(); query_with_output && query_with_output->settings_ast)
apply_query_settings(*query_with_output->settings_ast);
else if (insert && insert->settings_ast)
apply_query_settings(*insert->settings_ast);
InterpreterSetQuery::applySettingsFromQuery(parsed_query, client_context);
if (!connection->checkConnected(connection_parameters.timeouts))
connect();
ASTPtr input_function;
const auto * insert = parsed_query->as<ASTInsertQuery>();
if (insert && insert->select)
insert->tryFindInputFunction(input_function);

View File

@ -9,6 +9,7 @@
#include <Parsers/ASTQueryWithOutput.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
namespace DB
{
@ -45,9 +46,7 @@ static void applySettingsFromSelectWithUnion(const ASTSelectWithUnionQuery & sel
// It is flattened later, when we process UNION ALL/DISTINCT.
const auto * last_select = children.back()->as<ASTSelectQuery>();
if (last_select && last_select->settings())
{
InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext();
}
InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(/* ignore_setting_constraints= */ false);
}
void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMutablePtr context_)
@ -55,10 +54,20 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta
if (!ast)
return;
/// First apply the outermost settings. Then they could be overridden by deeper settings.
if (const auto * query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get()))
{
if (query_with_output->settings_ast)
InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false);
if (const auto * create_query = ast->as<ASTCreateQuery>(); create_query && create_query->select)
applySettingsFromSelectWithUnion(create_query->select->as<ASTSelectWithUnionQuery &>(), context_);
}
if (const auto * select_query = ast->as<ASTSelectQuery>())
{
if (auto new_settings = select_query->settings())
InterpreterSetQuery(new_settings, context_).executeForCurrentContext();
InterpreterSetQuery(new_settings, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false);
}
else if (const auto * select_with_union_query = ast->as<ASTSelectWithUnionQuery>())
{
@ -67,28 +76,15 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta
else if (const auto * explain_query = ast->as<ASTExplainQuery>())
{
if (explain_query->settings_ast)
InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext();
InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false);
applySettingsFromQuery(explain_query->getExplainedQuery(), context_);
}
else if (const auto * query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get()))
{
if (query_with_output->settings_ast)
InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext();
if (const auto * create_query = ast->as<ASTCreateQuery>())
{
if (create_query->select)
{
applySettingsFromSelectWithUnion(create_query->select->as<ASTSelectWithUnionQuery &>(), context_);
}
}
}
else if (auto * insert_query = ast->as<ASTInsertQuery>())
{
context_->setInsertFormat(insert_query->format);
if (insert_query->settings_ast)
InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext();
InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false);
}
}

View File

@ -23,7 +23,7 @@ public:
/** Set setting for current context (query context).
* It is used for interpretation of SETTINGS clause in SELECT query.
*/
void executeForCurrentContext(bool ignore_setting_constraints = false);
void executeForCurrentContext(bool ignore_setting_constraints);
bool supportsTransactions() const override { return true; }

View File

@ -25,7 +25,6 @@
#include <Parsers/ParserTablePropertiesQuery.h>
#include <Parsers/ParserWatchQuery.h>
#include <Parsers/ParserDescribeCacheQuery.h>
#include <Parsers/QueryWithOutputSettingsPushDownVisitor.h>
#include <Parsers/Access/ParserShowAccessEntitiesQuery.h>
#include <Parsers/Access/ParserShowAccessQuery.h>
#include <Parsers/Access/ParserShowCreateAccessEntityQuery.h>
@ -152,37 +151,55 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
/// These two sections are allowed in an arbitrary order.
ParserKeyword s_format(Keyword::FORMAT);
if (s_format.ignore(pos, expected))
{
ParserIdentifier format_p;
if (!format_p.parse(pos, query_with_output.format, expected))
return false;
setIdentifierSpecial(query_with_output.format);
query_with_output.children.push_back(query_with_output.format);
}
// SETTINGS key1 = value1, key2 = value2, ...
ParserKeyword s_settings(Keyword::SETTINGS);
if (!query_with_output.settings_ast && s_settings.ignore(pos, expected))
{
ParserSetQuery parser_settings(true);
if (!parser_settings.parse(pos, query_with_output.settings_ast, expected))
return false;
query_with_output.children.push_back(query_with_output.settings_ast);
// SETTINGS after FORMAT is not parsed by the SELECT parser (ParserSelectQuery)
// Pass them manually, to apply in InterpreterSelectQuery::initSettings()
if (query->as<ASTSelectWithUnionQuery>())
/** Why: let's take the following example:
* SELECT 1 UNION ALL SELECT 2 FORMAT TSV
* Each subquery can be put in parentheses and have its own settings:
* (SELECT 1 SETTINGS a=b) UNION ALL (SELECT 2 SETTINGS c=d) FORMAT TSV
* And the whole query can have settings:
* (SELECT 1 SETTINGS a=b) UNION ALL (SELECT 2 SETTINGS c=d) FORMAT TSV SETTINGS e=f
* A single query with output is parsed in the same way as the UNION ALL chain:
* SELECT 1 SETTINGS a=b FORMAT TSV SETTINGS e=f
* So while these forms have a slightly different meaning, they both exist:
* SELECT 1 SETTINGS a=b FORMAT TSV
* SELECT 1 FORMAT TSV SETTINGS e=f
* And due to this effect, the users expect that the FORMAT and SETTINGS may go in an arbitrary order.
* But while this work:
* (SELECT 1) UNION ALL (SELECT 2) FORMAT TSV SETTINGS d=f
* This does not work automatically, unless we explicitly allow different orders:
* (SELECT 1) UNION ALL (SELECT 2) SETTINGS d=f FORMAT TSV
* Inevitably, we also allow this:
* SELECT 1 SETTINGS a=b SETTINGS d=f FORMAT TSV
* ^^^^^^^^^^^^^^^^^^^^^
* Because this part is consumed into ASTSelectWithUnionQuery
* and the rest into ASTQueryWithOutput.
*/
for (size_t i = 0; i < 2; ++i)
{
if (!query_with_output.format && s_format.ignore(pos, expected))
{
auto settings = query_with_output.settings_ast->clone();
assert_cast<ASTSetQuery *>(settings.get())->print_in_format = false;
QueryWithOutputSettingsPushDownVisitor::Data data{settings};
QueryWithOutputSettingsPushDownVisitor(data).visit(query);
ParserIdentifier format_p;
if (!format_p.parse(pos, query_with_output.format, expected))
return false;
setIdentifierSpecial(query_with_output.format);
query_with_output.children.push_back(query_with_output.format);
}
else if (!query_with_output.settings_ast && s_settings.ignore(pos, expected))
{
// SETTINGS key1 = value1, key2 = value2, ...
ParserSetQuery parser_settings(true);
if (!parser_settings.parse(pos, query_with_output.settings_ast, expected))
return false;
query_with_output.children.push_back(query_with_output.settings_ast);
}
else
break;
}
node = std::move(query);

View File

@ -1,56 +0,0 @@
#include <Common/SettingsChanges.h>
#include <Parsers/QueryWithOutputSettingsPushDownVisitor.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTSubquery.h>
#include <iterator>
#include <algorithm>
namespace DB
{
bool QueryWithOutputSettingsPushDownMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
{
if (node->as<ASTSelectWithUnionQuery>())
return true;
if (node->as<ASTSubquery>())
return true;
if (child->as<ASTSelectQuery>())
return true;
return false;
}
void QueryWithOutputSettingsPushDownMatcher::visit(ASTPtr & ast, Data & data)
{
if (auto * select_query = ast->as<ASTSelectQuery>())
visit(*select_query, ast, data);
}
void QueryWithOutputSettingsPushDownMatcher::visit(ASTSelectQuery & select_query, ASTPtr &, Data & data)
{
ASTPtr select_settings_ast = select_query.settings();
if (!select_settings_ast)
{
select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, data.settings_ast->clone());
return;
}
SettingsChanges & select_settings = select_settings_ast->as<ASTSetQuery &>().changes;
SettingsChanges & settings = data.settings_ast->as<ASTSetQuery &>().changes;
for (auto & setting : settings)
{
auto it = std::find_if(select_settings.begin(), select_settings.end(), [&](auto & select_setting)
{
return select_setting.name == setting.name;
});
if (it == select_settings.end())
select_settings.push_back(setting);
else
it->value = setting.value;
}
}
}

View File

@ -1,39 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
class ASTSelectQuery;
struct SettingChange;
class SettingsChanges;
/// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query:
/// (since settings after FORMAT parsed separately not in the ParserSelectQuery but in ParserQueryWithOutput)
///
/// SELECT 1 FORMAT Null SETTINGS max_block_size = 1 ->
/// SELECT 1 SETTINGS max_block_size = 1 FORMAT Null SETTINGS max_block_size = 1
///
/// Otherwise settings after FORMAT will not be applied.
class QueryWithOutputSettingsPushDownMatcher
{
public:
using Visitor = InDepthNodeVisitor<QueryWithOutputSettingsPushDownMatcher, true>;
struct Data
{
const ASTPtr & settings_ast;
};
static bool needChildVisit(ASTPtr & node, const ASTPtr & child);
static void visit(ASTPtr & ast, Data & data);
private:
static void visit(ASTSelectQuery &, ASTPtr &, Data &);
};
using QueryWithOutputSettingsPushDownVisitor = QueryWithOutputSettingsPushDownMatcher::Visitor;
}

View File

@ -1,4 +1,3 @@
DROP TABLE IF EXISTS local_table;
DROP TABLE IF EXISTS other_table;

View File

@ -1,7 +1,7 @@
1
1
1
1
1
2
1
2
2

View File

@ -13,7 +13,7 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) FORMAT CSV SETTINGS max_block_size = 1'
# push down append
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_compress_block_size = 1 FORMAT CSV SETTINGS max_block_size = 1'
# overwrite on push down (since these settings goes latest)
# not overwrite on push down
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_block_size = 2 FORMAT CSV SETTINGS max_block_size = 1'
# on push-down
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_block_size = 1 FORMAT CSV'

View File

@ -1,2 +1 @@
select 42 settings compatibility=NULL; -- {clientError BAD_ARGUMENTS}
select 42 settings compatibility=NULL; -- {clientError BAD_GET}

View File

@ -0,0 +1,14 @@
1
2
1
2
1
2
1
1
3
3
3
3
3
1

View File

@ -0,0 +1,30 @@
SET max_block_size = 10, max_threads = 1;
-- Take the following example:
SELECT 1 UNION ALL SELECT 2 FORMAT TSV;
-- Each subquery can be put in parentheses and have its own settings:
(SELECT getSetting('max_block_size') SETTINGS max_block_size = 1) UNION ALL (SELECT getSetting('max_block_size') SETTINGS max_block_size = 2) FORMAT TSV;
-- And the whole query can have settings:
(SELECT getSetting('max_block_size') SETTINGS max_block_size = 1) UNION ALL (SELECT getSetting('max_block_size') SETTINGS max_block_size = 2) FORMAT TSV SETTINGS max_block_size = 3;
-- A single query with output is parsed in the same way as the UNION ALL chain:
SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 FORMAT TSV SETTINGS max_block_size = 3;
-- So while these forms have a slightly different meaning, they both exist:
SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 FORMAT TSV;
SELECT getSetting('max_block_size') FORMAT TSV SETTINGS max_block_size = 3;
-- And due to this effect, the users expect that the FORMAT and SETTINGS may go in an arbitrary order.
-- But while this work:
(SELECT getSetting('max_block_size')) UNION ALL (SELECT getSetting('max_block_size')) FORMAT TSV SETTINGS max_block_size = 3;
-- This does not work automatically, unless we explicitly allow different orders:
(SELECT getSetting('max_block_size')) UNION ALL (SELECT getSetting('max_block_size')) SETTINGS max_block_size = 3 FORMAT TSV;
-- Inevitably, we allow this:
SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 SETTINGS max_block_size = 3 FORMAT TSV;
/*^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^*/
-- Because this part is consumed into ASTSelectWithUnionQuery
-- and the rest into ASTQueryWithOutput.