mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #39263 from ClickHouse/fix_assertion_in_transactions
Fix assertion in transactions
This commit is contained in:
commit
611da87217
@ -12,16 +12,16 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_TRANSACTION;
|
||||
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)
|
||||
return {};
|
||||
return;
|
||||
|
||||
/// Maybe we should just throw an exception and do not support Ordinary database?
|
||||
auto default_timeout = std::chrono::milliseconds(10 * 1000);
|
||||
return storage->lockForShare(RWLockImpl::NO_QUERY, default_timeout);
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table {} belongs to database with Ordinary engine. "
|
||||
"This engine is deprecated and is not supported in transactions.", storage->getStorageID().getNameForLogs());
|
||||
}
|
||||
|
||||
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)
|
||||
{
|
||||
auto maybe_lock = getLockForOrdinary(storage);
|
||||
checkNotOrdinaryDatabase(storage);
|
||||
std::lock_guard lock{mutex};
|
||||
checkIsNotCancelled();
|
||||
storages.insert(storage);
|
||||
if (maybe_lock)
|
||||
table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock));
|
||||
creating_parts.push_back(new_part);
|
||||
}
|
||||
|
||||
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};
|
||||
@ -151,8 +149,6 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP
|
||||
part_to_remove->version.lockRemovalTID(tid, context);
|
||||
NOEXCEPT_SCOPE({
|
||||
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);
|
||||
});
|
||||
}
|
||||
@ -162,12 +158,10 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP
|
||||
|
||||
void MergeTreeTransaction::addMutation(const StoragePtr & table, const String & mutation_id)
|
||||
{
|
||||
auto maybe_lock = getLockForOrdinary(table);
|
||||
checkNotOrdinaryDatabase(table);
|
||||
std::lock_guard lock{mutex};
|
||||
checkIsNotCancelled();
|
||||
storages.insert(table);
|
||||
if (maybe_lock)
|
||||
table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock));
|
||||
mutations.emplace_back(table, mutation_id);
|
||||
}
|
||||
|
||||
|
@ -76,7 +76,6 @@ private:
|
||||
|
||||
/// Lists of changes made by transaction
|
||||
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 removing_parts TSA_GUARDED_BY(mutex);
|
||||
using RunningMutationsList = std::vector<std::pair<StoragePtr, String>>;
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long
|
||||
# Tags: long, no-ordinary-database
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/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
|
||||
# FIXME Replicated database executes ALTERs in separate context, so transaction info is lost
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/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
|
||||
# FIXME Replicated database executes ALTERs in separate context, so transaction info is lost
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long, no-replicated-database
|
||||
# Tags: long, no-replicated-database, no-ordinary-database
|
||||
|
||||
# shellcheck disable=SC2015
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/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
|
||||
# FIXME Replicated database executes ALTERs in separate context, so transaction info is lost
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/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
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
@ -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 It became flaky after upgrading to llvm-14 due to obscure freezes in tsan
|
||||
drop table if exists txn_counters;
|
||||
|
@ -1,3 +1,5 @@
|
||||
-- Tags: no-ordinary-database
|
||||
|
||||
drop table if exists mt1;
|
||||
drop table if exists mt2;
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long
|
||||
# Tags: long, no-ordinary-database
|
||||
|
||||
# shellcheck disable=SC2015
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
-- Tags: no-ordinary-database
|
||||
|
||||
CREATE TABLE landing (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
|
||||
|
Loading…
Reference in New Issue
Block a user