fix deadlock with DDLGuard

This commit is contained in:
Alexander Tokmakov 2020-10-13 18:00:36 +03:00
parent b3fc6d9b33
commit cb8d132cca
5 changed files with 67 additions and 8 deletions

View File

@ -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();
}
}

View File

@ -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();
};

View File

@ -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())
{

View 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";