Merge pull request #39263 from ClickHouse/fix_assertion_in_transactions

Fix assertion in transactions
This commit is contained in:
Alexander Tokmakov 2022-07-18 11:43:50 +03:00 committed by GitHub
commit 611da87217
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 20 additions and 23 deletions

View File

@ -12,16 +12,16 @@ namespace ErrorCodes
{ {
extern const int INVALID_TRANSACTION; extern const int INVALID_TRANSACTION;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
} }
static TableLockHolder getLockForOrdinary(const StoragePtr & storage) static void checkNotOrdinaryDatabase(const StoragePtr & storage)
{ {
if (storage->getStorageID().uuid != UUIDHelpers::Nil) if (storage->getStorageID().uuid != UUIDHelpers::Nil)
return {}; return;
/// Maybe we should just throw an exception and do not support Ordinary database? throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table {} belongs to database with Ordinary engine. "
auto default_timeout = std::chrono::milliseconds(10 * 1000); "This engine is deprecated and is not supported in transactions.", storage->getStorageID().getNameForLogs());
return storage->lockForShare(RWLockImpl::NO_QUERY, default_timeout);
} }
MergeTreeTransaction::MergeTreeTransaction(CSN snapshot_, LocalTID local_tid_, UUID host_id, std::list<CSN>::iterator snapshot_it_) MergeTreeTransaction::MergeTreeTransaction(CSN snapshot_, LocalTID local_tid_, UUID host_id, std::list<CSN>::iterator snapshot_it_)
@ -131,18 +131,16 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage
void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPartPtr & new_part) void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPartPtr & new_part)
{ {
auto maybe_lock = getLockForOrdinary(storage); checkNotOrdinaryDatabase(storage);
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
checkIsNotCancelled(); checkIsNotCancelled();
storages.insert(storage); storages.insert(storage);
if (maybe_lock)
table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock));
creating_parts.push_back(new_part); creating_parts.push_back(new_part);
} }
void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, const TransactionInfoContext & context) void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, const TransactionInfoContext & context)
{ {
auto maybe_lock = getLockForOrdinary(storage); checkNotOrdinaryDatabase(storage);
{ {
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
@ -151,8 +149,6 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP
part_to_remove->version.lockRemovalTID(tid, context); part_to_remove->version.lockRemovalTID(tid, context);
NOEXCEPT_SCOPE({ NOEXCEPT_SCOPE({
storages.insert(storage); storages.insert(storage);
if (maybe_lock)
table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock));
removing_parts.push_back(part_to_remove); removing_parts.push_back(part_to_remove);
}); });
} }
@ -162,12 +158,10 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP
void MergeTreeTransaction::addMutation(const StoragePtr & table, const String & mutation_id) void MergeTreeTransaction::addMutation(const StoragePtr & table, const String & mutation_id)
{ {
auto maybe_lock = getLockForOrdinary(table); checkNotOrdinaryDatabase(table);
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
checkIsNotCancelled(); checkIsNotCancelled();
storages.insert(table); storages.insert(table);
if (maybe_lock)
table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock));
mutations.emplace_back(table, mutation_id); mutations.emplace_back(table, mutation_id);
} }

View File

@ -76,7 +76,6 @@ private:
/// Lists of changes made by transaction /// Lists of changes made by transaction
std::unordered_set<StoragePtr> storages TSA_GUARDED_BY(mutex); std::unordered_set<StoragePtr> storages TSA_GUARDED_BY(mutex);
std::vector<TableLockHolder> table_read_locks_for_ordinary_db TSA_GUARDED_BY(mutex);
DataPartsVector creating_parts TSA_GUARDED_BY(mutex); DataPartsVector creating_parts TSA_GUARDED_BY(mutex);
DataPartsVector removing_parts TSA_GUARDED_BY(mutex); DataPartsVector removing_parts TSA_GUARDED_BY(mutex);
using RunningMutationsList = std::vector<std::pair<StoragePtr, String>>; using RunningMutationsList = std::vector<std::pair<StoragePtr, String>>;

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: long # Tags: long, no-ordinary-database
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: long, no-fasttest, no-replicated-database # Tags: long, no-fasttest, no-replicated-database, no-ordinary-database
# Looks like server does not listen https port in fasttest # Looks like server does not listen https port in fasttest
# FIXME Replicated database executes ALTERs in separate context, so transaction info is lost # FIXME Replicated database executes ALTERs in separate context, so transaction info is lost

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: no-fasttest, no-replicated-database # Tags: no-fasttest, no-replicated-database, no-ordinary-database
# Looks like server does not listen https port in fasttest # Looks like server does not listen https port in fasttest
# FIXME Replicated database executes ALTERs in separate context, so transaction info is lost # FIXME Replicated database executes ALTERs in separate context, so transaction info is lost

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: long, no-replicated-database # Tags: long, no-replicated-database, no-ordinary-database
# shellcheck disable=SC2015 # shellcheck disable=SC2015

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: no-fasttest, no-replicated-database # Tags: no-fasttest, no-replicated-database, no-ordinary-database
# Looks like server does not listen https port in fasttest # Looks like server does not listen https port in fasttest
# FIXME Replicated database executes ALTERs in separate context, so transaction info is lost # FIXME Replicated database executes ALTERs in separate context, so transaction info is lost

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: long, no-parallel # Tags: long, no-parallel, no-ordinary-database
# Test is too heavy, avoid parallel run in Flaky Check # Test is too heavy, avoid parallel run in Flaky Check
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)

View File

@ -1,4 +1,4 @@
-- Tags: no-s3-storage, no-tsan -- Tags: no-s3-storage, no-tsan, no-ordinary-database
-- FIXME this test fails with S3 due to a bug in DiskCacheWrapper -- FIXME this test fails with S3 due to a bug in DiskCacheWrapper
-- FIXME It became flaky after upgrading to llvm-14 due to obscure freezes in tsan -- FIXME It became flaky after upgrading to llvm-14 due to obscure freezes in tsan
drop table if exists txn_counters; drop table if exists txn_counters;

View File

@ -1,3 +1,5 @@
-- Tags: no-ordinary-database
drop table if exists mt1; drop table if exists mt1;
drop table if exists mt2; drop table if exists mt2;

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: long # Tags: long, no-ordinary-database
# shellcheck disable=SC2015 # shellcheck disable=SC2015

View File

@ -1,3 +1,5 @@
-- Tags: no-ordinary-database
CREATE TABLE landing (n Int64) engine=MergeTree order by n; CREATE TABLE landing (n Int64) engine=MergeTree order by n;
CREATE TABLE target (n Int64) engine=MergeTree order by n; CREATE TABLE target (n Int64) engine=MergeTree order by n;
CREATE MATERIALIZED VIEW landing_to_target TO target AS CREATE MATERIALIZED VIEW landing_to_target TO target AS