Merge pull request #66104 from ClickHouse/break_compatibility

Don't allow explicit UUIDs and ReplicatedMergeTree arguments in Replicated databases
This commit is contained in:
Alexander Tokmakov 2024-09-09 18:29:16 +00:00 committed by GitHub
commit 9ab3ee1b13
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
21 changed files with 148 additions and 31 deletions

View File

@ -913,11 +913,15 @@ void RestorerFromBackup::createTable(const QualifiedTableName & table_name)
table_info.database = DatabaseCatalog::instance().getDatabase(table_name.database); table_info.database = DatabaseCatalog::instance().getDatabase(table_name.database);
DatabasePtr database = table_info.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 /// Execute CREATE TABLE query (we call IDatabase::createTableRestoredFromBackup() to allow the database to do some
/// database-specific things). /// database-specific things).
database->createTableRestoredFromBackup( database->createTableRestoredFromBackup(
create_table_query, create_table_query,
context, query_context,
restore_coordination, restore_coordination,
std::chrono::duration_cast<std::chrono::milliseconds>(create_table_timeout).count()); std::chrono::duration_cast<std::chrono::milliseconds>(create_table_timeout).count());
} }

View File

@ -710,7 +710,8 @@ class IColumn;
M(UInt64, max_distributed_depth, 5, "Maximum distributed query depth", 0) \ 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_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_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, 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(Bool, cloud_mode, false, "Only available in ClickHouse Cloud", 0) \
M(UInt64, cloud_mode_engine, 1, "Only available in ClickHouse Cloud", 0) \ M(UInt64, cloud_mode_engine, 1, "Only available in ClickHouse Cloud", 0) \

View File

@ -76,7 +76,9 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"create_if_not_exists", false, false, "New setting."}, {"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"}, {"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_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", {"24.8",

View File

@ -907,18 +907,6 @@ void DatabaseReplicated::checkTableEngine(const ASTCreateQuery & query, ASTStora
bool maybe_replica_macros = info.expanded_other; bool maybe_replica_macros = info.expanded_other;
bool enable_functional_tests_helper = getContext()->getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros"); 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) if (maybe_shard_macros && maybe_replica_macros)
return; return;
@ -931,7 +919,9 @@ void DatabaseReplicated::checkTableEngine(const ASTCreateQuery & query, ASTStora
return; 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. " "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 " "If you really want to specify it explicitly, then you should use some macros "
"to distinguish different shards and replicas"); "to distinguish different shards and replicas");
@ -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 /// so we need to allow experimental features that can be used in a CREATE query
enableAllExperimentalSettings(query_context); 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, ""); auto txn = std::make_shared<ZooKeeperMetadataTransaction>(current_zookeeper, zookeeper_path, false, "");
query_context->initZooKeeperMetadataTransaction(txn); query_context->initZooKeeperMetadataTransaction(txn);
return query_context; return query_context;

View File

@ -1229,6 +1229,27 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
bool from_path = create.attach_from_path.has_value(); bool from_path = create.attach_from_path.has_value();
bool is_on_cluster = getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; 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 (is_replicated_database_internal && !internal)
{ {
if (create.uuid == UUIDHelpers::Nil) 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. /// 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. /// 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. /// 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

@ -12,6 +12,7 @@
#include <Common/Macros.h> #include <Common/Macros.h>
#include <Common/OptimizedRegularExpression.h> #include <Common/OptimizedRegularExpression.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Common/logger_useful.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
@ -189,7 +190,7 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
const String & engine_name, const String & engine_name,
ASTs & engine_args, ASTs & engine_args,
LoadingStrictnessLevel mode, LoadingStrictnessLevel mode,
const ContextPtr & context, const ContextPtr & local_context,
String & zookeeper_path, String & zookeeper_path,
String & replica_name, String & replica_name,
RenamingRestrictions & renaming_restrictions) RenamingRestrictions & renaming_restrictions)
@ -206,11 +207,11 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
{ {
/// Allow expressions in engine arguments. /// Allow expressions in engine arguments.
/// In new syntax argument can be literal or identifier or array/tuple of identifiers. /// 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_on_cluster = local_context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
bool is_replicated_database = 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"; DatabaseCatalog::instance().getDatabase(table_id.database_name)->getEngineName() == "Replicated";
/// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries
@ -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, /// 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). /// but now it's not allowed (and it was the only reason to unfold {uuid} macro).
info.table_id.uuid = UUIDHelpers::Nil; 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.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; ast_zk_path->value = zookeeper_path;
@ -251,11 +252,11 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
} }
if (!allow_uuid_macro) if (!allow_uuid_macro)
info.table_id.uuid = UUIDHelpers::Nil; 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.level = 0;
info.table_id.uuid = UUIDHelpers::Nil; 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. /// 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 /// NOTE: it may happen if table was created by older version of ClickHouse (< 20.10) and macros was not unfolded on table creation
@ -272,9 +273,24 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
bool has_arguments = (arg_num + 2 <= arg_cnt); 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>(); 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 (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 /// Get path and name from engine arguments
auto * ast_zk_path = engine_args[arg_num]->as<ASTLiteral>(); auto * ast_zk_path = engine_args[arg_num]->as<ASTLiteral>();
if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String) if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String)
@ -288,6 +304,15 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
else else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica name must be a string literal{}", verbose_help_message); 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); expand_macro(ast_zk_path, ast_replica_name);
} }
else if (is_extended_storage_def else if (is_extended_storage_def
@ -297,7 +322,6 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
{ {
/// Try use default values if arguments are not specified. /// Try use default values if arguments are not specified.
/// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. /// 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; zookeeper_path = server_settings.default_replica_path;
/// TODO maybe use hostname if {replica} is not defined? /// TODO maybe use hostname if {replica} is not defined?
replica_name = server_settings.default_replica_name; replica_name = server_settings.default_replica_name;
@ -322,7 +346,7 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
} }
/// Extracts a zookeeper path from a specified CREATE TABLE query. /// 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) if (!query.storage || !query.storage->engine)
return {}; return {};
@ -346,7 +370,7 @@ std::optional<String> extractZooKeeperPathFromReplicatedTableDef(const ASTCreate
try 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); zookeeper_path, replica_name, renaming_restrictions);
} }
catch (Exception & e) catch (Exception & e)

