ignore cluster if equals to db name

This commit is contained in:
Alexander Tokmakov 2022-07-06 19:54:05 +02:00
parent 3a94ae8f67
commit 30e9104b6e
9 changed files with 87 additions and 47 deletions

View File

@ -66,7 +66,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
{
BlockIO res;
if (!alter.cluster.empty())
if (!alter.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext()))
{
DDLQueryOnClusterParams params;
params.access_to_check = getRequiredAccess();

View File

@ -1563,7 +1563,7 @@ BlockIO InterpreterCreateQuery::execute()
auto & create = query_ptr->as<ASTCreateQuery &>();
bool is_create_database = create.database && !create.table;
if (!create.cluster.empty())
if (!create.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext()))
{
auto on_cluster_version = getContext()->getSettingsRef().distributed_ddl_entry_format_version;
if (is_create_database || on_cluster_version < DDLLogEntry::NORMALIZE_CREATE_ON_INITIATOR_VERSION)

View File

@ -54,7 +54,7 @@ InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, ContextMut
BlockIO InterpreterDropQuery::execute()
{
auto & drop = query_ptr->as<ASTDropQuery &>();
if (!drop.cluster.empty())
if (!drop.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext()))
{
DDLQueryOnClusterParams params;
params.access_to_check = getRequiredAccessForDDLOnCluster();

View File

@ -538,4 +538,26 @@ Strings DDLQueryStatusSource::getNewAndUpdate(const Strings & current_list_of_fi
}
bool maybeRemoveOnCluster(const ASTPtr & query_ptr, ContextPtr context)
{
const auto * query = dynamic_cast<const ASTQueryWithTableAndOutput *>(query_ptr.get());
if (!query || !query->table)
return false;
String database_name = query->getDatabase();
if (database_name.empty())
database_name = context->getCurrentDatabase();
auto * query_on_cluster = dynamic_cast<ASTQueryWithOnCluster *>(query_ptr.get());
if (database_name != query_on_cluster->cluster)
return false;
auto db = DatabaseCatalog::instance().tryGetDatabase(database_name);
if (!db || db->getEngineName() != "Replicated")
return false;
query_on_cluster->cluster.clear();
return true;
}
}

View File

@ -45,4 +45,6 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, c
BlockIO getDistributedDDLStatus(
const String & node_path, const DDLLogEntry & entry, ContextPtr context, const std::optional<Strings> & hosts_to_wait = {});
bool maybeRemoveOnCluster(const ASTPtr & query_ptr, ContextPtr context);
}

View File

@ -58,7 +58,7 @@ function thread6()
$CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS alter_table_$REPLICA;
CREATE TABLE alter_table_$REPLICA (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r_$REPLICA') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0;";
sleep 0.$RANDOM;
done
done
}

View File

