未验证 提交 790950ea 编写于 作者: T tavplubix 提交者: GitHub

Merge pull request #15934 from ClickHouse/ddl_queries_fixes

Fixes for DDL queries
......@@ -532,7 +532,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)
......@@ -834,7 +834,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;
......@@ -843,14 +843,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)
......@@ -860,4 +865,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();
};
......
......@@ -47,7 +47,7 @@ BlockIO InterpreterDropQuery::execute()
if (!drop.table.empty())
{
if (!drop.is_dictionary)
return executeToTable({drop.database, drop.table, drop.uuid}, drop);
return executeToTable(drop);
else
return executeToDictionary(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
}
......@@ -58,29 +58,26 @@ BlockIO InterpreterDropQuery::execute()
}
BlockIO InterpreterDropQuery::executeToTable(
const StorageID & table_id_,
const ASTDropQuery & query)
BlockIO InterpreterDropQuery::executeToTable(const ASTDropQuery & query)
{
if (query.temporary || table_id_.database_name.empty())
/// NOTE: it does not contain UUID, we will resolve it with locked DDLGuard
auto table_id = StorageID(query);
if (query.temporary || table_id.database_name.empty())
{
if (context.tryResolveStorageID(table_id_, Context::ResolveExternal))
return executeToTemporaryTable(table_id_.getTableName(), query.kind);
if (context.tryResolveStorageID(table_id, Context::ResolveExternal))
return executeToTemporaryTable(table_id.getTableName(), query.kind);
else
table_id.database_name = context.getCurrentDatabase();
}
if (query.temporary)
{
if (query.if_exists)
return {};
throw Exception("Temporary table " + backQuoteIfNeed(table_id_.table_name) + " doesn't exist",
throw Exception("Temporary table " + backQuoteIfNeed(table_id.table_name) + " doesn't exist",
ErrorCodes::UNKNOWN_TABLE);
}
auto table_id = query.if_exists ? context.tryResolveStorageID(table_id_, Context::ResolveOrdinary)
: context.resolveStorageID(table_id_, Context::ResolveOrdinary);
if (!table_id)
return {};
auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr);
/// If table was already dropped by anyone, an exception will be thrown
......@@ -92,6 +89,9 @@ BlockIO InterpreterDropQuery::executeToTable(
if (query_ptr->as<ASTDropQuery &>().is_view && !table->isView())
throw Exception("Table " + table_id.getNameForLogs() + " is not a View", ErrorCodes::LOGICAL_ERROR);
/// Now get UUID, so we can wait for table data to be finally dropped
table_id.uuid = database->tryGetTableUUID(table_id.table_name);
if (query.kind == ASTDropQuery::Kind::Detach)
{
context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id);
......@@ -251,6 +251,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS
ASTDropQuery query;
query.kind = kind;
query.if_exists = true;
query.database = database_name;
query.no_delay = no_delay;
......@@ -259,7 +260,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS
/// Reset reference counter of the StoragePtr to allow synchronous drop.
iterator->reset();
query.table = iterator->name();
executeToTable({query.database, query.table}, query);
executeToTable(query);
}
}
......
......@@ -31,7 +31,7 @@ private:
BlockIO executeToDatabase(const String & database_name, ASTDropQuery::Kind kind, bool if_exists, bool no_delay);
BlockIO executeToTable(const StorageID & table_id, const ASTDropQuery & query);
BlockIO executeToTable(const ASTDropQuery & query);
BlockIO executeToDictionary(const String & database_name, const String & dictionary_name, ASTDropQuery::Kind kind, bool if_exists, bool is_temporary, bool no_ddl_lock);
......
#!/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 "DETACH DATABASE IF EXISTS test_01150"
$CLICKHOUSE_CLIENT --query "ATTACH DATABASE IF NOT EXISTS test_01150"
$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01150";
#!/usr/bin/env bash
set -e
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
function thread_create {
while true; do
$CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2"
sleep 0.0$RANDOM
done
}
function thread_drop {
while true; do
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1"
sleep 0.0$RANDOM
done
}
function thread_rename {
while true; do
$CLICKHOUSE_CLIENT --query "RENAME TABLE $1 TO $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)'
sleep 0.0$RANDOM
done
}
function thread_select {
while true; do
$CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)'
sleep 0.0$RANDOM
done
}
function thread_insert {
while true; do
$CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT rand64(1), [toString(rand64(2))] FROM numbers($2)" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)'
sleep 0.0$RANDOM
done
}
function thread_insert_select {
while true; do
$CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)'
sleep 0.0$RANDOM
done
}
export -f thread_create
export -f thread_drop
export -f thread_rename
export -f thread_select
export -f thread_insert
export -f thread_insert_select
# Do randomized queries and expect nothing extraordinary happens.
function test_with_engine {
echo "Testing $1"
timeout 10 bash -c "thread_create t1 $1" &
timeout 10 bash -c "thread_create t2 $1" &
timeout 10 bash -c 'thread_drop t1' &
timeout 10 bash -c 'thread_drop t2' &
timeout 10 bash -c 'thread_rename t1 t2' &
timeout 10 bash -c 'thread_rename t2 t1' &
timeout 10 bash -c 'thread_select t1' &
timeout 10 bash -c 'thread_select t2' &
timeout 10 bash -c 'thread_insert t1 5' &
timeout 10 bash -c 'thread_insert t2 10' &
timeout 10 bash -c 'thread_insert_select t1 t2' &
timeout 10 bash -c 'thread_insert_select t2 t1' &
wait
echo "Done $1"
}
#test_with_engine TinyLog
#test_with_engine StripeLog
#test_with_engine Log
test_with_engine Memory
......@@ -12,21 +12,17 @@ function drop_database()
function drop_table()
{
${CLICKHOUSE_CLIENT} -nm <<EOL
DROP TABLE IF EXISTS db_01516.data3;
DROP TABLE IF EXISTS db_01516.data1;
DROP TABLE IF EXISTS db_01516.data2;
EOL
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS db_01516.data3;" 2>&1 | grep -F "Code: " | grep -Fv "is currently dropped or renamed"
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS db_01516.data1;" 2>&1 | grep -F "Code: " | grep -Fv "is currently dropped or renamed"
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS db_01516.data2;" 2>&1 | grep -F "Code: " | grep -Fv "is currently dropped or renamed"
}
function create()
{
${CLICKHOUSE_CLIENT} -nm <<EOL
CREATE DATABASE IF NOT EXISTS db_01516;
CREATE TABLE IF NOT EXISTS db_01516.data1 Engine=MergeTree() ORDER BY number AS SELECT * FROM numbers(1);
CREATE TABLE IF NOT EXISTS db_01516.data2 Engine=MergeTree() ORDER BY number AS SELECT * FROM numbers(1);
CREATE TABLE IF NOT EXISTS db_01516.data3 Engine=MergeTree() ORDER BY number AS SELECT * FROM numbers(1);
EOL
${CLICKHOUSE_CLIENT} -q "CREATE DATABASE IF NOT EXISTS db_01516;"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE IF NOT EXISTS db_01516.data1 Engine=MergeTree() ORDER BY number AS SELECT * FROM numbers(1);" 2>&1 | grep -F "Code: " | grep -Fv "is currently dropped or renamed"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE IF NOT EXISTS db_01516.data2 Engine=MergeTree() ORDER BY number AS SELECT * FROM numbers(1);" 2>&1 | grep -F "Code: " | grep -Fv "is currently dropped or renamed"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE IF NOT EXISTS db_01516.data3 Engine=MergeTree() ORDER BY number AS SELECT * FROM numbers(1);" 2>&1 | grep -F "Code: " | grep -Fv "is currently dropped or renamed"
}
for _ in {1..100}; do
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册