Skip to content

Commit

Permalink
Merge pull request ClickHouse#66104 from ClickHouse/break_compatibility
Browse files Browse the repository at this point in the history
Don't allow explicit UUIDs and ReplicatedMergeTree arguments in Replicated databases
  • Loading branch information
tavplubix authored Sep 9, 2024
2 parents 2b2cd8e + 238c9b5 commit 9ab3ee1
Show file tree
Hide file tree
Showing 21 changed files with 148 additions and 31 deletions.
6 changes: 5 additions & 1 deletion src/Backups/RestorerFromBackup.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -913,11 +913,15 @@ void RestorerFromBackup::createTable(const QualifiedTableName & table_name)
table_info.database = DatabaseCatalog::instance().getDatabase(table_name.database);
DatabasePtr database = table_info.database;

auto query_context = Context::createCopy(context);
query_context->setSetting("database_replicated_allow_explicit_uuid", 3);
query_context->setSetting("database_replicated_allow_replicated_engine_arguments", 3);

/// Execute CREATE TABLE query (we call IDatabase::createTableRestoredFromBackup() to allow the database to do some
/// database-specific things).
database->createTableRestoredFromBackup(
create_table_query,
context,
query_context,
restore_coordination,
std::chrono::duration_cast<std::chrono::milliseconds>(create_table_timeout).count());
}
Expand Down
3 changes: 2 additions & 1 deletion src/Core/Settings.h
Original file line number Diff line number Diff line change
Expand Up @@ -710,7 +710,8 @@ class IColumn;
M(UInt64, max_distributed_depth, 5, "Maximum distributed query depth", 0) \
M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \
M(Bool, database_replicated_allow_only_replicated_engine, false, "Allow to create only Replicated tables in database with engine Replicated", 0) \
M(Bool, database_replicated_allow_replicated_engine_arguments, true, "Allow to create only Replicated tables in database with engine Replicated with explicit arguments", 0) \
M(UInt64, database_replicated_allow_replicated_engine_arguments, 0, "0 - Don't allow to explicitly specify ZooKeeper path and replica name for *MergeTree tables in Replicated databases. 1 - Allow. 2 - Allow, but ignore the specified path and use default one instead.", 0) \
M(UInt64, database_replicated_allow_explicit_uuid, 0, "0 - Don't allow to explicitly specify UUIDs for tables in Replicated databases. 1 - Allow. 2 - Allow, but ignore the specified UUID and generate a random one instead.", 0) \
M(Bool, database_replicated_allow_heavy_create, false, "Allow long-running DDL queries (CREATE AS SELECT and POPULATE) in Replicated database engine. Note that it can block DDL queue for a long time.", 0) \
M(Bool, cloud_mode, false, "Only available in ClickHouse Cloud", 0) \
M(UInt64, cloud_mode_engine, 1, "Only available in ClickHouse Cloud", 0) \
Expand Down
4 changes: 3 additions & 1 deletion src/Core/SettingsChangesHistory.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,9 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"create_if_not_exists", false, false, "New setting."},
{"allow_materialized_view_with_bad_select", true, true, "Support (but not enable yet) stricter validation in CREATE MATERIALIZED VIEW"},
{"output_format_always_quote_identifiers", false, false, "New setting."},
{"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."}
{"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."},
{"database_replicated_allow_replicated_engine_arguments", 1, 0, "Don't allow explicit arguments by default"},
{"database_replicated_allow_explicit_uuid", 0, 0, "Added a new setting to disallow explicitly specifying table UUID"},
}
},
{"24.8",
Expand Down
19 changes: 6 additions & 13 deletions src/Databases/DatabaseReplicated.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -907,18 +907,6 @@ void DatabaseReplicated::checkTableEngine(const ASTCreateQuery & query, ASTStora
bool maybe_replica_macros = info.expanded_other;
bool enable_functional_tests_helper = getContext()->getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros");

if (!enable_functional_tests_helper)
{
if (query_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments)
LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments");
else
throw Exception(ErrorCodes::INCORRECT_QUERY,
"It's not allowed to specify explicit zookeeper_path and replica_name "
"for ReplicatedMergeTree arguments in Replicated database. If you really want to "
"specify them explicitly, enable setting "
"database_replicated_allow_replicated_engine_arguments.");
}

if (maybe_shard_macros && maybe_replica_macros)
return;

Expand All @@ -931,7 +919,9 @@ void DatabaseReplicated::checkTableEngine(const ASTCreateQuery & query, ASTStora
return;
}

throw Exception(ErrorCodes::INCORRECT_QUERY,
/// We will replace it with default arguments if the setting is 2
if (query_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments != 2)
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Explicit zookeeper_path and replica_name are specified in ReplicatedMergeTree arguments. "
"If you really want to specify it explicitly, then you should use some macros "
"to distinguish different shards and replicas");
Expand Down Expand Up @@ -1200,6 +1190,9 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
/// so we need to allow experimental features that can be used in a CREATE query
enableAllExperimentalSettings(query_context);

query_context->setSetting("database_replicated_allow_explicit_uuid", 3);
query_context->setSetting("database_replicated_allow_replicated_engine_arguments", 3);

auto txn = std::make_shared<ZooKeeperMetadataTransaction>(current_zookeeper, zookeeper_path, false, "");
query_context->initZooKeeperMetadataTransaction(txn);
return query_context;
Expand Down
21 changes: 21 additions & 0 deletions src/Interpreters/InterpreterCreateQuery.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1229,6 +1229,27 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
bool from_path = create.attach_from_path.has_value();
bool is_on_cluster = getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;

if (database->getEngineName() == "Replicated" && create.uuid != UUIDHelpers::Nil && !is_replicated_database_internal && !is_on_cluster && !create.attach)
{
if (getContext()->getSettingsRef().database_replicated_allow_explicit_uuid == 0)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "It's not allowed to explicitly specify UUIDs for tables in Replicated databases, "
"see database_replicated_allow_explicit_uuid");
}
else if (getContext()->getSettingsRef().database_replicated_allow_explicit_uuid == 1)
{
LOG_WARNING(&Poco::Logger::get("InterpreterCreateQuery"), "It's not recommended to explicitly specify UUIDs for tables in Replicated databases");
}
else if (getContext()->getSettingsRef().database_replicated_allow_explicit_uuid == 2)
{
UUID old_uuid = create.uuid;
create.uuid = UUIDHelpers::Nil;
create.generateRandomUUIDs();
LOG_WARNING(&Poco::Logger::get("InterpreterCreateQuery"), "Replaced a user-provided UUID ({}) with a random one ({}) "
"to make sure it's unique", old_uuid, create.uuid);
}
}