View File

@ -6,6 +6,7 @@
<distributed_ddl_task_timeout>120</distributed_ddl_task_timeout> <distributed_ddl_task_timeout>120</distributed_ddl_task_timeout>
<database_replicated_always_detach_permanently>1</database_replicated_always_detach_permanently> <database_replicated_always_detach_permanently>1</database_replicated_always_detach_permanently>
<database_replicated_enforce_synchronous_settings>1</database_replicated_enforce_synchronous_settings> <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> </default>
</profiles> </profiles>
</clickhouse> </clickhouse>

View File

@ -311,7 +311,8 @@ def test_replicated_database(cluster):
SETTINGS storage_policy = 'web'; SETTINGS storage_policy = 'web';
""".format( """.format(
uuids[0] uuids[0]
) ),
settings={"database_replicated_allow_explicit_uuid": 3},
) )
node2 = cluster.instances["node2"] node2 = cluster.instances["node2"]

View File

@ -5,6 +5,8 @@
<allow_experimental_alter_materialized_view_structure>1</allow_experimental_alter_materialized_view_structure> <allow_experimental_alter_materialized_view_structure>1</allow_experimental_alter_materialized_view_structure>
<allow_experimental_object_type>0</allow_experimental_object_type> <allow_experimental_object_type>0</allow_experimental_object_type>
<allow_suspicious_codecs>0</allow_suspicious_codecs> <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> </default>
</profiles> </profiles>
<users> <users>

View File

@ -5,6 +5,8 @@
<allow_experimental_alter_materialized_view_structure>1</allow_experimental_alter_materialized_view_structure> <allow_experimental_alter_materialized_view_structure>1</allow_experimental_alter_materialized_view_structure>
<allow_experimental_object_type>0</allow_experimental_object_type> <allow_experimental_object_type>0</allow_experimental_object_type>
<allow_suspicious_codecs>0</allow_suspicious_codecs> <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> <throw_on_unsupported_query_inside_transaction>0</throw_on_unsupported_query_inside_transaction>
</default> </default>

View File

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

View File

@ -6,6 +6,8 @@ DROP TABLE IF EXISTS without_deduplication;
DROP TABLE IF EXISTS with_deduplication_mv; DROP TABLE IF EXISTS with_deduplication_mv;
DROP TABLE IF EXISTS without_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) CREATE TABLE with_deduplication(x UInt32)
ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/with_deduplication', 'r1') ORDER BY x; ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/with_deduplication', 'r1') ORDER BY x;
CREATE TABLE without_deduplication(x UInt32) CREATE TABLE without_deduplication(x UInt32)

