mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-09 17:14:47 +00:00
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:
parent
0df33c94ba
commit
7cad0050c0
@ -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
|
||||
|
@ -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()
|
||||
|
@ -506,6 +506,7 @@ try
|
||||
processConfig();
|
||||
adjustSettings();
|
||||
initTTYBuffer(toProgressOption(config().getString("progress", "default")));
|
||||
ASTAlterCommand::setFormatAlterCommandsWithParentheses(true);
|
||||
|
||||
applyCmdSettings(global_context);
|
||||
|
||||
|
@ -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
|
||||
|
@ -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) \
|
||||
|
||||
|
@ -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))
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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_)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -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>
|
||||
{
|
||||
{
|
||||
|
@ -0,0 +1,3 @@
|
||||
<clickhouse>
|
||||
<format_alter_operations_with_parentheses>1</format_alter_operations_with_parentheses>
|
||||
</clickhouse>
|
45
tests/integration/test_unambiguous_alter_commands/test.py
Normal file
45
tests/integration/test_unambiguous_alter_commands/test.py
Normal 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
|
@ -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
|
@ -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
|
Loading…
Reference in New Issue
Block a user