mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Introduce compatiblity mode for SETTINGS after FORMAT
Add allow_settings_after_format_in_insert setting, OFF by default. Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> v2: s/parser_settings_after_format_compact/allow_settings_after_format_in_insert/ (suggested by vitlibar) v3: replace ParserSettings with a flag (requested by vitlibar)
This commit is contained in:
parent
c5ffbc688e
commit
33d99c8ffb
@ -519,6 +519,33 @@ Possible values:
|
||||
|
||||
Default value: `1`.
|
||||
|
||||
## allow_settings_after_format_in_insert {#allow_settings_after_format_in_insert}
|
||||
|
||||
Control whether `SETTINGS` after `FORMAT` in `INSERT` queries is allowed or not. It is not recommended to use this, since this may interpret part of `SETTINGS` as values.
|
||||
|
||||
Example:
|
||||
|
||||
```sql
|
||||
INSERT INTO FUNCTION null('foo String') SETTINGS max_threads=1 VALUES ('bar');
|
||||
```
|
||||
|
||||
But the following query will work only with `allow_settings_after_format_in_insert`:
|
||||
|
||||
```sql
|
||||
SET allow_settings_after_format_in_insert=1;
|
||||
INSERT INTO FUNCTION null('foo String') VALUES ('bar') SETTINGS max_threads=1;
|
||||
```
|
||||
|
||||
Possible values:
|
||||
|
||||
- 0 — Disallow.
|
||||
- 1 — Allow.
|
||||
|
||||
Default value: `0`.
|
||||
|
||||
!!! note "Warning"
|
||||
Use this setting only for backward compatibility if your use cases depend on old syntax.
|
||||
|
||||
## input_format_skip_unknown_fields {#settings-input-format-skip-unknown-fields}
|
||||
|
||||
Enables or disables skipping insertion of extra data.
|
||||
|
@ -54,6 +54,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
||||
("multiquery,n", "allow multiple queries in the same file")
|
||||
("obfuscate", "obfuscate instead of formatting")
|
||||
("backslash", "add a backslash at the end of each line of the formatted query")
|
||||
("allow_settings_after_format_in_insert", "Allow SETTINGS after FORMAT, but note, that this is not always safe")
|
||||
("seed", po::value<std::string>(), "seed (arbitrary string) that determines the result of obfuscation")
|
||||
;
|
||||
|
||||
@ -83,6 +84,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
||||
bool multiple = options.count("multiquery");
|
||||
bool obfuscate = options.count("obfuscate");
|
||||
bool backslash = options.count("backslash");
|
||||
bool allow_settings_after_format_in_insert = options.count("allow_settings_after_format_in_insert");
|
||||
|
||||
if (quiet && (hilite || oneline || obfuscate))
|
||||
{
|
||||
@ -154,7 +156,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
||||
const char * pos = query.data();
|
||||
const char * end = pos + query.size();
|
||||
|
||||
ParserQuery parser(end);
|
||||
ParserQuery parser(end, allow_settings_after_format_in_insert);
|
||||
do
|
||||
{
|
||||
ASTPtr res = parseQueryAndMovePosition(
|
||||
|
@ -268,7 +268,7 @@ void ClientBase::setupSignalHandler()
|
||||
|
||||
ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_multi_statements) const
|
||||
{
|
||||
ParserQuery parser(end);
|
||||
ParserQuery parser(end, global_context->getSettings().allow_settings_after_format_in_insert);
|
||||
ASTPtr res;
|
||||
|
||||
const auto & settings = global_context->getSettingsRef();
|
||||
|
@ -465,6 +465,7 @@ class IColumn;
|
||||
M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \
|
||||
M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \
|
||||
M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \
|
||||
M(Bool, allow_settings_after_format_in_insert, false, "Allow SETTINGS after FORMAT, but note, that this is not always safe (note: this is a compatibility setting).", 0) \
|
||||
M(Seconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \
|
||||
M(Seconds, periodic_live_view_refresh, DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC, "Interval after which periodically refreshed live view is forced to refresh.", 0) \
|
||||
M(Bool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \
|
||||
|
@ -142,10 +142,11 @@ void DDLTaskBase::parseQueryFromEntry(ContextPtr context)
|
||||
{
|
||||
const char * begin = entry.query.data();
|
||||
const char * end = begin + entry.query.size();
|
||||
const auto & settings = context->getSettingsRef();
|
||||
|
||||
ParserQuery parser_query(end);
|
||||
ParserQuery parser_query(end, settings.allow_settings_after_format_in_insert);
|
||||
String description;
|
||||
query = parseQuery(parser_query, begin, end, description, 0, context->getSettingsRef().max_parser_depth);
|
||||
query = parseQuery(parser_query, begin, end, description, 0, settings.max_parser_depth);
|
||||
}
|
||||
|
||||
ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/)
|
||||
|
@ -423,7 +423,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
String query_table;
|
||||
try
|
||||
{
|
||||
ParserQuery parser(end);
|
||||
ParserQuery parser(end, settings.allow_settings_after_format_in_insert);
|
||||
|
||||
/// TODO: parser should fail early when max_query_size limit is reached.
|
||||
ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
|
||||
|
@ -58,11 +58,11 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
ParserCreateTableQuery create_p;
|
||||
ParserSelectWithUnionQuery select_p;
|
||||
ParserInsertQuery insert_p(end);
|
||||
ParserInsertQuery insert_p(end, allow_settings_after_format_in_insert);
|
||||
ASTPtr query;
|
||||
if (kind == ASTExplainQuery::ExplainKind::ParsedAST)
|
||||
{
|
||||
ParserQuery p(end);
|
||||
ParserQuery p(end, allow_settings_after_format_in_insert);
|
||||
if (p.parse(pos, query, expected))
|
||||
explain_query->setExplainedQuery(std::move(query));
|
||||
else
|
||||
|
@ -10,11 +10,15 @@ class ParserExplainQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * end;
|
||||
bool allow_settings_after_format_in_insert;
|
||||
|
||||
const char * getName() const override { return "EXPLAIN"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
public:
|
||||
explicit ParserExplainQuery(const char* end_) : end(end_) {}
|
||||
explicit ParserExplainQuery(const char* end_, bool allow_settings_after_format_in_insert_)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -186,6 +186,31 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Read SETTINGS after FORMAT.
|
||||
///
|
||||
/// Note, that part of SETTINGS can be interpreted as values,
|
||||
/// hence it is done only under option.
|
||||
///
|
||||
/// Refs: https://github.com/ClickHouse/ClickHouse/issues/35100
|
||||
if (allow_settings_after_format_in_insert && s_settings.ignore(pos, expected))
|
||||
{
|
||||
if (settings_ast)
|
||||
throw Exception("You have SETTINGS before and after FORMAT, "
|
||||
"this is not allowed. "
|
||||
"Consider switching to SETTINGS before FORMAT "
|
||||
"and disable allow_settings_after_format_in_insert.",
|
||||
ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
/// Settings are written like SET query, so parse them with ParserSetQuery
|
||||
ParserSetQuery parser_settings(true);
|
||||
if (!parser_settings.parse(pos, settings_ast, expected))
|
||||
return false;
|
||||
/// In case of INSERT INTO ... VALUES SETTINGS ... (...), (...), ...
|
||||
/// we should move data pointer after all settings.
|
||||
if (data != nullptr)
|
||||
data = pos->begin;
|
||||
}
|
||||
|
||||
if (select)
|
||||
{
|
||||
/// Copy SETTINGS from the INSERT ... SELECT ... SETTINGS
|
||||
|
@ -26,11 +26,15 @@ class ParserInsertQuery : public IParserBase
|
||||
{
|
||||
private:
|
||||
const char * end;
|
||||
bool allow_settings_after_format_in_insert;
|
||||
|
||||
const char * getName() const override { return "INSERT query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
public:
|
||||
explicit ParserInsertQuery(const char * end_) : end(end_) {}
|
||||
explicit ParserInsertQuery(const char * end_, bool allow_settings_after_format_in_insert_)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
};
|
||||
|
||||
/** Insert accepts an identifier and an asterisk with variants.
|
||||
|
@ -30,8 +30,8 @@ namespace DB
|
||||
|
||||
bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserQueryWithOutput query_with_output_p(end);
|
||||
ParserInsertQuery insert_p(end);
|
||||
ParserQueryWithOutput query_with_output_p(end, allow_settings_after_format_in_insert);
|
||||
ParserInsertQuery insert_p(end, allow_settings_after_format_in_insert);
|
||||
ParserUseQuery use_p;
|
||||
ParserSetQuery set_p;
|
||||
ParserSystemQuery system_p;
|
||||
|
@ -10,12 +10,16 @@ class ParserQuery : public IParserBase
|
||||
{
|
||||
private:
|
||||
const char * end;
|
||||
bool allow_settings_after_format_in_insert;
|
||||
|
||||
const char * getName() const override { return "Query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
public:
|
||||
explicit ParserQuery(const char * end_) : end(end_) {}
|
||||
explicit ParserQuery(const char * end_, bool allow_settings_after_format_in_insert_ = false)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -49,7 +49,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
ParserShowCreateAccessEntityQuery show_create_access_entity_p;
|
||||
ParserShowGrantsQuery show_grants_p;
|
||||
ParserShowPrivilegesQuery show_privileges_p;
|
||||
ParserExplainQuery explain_p(end);
|
||||
ParserExplainQuery explain_p(end, allow_settings_after_format_in_insert);
|
||||
|
||||
ASTPtr query;
|
||||
|
||||
|
@ -12,10 +12,16 @@ class ParserQueryWithOutput : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * end;
|
||||
bool allow_settings_after_format_in_insert;
|
||||
|
||||
const char * getName() const override { return "Query with output"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
public:
|
||||
explicit ParserQueryWithOutput(const char * end_) : end(end_) {}
|
||||
explicit ParserQueryWithOutput(const char * end_, bool allow_settings_after_format_in_insert_ = false)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -386,7 +386,8 @@ std::pair<const char *, bool> splitMultipartQuery(
|
||||
const std::string & queries,
|
||||
std::vector<std::string> & queries_list,
|
||||
size_t max_query_size,
|
||||
size_t max_parser_depth)
|
||||
size_t max_parser_depth,
|
||||
bool allow_settings_after_format_in_insert)
|
||||
{
|
||||
ASTPtr ast;
|
||||
|
||||
@ -394,7 +395,7 @@ std::pair<const char *, bool> splitMultipartQuery(
|
||||
const char * pos = begin; /// parser moves pos from begin to the end of current query
|
||||
const char * end = begin + queries.size();
|
||||
|
||||
ParserQuery parser(end);
|
||||
ParserQuery parser(end, allow_settings_after_format_in_insert);
|
||||
|
||||
queries_list.clear();
|
||||
|
||||
|
@ -61,6 +61,7 @@ std::pair<const char *, bool> splitMultipartQuery(
|
||||
const std::string & queries,
|
||||
std::vector<std::string> & queries_list,
|
||||
size_t max_query_size,
|
||||
size_t max_parser_depth);
|
||||
size_t max_parser_depth,
|
||||
bool allow_settings_after_format_in_insert);
|
||||
|
||||
}
|
||||
|
@ -868,7 +868,7 @@ namespace
|
||||
query_text = std::move(*(query_info.mutable_query()));
|
||||
const char * begin = query_text.data();
|
||||
const char * end = begin + query_text.size();
|
||||
ParserQuery parser(end);
|
||||
ParserQuery parser(end, settings.allow_settings_after_format_in_insert);
|
||||
ast = parseQuery(parser, begin, end, "", settings.max_query_size, settings.max_parser_depth);
|
||||
|
||||
/// Choose input format.
|
||||
|
@ -275,7 +275,10 @@ void PostgreSQLHandler::processQuery()
|
||||
|
||||
const auto & settings = session->sessionContext()->getSettingsRef();
|
||||
std::vector<String> queries;
|
||||
auto parse_res = splitMultipartQuery(query->query, queries, settings.max_query_size, settings.max_parser_depth);
|
||||
auto parse_res = splitMultipartQuery(query->query, queries,
|
||||
settings.max_query_size,
|
||||
settings.max_parser_depth,
|
||||
settings.allow_settings_after_format_in_insert);
|
||||
if (!parse_res.second)
|
||||
throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
|
@ -71,13 +71,18 @@ static String clusterNameFromDDLQuery(ContextPtr context, const DDLTask & task)
|
||||
{
|
||||
const char * begin = task.entry.query.data();
|
||||
const char * end = begin + task.entry.query.size();
|
||||
String cluster_name;
|
||||
ParserQuery parser_query(end);
|
||||
const auto & settings = context->getSettingsRef();
|
||||
|
||||
String description = fmt::format("from {}", task.entry_path);
|
||||
ParserQuery parser_query(end, settings.allow_settings_after_format_in_insert);
|
||||
ASTPtr query = parseQuery(parser_query, begin, end, description,
|
||||
context->getSettingsRef().max_query_size, context->getSettingsRef().max_parser_depth);
|
||||
settings.max_query_size,
|
||||
settings.max_parser_depth);
|
||||
|
||||
String cluster_name;
|
||||
if (const auto * query_on_cluster = dynamic_cast<const ASTQueryWithOnCluster *>(query.get()))
|
||||
cluster_name = query_on_cluster->cluster;
|
||||
|
||||
return cluster_name;
|
||||
}
|
||||
|
||||
|
@ -2,6 +2,17 @@ insert into foo settings max_threads=1
|
||||
Syntax error (query): failed at position 40 (end of query):
|
||||
insert into foo format tsv settings max_threads=1
|
||||
Can't format ASTInsertQuery with data, since data will be lost.
|
||||
[multi] insert into foo format tsv settings max_threads=1
|
||||
INSERT INTO foo
|
||||
SETTINGS max_threads = 1
|
||||
FORMAT tsv
|
||||
[oneline] insert into foo format tsv settings max_threads=1
|
||||
INSERT INTO foo SETTINGS max_threads = 1 FORMAT tsv
|
||||
insert into foo settings max_threads=1 format tsv settings max_threads=1
|
||||
You have SETTINGS before and after FORMAT
|
||||
Cannot parse input: expected '\n' before: 'settings max_threads=1 1'
|
||||
1
|
||||
You have SETTINGS before and after FORMAT
|
||||
[multi] insert into foo values
|
||||
INSERT INTO foo FORMAT Values
|
||||
[oneline] insert into foo values
|
||||
|
@ -9,21 +9,32 @@ function run_format()
|
||||
local q="$1" && shift
|
||||
|
||||
echo "$q"
|
||||
$CLICKHOUSE_FORMAT <<<"$q"
|
||||
$CLICKHOUSE_FORMAT "$@" <<<"$q"
|
||||
}
|
||||
function run_format_both()
|
||||
{
|
||||
local q="$1" && shift
|
||||
|
||||
echo "[multi] $q"
|
||||
$CLICKHOUSE_FORMAT <<<"$q"
|
||||
$CLICKHOUSE_FORMAT "$@" <<<"$q"
|
||||
echo "[oneline] $q"
|
||||
$CLICKHOUSE_FORMAT --oneline <<<"$q"
|
||||
$CLICKHOUSE_FORMAT --oneline "$@" <<<"$q"
|
||||
}
|
||||
|
||||
# NOTE: that those queries may work slow, due to stack trace obtaining
|
||||
run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):")
|
||||
# compatibility
|
||||
run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.")
|
||||
run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert
|
||||
run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT")
|
||||
# and via server (since this is a separate code path)
|
||||
$CLICKHOUSE_CLIENT -q 'drop table if exists data_02263'
|
||||
$CLICKHOUSE_CLIENT -q 'create table data_02263 (key Int) engine=Memory()'
|
||||
$CLICKHOUSE_CLIENT -q 'insert into data_02263 format TSV settings max_threads=1 1' 2> >(grep -m1 -F -o "Cannot parse input: expected '\n' before: 'settings max_threads=1 1'")
|
||||
$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 format TSV settings max_threads=1 1'
|
||||
$CLICKHOUSE_CLIENT -q 'select * from data_02263'
|
||||
$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 settings max_threads=1 format tsv settings max_threads=1' 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT")
|
||||
$CLICKHOUSE_CLIENT -q 'drop table data_02263'
|
||||
|
||||
run_format_both 'insert into foo values'
|
||||
run_format_both 'insert into foo select 1'
|
||||
|
Loading…
Reference in New Issue
Block a user