if (is_replicated_database_internal && !internal)
{
if (create.uuid == UUIDHelpers::Nil)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,6 @@ using ContextPtr = std::shared_ptr<const Context>;
/// Extracts a zookeeper path from a specified CREATE TABLE query.
/// The function checks the table engine and if it is Replicated*MergeTree then it takes the first argument and expands macros in it.
/// Returns std::nullopt if the specified CREATE query doesn't describe a Replicated table or its arguments can't be evaluated.
std::optional<String> extractZooKeeperPathFromReplicatedTableDef(const ASTCreateQuery & create_query, const ContextPtr & context);
std::optional<String> extractZooKeeperPathFromReplicatedTableDef(const ASTCreateQuery & create_query, const ContextPtr & local_context);

}
46 changes: 35 additions & 11 deletions src/Storages/MergeTree/registerStorageMergeTree.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
#include <Common/Macros.h>
#include <Common/OptimizedRegularExpression.h>
#include <Common/typeid_cast.h>
#include <Common/logger_useful.h>

#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
Expand Down Expand Up @@ -189,7 +190,7 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
const String & engine_name,
ASTs & engine_args,
LoadingStrictnessLevel mode,
const ContextPtr & context,
const ContextPtr & local_context,
String & zookeeper_path,
String & replica_name,
RenamingRestrictions & renaming_restrictions)
Expand All @@ -206,11 +207,11 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
{
/// Allow expressions in engine arguments.
/// In new syntax argument can be literal or identifier or array/tuple of identifiers.
evaluateEngineArgs(engine_args, context);
evaluateEngineArgs(engine_args, local_context);
}

bool is_on_cluster = context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
bool is_replicated_database = context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY &&
bool is_on_cluster = local_context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
bool is_replicated_database = local_context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY &&
DatabaseCatalog::instance().getDatabase(table_id.database_name)->getEngineName() == "Replicated";

/// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries
Expand All @@ -230,10 +231,10 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
/// We did unfold it in previous versions to make moving table from Atomic to Ordinary database work correctly,
/// but now it's not allowed (and it was the only reason to unfold {uuid} macro).
info.table_id.uuid = UUIDHelpers::Nil;
zookeeper_path = context->getMacros()->expand(zookeeper_path, info);
zookeeper_path = local_context->getMacros()->expand(zookeeper_path, info);

info.level = 0;
replica_name = context->getMacros()->expand(replica_name, info);
replica_name = local_context->getMacros()->expand(replica_name, info);
}

