ClickHouse/src/Interpreters/MergeTreeTransaction.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

90 lines
3.2 KiB
C++
Raw Normal View History

2021-03-31 17:55:04 +00:00
#pragma once
2021-11-08 18:56:09 +00:00
#include <Interpreters/TransactionVersionMetadata.h>
2021-04-09 12:53:51 +00:00
#include <boost/noncopyable.hpp>
2021-06-04 09:26:47 +00:00
#include <Storages/IStorage_fwd.h>
2022-03-31 11:47:38 +00:00
#include <Storages/TableLockHolder.h>
2022-03-10 21:29:58 +00:00
#include <Common/Stopwatch.h>
2022-03-15 13:35:48 +00:00
#include <base/scope_guard.h>
2021-06-04 09:26:47 +00:00
#include <list>
#include <unordered_set>
2021-03-31 17:55:04 +00:00
namespace DB
{
2021-04-08 17:20:45 +00:00
class IMergeTreeDataPart;
using DataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
using DataPartsVector = std::vector<DataPartPtr>;
2022-04-07 16:17:43 +00:00
/// This object is responsible for tracking all changes that some transaction is making in MergeTree tables.
/// It collects all changes that queries of current transaction made in data part sets of all MergeTree tables
2022-06-03 12:35:15 +00:00
/// to either make them visible when transaction commits or undo when transaction rolls back.
2021-04-09 12:53:51 +00:00
class MergeTreeTransaction : public std::enable_shared_from_this<MergeTreeTransaction>, private boost::noncopyable
2021-03-31 17:55:04 +00:00
{
friend class TransactionLog;
public:
enum State
{
RUNNING,
2022-05-20 15:35:29 +00:00
COMMITTING,
2021-03-31 17:55:04 +00:00
COMMITTED,
ROLLED_BACK,
};
2022-06-27 20:48:27 +00:00
CSN getSnapshot() const { return snapshot.load(std::memory_order_relaxed); }
2022-03-18 13:33:59 +00:00
void setSnapshot(CSN new_snapshot);
2021-04-09 12:53:51 +00:00
State getState() const;
2021-03-31 17:55:04 +00:00
const TransactionID tid;
2022-06-27 20:48:27 +00:00
MergeTreeTransaction(CSN snapshot_, LocalTID local_tid_, UUID host_id, std::list<CSN>::iterator snapshot_it_);
2021-03-31 17:55:04 +00:00
2021-06-04 09:26:47 +00:00
void addNewPart(const StoragePtr & storage, const DataPartPtr & new_part);
2022-03-09 20:38:18 +00:00
void removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, const TransactionInfoContext & context);
2021-04-08 17:20:45 +00:00
2021-12-14 20:06:34 +00:00
void addMutation(const StoragePtr & table, const String & mutation_id);
2021-06-04 09:26:47 +00:00
static void addNewPart(const StoragePtr & storage, const DataPartPtr & new_part, MergeTreeTransaction * txn);
static void removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, MergeTreeTransaction * txn);
static void addNewPartAndRemoveCovered(const StoragePtr & storage, const DataPartPtr & new_part, const DataPartsVector & covered_parts, MergeTreeTransaction * txn);
2021-04-08 17:20:45 +00:00
bool isReadOnly() const;
2021-04-09 12:53:51 +00:00
void onException();
2021-05-18 17:07:29 +00:00
String dumpDescription() const;
2022-03-10 21:29:58 +00:00
Float64 elapsedSeconds() const { return elapsed.elapsedSeconds(); }
2022-05-23 18:53:33 +00:00
/// Waits for transaction state to become not equal to the state corresponding to current_state_csn
bool waitStateChange(CSN current_state_csn) const;
CSN getCSN() const { return csn; }
2022-05-20 20:08:46 +00:00
2021-03-31 17:55:04 +00:00
private:
2022-03-15 13:35:48 +00:00
scope_guard beforeCommit();
2021-04-09 12:53:51 +00:00
void afterCommit(CSN assigned_csn) noexcept;
2021-12-14 20:06:34 +00:00
bool rollback() noexcept;
2022-03-10 21:29:58 +00:00
void checkIsNotCancelled() const;
mutable std::mutex mutex;
Stopwatch elapsed;
2021-04-08 17:20:45 +00:00
2022-03-18 13:33:59 +00:00
/// Usually it's equal to tid.start_csn, but can be changed by SET SNAPSHOT query (for introspection purposes and time-traveling)
2022-06-27 20:48:27 +00:00
std::atomic<CSN> snapshot;
const std::list<CSN>::iterator snapshot_in_use_it;
2021-03-31 17:55:04 +00:00
2022-03-18 11:01:26 +00:00
/// Lists of changes made by transaction
2022-06-27 20:48:27 +00:00
std::unordered_set<StoragePtr> storages TSA_GUARDED_BY(mutex);
DataPartsVector creating_parts TSA_GUARDED_BY(mutex);
DataPartsVector removing_parts TSA_GUARDED_BY(mutex);
2022-03-10 21:29:58 +00:00
using RunningMutationsList = std::vector<std::pair<StoragePtr, String>>;
2022-06-27 20:48:27 +00:00
RunningMutationsList mutations TSA_GUARDED_BY(mutex);
2022-03-18 11:01:26 +00:00
std::atomic<CSN> csn;
2021-03-31 17:55:04 +00:00
};
using MergeTreeTransactionPtr = std::shared_ptr<MergeTreeTransaction>;
}