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 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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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>>;
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
|
||||||
|
@ -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)
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
Loading…
Reference in New Issue
Block a user