Disambiguate alter commands list (#59532)

* Add optional parentheses to TTL element and alter command

* Add some basic tests

* Extend tests with more test cases

* Add one more test case

* Add server setting to control new formatting behavior

* Automatic style fix

* Fix segfault in `clickhouse-format` related to new server setting

* Remove format_ttl_expressions_with_parentheses

* Fix unit test compilation failures

* Introduce global flag to control the new formatting behavior

* Revert "Fix segfault in `clickhouse-format` related to new server setting"

This reverts commit d7131a3145.

* Revert accidental changes

* Revert accidental include reorder

* Revert accidental changes

* Enable new format in client and local server

---------

Co-authored-by: robot-clickhouse <robot-clickhouse@users.noreply.github.com>
This commit is contained in:
János Benjamin Antal 2024-02-22 19:17:15 +01:00 committed by GitHub
parent 0df33c94ba
commit 7cad0050c0
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
16 changed files with 133 additions and 19 deletions

View File

@ -472,10 +472,10 @@ The value 0 means that you can delete all tables without any restrictions.
``` xml
<max_table_size_to_drop>0</max_table_size_to_drop>
```
## max\_database\_num\_to\_warn {#max-database-num-to-warn}
If the number of attached databases exceeds the specified value, clickhouse server will add warning messages to `system.warnings` table.
## max\_database\_num\_to\_warn {#max-database-num-to-warn}
If the number of attached databases exceeds the specified value, clickhouse server will add warning messages to `system.warnings` table.
Default value: 1000
**Example**
@ -483,10 +483,10 @@ Default value: 1000
``` xml
<max_database_num_to_warn>50</max_database_num_to_warn>
```
## max\_table\_num\_to\_warn {#max-table-num-to-warn}
If the number of attached tables exceeds the specified value, clickhouse server will add warning messages to `system.warnings` table.
Default value: 5000
## max\_table\_num\_to\_warn {#max-table-num-to-warn}
If the number of attached tables exceeds the specified value, clickhouse server will add warning messages to `system.warnings` table.
Default value: 5000
**Example**
@ -495,9 +495,9 @@ Default value: 5000
```
## max\_part\_num\_to\_warn {#max-part-num-to-warn}
If the number of active parts exceeds the specified value, clickhouse server will add warning messages to `system.warnings` table.
Default value: 100000
## max\_part\_num\_to\_warn {#max-part-num-to-warn}
If the number of active parts exceeds the specified value, clickhouse server will add warning messages to `system.warnings` table.
Default value: 100000
**Example**
@ -2873,3 +2873,11 @@ A limit on the number of materialized views attached to a table.
Note that only directly dependent views are considered here, and the creation of one view on top of another view is not considered.
Default value: `0`.
## format_alter_operations_with_parentheses {#format_alter_operations_with_parentheses}
If set to true, then alter operations will be surrounded by parentheses in formatted queries. This makes the parsing of formatted alter queries less ambiguous.
Type: Bool
Default: 0

View File

@ -330,6 +330,7 @@ try
processConfig();
adjustSettings();
initTTYBuffer(toProgressOption(config().getString("progress", "default")));
ASTAlterCommand::setFormatAlterCommandsWithParentheses(true);
{
// All that just to set DB::CurrentThread::get().getGlobalContext()

View File

@ -506,6 +506,7 @@ try
processConfig();
adjustSettings();
initTTYBuffer(toProgressOption(config().getString("progress", "default")));
ASTAlterCommand::setFormatAlterCommandsWithParentheses(true);
applyCmdSettings(global_context);

View File

@ -623,6 +623,8 @@ try
ServerSettings server_settings;
server_settings.loadSettingsFromConfig(config());
ASTAlterCommand::setFormatAlterCommandsWithParentheses(server_settings.format_alter_operations_with_parentheses);
StackTrace::setShowAddresses(server_settings.show_addresses_in_stack_traces);
#if USE_HDFS

View File

@ -115,6 +115,7 @@ namespace DB
M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \
M(UInt64, max_materialized_views_count_for_table, 0, "A limit on the number of materialized views attached to a table.", 0) \
M(UInt32, max_database_replicated_create_table_thread_pool_size, 1, "The number of threads to create tables during replica recovery in DatabaseReplicated. Zero means number of threads equal number of cores.", 0) \
M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \
M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \
M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \

View File

@ -277,9 +277,11 @@ BlockIO InterpreterKillQueryQuery::execute()
code = CancellationCode::NotFound;
else
{
ParserAlterCommand parser;
const auto alter_command = command_col.getDataAt(i).toString();
const auto with_round_bracket = alter_command.front() == '(';
ParserAlterCommand parser{with_round_bracket};
auto command_ast
= parseQuery(parser, command_col.getDataAt(i).toString(), 0, getContext()->getSettingsRef().max_parser_depth);
= parseQuery(parser, alter_command, 0, getContext()->getSettingsRef().max_parser_depth);
required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand(
command_ast->as<const ASTAlterCommand &>(), table_id.database_name, table_id.table_name);
if (!access->isGranted(required_access_rights))

View File

@ -1,6 +1,7 @@
#include <iomanip>
#include <IO/Operators.h>
#include <Parsers/ASTAlterQuery.h>
#include <Core/ServerSettings.h>
#include <IO/Operators.h>
#include <Common/quoteString.h>
@ -69,6 +70,9 @@ ASTPtr ASTAlterCommand::clone() const
void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (format_alter_commands_with_parentheses)
settings.ostr << "(";
if (type == ASTAlterCommand::ADD_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD COLUMN " << (if_not_exists ? "IF NOT EXISTS " : "")
@ -486,6 +490,9 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState &
}
else
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected type of ALTER");
if (format_alter_commands_with_parentheses)
settings.ostr << ")";
}
void ASTAlterCommand::forEachPointerToChild(std::function<void(void**)> f)

View File

@ -226,10 +226,16 @@ public:
ASTPtr clone() const override;
// This function is only meant to be called during application startup
// For reasons see https://github.com/ClickHouse/ClickHouse/pull/59532
static void setFormatAlterCommandsWithParentheses(bool value) { format_alter_commands_with_parentheses = value; }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void forEachPointerToChild(std::function<void(void**)> f) override;
static inline bool format_alter_commands_with_parentheses = false;
};
class ASTAlterQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster

View File

@ -118,6 +118,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY");
ParserKeyword s_apply_deleted_mask("APPLY DELETED MASK");
ParserToken parser_opening_round_bracket(TokenType::OpeningRoundBracket);
ParserToken parser_closing_round_bracket(TokenType::ClosingRoundBracket);
ParserCompoundIdentifier parser_name;
ParserStringLiteral parser_string_literal;
ParserStringAndSubstitution parser_string_and_substituion;
@ -166,6 +169,12 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ASTPtr command_rename_to;
ASTPtr command_sql_security;
if (with_round_bracket)
{
if (!parser_opening_round_bracket.ignore(pos, expected))
return false;
}
switch (alter_object)
{
case ASTAlterQuery::AlterObjectType::LIVE_VIEW:
@ -896,6 +905,12 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
}
}
if (with_round_bracket)
{
if (!parser_closing_round_bracket.ignore(pos, expected))
return false;
}
if (command_col_decl)
command->col_decl = command->children.emplace_back(std::move(command_col_decl)).get();
if (command_column)
@ -951,7 +966,10 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
node = command_list;
ParserToken s_comma(TokenType::Comma);
ParserAlterCommand p_command(alter_object);
const auto with_round_bracket = pos->type == TokenType::OpeningRoundBracket;
ParserAlterCommand p_command(with_round_bracket, alter_object);
do
{

View File

@ -61,10 +61,15 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
public:
bool with_round_bracket;
ASTAlterQuery::AlterObjectType alter_object;
ParserAlterCommand(ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE)
: alter_object(alter_object_) {}
explicit ParserAlterCommand(
bool with_round_bracket_, ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE)
: with_round_bracket(with_round_bracket_), alter_object(alter_object_)
{
}
};

View File

@ -132,7 +132,7 @@ INSTANTIATE_TEST_SUITE_P(ParserOptimizeQuery, ParserTest,
INSTANTIATE_TEST_SUITE_P(ParserOptimizeQuery_FAIL, ParserTest,
::testing::Combine(
::testing::Values(std::make_shared<ParserAlterCommand>()),
::testing::Values(std::make_shared<ParserAlterCommand>(false)),
::testing::ValuesIn(std::initializer_list<ParserTestCase>
{
{
@ -159,7 +159,7 @@ INSTANTIATE_TEST_SUITE_P(ParserOptimizeQuery_FAIL, ParserTest,
INSTANTIATE_TEST_SUITE_P(ParserAlterCommand_MODIFY_COMMENT, ParserTest,
::testing::Combine(
::testing::Values(std::make_shared<ParserAlterCommand>()),
::testing::Values(std::make_shared<ParserAlterCommand>(false)),
::testing::ValuesIn(std::initializer_list<ParserTestCase>
{
{

View File

@ -0,0 +1,3 @@
<clickhouse>
<format_alter_operations_with_parentheses>1</format_alter_operations_with_parentheses>
</clickhouse>

View File

@ -0,0 +1,45 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
main_configs=[
"configs/format_alter_operations_with_parentheses.xml",
],
)
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_alter():
INPUT = """
SELECT '--- Alter commands in parens';
SELECT formatQuery('ALTER TABLE a (MODIFY TTL expr GROUP BY some_key), (ADD COLUMN a Int64)');
SELECT formatQuery('ALTER TABLE a (MODIFY TTL expr TO VOLUME \\'vol1\\', expr2 + INTERVAL 2 YEAR TO VOLUME \\'vol2\\'), (DROP COLUMN c)');
SELECT '--- Check only consistent parens around alter commands are accepted';
SELECT formatQuery('ALTER TABLE a (DROP COLUMN b), DROP COLUMN c'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a DROP COLUMN b, (DROP COLUMN c)'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a (DROP COLUMN b), (DROP COLUMN c)');
SELECT formatQuery('ALTER TABLE a DROP COLUMN b, DROP COLUMN c'); -- Make sure it is backward compatible
"""
EXPECTED_OUTPUT = """--- Alter commands in parens
ALTER TABLE a\\n (MODIFY TTL expr GROUP BY some_key),\\n (ADD COLUMN `a` Int64)
ALTER TABLE a\\n (MODIFY TTL expr TO VOLUME \\'vol1\\', expr2 + toIntervalYear(2) TO VOLUME \\'vol2\\'),\\n (DROP COLUMN c)
--- Check only consistent parens around alter commands are accepted
ALTER TABLE a\\n (DROP COLUMN b),\\n (DROP COLUMN c)
ALTER TABLE a\\n (DROP COLUMN b),\\n (DROP COLUMN c)
"""
result = node.query(INPUT)
assert result == EXPECTED_OUTPUT

View File

@ -0,0 +1,6 @@
--- Alter commands in parens
ALTER TABLE a\n MODIFY TTL expr GROUP BY some_key,\n ADD COLUMN `a` Int64
ALTER TABLE a\n MODIFY TTL expr TO VOLUME \'vol1\', expr2 + toIntervalYear(2) TO VOLUME \'vol2\',\n DROP COLUMN c
--- Check only consistent parens around alter commands are accepted
ALTER TABLE a\n DROP COLUMN b,\n DROP COLUMN c
ALTER TABLE a\n DROP COLUMN b,\n DROP COLUMN c

View File

@ -0,0 +1,9 @@
SELECT '--- Alter commands in parens';
SELECT formatQuery('ALTER TABLE a (MODIFY TTL expr GROUP BY some_key), (ADD COLUMN a Int64)');
SELECT formatQuery('ALTER TABLE a (MODIFY TTL expr TO VOLUME \'vol1\', expr2 + INTERVAL 2 YEAR TO VOLUME \'vol2\'), (DROP COLUMN c)');
SELECT '--- Check only consistent parens around alter commands are accepted';
SELECT formatQuery('ALTER TABLE a (DROP COLUMN b), DROP COLUMN c'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a DROP COLUMN b, (DROP COLUMN c)'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a (DROP COLUMN b), (DROP COLUMN c)');
SELECT formatQuery('ALTER TABLE a DROP COLUMN b, DROP COLUMN c'); -- Make sure it is backward compatible