mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #60197 from ClickHouse/format-settings-parsing
Fix unexpected behavior with `FORMAT` and `SETTINGS` parsing
This commit is contained in:
commit
5a6d765fa2
@ -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();
|
||||
}
|
||||
|
||||
|
||||
|
@ -1920,7 +1920,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 (...)
|
||||
{
|
||||
|
@ -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({""});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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 (...)
|
||||
{
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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; }
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
|
||||
}
|
@ -1,4 +1,3 @@
|
||||
|
||||
DROP TABLE IF EXISTS local_table;
|
||||
DROP TABLE IF EXISTS other_table;
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
2
|
||||
1
|
||||
2
|
||||
2
|
||||
|
@ -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'
|
||||
|
@ -1,2 +1 @@
|
||||
select 42 settings compatibility=NULL; -- {clientError BAD_ARGUMENTS}
|
||||
|
||||
select 42 settings compatibility=NULL; -- {clientError BAD_GET}
|
||||
|
@ -0,0 +1,14 @@
|
||||
1
|
||||
2
|
||||
1
|
||||
2
|
||||
1
|
||||
2
|
||||
1
|
||||
1
|
||||
3
|
||||
3
|
||||
3
|
||||
3
|
||||
3
|
||||
1
|
30
tests/queries/0_stateless/03172_format_settings_clauses.sql
Normal file
30
tests/queries/0_stateless/03172_format_settings_clauses.sql
Normal 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.
|
Loading…
Reference in New Issue
Block a user