mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
ignore cluster if equals to db name
This commit is contained in:
parent
3a94ae8f67
commit
30e9104b6e
@ -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();
|
||||
|
@ -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)
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -8,6 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
function create_db()
|
||||
{
|
||||
while true; do
|
||||
SHARD=$(($RANDOM % 2))
|
||||
REPLICA=$(($RANDOM % 2))
|
||||
SUFFIX=$(($RANDOM % 16))
|
||||
@ -17,53 +18,64 @@ function create_db()
|
||||
"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()
|
||||
{
|
||||
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()
|
||||
{
|
||||
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()
|
||||
{
|
||||
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()
|
||||
{
|
||||
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 update n = n + (select max(n) from merge(REGEXP('${CLICKHOUSE_DATABASE}.*'), '.*')) where 1 settings allow_nondeterministic_mutations=1" \
|
||||
"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()
|
||||
{
|
||||
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
|
||||
|
||||
|
@ -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 ,
|
||||
|
Loading…
Reference in New Issue
Block a user