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