ast_zk_path->value = zookeeper_path;
Expand All @@ -251,11 +252,11 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
}
if (!allow_uuid_macro)
info.table_id.uuid = UUIDHelpers::Nil;
zookeeper_path = context->getMacros()->expand(zookeeper_path, info);
zookeeper_path = local_context->getMacros()->expand(zookeeper_path, info);

info.level = 0;
info.table_id.uuid = UUIDHelpers::Nil;
replica_name = context->getMacros()->expand(replica_name, info);
replica_name = local_context->getMacros()->expand(replica_name, info);

/// We do not allow renaming table with these macros in metadata, because zookeeper_path will be broken after RENAME TABLE.
/// NOTE: it may happen if table was created by older version of ClickHouse (< 20.10) and macros was not unfolded on table creation
Expand All @@ -272,9 +273,24 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(

bool has_arguments = (arg_num + 2 <= arg_cnt);
bool has_valid_arguments = has_arguments && engine_args[arg_num]->as<ASTLiteral>() && engine_args[arg_num + 1]->as<ASTLiteral>();
const auto & server_settings = local_context->getServerSettings();

if (has_valid_arguments)
{
if (is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 0)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"It's not allowed to specify explicit zookeeper_path and replica_name "
"for ReplicatedMergeTree arguments in Replicated database. If you really want to "
"specify them explicitly, enable setting "
"database_replicated_allow_replicated_engine_arguments.");
}
else if (is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 1)
{
LOG_WARNING(&Poco::Logger::get("registerStorageMergeTree"), "It's not recommended to explicitly specify "
"zookeeper_path and replica_name in ReplicatedMergeTree arguments");
}

/// Get path and name from engine arguments
auto * ast_zk_path = engine_args[arg_num]->as<ASTLiteral>();
if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String)
Expand All @@ -288,6 +304,15 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica name must be a string literal{}", verbose_help_message);


if (is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 2)
{
LOG_WARNING(&Poco::Logger::get("registerStorageMergeTree"), "Replacing user-provided ZooKeeper path and replica name ({}, {}) "
"with default arguments", zookeeper_path, replica_name);
engine_args[arg_num]->as<ASTLiteral>()->value = zookeeper_path = server_settings.default_replica_path;
engine_args[arg_num + 1]->as<ASTLiteral>()->value = replica_name = server_settings.default_replica_name;
}