@ -8,62 +8,74 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
function create_db()
{
SHARD=$(($RANDOM % 2))
REPLICA=$(($RANDOM % 2))
SUFFIX=$(($RANDOM % 16))
# Multiple database replicas on one server are actually not supported (until we have namespaces).
# So CREATE TABLE queries will fail on all replicas except one. But it's still makes sense for a stress test.
$CLICKHOUSE_CLIENT --allow_experimental_database_replicated=1 --query \
"create database if not exists ${CLICKHOUSE_DATABASE}_repl_$SUFFIX engine=Replicated('/test/01111/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '$SHARD', '$REPLICA')" \
2>&1| grep -Fa "Exception: " | grep -Fv "REPLICA_IS_ALREADY_EXIST" | grep -Fiv "Will not try to start it up" | grep -Fv "Coordination::Exception"
sleep 0.$RANDOM
while true; do
SHARD=$(($RANDOM % 2))
REPLICA=$(($RANDOM % 2))
SUFFIX=$(($RANDOM % 16))
# Multiple database replicas on one server are actually not supported (until we have namespaces).
# So CREATE TABLE queries will fail on all replicas except one. But it's still makes sense for a stress test.
$CLICKHOUSE_CLIENT --allow_experimental_database_replicated=1 --query \
"create database if not exists ${CLICKHOUSE_DATABASE}_repl_$SUFFIX engine=Replicated('/test/01111/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '$SHARD', '$REPLICA')" \
2>&1| grep -Fa "Exception: " | grep -Fv "REPLICA_IS_ALREADY_EXIST" | grep -Fiv "Will not try to start it up" | grep -Fv "Coordination::Exception"
sleep 0.$RANDOM
done
}
function drop_db()
{
database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1")
if [[ "$database" == "$CLICKHOUSE_DATABASE" ]]; then return; fi
if [ -z "$database" ]; then return; fi
$CLICKHOUSE_CLIENT -n --query \
"drop database if exists $database" 2>&1| grep -Fa "Exception: "
sleep 0.$RANDOM
while true; do
database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1")
if [[ "$database" == "$CLICKHOUSE_DATABASE" ]]; then return; fi
if [ -z "$database" ]; then return; fi
$CLICKHOUSE_CLIENT -n --query \
"drop database if exists $database" 2>&1| grep -Fa "Exception: "
sleep 0.$RANDOM
done
}
function sync_db()
{
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 --receive_timeout=1 -q \
"system sync database replica $database" 2>&1| grep -Fa "Exception: " | grep -Fv TIMEOUT_EXCEEDED | grep -Fv "only with Replicated engine" | grep -Fv UNKNOWN_DATABASE
sleep 0.$RANDOM
while true; do
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 --receive_timeout=1 -q \
"system sync database replica $database" 2>&1| grep -Fa "Exception: " | grep -Fv TIMEOUT_EXCEEDED | grep -Fv "only with Replicated engine" | grep -Fv UNKNOWN_DATABASE
sleep 0.$RANDOM
done
}
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" \
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
while true; do
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" \
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
}
function alter_table()
{
table=$($CLICKHOUSE_CLIENT -q "select database || '.' || name from system.tables where database like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1")
if [ -z "$table" ]; then return; fi
$CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \
"alter table $table update n = n + (select max(n) from merge(REGEXP('${CLICKHOUSE_DATABASE}.*'), '.*')) where 1 settings allow_nondeterministic_mutations=1" \
2>&1| grep -Fa "Exception: " | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE | grep -Fv UNKNOWN_TABLE | grep -Fv TABLE_IS_READ_ONLY
sleep 0.$RANDOM
while true; do
table=$($CLICKHOUSE_CLIENT -q "select database || '.' || name from system.tables where database like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1")
if [ -z "$table" ]; then return; fi
$CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \
"alter table $table on cluster $database update n = n + (select max(n) from merge(REGEXP('${CLICKHOUSE_DATABASE}.*'), '.*')) where 1 settings allow_nondeterministic_mutations=1" \
2>&1| grep -Fa "Exception: " | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE | grep -Fv UNKNOWN_TABLE | grep -Fv TABLE_IS_READ_ONLY
sleep 0.$RANDOM
done
}
function insert()
{
table=$($CLICKHOUSE_CLIENT -q "select database || '.' || name from system.tables where database like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1")
if [ -z "$table" ]; then return; fi
$CLICKHOUSE_CLIENT -q \
"insert into $table values ($RANDOM)" 2>&1| grep -Fa "Exception: " | grep -Fv UNKNOWN_DATABASE | grep -Fv UNKNOWN_TABLE | grep -Fv TABLE_IS_READ_ONLY
while true; do
table=$($CLICKHOUSE_CLIENT -q "select database || '.' || name from system.tables where database like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1")
if [ -z "$table" ]; then return; fi
$CLICKHOUSE_CLIENT -q \
"insert into $table values ($RANDOM)" 2>&1| grep -Fa "Exception: " | grep -Fv UNKNOWN_DATABASE | grep -Fv UNKNOWN_TABLE | grep -Fv TABLE_IS_READ_ONLY
done
}
@ -77,14 +89,14 @@ export -f insert
TIMEOUT=30
clickhouse_client_loop_timeout $TIMEOUT create_db &
clickhouse_client_loop_timeout $TIMEOUT sync_db &
clickhouse_client_loop_timeout $TIMEOUT create_table &
clickhouse_client_loop_timeout $TIMEOUT alter_table &
clickhouse_client_loop_timeout $TIMEOUT insert &
timeout $TIMEOUT bash -c create_db &
timeout $TIMEOUT bash -c sync_db &
timeout $TIMEOUT bash -c create_table &
timeout $TIMEOUT bash -c alter_table &
timeout $TIMEOUT bash -c insert &
sleep 1 # give other queries a head start
clickhouse_client_loop_timeout $TIMEOUT drop_db &
timeout $TIMEOUT bash -c drop_db &
wait

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_cluster\', \'default\', \'local_t_l5ydey\', rand())

View File

@ -1,3 +1,7 @@
-- 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 ,