fix tests

This commit is contained in:
Alexander Tokmakov 2022-07-07 23:59:15 +02:00
parent 388872550a
commit 63fc95903c
8 changed files with 13 additions and 10 deletions

View File

@ -1517,6 +1517,11 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, Cont
if (cluster->maybeCrossReplication())
{
auto on_cluster_version = local_context->getSettingsRef().distributed_ddl_entry_format_version;
if (DDLLogEntry::NORMALIZE_CREATE_ON_INITIATOR_VERSION <= on_cluster_version)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Value {} of setting distributed_ddl_entry_format_version "
"is incompatible with cross-replication", on_cluster_version);
/// Check that {uuid} macro is not used in zookeeper_path for ReplicatedMergeTree.
/// Otherwise replicas will generate different paths.
if (!create.storage)

View File

@ -6,7 +6,6 @@
<database_replicated_initial_query_timeout_sec>120</database_replicated_initial_query_timeout_sec>
<distributed_ddl_task_timeout>120</distributed_ddl_task_timeout>
<database_replicated_always_detach_permanently>1</database_replicated_always_detach_permanently>
<distributed_ddl_entry_format_version>2</distributed_ddl_entry_format_version>
</default>
</profiles>
</clickhouse>

View File

@ -50,7 +50,7 @@ function create_table()
database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1")
if [ -z "$database" ]; then return; fi
$CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \
"create table $database.rmt_$RANDOM (n int) engine=ReplicatedMergeTree order by tuple() -- suppress CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" \
"create table $database.rmt_$RANDOM (n int) engine=ReplicatedMergeTree order by tuple() -- suppress $CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" \
2>&1| grep -Fa "Exception: " | grep -Fv "Macro 'uuid' and empty arguments" | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE
sleep 0.$RANDOM
done

View File

@ -8,6 +8,8 @@ DROP TABLE IF EXISTS demo_loan_01568_dist;
CREATE DATABASE shard_0;
CREATE DATABASE shard_1;
CREATE TABLE demo_loan_01568 ON CLUSTER test_cluster_two_shards_different_databases ( `id` Int64 COMMENT 'id', `date_stat` Date COMMENT 'date of stat', `customer_no` String COMMENT 'customer no', `loan_principal` Float64 COMMENT 'loan principal' ) ENGINE=ReplacingMergeTree() ORDER BY id PARTITION BY toYYYYMM(date_stat); -- { serverError 48 }
SET distributed_ddl_entry_format_version = 2;
CREATE TABLE demo_loan_01568 ON CLUSTER test_cluster_two_shards_different_databases ( `id` Int64 COMMENT 'id', `date_stat` Date COMMENT 'date of stat', `customer_no` String COMMENT 'customer no', `loan_principal` Float64 COMMENT 'loan principal' ) ENGINE=ReplacingMergeTree() ORDER BY id PARTITION BY toYYYYMM(date_stat); -- { serverError 371 }
SET distributed_ddl_output_mode='throw';
CREATE TABLE shard_0.demo_loan_01568 ON CLUSTER test_cluster_two_shards_different_databases ( `id` Int64 COMMENT 'id', `date_stat` Date COMMENT 'date of stat', `customer_no` String COMMENT 'customer no', `loan_principal` Float64 COMMENT 'loan principal' ) ENGINE=ReplacingMergeTree() ORDER BY id PARTITION BY toYYYYMM(date_stat);

View File

@ -38,7 +38,6 @@ LOG_COMMENT="${CLICKHOUSE_LOG_COMMENT}_$RAND_COMMENT"
CLICKHOUSE_CLIENT_WITH_SETTINGS=${CLICKHOUSE_CLIENT/--log_comment ${CLICKHOUSE_LOG_COMMENT}/--log_comment ${LOG_COMMENT}}
CLICKHOUSE_CLIENT_WITH_SETTINGS+=" --output_format_parallel_formatting=0 "
CLICKHOUSE_CLIENT_WITH_SETTINGS+=" --distributed_ddl_entry_format_version=2 "
CLIENT=${CLICKHOUSE_CLIENT_WITH_SETTINGS}
CLIENT+=" --distributed_ddl_task_timeout=$TIMEOUT "

View File

@ -11,8 +11,8 @@ ${CLICKHOUSE_CLIENT} -q "CREATE USER user_${CLICKHOUSE_DATABASE} settings databa
${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO user_${CLICKHOUSE_DATABASE}"
${CLICKHOUSE_CLIENT} --allow_experimental_database_replicated=1 --query "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_memory (x UInt32) engine = Memory;"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "set distributed_ddl_entry_format_version=2; CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" 2>&1 | grep -o "Only tables with a Replicated engine"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -n --query "set distributed_ddl_entry_format_version=2; CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "set distributed_ddl_entry_format_version=2; CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt (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_mt (x UInt32) engine = MergeTree order by x;" 2>&1 | grep -o "Only tables with a Replicated engine"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt (x UInt32) engine = ReplicatedMergeTree order by x;"
${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db"
${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}"

View File

@ -1,3 +1,3 @@
1 2 3 4
5 6 7 8
CREATE TABLE default.t_l5ydey\n(\n `c_qv5rv` Int32,\n `c_rutjs4` Int32,\n `c_wmj` Int32,\n `c_m3` String\n)\nENGINE = Distributed(\'test_cluster\', \'default\', \'local_t_l5ydey\', rand())
CREATE TABLE default.t_l5ydey\n(\n `c_qv5rv` Int32,\n `c_rutjs4` Int32,\n `c_wmj` Int32,\n `c_m3` String\n)\nENGINE = Distributed(\'test_shard_localhost\', \'default\', \'local_t_l5ydey\', rand())

View File

@ -1,8 +1,6 @@
-- Tags: no-replicated-database
-- Tag no-replicated-database: ON CLUSTER is not allowed
set distributed_ddl_entry_format_version=3;
create table local_t_l5ydey on cluster test_shard_localhost (
c_qv5rv INTEGER ,
c_rutjs4 INTEGER ,
@ -12,7 +10,7 @@ create table local_t_l5ydey on cluster test_shard_localhost (
) engine=ReplicatedMergeTree('/clickhouse/tables/test_{database}/{shard}/local_t_l5ydey', '{replica}');
create table t_l5ydey on cluster test_shard_localhost as local_t_l5ydey
engine=Distributed('test_cluster', currentDatabase(),'local_t_l5ydey', rand());
engine=Distributed('test_shard_localhost', currentDatabase(),'local_t_l5ydey', rand());
insert into local_t_l5ydey values (1, 2, 3, '4');
insert into t_l5ydey values (5, 6, 7, '8');