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>;
|
|
|
|
|
|
|
|
}
|