mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-09 17:14:47 +00:00
don't allow explicit uuids and rmt args in replicated db
This commit is contained in:
parent
bdc7157670
commit
4e350ff44a
@ -731,7 +731,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, 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) \
|
||||||
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) \
|
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) \
|
||||||
|
@ -774,18 +774,6 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_
|
|||||||
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;
|
||||||
|
|
||||||
|
@ -1055,6 +1055,26 @@ 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)
|
||||||
|
{
|
||||||
|
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 (is_replicated_database_internal && !internal)
|
||||||
{
|
{
|
||||||
if (create.uuid == UUIDHelpers::Nil)
|
if (create.uuid == UUIDHelpers::Nil)
|
||||||
|
@ -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);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -9,6 +9,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>
|
||||||
@ -185,7 +186,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)
|
||||||
@ -202,11 +203,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
|
||||||
@ -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,
|
/// 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;
|
||||||
@ -247,11 +248,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
|
||||||
@ -268,9 +269,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)
|
||||||
@ -284,6 +300,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_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);
|
expand_macro(ast_zk_path, ast_replica_name);
|
||||||
}
|
}
|
||||||
else if (is_extended_storage_def
|
else if (is_extended_storage_def
|
||||||
@ -293,7 +318,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;
|
||||||
@ -318,7 +342,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 {};
|
||||||
@ -342,7 +366,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)
|
||||||
|
@ -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>
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
<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>
|
||||||
</default>
|
</default>
|
||||||
</profiles>
|
</profiles>
|
||||||
<users>
|
<users>
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
<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>
|
||||||
|
|
||||||
<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>
|
||||||
|
@ -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
|
41
tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh
Executable file
41
tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh
Executable 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"
|
Loading…
Reference in New Issue
Block a user