2020-10-30 14:16:47 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Common/ZooKeeper/IKeeper.h>
|
|
|
|
#include <Common/ConcurrentBoundedQueue.h>
|
|
|
|
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
2021-02-03 20:32:15 +00:00
|
|
|
#include <Coordination/SessionExpiryQueue.h>
|
2021-05-28 11:52:19 +00:00
|
|
|
#include <Coordination/ACLMap.h>
|
2021-02-26 13:53:34 +00:00
|
|
|
#include <Coordination/SnapshotableHashTable.h>
|
2020-11-19 16:06:19 +00:00
|
|
|
#include <unordered_map>
|
|
|
|
#include <unordered_set>
|
2021-01-19 15:51:52 +00:00
|
|
|
#include <vector>
|
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 &)>;
|
2021-02-19 07:25:55 +00:00
|
|
|
using ChildrenSet = std::unordered_set<std::string>;
|
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
|
|
|
|
{
|
|
|
|
String data;
|
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-02-19 09:24:50 +00:00
|
|
|
ChildrenSet children{};
|
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
|
|
|
|
2021-12-09 10:15:53 +00:00
|
|
|
Node()
|
2021-12-09 10:04:29 +00:00
|
|
|
{
|
2021-12-08 06:52:21 +00:00
|
|
|
size_bytes = sizeof(size_bytes);
|
|
|
|
size_bytes += data.size();
|
|
|
|
size_bytes += sizeof(acl_id);
|
|
|
|
size_bytes += sizeof(is_sequental);
|
|
|
|
size_bytes += sizeof(stat);
|
|
|
|
size_bytes += sizeof(seq_num);
|
|
|
|
}
|
2021-11-19 09:30:58 +00:00
|
|
|
/// Object memory size
|
2021-12-09 10:15:53 +00:00
|
|
|
uint64_t sizeInBytes() const
|
2021-12-09 10:04:29 +00:00
|
|
|
{
|
2021-12-08 06:52:21 +00:00
|
|
|
return size_bytes;
|
|
|
|
}
|
2020-10-30 14:16:47 +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
|
|
|
|
|
|
|
struct RequestForSession
|
2020-11-26 14:57:32 +00:00
|
|
|
{
|
|
|
|
int64_t session_id;
|
2021-01-19 14:22:28 +00:00
|
|
|
Coordination::ZooKeeperRequestPtr request;
|
2020-11-26 14:57:32 +00:00
|
|
|
};
|
|
|
|
|
2021-05-22 16:07:47 +00:00
|
|
|
struct AuthID
|
|
|
|
{
|
|
|
|
std::string scheme;
|
|
|
|
std::string id;
|
|
|
|
|
|
|
|
bool operator==(const AuthID & other) const
|
|
|
|
{
|
|
|
|
return scheme == other.scheme && id == other.id;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
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>>;
|
2021-01-19 14:22:28 +00:00
|
|
|
using SessionIDs = std::vector<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
|
|
|
public:
|
|
|
|
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
|
|
|
|
|
|
|
/// Mapping session_id -> set of ephemeral nodes paths
|
2020-11-19 16:06:19 +00:00
|
|
|
Ephemerals ephemerals;
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Mapping sessuib_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};
|
|
|
|
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;
|
|
|
|
Watches list_watches; /// Watches for 'list' request (watches on children).
|
|
|
|
|
2021-01-19 14:22:28 +00:00
|
|
|
void clearDeadWatches(int64_t session_id);
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Get current zxid
|
2021-02-25 08:34:05 +00:00
|
|
|
int64_t getZXID() const
|
2020-10-30 14:16:47 +00:00
|
|
|
{
|
2021-02-25 08:34:05 +00:00
|
|
|
return zxid;
|
2021-01-19 14:22:28 +00:00
|
|
|
}
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2021-05-22 16:07:47 +00:00
|
|
|
const String superdigest;
|
|
|
|
|
2020-10-30 14:16:47 +00:00
|
|
|
public:
|
2021-05-23 17:54:42 +00:00
|
|
|
KeeperStorage(int64_t tick_time_ms, const String & superdigest_);
|
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
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Process user request and return response.
|
|
|
|
/// check_acl = false only when converting data from ZooKeeper.
|
2021-06-21 15:45:45 +00:00
|
|
|
ResponsesForSessions processRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id, std::optional<int64_t> new_last_zxid, bool check_acl = true);
|
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.
|
2021-02-26 14:54:59 +00:00
|
|
|
void enableSnapshotMode()
|
|
|
|
{
|
|
|
|
container.enableSnapshotMode();
|
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Turn off snapshot mode.
|
2021-02-26 14:54:59 +00:00
|
|
|
void disableSnapshotMode()
|
|
|
|
{
|
|
|
|
container.disableSnapshotMode();
|
|
|
|
}
|
|
|
|
|
|
|
|
Container::const_iterator getSnapshotIteratorBegin() const
|
|
|
|
{
|
|
|
|
return container.begin();
|
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Clear outdated data from internal container.
|
2021-02-26 14:54:59 +00:00
|
|
|
void clearGarbageAfterSnapshot()
|
|
|
|
{
|
|
|
|
container.clearOutdatedNodes();
|
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Get all active sessions
|
2021-02-26 14:54:59 +00:00
|
|
|
const SessionAndTimeout & getActiveSessions() const
|
|
|
|
{
|
|
|
|
return session_and_timeout;
|
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Get all dead sessions
|
2021-09-02 20:37:34 +00:00
|
|
|
std::vector<int64_t> getDeadSessions()
|
2021-02-03 20:32:15 +00:00
|
|
|
{
|
|
|
|
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
|
|
|
|
uint64_t getNodesCount() const
|
2021-10-27 12:26:42 +00:00
|
|
|
{
|
|
|
|
return container.size();
|
|
|
|
}
|
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
uint64_t getApproximateDataSize() const
|
2021-10-27 12:26:42 +00:00
|
|
|
{
|
2021-11-19 07:52:35 +00:00
|
|
|
return container.getApproximateDataSize();
|
2021-10-27 12:26:42 +00:00
|
|
|
}
|
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
uint64_t getTotalWatchesCount() const;
|
2021-11-05 10:21:34 +00:00
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
uint64_t getWatchedPathsCount() const
|
2021-11-05 10:21:34 +00:00
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
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;
|
|
|
|
|
|
|
|
uint64_t getSessionWithEphemeralNodesCount() const
|
2021-10-27 12:26:42 +00:00
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
return ephemerals.size();
|
2021-10-27 12:26:42 +00:00
|
|
|
}
|
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;
|
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
|
|
|
}
|