don't allow explicit uuids and rmt args in replicated db

This commit is contained in:
Alexander Tokmakov 2024-07-04 18:48:41 +02:00
parent bdc7157670
commit 4e350ff44a
10 changed files with 112 additions and 25 deletions

View File

@ -731,7 +731,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, cloud_mode, false, "Only available in ClickHouse Cloud", 0) \
M(UInt64, cloud_mode_engine, 1, "Only available in ClickHouse Cloud", 0) \
M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result, one of: 'none', 'throw', 'null_status_on_timeout', 'never_throw', 'none_only_active', 'throw_only_active', 'null_status_on_timeout_only_active'", 0) \

View File

@ -774,18 +774,6 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_
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;

View File

@ -1055,6 +1055,26 @@ 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)
{
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 == 0)
{
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.generateRandomUUID(/*always_generate_new_uuid*/ true);
LOG_INFO(&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)

View File

@ -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);
}

View File

@ -9,6 +9,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>
@ -185,7 +186,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)
@ -202,11 +203,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
@ -226,10 +227,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;
@ -247,11 +248,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
@ -268,9 +269,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)
@ -284,6 +300,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_INFO(&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
@ -293,7 +318,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;
@ -318,7 +342,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 {};
@ -342,7 +366,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)

View File

@ -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>

View File

@ -5,6 +5,7 @@
<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>
</default>
</profiles>
<users>

View File

@ -5,6 +5,7 @@
<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>
<throw_on_unsupported_query_inside_transaction>0</throw_on_unsupported_query_inside_transaction>
</default>

View File

@ -0,0 +1,10 @@
2
3
m1
m2
rmt1
rmt2
02858000-1000-4000-8000-0000000000
0
CREATE TABLE rdb_default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/default/auto_{shard}\', \'1auto_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192
CREATE TABLE rdb_default.rmt2\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{uuid}/{shard}\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192

View File

@ -0,0 +1,41 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
db=$CLICKHOUSE_DATABASE
if [[ $($CLICKHOUSE_CLIENT -q "SELECT engine = 'Replicated' FROM system.databases WHERE name='$CLICKHOUSE_DATABASE'") != 1 ]]; then
$CLICKHOUSE_CLIENT -q "CREATE DATABASE rdb_$CLICKHOUSE_DATABASE ENGINE=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', '1', '1')"
db="rdb_$CLICKHOUSE_DATABASE"
fi
$CLICKHOUSE_CLIENT --database_replicated_allow_explicit_uuid=0 -q "CREATE TABLE $db.m0
UUID '02858000-1000-4000-8000-000000000000' (n int) ENGINE=Memory" 2>&1| grep -Fac "database_replicated_allow_explicit_uuid"
$CLICKHOUSE_CLIENT --database_replicated_allow_explicit_uuid=1 -q "CREATE TABLE $db.m1
UUID '02858000-1000-4000-8000-0000000000$(($RANDOM % 10))$(($RANDOM % 10))' (n int) ENGINE=Memory"
$CLICKHOUSE_CLIENT --database_replicated_allow_explicit_uuid=2 -q "CREATE TABLE $db.m2
UUID '02858000-1000-4000-8000-000000000002' (n int) ENGINE=Memory"
$CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=0 -q "CREATE TABLE $db.rmt0 (n int)
ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_DATABASE', '1') ORDER BY n" 2>&1| grep -Fac "database_replicated_allow_replicated_engine_arguments"
$CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=1 -q "CREATE TABLE $db.rmt1 (n int)
ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_DATABASE', '1') ORDER BY n"
$CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=2 -q "CREATE TABLE $db.rmt2 (n int)
ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_DATABASE', '1') ORDER BY n"
$CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database='$db' ORDER BY name"
$CLICKHOUSE_CLIENT -q "SELECT substring(toString(uuid) as s, 1, length(s) - 2) FROM system.tables WHERE database='$db' and name='m1'"
$CLICKHOUSE_CLIENT -q "SELECT toString(uuid) LIKE '02858000%' FROM system.tables WHERE database='$db' and name='m2'"
$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt1"
$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt2"
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS rdb_$CLICKHOUSE_DATABASE"