2020-10-30 14:16:47 +00:00
|
|
|
#pragma once
|
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
#include <unordered_map>
|
|
|
|
#include <vector>
|
2021-05-28 11:52:19 +00:00
|
|
|
#include <Coordination/ACLMap.h>
|
2022-05-09 08:32:25 +00:00
|
|
|
#include <Coordination/SessionExpiryQueue.h>
|
2021-02-26 13:53:34 +00:00
|
|
|
#include <Coordination/SnapshotableHashTable.h>
|
2022-01-10 19:01:41 +00:00
|
|
|
#include <IO/WriteBufferFromString.h>
|
2022-05-09 08:32:25 +00:00
|
|
|
#include <Common/ConcurrentBoundedQueue.h>
|
|
|
|
#include <Common/ZooKeeper/IKeeper.h>
|
|
|
|
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
2022-07-23 14:27:44 +00:00
|
|
|
#include <Coordination/KeeperContext.h>
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2022-01-21 14:26:50 +00:00
|
|
|
#include <absl/container/flat_hash_set.h>
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2021-01-21 15:09:48 +00:00
|
|
|
namespace DB
|
2020-10-30 14:16:47 +00:00
|
|
|
{
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageRequestProcessor;
|
|
|
|
using KeeperStorageRequestProcessorPtr = std::shared_ptr<KeeperStorageRequestProcessor>;
|
2020-11-10 13:43:10 +00:00
|
|
|
using ResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr &)>;
|
2022-01-21 14:26:50 +00:00
|
|
|
using ChildrenSet = absl::flat_hash_set<StringRef, StringRefHash>;
|
2021-03-01 13:33:34 +00:00
|
|
|
using SessionAndTimeout = std::unordered_map<int64_t, int64_t>;
|
|
|
|
|
2021-03-29 08:24:56 +00:00
|
|
|
struct KeeperStorageSnapshot;
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Keeper state machine almost equal to the ZooKeeper's state machine.
|
|
|
|
/// Implements all logic of operations, data changes, sessions allocation.
|
|
|
|
/// In-memory and not thread safe.
|
2021-03-29 08:24:56 +00:00
|
|
|
class KeeperStorage
|
2020-10-30 14:16:47 +00:00
|
|
|
{
|
2020-10-30 19:57:30 +00:00
|
|
|
public:
|
2020-10-30 14:16:47 +00:00
|
|
|
struct Node
|
|
|
|
{
|
2021-05-28 11:52:19 +00:00
|
|
|
uint64_t acl_id = 0; /// 0 -- no ACL by default
|
2020-10-30 14:16:47 +00:00
|
|
|
bool is_sequental = false;
|
|
|
|
Coordination::Stat stat{};
|
|
|
|
int32_t seq_num = 0;
|
2021-12-09 10:05:11 +00:00
|
|
|
uint64_t size_bytes; // save size to avoid calculate every time
|
2021-10-27 12:26:42 +00:00
|
|
|
|
2022-04-05 06:27:03 +00:00
|
|
|
Node() : size_bytes(sizeof(Node)) { }
|
|
|
|
|
2021-11-19 09:30:58 +00:00
|
|
|
/// Object memory size
|
2022-05-09 08:32:25 +00:00
|
|
|
uint64_t sizeInBytes() const { return size_bytes; }
|
2022-04-05 06:27:03 +00:00
|
|
|
|
|
|
|
void setData(String new_data);
|
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
const auto & getData() const noexcept { return data; }
|
2022-04-05 06:27:03 +00:00
|
|
|
|
|
|
|
void addChild(StringRef child_path);
|
|
|
|
|
|
|
|
void removeChild(StringRef child_path);
|
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
const auto & getChildren() const noexcept { return children; }
|
|
|
|
|
2022-06-15 12:48:30 +00:00
|
|
|
// Invalidate the calculated digest so it's recalculated again on the next
|
|
|
|
// getDigest call
|
2022-05-13 13:43:42 +00:00
|
|
|
void invalidateDigestCache() const;
|
2022-06-15 12:48:30 +00:00
|
|
|
|
|
|
|
// get the calculated digest of the node
|
2022-05-13 13:43:42 +00:00
|
|
|
UInt64 getDigest(std::string_view path) const;
|
2022-05-19 09:45:38 +00:00
|
|
|
|
2022-06-15 12:48:30 +00:00
|
|
|
// copy only necessary information for preprocessing and digest calculation
|
|
|
|
// (e.g. we don't need to copy list of children)
|
2022-05-19 09:45:38 +00:00
|
|
|
void shallowCopy(const Node & other);
|
2022-05-13 13:43:42 +00:00
|
|
|
|
2022-04-05 06:27:03 +00:00
|
|
|
private:
|
|
|
|
String data;
|
|
|
|
ChildrenSet children{};
|
2022-05-13 13:43:42 +00:00
|
|
|
mutable std::optional<UInt64> cached_digest;
|
2020-10-30 14:16:47 +00:00
|
|
|
};
|
|
|
|
|
2022-05-16 12:12:29 +00:00
|
|
|
enum DigestVersion : uint8_t
|
|
|
|
{
|
|
|
|
NO_DIGEST = 0,
|
2022-07-29 07:36:19 +00:00
|
|
|
V0 = 1,
|
|
|
|
V1 = 2 // added system nodes that modify the digest on startup so digest from V0 is invalid
|
2022-05-16 12:12:29 +00:00
|
|
|
};
|
|
|
|
|
2022-07-29 07:36:19 +00:00
|
|
|
static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V1;
|
2022-05-16 12:12:29 +00:00
|
|
|
|
2021-01-19 14:22:28 +00:00
|
|
|
struct ResponseForSession
|
|
|
|
{
|
|
|
|
int64_t session_id;
|
|
|
|
Coordination::ZooKeeperResponsePtr response;
|
|
|
|
};
|
2021-01-19 14:45:45 +00:00
|
|
|
using ResponsesForSessions = std::vector<ResponseForSession>;
|
2021-01-19 14:22:28 +00:00
|
|
|
|
2022-05-16 12:12:29 +00:00
|
|
|
struct Digest
|
|
|
|
{
|
|
|
|
DigestVersion version{DigestVersion::NO_DIGEST};
|
|
|
|
uint64_t value{0};
|
|
|
|
};
|
|
|
|
|
|
|
|
static bool checkDigest(const Digest & first, const Digest & second)
|
|
|
|
{
|
|
|
|
if (first.version != second.version)
|
|
|
|
return true;
|
|
|
|
|
|
|
|
if (first.version == DigestVersion::NO_DIGEST)
|
|
|
|
return true;
|
|
|
|
|
|
|
|
return first.value == second.value;
|
|
|
|
}
|
|
|
|
|
2021-01-19 14:22:28 +00:00
|
|
|
struct RequestForSession
|
2020-11-26 14:57:32 +00:00
|
|
|
{
|
|
|
|
int64_t session_id;
|
2022-01-06 13:14:45 +00:00
|
|
|
int64_t time;
|
2021-01-19 14:22:28 +00:00
|
|
|
Coordination::ZooKeeperRequestPtr request;
|
2022-05-12 08:58:36 +00:00
|
|
|
int64_t zxid{0};
|
2022-05-17 08:11:08 +00:00
|
|
|
std::optional<Digest> digest;
|
2020-11-26 14:57:32 +00:00
|
|
|
};
|
|
|
|
|
2021-05-22 16:07:47 +00:00
|
|
|
struct AuthID
|
|
|
|
{
|
|
|
|
std::string scheme;
|
|
|
|
std::string id;
|
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
bool operator==(const AuthID & other) const { return scheme == other.scheme && id == other.id; }
|
2021-05-22 16:07:47 +00:00
|
|
|
};
|
|
|
|
|
2021-01-19 14:45:45 +00:00
|
|
|
using RequestsForSessions = std::vector<RequestForSession>;
|
2021-01-19 14:22:28 +00:00
|
|
|
|
2021-02-26 13:53:34 +00:00
|
|
|
using Container = SnapshotableHashTable<Node>;
|
2021-02-19 07:05:52 +00:00
|
|
|
using Ephemerals = std::unordered_map<int64_t, std::unordered_set<std::string>>;
|
|
|
|
using SessionAndWatcher = std::unordered_map<int64_t, std::unordered_set<std::string>>;
|
keeper: store only unique session IDs for watches
This should speed up keeper, especially in case of incorrect usage (like
the case that had been fixed in #37640), especially in case on non
release build.
And also this should fix SIGKILL in stress tests.
You will find some details for one of such SIGKILL in `<details>` tag [1]:
<details>
$ pigz -cd clickhouse-server.stress.log.gz | tail
2022.05.27 16:17:24.882971 [ 637 ] {} <Trace> BackgroundSchedulePool/BgSchPool: Waiting for threads to finish.
2022.05.27 16:17:24.896749 [ 637 ] {} <Debug> MemoryTracker: Peak memory usage (for query): 4.09 MiB.
2022.05.27 16:17:24.907163 [ 637 ] {} <Debug> Application: Shut down storages.
2022.05.27 16:17:24.907233 [ 637 ] {} <Debug> Application: Waiting for current connections to servers for tables to finish.
2022.05.27 16:17:24.934335 [ 637 ] {} <Information> Application: Closed all listening sockets. Waiting for 1 outstanding connections.
2022.05.27 16:17:29.843491 [ 637 ] {} <Information> Application: Closed connections to servers for tables. But 1 remain. Probably some tables of other users cannot finish their connections after context shutdown.
2022.05.27 16:17:29.843632 [ 637 ] {} <Debug> KeeperDispatcher: Shutting down storage dispatcher
2022.05.27 16:17:34.612616 [ 688 ] {} <Test> virtual Coordination::ZooKeeperRequest::~ZooKeeperRequest(): Processing of request xid=2147483647 took 10000 ms
2022.05.27 16:17:54.612109 [ 3176 ] {} <Debug> KeeperTCPHandler: Session #12 expired
2022.05.27 16:19:59.823038 [ 635 ] {} <Fatal> Application: Child process was terminated by signal 9 (KILL). If it is not done by 'forcestop' command or manually, the possible cause is OOM Killer (see 'dmesg' and look at the '/var/log/kern.log' for the details).
Thread 26 (Thread 0x7f1c7703f700 (LWP 708)):
0 0x000000000b074b2a in __tsan::MemoryAccessImpl(__tsan::ThreadState*, unsigned long, int, bool, bool, unsigned long long*, __tsan::Shadow) ()
1 0x000000000b08630c in __tsan::MemoryAccessRange(__tsan::ThreadState*, unsigned long, unsigned long, unsigned long, bool) ()
2 0x000000000b01ff03 in memmove ()
3 0x000000001bbc8996 in std::__1::__move<long, long> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:57
4 std::__1::move<long*, long*> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:70
5 std::__1::vector<long, std::__1::allocator<long> >::erase (this=0x7b1400584c48, __position=...) at ../contrib/libcxx/include/vector:1608
6 DB::KeeperStorage::clearDeadWatches (this=0x7b5800001ad8, this@entry=0x7b5800001800, session_id=session_id@entry=12) at ../src/Coordination/KeeperStorage.cpp:1228
7 0x000000001bbc5c55 in DB::KeeperStorage::processRequest (this=0x7b5800001800, zk_request=..., session_id=12, time=1, new_last_zxid=..., check_acl=true) at ../src/Coordination/KeeperStorage.cpp:1122
8 0x000000001bba06a3 in DB::KeeperStateMachine::commit (this=<optimized out>, log_idx=3549, data=...) at ../src/Coordination/KeeperStateMachine.cpp:143
9 0x000000001bba6193 in nuraft::state_machine::commit_ext (this=0x7b4c00001f98, params=...) at ../contrib/NuRaft/include/libnuraft/state_machine.hxx:75
10 0x00000000202c5a55 in nuraft::raft_server::commit_app_log (this=this@entry=0x7b6c00002a18, idx_to_commit=idx_to_commit@entry=3549, le=..., need_to_handle_commit_elem=true, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:311
11 0x00000000202c4f98 in nuraft::raft_server::commit_in_bg_exec (this=<optimized out>, this@entry=0x7b6c00002a18, timeout_ms=timeout_ms@entry=0, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:241
12 0x00000000202c4613 in nuraft::raft_server::commit_in_bg (this=this@entry=0x7b6c00002a18) at ../contrib/NuRaft/src/handle_commit.cxx:149
...
Thread 28 (Thread 0x7f1c7603d700 (LWP 710)):
0 0x00007f1d22a6d110 in __lll_lock_wait () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x00007f1d22a650a3 in pthread_mutex_lock () from /lib/x86_64-linux-gnu/libpthread.so.0
2 0x000000000b0337b0 in pthread_mutex_lock ()
3 0x00000000221884da in std::__1::__libcpp_mutex_lock (__m=0x7b4c00002088) at ../contrib/libcxx/include/__threading_support:303
4 std::__1::mutex::lock (this=0x7b4c00002088) at ../contrib/libcxx/src/mutex.cpp:33
5 0x000000001bba4188 in std::__1::lock_guard<std::__1::mutex>::lock_guard (__m=..., this=<optimized out>) at ../contrib/libcxx/include/__mutex_base:91
6 DB::KeeperStateMachine::getDeadSessions (this=0x7b4c00001f98) at ../src/Coordination/KeeperStateMachine.cpp:360
7 0x000000001bb79b4b in DB::KeeperServer::getDeadSessions (this=0x7b4400012700) at ../src/Coordination/KeeperServer.cpp:572
8 0x000000001bb64d1a in DB::KeeperDispatcher::sessionCleanerTask (this=<optimized out>, this@entry=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:399
...
Thread 1 (Thread 0x7f1d227148c0 (LWP 637)):
0 0x00007f1d22a69376 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x000000000b0895e0 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*), void (*)(void*), void*) ()
2 0x000000000b017091 in pthread_cond_wait ()
3 0x0000000020569d98 in Poco::EventImpl::waitImpl (this=0x7b2000008798) at ../contrib/poco/Foundation/src/Event_POSIX.cpp:106
4 0x000000001bb636cf in Poco::Event::wait (this=0x7b2000008798) at ../contrib/poco/Foundation/include/Poco/Event.h:97
5 ThreadFromGlobalPool::join (this=<optimized out>) at ../src/Common/ThreadPool.h:217
6 DB::KeeperDispatcher::shutdown (this=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:322
7 0x0000000019ca8bfc in DB::Context::shutdownKeeperDispatcher (this=<optimized out>) at ../src/Interpreters/Context.cpp:2111
8 0x000000000b0a979b in DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_9::operator()() const (this=0x7ffcde44f0a0) at ../programs/server/Server.cpp:1407
</details>
[1]: https://s3.amazonaws.com/clickhouse-test-reports/37593/2613149f6bf4f242bbbf2c3c8539b5176fd77286/stress_test__thread__actions_.html
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-05-30 08:43:49 +00:00
|
|
|
using SessionIDs = std::unordered_set<int64_t>;
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2021-05-21 21:19:22 +00:00
|
|
|
/// Just vector of SHA1 from user:password
|
2021-05-22 16:07:47 +00:00
|
|
|
using AuthIDs = std::vector<AuthID>;
|
2021-05-21 21:19:22 +00:00
|
|
|
using SessionAndAuth = std::unordered_map<int64_t, AuthIDs>;
|
2021-01-19 14:22:28 +00:00
|
|
|
using Watches = std::map<String /* path, relative of root_path */, SessionIDs>;
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
int64_t session_id_counter{1};
|
|
|
|
|
|
|
|
SessionAndAuth session_and_auth;
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Main hashtable with nodes. Contain all information about data.
|
|
|
|
/// All other structures expect session_and_timeout can be restored from
|
|
|
|
/// container.
|
2020-10-30 14:16:47 +00:00
|
|
|
Container container;
|
2021-08-24 12:30:31 +00:00
|
|
|
|
2022-05-17 06:45:51 +00:00
|
|
|
// Applying ZooKeeper request to storage consists of two steps:
|
|
|
|
// - preprocessing which, instead of applying the changes directly to storage,
|
|
|
|
// generates deltas with those changes, denoted with the request ZXID
|
|
|
|
// - processing which applies deltas with the correct ZXID to the storage
|
|
|
|
//
|
|
|
|
// Delta objects allow us two things:
|
|
|
|
// - fetch the latest, uncommitted state of an object by getting the committed
|
|
|
|
// state of that same object from the storage and applying the deltas
|
|
|
|
// in the same order as they are defined
|
|
|
|
// - quickly commit the changes to the storage
|
2022-05-09 08:32:25 +00:00
|
|
|
struct CreateNodeDelta
|
2022-05-05 10:32:41 +00:00
|
|
|
{
|
2022-05-06 12:25:25 +00:00
|
|
|
Coordination::Stat stat;
|
|
|
|
bool is_sequental;
|
|
|
|
Coordination::ACLs acls;
|
|
|
|
String data;
|
|
|
|
};
|
|
|
|
|
|
|
|
struct RemoveNodeDelta
|
|
|
|
{
|
|
|
|
int32_t version{-1};
|
2022-05-23 14:37:57 +00:00
|
|
|
int64_t ephemeral_owner{0};
|
2022-05-06 12:25:25 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
struct UpdateNodeDelta
|
|
|
|
{
|
|
|
|
std::function<void(Node &)> update_fn;
|
|
|
|
int32_t version{-1};
|
|
|
|
};
|
|
|
|
|
|
|
|
struct SetACLDelta
|
|
|
|
{
|
|
|
|
Coordination::ACLs acls;
|
|
|
|
int32_t version{-1};
|
|
|
|
};
|
|
|
|
|
|
|
|
struct ErrorDelta
|
|
|
|
{
|
|
|
|
Coordination::Error error;
|
|
|
|
};
|
|
|
|
|
2022-05-09 07:02:11 +00:00
|
|
|
struct FailedMultiDelta
|
|
|
|
{
|
|
|
|
std::vector<Coordination::Error> error_codes;
|
|
|
|
};
|
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
// Denotes end of a subrequest in multi request
|
2022-05-09 07:02:11 +00:00
|
|
|
struct SubDeltaEnd
|
|
|
|
{
|
|
|
|
};
|
|
|
|
|
2022-05-10 12:53:18 +00:00
|
|
|
struct AddAuthDelta
|
|
|
|
{
|
|
|
|
int64_t session_id;
|
|
|
|
AuthID auth_id;
|
|
|
|
};
|
|
|
|
|
2022-05-13 13:43:42 +00:00
|
|
|
using Operation = std::
|
|
|
|
variant<CreateNodeDelta, RemoveNodeDelta, UpdateNodeDelta, SetACLDelta, AddAuthDelta, ErrorDelta, SubDeltaEnd, FailedMultiDelta>;
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
struct Delta
|
|
|
|
{
|
2022-05-09 08:32:25 +00:00
|
|
|
Delta(String path_, int64_t zxid_, Operation operation_) : path(std::move(path_)), zxid(zxid_), operation(std::move(operation_)) { }
|
2022-05-05 10:32:41 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
Delta(int64_t zxid_, Coordination::Error error) : Delta("", zxid_, ErrorDelta{error}) { }
|
2022-05-09 07:02:11 +00:00
|
|
|
|
2022-05-10 12:53:18 +00:00
|
|
|
Delta(int64_t zxid_, Operation subdelta) : Delta("", zxid_, subdelta) { }
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
String path;
|
2022-05-05 10:32:41 +00:00
|
|
|
int64_t zxid;
|
2022-05-06 12:25:25 +00:00
|
|
|
Operation operation;
|
2022-05-05 10:32:41 +00:00
|
|
|
};
|
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
struct UncommittedState
|
2022-05-05 10:32:41 +00:00
|
|
|
{
|
2022-05-11 09:08:39 +00:00
|
|
|
explicit UncommittedState(KeeperStorage & storage_) : storage(storage_) { }
|
2022-05-05 10:32:41 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
void addDeltas(std::vector<Delta> new_deltas);
|
|
|
|
void commit(int64_t commit_zxid);
|
|
|
|
void rollback(int64_t rollback_zxid);
|
2022-05-13 13:43:42 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
std::shared_ptr<Node> getNode(StringRef path) const;
|
|
|
|
Coordination::ACLs getACLs(StringRef path) const;
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
void applyDelta(const Delta & delta);
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-17 08:11:08 +00:00
|
|
|
bool hasACL(int64_t session_id, bool is_local, std::function<bool(const AuthID &)> predicate)
|
2022-05-10 12:53:18 +00:00
|
|
|
{
|
|
|
|
for (const auto & session_auth : storage.session_and_auth[session_id])
|
|
|
|
{
|
|
|
|
if (predicate(session_auth))
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (is_local)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
for (const auto & delta : deltas)
|
|
|
|
{
|
2022-05-17 06:45:51 +00:00
|
|
|
if (const auto * auth_delta = std::get_if<KeeperStorage::AddAuthDelta>(&delta.operation);
|
2022-05-10 12:53:18 +00:00
|
|
|
auth_delta && auth_delta->session_id == session_id && predicate(auth_delta->auth_id))
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2022-06-15 12:48:30 +00:00
|
|
|
std::shared_ptr<Node> tryGetNodeFromStorage(StringRef path) const;
|
2022-05-19 09:45:38 +00:00
|
|
|
|
|
|
|
struct UncommittedNode
|
|
|
|
{
|
|
|
|
std::shared_ptr<Node> node{nullptr};
|
|
|
|
Coordination::ACLs acls{};
|
|
|
|
int64_t zxid{0};
|
|
|
|
};
|
2022-05-05 10:32:41 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
mutable std::unordered_map<std::string, UncommittedNode> nodes;
|
2022-06-14 07:37:02 +00:00
|
|
|
std::list<Delta> deltas;
|
2022-05-05 10:32:41 +00:00
|
|
|
KeeperStorage & storage;
|
|
|
|
};
|
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
UncommittedState uncommitted_state{*this};
|
2022-05-05 10:32:41 +00:00
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::Error commit(int64_t zxid);
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-17 06:45:51 +00:00
|
|
|
// Create node in the storage
|
|
|
|
// Returns false if it failed to create the node, true otherwise
|
|
|
|
// We don't care about the exact failure because we should've caught it during preprocessing
|
2022-05-09 08:32:25 +00:00
|
|
|
bool createNode(
|
|
|
|
const std::string & path,
|
|
|
|
String data,
|
|
|
|
const Coordination::Stat & stat,
|
|
|
|
bool is_sequental,
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ACLs node_acls);
|
2022-05-17 06:45:51 +00:00
|
|
|
|
|
|
|
// Remove node in the storage
|
|
|
|
// Returns false if it failed to remove the node, true otherwise
|
|
|
|
// We don't care about the exact failure because we should've caught it during preprocessing
|
2022-05-06 12:25:25 +00:00
|
|
|
bool removeNode(const std::string & path, int32_t version);
|
|
|
|
|
2022-05-10 12:53:18 +00:00
|
|
|
bool checkACL(StringRef path, int32_t permissions, int64_t session_id, bool is_local);
|
|
|
|
|
2022-05-23 14:37:57 +00:00
|
|
|
void unregisterEphemeralPath(int64_t session_id, const std::string & path);
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Mapping session_id -> set of ephemeral nodes paths
|
2020-11-19 16:06:19 +00:00
|
|
|
Ephemerals ephemerals;
|
2022-04-05 06:27:03 +00:00
|
|
|
/// Mapping session_id -> set of watched nodes paths
|
2020-11-26 14:57:32 +00:00
|
|
|
SessionAndWatcher sessions_and_watchers;
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Expiration queue for session, allows to get dead sessions at some point of time
|
2021-02-03 20:32:15 +00:00
|
|
|
SessionExpiryQueue session_expiry_queue;
|
2021-08-24 12:30:31 +00:00
|
|
|
/// All active sessions with timeout
|
2021-02-03 20:32:15 +00:00
|
|
|
SessionAndTimeout session_and_timeout;
|
2021-08-24 12:30:31 +00:00
|
|
|
|
|
|
|
/// ACLMap for more compact ACLs storage inside nodes.
|
2021-05-28 11:52:19 +00:00
|
|
|
ACLMap acl_map;
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Global id of all requests applied to storage
|
2021-01-21 13:53:10 +00:00
|
|
|
int64_t zxid{0};
|
2022-05-13 13:43:42 +00:00
|
|
|
|
2022-06-15 13:46:27 +00:00
|
|
|
// older Keeper node (pre V5 snapshots) can create snapshots and receive logs from newer Keeper nodes
|
|
|
|
// this can lead to some inconsistencies, e.g. from snapshot it will use log_idx as zxid
|
|
|
|
// while the log will have a smaller zxid because it's generated by the newer nodes
|
|
|
|
// we save the value loaded from snapshot to know when is it okay to have
|
|
|
|
// smaller zxid in newer requests
|
|
|
|
int64_t old_snapshot_zxid{0};
|
|
|
|
|
2022-05-13 13:43:42 +00:00
|
|
|
struct TransactionInfo
|
|
|
|
{
|
|
|
|
int64_t zxid;
|
2022-05-16 12:12:29 +00:00
|
|
|
Digest nodes_digest;
|
2022-05-13 13:43:42 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
std::deque<TransactionInfo> uncommitted_transactions;
|
|
|
|
|
2022-05-16 12:12:29 +00:00
|
|
|
uint64_t nodes_digest{0};
|
2022-05-13 13:43:42 +00:00
|
|
|
|
2021-01-21 13:53:10 +00:00
|
|
|
bool finalized{false};
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Currently active watches (node_path -> subscribed sessions)
|
2020-10-30 14:16:47 +00:00
|
|
|
Watches watches;
|
2022-05-09 08:32:25 +00:00
|
|
|
Watches list_watches; /// Watches for 'list' request (watches on children).
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2021-01-19 14:22:28 +00:00
|
|
|
void clearDeadWatches(int64_t session_id);
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2022-05-12 08:58:36 +00:00
|
|
|
/// Get current committed zxid
|
2022-05-09 08:32:25 +00:00
|
|
|
int64_t getZXID() const { return zxid; }
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2022-05-12 08:58:36 +00:00
|
|
|
int64_t getNextZXID() const
|
|
|
|
{
|
2022-05-13 13:43:42 +00:00
|
|
|
if (uncommitted_transactions.empty())
|
2022-05-12 08:58:36 +00:00
|
|
|
return zxid + 1;
|
|
|
|
|
2022-05-13 13:43:42 +00:00
|
|
|
return uncommitted_transactions.back().zxid + 1;
|
2022-05-12 08:58:36 +00:00
|
|
|
}
|
|
|
|
|
2022-05-16 12:12:29 +00:00
|
|
|
Digest getNodesDigest(bool committed) const;
|
2022-05-12 08:58:36 +00:00
|
|
|
|
2022-07-23 14:27:44 +00:00
|
|
|
KeeperContextPtr keeper_context;
|
2022-05-16 13:08:10 +00:00
|
|
|
|
2021-05-22 16:07:47 +00:00
|
|
|
const String superdigest;
|
|
|
|
|
2022-07-21 09:31:06 +00:00
|
|
|
bool initialized{false};
|
|
|
|
|
2022-07-23 14:27:44 +00:00
|
|
|
KeeperStorage(int64_t tick_time_ms, const String & superdigest_, const KeeperContextPtr & keeper_context_, bool initialize_system_nodes = true);
|
2022-07-21 09:31:06 +00:00
|
|
|
|
|
|
|
void initializeSystemNodes();
|
2021-01-19 14:22:28 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Allocate new session id with the specified timeouts
|
2021-02-04 08:28:11 +00:00
|
|
|
int64_t getSessionID(int64_t session_timeout_ms)
|
2021-01-21 14:34:34 +00:00
|
|
|
{
|
2021-02-03 20:32:15 +00:00
|
|
|
auto result = session_id_counter++;
|
|
|
|
session_and_timeout.emplace(result, session_timeout_ms);
|
2021-09-02 11:40:54 +00:00
|
|
|
session_expiry_queue.addNewSessionOrUpdate(result, session_timeout_ms);
|
2021-02-03 20:32:15 +00:00
|
|
|
return result;
|
2021-01-21 14:34:34 +00:00
|
|
|
}
|
2021-01-21 20:01:25 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Add session id. Used when restoring KeeperStorage from snapshot.
|
2021-03-01 13:33:34 +00:00
|
|
|
void addSessionID(int64_t session_id, int64_t session_timeout_ms)
|
|
|
|
{
|
|
|
|
session_and_timeout.emplace(session_id, session_timeout_ms);
|
2021-09-02 11:40:54 +00:00
|
|
|
session_expiry_queue.addNewSessionOrUpdate(session_id, session_timeout_ms);
|
2021-03-01 13:33:34 +00:00
|
|
|
}
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
UInt64 calculateNodesDigest(UInt64 current_digest, const std::vector<Delta> & new_deltas) const;
|
2022-05-13 13:43:42 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Process user request and return response.
|
|
|
|
/// check_acl = false only when converting data from ZooKeeper.
|
2022-05-09 08:32:25 +00:00
|
|
|
ResponsesForSessions processRequest(
|
|
|
|
const Coordination::ZooKeeperRequestPtr & request,
|
|
|
|
int64_t session_id,
|
|
|
|
std::optional<int64_t> new_last_zxid,
|
|
|
|
bool check_acl = true,
|
|
|
|
bool is_local = false);
|
2022-05-09 09:16:05 +00:00
|
|
|
void preprocessRequest(
|
2022-05-13 13:43:42 +00:00
|
|
|
const Coordination::ZooKeeperRequestPtr & request,
|
|
|
|
int64_t session_id,
|
|
|
|
int64_t time,
|
|
|
|
int64_t new_last_zxid,
|
2022-05-16 13:08:10 +00:00
|
|
|
bool check_acl = true,
|
2022-05-17 08:11:08 +00:00
|
|
|
std::optional<Digest> digest = std::nullopt);
|
2022-05-10 13:04:35 +00:00
|
|
|
void rollbackRequest(int64_t rollback_zxid);
|
2021-02-04 12:07:41 +00:00
|
|
|
|
|
|
|
void finalize();
|
2021-02-03 20:32:15 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Set of methods for creating snapshots
|
|
|
|
|
|
|
|
/// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version.
|
2022-05-09 08:32:25 +00:00
|
|
|
void enableSnapshotMode(size_t up_to_version) { container.enableSnapshotMode(up_to_version); }
|
2021-02-26 14:54:59 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Turn off snapshot mode.
|
2022-05-09 08:32:25 +00:00
|
|
|
void disableSnapshotMode() { container.disableSnapshotMode(); }
|
2021-02-26 14:54:59 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
Container::const_iterator getSnapshotIteratorBegin() const { return container.begin(); }
|
2021-02-26 14:54:59 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Clear outdated data from internal container.
|
2022-05-09 08:32:25 +00:00
|
|
|
void clearGarbageAfterSnapshot() { container.clearOutdatedNodes(); }
|
2021-02-26 14:54:59 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Get all active sessions
|
2022-05-09 08:32:25 +00:00
|
|
|
const SessionAndTimeout & getActiveSessions() const { return session_and_timeout; }
|
2021-02-26 14:54:59 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Get all dead sessions
|
2022-05-09 08:32:25 +00:00
|
|
|
std::vector<int64_t> getDeadSessions() const { return session_expiry_queue.getExpiredSessions(); }
|
2021-10-27 12:26:42 +00:00
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
/// Introspection functions mostly used in 4-letter commands
|
2022-05-09 08:32:25 +00:00
|
|
|
uint64_t getNodesCount() const { return container.size(); }
|
2021-10-27 12:26:42 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
uint64_t getApproximateDataSize() const { return container.getApproximateDataSize(); }
|
2021-10-27 12:26:42 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
uint64_t getArenaDataSize() const { return container.keyArenaSize(); }
|
2022-01-19 11:46:29 +00:00
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
uint64_t getTotalWatchesCount() const;
|
2021-11-05 10:21:34 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
uint64_t getWatchedPathsCount() const { return watches.size() + list_watches.size(); }
|
2021-11-05 10:21:34 +00:00
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
uint64_t getSessionsWithWatchesCount() const;
|
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
uint64_t getSessionWithEphemeralNodesCount() const { return ephemerals.size(); }
|
2021-11-18 20:17:22 +00:00
|
|
|
uint64_t getTotalEphemeralNodesCount() const;
|
2021-10-27 12:26:42 +00:00
|
|
|
|
2021-11-05 10:21:34 +00:00
|
|
|
void dumpWatches(WriteBufferFromOwnString & buf) const;
|
|
|
|
void dumpWatchesByPath(WriteBufferFromOwnString & buf) const;
|
2021-11-18 20:17:22 +00:00
|
|
|
void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const;
|
2022-05-16 13:08:10 +00:00
|
|
|
|
|
|
|
private:
|
|
|
|
void removeDigest(const Node & node, std::string_view path);
|
|
|
|
void addDigest(const Node & node, std::string_view path);
|
2020-10-30 14:16:47 +00:00
|
|
|
};
|
2020-11-11 13:55:28 +00:00
|
|
|
|
2021-03-29 08:24:56 +00:00
|
|
|
using KeeperStoragePtr = std::unique_ptr<KeeperStorage>;
|
2021-03-18 20:55:11 +00:00
|
|
|
|
2020-10-30 14:16:47 +00:00
|
|
|
}
|