mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
some preparement to support RocksDB as alternative keeper storage
This commit is contained in:
parent
a5e17dc919
commit
3bccce2eae
@ -44,16 +44,17 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv)
|
||||
keeper_context->setDigestEnabled(true);
|
||||
keeper_context->setSnapshotDisk(std::make_shared<DiskLocal>("Keeper-snapshots", options["output-dir"].as<std::string>()));
|
||||
|
||||
DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false);
|
||||
/// TODO(hanfei): support rocksdb here
|
||||
DB::KeeperMemoryStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false);
|
||||
|
||||
DB::deserializeKeeperStorageFromSnapshotsDir(storage, options["zookeeper-snapshots-dir"].as<std::string>(), logger);
|
||||
storage.initializeSystemNodes();
|
||||
|
||||
DB::deserializeLogsAndApplyToStorage(storage, options["zookeeper-logs-dir"].as<std::string>(), logger);
|
||||
DB::SnapshotMetadataPtr snapshot_meta = std::make_shared<DB::SnapshotMetadata>(storage.getZXID(), 1, std::make_shared<nuraft::cluster_config>());
|
||||
DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta);
|
||||
DB::KeeperStorageSnapshot<DB::KeeperMemoryStorage> snapshot(&storage, snapshot_meta);
|
||||
|
||||
DB::KeeperSnapshotManager manager(1, keeper_context);
|
||||
DB::KeeperSnapshotManager<DB::KeeperMemoryStorage> manager(1, keeper_context);
|
||||
auto snp = manager.serializeSnapshotToBuffer(snapshot);
|
||||
auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID());
|
||||
std::cout << "Snapshot serialized to path:" << fs::path(file_info.disk->getPath()) / file_info.path << std::endl;
|
||||
|
@ -66,13 +66,13 @@ void KeeperDispatcher::requestThread()
|
||||
RaftAppendResult prev_result = nullptr;
|
||||
/// Requests from previous iteration. We store them to be able
|
||||
/// to send errors to the client.
|
||||
KeeperStorage::RequestsForSessions prev_batch;
|
||||
KeeperStorageBase::RequestsForSessions prev_batch;
|
||||
|
||||
auto & shutdown_called = keeper_context->shutdown_called;
|
||||
|
||||
while (!shutdown_called)
|
||||
{
|
||||
KeeperStorage::RequestForSession request;
|
||||
KeeperStorageBase::RequestForSession request;
|
||||
|
||||
auto coordination_settings = configuration_and_settings->coordination_settings;
|
||||
uint64_t max_wait = coordination_settings->operation_timeout_ms.totalMilliseconds();
|
||||
@ -93,7 +93,7 @@ void KeeperDispatcher::requestThread()
|
||||
if (shutdown_called)
|
||||
break;
|
||||
|
||||
KeeperStorage::RequestsForSessions current_batch;
|
||||
KeeperStorageBase::RequestsForSessions current_batch;
|
||||
size_t current_batch_bytes_size = 0;
|
||||
|
||||
bool has_read_request = false;
|
||||
@ -244,7 +244,7 @@ void KeeperDispatcher::responseThread()
|
||||
auto & shutdown_called = keeper_context->shutdown_called;
|
||||
while (!shutdown_called)
|
||||
{
|
||||
KeeperStorage::ResponseForSession response_for_session;
|
||||
KeeperStorageBase::ResponseForSession response_for_session;
|
||||
|
||||
uint64_t max_wait = configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds();
|
||||
|
||||
@ -340,7 +340,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ
|
||||
return false;
|
||||
}
|
||||
|
||||
KeeperStorage::RequestForSession request_info;
|
||||
KeeperStorageBase::RequestForSession request_info;
|
||||
request_info.request = request;
|
||||
using namespace std::chrono;
|
||||
request_info.time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count();
|
||||
@ -386,7 +386,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf
|
||||
snapshots_queue,
|
||||
keeper_context,
|
||||
snapshot_s3,
|
||||
[this](uint64_t log_idx, const KeeperStorage::RequestForSession & request_for_session)
|
||||
[this](uint64_t log_idx, const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
{
|
||||
/// check if we have queue of read requests depending on this request to be committed
|
||||
@ -486,7 +486,7 @@ void KeeperDispatcher::shutdown()
|
||||
update_configuration_thread.join();
|
||||
}
|
||||
|
||||
KeeperStorage::RequestForSession request_for_session;
|
||||
KeeperStorageBase::RequestForSession request_for_session;
|
||||
|
||||
/// Set session expired for all pending requests
|
||||
while (requests_queue && requests_queue->tryPop(request_for_session))
|
||||
@ -497,7 +497,7 @@ void KeeperDispatcher::shutdown()
|
||||
setResponse(request_for_session.session_id, response);
|
||||
}
|
||||
|
||||
KeeperStorage::RequestsForSessions close_requests;
|
||||
KeeperStorageBase::RequestsForSessions close_requests;
|
||||
{
|
||||
/// Clear all registered sessions
|
||||
std::lock_guard lock(session_to_response_callback_mutex);
|
||||
@ -511,7 +511,7 @@ void KeeperDispatcher::shutdown()
|
||||
auto request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close);
|
||||
request->xid = Coordination::CLOSE_XID;
|
||||
using namespace std::chrono;
|
||||
KeeperStorage::RequestForSession request_info
|
||||
KeeperStorageBase::RequestForSession request_info
|
||||
{
|
||||
.session_id = session,
|
||||
.time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count(),
|
||||
@ -609,7 +609,7 @@ void KeeperDispatcher::sessionCleanerTask()
|
||||
auto request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close);
|
||||
request->xid = Coordination::CLOSE_XID;
|
||||
using namespace std::chrono;
|
||||
KeeperStorage::RequestForSession request_info
|
||||
KeeperStorageBase::RequestForSession request_info
|
||||
{
|
||||
.session_id = dead_session,
|
||||
.time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count(),
|
||||
@ -657,16 +657,16 @@ void KeeperDispatcher::finishSession(int64_t session_id)
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSessions & requests_for_sessions, Coordination::Error error)
|
||||
void KeeperDispatcher::addErrorResponses(const KeeperStorageBase::RequestsForSessions & requests_for_sessions, Coordination::Error error)
|
||||
{
|
||||
for (const auto & request_for_session : requests_for_sessions)
|
||||
{
|
||||
KeeperStorage::ResponsesForSessions responses;
|
||||
KeeperStorageBase::ResponsesForSessions responses;
|
||||
auto response = request_for_session.request->makeResponse();
|
||||
response->xid = request_for_session.request->xid;
|
||||
response->zxid = 0;
|
||||
response->error = error;
|
||||
if (!responses_queue.push(DB::KeeperStorage::ResponseForSession{request_for_session.session_id, response}))
|
||||
if (!responses_queue.push(DB::KeeperStorageBase::ResponseForSession{request_for_session.session_id, response}))
|
||||
throw Exception(ErrorCodes::SYSTEM_ERROR,
|
||||
"Could not push error response xid {} zxid {} error message {} to responses queue",
|
||||
response->xid,
|
||||
@ -675,7 +675,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession
|
||||
}
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions)
|
||||
nuraft::ptr<nuraft::buffer> KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorageBase::RequestsForSessions & requests_for_sessions)
|
||||
{
|
||||
if (!result->has_result())
|
||||
result->get();
|
||||
@ -697,7 +697,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms)
|
||||
{
|
||||
/// New session id allocation is a special request, because we cannot process it in normal
|
||||
/// way: get request -> put to raft -> set response for registered callback.
|
||||
KeeperStorage::RequestForSession request_info;
|
||||
KeeperStorageBase::RequestForSession request_info;
|
||||
std::shared_ptr<Coordination::ZooKeeperSessionIDRequest> request = std::make_shared<Coordination::ZooKeeperSessionIDRequest>();
|
||||
/// Internal session id. It's a temporary number which is unique for each client on this server
|
||||
/// but can be same on different servers.
|
||||
|
@ -27,7 +27,7 @@ using ZooKeeperResponseCallback = std::function<void(const Coordination::ZooKeep
|
||||
class KeeperDispatcher
|
||||
{
|
||||
private:
|
||||
using RequestsQueue = ConcurrentBoundedQueue<KeeperStorage::RequestForSession>;
|
||||
using RequestsQueue = ConcurrentBoundedQueue<KeeperStorageBase::RequestForSession>;
|
||||
using SessionToResponseCallback = std::unordered_map<int64_t, ZooKeeperResponseCallback>;
|
||||
using ClusterUpdateQueue = ConcurrentBoundedQueue<ClusterUpdateAction>;
|
||||
|
||||
@ -96,11 +96,11 @@ private:
|
||||
|
||||
/// Add error responses for requests to responses queue.
|
||||
/// Clears requests.
|
||||
void addErrorResponses(const KeeperStorage::RequestsForSessions & requests_for_sessions, Coordination::Error error);
|
||||
void addErrorResponses(const KeeperStorageBase::RequestsForSessions & requests_for_sessions, Coordination::Error error);
|
||||
|
||||
/// Forcefully wait for result and sets errors if something when wrong.
|
||||
/// Clears both arguments
|
||||
nuraft::ptr<nuraft::buffer> forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions);
|
||||
nuraft::ptr<nuraft::buffer> forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorageBase::RequestsForSessions & requests_for_sessions);
|
||||
|
||||
public:
|
||||
std::mutex read_request_queue_mutex;
|
||||
@ -108,7 +108,7 @@ public:
|
||||
std::atomic<uint64_t> our_last_committed_log_idx = 0;
|
||||
|
||||
/// queue of read requests that can be processed after a request with specific session ID and XID is committed
|
||||
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, KeeperStorage::RequestsForSessions>> read_request_queue;
|
||||
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, KeeperStorageBase::RequestsForSessions>> read_request_queue;
|
||||
|
||||
/// Just allocate some objects, real initialization is done by `intialize method`
|
||||
KeeperDispatcher();
|
||||
@ -189,7 +189,7 @@ public:
|
||||
|
||||
Keeper4LWInfo getKeeper4LWInfo() const;
|
||||
|
||||
const KeeperStateMachine & getStateMachine() const
|
||||
const IKeeperStateMachine & getStateMachine() const
|
||||
{
|
||||
return *server->getKeeperStateMachine();
|
||||
}
|
||||
|
@ -116,7 +116,7 @@ KeeperServer::KeeperServer(
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
KeeperContextPtr keeper_context_,
|
||||
KeeperSnapshotManagerS3 & snapshot_manager_s3,
|
||||
KeeperStateMachine::CommitCallback commit_callback)
|
||||
IKeeperStateMachine::CommitCallback commit_callback)
|
||||
: server_id(configuration_and_settings_->server_id)
|
||||
, coordination_settings(configuration_and_settings_->coordination_settings)
|
||||
, log(&Poco::Logger::get("KeeperServer"))
|
||||
@ -128,7 +128,7 @@ KeeperServer::KeeperServer(
|
||||
if (coordination_settings->quorum_reads)
|
||||
LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower.");
|
||||
|
||||
state_machine = nuraft::cs_new<KeeperStateMachine>(
|
||||
state_machine = nuraft::cs_new<KeeperStateMachine<KeeperMemoryStorage>>(
|
||||
responses_queue_,
|
||||
snapshots_queue_,
|
||||
coordination_settings,
|
||||
@ -478,7 +478,7 @@ namespace
|
||||
{
|
||||
|
||||
// Serialize the request for the log entry
|
||||
nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorage::RequestForSession & request_for_session)
|
||||
nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
DB::WriteBufferFromNuraftBuffer write_buf;
|
||||
DB::writeIntBinary(request_for_session.session_id, write_buf);
|
||||
@ -486,7 +486,7 @@ nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorage::RequestFor
|
||||
DB::writeIntBinary(request_for_session.time, write_buf);
|
||||
/// we fill with dummy values to eliminate unnecessary copy later on when we will write correct values
|
||||
DB::writeIntBinary(static_cast<int64_t>(0), write_buf); /// zxid
|
||||
DB::writeIntBinary(KeeperStorage::DigestVersion::NO_DIGEST, write_buf); /// digest version or NO_DIGEST flag
|
||||
DB::writeIntBinary(KeeperStorageBase::DigestVersion::NO_DIGEST, write_buf); /// digest version or NO_DIGEST flag
|
||||
DB::writeIntBinary(static_cast<uint64_t>(0), write_buf); /// digest value
|
||||
/// if new fields are added, update KeeperStateMachine::ZooKeeperLogSerializationVersion along with parseRequest function and PreAppendLog callback handler
|
||||
return write_buf.getBuffer();
|
||||
@ -494,7 +494,7 @@ nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorage::RequestFor
|
||||
|
||||
}
|
||||
|
||||
void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession & request_for_session)
|
||||
void KeeperServer::putLocalReadRequest(const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
if (!request_for_session.request->isReadRequest())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot process non-read request locally");
|
||||
@ -502,7 +502,7 @@ void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession &
|
||||
state_machine->processReadRequest(request_for_session);
|
||||
}
|
||||
|
||||
RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForSessions & requests_for_sessions)
|
||||
RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorageBase::RequestsForSessions & requests_for_sessions)
|
||||
{
|
||||
std::vector<nuraft::ptr<nuraft::buffer>> entries;
|
||||
for (const auto & request_for_session : requests_for_sessions)
|
||||
@ -725,7 +725,7 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
|
||||
|
||||
auto entry_buf = entry->get_buf_ptr();
|
||||
|
||||
KeeperStateMachine::ZooKeeperLogSerializationVersion serialization_version;
|
||||
IKeeperStateMachine::ZooKeeperLogSerializationVersion serialization_version;
|
||||
auto request_for_session = state_machine->parseRequest(*entry_buf, /*final=*/false, &serialization_version);
|
||||
request_for_session->zxid = next_zxid;
|
||||
if (!state_machine->preprocess(*request_for_session))
|
||||
@ -735,10 +735,10 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
|
||||
|
||||
/// older versions of Keeper can send logs that are missing some fields
|
||||
size_t bytes_missing = 0;
|
||||
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
bytes_missing += sizeof(request_for_session->time);
|
||||
|
||||
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_ZXID_DIGEST)
|
||||
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_ZXID_DIGEST)
|
||||
bytes_missing += sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value);
|
||||
|
||||
if (bytes_missing != 0)
|
||||
@ -752,19 +752,19 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
|
||||
size_t write_buffer_header_size
|
||||
= sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value);
|
||||
|
||||
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
write_buffer_header_size += sizeof(request_for_session->time);
|
||||
|
||||
auto * buffer_start = reinterpret_cast<BufferBase::Position>(entry_buf->data_begin() + entry_buf->size() - write_buffer_header_size);
|
||||
|
||||
WriteBufferFromPointer write_buf(buffer_start, write_buffer_header_size);
|
||||
|
||||
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
writeIntBinary(request_for_session->time, write_buf);
|
||||
|
||||
writeIntBinary(request_for_session->zxid, write_buf);
|
||||
writeIntBinary(request_for_session->digest->version, write_buf);
|
||||
if (request_for_session->digest->version != KeeperStorage::NO_DIGEST)
|
||||
if (request_for_session->digest->version != KeeperStorageBase::NO_DIGEST)
|
||||
writeIntBinary(request_for_session->digest->value, write_buf);
|
||||
|
||||
write_buf.finalize();
|
||||
|
@ -24,7 +24,7 @@ private:
|
||||
|
||||
CoordinationSettingsPtr coordination_settings;
|
||||
|
||||
nuraft::ptr<KeeperStateMachine> state_machine;
|
||||
nuraft::ptr<IKeeperStateMachine> state_machine;
|
||||
|
||||
nuraft::ptr<KeeperStateManager> state_manager;
|
||||
|
||||
@ -79,26 +79,26 @@ public:
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
KeeperContextPtr keeper_context_,
|
||||
KeeperSnapshotManagerS3 & snapshot_manager_s3,
|
||||
KeeperStateMachine::CommitCallback commit_callback);
|
||||
IKeeperStateMachine::CommitCallback commit_callback);
|
||||
|
||||
/// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings.
|
||||
void startup(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6 = true);
|
||||
|
||||
/// Put local read request and execute in state machine directly and response into
|
||||
/// responses queue
|
||||
void putLocalReadRequest(const KeeperStorage::RequestForSession & request);
|
||||
void putLocalReadRequest(const KeeperStorageBase::RequestForSession & request);
|
||||
|
||||
bool isRecovering() const { return is_recovering; }
|
||||
bool reconfigEnabled() const { return enable_reconfiguration; }
|
||||
|
||||
/// Put batch of requests into Raft and get result of put. Responses will be set separately into
|
||||
/// responses_queue.
|
||||
RaftAppendResult putRequestBatch(const KeeperStorage::RequestsForSessions & requests);
|
||||
RaftAppendResult putRequestBatch(const KeeperStorageBase::RequestsForSessions & requests);
|
||||
|
||||
/// Return set of the non-active sessions
|
||||
std::vector<int64_t> getDeadSessions();
|
||||
|
||||
nuraft::ptr<KeeperStateMachine> getKeeperStateMachine() const { return state_machine; }
|
||||
nuraft::ptr<IKeeperStateMachine> getKeeperStateMachine() const { return state_machine; }
|
||||
|
||||
void forceRecovery();
|
||||
|
||||
|
@ -68,7 +68,8 @@ namespace
|
||||
return base;
|
||||
}
|
||||
|
||||
void writeNode(const KeeperStorage::Node & node, SnapshotVersion version, WriteBuffer & out)
|
||||
template<typename Node>
|
||||
void writeNode(const Node & node, SnapshotVersion version, WriteBuffer & out)
|
||||
{
|
||||
writeBinary(node.getData(), out);
|
||||
|
||||
@ -96,7 +97,8 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
void readNode(KeeperStorage::Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map)
|
||||
template<typename Node>
|
||||
void readNode(Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map)
|
||||
{
|
||||
String new_data;
|
||||
readBinary(new_data, in);
|
||||
@ -169,7 +171,8 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context)
|
||||
template<typename Storage>
|
||||
void KeeperStorageSnapshot<Storage>::serialize(const KeeperStorageSnapshot<Storage> & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context)
|
||||
{
|
||||
writeBinary(static_cast<uint8_t>(snapshot.version), out);
|
||||
serializeSnapshotMetadata(snapshot.snapshot_meta, out);
|
||||
@ -179,11 +182,11 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
|
||||
writeBinary(snapshot.zxid, out);
|
||||
if (keeper_context->digestEnabled())
|
||||
{
|
||||
writeBinary(static_cast<uint8_t>(KeeperStorage::CURRENT_DIGEST_VERSION), out);
|
||||
writeBinary(static_cast<uint8_t>(Storage::CURRENT_DIGEST_VERSION), out);
|
||||
writeBinary(snapshot.nodes_digest, out);
|
||||
}
|
||||
else
|
||||
writeBinary(static_cast<uint8_t>(KeeperStorage::NO_DIGEST), out);
|
||||
writeBinary(static_cast<uint8_t>(Storage::NO_DIGEST), out);
|
||||
}
|
||||
|
||||
writeBinary(snapshot.session_id, out);
|
||||
@ -256,7 +259,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
|
||||
writeBinary(session_id, out);
|
||||
writeBinary(timeout, out);
|
||||
|
||||
KeeperStorage::AuthIDs ids;
|
||||
KeeperStorageBase::AuthIDs ids;
|
||||
if (snapshot.session_and_auth.contains(session_id))
|
||||
ids = snapshot.session_and_auth.at(session_id);
|
||||
|
||||
@ -277,7 +280,8 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context)
|
||||
template<typename Storage>
|
||||
void KeeperStorageSnapshot<Storage>::deserialize(SnapshotDeserializationResult<Storage> & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context)
|
||||
{
|
||||
uint8_t version;
|
||||
readBinary(version, in);
|
||||
@ -286,7 +290,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported snapshot version {}", version);
|
||||
|
||||
deserialization_result.snapshot_meta = deserializeSnapshotMetadata(in);
|
||||
KeeperStorage & storage = *deserialization_result.storage;
|
||||
Storage & storage = *deserialization_result.storage;
|
||||
|
||||
bool recalculate_digest = keeper_context->digestEnabled();
|
||||
if (version >= SnapshotVersion::V5)
|
||||
@ -294,11 +298,11 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
readBinary(storage.zxid, in);
|
||||
uint8_t digest_version;
|
||||
readBinary(digest_version, in);
|
||||
if (digest_version != KeeperStorage::DigestVersion::NO_DIGEST)
|
||||
if (digest_version != Storage::DigestVersion::NO_DIGEST)
|
||||
{
|
||||
uint64_t nodes_digest;
|
||||
readBinary(nodes_digest, in);
|
||||
if (digest_version == KeeperStorage::CURRENT_DIGEST_VERSION)
|
||||
if (digest_version == Storage::CURRENT_DIGEST_VERSION)
|
||||
{
|
||||
storage.nodes_digest = nodes_digest;
|
||||
recalculate_digest = false;
|
||||
@ -361,7 +365,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
{
|
||||
std::string path;
|
||||
readBinary(path, in);
|
||||
KeeperStorage::Node node{};
|
||||
typename Storage::Node node{};
|
||||
readNode(node, in, current_version, storage.acl_map);
|
||||
|
||||
using enum Coordination::PathMatchResult;
|
||||
@ -389,7 +393,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT)
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg);
|
||||
node = KeeperStorage::Node{};
|
||||
node = typename Storage::Node{};
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
@ -417,7 +421,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
{
|
||||
auto parent_path = parentNodePath(itr.key);
|
||||
storage.container.updateValue(
|
||||
parent_path, [version, path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseNodeName(path), /*update_size*/ version < SnapshotVersion::V4); });
|
||||
parent_path, [version, path = itr.key](typename Storage::Node & value) { value.addChild(getBaseNodeName(path), /*update_size*/ version < SnapshotVersion::V4); });
|
||||
}
|
||||
}
|
||||
|
||||
@ -457,14 +461,14 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
size_t session_auths_size;
|
||||
readBinary(session_auths_size, in);
|
||||
|
||||
KeeperStorage::AuthIDs ids;
|
||||
typename Storage::AuthIDs ids;
|
||||
size_t session_auth_counter = 0;
|
||||
while (session_auth_counter < session_auths_size)
|
||||
{
|
||||
String scheme, id;
|
||||
readBinary(scheme, in);
|
||||
readBinary(id, in);
|
||||
ids.emplace_back(KeeperStorage::AuthID{scheme, id});
|
||||
ids.emplace_back(typename Storage::AuthID{scheme, id});
|
||||
|
||||
session_auth_counter++;
|
||||
}
|
||||
@ -487,7 +491,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
}
|
||||
}
|
||||
|
||||
KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_)
|
||||
template<typename Storage>
|
||||
KeeperStorageSnapshot<Storage>::KeeperStorageSnapshot(Storage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_)
|
||||
: storage(storage_)
|
||||
, snapshot_meta(std::make_shared<SnapshotMetadata>(up_to_log_idx_, 0, std::make_shared<nuraft::cluster_config>()))
|
||||
, session_id(storage->session_id_counter)
|
||||
@ -504,8 +509,9 @@ KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t
|
||||
session_and_auth = storage->session_and_auth;
|
||||
}
|
||||
|
||||
KeeperStorageSnapshot::KeeperStorageSnapshot(
|
||||
KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_)
|
||||
template<typename Storage>
|
||||
KeeperStorageSnapshot<Storage>::KeeperStorageSnapshot(
|
||||
Storage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_)
|
||||
: storage(storage_)
|
||||
, snapshot_meta(snapshot_meta_)
|
||||
, session_id(storage->session_id_counter)
|
||||
@ -522,12 +528,14 @@ KeeperStorageSnapshot::KeeperStorageSnapshot(
|
||||
session_and_auth = storage->session_and_auth;
|
||||
}
|
||||
|
||||
KeeperStorageSnapshot::~KeeperStorageSnapshot()
|
||||
template<typename Storage>
|
||||
KeeperStorageSnapshot<Storage>::~KeeperStorageSnapshot()
|
||||
{
|
||||
storage->disableSnapshotMode();
|
||||
}
|
||||
|
||||
KeeperSnapshotManager::KeeperSnapshotManager(
|
||||
template<typename Storage>
|
||||
KeeperSnapshotManager<Storage>::KeeperSnapshotManager(
|
||||
size_t snapshots_to_keep_,
|
||||
const KeeperContextPtr & keeper_context_,
|
||||
bool compress_snapshots_zstd_,
|
||||
@ -606,7 +614,8 @@ KeeperSnapshotManager::KeeperSnapshotManager(
|
||||
moveSnapshotsIfNeeded();
|
||||
}
|
||||
|
||||
SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx)
|
||||
template<typename Storage>
|
||||
SnapshotFileInfo KeeperSnapshotManager<Storage>::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx)
|
||||
{
|
||||
ReadBufferFromNuraftBuffer reader(buffer);
|
||||
|
||||
@ -634,7 +643,8 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::bu
|
||||
return {snapshot_file_name, disk};
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk()
|
||||
template<typename Storage>
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager<Storage>::deserializeLatestSnapshotBufferFromDisk()
|
||||
{
|
||||
while (!existing_snapshots.empty())
|
||||
{
|
||||
@ -655,7 +665,8 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeLatestSnapshotBuff
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const
|
||||
template<typename Storage>
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager<Storage>::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const
|
||||
{
|
||||
const auto & [snapshot_path, snapshot_disk] = existing_snapshots.at(up_to_log_idx);
|
||||
WriteBufferFromNuraftBuffer writer;
|
||||
@ -664,7 +675,8 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeSnapshotBufferFrom
|
||||
return writer.getBuffer();
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const
|
||||
template<typename Storage>
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager<Storage>::serializeSnapshotToBuffer(const KeeperStorageSnapshot<Storage> & snapshot) const
|
||||
{
|
||||
std::unique_ptr<WriteBufferFromNuraftBuffer> writer = std::make_unique<WriteBufferFromNuraftBuffer>();
|
||||
auto * buffer_raw_ptr = writer.get();
|
||||
@ -674,13 +686,13 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::serializeSnapshotToBuffer(con
|
||||
else
|
||||
compressed_writer = std::make_unique<CompressedWriteBuffer>(*writer);
|
||||
|
||||
KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context);
|
||||
KeeperStorageSnapshot<Storage>::serialize(snapshot, *compressed_writer, keeper_context);
|
||||
compressed_writer->finalize();
|
||||
return buffer_raw_ptr->getBuffer();
|
||||
}
|
||||
|
||||
|
||||
bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer)
|
||||
template<typename Storage>
|
||||
bool KeeperSnapshotManager<Storage>::isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer)
|
||||
{
|
||||
static constexpr unsigned char ZSTD_COMPRESSED_MAGIC[4] = {0x28, 0xB5, 0x2F, 0xFD};
|
||||
|
||||
@ -691,7 +703,8 @@ bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer)
|
||||
return memcmp(magic_from_buffer, ZSTD_COMPRESSED_MAGIC, 4) == 0;
|
||||
}
|
||||
|
||||
SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const
|
||||
template<typename Storage>
|
||||
SnapshotDeserializationResult<Storage> KeeperSnapshotManager<Storage>::deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const
|
||||
{
|
||||
bool is_zstd_compressed = isZstdCompressed(buffer);
|
||||
|
||||
@ -703,14 +716,15 @@ SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuff
|
||||
else
|
||||
compressed_reader = std::make_unique<CompressedReadBuffer>(*reader);
|
||||
|
||||
SnapshotDeserializationResult result;
|
||||
result.storage = std::make_unique<KeeperStorage>(storage_tick_time, superdigest, keeper_context, /* initialize_system_nodes */ false);
|
||||
KeeperStorageSnapshot::deserialize(result, *compressed_reader, keeper_context);
|
||||
SnapshotDeserializationResult<Storage> result;
|
||||
result.storage = std::make_unique<Storage>(storage_tick_time, superdigest, keeper_context, /* initialize_system_nodes */ false);
|
||||
KeeperStorageSnapshot<Storage>::deserialize(result, *compressed_reader, keeper_context);
|
||||
result.storage->initializeSystemNodes();
|
||||
return result;
|
||||
}
|
||||
|
||||
SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot()
|
||||
template<typename Storage>
|
||||
SnapshotDeserializationResult<Storage> KeeperSnapshotManager<Storage>::restoreFromLatestSnapshot()
|
||||
{
|
||||
if (existing_snapshots.empty())
|
||||
return {};
|
||||
@ -721,23 +735,27 @@ SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot()
|
||||
return deserializeSnapshotFromBuffer(buffer);
|
||||
}
|
||||
|
||||
DiskPtr KeeperSnapshotManager::getDisk() const
|
||||
template<typename Storage>
|
||||
DiskPtr KeeperSnapshotManager<Storage>::getDisk() const
|
||||
{
|
||||
return keeper_context->getSnapshotDisk();
|
||||
}
|
||||
|
||||
DiskPtr KeeperSnapshotManager::getLatestSnapshotDisk() const
|
||||
template<typename Storage>
|
||||
DiskPtr KeeperSnapshotManager<Storage>::getLatestSnapshotDisk() const
|
||||
{
|
||||
return keeper_context->getLatestSnapshotDisk();
|
||||
}
|
||||
|
||||
void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded()
|
||||
template<typename Storage>
|
||||
void KeeperSnapshotManager<Storage>::removeOutdatedSnapshotsIfNeeded()
|
||||
{
|
||||
while (existing_snapshots.size() > snapshots_to_keep)
|
||||
removeSnapshot(existing_snapshots.begin()->first);
|
||||
}
|
||||
|
||||
void KeeperSnapshotManager::moveSnapshotsIfNeeded()
|
||||
template<typename Storage>
|
||||
void KeeperSnapshotManager<Storage>::moveSnapshotsIfNeeded()
|
||||
{
|
||||
/// move snapshots to correct disks
|
||||
|
||||
@ -767,7 +785,8 @@ void KeeperSnapshotManager::moveSnapshotsIfNeeded()
|
||||
|
||||
}
|
||||
|
||||
void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx)
|
||||
template<typename Storage>
|
||||
void KeeperSnapshotManager<Storage>::removeSnapshot(uint64_t log_idx)
|
||||
{
|
||||
auto itr = existing_snapshots.find(log_idx);
|
||||
if (itr == existing_snapshots.end())
|
||||
@ -777,7 +796,8 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx)
|
||||
existing_snapshots.erase(itr);
|
||||
}
|
||||
|
||||
SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot)
|
||||
template<typename Storage>
|
||||
SnapshotFileInfo KeeperSnapshotManager<Storage>::serializeSnapshotToDisk(const KeeperStorageSnapshot<Storage> & snapshot)
|
||||
{
|
||||
auto up_to_log_idx = snapshot.snapshot_meta->get_last_log_idx();
|
||||
auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd);
|
||||
@ -796,7 +816,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor
|
||||
else
|
||||
compressed_writer = std::make_unique<CompressedWriteBuffer>(*writer);
|
||||
|
||||
KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context);
|
||||
KeeperStorageSnapshot<Storage>::serialize(snapshot, *compressed_writer, keeper_context);
|
||||
compressed_writer->finalize();
|
||||
compressed_writer->sync();
|
||||
|
||||
@ -809,4 +829,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor
|
||||
return {snapshot_file_name, disk};
|
||||
}
|
||||
|
||||
template struct KeeperStorageSnapshot<KeeperMemoryStorage>;
|
||||
template class KeeperSnapshotManager<KeeperMemoryStorage>;
|
||||
|
||||
}
|
||||
|
@ -29,16 +29,19 @@ enum SnapshotVersion : uint8_t
|
||||
static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V5;
|
||||
|
||||
/// What is stored in binary snapshot
|
||||
template<typename Storage>
|
||||
struct SnapshotDeserializationResult
|
||||
{
|
||||
/// Storage
|
||||
KeeperStoragePtr storage;
|
||||
std::unique_ptr<Storage> storage;
|
||||
/// Snapshot metadata (up_to_log_idx and so on)
|
||||
SnapshotMetadataPtr snapshot_meta;
|
||||
/// Cluster config
|
||||
ClusterConfigPtr cluster_config;
|
||||
};
|
||||
|
||||
struct IKeeperStorageSnapshot {};
|
||||
|
||||
/// In memory keeper snapshot. Keeper Storage based on a hash map which can be
|
||||
/// turned into snapshot mode. This operation is fast and KeeperStorageSnapshot
|
||||
/// class do it in constructor. It also copies iterators from storage hash table
|
||||
@ -47,21 +50,22 @@ struct SnapshotDeserializationResult
|
||||
///
|
||||
/// This representation of snapshot have to be serialized into NuRaft
|
||||
/// buffer and send over network or saved to file.
|
||||
struct KeeperStorageSnapshot
|
||||
template<typename Storage>
|
||||
struct KeeperStorageSnapshot : IKeeperStorageSnapshot
|
||||
{
|
||||
public:
|
||||
KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr);
|
||||
KeeperStorageSnapshot(Storage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr);
|
||||
|
||||
KeeperStorageSnapshot(
|
||||
KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_ = nullptr);
|
||||
Storage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_ = nullptr);
|
||||
|
||||
~KeeperStorageSnapshot();
|
||||
|
||||
static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context);
|
||||
static void serialize(const KeeperStorageSnapshot<Storage> & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context);
|
||||
|
||||
static void deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context);
|
||||
static void deserialize(SnapshotDeserializationResult<Storage> & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context);
|
||||
|
||||
KeeperStorage * storage;
|
||||
Storage * storage;
|
||||
|
||||
SnapshotVersion version = CURRENT_SNAPSHOT_VERSION;
|
||||
/// Snapshot metadata
|
||||
@ -72,11 +76,11 @@ public:
|
||||
/// so we have for loop for (i = 0; i < snapshot_container_size; ++i) { doSmth(begin + i); }
|
||||
size_t snapshot_container_size;
|
||||
/// Iterator to the start of the storage
|
||||
KeeperStorage::Container::const_iterator begin;
|
||||
Storage::Container::const_iterator begin;
|
||||
/// Active sessions and their timeouts
|
||||
SessionAndTimeout session_and_timeout;
|
||||
/// Sessions credentials
|
||||
KeeperStorage::SessionAndAuth session_and_auth;
|
||||
Storage::SessionAndAuth session_and_auth;
|
||||
/// ACLs cache for better performance. Without we cannot deserialize storage.
|
||||
std::unordered_map<uint64_t, Coordination::ACLs> acl_map;
|
||||
/// Cluster config from snapshot, can be empty
|
||||
@ -93,14 +97,12 @@ struct SnapshotFileInfo
|
||||
DiskPtr disk;
|
||||
};
|
||||
|
||||
using KeeperStorageSnapshotPtr = std::shared_ptr<KeeperStorageSnapshot>;
|
||||
using KeeperStorageSnapshotPtr = std::shared_ptr<IKeeperStorageSnapshot>;
|
||||
using CreateSnapshotCallback = std::function<SnapshotFileInfo(KeeperStorageSnapshotPtr &&)>;
|
||||
|
||||
|
||||
using SnapshotMetaAndStorage = std::pair<SnapshotMetadataPtr, KeeperStoragePtr>;
|
||||
|
||||
/// Class responsible for snapshots serialization and deserialization. Each snapshot
|
||||
/// has it's path on disk and log index.
|
||||
template<typename Storage>
|
||||
class KeeperSnapshotManager
|
||||
{
|
||||
public:
|
||||
@ -112,18 +114,18 @@ public:
|
||||
size_t storage_tick_time_ = 500);
|
||||
|
||||
/// Restore storage from latest available snapshot
|
||||
SnapshotDeserializationResult restoreFromLatestSnapshot();
|
||||
SnapshotDeserializationResult<Storage> restoreFromLatestSnapshot();
|
||||
|
||||
/// Compress snapshot and serialize it to buffer
|
||||
nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const;
|
||||
nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot<Storage> & snapshot) const;
|
||||
|
||||
/// Serialize already compressed snapshot to disk (return path)
|
||||
SnapshotFileInfo serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx);
|
||||
|
||||
/// Serialize snapshot directly to disk
|
||||
SnapshotFileInfo serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot);
|
||||
SnapshotFileInfo serializeSnapshotToDisk(const KeeperStorageSnapshot<Storage> & snapshot);
|
||||
|
||||
SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
|
||||
SnapshotDeserializationResult<Storage> deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
|
||||
|
||||
/// Deserialize snapshot with log index up_to_log_idx from disk into compressed nuraft buffer.
|
||||
nuraft::ptr<nuraft::buffer> deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const;
|
||||
|
@ -40,7 +40,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
KeeperStateMachine::KeeperStateMachine(
|
||||
IKeeperStateMachine::IKeeperStateMachine(
|
||||
ResponsesQueue & responses_queue_,
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
const CoordinationSettingsPtr & coordination_settings_,
|
||||
@ -50,12 +50,6 @@ KeeperStateMachine::KeeperStateMachine(
|
||||
const std::string & superdigest_)
|
||||
: commit_callback(commit_callback_)
|
||||
, coordination_settings(coordination_settings_)
|
||||
, snapshot_manager(
|
||||
coordination_settings->snapshots_to_keep,
|
||||
keeper_context_,
|
||||
coordination_settings->compress_snapshots_with_zstd_format,
|
||||
superdigest_,
|
||||
coordination_settings->dead_session_check_period_ms.totalMilliseconds())
|
||||
, responses_queue(responses_queue_)
|
||||
, snapshots_queue(snapshots_queue_)
|
||||
, min_request_size_to_cache(coordination_settings_->min_request_size_for_cache)
|
||||
@ -67,6 +61,32 @@ KeeperStateMachine::KeeperStateMachine(
|
||||
{
|
||||
}
|
||||
|
||||
template<typename Storage>
|
||||
KeeperStateMachine<Storage>::KeeperStateMachine(
|
||||
ResponsesQueue & responses_queue_,
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
const CoordinationSettingsPtr & coordination_settings_,
|
||||
const KeeperContextPtr & keeper_context_,
|
||||
KeeperSnapshotManagerS3 * snapshot_manager_s3_,
|
||||
IKeeperStateMachine::CommitCallback commit_callback_,
|
||||
const std::string & superdigest_)
|
||||
: IKeeperStateMachine(
|
||||
responses_queue_,
|
||||
snapshots_queue_,
|
||||
coordination_settings_,
|
||||
keeper_context_,
|
||||
snapshot_manager_s3_,
|
||||
commit_callback_,
|
||||
superdigest_),
|
||||
snapshot_manager(
|
||||
coordination_settings->snapshots_to_keep,
|
||||
keeper_context_,
|
||||
coordination_settings->compress_snapshots_with_zstd_format,
|
||||
superdigest_,
|
||||
coordination_settings->dead_session_check_period_ms.totalMilliseconds())
|
||||
{
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
@ -77,7 +97,8 @@ bool isLocalDisk(const IDisk & disk)
|
||||
|
||||
}
|
||||
|
||||
void KeeperStateMachine::init()
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::init()
|
||||
{
|
||||
/// Do everything without mutexes, no other threads exist.
|
||||
LOG_DEBUG(log, "Totally have {} snapshots", snapshot_manager.totalSnapshots());
|
||||
@ -129,7 +150,7 @@ void KeeperStateMachine::init()
|
||||
}
|
||||
|
||||
if (!storage)
|
||||
storage = std::make_unique<KeeperStorage>(
|
||||
storage = std::make_unique<Storage>(
|
||||
coordination_settings->dead_session_check_period_ms.totalMilliseconds(), superdigest, keeper_context);
|
||||
}
|
||||
|
||||
@ -137,12 +158,12 @@ namespace
|
||||
{
|
||||
|
||||
void assertDigest(
|
||||
const KeeperStorage::Digest & first,
|
||||
const KeeperStorage::Digest & second,
|
||||
const KeeperStorageBase::Digest & first,
|
||||
const KeeperStorageBase::Digest & second,
|
||||
const Coordination::ZooKeeperRequest & request,
|
||||
bool committing)
|
||||
{
|
||||
if (!KeeperStorage::checkDigest(first, second))
|
||||
if (!KeeperStorageBase::checkDigest(first, second))
|
||||
{
|
||||
LOG_FATAL(
|
||||
&Poco::Logger::get("KeeperStateMachine"),
|
||||
@ -160,7 +181,8 @@ void assertDigest(
|
||||
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine::pre_commit(uint64_t log_idx, nuraft::buffer & data)
|
||||
template<typename Storage>
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine<Storage>::pre_commit(uint64_t log_idx, nuraft::buffer & data)
|
||||
{
|
||||
auto request_for_session = parseRequest(data, /*final=*/false);
|
||||
if (!request_for_session->zxid)
|
||||
@ -175,10 +197,10 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::pre_commit(uint64_t log_idx, nur
|
||||
return result;
|
||||
}
|
||||
|
||||
std::shared_ptr<KeeperStorage::RequestForSession> KeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version)
|
||||
std::shared_ptr<KeeperStorageBase::RequestForSession> IKeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version)
|
||||
{
|
||||
ReadBufferFromNuraftBuffer buffer(data);
|
||||
auto request_for_session = std::make_shared<KeeperStorage::RequestForSession>();
|
||||
auto request_for_session = std::make_shared<KeeperStorageBase::RequestForSession>();
|
||||
readIntBinary(request_for_session->session_id, buffer);
|
||||
|
||||
int32_t length;
|
||||
@ -251,7 +273,7 @@ std::shared_ptr<KeeperStorage::RequestForSession> KeeperStateMachine::parseReque
|
||||
|
||||
request_for_session->digest.emplace();
|
||||
readIntBinary(request_for_session->digest->version, buffer);
|
||||
if (request_for_session->digest->version != KeeperStorage::DigestVersion::NO_DIGEST || !buffer.eof())
|
||||
if (request_for_session->digest->version != KeeperStorageBase::DigestVersion::NO_DIGEST || !buffer.eof())
|
||||
readIntBinary(request_for_session->digest->value, buffer);
|
||||
}
|
||||
|
||||
@ -267,7 +289,8 @@ std::shared_ptr<KeeperStorage::RequestForSession> KeeperStateMachine::parseReque
|
||||
return request_for_session;
|
||||
}
|
||||
|
||||
bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session)
|
||||
template<typename Storage>
|
||||
bool KeeperStateMachine<Storage>::preprocess(const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
const auto op_num = request_for_session.request->getOpNum();
|
||||
if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig)
|
||||
@ -301,10 +324,11 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req
|
||||
return true;
|
||||
}
|
||||
|
||||
void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session)
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::reconfigure(const KeeperStorageBase::RequestForSession& request_for_session)
|
||||
{
|
||||
std::lock_guard _(storage_and_responses_lock);
|
||||
KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session);
|
||||
KeeperStorageBase::ResponseForSession response = processReconfiguration(request_for_session);
|
||||
if (!responses_queue.push(response))
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed);
|
||||
@ -314,8 +338,9 @@ void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& req
|
||||
}
|
||||
}
|
||||
|
||||
KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration(
|
||||
const KeeperStorage::RequestForSession & request_for_session)
|
||||
template<typename Storage>
|
||||
KeeperStorageBase::ResponseForSession KeeperStateMachine<Storage>::processReconfiguration(
|
||||
const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest);
|
||||
|
||||
@ -324,7 +349,7 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration(
|
||||
const int64_t zxid = request_for_session.zxid;
|
||||
|
||||
using enum Coordination::Error;
|
||||
auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorage::ResponseForSession
|
||||
auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorageBase::ResponseForSession
|
||||
{
|
||||
auto res = std::make_shared<Coordination::ZooKeeperReconfigResponse>();
|
||||
res->xid = request.xid;
|
||||
@ -381,7 +406,8 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration(
|
||||
return { session_id, std::move(response) };
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data)
|
||||
template<typename Storage>
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine<Storage>::commit(const uint64_t log_idx, nuraft::buffer & data)
|
||||
{
|
||||
auto request_for_session = parseRequest(data, true);
|
||||
if (!request_for_session->zxid)
|
||||
@ -392,7 +418,7 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
|
||||
if (!keeper_context->local_logs_preprocessed)
|
||||
preprocess(*request_for_session);
|
||||
|
||||
auto try_push = [this](const KeeperStorage::ResponseForSession& response)
|
||||
auto try_push = [this](const KeeperStorageBase::ResponseForSession& response)
|
||||
{
|
||||
if (!responses_queue.push(response))
|
||||
{
|
||||
@ -412,7 +438,7 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
|
||||
std::shared_ptr<Coordination::ZooKeeperSessionIDResponse> response = std::make_shared<Coordination::ZooKeeperSessionIDResponse>();
|
||||
response->internal_id = session_id_request.internal_id;
|
||||
response->server_id = session_id_request.server_id;
|
||||
KeeperStorage::ResponseForSession response_for_session;
|
||||
KeeperStorageBase::ResponseForSession response_for_session;
|
||||
response_for_session.session_id = -1;
|
||||
response_for_session.response = response;
|
||||
|
||||
@ -431,7 +457,7 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
|
||||
}
|
||||
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
KeeperStorage::ResponsesForSessions responses_for_sessions
|
||||
KeeperStorageBase::ResponsesForSessions responses_for_sessions
|
||||
= storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid);
|
||||
for (auto & response_for_session : responses_for_sessions)
|
||||
try_push(response_for_session);
|
||||
@ -448,7 +474,8 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
|
||||
template<typename Storage>
|
||||
bool KeeperStateMachine<Storage>::apply_snapshot(nuraft::snapshot & s)
|
||||
{
|
||||
LOG_DEBUG(log, "Applying snapshot {}", s.get_last_log_idx());
|
||||
nuraft::ptr<nuraft::buffer> latest_snapshot_ptr;
|
||||
@ -475,7 +502,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
|
||||
{ /// deserialize and apply snapshot to storage
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
|
||||
SnapshotDeserializationResult snapshot_deserialization_result;
|
||||
SnapshotDeserializationResult<Storage> snapshot_deserialization_result;
|
||||
if (latest_snapshot_ptr)
|
||||
snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr);
|
||||
else
|
||||
@ -496,7 +523,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
|
||||
}
|
||||
|
||||
|
||||
void KeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr<nuraft::cluster_config> & new_conf)
|
||||
void IKeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr<nuraft::cluster_config> & new_conf)
|
||||
{
|
||||
std::lock_guard lock(cluster_config_lock);
|
||||
auto tmp = new_conf->serialize();
|
||||
@ -504,7 +531,7 @@ void KeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr<nuraf
|
||||
last_committed_idx = log_idx;
|
||||
}
|
||||
|
||||
void KeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data)
|
||||
void IKeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data)
|
||||
{
|
||||
auto request_for_session = parseRequest(data, true);
|
||||
// If we received a log from an older node, use the log_idx as the zxid
|
||||
@ -516,7 +543,8 @@ void KeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data)
|
||||
rollbackRequest(*request_for_session, false);
|
||||
}
|
||||
|
||||
void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing)
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing)
|
||||
{
|
||||
if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID)
|
||||
return;
|
||||
@ -525,7 +553,8 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession
|
||||
storage->rollbackRequest(request_for_session.zxid, allow_missing);
|
||||
}
|
||||
|
||||
void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing)
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::rollbackRequestNoLock(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing)
|
||||
{
|
||||
if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID)
|
||||
return;
|
||||
@ -533,14 +562,15 @@ void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSe
|
||||
storage->rollbackRequest(request_for_session.zxid, allow_missing);
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::snapshot> KeeperStateMachine::last_snapshot()
|
||||
nuraft::ptr<nuraft::snapshot> IKeeperStateMachine::last_snapshot()
|
||||
{
|
||||
/// Just return the latest snapshot.
|
||||
std::lock_guard lock(snapshots_lock);
|
||||
return latest_snapshot_meta;
|
||||
}
|
||||
|
||||
void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done)
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done)
|
||||
{
|
||||
LOG_DEBUG(log, "Creating snapshot {}", s.get_last_log_idx());
|
||||
|
||||
@ -549,12 +579,13 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res
|
||||
CreateSnapshotTask snapshot_task;
|
||||
{ /// lock storage for a short period time to turn on "snapshot mode". After that we can read consistent storage state without locking.
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
snapshot_task.snapshot = std::make_shared<KeeperStorageSnapshot>(storage.get(), snapshot_meta_copy, getClusterConfig());
|
||||
snapshot_task.snapshot = std::make_shared<KeeperStorageSnapshot<Storage>>(storage.get(), snapshot_meta_copy, getClusterConfig());
|
||||
}
|
||||
|
||||
/// create snapshot task for background execution (in snapshot thread)
|
||||
snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot)
|
||||
snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot_)
|
||||
{
|
||||
auto * snapshot = typeid_cast<KeeperStorageSnapshot<Storage> *>(snapshot_.get());
|
||||
nuraft::ptr<std::exception> exception(nullptr);
|
||||
bool ret = true;
|
||||
try
|
||||
@ -601,7 +632,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res
|
||||
/// Turn off "snapshot mode" and clear outdate part of storage state
|
||||
storage->clearGarbageAfterSnapshot();
|
||||
LOG_TRACE(log, "Cleared garbage after snapshot");
|
||||
snapshot.reset();
|
||||
snapshot_.reset();
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
@ -638,7 +669,8 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res
|
||||
LOG_WARNING(log, "Cannot push snapshot task into queue");
|
||||
}
|
||||
|
||||
void KeeperStateMachine::save_logical_snp_obj(
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::save_logical_snp_obj(
|
||||
nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool /*is_first_obj*/, bool /*is_last_obj*/)
|
||||
{
|
||||
LOG_DEBUG(log, "Saving snapshot {} obj_id {}", s.get_last_log_idx(), obj_id);
|
||||
@ -703,7 +735,7 @@ static int bufferFromFile(Poco::Logger * log, const std::string & path, nuraft::
|
||||
return 0;
|
||||
}
|
||||
|
||||
int KeeperStateMachine::read_logical_snp_obj(
|
||||
int IKeeperStateMachine::read_logical_snp_obj(
|
||||
nuraft::snapshot & s, void *& /*user_snp_ctx*/, uint64_t obj_id, nuraft::ptr<nuraft::buffer> & data_out, bool & is_last_obj)
|
||||
{
|
||||
LOG_DEBUG(log, "Reading snapshot {} obj_id {}", s.get_last_log_idx(), obj_id);
|
||||
@ -743,7 +775,8 @@ int KeeperStateMachine::read_logical_snp_obj(
|
||||
return 1;
|
||||
}
|
||||
|
||||
void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSession & request_for_session)
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
/// Pure local request, just process it with storage
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
@ -754,103 +787,120 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi
|
||||
LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response.session_id);
|
||||
}
|
||||
|
||||
void KeeperStateMachine::shutdownStorage()
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::shutdownStorage()
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
storage->finalize();
|
||||
}
|
||||
|
||||
std::vector<int64_t> KeeperStateMachine::getDeadSessions()
|
||||
template<typename Storage>
|
||||
std::vector<int64_t> KeeperStateMachine<Storage>::getDeadSessions()
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getDeadSessions();
|
||||
}
|
||||
|
||||
int64_t KeeperStateMachine::getNextZxid() const
|
||||
template<typename Storage>
|
||||
int64_t KeeperStateMachine<Storage>::getNextZxid() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getNextZXID();
|
||||
}
|
||||
|
||||
KeeperStorage::Digest KeeperStateMachine::getNodesDigest() const
|
||||
template<typename Storage>
|
||||
KeeperStorageBase::Digest KeeperStateMachine<Storage>::getNodesDigest() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getNodesDigest(false);
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getLastProcessedZxid() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getLastProcessedZxid() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getZXID();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getNodesCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getNodesCount() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getNodesCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getTotalWatchesCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getTotalWatchesCount() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getTotalWatchesCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getWatchedPathsCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getWatchedPathsCount() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getWatchedPathsCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getSessionsWithWatchesCount() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getSessionsWithWatchesCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getTotalEphemeralNodesCount() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getTotalEphemeralNodesCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getSessionWithEphemeralNodesCount() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getSessionWithEphemeralNodesCount();
|
||||
}
|
||||
|
||||
void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::dumpWatches(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
storage->dumpWatches(buf);
|
||||
}
|
||||
|
||||
void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::dumpWatchesByPath(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
storage->dumpWatchesByPath(buf);
|
||||
}
|
||||
|
||||
void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
storage->dumpSessionsAndEphemerals(buf);
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getApproximateDataSize() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getApproximateDataSize() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getApproximateDataSize();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getKeyArenaSize() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getKeyArenaSize() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getArenaDataSize();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getLatestSnapshotBufSize() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getLatestSnapshotBufSize() const
|
||||
{
|
||||
std::lock_guard lock(snapshots_lock);
|
||||
if (latest_snapshot_buf)
|
||||
@ -858,7 +908,7 @@ uint64_t KeeperStateMachine::getLatestSnapshotBufSize() const
|
||||
return 0;
|
||||
}
|
||||
|
||||
ClusterConfigPtr KeeperStateMachine::getClusterConfig() const
|
||||
ClusterConfigPtr IKeeperStateMachine::getClusterConfig() const
|
||||
{
|
||||
std::lock_guard lock(cluster_config_lock);
|
||||
if (cluster_config)
|
||||
@ -870,11 +920,15 @@ ClusterConfigPtr KeeperStateMachine::getClusterConfig() const
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
void KeeperStateMachine::recalculateStorageStats()
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::recalculateStorageStats()
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
LOG_INFO(log, "Recalculating storage stats");
|
||||
storage->recalculateStats();
|
||||
LOG_INFO(log, "Done recalculating storage stats");
|
||||
}
|
||||
|
||||
template class KeeperStateMachine<KeeperMemoryStorage>;
|
||||
|
||||
}
|
||||
|
@ -12,27 +12,25 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
using ResponsesQueue = ConcurrentBoundedQueue<KeeperStorage::ResponseForSession>;
|
||||
using ResponsesQueue = ConcurrentBoundedQueue<KeeperStorageBase::ResponseForSession>;
|
||||
using SnapshotsQueue = ConcurrentBoundedQueue<CreateSnapshotTask>;
|
||||
|
||||
/// ClickHouse Keeper state machine. Wrapper for KeeperStorage.
|
||||
/// Responsible for entries commit, snapshots creation and so on.
|
||||
class KeeperStateMachine : public nuraft::state_machine
|
||||
class IKeeperStateMachine : public nuraft::state_machine
|
||||
{
|
||||
public:
|
||||
using CommitCallback = std::function<void(uint64_t, const KeeperStorage::RequestForSession &)>;
|
||||
using CommitCallback = std::function<void(uint64_t, const KeeperStorageBase::RequestForSession &)>;
|
||||
|
||||
KeeperStateMachine(
|
||||
IKeeperStateMachine(
|
||||
ResponsesQueue & responses_queue_,
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
const CoordinationSettingsPtr & coordination_settings_,
|
||||
const KeeperContextPtr & keeper_context_,
|
||||
KeeperSnapshotManagerS3 * snapshot_manager_s3_,
|
||||
CommitCallback commit_callback_ = {},
|
||||
const std::string & superdigest_ = "");
|
||||
CommitCallback commit_callback_,
|
||||
const std::string & superdisgest_);
|
||||
|
||||
/// Read state from the latest snapshot
|
||||
void init();
|
||||
virtual void init() = 0;
|
||||
|
||||
enum ZooKeeperLogSerializationVersion
|
||||
{
|
||||
@ -49,89 +47,66 @@ public:
|
||||
///
|
||||
/// final - whether it's the final time we will fetch the request so we can safely remove it from cache
|
||||
/// serialization_version - information about which fields were parsed from the buffer so we can modify the buffer accordingly
|
||||
std::shared_ptr<KeeperStorage::RequestForSession> parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version = nullptr);
|
||||
std::shared_ptr<KeeperStorageBase::RequestForSession> parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version = nullptr);
|
||||
|
||||
bool preprocess(const KeeperStorage::RequestForSession & request_for_session);
|
||||
virtual bool preprocess(const KeeperStorageBase::RequestForSession & request_for_session) = 0;
|
||||
|
||||
nuraft::ptr<nuraft::buffer> pre_commit(uint64_t log_idx, nuraft::buffer & data) override;
|
||||
|
||||
nuraft::ptr<nuraft::buffer> commit(const uint64_t log_idx, nuraft::buffer & data) override; /// NOLINT
|
||||
|
||||
/// Save new cluster config to our snapshot (copy of the config stored in StateManager)
|
||||
void commit_config(const uint64_t log_idx, nuraft::ptr<nuraft::cluster_config> & new_conf) override; /// NOLINT
|
||||
|
||||
void rollback(uint64_t log_idx, nuraft::buffer & data) override;
|
||||
|
||||
// allow_missing - whether the transaction we want to rollback can be missing from storage
|
||||
// (can happen in case of exception during preprocessing)
|
||||
void rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing);
|
||||
|
||||
void rollbackRequestNoLock(
|
||||
const KeeperStorage::RequestForSession & request_for_session,
|
||||
bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS;
|
||||
virtual void rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) = 0;
|
||||
|
||||
uint64_t last_commit_index() override { return last_committed_idx; }
|
||||
|
||||
/// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state.
|
||||
bool apply_snapshot(nuraft::snapshot & s) override;
|
||||
|
||||
nuraft::ptr<nuraft::snapshot> last_snapshot() override;
|
||||
|
||||
/// Create new snapshot from current state.
|
||||
void create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done) override;
|
||||
void create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done) override = 0;
|
||||
|
||||
/// Save snapshot which was send by leader to us. After that we will apply it in apply_snapshot.
|
||||
void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override;
|
||||
void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override = 0;
|
||||
|
||||
/// Better name is `serialize snapshot` -- save existing snapshot (created by create_snapshot) into
|
||||
/// in-memory buffer data_out.
|
||||
int read_logical_snp_obj(
|
||||
nuraft::snapshot & s, void *& user_snp_ctx, uint64_t obj_id, nuraft::ptr<nuraft::buffer> & data_out, bool & is_last_obj) override;
|
||||
|
||||
// This should be used only for tests or keeper-data-dumper because it violates
|
||||
// TSA -- we can't acquire the lock outside of this class or return a storage under lock
|
||||
// in a reasonable way.
|
||||
KeeperStorage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS
|
||||
{
|
||||
return *storage;
|
||||
}
|
||||
|
||||
void shutdownStorage();
|
||||
virtual void shutdownStorage() = 0;
|
||||
|
||||
ClusterConfigPtr getClusterConfig() const;
|
||||
|
||||
/// Process local read request
|
||||
void processReadRequest(const KeeperStorage::RequestForSession & request_for_session);
|
||||
virtual void processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) = 0;
|
||||
|
||||
std::vector<int64_t> getDeadSessions();
|
||||
virtual std::vector<int64_t> getDeadSessions() = 0;
|
||||
|
||||
int64_t getNextZxid() const;
|
||||
virtual int64_t getNextZxid() const = 0;
|
||||
|
||||
KeeperStorage::Digest getNodesDigest() const;
|
||||
virtual KeeperStorageBase::Digest getNodesDigest() const = 0;
|
||||
|
||||
/// Introspection functions for 4lw commands
|
||||
uint64_t getLastProcessedZxid() const;
|
||||
virtual uint64_t getLastProcessedZxid() const = 0;
|
||||
|
||||
uint64_t getNodesCount() const;
|
||||
uint64_t getTotalWatchesCount() const;
|
||||
uint64_t getWatchedPathsCount() const;
|
||||
uint64_t getSessionsWithWatchesCount() const;
|
||||
virtual uint64_t getNodesCount() const = 0;
|
||||
virtual uint64_t getTotalWatchesCount() const = 0;
|
||||
virtual uint64_t getWatchedPathsCount() const = 0;
|
||||
virtual uint64_t getSessionsWithWatchesCount() const = 0;
|
||||
|
||||
void dumpWatches(WriteBufferFromOwnString & buf) const;
|
||||
void dumpWatchesByPath(WriteBufferFromOwnString & buf) const;
|
||||
void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const;
|
||||
virtual void dumpWatches(WriteBufferFromOwnString & buf) const = 0;
|
||||
virtual void dumpWatchesByPath(WriteBufferFromOwnString & buf) const = 0;
|
||||
virtual void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const = 0;
|
||||
|
||||
uint64_t getSessionWithEphemeralNodesCount() const;
|
||||
uint64_t getTotalEphemeralNodesCount() const;
|
||||
uint64_t getApproximateDataSize() const;
|
||||
uint64_t getKeyArenaSize() const;
|
||||
uint64_t getLatestSnapshotBufSize() const;
|
||||
virtual uint64_t getSessionWithEphemeralNodesCount() const = 0;
|
||||
virtual uint64_t getTotalEphemeralNodesCount() const = 0;
|
||||
virtual uint64_t getApproximateDataSize() const = 0;
|
||||
virtual uint64_t getKeyArenaSize() const = 0;
|
||||
virtual uint64_t getLatestSnapshotBufSize() const = 0;
|
||||
|
||||
void recalculateStorageStats();
|
||||
virtual void recalculateStorageStats() = 0;
|
||||
|
||||
void reconfigure(const KeeperStorage::RequestForSession& request_for_session);
|
||||
virtual void reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) = 0;
|
||||
|
||||
private:
|
||||
protected:
|
||||
CommitCallback commit_callback;
|
||||
/// In our state machine we always have a single snapshot which is stored
|
||||
/// in memory in compressed (serialized) format.
|
||||
@ -141,12 +116,7 @@ private:
|
||||
|
||||
CoordinationSettingsPtr coordination_settings;
|
||||
|
||||
/// Main state machine logic
|
||||
KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock);
|
||||
|
||||
/// Save/Load and Serialize/Deserialize logic for snapshots.
|
||||
KeeperSnapshotManager snapshot_manager;
|
||||
|
||||
/// Put processed responses into this queue
|
||||
ResponsesQueue & responses_queue;
|
||||
|
||||
@ -163,7 +133,7 @@ private:
|
||||
/// for request.
|
||||
mutable std::mutex storage_and_responses_lock;
|
||||
|
||||
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, std::shared_ptr<KeeperStorage::RequestForSession>>> parsed_request_cache;
|
||||
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, std::shared_ptr<KeeperStorageBase::RequestForSession>>> parsed_request_cache;
|
||||
uint64_t min_request_size_to_cache{0};
|
||||
/// we only need to protect the access to the map itself
|
||||
/// requests can be modified from anywhere without lock because a single request
|
||||
@ -188,8 +158,105 @@ private:
|
||||
|
||||
KeeperSnapshotManagerS3 * snapshot_manager_s3;
|
||||
|
||||
KeeperStorage::ResponseForSession processReconfiguration(
|
||||
const KeeperStorage::RequestForSession& request_for_session)
|
||||
TSA_REQUIRES(storage_and_responses_lock);
|
||||
virtual KeeperStorageBase::ResponseForSession processReconfiguration(
|
||||
const KeeperStorageBase::RequestForSession& request_for_session)
|
||||
TSA_REQUIRES(storage_and_responses_lock) = 0;
|
||||
|
||||
};
|
||||
|
||||
/// ClickHouse Keeper state machine. Wrapper for KeeperStorage.
|
||||
/// Responsible for entries commit, snapshots creation and so on.
|
||||
template<typename Storage>
|
||||
class KeeperStateMachine : public IKeeperStateMachine
|
||||
{
|
||||
public:
|
||||
/// using CommitCallback = std::function<void(uint64_t, const KeeperStorage::RequestForSession &)>;
|
||||
|
||||
KeeperStateMachine(
|
||||
ResponsesQueue & responses_queue_,
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
const CoordinationSettingsPtr & coordination_settings_,
|
||||
const KeeperContextPtr & keeper_context_,
|
||||
KeeperSnapshotManagerS3 * snapshot_manager_s3_,
|
||||
CommitCallback commit_callback_ = {},
|
||||
const std::string & superdigest_ = "");
|
||||
|
||||
/// Read state from the latest snapshot
|
||||
void init() override;
|
||||
|
||||
bool preprocess(const KeeperStorageBase::RequestForSession & request_for_session) override;
|
||||
|
||||
nuraft::ptr<nuraft::buffer> pre_commit(uint64_t log_idx, nuraft::buffer & data) override;
|
||||
|
||||
nuraft::ptr<nuraft::buffer> commit(const uint64_t log_idx, nuraft::buffer & data) override; /// NOLINT
|
||||
|
||||
// allow_missing - whether the transaction we want to rollback can be missing from storage
|
||||
// (can happen in case of exception during preprocessing)
|
||||
void rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) override;
|
||||
|
||||
void rollbackRequestNoLock(
|
||||
const KeeperStorageBase::RequestForSession & request_for_session,
|
||||
bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS;
|
||||
|
||||
/// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state.
|
||||
bool apply_snapshot(nuraft::snapshot & s) override;
|
||||
|
||||
/// Create new snapshot from current state.
|
||||
void create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done) override;
|
||||
|
||||
/// Save snapshot which was send by leader to us. After that we will apply it in apply_snapshot.
|
||||
void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override;
|
||||
|
||||
// This should be used only for tests or keeper-data-dumper because it violates
|
||||
// TSA -- we can't acquire the lock outside of this class or return a storage under lock
|
||||
// in a reasonable way.
|
||||
Storage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS
|
||||
{
|
||||
return *storage;
|
||||
}
|
||||
|
||||
void shutdownStorage() override;
|
||||
|
||||
/// Process local read request
|
||||
void processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) override;
|
||||
|
||||
std::vector<int64_t> getDeadSessions() override;
|
||||
|
||||
int64_t getNextZxid() const override;
|
||||
|
||||
KeeperStorageBase::Digest getNodesDigest() const override;
|
||||
|
||||
/// Introspection functions for 4lw commands
|
||||
uint64_t getLastProcessedZxid() const override;
|
||||
|
||||
uint64_t getNodesCount() const override;
|
||||
uint64_t getTotalWatchesCount() const override;
|
||||
uint64_t getWatchedPathsCount() const override;
|
||||
uint64_t getSessionsWithWatchesCount() const override;
|
||||
|
||||
void dumpWatches(WriteBufferFromOwnString & buf) const override;
|
||||
void dumpWatchesByPath(WriteBufferFromOwnString & buf) const override;
|
||||
void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const override;
|
||||
|
||||
uint64_t getSessionWithEphemeralNodesCount() const override;
|
||||
uint64_t getTotalEphemeralNodesCount() const override;
|
||||
uint64_t getApproximateDataSize() const override;
|
||||
uint64_t getKeyArenaSize() const override;
|
||||
uint64_t getLatestSnapshotBufSize() const override;
|
||||
|
||||
void recalculateStorageStats() override;
|
||||
|
||||
void reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) override;
|
||||
|
||||
private:
|
||||
/// Main state machine logic
|
||||
std::unique_ptr<Storage> storage; //TSA_PT_GUARDED_BY(storage_and_responses_lock);
|
||||
|
||||
/// Save/Load and Serialize/Deserialize logic for snapshots.
|
||||
KeeperSnapshotManager<Storage> snapshot_manager;
|
||||
|
||||
KeeperStorageBase::ResponseForSession processReconfiguration(
|
||||
const KeeperStorageBase::RequestForSession& request_for_session)
|
||||
TSA_REQUIRES(storage_and_responses_lock) override;
|
||||
};
|
||||
}
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -16,21 +16,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct KeeperStorageRequestProcessor;
|
||||
using KeeperStorageRequestProcessorPtr = std::shared_ptr<KeeperStorageRequestProcessor>;
|
||||
using ResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr &)>;
|
||||
using ChildrenSet = absl::flat_hash_set<StringRef, StringRefHash>;
|
||||
using SessionAndTimeout = std::unordered_map<int64_t, int64_t>;
|
||||
|
||||
struct KeeperStorageSnapshot;
|
||||
|
||||
/// 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.
|
||||
class KeeperStorage
|
||||
{
|
||||
public:
|
||||
struct Node
|
||||
struct KeeperMemNode
|
||||
{
|
||||
uint64_t acl_id = 0; /// 0 -- no ACL by default
|
||||
bool is_sequental = false;
|
||||
@ -38,7 +28,7 @@ public:
|
||||
int32_t seq_num = 0;
|
||||
uint64_t size_bytes; // save size to avoid calculate every time
|
||||
|
||||
Node() : size_bytes(sizeof(Node)) { }
|
||||
KeeperMemNode() : size_bytes(sizeof(KeeperMemNode)) { }
|
||||
|
||||
/// Object memory size
|
||||
uint64_t sizeInBytes() const { return size_bytes; }
|
||||
@ -62,7 +52,7 @@ public:
|
||||
|
||||
// copy only necessary information for preprocessing and digest calculation
|
||||
// (e.g. we don't need to copy list of children)
|
||||
void shallowCopy(const Node & other);
|
||||
void shallowCopy(const KeeperMemNode & other);
|
||||
|
||||
void recalculateSize();
|
||||
|
||||
@ -72,6 +62,9 @@ public:
|
||||
mutable std::optional<UInt64> cached_digest;
|
||||
};
|
||||
|
||||
class KeeperStorageBase
|
||||
{
|
||||
public:
|
||||
enum DigestVersion : uint8_t
|
||||
{
|
||||
NO_DIGEST = 0,
|
||||
@ -79,7 +72,11 @@ public:
|
||||
V2 = 2 // added system nodes that modify the digest on startup so digest from V0 is invalid
|
||||
};
|
||||
|
||||
static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V2;
|
||||
struct Digest
|
||||
{
|
||||
DigestVersion version{DigestVersion::NO_DIGEST};
|
||||
uint64_t value{0};
|
||||
};
|
||||
|
||||
struct ResponseForSession
|
||||
{
|
||||
@ -88,11 +85,34 @@ public:
|
||||
};
|
||||
using ResponsesForSessions = std::vector<ResponseForSession>;
|
||||
|
||||
struct Digest
|
||||
struct RequestForSession
|
||||
{
|
||||
DigestVersion version{DigestVersion::NO_DIGEST};
|
||||
uint64_t value{0};
|
||||
int64_t session_id;
|
||||
int64_t time{0};
|
||||
Coordination::ZooKeeperRequestPtr request;
|
||||
int64_t zxid{0};
|
||||
std::optional<Digest> digest;
|
||||
int64_t log_idx{0};
|
||||
};
|
||||
using RequestsForSessions = std::vector<RequestForSession>;
|
||||
|
||||
struct AuthID
|
||||
{
|
||||
std::string scheme;
|
||||
std::string id;
|
||||
|
||||
bool operator==(const AuthID & other) const { return scheme == other.scheme && id == other.id; }
|
||||
};
|
||||
|
||||
// using Container = SnapshotableHashTable<Node>;
|
||||
using Ephemerals = std::unordered_map<int64_t, std::unordered_set<std::string>>;
|
||||
using SessionAndWatcher = std::unordered_map<int64_t, std::unordered_set<std::string>>;
|
||||
using SessionIDs = std::unordered_set<int64_t>;
|
||||
|
||||
/// Just vector of SHA1 from user:password
|
||||
using AuthIDs = std::vector<AuthID>;
|
||||
using SessionAndAuth = std::unordered_map<int64_t, AuthIDs>;
|
||||
using Watches = std::unordered_map<String /* path, relative of root_path */, SessionIDs>;
|
||||
|
||||
static bool checkDigest(const Digest & first, const Digest & second)
|
||||
{
|
||||
@ -105,38 +125,22 @@ public:
|
||||
return first.value == second.value;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
/// 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.
|
||||
template<typename Container_>
|
||||
class KeeperStorage : public KeeperStorageBase
|
||||
{
|
||||
public:
|
||||
using Container = Container_;
|
||||
using Node = Container::Node;
|
||||
|
||||
static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V2;
|
||||
|
||||
static String generateDigest(const String & userdata);
|
||||
|
||||
struct RequestForSession
|
||||
{
|
||||
int64_t session_id;
|
||||
int64_t time{0};
|
||||
Coordination::ZooKeeperRequestPtr request;
|
||||
int64_t zxid{0};
|
||||
std::optional<Digest> digest;
|
||||
int64_t log_idx{0};
|
||||
};
|
||||
|
||||
struct AuthID
|
||||
{
|
||||
std::string scheme;
|
||||
std::string id;
|
||||
|
||||
bool operator==(const AuthID & other) const { return scheme == other.scheme && id == other.id; }
|
||||
};
|
||||
|
||||
using RequestsForSessions = std::vector<RequestForSession>;
|
||||
|
||||
using Container = SnapshotableHashTable<Node>;
|
||||
using Ephemerals = std::unordered_map<int64_t, std::unordered_set<std::string>>;
|
||||
using SessionAndWatcher = std::unordered_map<int64_t, std::unordered_set<std::string>>;
|
||||
using SessionIDs = std::unordered_set<int64_t>;
|
||||
|
||||
/// Just vector of SHA1 from user:password
|
||||
using AuthIDs = std::vector<AuthID>;
|
||||
using SessionAndAuth = std::unordered_map<int64_t, AuthIDs>;
|
||||
using Watches = std::unordered_map<String /* path, relative of root_path */, SessionIDs>;
|
||||
|
||||
int64_t session_id_counter{1};
|
||||
|
||||
SessionAndAuth session_and_auth;
|
||||
@ -307,7 +311,7 @@ public:
|
||||
std::unordered_map<std::string, std::list<const Delta *>, Hash, Equal> deltas_for_path;
|
||||
|
||||
std::list<Delta> deltas;
|
||||
KeeperStorage & storage;
|
||||
KeeperStorage<Container> & storage;
|
||||
};
|
||||
|
||||
UncommittedState uncommitted_state{*this};
|
||||
@ -487,6 +491,6 @@ private:
|
||||
void addDigest(const Node & node, std::string_view path);
|
||||
};
|
||||
|
||||
using KeeperStoragePtr = std::unique_ptr<KeeperStorage>;
|
||||
using KeeperMemoryStorage = KeeperStorage<SnapshotableHashTable<KeeperMemNode>>;
|
||||
|
||||
}
|
||||
|
@ -121,6 +121,7 @@ private:
|
||||
|
||||
public:
|
||||
|
||||
using Node = V;
|
||||
using iterator = typename List::iterator;
|
||||
using const_iterator = typename List::const_iterator;
|
||||
using ValueUpdater = std::function<void(V & value)>;
|
||||
|
@ -43,7 +43,8 @@ void deserializeSnapshotMagic(ReadBuffer & in)
|
||||
throw Exception(ErrorCodes::CORRUPTED_DATA, "Incorrect magic header in file, expected {}, got {}", SNP_HEADER, magic_header);
|
||||
}
|
||||
|
||||
int64_t deserializeSessionAndTimeout(KeeperStorage & storage, ReadBuffer & in)
|
||||
template<typename Storage>
|
||||
int64_t deserializeSessionAndTimeout(Storage & storage, ReadBuffer & in)
|
||||
{
|
||||
int32_t count;
|
||||
Coordination::read(count, in);
|
||||
@ -62,7 +63,8 @@ int64_t deserializeSessionAndTimeout(KeeperStorage & storage, ReadBuffer & in)
|
||||
return max_session_id;
|
||||
}
|
||||
|
||||
void deserializeACLMap(KeeperStorage & storage, ReadBuffer & in)
|
||||
template<typename Storage>
|
||||
void deserializeACLMap(Storage & storage, ReadBuffer & in)
|
||||
{
|
||||
int32_t count;
|
||||
Coordination::read(count, in);
|
||||
@ -90,7 +92,8 @@ void deserializeACLMap(KeeperStorage & storage, ReadBuffer & in)
|
||||
}
|
||||
}
|
||||
|
||||
int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * log)
|
||||
template<typename Storage>
|
||||
int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, Poco::Logger * log)
|
||||
{
|
||||
int64_t max_zxid = 0;
|
||||
std::string path;
|
||||
@ -98,7 +101,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L
|
||||
size_t count = 0;
|
||||
while (path != "/")
|
||||
{
|
||||
KeeperStorage::Node node{};
|
||||
typename Storage::Node node{};
|
||||
String data;
|
||||
Coordination::read(data, in);
|
||||
node.setData(std::move(data));
|
||||
@ -140,14 +143,15 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L
|
||||
if (itr.key != "/")
|
||||
{
|
||||
auto parent_path = parentNodePath(itr.key);
|
||||
storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseNodeName(my_path)); ++value.stat.numChildren; });
|
||||
storage.container.updateValue(parent_path, [my_path = itr.key] (typename Storage::Node & value) { value.addChild(getBaseNodeName(my_path)); ++value.stat.numChildren; });
|
||||
}
|
||||
}
|
||||
|
||||
return max_zxid;
|
||||
}
|
||||
|
||||
void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log)
|
||||
template<typename Storage>
|
||||
void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, Poco::Logger * log)
|
||||
{
|
||||
LOG_INFO(log, "Deserializing storage snapshot {}", snapshot_path);
|
||||
int64_t zxid = getZxidFromName(snapshot_path);
|
||||
@ -186,7 +190,8 @@ void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::st
|
||||
LOG_INFO(log, "Finished, snapshot ZXID {}", storage.zxid);
|
||||
}
|
||||
|
||||
void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, Poco::Logger * log)
|
||||
template<typename Storage>
|
||||
void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, Poco::Logger * log)
|
||||
{
|
||||
namespace fs = std::filesystem;
|
||||
std::map<int64_t, std::string> existing_snapshots;
|
||||
@ -474,7 +479,8 @@ bool hasErrorsInMultiRequest(Coordination::ZooKeeperRequestPtr request)
|
||||
|
||||
}
|
||||
|
||||
bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*log*/)
|
||||
template<typename Storage>
|
||||
bool deserializeTxn(Storage & storage, ReadBuffer & in, Poco::Logger * /*log*/)
|
||||
{
|
||||
int64_t checksum;
|
||||
Coordination::read(checksum, in);
|
||||
@ -529,7 +535,8 @@ bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*l
|
||||
return true;
|
||||
}
|
||||
|
||||
void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, Poco::Logger * log)
|
||||
template<typename Storage>
|
||||
void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_path, Poco::Logger * log)
|
||||
{
|
||||
ReadBufferFromFile reader(log_path);
|
||||
|
||||
@ -553,7 +560,8 @@ void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string
|
||||
LOG_INFO(log, "Finished {} deserialization, totally read {} records", log_path, counter);
|
||||
}
|
||||
|
||||
void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, Poco::Logger * log)
|
||||
template<typename Storage>
|
||||
void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, Poco::Logger * log)
|
||||
{
|
||||
namespace fs = std::filesystem;
|
||||
std::map<int64_t, std::string> existing_logs;
|
||||
@ -589,4 +597,9 @@ void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string
|
||||
}
|
||||
}
|
||||
|
||||
template void deserializeKeeperStorageFromSnapshot<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & snapshot_path, Poco::Logger * log);
|
||||
template void deserializeKeeperStorageFromSnapshotsDir<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & path, Poco::Logger * log);
|
||||
template void deserializeLogAndApplyToStorage<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & log_path, Poco::Logger * log);
|
||||
template void deserializeLogsAndApplyToStorage<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & path, Poco::Logger * log);
|
||||
|
||||
}
|
||||
|
@ -5,12 +5,16 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log);
|
||||
template<typename Storage>
|
||||
void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, Poco::Logger * log);
|
||||
|
||||
void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, Poco::Logger * log);
|
||||
template<typename Storage>
|
||||
void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, Poco::Logger * log);
|
||||
|
||||
void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, Poco::Logger * log);
|
||||
template<typename Storage>
|
||||
void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_path, Poco::Logger * log);
|
||||
|
||||
void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, Poco::Logger * log);
|
||||
template<typename Storage>
|
||||
void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, Poco::Logger * log);
|
||||
|
||||
}
|
||||
|
@ -1320,7 +1320,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize)
|
||||
EXPECT_EQ(hello.getApproximateDataSize(), 0);
|
||||
|
||||
/// Node
|
||||
using Node = DB::KeeperStorage::Node;
|
||||
using Node = DB::KeeperMemoryStorage::Node;
|
||||
DB::SnapshotableHashTable<Node> world;
|
||||
Node n1;
|
||||
n1.setData("1234");
|
||||
@ -1359,9 +1359,9 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize)
|
||||
EXPECT_EQ(world.getApproximateDataSize(), 0);
|
||||
}
|
||||
|
||||
void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0)
|
||||
void addNode(DB::KeeperMemoryStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0)
|
||||
{
|
||||
using Node = DB::KeeperStorage::Node;
|
||||
using Node = DB::KeeperMemoryStorage::Node;
|
||||
Node node{};
|
||||
node.setData(data);
|
||||
node.stat.ephemeralOwner = ephemeral_owner;
|
||||
@ -1383,9 +1383,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple)
|
||||
ChangelogDirTest test("./snapshots");
|
||||
setSnapshotDirectory("./snapshots");
|
||||
|
||||
DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression);
|
||||
DB::KeeperSnapshotManager<DB::KeeperMemoryStorage> manager(3, keeper_context, params.enable_compression);
|
||||
|
||||
DB::KeeperStorage storage(500, "", keeper_context);
|
||||
DB::KeeperMemoryStorage storage(500, "", keeper_context);
|
||||
addNode(storage, "/hello", "world", 1);
|
||||
addNode(storage, "/hello/somepath", "somedata", 3);
|
||||
storage.session_id_counter = 5;
|
||||
@ -1395,7 +1395,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple)
|
||||
storage.getSessionID(130);
|
||||
storage.getSessionID(130);
|
||||
|
||||
DB::KeeperStorageSnapshot snapshot(&storage, 2);
|
||||
DB::KeeperStorageSnapshot<DB::KeeperMemoryStorage> snapshot(&storage, 2);
|
||||
|
||||
EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 2);
|
||||
EXPECT_EQ(snapshot.session_id, 7);
|
||||
@ -1433,9 +1433,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites)
|
||||
ChangelogDirTest test("./snapshots");
|
||||
setSnapshotDirectory("./snapshots");
|
||||
|
||||
DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression);
|
||||
DB::KeeperSnapshotManager<DB::KeeperMemoryStorage> manager(3, keeper_context, params.enable_compression);
|
||||
|
||||
DB::KeeperStorage storage(500, "", keeper_context);
|
||||
DB::KeeperMemoryStorage storage(500, "", keeper_context);
|
||||
storage.getSessionID(130);
|
||||
|
||||
for (size_t i = 0; i < 50; ++i)
|
||||
@ -1443,7 +1443,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites)
|
||||
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
|
||||
}
|
||||
|
||||
DB::KeeperStorageSnapshot snapshot(&storage, 50);
|
||||
DB::KeeperStorageSnapshot<DB::KeeperMemoryStorage> snapshot(&storage, 50);
|
||||
EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 50);
|
||||
EXPECT_EQ(snapshot.snapshot_container_size, 54);
|
||||
|
||||
@ -1476,9 +1476,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots)
|
||||
ChangelogDirTest test("./snapshots");
|
||||
setSnapshotDirectory("./snapshots");
|
||||
|
||||
DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression);
|
||||
DB::KeeperSnapshotManager<DB::KeeperMemoryStorage> manager(3, keeper_context, params.enable_compression);
|
||||
|
||||
DB::KeeperStorage storage(500, "", keeper_context);
|
||||
DB::KeeperMemoryStorage storage(500, "", keeper_context);
|
||||
storage.getSessionID(130);
|
||||
|
||||
for (size_t j = 1; j <= 5; ++j)
|
||||
@ -1488,7 +1488,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots)
|
||||
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
|
||||
}
|
||||
|
||||
DB::KeeperStorageSnapshot snapshot(&storage, j * 50);
|
||||
DB::KeeperStorageSnapshot<DB::KeeperMemoryStorage> snapshot(&storage, j * 50);
|
||||
auto buf = manager.serializeSnapshotToBuffer(snapshot);
|
||||
manager.serializeSnapshotBufferToDisk(*buf, j * 50);
|
||||
EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin" + params.extension));
|
||||
@ -1517,15 +1517,15 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode)
|
||||
ChangelogDirTest test("./snapshots");
|
||||
setSnapshotDirectory("./snapshots");
|
||||
|
||||
DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression);
|
||||
DB::KeeperStorage storage(500, "", keeper_context);
|
||||
DB::KeeperSnapshotManager<DB::KeeperMemoryStorage> manager(3, keeper_context, params.enable_compression);
|
||||
DB::KeeperMemoryStorage storage(500, "", keeper_context);
|
||||
for (size_t i = 0; i < 50; ++i)
|
||||
{
|
||||
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
|
||||
}
|
||||
|
||||
{
|
||||
DB::KeeperStorageSnapshot snapshot(&storage, 50);
|
||||
DB::KeeperStorageSnapshot<DB::KeeperMemoryStorage> snapshot(&storage, 50);
|
||||
for (size_t i = 0; i < 50; ++i)
|
||||
{
|
||||
addNode(storage, "/hello_" + std::to_string(i), "wlrd_" + std::to_string(i));
|
||||
@ -1571,14 +1571,14 @@ TEST_P(CoordinationTest, TestStorageSnapshotBroken)
|
||||
ChangelogDirTest test("./snapshots");
|
||||
setSnapshotDirectory("./snapshots");
|
||||
|
||||
DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression);
|
||||
DB::KeeperStorage storage(500, "", keeper_context);
|
||||
DB::KeeperSnapshotManager<DB::KeeperMemoryStorage> manager(3, keeper_context, params.enable_compression);
|
||||
DB::KeeperMemoryStorage storage(500, "", keeper_context);
|
||||
for (size_t i = 0; i < 50; ++i)
|
||||
{
|
||||
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
|
||||
}
|
||||
{
|
||||
DB::KeeperStorageSnapshot snapshot(&storage, 50);
|
||||
DB::KeeperStorageSnapshot<DB::KeeperMemoryStorage> snapshot(&storage, 50);
|
||||
auto buf = manager.serializeSnapshotToBuffer(snapshot);
|
||||
manager.serializeSnapshotBufferToDisk(*buf, 50);
|
||||
}
|
||||
@ -1602,7 +1602,7 @@ nuraft::ptr<nuraft::buffer> getBufferFromZKRequest(int64_t session_id, int64_t z
|
||||
auto time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count();
|
||||
DB::writeIntBinary(time, buf);
|
||||
DB::writeIntBinary(zxid, buf);
|
||||
DB::writeIntBinary(DB::KeeperStorage::DigestVersion::NO_DIGEST, buf);
|
||||
DB::writeIntBinary(DB::KeeperMemoryStorage::DigestVersion::NO_DIGEST, buf);
|
||||
return buf.getBuffer();
|
||||
}
|
||||
|
||||
@ -1629,7 +1629,7 @@ void testLogAndStateMachine(
|
||||
|
||||
ResponsesQueue queue(std::numeric_limits<size_t>::max());
|
||||
SnapshotsQueue snapshots_queue{1};
|
||||
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, settings, keeper_context, nullptr);
|
||||
auto state_machine = std::make_shared<KeeperStateMachine<DB::KeeperMemoryStorage>>(queue, snapshots_queue, settings, keeper_context, nullptr);
|
||||
state_machine->init();
|
||||
DB::KeeperLogStore changelog(
|
||||
DB::LogFileSettings{
|
||||
@ -1672,7 +1672,7 @@ void testLogAndStateMachine(
|
||||
}
|
||||
|
||||
SnapshotsQueue snapshots_queue1{1};
|
||||
auto restore_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue1, settings, keeper_context, nullptr);
|
||||
auto restore_machine = std::make_shared<KeeperStateMachine<DB::KeeperMemoryStorage>>(queue, snapshots_queue1, settings, keeper_context, nullptr);
|
||||
restore_machine->init();
|
||||
EXPECT_EQ(restore_machine->last_commit_index(), total_logs - total_logs % settings->snapshot_distance);
|
||||
|
||||
@ -1791,7 +1791,7 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove)
|
||||
|
||||
ResponsesQueue queue(std::numeric_limits<size_t>::max());
|
||||
SnapshotsQueue snapshots_queue{1};
|
||||
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, settings, keeper_context, nullptr);
|
||||
auto state_machine = std::make_shared<KeeperStateMachine<DB::KeeperMemoryStorage>>(queue, snapshots_queue, settings, keeper_context, nullptr);
|
||||
state_machine->init();
|
||||
|
||||
std::shared_ptr<ZooKeeperCreateRequest> request_c = std::make_shared<ZooKeeperCreateRequest>();
|
||||
@ -1825,11 +1825,11 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte
|
||||
ResponsesQueue queue(std::numeric_limits<size_t>::max());
|
||||
SnapshotsQueue snapshots_queue{1};
|
||||
|
||||
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, settings, keeper_context, nullptr);
|
||||
auto state_machine = std::make_shared<KeeperStateMachine<DB::KeeperMemoryStorage>>(queue, snapshots_queue, settings, keeper_context, nullptr);
|
||||
state_machine->init();
|
||||
|
||||
String user_auth_data = "test_user:test_password";
|
||||
String digest = KeeperStorage::generateDigest(user_auth_data);
|
||||
String digest = KeeperMemoryStorage::generateDigest(user_auth_data);
|
||||
|
||||
std::shared_ptr<ZooKeeperAuthRequest> auth_req = std::make_shared<ZooKeeperAuthRequest>();
|
||||
auth_req->scheme = "digest";
|
||||
@ -1877,11 +1877,11 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted)
|
||||
ResponsesQueue queue(std::numeric_limits<size_t>::max());
|
||||
SnapshotsQueue snapshots_queue{1};
|
||||
|
||||
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, settings, keeper_context, nullptr);
|
||||
auto state_machine = std::make_shared<KeeperStateMachine<DB::KeeperMemoryStorage>>(queue, snapshots_queue, settings, keeper_context, nullptr);
|
||||
state_machine->init();
|
||||
|
||||
String user_auth_data = "test_user:test_password";
|
||||
String digest = KeeperStorage::generateDigest(user_auth_data);
|
||||
String digest = KeeperMemoryStorage::generateDigest(user_auth_data);
|
||||
|
||||
std::shared_ptr<ZooKeeperAuthRequest> auth_req = std::make_shared<ZooKeeperAuthRequest>();
|
||||
auth_req->scheme = "digest";
|
||||
@ -2104,9 +2104,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions)
|
||||
ChangelogDirTest test("./snapshots");
|
||||
setSnapshotDirectory("./snapshots");
|
||||
|
||||
DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression);
|
||||
DB::KeeperSnapshotManager<DB::KeeperMemoryStorage> manager(3, keeper_context, params.enable_compression);
|
||||
|
||||
DB::KeeperStorage storage(500, "", keeper_context);
|
||||
DB::KeeperMemoryStorage storage(500, "", keeper_context);
|
||||
addNode(storage, "/hello", "world", 1);
|
||||
addNode(storage, "/hello/somepath", "somedata", 3);
|
||||
storage.session_id_counter = 5;
|
||||
@ -2116,13 +2116,13 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions)
|
||||
storage.getSessionID(130);
|
||||
storage.getSessionID(130);
|
||||
|
||||
DB::KeeperStorageSnapshot snapshot(&storage, 2);
|
||||
DB::KeeperStorageSnapshot<DB::KeeperMemoryStorage> snapshot(&storage, 2);
|
||||
|
||||
auto buf = manager.serializeSnapshotToBuffer(snapshot);
|
||||
manager.serializeSnapshotBufferToDisk(*buf, 2);
|
||||
EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + params.extension));
|
||||
|
||||
DB::KeeperSnapshotManager new_manager(3, keeper_context, !params.enable_compression);
|
||||
DB::KeeperSnapshotManager<DB::KeeperMemoryStorage> new_manager(3, keeper_context, !params.enable_compression);
|
||||
|
||||
auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2);
|
||||
|
||||
@ -2308,9 +2308,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotEqual)
|
||||
std::optional<UInt128> snapshot_hash;
|
||||
for (size_t i = 0; i < 15; ++i)
|
||||
{
|
||||
DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression);
|
||||
DB::KeeperSnapshotManager<DB::KeeperMemoryStorage> manager(3, keeper_context, params.enable_compression);
|
||||
|
||||
DB::KeeperStorage storage(500, "", keeper_context);
|
||||
DB::KeeperMemoryStorage storage(500, "", keeper_context);
|
||||
addNode(storage, "/hello", "");
|
||||
for (size_t j = 0; j < 5000; ++j)
|
||||
{
|
||||
@ -2326,7 +2326,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotEqual)
|
||||
for (size_t j = 0; j < 3333; ++j)
|
||||
storage.getSessionID(130 * j);
|
||||
|
||||
DB::KeeperStorageSnapshot snapshot(&storage, storage.zxid);
|
||||
DB::KeeperStorageSnapshot<DB::KeeperMemoryStorage> snapshot(&storage, storage.zxid);
|
||||
|
||||
auto buf = manager.serializeSnapshotToBuffer(snapshot);
|
||||
|
||||
@ -2389,7 +2389,7 @@ TEST_P(CoordinationTest, TestUncommittedStateBasicCrud)
|
||||
using namespace DB;
|
||||
using namespace Coordination;
|
||||
|
||||
DB::KeeperStorage storage{500, "", keeper_context};
|
||||
DB::KeeperMemoryStorage storage{500, "", keeper_context};
|
||||
|
||||
constexpr std::string_view path = "/test";
|
||||
|
||||
@ -2506,7 +2506,7 @@ TEST_P(CoordinationTest, TestListRequestTypes)
|
||||
using namespace DB;
|
||||
using namespace Coordination;
|
||||
|
||||
KeeperStorage storage{500, "", keeper_context};
|
||||
KeeperMemoryStorage storage{500, "", keeper_context};
|
||||
|
||||
int32_t zxid = 0;
|
||||
|
||||
@ -2660,7 +2660,7 @@ TEST_P(CoordinationTest, TestDurableState)
|
||||
TEST_P(CoordinationTest, TestFeatureFlags)
|
||||
{
|
||||
using namespace Coordination;
|
||||
KeeperStorage storage{500, "", keeper_context};
|
||||
KeeperMemoryStorage storage{500, "", keeper_context};
|
||||
auto request = std::make_shared<ZooKeeperGetRequest>();
|
||||
request->path = DB::keeper_api_feature_flags_path;
|
||||
auto responses = storage.processRequest(request, 0, std::nullopt, true, true);
|
||||
@ -2679,7 +2679,7 @@ TEST_P(CoordinationTest, TestSystemNodeModify)
|
||||
|
||||
// On INIT we abort when a system path is modified
|
||||
keeper_context->setServerState(KeeperContext::Phase::RUNNING);
|
||||
KeeperStorage storage{500, "", keeper_context};
|
||||
KeeperMemoryStorage storage{500, "", keeper_context};
|
||||
const auto assert_create = [&](const std::string_view path, const auto expected_code)
|
||||
{
|
||||
auto request = std::make_shared<ZooKeeperCreateRequest>();
|
||||
@ -2771,7 +2771,7 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest)
|
||||
using namespace DB;
|
||||
using namespace Coordination;
|
||||
|
||||
KeeperStorage storage{500, "", keeper_context};
|
||||
KeeperMemoryStorage storage{500, "", keeper_context};
|
||||
|
||||
int32_t zxid = 0;
|
||||
|
||||
@ -2850,7 +2850,7 @@ TEST_P(CoordinationTest, TestReapplyingDeltas)
|
||||
create_request->path = "/test/data";
|
||||
create_request->is_sequential = true;
|
||||
|
||||
const auto process_create = [](KeeperStorage & storage, const auto & request, int64_t zxid)
|
||||
const auto process_create = [](KeeperMemoryStorage & storage, const auto & request, int64_t zxid)
|
||||
{
|
||||
storage.preprocessRequest(request, 1, 0, zxid);
|
||||
auto responses = storage.processRequest(request, 1, zxid);
|
||||
@ -2871,19 +2871,19 @@ TEST_P(CoordinationTest, TestReapplyingDeltas)
|
||||
process_create(storage, create_request, zxid);
|
||||
};
|
||||
|
||||
KeeperStorage storage1{500, "", keeper_context};
|
||||
KeeperMemoryStorage storage1{500, "", keeper_context};
|
||||
commit_initial_data(storage1);
|
||||
|
||||
for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid)
|
||||
storage1.preprocessRequest(create_request, 1, 0, zxid, /*check_acl=*/true, /*digest=*/std::nullopt, /*log_idx=*/zxid);
|
||||
|
||||
/// create identical new storage
|
||||
KeeperStorage storage2{500, "", keeper_context};
|
||||
KeeperMemoryStorage storage2{500, "", keeper_context};
|
||||
commit_initial_data(storage2);
|
||||
|
||||
storage1.applyUncommittedState(storage2, initial_zxid);
|
||||
|
||||
const auto commit_unprocessed = [&](KeeperStorage & storage)
|
||||
const auto commit_unprocessed = [&](KeeperMemoryStorage & storage)
|
||||
{
|
||||
for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid)
|
||||
{
|
||||
@ -2896,7 +2896,7 @@ TEST_P(CoordinationTest, TestReapplyingDeltas)
|
||||
commit_unprocessed(storage1);
|
||||
commit_unprocessed(storage2);
|
||||
|
||||
const auto get_children = [&](KeeperStorage & storage)
|
||||
const auto get_children = [&](KeeperMemoryStorage & storage)
|
||||
{
|
||||
const auto list_request = std::make_shared<ZooKeeperListRequest>();
|
||||
list_request->path = "/test";
|
||||
|
Loading…
Reference in New Issue
Block a user