ClickHouse/src/Interpreters/TransactionLog.h

115 lines
3.2 KiB
C++
Raw Normal View History

2021-03-31 17:55:04 +00:00
#pragma once
#include <Interpreters/MergeTreeTransaction.h>
2021-05-17 11:14:09 +00:00
#include <Interpreters/MergeTreeTransactionHolder.h>
2021-12-28 11:23:35 +00:00
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ThreadPool.h>
2021-03-31 17:55:04 +00:00
#include <boost/noncopyable.hpp>
#include <mutex>
2021-04-08 17:20:45 +00:00
#include <unordered_map>
2021-03-31 17:55:04 +00:00
namespace DB
{
2022-01-19 18:29:31 +00:00
/// We want to create a TransactionLog object lazily and avoid creation if it's not needed.
/// But we also want to call shutdown() in a specific place to avoid race conditions.
/// We cannot simply use return-static-variable pattern,
/// because a call to shutdown() may construct unnecessary object in this case.
template <typename Derived>
class SingletonHelper : private boost::noncopyable
{
public:
static Derived & instance()
{
Derived * ptr = instance_raw_ptr.load();
if (likely(ptr))
return *ptr;
std::lock_guard lock{instance_mutex};
if (!instance_holder.has_value())
{
instance_holder.emplace();
instance_raw_ptr = &instance_holder.value();
}
return instance_holder.value();
}
static void shutdownIfAny()
{
std::lock_guard lock{instance_mutex};
if (instance_holder.has_value())
instance_holder->shutdown();
}
private:
static inline std::atomic<Derived *> instance_raw_ptr;
static inline std::optional<Derived> instance_holder;
static inline std::mutex instance_mutex;
};
2022-01-14 14:03:00 +00:00
class TransactionsInfoLog;
using TransactionsInfoLogPtr = std::shared_ptr<TransactionsInfoLog>;
2021-12-28 11:23:35 +00:00
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
2022-01-19 18:29:31 +00:00
class TransactionLog final : public SingletonHelper<TransactionLog>
2021-03-31 17:55:04 +00:00
{
public:
2022-01-19 18:29:31 +00:00
//static TransactionLog & instance();
2021-03-31 17:55:04 +00:00
TransactionLog();
2021-12-28 11:23:35 +00:00
~TransactionLog();
2022-01-19 18:29:31 +00:00
void shutdown();
2021-03-31 17:55:04 +00:00
Snapshot getLatestSnapshot() const;
2021-12-30 13:15:28 +00:00
Snapshot getOldestSnapshot() const;
2021-03-31 17:55:04 +00:00
/// Allocated TID, returns transaction object
MergeTreeTransactionPtr beginTransaction();
CSN commitTransaction(const MergeTreeTransactionPtr & txn);
2021-04-09 12:53:51 +00:00
void rollbackTransaction(const MergeTreeTransactionPtr & txn) noexcept;
2021-03-31 17:55:04 +00:00
2021-04-08 17:20:45 +00:00
CSN getCSN(const TransactionID & tid) const;
CSN getCSN(const TIDHash & tid) const;
MergeTreeTransactionPtr tryGetRunningTransaction(const TIDHash & tid);
2021-06-04 09:26:47 +00:00
2021-03-31 17:55:04 +00:00
private:
2021-12-28 11:23:35 +00:00
void loadLogFromZooKeeper();
void runUpdatingThread();
void loadEntries(Strings::const_iterator beg, Strings::const_iterator end);
void loadNewEntries();
static UInt64 parseCSN(const String & csn_node_name);
static TransactionID parseTID(const String & csn_node_content);
static String writeTID(const TransactionID & tid);
ContextPtr global_context;
2021-05-18 17:07:29 +00:00
Poco::Logger * log;
2021-04-08 17:20:45 +00:00
std::atomic<CSN> latest_snapshot;
2021-03-31 17:55:04 +00:00
std::atomic<LocalTID> local_tid_counter;
2021-04-08 17:20:45 +00:00
/// FIXME Transactions: it's probably a bad idea to use global mutex here
mutable std::mutex commit_mutex;
std::unordered_map<TIDHash, CSN> tid_to_csn;
mutable std::mutex running_list_mutex;
std::unordered_map<TIDHash, MergeTreeTransactionPtr> running_list;
2021-06-04 09:26:47 +00:00
std::list<Snapshot> snapshots_in_use;
2021-12-28 11:23:35 +00:00
String zookeeper_path;
ZooKeeperPtr zookeeper;
String last_loaded_entry;
zkutil::EventPtr log_updated_event = std::make_shared<Poco::Event>();
std::atomic_bool stop_flag = false;
ThreadFromGlobalPool updating_thread;
2021-03-31 17:55:04 +00:00
};
}