View File

@ -1,4 +1,4 @@
-- Tags: no-ordinary-database -- Tags: no-ordinary-database, no-parallel
DROP TABLE IF EXISTS test_00609; DROP TABLE IF EXISTS test_00609;
DROP TABLE IF EXISTS test_mv_00609; DROP TABLE IF EXISTS test_mv_00609;
@ -6,6 +6,7 @@ DROP TABLE IF EXISTS test_mv_00609;
create table test_00609 (a Int8) engine=Memory; create table test_00609 (a Int8) engine=Memory;
insert into test_00609 values (1); insert into test_00609 values (1);
set database_replicated_allow_explicit_uuid=3;
set allow_deprecated_syntax_for_merge_tree=1; 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; 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;

View File

@ -7,6 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh
. "$CURDIR"/../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 # there are some issues with Atomic database, let's generate it uniq
# otherwise flaky check will not pass. # otherwise flaky check will not pass.
uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase())") uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase())")

View File

@ -7,6 +7,8 @@ DROP TABLE IF EXISTS rmt1;
DROP TABLE IF EXISTS rmt2; DROP TABLE IF EXISTS rmt2;
DROP TABLE IF EXISTS rmt3; 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; CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/test_01148/{shard}/{database}/{table}', '{replica}') ORDER BY n;
SHOW CREATE TABLE rmt; SHOW CREATE TABLE rmt;
RENAME TABLE rmt TO rmt1; RENAME TABLE rmt TO rmt1;

View File

@ -14,6 +14,8 @@ INSERT INTO src VALUES (3), (4);
SELECT * FROM mv ORDER BY n; SELECT * FROM mv ORDER BY n;
DROP TABLE mv SYNC; DROP TABLE mv SYNC;
SET database_replicated_allow_explicit_uuid=3;
SET show_table_uuid_in_table_create_query_if_not_nil=1; 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; 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; 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;

View File

@ -15,7 +15,7 @@ ${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO us
${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}"
${CLICKHOUSE_CLIENT} -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" ${CLICKHOUSE_CLIENT} -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 80 }" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 36 }"
${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db"
${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}"

View File

@ -0,0 +1,10 @@
2
3
m1
m2
rmt1
rmt2
02858000-1000-4000-8000-000000000
0
CREATE TABLE default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/02858_explicit_uuid_and_zk_path_default/rmt/{shard}\', \'_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192
CREATE TABLE 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,44 @@
#!/usr/bin/env bash
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=error
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_TEST_ZOOKEEPER_PREFIX/rdb', '1', '1')"
db="rdb_$CLICKHOUSE_DATABASE"
fi
$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --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 --distributed_ddl_output_mode=none --database_replicated_allow_explicit_uuid=1 -q "CREATE TABLE $db.m1
UUID '02858000-1000-4000-8000-000000000$(($RANDOM % 10))$(($RANDOM % 10))$(($RANDOM % 10))' (n int) ENGINE=Memory"
$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_explicit_uuid=2 -q "CREATE TABLE $db.m2
UUID '02858000-1000-4000-8000-000000000002' (n int) ENGINE=Memory"
$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_replicated_engine_arguments=0 -q "CREATE TABLE $db.rmt0 (n int)
ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt/{shard}', '_{replica}') ORDER BY n" 2>&1| grep -Fac "database_replicated_allow_replicated_engine_arguments"
$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_replicated_engine_arguments=1 -q "CREATE TABLE $db.rmt1 (n int)
ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt/{shard}', '_{replica}') ORDER BY n"
$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_replicated_engine_arguments=2 -q "CREATE TABLE $db.rmt2 (n int)
ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt/{shard}', '_{replica}') 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) - 3) 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" | sed "s/$db/default/g"
$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt2" | sed "s/$db/default/g"
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS rdb_$CLICKHOUSE_DATABASE"

View File

@ -5,6 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh . "$CUR_DIR"/../shell_config.sh
CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --database_replicated_allow_explicit_uuid 3 --database_replicated_allow_replicated_engine_arguments 3"
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test_exception_replicated SYNC" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test_exception_replicated SYNC"