mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
fix deadlock with DDLGuard
This commit is contained in:
parent
b3fc6d9b33
commit
cb8d132cca
@ -530,7 +530,7 @@ std::unique_ptr<DDLGuard> DatabaseCatalog::getDDLGuard(const String & database,
|
||||
std::unique_lock lock(ddl_guards_mutex);
|
||||
auto db_guard_iter = ddl_guards.try_emplace(database).first;
|
||||
DatabaseGuard & db_guard = db_guard_iter->second;
|
||||
return std::make_unique<DDLGuard>(db_guard.first, db_guard.second, std::move(lock), table);
|
||||
return std::make_unique<DDLGuard>(db_guard.first, db_guard.second, std::move(lock), table, database);
|
||||
}
|
||||
|
||||
std::unique_lock<std::shared_mutex> DatabaseCatalog::getExclusiveDDLGuardForDatabase(const String & database)
|
||||
@ -832,7 +832,7 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid)
|
||||
}
|
||||
|
||||
|
||||
DDLGuard::DDLGuard(Map & map_, std::shared_mutex & db_mutex_, std::unique_lock<std::mutex> guards_lock_, const String & elem)
|
||||
DDLGuard::DDLGuard(Map & map_, std::shared_mutex & db_mutex_, std::unique_lock<std::mutex> guards_lock_, const String & elem, const String & database_name)
|
||||
: map(map_), db_mutex(db_mutex_), guards_lock(std::move(guards_lock_))
|
||||
{
|
||||
it = map.emplace(elem, Entry{std::make_unique<std::mutex>(), 0}).first;
|
||||
@ -841,14 +841,19 @@ DDLGuard::DDLGuard(Map & map_, std::shared_mutex & db_mutex_, std::unique_lock<s
|
||||
table_lock = std::unique_lock(*it->second.mutex);
|
||||
bool is_database = elem.empty();
|
||||
if (!is_database)
|
||||
db_mutex.lock_shared();
|
||||
{
|
||||
|
||||
bool locked_database_for_read = db_mutex.try_lock_shared();
|
||||
if (!locked_database_for_read)
|
||||
{
|
||||
removeTableLock();
|
||||
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} is currently dropped or renamed", database_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
DDLGuard::~DDLGuard()
|
||||
void DDLGuard::removeTableLock()
|
||||
{
|
||||
bool is_database = it->first.empty();
|
||||
if (!is_database)
|
||||
db_mutex.unlock_shared();
|
||||
guards_lock.lock();
|
||||
--it->second.counter;
|
||||
if (!it->second.counter)
|
||||
@ -858,4 +863,12 @@ DDLGuard::~DDLGuard()
|
||||
}
|
||||
}
|
||||
|
||||
DDLGuard::~DDLGuard()
|
||||
{
|
||||
bool is_database = it->first.empty();
|
||||
if (!is_database)
|
||||
db_mutex.unlock_shared();
|
||||
removeTableLock();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -51,7 +51,7 @@ public:
|
||||
/// NOTE: using std::map here (and not std::unordered_map) to avoid iterator invalidation on insertion.
|
||||
using Map = std::map<String, Entry>;
|
||||
|
||||
DDLGuard(Map & map_, std::shared_mutex & db_mutex_, std::unique_lock<std::mutex> guards_lock_, const String & elem);
|
||||
DDLGuard(Map & map_, std::shared_mutex & db_mutex_, std::unique_lock<std::mutex> guards_lock_, const String & elem, const String & database_name);
|
||||
~DDLGuard();
|
||||
|
||||
private:
|
||||
@ -60,6 +60,8 @@ private:
|
||||
Map::iterator it;
|
||||
std::unique_lock<std::mutex> guards_lock;
|
||||
std::unique_lock<std::mutex> table_lock;
|
||||
|
||||
void removeTableLock();
|
||||
};
|
||||
|
||||
|
||||
|
@ -247,6 +247,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS
|
||||
|
||||
ASTDropQuery query;
|
||||
query.kind = kind;
|
||||
query.if_exists = true;
|
||||
query.database = database_name;
|
||||
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
|
43
tests/queries/0_stateless/01150_ddl_guard_rwr.sh
Executable file
43
tests/queries/0_stateless/01150_ddl_guard_rwr.sh
Executable file
@ -0,0 +1,43 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS test_01150"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01150"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE test_01150.t1 (x UInt64, s Array(Nullable(String))) ENGINE = Memory"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE test_01150.t2 (x UInt64, s Array(Nullable(String))) ENGINE = Memory"
|
||||
|
||||
function thread_detach_attach {
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT --query "DETACH DATABASE test_01150" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (219)'
|
||||
sleep 0.0$RANDOM
|
||||
$CLICKHOUSE_CLIENT --query "ATTACH DATABASE test_01150" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (82)'
|
||||
sleep 0.0$RANDOM
|
||||
done
|
||||
}
|
||||
|
||||
function thread_rename {
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT --query "RENAME TABLE test_01150.t1 TO test_01150.t2_tmp, test_01150.t2 TO test_01150.t1, test_01150.t2_tmp TO test_01150.t2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (81|60|57|521)'
|
||||
sleep 0.0$RANDOM
|
||||
$CLICKHOUSE_CLIENT --query "RENAME TABLE test_01150.t2 TO test_01150.t1, test_01150.t2_tmp TO test_01150.t2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (81|60|57|521)'
|
||||
sleep 0.0$RANDOM
|
||||
$CLICKHOUSE_CLIENT --query "RENAME TABLE test_01150.t2_tmp TO test_01150.t2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (81|60|57|521)'
|
||||
sleep 0.0$RANDOM
|
||||
done
|
||||
}
|
||||
|
||||
export -f thread_detach_attach
|
||||
export -f thread_rename
|
||||
|
||||
timeout 20 bash -c "thread_detach_attach" &
|
||||
timeout 20 bash -c 'thread_rename' &
|
||||
wait
|
||||
sleep 1
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "ATTACH DATABASE IF NOT EXISTS test_01150"
|
||||
$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01150";
|
Loading…
Reference in New Issue
Block a user