This commit is contained in:
Alexander Tokmakov 2024-07-04 20:49:10 +02:00
parent aa9591419d
commit 613636bb1b
8 changed files with 24 additions and 16 deletions

View File

@ -63,6 +63,8 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."},
{"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."},
{"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"},
{"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.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"},
{"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"},

View File

@ -786,7 +786,8 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_
return;
}
throw Exception(ErrorCodes::INCORRECT_QUERY,
if (query_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 0)
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");

View File

@ -1062,15 +1062,15 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
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)
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)
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 ({}) "
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);
}
}

View File

@ -303,7 +303,7 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
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 ({}, {}) "
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;

View File

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

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 "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}" -n --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}" -n --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}" -n --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} -q "DROP USER user_${CLICKHOUSE_DATABASE}"

View File

@ -6,5 +6,5 @@ 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.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/02858_explicit_uuid_and_zk_path_default/rmt/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

@ -1,33 +1,36 @@
#!/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 --database_replicated_allow_explicit_uuid=0 -q "CREATE TABLE $db.m0
$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 --database_replicated_allow_explicit_uuid=1 -q "CREATE TABLE $db.m1
$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --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
$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 --database_replicated_allow_replicated_engine_arguments=0 -q "CREATE TABLE $db.rmt0 (n int)
ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') 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=0 -q "CREATE TABLE $db.rmt0 (n int)
ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt', '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_TEST_ZOOKEEPER_PREFIX', '1') ORDER BY n"
$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', '1') ORDER BY n"
$CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=2 -q "CREATE TABLE $db.rmt2 (n int)
ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') 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', '1') ORDER BY n"
$CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database='$db' ORDER BY name"