expand_macro(ast_zk_path, ast_replica_name);
}
else if (is_extended_storage_def
Expand All @@ -297,7 +322,6 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
{
/// Try use default values if arguments are not specified.
/// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic.
const auto & server_settings = context->getServerSettings();
zookeeper_path = server_settings.default_replica_path;
/// TODO maybe use hostname if {replica} is not defined?
replica_name = server_settings.default_replica_name;
Expand All @@ -322,7 +346,7 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
}

/// Extracts a zookeeper path from a specified CREATE TABLE query.
std::optional<String> extractZooKeeperPathFromReplicatedTableDef(const ASTCreateQuery & query, const ContextPtr & context)
std::optional<String> extractZooKeeperPathFromReplicatedTableDef(const ASTCreateQuery & query, const ContextPtr & local_context)
{
if (!query.storage || !query.storage->engine)
return {};
Expand All @@ -346,7 +370,7 @@ std::optional<String> extractZooKeeperPathFromReplicatedTableDef(const ASTCreate

try
{
extractZooKeeperPathAndReplicaNameFromEngineArgs(query, table_id, engine_name, engine_args, mode, context,
extractZooKeeperPathAndReplicaNameFromEngineArgs(query, table_id, engine_name, engine_args, mode, local_context,
zookeeper_path, replica_name, renaming_restrictions);
}
catch (Exception & e)
Expand Down
1 change: 1 addition & 0 deletions tests/config/users.d/database_replicated.xml
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
<distributed_ddl_task_timeout>120</distributed_ddl_task_timeout>
<database_replicated_always_detach_permanently>1</database_replicated_always_detach_permanently>
<database_replicated_enforce_synchronous_settings>1</database_replicated_enforce_synchronous_settings>
<database_replicated_allow_replicated_engine_arguments>3</database_replicated_allow_replicated_engine_arguments>
</default>
</profiles>
</clickhouse>
3 changes: 2 additions & 1 deletion tests/integration/test_disk_over_web_server/test.py
Original file line number Diff line number Diff line change
Expand Up @@ -311,7 +311,8 @@ def test_replicated_database(cluster):
SETTINGS storage_policy = 'web';
""".format(
uuids[0]
)
),
settings={"database_replicated_allow_explicit_uuid": 3},
)

node2 = cluster.instances["node2"]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,8 @@
<allow_experimental_alter_materialized_view_structure>1</allow_experimental_alter_materialized_view_structure>
<allow_experimental_object_type>0</allow_experimental_object_type>
<allow_suspicious_codecs>0</allow_suspicious_codecs>
<database_replicated_allow_replicated_engine_arguments>3</database_replicated_allow_replicated_engine_arguments>
<database_replicated_allow_explicit_uuid>3</database_replicated_allow_explicit_uuid>
</default>
</profiles>
<users>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,8 @@
<allow_experimental_alter_materialized_view_structure>1</allow_experimental_alter_materialized_view_structure>
<allow_experimental_object_type>0</allow_experimental_object_type>
<allow_suspicious_codecs>0</allow_suspicious_codecs>
<database_replicated_allow_replicated_engine_arguments>3</database_replicated_allow_replicated_engine_arguments>
<database_replicated_allow_explicit_uuid>3</database_replicated_allow_explicit_uuid>

<throw_on_unsupported_query_inside_transaction>0</throw_on_unsupported_query_inside_transaction>
</default>
Expand Down
Original file line number Diff line number Diff line change
@@ -1,6 +1,8 @@
<clickhouse>
<profiles>
<default>
<database_replicated_allow_replicated_engine_arguments>3</database_replicated_allow_replicated_engine_arguments>
<database_replicated_allow_explicit_uuid>3</database_replicated_allow_explicit_uuid>
</default>
</profiles>
<users>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,8 @@ DROP TABLE IF EXISTS without_deduplication;
DROP TABLE IF EXISTS with_deduplication_mv;
DROP TABLE IF EXISTS without_deduplication_mv;

SET database_replicated_allow_explicit_uuid=3;
SET database_replicated_allow_replicated_engine_arguments=3;
CREATE TABLE with_deduplication(x UInt32)
ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/with_deduplication', 'r1') ORDER BY x;
CREATE TABLE without_deduplication(x UInt32)
Expand Down
3 changes: 2 additions & 1 deletion tests/queries/0_stateless/00609_mv_index_in_in.sql
Original file line number Diff line number Diff line change
@@ -1,11 +1,12 @@
-- Tags: no-ordinary-database
-- Tags: no-ordinary-database, no-parallel

DROP TABLE IF EXISTS test_00609;
DROP TABLE IF EXISTS test_mv_00609;

create table test_00609 (a Int8) engine=Memory;

insert into test_00609 values (1);
set database_replicated_allow_explicit_uuid=3;
set allow_deprecated_syntax_for_merge_tree=1;
create materialized view test_mv_00609 uuid '00000609-1000-4000-8000-000000000001' Engine=MergeTree(date, (a), 8192) populate as select a, toDate('2000-01-01') date from test_00609;

Expand Down
2 changes: 2 additions & 0 deletions tests/queries/0_stateless/00738_lock_for_inner_table.sh
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh

CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --database_replicated_allow_explicit_uuid 3"

# there are some issues with Atomic database, let's generate it uniq
# otherwise flaky check will not pass.
uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase())")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,8 @@ DROP TABLE IF EXISTS rmt1;
DROP TABLE IF EXISTS rmt2;
DROP TABLE IF EXISTS rmt3;

SET database_replicated_allow_replicated_engine_arguments=1;

CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/test_01148/{shard}/{database}/{table}', '{replica}') ORDER BY n;
SHOW CREATE TABLE rmt;
RENAME TABLE rmt TO rmt1;
Expand Down
2 changes: 2 additions & 0 deletions tests/queries/0_stateless/01153_attach_mv_uuid.sql
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@ INSERT INTO src VALUES (3), (4);
SELECT * FROM mv ORDER BY n;
DROP TABLE mv SYNC;

SET database_replicated_allow_explicit_uuid=3;

SET show_table_uuid_in_table_create_query_if_not_nil=1;
CREATE TABLE ".inner_id.e15f3ab5-6cae-4df3-b879-f40deafd82c2" (n Int32, n2 Int64) ENGINE = MergeTree PARTITION BY n % 10 ORDER BY n;
ATTACH MATERIALIZED VIEW mv UUID 'e15f3ab5-6cae-4df3-b879-f40deafd82c2' (n Int32, n2 Int64) ENGINE = MergeTree PARTITION BY n % 10 ORDER BY n AS SELECT n, n * n AS n2 FROM src;
Expand Down
Loading

0 comments on commit 9ab3ee1

Please sign in to comment.