Merge pull request #19580 from ClickHouse/in_memory_raft

In memory coordination inside ClickHouse
This commit is contained in:
alesapin 2021-02-13 10:19:23 +03:00 committed by GitHub
commit f80137626a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
74 changed files with 3760 additions and 469 deletions

2
.gitmodules vendored
View File

@ -220,4 +220,4 @@
url = https://github.com/ClickHouse-Extras/boringssl.git
[submodule "contrib/NuRaft"]
path = contrib/NuRaft
url = https://github.com/eBay/NuRaft.git
url = https://github.com/ClickHouse-Extras/NuRaft.git

View File

@ -11,7 +11,7 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/NuRaft/CMakeLists.txt")
return()
endif ()
if (NOT OS_FREEBSD)
if (NOT OS_FREEBSD AND NOT OS_DARWIN)
set (USE_NURAFT 1)
set (NURAFT_LIBRARY nuraft)
@ -20,5 +20,5 @@ if (NOT OS_FREEBSD)
message (STATUS "Using NuRaft=${USE_NURAFT}: ${NURAFT_INCLUDE_DIR} : ${NURAFT_LIBRARY}")
else()
set (USE_NURAFT 0)
message (STATUS "Using internal NuRaft library on FreeBSD is not supported")
message (STATUS "Using internal NuRaft library on FreeBSD and Darwin is not supported")
endif()

2
contrib/NuRaft vendored

@ -1 +1 @@
Subproject commit 410bd149da84cdde60b4436b02b738749f4e87e1
Subproject commit 7adf7ae33e7d5c307342431b577c8ab1025ee793

2
contrib/boost vendored

@ -1 +1 @@
Subproject commit 8e259cd2a6b60d75dd17e73432f11bb7b9351bb1
Subproject commit 48f40ebb539220d328958f8823b094c0b07a4e79

View File

@ -30,7 +30,12 @@ set(SRCS
add_library(nuraft ${SRCS})
target_compile_definitions(nuraft PRIVATE USE_BOOST_ASIO=1 BOOST_ASIO_STANDALONE=1)
if (NOT OPENSSL_SSL_LIBRARY OR NOT OPENSSL_CRYPTO_LIBRARY)
target_compile_definitions(nuraft PRIVATE USE_BOOST_ASIO=1 BOOST_ASIO_STANDALONE=1 SSL_LIBRARY_NOT_FOUND=1)
else()
target_compile_definitions(nuraft PRIVATE USE_BOOST_ASIO=1 BOOST_ASIO_STANDALONE=1)
endif()
target_include_directories (nuraft SYSTEM PRIVATE ${LIBRARY_DIR}/include/libnuraft)
# for some reason include "asio.h" directly without "boost/" prefix.

View File

@ -163,6 +163,7 @@ function clone_submodules
contrib/xz
contrib/dragonbox
contrib/fast_float
contrib/NuRaft
)
git submodule sync
@ -182,6 +183,7 @@ function run_cmake
"-DENABLE_EMBEDDED_COMPILER=0"
"-DENABLE_THINLTO=0"
"-DUSE_UNWIND=1"
"-DENABLE_NURAFT=1"
)
# TODO remove this? we don't use ccache anyway. An option would be to download it

View File

@ -59,7 +59,6 @@
#include <Disks/registerDisks.h>
#include <Common/Config/ConfigReloader.h>
#include <Server/HTTPHandlerFactory.h>
#include <Server/TestKeeperTCPHandlerFactory.h>
#include "MetricsTransmitter.h"
#include <Common/StatusFile.h>
#include <Server/TCPHandlerFactory.h>
@ -94,6 +93,9 @@
# include <Server/GRPCServer.h>
#endif
#if USE_NURAFT
# include <Server/NuKeeperTCPHandlerFactory.h>
#endif
namespace CurrentMetrics
{
@ -842,23 +844,33 @@ int Server::main(const std::vector<std::string> & /*args*/)
listen_try = true;
}
for (const auto & listen_host : listen_hosts)
if (config().has("test_keeper_server"))
{
/// TCP TestKeeper
const char * port_name = "test_keeper_server.tcp_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
#if USE_NURAFT
/// Initialize test keeper RAFT. Do nothing if no nu_keeper_server in config.
global_context->initializeNuKeeperStorageDispatcher();
for (const auto & listen_host : listen_hosts)
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(socket, listen_host, port);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
servers_to_start_before_tables->emplace_back(
port_name,
std::make_unique<Poco::Net::TCPServer>(
new TestKeeperTCPHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams));
/// TCP NuKeeper
const char * port_name = "test_keeper_server.tcp_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(socket, listen_host, port);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
servers_to_start_before_tables->emplace_back(
port_name,
std::make_unique<Poco::Net::TCPServer>(
new NuKeeperTCPHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for connections to NuKeeper (tcp): {}", address.toString());
});
}
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination.");
#endif
LOG_INFO(log, "Listening for connections to fake zookeeper (tcp): {}", address.toString());
});
}
for (auto & server : *servers_to_start_before_tables)
@ -898,6 +910,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections);
else
LOG_INFO(log, "Closed connections to servers for tables.");
global_context->shutdownNuKeeperStorageDispatcher();
}
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.

View File

@ -60,6 +60,7 @@ add_subdirectory (Processors)
add_subdirectory (Formats)
add_subdirectory (Compression)
add_subdirectory (Server)
add_subdirectory (Coordination)
set(dbms_headers)
@ -192,6 +193,10 @@ add_object_library(clickhouse_processors_merges_algorithms Processors/Merges/Alg
add_object_library(clickhouse_processors_queryplan Processors/QueryPlan)
add_object_library(clickhouse_processors_queryplan_optimizations Processors/QueryPlan/Optimizations)
if (USE_NURAFT)
add_object_library(clickhouse_coordination Coordination)
endif()
set (DBMS_COMMON_LIBRARIES)
# libgcc_s does not provide an implementation of an atomics library. Instead,
# GCCs libatomic library can be used to supply these when using libgcc_s.
@ -314,7 +319,7 @@ if (USE_KRB5)
endif()
if (USE_NURAFT)
dbms_target_link_libraries(PRIVATE ${NURAFT_LIBRARY})
dbms_target_link_libraries(PUBLIC ${NURAFT_LIBRARY})
endif()
if(RE2_INCLUDE_DIR)

View File

@ -534,6 +534,7 @@
M(565, TOO_MANY_PARTITIONS) \
M(566, CANNOT_RMDIR) \
M(567, DUPLICATED_PART_UUIDS) \
M(568, RAFT_ERROR) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -1,139 +0,0 @@
#include <Common/ZooKeeper/TestKeeperStorageDispatcher.h>
#include <Common/setThreadName.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int TIMEOUT_EXCEEDED;
}
}
namespace zkutil
{
void TestKeeperStorageDispatcher::processingThread()
{
setThreadName("TestKeeperSProc");
while (!shutdown)
{
RequestInfo info;
UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds());
if (requests_queue.tryPop(info, max_wait))
{
if (shutdown)
break;
try
{
auto responses = storage.processRequest(info.request, info.session_id);
for (const auto & response_for_session : responses)
setResponse(response_for_session.session_id, response_for_session.response);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
}
void TestKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response)
{
std::lock_guard lock(session_to_response_callback_mutex);
auto session_writer = session_to_response_callback.find(session_id);
if (session_writer == session_to_response_callback.end())
return;
session_writer->second(response);
/// Session closed, no more writes
if (response->xid != Coordination::WATCH_XID && response->getOpNum() == Coordination::OpNum::Close)
session_to_response_callback.erase(session_writer);
}
void TestKeeperStorageDispatcher::finalize()
{
{
std::lock_guard lock(push_request_mutex);
if (shutdown)
return;
shutdown = true;
if (processing_thread.joinable())
processing_thread.join();
}
RequestInfo info;
TestKeeperStorage::RequestsForSessions expired_requests;
while (requests_queue.tryPop(info))
expired_requests.push_back(TestKeeperStorage::RequestForSession{info.session_id, info.request});
auto expired_responses = storage.finalize(expired_requests);
for (const auto & response_for_session : expired_responses)
setResponse(response_for_session.session_id, response_for_session.response);
}
void TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id)
{
{
std::lock_guard lock(session_to_response_callback_mutex);
if (session_to_response_callback.count(session_id) == 0)
throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknown session id {}", session_id);
}
RequestInfo request_info;
request_info.time = clock::now();
request_info.request = request;
request_info.session_id = session_id;
std::lock_guard lock(push_request_mutex);
/// Put close requests without timeouts
if (request->getOpNum() == Coordination::OpNum::Close)
requests_queue.push(std::move(request_info));
else if (!requests_queue.tryPush(std::move(request_info), operation_timeout.totalMilliseconds()))
throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED);
}
TestKeeperStorageDispatcher::TestKeeperStorageDispatcher()
{
processing_thread = ThreadFromGlobalPool([this] { processingThread(); });
}
TestKeeperStorageDispatcher::~TestKeeperStorageDispatcher()
{
try
{
finalize();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void TestKeeperStorageDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCallback callback)
{
std::lock_guard lock(session_to_response_callback_mutex);
if (!session_to_response_callback.try_emplace(session_id, callback).second)
throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Session with id {} already registered in dispatcher", session_id);
}
void TestKeeperStorageDispatcher::finishSession(int64_t session_id)
{
std::lock_guard lock(session_to_response_callback_mutex);
auto session_it = session_to_response_callback.find(session_id);
if (session_it != session_to_response_callback.end())
session_to_response_callback.erase(session_it);
}
}

View File

@ -1,60 +0,0 @@
#pragma once
#include <Common/ThreadPool.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/ZooKeeper/TestKeeperStorage.h>
#include <functional>
namespace zkutil
{
using ZooKeeperResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr & response)>;
class TestKeeperStorageDispatcher
{
private:
Poco::Timespan operation_timeout{0, Coordination::DEFAULT_OPERATION_TIMEOUT_MS * 1000};
using clock = std::chrono::steady_clock;
struct RequestInfo
{
Coordination::ZooKeeperRequestPtr request;
clock::time_point time;
int64_t session_id;
};
std::mutex push_request_mutex;
using RequestsQueue = ConcurrentBoundedQueue<RequestInfo>;
RequestsQueue requests_queue{1};
std::atomic<bool> shutdown{false};
using SessionToResponseCallback = std::unordered_map<int64_t, ZooKeeperResponseCallback>;
std::mutex session_to_response_callback_mutex;
SessionToResponseCallback session_to_response_callback;
ThreadFromGlobalPool processing_thread;
TestKeeperStorage storage;
private:
void processingThread();
void finalize();
void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response);
public:
TestKeeperStorageDispatcher();
~TestKeeperStorageDispatcher();
void putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id);
int64_t getSessionID()
{
return storage.getSessionID();
}
void registerSession(int64_t session_id, ZooKeeperResponseCallback callback);
/// Call if we don't need any responses for this session no more (session was expired)
void finishSession(int64_t session_id);
};
}

View File

@ -37,6 +37,26 @@ void ZooKeeperRequest::write(WriteBuffer & out) const
out.next();
}
void ZooKeeperSyncRequest::writeImpl(WriteBuffer & out) const
{
Coordination::write(path, out);
}
void ZooKeeperSyncRequest::readImpl(ReadBuffer & in)
{
Coordination::read(path, in);
}
void ZooKeeperSyncResponse::readImpl(ReadBuffer & in)
{
Coordination::read(path, in);
}
void ZooKeeperSyncResponse::writeImpl(WriteBuffer & out) const
{
Coordination::write(path, out);
}
void ZooKeeperWatchResponse::readImpl(ReadBuffer & in)
{
Coordination::read(type, in);
@ -51,6 +71,13 @@ void ZooKeeperWatchResponse::writeImpl(WriteBuffer & out) const
Coordination::write(path, out);
}
void ZooKeeperWatchResponse::write(WriteBuffer & out) const
{
if (error == Error::ZOK)
ZooKeeperResponse::write(out);
/// skip bad responses for watches
}
void ZooKeeperAuthRequest::writeImpl(WriteBuffer & out) const
{
Coordination::write(type, out);
@ -326,6 +353,12 @@ void ZooKeeperMultiRequest::readImpl(ReadBuffer & in)
}
}
bool ZooKeeperMultiRequest::isReadRequest() const
{
/// Possibly we can do better
return false;
}
void ZooKeeperMultiResponse::readImpl(ReadBuffer & in)
{
for (auto & response : responses)
@ -410,6 +443,7 @@ void ZooKeeperMultiResponse::writeImpl(WriteBuffer & out) const
}
ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return std::make_shared<ZooKeeperHeartbeatResponse>(); }
ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return std::make_shared<ZooKeeperSyncResponse>(); }
ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return std::make_shared<ZooKeeperAuthResponse>(); }
ZooKeeperResponsePtr ZooKeeperCreateRequest::makeResponse() const { return std::make_shared<ZooKeeperCreateResponse>(); }
ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return std::make_shared<ZooKeeperRemoveResponse>(); }
@ -465,6 +499,7 @@ void registerZooKeeperRequest(ZooKeeperRequestFactory & factory)
ZooKeeperRequestFactory::ZooKeeperRequestFactory()
{
registerZooKeeperRequest<OpNum::Heartbeat, ZooKeeperHeartbeatRequest>(*this);
registerZooKeeperRequest<OpNum::Sync, ZooKeeperSyncRequest>(*this);
registerZooKeeperRequest<OpNum::Auth, ZooKeeperAuthRequest>(*this);
registerZooKeeperRequest<OpNum::Close, ZooKeeperCloseRequest>(*this);
registerZooKeeperRequest<OpNum::Create, ZooKeeperCreateRequest>(*this);

View File

@ -30,7 +30,7 @@ struct ZooKeeperResponse : virtual Response
virtual ~ZooKeeperResponse() override = default;
virtual void readImpl(ReadBuffer &) = 0;
virtual void writeImpl(WriteBuffer &) const = 0;
void write(WriteBuffer & out) const;
virtual void write(WriteBuffer & out) const;
virtual OpNum getOpNum() const = 0;
};
@ -60,6 +60,7 @@ struct ZooKeeperRequest : virtual Request
static std::shared_ptr<ZooKeeperRequest> read(ReadBuffer & in);
virtual ZooKeeperResponsePtr makeResponse() const = 0;
virtual bool isReadRequest() const = 0;
};
using ZooKeeperRequestPtr = std::shared_ptr<ZooKeeperRequest>;
@ -71,6 +72,26 @@ struct ZooKeeperHeartbeatRequest final : ZooKeeperRequest
void writeImpl(WriteBuffer &) const override {}
void readImpl(ReadBuffer &) override {}
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return false; }
};
struct ZooKeeperSyncRequest final : ZooKeeperRequest
{
String path;
String getPath() const override { return path; }
OpNum getOpNum() const override { return OpNum::Sync; }
void writeImpl(WriteBuffer & out) const override;
void readImpl(ReadBuffer & in) override;
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return false; }
};
struct ZooKeeperSyncResponse final : ZooKeeperResponse
{
String path;
void readImpl(ReadBuffer & in) override;
void writeImpl(WriteBuffer & out) const override;
OpNum getOpNum() const override { return OpNum::Sync; }
};
struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse
@ -86,6 +107,8 @@ struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse
void writeImpl(WriteBuffer & out) const override;
void write(WriteBuffer & out) const override;
OpNum getOpNum() const override
{
throw Exception("OpNum for watch response doesn't exist", Error::ZRUNTIMEINCONSISTENCY);
@ -104,6 +127,7 @@ struct ZooKeeperAuthRequest final : ZooKeeperRequest
void readImpl(ReadBuffer & in) override;
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return false; }
};
struct ZooKeeperAuthResponse final : ZooKeeperResponse
@ -122,6 +146,7 @@ struct ZooKeeperCloseRequest final : ZooKeeperRequest
void readImpl(ReadBuffer &) override {}
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return false; }
};
struct ZooKeeperCloseResponse final : ZooKeeperResponse
@ -146,6 +171,7 @@ struct ZooKeeperCreateRequest final : public CreateRequest, ZooKeeperRequest
void readImpl(ReadBuffer & in) override;
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return false; }
};
struct ZooKeeperCreateResponse final : CreateResponse, ZooKeeperResponse
@ -167,6 +193,7 @@ struct ZooKeeperRemoveRequest final : RemoveRequest, ZooKeeperRequest
void readImpl(ReadBuffer & in) override;
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return false; }
};
struct ZooKeeperRemoveResponse final : RemoveResponse, ZooKeeperResponse
@ -183,6 +210,7 @@ struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest
void readImpl(ReadBuffer & in) override;
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return !has_watch; }
};
struct ZooKeeperExistsResponse final : ExistsResponse, ZooKeeperResponse
@ -199,6 +227,7 @@ struct ZooKeeperGetRequest final : GetRequest, ZooKeeperRequest
void readImpl(ReadBuffer & in) override;
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return !has_watch; }
};
struct ZooKeeperGetResponse final : GetResponse, ZooKeeperResponse
@ -217,6 +246,7 @@ struct ZooKeeperSetRequest final : SetRequest, ZooKeeperRequest
void writeImpl(WriteBuffer & out) const override;
void readImpl(ReadBuffer & in) override;
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return false; }
};
struct ZooKeeperSetResponse final : SetResponse, ZooKeeperResponse
@ -232,6 +262,7 @@ struct ZooKeeperListRequest : ListRequest, ZooKeeperRequest
void writeImpl(WriteBuffer & out) const override;
void readImpl(ReadBuffer & in) override;
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return !has_watch; }
};
struct ZooKeeperSimpleListRequest final : ZooKeeperListRequest
@ -261,6 +292,7 @@ struct ZooKeeperCheckRequest final : CheckRequest, ZooKeeperRequest
void readImpl(ReadBuffer & in) override;
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return !has_watch; }
};
struct ZooKeeperCheckResponse final : CheckResponse, ZooKeeperResponse
@ -290,6 +322,7 @@ struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest
void readImpl(ReadBuffer & in) override;
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override;
};
struct ZooKeeperMultiResponse final : MultiResponse, ZooKeeperResponse

View File

@ -15,6 +15,7 @@ static const std::unordered_set<int32_t> VALID_OPERATIONS =
static_cast<int32_t>(OpNum::Get),
static_cast<int32_t>(OpNum::Set),
static_cast<int32_t>(OpNum::SimpleList),
static_cast<int32_t>(OpNum::Sync),
static_cast<int32_t>(OpNum::Heartbeat),
static_cast<int32_t>(OpNum::List),
static_cast<int32_t>(OpNum::Check),
@ -48,6 +49,8 @@ std::string toString(OpNum op_num)
return "Check";
case OpNum::Multi:
return "Multi";
case OpNum::Sync:
return "Sync";
case OpNum::Heartbeat:
return "Heartbeat";
case OpNum::Auth:

View File

@ -24,6 +24,7 @@ enum class OpNum : int32_t
Get = 4,
Set = 5,
SimpleList = 8,
Sync = 9,
Heartbeat = 11,
List = 12,
Check = 13,

View File

@ -3,6 +3,13 @@
namespace Coordination
{
void write(size_t x, WriteBuffer & out)
{
x = __builtin_bswap64(x);
writeBinary(x, out);
}
void write(int64_t x, WriteBuffer & out)
{
x = __builtin_bswap64(x);
@ -57,6 +64,12 @@ void write(const Error & x, WriteBuffer & out)
write(static_cast<int32_t>(x), out);
}
void read(size_t & x, ReadBuffer & in)
{
readBinary(x, in);
x = __builtin_bswap64(x);
}
void read(int64_t & x, ReadBuffer & in)
{
readBinary(x, in);

View File

@ -13,6 +13,7 @@ namespace Coordination
using namespace DB;
void write(size_t x, WriteBuffer & out);
void write(int64_t x, WriteBuffer & out);
void write(int32_t x, WriteBuffer & out);
void write(OpNum x, WriteBuffer & out);
@ -37,6 +38,7 @@ void write(const std::vector<T> & arr, WriteBuffer & out)
write(elem, out);
}
void read(size_t & x, ReadBuffer & in);
void read(int64_t & x, ReadBuffer & in);
void read(int32_t & x, ReadBuffer & in);
void read(OpNum & x, ReadBuffer & in);

View File

@ -83,8 +83,6 @@ SRCS(
WeakHash.cpp
ZooKeeper/IKeeper.cpp
ZooKeeper/TestKeeper.cpp
ZooKeeper/TestKeeperStorage.cpp
ZooKeeper/TestKeeperStorageDispatcher.cpp
ZooKeeper/ZooKeeper.cpp
ZooKeeper/ZooKeeperCommon.cpp
ZooKeeper/ZooKeeperConstants.cpp

View File

View File

@ -0,0 +1,35 @@
#include <Coordination/CoordinationSettings.h>
#include <Core/Settings.h>
#include <common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_SETTING;
}
IMPLEMENT_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
{
if (!config.has(config_elem))
return;
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(config_elem, config_keys);
try
{
for (const String & key : config_keys)
set(key, config.getString(config_elem + "." + key));
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::UNKNOWN_SETTING)
e.addMessage("in Coordination settings config");
throw;
}
}
}

View File

@ -0,0 +1,43 @@
#pragma once
#include <Core/Defines.h>
#include <Core/BaseSettings.h>
#include <Core/SettingsEnums.h>
#include <Common/ZooKeeper/ZooKeeperConstants.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
struct Settings;
/** These settings represent fine tunes for internal details of Coordination storages
* and should not be changed by the user without a reason.
*/
#define LIST_OF_COORDINATION_SETTINGS(M) \
M(Milliseconds, session_timeout_ms, Coordination::DEFAULT_SESSION_TIMEOUT_MS, "Default client session timeout", 0) \
M(Milliseconds, operation_timeout_ms, Coordination::DEFAULT_OPERATION_TIMEOUT_MS, "Default client operation timeout", 0) \
M(Milliseconds, dead_session_check_period_ms, 500, "How often leader will check sessions to consider them dead and remove", 0) \
M(Milliseconds, heart_beat_interval_ms, 500, "Heartbeat interval between quorum nodes", 0) \
M(Milliseconds, election_timeout_lower_bound_ms, 1000, "Lower bound of election timer (avoid too often leader elections)", 0) \
M(Milliseconds, election_timeout_upper_bound_ms, 2000, "Lower bound of election timer (avoid too often leader elections)", 0) \
M(UInt64, reserved_log_items, 5000, "How many log items to store (don't remove during compaction)", 0) \
M(UInt64, snapshot_distance, 5000, "How many log items we have to collect to write new snapshot", 0) \
M(UInt64, max_stored_snapshots, 3, "How many snapshots we want to store", 0) \
M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \
M(Milliseconds, shutdown_timeout, 5000, "How many time we will until RAFT shutdown", 0) \
M(Milliseconds, startup_timeout, 30000, "How many time we will until RAFT to start", 0) \
M(LogsLevel, raft_logs_level, LogsLevel::information, "Log internal RAFT logs into main server log level. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0)
DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
struct CoordinationSettings : public BaseSettings<CoordinationSettingsTraits>
{
void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
};
using CoordinationSettingsPtr = std::shared_ptr<CoordinationSettings>;
}

View File

@ -0,0 +1,194 @@
#include <Coordination/InMemoryLogStore.h>
namespace DB
{
namespace
{
using namespace nuraft;
ptr<log_entry> makeClone(const ptr<log_entry> & entry)
{
ptr<log_entry> clone = cs_new<log_entry>(entry->get_term(), buffer::clone(entry->get_buf()), entry->get_val_type());
return clone;
}
}
InMemoryLogStore::InMemoryLogStore()
: start_idx(1)
{
nuraft::ptr<nuraft::buffer> buf = nuraft::buffer::alloc(sizeof(size_t));
logs[0] = nuraft::cs_new<nuraft::log_entry>(0, buf);
}
size_t InMemoryLogStore::start_index() const
{
return start_idx;
}
size_t InMemoryLogStore::next_slot() const
{
std::lock_guard<std::mutex> l(logs_lock);
// Exclude the dummy entry.
return start_idx + logs.size() - 1;
}
nuraft::ptr<nuraft::log_entry> InMemoryLogStore::last_entry() const
{
size_t next_idx = next_slot();
std::lock_guard<std::mutex> lock(logs_lock);
auto entry = logs.find(next_idx - 1);
if (entry == logs.end())
entry = logs.find(0);
return makeClone(entry->second);
}
size_t InMemoryLogStore::append(nuraft::ptr<nuraft::log_entry> & entry)
{
ptr<log_entry> clone = makeClone(entry);
std::lock_guard<std::mutex> l(logs_lock);
size_t idx = start_idx + logs.size() - 1;
logs[idx] = clone;
return idx;
}
void InMemoryLogStore::write_at(size_t index, nuraft::ptr<nuraft::log_entry> & entry)
{
nuraft::ptr<log_entry> clone = makeClone(entry);
// Discard all logs equal to or greater than `index.
std::lock_guard<std::mutex> l(logs_lock);
auto itr = logs.lower_bound(index);
while (itr != logs.end())
itr = logs.erase(itr);
logs[index] = clone;
}
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> InMemoryLogStore::log_entries(size_t start, size_t end)
{
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> ret =
nuraft::cs_new<std::vector<nuraft::ptr<nuraft::log_entry>>>();
ret->resize(end - start);
size_t cc = 0;
for (size_t ii = start; ii < end; ++ii)
{
nuraft::ptr<nuraft::log_entry> src = nullptr;
{
std::lock_guard<std::mutex> l(logs_lock);
auto entry = logs.find(ii);
if (entry == logs.end())
{
entry = logs.find(0);
assert(0);
}
src = entry->second;
}
(*ret)[cc++] = makeClone(src);
}
return ret;
}
nuraft::ptr<nuraft::log_entry> InMemoryLogStore::entry_at(size_t index)
{
nuraft::ptr<nuraft::log_entry> src = nullptr;
{
std::lock_guard<std::mutex> l(logs_lock);
auto entry = logs.find(index);
if (entry == logs.end())
entry = logs.find(0);
src = entry->second;
}
return makeClone(src);
}
size_t InMemoryLogStore::term_at(size_t index)
{
size_t term = 0;
{
std::lock_guard<std::mutex> l(logs_lock);
auto entry = logs.find(index);
if (entry == logs.end())
entry = logs.find(0);
term = entry->second->get_term();
}
return term;
}
nuraft::ptr<nuraft::buffer> InMemoryLogStore::pack(size_t index, Int32 cnt)
{
std::vector<nuraft::ptr<nuraft::buffer>> returned_logs;
size_t size_total = 0;
for (size_t ii = index; ii < index + cnt; ++ii)
{
ptr<log_entry> le = nullptr;
{
std::lock_guard<std::mutex> l(logs_lock);
le = logs[ii];
}
assert(le.get());
nuraft::ptr<nuraft::buffer> buf = le->serialize();
size_total += buf->size();
returned_logs.push_back(buf);
}
nuraft::ptr<buffer> buf_out = nuraft::buffer::alloc(sizeof(int32) + cnt * sizeof(int32) + size_total);
buf_out->pos(0);
buf_out->put(static_cast<Int32>(cnt));
for (auto & entry : returned_logs)
{
nuraft::ptr<nuraft::buffer> & bb = entry;
buf_out->put(static_cast<Int32>(bb->size()));
buf_out->put(*bb);
}
return buf_out;
}
void InMemoryLogStore::apply_pack(size_t index, nuraft::buffer & pack)
{
pack.pos(0);
Int32 num_logs = pack.get_int();
for (Int32 ii = 0; ii < num_logs; ++ii)
{
size_t cur_idx = index + ii;
Int32 buf_size = pack.get_int();
nuraft::ptr<nuraft::buffer> buf_local = nuraft::buffer::alloc(buf_size);
pack.get(buf_local);
nuraft::ptr<nuraft::log_entry> le = nuraft::log_entry::deserialize(*buf_local);
{
std::lock_guard<std::mutex> l(logs_lock);
logs[cur_idx] = le;
}
}
{
std::lock_guard<std::mutex> l(logs_lock);
auto entry = logs.upper_bound(0);
if (entry != logs.end())
start_idx = entry->first;
else
start_idx = 1;
}
}
bool InMemoryLogStore::compact(size_t last_log_index)
{
std::lock_guard<std::mutex> l(logs_lock);
for (size_t ii = start_idx; ii <= last_log_index; ++ii)
{
auto entry = logs.find(ii);
if (entry != logs.end())
logs.erase(entry);
}
start_idx = last_log_index + 1;
return true;
}
}

View File

@ -0,0 +1,47 @@
#pragma once
#include <atomic>
#include <map>
#include <mutex>
#include <Core/Types.h>
#include <libnuraft/log_store.hxx> // Y_IGNORE
namespace DB
{
class InMemoryLogStore : public nuraft::log_store
{
public:
InMemoryLogStore();
size_t start_index() const override;
size_t next_slot() const override;
nuraft::ptr<nuraft::log_entry> last_entry() const override;
size_t append(nuraft::ptr<nuraft::log_entry> & entry) override;
void write_at(size_t index, nuraft::ptr<nuraft::log_entry> & entry) override;
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> log_entries(size_t start, size_t end) override;
nuraft::ptr<nuraft::log_entry> entry_at(size_t index) override;
size_t term_at(size_t index) override;
nuraft::ptr<nuraft::buffer> pack(size_t index, Int32 cnt) override;
void apply_pack(size_t index, nuraft::buffer & pack) override;
bool compact(size_t last_log_index) override;
bool flush() override { return true; }
private:
std::map<size_t, nuraft::ptr<nuraft::log_entry>> logs;
mutable std::mutex logs_lock;
std::atomic<size_t> start_idx;
};
}

View File

@ -0,0 +1,78 @@
#include <Coordination/InMemoryStateManager.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int RAFT_ERROR;
}
InMemoryStateManager::InMemoryStateManager(int server_id_, const std::string & host, int port)
: my_server_id(server_id_)
, my_port(port)
, log_store(nuraft::cs_new<InMemoryLogStore>())
, cluster_config(nuraft::cs_new<nuraft::cluster_config>())
{
auto peer_config = nuraft::cs_new<nuraft::srv_config>(my_server_id, host + ":" + std::to_string(port));
cluster_config->get_servers().push_back(peer_config);
}
InMemoryStateManager::InMemoryStateManager(
int my_server_id_,
const std::string & config_prefix,
const Poco::Util::AbstractConfiguration & config)
: my_server_id(my_server_id_)
, log_store(nuraft::cs_new<InMemoryLogStore>())
, cluster_config(nuraft::cs_new<nuraft::cluster_config>())
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
for (const auto & server_key : keys)
{
std::string full_prefix = config_prefix + "." + server_key;
int server_id = config.getInt(full_prefix + ".id");
std::string hostname = config.getString(full_prefix + ".hostname");
int port = config.getInt(full_prefix + ".port");
bool can_become_leader = config.getBool(full_prefix + ".can_become_leader", true);
int32_t priority = config.getInt(full_prefix + ".priority", 1);
bool start_as_follower = config.getBool(full_prefix + ".start_as_follower", false);
if (start_as_follower)
start_as_follower_servers.insert(server_id);
auto endpoint = hostname + ":" + std::to_string(port);
auto peer_config = nuraft::cs_new<nuraft::srv_config>(server_id, 0, endpoint, "", !can_become_leader, priority);
if (server_id == my_server_id)
{
my_server_config = peer_config;
my_port = port;
}
cluster_config->get_servers().push_back(peer_config);
}
if (!my_server_config)
throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section");
if (start_as_follower_servers.size() == cluster_config->get_servers().size())
throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without <start_as_follower>)");
}
void InMemoryStateManager::save_config(const nuraft::cluster_config & config)
{
// Just keep in memory in this example.
// Need to write to disk here, if want to make it durable.
nuraft::ptr<nuraft::buffer> buf = config.serialize();
cluster_config = nuraft::cluster_config::deserialize(*buf);
}
void InMemoryStateManager::save_state(const nuraft::srv_state & state)
{
// Just keep in memory in this example.
// Need to write to disk here, if want to make it durable.
nuraft::ptr<nuraft::buffer> buf = state.serialize();
server_state = nuraft::srv_state::deserialize(*buf);
}
}

View File

@ -0,0 +1,58 @@
#pragma once
#include <Core/Types.h>
#include <string>
#include <Coordination/InMemoryLogStore.h>
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
class InMemoryStateManager : public nuraft::state_mgr
{
public:
InMemoryStateManager(
int server_id_,
const std::string & config_prefix,
const Poco::Util::AbstractConfiguration & config);
InMemoryStateManager(
int server_id_,
const std::string & host,
int port);
nuraft::ptr<nuraft::cluster_config> load_config() override { return cluster_config; }
void save_config(const nuraft::cluster_config & config) override;
void save_state(const nuraft::srv_state & state) override;
nuraft::ptr<nuraft::srv_state> read_state() override { return server_state; }
nuraft::ptr<nuraft::log_store> load_log_store() override { return log_store; }
Int32 server_id() override { return my_server_id; }
nuraft::ptr<nuraft::srv_config> get_srv_config() const { return my_server_config; }
void system_exit(const int /* exit_code */) override {}
int getPort() const { return my_port; }
bool shouldStartAsFollower() const
{
return start_as_follower_servers.count(my_server_id);
}
private:
int my_server_id;
int my_port;
std::unordered_set<int> start_as_follower_servers;
nuraft::ptr<InMemoryLogStore> log_store;
nuraft::ptr<nuraft::srv_config> my_server_config;
nuraft::ptr<nuraft::cluster_config> cluster_config;
nuraft::ptr<nuraft::srv_state> server_state;
};
}

View File

@ -0,0 +1,47 @@
#pragma once
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <common/logger_useful.h>
#include <Core/SettingsEnums.h>
namespace DB
{
class LoggerWrapper : public nuraft::logger
{
public:
LoggerWrapper(const std::string & name, LogsLevel level_)
: log(&Poco::Logger::get(name))
, level(static_cast<int>(level_))
{
log->setLevel(level);
}
void put_details(
int level_,
const char * /* source_file */,
const char * /* func_name */,
size_t /* line_number */,
const std::string & msg) override
{
LOG_IMPL(log, static_cast<DB::LogsLevel>(level_), static_cast<Poco::Message::Priority>(level_), msg);
}
void set_level(int level_) override
{
level_ = std::min(6, std::max(1, level_));
log->setLevel(level_);
level = level_;
}
int get_level() override
{
return level;
}
private:
Poco::Logger * log;
std::atomic<int> level;
};
}

View File

@ -0,0 +1,24 @@
#pragma once
#include <Common/ZooKeeper/ZooKeeperCommon.h>
namespace DB
{
struct NuKeeperRequest
{
int64_t session_id;
Coordination::ZooKeeperRequestPtr request;
};
using NuKeeperRequests = std::vector<NuKeeperRequest>;
struct NuKeeperResponse
{
int64_t session_id;
Coordination::ZooKeeperRequestPtr response;
};
using NuKeeperResponses = std::vector<NuKeeperResponse>;
}

View File

@ -0,0 +1,182 @@
#include <Coordination/NuKeeperServer.h>
#include <Coordination/LoggerWrapper.h>
#include <Coordination/NuKeeperStateMachine.h>
#include <Coordination/InMemoryStateManager.h>
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <Coordination/ReadBufferFromNuraftBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <chrono>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <string>
namespace DB
{
namespace ErrorCodes
{
extern const int RAFT_ERROR;
}
NuKeeperServer::NuKeeperServer(
int server_id_,
const CoordinationSettingsPtr & coordination_settings_,
const Poco::Util::AbstractConfiguration & config,
ResponsesQueue & responses_queue_)
: server_id(server_id_)
, coordination_settings(coordination_settings_)
, state_machine(nuraft::cs_new<NuKeeperStateMachine>(responses_queue_, coordination_settings))
, state_manager(nuraft::cs_new<InMemoryStateManager>(server_id, "test_keeper_server.raft_configuration", config))
, responses_queue(responses_queue_)
{
}
void NuKeeperServer::startup()
{
nuraft::raft_params params;
params.heart_beat_interval_ = coordination_settings->heart_beat_interval_ms.totalMilliseconds();
params.election_timeout_lower_bound_ = coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds();
params.election_timeout_upper_bound_ = coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds();
params.reserved_log_items_ = coordination_settings->reserved_log_items;
params.snapshot_distance_ = coordination_settings->snapshot_distance;
params.client_req_timeout_ = coordination_settings->operation_timeout_ms.totalMilliseconds();
params.auto_forwarding_ = coordination_settings->auto_forwarding;
params.auto_forwarding_req_timeout_ = coordination_settings->operation_timeout_ms.totalMilliseconds() * 2;
params.return_method_ = nuraft::raft_params::blocking;
nuraft::asio_service::options asio_opts{};
nuraft::raft_server::init_options init_options;
init_options.skip_initial_election_timeout_ = state_manager->shouldStartAsFollower();
init_options.raft_callback_ = [this] (nuraft::cb_func::Type type, nuraft::cb_func::Param * param)
{
return callbackFunc(type, param);
};
raft_instance = launcher.init(
state_machine, state_manager, nuraft::cs_new<LoggerWrapper>("RaftInstance", coordination_settings->raft_logs_level), state_manager->getPort(),
asio_opts, params, init_options);
if (!raft_instance)
throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance");
}
void NuKeeperServer::shutdown()
{
state_machine->shutdownStorage();
if (!launcher.shutdown(coordination_settings->shutdown_timeout.totalSeconds()))
LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Failed to shutdown RAFT server in {} seconds", 5);
}
namespace
{
nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(int64_t session_id, const Coordination::ZooKeeperRequestPtr & request)
{
DB::WriteBufferFromNuraftBuffer buf;
DB::writeIntBinary(session_id, buf);
request->write(buf);
return buf.getBuffer();
}
}
void NuKeeperServer::putRequest(const NuKeeperStorage::RequestForSession & request_for_session)
{
auto [session_id, request] = request_for_session;
if (isLeaderAlive() && request->isReadRequest())
{
state_machine->processReadRequest(request_for_session);
}
else
{
std::vector<nuraft::ptr<nuraft::buffer>> entries;
entries.push_back(getZooKeeperLogEntry(session_id, request));
std::lock_guard lock(append_entries_mutex);
auto result = raft_instance->append_entries(entries);
if (!result->get_accepted())
{
NuKeeperStorage::ResponsesForSessions responses;
auto response = request->makeResponse();
response->xid = request->xid;
response->zxid = 0;
response->error = Coordination::Error::ZOPERATIONTIMEOUT;
responses_queue.push(DB::NuKeeperStorage::ResponseForSession{session_id, response});
}
if (result->get_result_code() == nuraft::cmd_result_code::TIMEOUT)
{
NuKeeperStorage::ResponsesForSessions responses;
auto response = request->makeResponse();
response->xid = request->xid;
response->zxid = 0;
response->error = Coordination::Error::ZOPERATIONTIMEOUT;
responses_queue.push(DB::NuKeeperStorage::ResponseForSession{session_id, response});
}
else if (result->get_result_code() != nuraft::cmd_result_code::OK)
throw Exception(ErrorCodes::RAFT_ERROR, "Requests result failed with code {} and message: '{}'", result->get_result_code(), result->get_result_str());
}
}
int64_t NuKeeperServer::getSessionID(int64_t session_timeout_ms)
{
auto entry = nuraft::buffer::alloc(sizeof(int64_t));
/// Just special session request
nuraft::buffer_serializer bs(entry);
bs.put_i64(session_timeout_ms);
std::lock_guard lock(append_entries_mutex);
auto result = raft_instance->append_entries({entry});
if (!result->get_accepted())
throw Exception(ErrorCodes::RAFT_ERROR, "Cannot send session_id request to RAFT");
if (result->get_result_code() != nuraft::cmd_result_code::OK)
throw Exception(ErrorCodes::RAFT_ERROR, "session_id request failed to RAFT");
auto resp = result->get();
if (resp == nullptr)
throw Exception(ErrorCodes::RAFT_ERROR, "Received nullptr as session_id");
nuraft::buffer_serializer bs_resp(resp);
return bs_resp.get_i64();
}
bool NuKeeperServer::isLeader() const
{
return raft_instance->is_leader();
}
bool NuKeeperServer::isLeaderAlive() const
{
return raft_instance->is_leader_alive();
}
nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * /* param */)
{
if (type == nuraft::cb_func::Type::BecomeFresh || type == nuraft::cb_func::Type::BecomeLeader)
{
std::unique_lock lock(initialized_mutex);
initialized_flag = true;
initialized_cv.notify_all();
}
return nuraft::cb_func::ReturnCode::Ok;
}
void NuKeeperServer::waitInit()
{
std::unique_lock lock(initialized_mutex);
int64_t timeout = coordination_settings->startup_timeout.totalMilliseconds();
if (!initialized_cv.wait_for(lock, std::chrono::milliseconds(timeout), [&] { return initialized_flag; }))
throw Exception(ErrorCodes::RAFT_ERROR, "Failed to wait RAFT initialization");
}
std::unordered_set<int64_t> NuKeeperServer::getDeadSessions()
{
return state_machine->getDeadSessions();
}
}

View File

@ -0,0 +1,63 @@
#pragma once
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <Coordination/InMemoryLogStore.h>
#include <Coordination/InMemoryStateManager.h>
#include <Coordination/NuKeeperStateMachine.h>
#include <Coordination/NuKeeperStorage.h>
#include <Coordination/CoordinationSettings.h>
#include <unordered_map>
namespace DB
{
class NuKeeperServer
{
private:
int server_id;
CoordinationSettingsPtr coordination_settings;
nuraft::ptr<NuKeeperStateMachine> state_machine;
nuraft::ptr<InMemoryStateManager> state_manager;
nuraft::raft_launcher launcher;
nuraft::ptr<nuraft::raft_server> raft_instance;
std::mutex append_entries_mutex;
ResponsesQueue & responses_queue;
std::mutex initialized_mutex;
bool initialized_flag = false;
std::condition_variable initialized_cv;
nuraft::cb_func::ReturnCode callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * param);
public:
NuKeeperServer(
int server_id_,
const CoordinationSettingsPtr & coordination_settings_,
const Poco::Util::AbstractConfiguration & config,
ResponsesQueue & responses_queue_);
void startup();
void putRequest(const NuKeeperStorage::RequestForSession & request);
int64_t getSessionID(int64_t session_timeout_ms);
std::unordered_set<int64_t> getDeadSessions();
bool isLeader() const;
bool isLeaderAlive() const;
void waitInit();
void shutdown();
};
}

View File

@ -0,0 +1,260 @@
#include <Coordination/NuKeeperStateMachine.h>
#include <Coordination/ReadBufferFromNuraftBuffer.h>
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <IO/ReadHelpers.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <Coordination/NuKeeperStorageSerializer.h>
namespace DB
{
NuKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data)
{
ReadBufferFromNuraftBuffer buffer(data);
NuKeeperStorage::RequestForSession request_for_session;
readIntBinary(request_for_session.session_id, buffer);
int32_t length;
Coordination::read(length, buffer);
int32_t xid;
Coordination::read(xid, buffer);
Coordination::OpNum opnum;
Coordination::read(opnum, buffer);
request_for_session.request = Coordination::ZooKeeperRequestFactory::instance().get(opnum);
request_for_session.request->xid = xid;
request_for_session.request->readImpl(buffer);
return request_for_session;
}
nuraft::ptr<nuraft::buffer> writeResponses(NuKeeperStorage::ResponsesForSessions & responses)
{
WriteBufferFromNuraftBuffer buffer;
for (const auto & response_and_session : responses)
{
writeIntBinary(response_and_session.session_id, buffer);
response_and_session.response->write(buffer);
}
return buffer.getBuffer();
}
NuKeeperStateMachine::NuKeeperStateMachine(ResponsesQueue & responses_queue_, const CoordinationSettingsPtr & coordination_settings_)
: coordination_settings(coordination_settings_)
, storage(coordination_settings->dead_session_check_period_ms.totalMilliseconds())
, responses_queue(responses_queue_)
, last_committed_idx(0)
, log(&Poco::Logger::get("NuRaftStateMachine"))
{
LOG_DEBUG(log, "Created nukeeper state machine");
}
nuraft::ptr<nuraft::buffer> NuKeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data)
{
if (data.size() == sizeof(int64_t))
{
nuraft::buffer_serializer timeout_data(data);
int64_t session_timeout_ms = timeout_data.get_i64();
auto response = nuraft::buffer::alloc(sizeof(int64_t));
int64_t session_id;
nuraft::buffer_serializer bs(response);
{
std::lock_guard lock(storage_lock);
session_id = storage.getSessionID(session_timeout_ms);
bs.put_i64(session_id);
}
LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_timeout_ms);
last_committed_idx = log_idx;
return response;
}
else
{
auto request_for_session = parseRequest(data);
NuKeeperStorage::ResponsesForSessions responses_for_sessions;
{
std::lock_guard lock(storage_lock);
responses_for_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id);
for (auto & response_for_session : responses_for_sessions)
responses_queue.push(response_for_session);
}
last_committed_idx = log_idx;
return nullptr;
}
}
bool NuKeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
{
LOG_DEBUG(log, "Applying snapshot {}", s.get_last_log_idx());
StorageSnapshotPtr snapshot;
{
std::lock_guard<std::mutex> lock(snapshots_lock);
auto entry = snapshots.find(s.get_last_log_idx());
if (entry == snapshots.end())
return false;
snapshot = entry->second;
}
std::lock_guard lock(storage_lock);
storage = snapshot->storage;
last_committed_idx = s.get_last_log_idx();
return true;
}
nuraft::ptr<nuraft::snapshot> NuKeeperStateMachine::last_snapshot()
{
// Just return the latest snapshot.
std::lock_guard<std::mutex> lock(snapshots_lock);
auto entry = snapshots.rbegin();
if (entry == snapshots.rend())
return nullptr;
return entry->second->snapshot;
}
NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::createSnapshotInternal(nuraft::snapshot & s)
{
nuraft::ptr<nuraft::buffer> snp_buf = s.serialize();
nuraft::ptr<nuraft::snapshot> ss = nuraft::snapshot::deserialize(*snp_buf);
std::lock_guard lock(storage_lock);
return std::make_shared<NuKeeperStateMachine::StorageSnapshot>(ss, storage);
}
NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nuraft::snapshot & s, nuraft::buffer & in)
{
nuraft::ptr<nuraft::buffer> snp_buf = s.serialize();
nuraft::ptr<nuraft::snapshot> ss = nuraft::snapshot::deserialize(*snp_buf);
NuKeeperStorageSerializer serializer;
ReadBufferFromNuraftBuffer reader(in);
NuKeeperStorage new_storage(coordination_settings->dead_session_check_period_ms.totalMilliseconds());
serializer.deserialize(new_storage, reader);
return std::make_shared<StorageSnapshot>(ss, new_storage);
}
void NuKeeperStateMachine::writeSnapshot(const NuKeeperStateMachine::StorageSnapshotPtr & snapshot, nuraft::ptr<nuraft::buffer> & out)
{
NuKeeperStorageSerializer serializer;
WriteBufferFromNuraftBuffer writer;
serializer.serialize(snapshot->storage, writer);
out = writer.getBuffer();
}
void NuKeeperStateMachine::create_snapshot(
nuraft::snapshot & s,
nuraft::async_result<bool>::handler_type & when_done)
{
LOG_DEBUG(log, "Creating snapshot {}", s.get_last_log_idx());
auto snapshot = createSnapshotInternal(s);
{
std::lock_guard<std::mutex> lock(snapshots_lock);
snapshots[s.get_last_log_idx()] = snapshot;
size_t num = snapshots.size();
if (num > coordination_settings->max_stored_snapshots)
{
auto entry = snapshots.begin();
for (size_t i = 0; i < num - coordination_settings->max_stored_snapshots; ++i)
{
if (entry == snapshots.end())
break;
entry = snapshots.erase(entry);
}
}
}
nuraft::ptr<std::exception> except(nullptr);
bool ret = true;
when_done(ret, except);
}
void NuKeeperStateMachine::save_logical_snp_obj(
nuraft::snapshot & s,
size_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);
if (obj_id == 0)
{
auto new_snapshot = createSnapshotInternal(s);
std::lock_guard<std::mutex> lock(snapshots_lock);
snapshots.try_emplace(s.get_last_log_idx(), std::move(new_snapshot));
}
else
{
auto received_snapshot = readSnapshot(s, data);
std::lock_guard<std::mutex> lock(snapshots_lock);
snapshots[s.get_last_log_idx()] = std::move(received_snapshot);
}
obj_id++;
}
int NuKeeperStateMachine::read_logical_snp_obj(
nuraft::snapshot & s,
void* & /*user_snp_ctx*/,
ulong 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);
StorageSnapshotPtr required_snapshot;
{
std::lock_guard<std::mutex> lock(snapshots_lock);
auto entry = snapshots.find(s.get_last_log_idx());
if (entry == snapshots.end())
{
// Snapshot doesn't exist.
data_out = nullptr;
is_last_obj = true;
return 0;
}
required_snapshot = entry->second;
}
if (obj_id == 0)
{
auto new_snapshot = createSnapshotInternal(s);
writeSnapshot(new_snapshot, data_out);
is_last_obj = false;
}
else
{
writeSnapshot(required_snapshot, data_out);
is_last_obj = true;
}
return 0;
}
void NuKeeperStateMachine::processReadRequest(const NuKeeperStorage::RequestForSession & request_for_session)
{
NuKeeperStorage::ResponsesForSessions responses;
{
std::lock_guard lock(storage_lock);
responses = storage.processRequest(request_for_session.request, request_for_session.session_id);
}
for (const auto & response : responses)
responses_queue.push(response);
}
std::unordered_set<int64_t> NuKeeperStateMachine::getDeadSessions()
{
std::lock_guard lock(storage_lock);
return storage.getDeadSessions();
}
void NuKeeperStateMachine::shutdownStorage()
{
std::lock_guard lock(storage_lock);
storage.finalize();
}
}

View File

@ -0,0 +1,99 @@
#pragma once
#include <Coordination/NuKeeperStorage.h>
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <common/logger_useful.h>
#include <Coordination/ThreadSafeQueue.h>
#include <Coordination/CoordinationSettings.h>
namespace DB
{
using ResponsesQueue = ThreadSafeQueue<NuKeeperStorage::ResponseForSession>;
class NuKeeperStateMachine : public nuraft::state_machine
{
public:
NuKeeperStateMachine(ResponsesQueue & responses_queue_, const CoordinationSettingsPtr & coordination_settings_);
nuraft::ptr<nuraft::buffer> pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; }
nuraft::ptr<nuraft::buffer> commit(const size_t log_idx, nuraft::buffer & data) override;
void rollback(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override {}
size_t last_commit_index() override { return last_committed_idx; }
bool apply_snapshot(nuraft::snapshot & s) override;
nuraft::ptr<nuraft::snapshot> last_snapshot() override;
void create_snapshot(
nuraft::snapshot & s,
nuraft::async_result<bool>::handler_type & when_done) override;
void save_logical_snp_obj(
nuraft::snapshot & s,
size_t & obj_id,
nuraft::buffer & data,
bool is_first_obj,
bool is_last_obj) override;
int read_logical_snp_obj(
nuraft::snapshot & s,
void* & user_snp_ctx,
ulong obj_id,
nuraft::ptr<nuraft::buffer> & data_out,
bool & is_last_obj) override;
NuKeeperStorage & getStorage()
{
return storage;
}
void processReadRequest(const NuKeeperStorage::RequestForSession & request_for_session);
std::unordered_set<int64_t> getDeadSessions();
void shutdownStorage();
private:
struct StorageSnapshot
{
StorageSnapshot(const nuraft::ptr<nuraft::snapshot> & s, const NuKeeperStorage & storage_)
: snapshot(s)
, storage(storage_)
{}
nuraft::ptr<nuraft::snapshot> snapshot;
NuKeeperStorage storage;
};
using StorageSnapshotPtr = std::shared_ptr<StorageSnapshot>;
StorageSnapshotPtr createSnapshotInternal(nuraft::snapshot & s);
StorageSnapshotPtr readSnapshot(nuraft::snapshot & s, nuraft::buffer & in);
static void writeSnapshot(const StorageSnapshotPtr & snapshot, nuraft::ptr<nuraft::buffer> & out);
CoordinationSettingsPtr coordination_settings;
NuKeeperStorage storage;
ResponsesQueue & responses_queue;
/// Mutex for snapshots
std::mutex snapshots_lock;
/// Lock for storage
std::mutex storage_lock;
/// Fake snapshot storage
std::map<uint64_t, StorageSnapshotPtr> snapshots;
/// Last committed Raft log number.
std::atomic<size_t> last_committed_idx;
Poco::Logger * log;
};
}

View File

@ -1,4 +1,4 @@
#include <Common/ZooKeeper/TestKeeperStorage.h>
#include <Coordination/NuKeeperStorage.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/setThreadName.h>
#include <mutex>
@ -17,13 +17,6 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
}
namespace zkutil
{
using namespace DB;
static String parentPath(const String & path)
{
auto rslash_pos = path.rfind('/');
@ -38,20 +31,20 @@ static String baseName(const String & path)
return path.substr(rslash_pos + 1);
}
static TestKeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches, Coordination::Event event_type)
static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches, Coordination::Event event_type)
{
TestKeeperStorage::ResponsesForSessions result;
NuKeeperStorage::ResponsesForSessions result;
auto it = watches.find(path);
if (it != watches.end())
{
std::shared_ptr<Coordination::ZooKeeperWatchResponse> watch_response = std::make_shared<Coordination::ZooKeeperWatchResponse>();
watch_response->path = path;
watch_response->xid = -1;
watch_response->xid = Coordination::WATCH_XID;
watch_response->zxid = -1;
watch_response->type = event_type;
watch_response->state = Coordination::State::CONNECTED;
for (auto watcher_session : it->second)
result.push_back(TestKeeperStorage::ResponseForSession{watcher_session, watch_response});
result.push_back(NuKeeperStorage::ResponseForSession{watcher_session, watch_response});
watches.erase(it);
}
@ -62,58 +55,69 @@ static TestKeeperStorage::ResponsesForSessions processWatchesImpl(const String &
{
std::shared_ptr<Coordination::ZooKeeperWatchResponse> watch_list_response = std::make_shared<Coordination::ZooKeeperWatchResponse>();
watch_list_response->path = parent_path;
watch_list_response->xid = -1;
watch_list_response->xid = Coordination::WATCH_XID;
watch_list_response->zxid = -1;
watch_list_response->type = Coordination::Event::CHILD;
watch_list_response->state = Coordination::State::CONNECTED;
for (auto watcher_session : it->second)
result.push_back(TestKeeperStorage::ResponseForSession{watcher_session, watch_list_response});
result.push_back(NuKeeperStorage::ResponseForSession{watcher_session, watch_list_response});
list_watches.erase(it);
}
return result;
}
TestKeeperStorage::TestKeeperStorage()
NuKeeperStorage::NuKeeperStorage(int64_t tick_time_ms)
: session_expiry_queue(tick_time_ms)
{
container.emplace("/", Node());
}
using Undo = std::function<void()>;
struct TestKeeperStorageRequest
struct NuKeeperStorageRequest
{
Coordination::ZooKeeperRequestPtr zk_request;
explicit TestKeeperStorageRequest(const Coordination::ZooKeeperRequestPtr & zk_request_)
explicit NuKeeperStorageRequest(const Coordination::ZooKeeperRequestPtr & zk_request_)
: zk_request(zk_request_)
{}
virtual std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const = 0;
virtual TestKeeperStorage::ResponsesForSessions processWatches(TestKeeperStorage::Watches & /*watches*/, TestKeeperStorage::Watches & /*list_watches*/) const { return {}; }
virtual std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const = 0;
virtual NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & /*watches*/, NuKeeperStorage::Watches & /*list_watches*/) const { return {}; }
virtual ~TestKeeperStorageRequest() = default;
virtual ~NuKeeperStorageRequest() = default;
};
struct TestKeeperStorageHeartbeatRequest final : public TestKeeperStorageRequest
struct NuKeeperStorageHeartbeatRequest final : public NuKeeperStorageRequest
{
using TestKeeperStorageRequest::TestKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(TestKeeperStorage::Container & /* container */, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & /* container */, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override
{
return {zk_request->makeResponse(), {}};
}
};
struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest
struct NuKeeperStorageSyncRequest final : public NuKeeperStorageRequest
{
using TestKeeperStorageRequest::TestKeeperStorageRequest;
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & /* container */, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override
{
auto response = zk_request->makeResponse();
dynamic_cast<Coordination::ZooKeeperSyncResponse *>(response.get())->path = dynamic_cast<Coordination::ZooKeeperSyncRequest *>(zk_request.get())->path;
return {response, {}};
}
};
TestKeeperStorage::ResponsesForSessions processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override
struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest
{
using NuKeeperStorageRequest::NuKeeperStorageRequest;
NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override
{
return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED);
}
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Undo undo;
@ -138,8 +142,7 @@ struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest
}
else
{
TestKeeperStorage::Node created_node;
created_node.seq_num = 0;
NuKeeperStorage::Node created_node;
created_node.stat.czxid = zxid;
created_node.stat.mzxid = zxid;
created_node.stat.ctime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1);
@ -193,10 +196,10 @@ struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest
}
};
struct TestKeeperStorageGetRequest final : public TestKeeperStorageRequest
struct NuKeeperStorageGetRequest final : public NuKeeperStorageRequest
{
using TestKeeperStorageRequest::TestKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperGetResponse & response = dynamic_cast<Coordination::ZooKeeperGetResponse &>(*response_ptr);
@ -218,10 +221,10 @@ struct TestKeeperStorageGetRequest final : public TestKeeperStorageRequest
}
};
struct TestKeeperStorageRemoveRequest final : public TestKeeperStorageRequest
struct NuKeeperStorageRemoveRequest final : public NuKeeperStorageRequest
{
using TestKeeperStorageRequest::TestKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & ephemerals, int64_t /*zxid*/, int64_t session_id) const override
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t /*zxid*/, int64_t session_id) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperRemoveResponse & response = dynamic_cast<Coordination::ZooKeeperRemoveResponse &>(*response_ptr);
@ -268,16 +271,16 @@ struct TestKeeperStorageRemoveRequest final : public TestKeeperStorageRequest
return { response_ptr, undo };
}
TestKeeperStorage::ResponsesForSessions processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override
NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override
{
return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED);
}
};
struct TestKeeperStorageExistsRequest final : public TestKeeperStorageRequest
struct NuKeeperStorageExistsRequest final : public NuKeeperStorageRequest
{
using TestKeeperStorageRequest::TestKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /* session_id */) const override
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /* session_id */) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperExistsResponse & response = dynamic_cast<Coordination::ZooKeeperExistsResponse &>(*response_ptr);
@ -298,10 +301,10 @@ struct TestKeeperStorageExistsRequest final : public TestKeeperStorageRequest
}
};
struct TestKeeperStorageSetRequest final : public TestKeeperStorageRequest
struct NuKeeperStorageSetRequest final : public NuKeeperStorageRequest
{
using TestKeeperStorageRequest::TestKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t zxid, int64_t /* session_id */) const override
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t zxid, int64_t /* session_id */) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperSetResponse & response = dynamic_cast<Coordination::ZooKeeperSetResponse &>(*response_ptr);
@ -341,17 +344,17 @@ struct TestKeeperStorageSetRequest final : public TestKeeperStorageRequest
return { response_ptr, undo };
}
TestKeeperStorage::ResponsesForSessions processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override
NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override
{
return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED);
}
};
struct TestKeeperStorageListRequest final : public TestKeeperStorageRequest
struct NuKeeperStorageListRequest final : public NuKeeperStorageRequest
{
using TestKeeperStorageRequest::TestKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperListResponse & response = dynamic_cast<Coordination::ZooKeeperListResponse &>(*response_ptr);
@ -387,10 +390,10 @@ struct TestKeeperStorageListRequest final : public TestKeeperStorageRequest
}
};
struct TestKeeperStorageCheckRequest final : public TestKeeperStorageRequest
struct NuKeeperStorageCheckRequest final : public NuKeeperStorageRequest
{
using TestKeeperStorageRequest::TestKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperCheckResponse & response = dynamic_cast<Coordination::ZooKeeperCheckResponse &>(*response_ptr);
@ -413,11 +416,11 @@ struct TestKeeperStorageCheckRequest final : public TestKeeperStorageRequest
}
};
struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest
struct NuKeeperStorageMultiRequest final : public NuKeeperStorageRequest
{
std::vector<TestKeeperStorageRequestPtr> concrete_requests;
explicit TestKeeperStorageMultiRequest(const Coordination::ZooKeeperRequestPtr & zk_request_)
: TestKeeperStorageRequest(zk_request_)
std::vector<NuKeeperStorageRequestPtr> concrete_requests;
explicit NuKeeperStorageMultiRequest(const Coordination::ZooKeeperRequestPtr & zk_request_)
: NuKeeperStorageRequest(zk_request_)
{
Coordination::ZooKeeperMultiRequest & request = dynamic_cast<Coordination::ZooKeeperMultiRequest &>(*zk_request);
concrete_requests.reserve(request.requests.size());
@ -427,26 +430,26 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest
auto sub_zk_request = std::dynamic_pointer_cast<Coordination::ZooKeeperRequest>(sub_request);
if (sub_zk_request->getOpNum() == Coordination::OpNum::Create)
{
concrete_requests.push_back(std::make_shared<TestKeeperStorageCreateRequest>(sub_zk_request));
concrete_requests.push_back(std::make_shared<NuKeeperStorageCreateRequest>(sub_zk_request));
}
else if (sub_zk_request->getOpNum() == Coordination::OpNum::Remove)
{
concrete_requests.push_back(std::make_shared<TestKeeperStorageRemoveRequest>(sub_zk_request));
concrete_requests.push_back(std::make_shared<NuKeeperStorageRemoveRequest>(sub_zk_request));
}
else if (sub_zk_request->getOpNum() == Coordination::OpNum::Set)
{
concrete_requests.push_back(std::make_shared<TestKeeperStorageSetRequest>(sub_zk_request));
concrete_requests.push_back(std::make_shared<NuKeeperStorageSetRequest>(sub_zk_request));
}
else if (sub_zk_request->getOpNum() == Coordination::OpNum::Check)
{
concrete_requests.push_back(std::make_shared<TestKeeperStorageCheckRequest>(sub_zk_request));
concrete_requests.push_back(std::make_shared<NuKeeperStorageCheckRequest>(sub_zk_request));
}
else
throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal command as part of multi ZooKeeper request {}", sub_zk_request->getOpNum());
}
}
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperMultiResponse & response = dynamic_cast<Coordination::ZooKeeperMultiResponse &>(*response_ptr);
@ -499,9 +502,9 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest
}
}
TestKeeperStorage::ResponsesForSessions processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override
NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override
{
TestKeeperStorage::ResponsesForSessions result;
NuKeeperStorage::ResponsesForSessions result;
for (const auto & generic_request : concrete_requests)
{
auto responses = generic_request->processWatches(watches, list_watches);
@ -511,75 +514,49 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest
}
};
struct TestKeeperStorageCloseRequest final : public TestKeeperStorageRequest
struct NuKeeperStorageCloseRequest final : public NuKeeperStorageRequest
{
using TestKeeperStorageRequest::TestKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(TestKeeperStorage::Container &, TestKeeperStorage::Ephemerals &, int64_t, int64_t) const override
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container &, NuKeeperStorage::Ephemerals &, int64_t, int64_t) const override
{
throw DB::Exception("Called process on close request", ErrorCodes::LOGICAL_ERROR);
}
};
TestKeeperStorage::ResponsesForSessions TestKeeperStorage::finalize(const RequestsForSessions & expired_requests)
void NuKeeperStorage::finalize()
{
if (finalized)
throw DB::Exception("Testkeeper storage already finalized", ErrorCodes::LOGICAL_ERROR);
finalized = true;
ResponsesForSessions finalize_results;
auto finish_watch = [] (const auto & watch_pair) -> ResponsesForSessions
{
ResponsesForSessions results;
std::shared_ptr<Coordination::ZooKeeperWatchResponse> response = std::make_shared<Coordination::ZooKeeperWatchResponse>();
response->type = Coordination::SESSION;
response->state = Coordination::EXPIRED_SESSION;
response->error = Coordination::Error::ZSESSIONEXPIRED;
for (const auto & [session_id, ephemerals_paths] : ephemerals)
for (const String & ephemeral_path : ephemerals_paths)
container.erase(ephemeral_path);
for (auto & watcher_session : watch_pair.second)
results.push_back(ResponseForSession{watcher_session, response});
return results;
};
for (auto & path_watch : watches)
{
auto watch_responses = finish_watch(path_watch);
finalize_results.insert(finalize_results.end(), watch_responses.begin(), watch_responses.end());
}
ephemerals.clear();
watches.clear();
for (auto & path_watch : list_watches)
{
auto list_watch_responses = finish_watch(path_watch);
finalize_results.insert(finalize_results.end(), list_watch_responses.begin(), list_watch_responses.end());
}
list_watches.clear();
sessions_and_watchers.clear();
for (const auto & [session_id, zk_request] : expired_requests)
{
auto response = zk_request->makeResponse();
response->error = Coordination::Error::ZSESSIONEXPIRED;
finalize_results.push_back(ResponseForSession{session_id, response});
}
return finalize_results;
session_expiry_queue.clear();
}
class TestKeeperWrapperFactory final : private boost::noncopyable
class NuKeeperWrapperFactory final : private boost::noncopyable
{
public:
using Creator = std::function<TestKeeperStorageRequestPtr(const Coordination::ZooKeeperRequestPtr &)>;
using Creator = std::function<NuKeeperStorageRequestPtr(const Coordination::ZooKeeperRequestPtr &)>;
using OpNumToRequest = std::unordered_map<Coordination::OpNum, Creator>;
static TestKeeperWrapperFactory & instance()
static NuKeeperWrapperFactory & instance()
{
static TestKeeperWrapperFactory factory;
static NuKeeperWrapperFactory factory;
return factory;
}
TestKeeperStorageRequestPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const
NuKeeperStorageRequestPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const
{
auto it = op_num_to_request.find(zk_request->getOpNum());
if (it == op_num_to_request.end())
@ -596,36 +573,37 @@ public:
private:
OpNumToRequest op_num_to_request;
TestKeeperWrapperFactory();
NuKeeperWrapperFactory();
};
template<Coordination::OpNum num, typename RequestT>
void registerTestKeeperRequestWrapper(TestKeeperWrapperFactory & factory)
void registerNuKeeperRequestWrapper(NuKeeperWrapperFactory & factory)
{
factory.registerRequest(num, [] (const Coordination::ZooKeeperRequestPtr & zk_request) { return std::make_shared<RequestT>(zk_request); });
}
TestKeeperWrapperFactory::TestKeeperWrapperFactory()
NuKeeperWrapperFactory::NuKeeperWrapperFactory()
{
registerTestKeeperRequestWrapper<Coordination::OpNum::Heartbeat, TestKeeperStorageHeartbeatRequest>(*this);
//registerTestKeeperRequestWrapper<Coordination::OpNum::Auth, TestKeeperStorageAuthRequest>(*this);
registerTestKeeperRequestWrapper<Coordination::OpNum::Close, TestKeeperStorageCloseRequest>(*this);
registerTestKeeperRequestWrapper<Coordination::OpNum::Create, TestKeeperStorageCreateRequest>(*this);
registerTestKeeperRequestWrapper<Coordination::OpNum::Remove, TestKeeperStorageRemoveRequest>(*this);
registerTestKeeperRequestWrapper<Coordination::OpNum::Exists, TestKeeperStorageExistsRequest>(*this);
registerTestKeeperRequestWrapper<Coordination::OpNum::Get, TestKeeperStorageGetRequest>(*this);
registerTestKeeperRequestWrapper<Coordination::OpNum::Set, TestKeeperStorageSetRequest>(*this);
registerTestKeeperRequestWrapper<Coordination::OpNum::List, TestKeeperStorageListRequest>(*this);
registerTestKeeperRequestWrapper<Coordination::OpNum::SimpleList, TestKeeperStorageListRequest>(*this);
registerTestKeeperRequestWrapper<Coordination::OpNum::Check, TestKeeperStorageCheckRequest>(*this);
registerTestKeeperRequestWrapper<Coordination::OpNum::Multi, TestKeeperStorageMultiRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Heartbeat, NuKeeperStorageHeartbeatRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Sync, NuKeeperStorageSyncRequest>(*this);
//registerNuKeeperRequestWrapper<Coordination::OpNum::Auth, NuKeeperStorageAuthRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Close, NuKeeperStorageCloseRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Create, NuKeeperStorageCreateRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Remove, NuKeeperStorageRemoveRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Exists, NuKeeperStorageExistsRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Get, NuKeeperStorageGetRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Set, NuKeeperStorageSetRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::List, NuKeeperStorageListRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::SimpleList, NuKeeperStorageListRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Check, NuKeeperStorageCheckRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Multi, NuKeeperStorageMultiRequest>(*this);
}
TestKeeperStorage::ResponsesForSessions TestKeeperStorage::processRequest(const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id)
NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id)
{
TestKeeperStorage::ResponsesForSessions results;
NuKeeperStorage::ResponsesForSessions results;
if (zk_request->getOpNum() == Coordination::OpNum::Close)
{
auto it = ephemerals.find(session_id);
@ -645,12 +623,24 @@ TestKeeperStorage::ResponsesForSessions TestKeeperStorage::processRequest(const
auto response = std::make_shared<Coordination::ZooKeeperCloseResponse>();
response->xid = zk_request->xid;
response->zxid = getZXID();
session_expiry_queue.remove(session_id);
session_and_timeout.erase(session_id);
results.push_back(ResponseForSession{session_id, response});
}
else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat)
{
session_expiry_queue.update(session_id, session_and_timeout[session_id]);
NuKeeperStorageRequestPtr storage_request = NuKeeperWrapperFactory::instance().get(zk_request);
auto [response, _] = storage_request->process(container, ephemerals, zxid, session_id);
response->xid = zk_request->xid;
response->zxid = getZXID();
results.push_back(ResponseForSession{session_id, response});
}
else
{
TestKeeperStorageRequestPtr storage_request = TestKeeperWrapperFactory::instance().get(zk_request);
NuKeeperStorageRequestPtr storage_request = NuKeeperWrapperFactory::instance().get(zk_request);
auto [response, _] = storage_request->process(container, ephemerals, zxid, session_id);
if (zk_request->has_watch)
@ -669,15 +659,6 @@ TestKeeperStorage::ResponsesForSessions TestKeeperStorage::processRequest(const
watches[zk_request->getPath()].emplace_back(session_id);
sessions_and_watchers[session_id].emplace(zk_request->getPath());
}
else
{
std::shared_ptr<Coordination::ZooKeeperWatchResponse> watch_response = std::make_shared<Coordination::ZooKeeperWatchResponse>();
watch_response->path = zk_request->getPath();
watch_response->xid = -1;
watch_response->error = response->error;
watch_response->type = Coordination::Event::NOTWATCHING;
results.push_back(ResponseForSession{session_id, watch_response});
}
}
if (response->error == Coordination::Error::ZOK)
@ -696,7 +677,7 @@ TestKeeperStorage::ResponsesForSessions TestKeeperStorage::processRequest(const
}
void TestKeeperStorage::clearDeadWatches(int64_t session_id)
void NuKeeperStorage::clearDeadWatches(int64_t session_id)
{
auto watches_it = sessions_and_watchers.find(session_id);
if (watches_it != sessions_and_watchers.end())

View File

@ -4,27 +4,28 @@
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Coordination/SessionExpiryQueue.h>
#include <unordered_map>
#include <unordered_set>
#include <vector>
namespace zkutil
namespace DB
{
using namespace DB;
struct TestKeeperStorageRequest;
using TestKeeperStorageRequestPtr = std::shared_ptr<TestKeeperStorageRequest>;
struct NuKeeperStorageRequest;
using NuKeeperStorageRequestPtr = std::shared_ptr<NuKeeperStorageRequest>;
using ResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr &)>;
class TestKeeperStorage
class NuKeeperStorage
{
public:
std::atomic<int64_t> session_id_counter{0};
int64_t session_id_counter{0};
struct Node
{
String data;
Coordination::ACLs acls;
Coordination::ACLs acls{};
bool is_ephemeral = false;
bool is_sequental = false;
Coordination::Stat stat{};
@ -50,6 +51,7 @@ public:
using Container = std::map<std::string, Node>;
using Ephemerals = std::unordered_map<int64_t, std::unordered_set<String>>;
using SessionAndWatcher = std::unordered_map<int64_t, std::unordered_set<String>>;
using SessionAndTimeout = std::unordered_map<int64_t, long>;
using SessionIDs = std::vector<int64_t>;
using Watches = std::map<String /* path, relative of root_path */, SessionIDs>;
@ -57,9 +59,11 @@ public:
Container container;
Ephemerals ephemerals;
SessionAndWatcher sessions_and_watchers;
SessionExpiryQueue session_expiry_queue;
SessionAndTimeout session_and_timeout;
std::atomic<int64_t> zxid{0};
std::atomic<bool> finalized{false};
int64_t zxid{0};
bool finalized{false};
Watches watches;
Watches list_watches; /// Watches for 'list' request (watches on children).
@ -68,18 +72,27 @@ public:
int64_t getZXID()
{
return zxid.fetch_add(1);
return zxid++;
}
public:
TestKeeperStorage();
NuKeeperStorage(int64_t tick_time_ms);
int64_t getSessionID(int64_t session_timeout_ms)
{
auto result = session_id_counter++;
session_and_timeout.emplace(result, session_timeout_ms);
session_expiry_queue.update(result, session_timeout_ms);
return result;
}
ResponsesForSessions processRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id);
ResponsesForSessions finalize(const RequestsForSessions & expired_requests);
int64_t getSessionID()
void finalize();
std::unordered_set<int64_t> getDeadSessions()
{
return session_id_counter.fetch_add(1);
return session_expiry_queue.getExpiredSessions();
}
};

View File

@ -0,0 +1,237 @@
#include <Coordination/NuKeeperStorageDispatcher.h>
#include <Common/setThreadName.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int TIMEOUT_EXCEEDED;
}
NuKeeperStorageDispatcher::NuKeeperStorageDispatcher()
: coordination_settings(std::make_shared<CoordinationSettings>())
, log(&Poco::Logger::get("NuKeeperDispatcher"))
{
}
void NuKeeperStorageDispatcher::requestThread()
{
setThreadName("NuKeeperReqT");
while (!shutdown_called)
{
NuKeeperStorage::RequestForSession request;
UInt64 max_wait = UInt64(coordination_settings->operation_timeout_ms.totalMilliseconds());
if (requests_queue.tryPop(request, max_wait))
{
if (shutdown_called)
break;
try
{
server->putRequest(request);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
}
void NuKeeperStorageDispatcher::responseThread()
{
setThreadName("NuKeeperRspT");
while (!shutdown_called)
{
NuKeeperStorage::ResponseForSession response_for_session;
UInt64 max_wait = UInt64(coordination_settings->operation_timeout_ms.totalMilliseconds());
if (responses_queue.tryPop(response_for_session, max_wait))
{
if (shutdown_called)
break;
try
{
setResponse(response_for_session.session_id, response_for_session.response);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
}
void NuKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response)
{
std::lock_guard lock(session_to_response_callback_mutex);
auto session_writer = session_to_response_callback.find(session_id);
if (session_writer == session_to_response_callback.end())
return;
session_writer->second(response);
/// Session closed, no more writes
if (response->xid != Coordination::WATCH_XID && response->getOpNum() == Coordination::OpNum::Close)
session_to_response_callback.erase(session_writer);
}
bool NuKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id)
{
{
std::lock_guard lock(session_to_response_callback_mutex);
if (session_to_response_callback.count(session_id) == 0)
return false;
}
NuKeeperStorage::RequestForSession request_info;
request_info.request = request;
request_info.session_id = session_id;
std::lock_guard lock(push_request_mutex);
/// Put close requests without timeouts
if (request->getOpNum() == Coordination::OpNum::Close)
requests_queue.push(std::move(request_info));
else if (!requests_queue.tryPush(std::move(request_info), coordination_settings->operation_timeout_ms.totalMilliseconds()))
throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED);
return true;
}
void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config)
{
LOG_DEBUG(log, "Initializing storage dispatcher");
int myid = config.getInt("test_keeper_server.server_id");
coordination_settings->loadFromConfig("test_keeper_server.coordination_settings", config);
server = std::make_unique<NuKeeperServer>(myid, coordination_settings, config, responses_queue);
try
{
LOG_DEBUG(log, "Waiting server to initialize");
server->startup();
LOG_DEBUG(log, "Server initialized, waiting for quorum");
server->waitInit();
LOG_DEBUG(log, "Quorum initialized");
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
throw;
}
request_thread = ThreadFromGlobalPool([this] { requestThread(); });
responses_thread = ThreadFromGlobalPool([this] { responseThread(); });
session_cleaner_thread = ThreadFromGlobalPool([this] { sessionCleanerTask(); });
LOG_DEBUG(log, "Dispatcher initialized");
}
void NuKeeperStorageDispatcher::shutdown()
{
try
{
{
std::lock_guard lock(push_request_mutex);
if (shutdown_called)
return;
LOG_DEBUG(log, "Shutting down storage dispatcher");
shutdown_called = true;
if (session_cleaner_thread.joinable())
session_cleaner_thread.join();
if (request_thread.joinable())
request_thread.join();
if (responses_thread.joinable())
responses_thread.join();
}
if (server)
server->shutdown();
NuKeeperStorage::RequestForSession request_for_session;
while (requests_queue.tryPop(request_for_session))
{
auto response = request_for_session.request->makeResponse();
response->error = Coordination::Error::ZSESSIONEXPIRED;
setResponse(request_for_session.session_id, response);
}
session_to_response_callback.clear();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
LOG_DEBUG(log, "Dispatcher shut down");
}
NuKeeperStorageDispatcher::~NuKeeperStorageDispatcher()
{
shutdown();
}
void NuKeeperStorageDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCallback callback)
{
std::lock_guard lock(session_to_response_callback_mutex);
if (!session_to_response_callback.try_emplace(session_id, callback).second)
throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Session with id {} already registered in dispatcher", session_id);
}
void NuKeeperStorageDispatcher::sessionCleanerTask()
{
while (true)
{
if (shutdown_called)
return;
try
{
if (isLeader())
{
auto dead_sessions = server->getDeadSessions();
for (int64_t dead_session : dead_sessions)
{
LOG_INFO(log, "Found dead session {}, will try to close it", dead_session);
Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close);
request->xid = Coordination::CLOSE_XID;
NuKeeperStorage::RequestForSession request_info;
request_info.request = request;
request_info.session_id = dead_session;
{
std::lock_guard lock(push_request_mutex);
requests_queue.push(std::move(request_info));
}
finishSession(dead_session);
LOG_INFO(log, "Dead session close request pushed");
}
}
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
std::this_thread::sleep_for(std::chrono::milliseconds(coordination_settings->dead_session_check_period_ms.totalMilliseconds()));
}
}
void NuKeeperStorageDispatcher::finishSession(int64_t session_id)
{
std::lock_guard lock(session_to_response_callback_mutex);
auto session_it = session_to_response_callback.find(session_id);
if (session_it != session_to_response_callback.end())
session_to_response_callback.erase(session_it);
}
}

View File

@ -0,0 +1,89 @@
#pragma once
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
# include "config_core.h"
#endif
#if USE_NURAFT
#include <Common/ThreadPool.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/Exception.h>
#include <common/logger_useful.h>
#include <functional>
#include <Coordination/NuKeeperServer.h>
#include <Coordination/CoordinationSettings.h>
namespace DB
{
using ZooKeeperResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr & response)>;
class NuKeeperStorageDispatcher
{
private:
std::mutex push_request_mutex;
CoordinationSettingsPtr coordination_settings;
using RequestsQueue = ConcurrentBoundedQueue<NuKeeperStorage::RequestForSession>;
RequestsQueue requests_queue{1};
ResponsesQueue responses_queue;
std::atomic<bool> shutdown_called{false};
using SessionToResponseCallback = std::unordered_map<int64_t, ZooKeeperResponseCallback>;
std::mutex session_to_response_callback_mutex;
SessionToResponseCallback session_to_response_callback;
ThreadFromGlobalPool request_thread;
ThreadFromGlobalPool responses_thread;
ThreadFromGlobalPool session_cleaner_thread;
std::unique_ptr<NuKeeperServer> server;
Poco::Logger * log;
private:
void requestThread();
void responseThread();
void sessionCleanerTask();
void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response);
public:
NuKeeperStorageDispatcher();
void initialize(const Poco::Util::AbstractConfiguration & config);
void shutdown();
~NuKeeperStorageDispatcher();
bool putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id);
bool isLeader() const
{
return server->isLeader();
}
bool hasLeader() const
{
return server->isLeaderAlive();
}
int64_t getSessionID(long session_timeout_ms)
{
return server->getSessionID(session_timeout_ms);
}
void registerSession(int64_t session_id, ZooKeeperResponseCallback callback);
/// Call if we don't need any responses for this session no more (session was expired)
void finishSession(int64_t session_id);
};
}
#endif

View File

@ -0,0 +1,87 @@
#include <Coordination/NuKeeperStorageSerializer.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
namespace DB
{
namespace
{
void writeNode(const NuKeeperStorage::Node & node, WriteBuffer & out)
{
Coordination::write(node.data, out);
Coordination::write(node.acls, out);
Coordination::write(node.is_ephemeral, out);
Coordination::write(node.is_sequental, out);
Coordination::write(node.stat, out);
Coordination::write(node.seq_num, out);
}
void readNode(NuKeeperStorage::Node & node, ReadBuffer & in)
{
Coordination::read(node.data, in);
Coordination::read(node.acls, in);
Coordination::read(node.is_ephemeral, in);
Coordination::read(node.is_sequental, in);
Coordination::read(node.stat, in);
Coordination::read(node.seq_num, in);
}
}
void NuKeeperStorageSerializer::serialize(const NuKeeperStorage & storage, WriteBuffer & out)
{
Coordination::write(storage.zxid, out);
Coordination::write(storage.session_id_counter, out);
Coordination::write(storage.container.size(), out);
for (const auto & [path, node] : storage.container)
{
Coordination::write(path, out);
writeNode(node, out);
}
Coordination::write(storage.ephemerals.size(), out);
for (const auto & [session_id, paths] : storage.ephemerals)
{
Coordination::write(session_id, out);
Coordination::write(paths.size(), out);
for (const auto & path : paths)
Coordination::write(path, out);
}
}
void NuKeeperStorageSerializer::deserialize(NuKeeperStorage & storage, ReadBuffer & in)
{
int64_t session_id_counter, zxid;
Coordination::read(zxid, in);
Coordination::read(session_id_counter, in);
storage.zxid = zxid;
storage.session_id_counter = session_id_counter;
size_t container_size;
Coordination::read(container_size, in);
while (storage.container.size() < container_size)
{
std::string path;
Coordination::read(path, in);
NuKeeperStorage::Node node;
readNode(node, in);
storage.container[path] = node;
}
size_t ephemerals_size;
Coordination::read(ephemerals_size, in);
while (storage.ephemerals.size() < ephemerals_size)
{
int64_t session_id;
size_t ephemerals_for_session;
Coordination::read(session_id, in);
Coordination::read(ephemerals_for_session, in);
while (storage.ephemerals[session_id].size() < ephemerals_for_session)
{
std::string ephemeral_path;
Coordination::read(ephemeral_path, in);
storage.ephemerals[session_id].emplace(ephemeral_path);
}
}
}
}

View File

@ -0,0 +1,17 @@
#pragma once
#include <Coordination/NuKeeperStorage.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
namespace DB
{
class NuKeeperStorageSerializer
{
public:
static void serialize(const NuKeeperStorage & storage, WriteBuffer & out);
static void deserialize(NuKeeperStorage & storage, ReadBuffer & in);
};
}

View File

@ -0,0 +1,20 @@
#pragma once
#include <IO/ReadBufferFromMemory.h>
#include <libnuraft/nuraft.hxx> // Y_IGNORE
namespace DB
{
class ReadBufferFromNuraftBuffer : public ReadBufferFromMemory
{
public:
explicit ReadBufferFromNuraftBuffer(nuraft::ptr<nuraft::buffer> buffer)
: ReadBufferFromMemory(buffer->data_begin(), buffer->size())
{}
explicit ReadBufferFromNuraftBuffer(nuraft::buffer & buffer)
: ReadBufferFromMemory(buffer.data_begin(), buffer.size())
{}
};
}

View File

@ -0,0 +1,83 @@
#include <Coordination/SessionExpiryQueue.h>
#include <common/logger_useful.h>
namespace DB
{
bool SessionExpiryQueue::remove(int64_t session_id)
{
auto session_it = session_to_timeout.find(session_id);
if (session_it != session_to_timeout.end())
{
auto set_it = expiry_to_sessions.find(session_it->second);
if (set_it != expiry_to_sessions.end())
set_it->second.erase(session_id);
return true;
}
return false;
}
bool SessionExpiryQueue::update(int64_t session_id, int64_t timeout_ms)
{
auto session_it = session_to_timeout.find(session_id);
int64_t now = getNowMilliseconds();
int64_t new_expiry_time = roundToNextInterval(now + timeout_ms);
if (session_it != session_to_timeout.end())
{
if (new_expiry_time == session_it->second)
return false;
auto set_it = expiry_to_sessions.find(new_expiry_time);
if (set_it == expiry_to_sessions.end())
std::tie(set_it, std::ignore) = expiry_to_sessions.emplace(new_expiry_time, std::unordered_set<int64_t>());
set_it->second.insert(session_id);
int64_t prev_expiry_time = session_it->second;
if (prev_expiry_time != new_expiry_time)
{
auto prev_set_it = expiry_to_sessions.find(prev_expiry_time);
if (prev_set_it != expiry_to_sessions.end())
prev_set_it->second.erase(session_id);
}
session_it->second = new_expiry_time;
return true;
}
else
{
session_to_timeout[session_id] = new_expiry_time;
auto set_it = expiry_to_sessions.find(new_expiry_time);
if (set_it == expiry_to_sessions.end())
std::tie(set_it, std::ignore) = expiry_to_sessions.emplace(new_expiry_time, std::unordered_set<int64_t>());
set_it->second.insert(session_id);
return false;
}
}
std::unordered_set<int64_t> SessionExpiryQueue::getExpiredSessions()
{
int64_t now = getNowMilliseconds();
if (now < next_expiration_time)
return {};
auto set_it = expiry_to_sessions.find(next_expiration_time);
int64_t new_expiration_time = next_expiration_time + expiration_interval;
next_expiration_time = new_expiration_time;
if (set_it != expiry_to_sessions.end())
{
auto result = set_it->second;
expiry_to_sessions.erase(set_it);
return result;
}
return {};
}
void SessionExpiryQueue::clear()
{
session_to_timeout.clear();
expiry_to_sessions.clear();
}
}

View File

@ -0,0 +1,45 @@
#pragma once
#include <unordered_map>
#include <unordered_set>
#include <chrono>
namespace DB
{
class SessionExpiryQueue
{
private:
std::unordered_map<int64_t, int64_t> session_to_timeout;
std::unordered_map<int64_t, std::unordered_set<int64_t>> expiry_to_sessions;
int64_t expiration_interval;
int64_t next_expiration_time;
static int64_t getNowMilliseconds()
{
using namespace std::chrono;
return duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count();
}
int64_t roundToNextInterval(int64_t time) const
{
return (time / expiration_interval + 1) * expiration_interval;
}
public:
explicit SessionExpiryQueue(int64_t expiration_interval_)
: expiration_interval(expiration_interval_)
, next_expiration_time(roundToNextInterval(getNowMilliseconds()))
{
}
bool remove(int64_t session_id);
bool update(int64_t session_id, int64_t timeout_ms);
std::unordered_set<int64_t> getExpiredSessions();
void clear();
};
}

View File

@ -0,0 +1,167 @@
#include <Coordination/SummingStateMachine.h>
#include <iostream>
#include <cstring>
namespace DB
{
static constexpr int MAX_SNAPSHOTS = 3;
static int64_t deserializeValue(nuraft::buffer & buffer)
{
nuraft::buffer_serializer bs(buffer);
int64_t result;
memcpy(&result, bs.get_raw(buffer.size()), sizeof(result));
return result;
}
SummingStateMachine::SummingStateMachine()
: value(0)
, last_committed_idx(0)
{
}
nuraft::ptr<nuraft::buffer> SummingStateMachine::commit(const size_t log_idx, nuraft::buffer & data)
{
int64_t value_to_add = deserializeValue(data);
value += value_to_add;
last_committed_idx = log_idx;
// Return Raft log number as a return result.
nuraft::ptr<nuraft::buffer> ret = nuraft::buffer::alloc(sizeof(log_idx));
nuraft::buffer_serializer bs(ret);
bs.put_u64(log_idx);
return ret;
}
bool SummingStateMachine::apply_snapshot(nuraft::snapshot & s)
{
std::lock_guard<std::mutex> ll(snapshots_lock);
auto entry = snapshots.find(s.get_last_log_idx());
if (entry == snapshots.end())
return false;
auto ctx = entry->second;
value = ctx->value;
return true;
}
nuraft::ptr<nuraft::snapshot> SummingStateMachine::last_snapshot()
{
// Just return the latest snapshot.
std::lock_guard<std::mutex> ll(snapshots_lock);
auto entry = snapshots.rbegin();
if (entry == snapshots.rend())
return nullptr;
auto ctx = entry->second;
return ctx->snapshot;
}
void SummingStateMachine::createSnapshotInternal(nuraft::snapshot & s)
{
// Clone snapshot from `s`.
nuraft::ptr<nuraft::buffer> snp_buf = s.serialize();
nuraft::ptr<nuraft::snapshot> ss = nuraft::snapshot::deserialize(*snp_buf);
// Put into snapshot map.
auto ctx = cs_new<SingleValueSnapshotContext>(ss, value);
snapshots[s.get_last_log_idx()] = ctx;
// Maintain last 3 snapshots only.
int num = snapshots.size();
auto entry = snapshots.begin();
for (int ii = 0; ii < num - MAX_SNAPSHOTS; ++ii)
{
if (entry == snapshots.end())
break;
entry = snapshots.erase(entry);
}
}
void SummingStateMachine::save_logical_snp_obj(
nuraft::snapshot & s,
size_t & obj_id,
nuraft::buffer & data,
bool /*is_first_obj*/,
bool /*is_last_obj*/)
{
if (obj_id == 0)
{
// Object ID == 0: it contains dummy value, create snapshot context.
createSnapshotInternal(s);
}
else
{
// Object ID > 0: actual snapshot value.
nuraft::buffer_serializer bs(data);
int64_t local_value = static_cast<int64_t>(bs.get_u64());
std::lock_guard<std::mutex> ll(snapshots_lock);
auto entry = snapshots.find(s.get_last_log_idx());
assert(entry != snapshots.end());
entry->second->value = local_value;
}
// Request next object.
obj_id++;
}
int SummingStateMachine::read_logical_snp_obj(
nuraft::snapshot & s,
void* & /*user_snp_ctx*/,
size_t obj_id,
nuraft::ptr<nuraft::buffer> & data_out,
bool & is_last_obj)
{
nuraft::ptr<SingleValueSnapshotContext> ctx = nullptr;
{
std::lock_guard<std::mutex> ll(snapshots_lock);
auto entry = snapshots.find(s.get_last_log_idx());
if (entry == snapshots.end())
{
// Snapshot doesn't exist.
data_out = nullptr;
is_last_obj = true;
return 0;
}
ctx = entry->second;
}
if (obj_id == 0)
{
// Object ID == 0: first object, put dummy data.
data_out = nuraft::buffer::alloc(sizeof(Int32));
nuraft::buffer_serializer bs(data_out);
bs.put_i32(0);
is_last_obj = false;
}
else
{
// Object ID > 0: second object, put actual value.
data_out = nuraft::buffer::alloc(sizeof(size_t));
nuraft::buffer_serializer bs(data_out);
bs.put_u64(ctx->value);
is_last_obj = true;
}
return 0;
}
void SummingStateMachine::create_snapshot(
nuraft::snapshot & s,
nuraft::async_result<bool>::handler_type & when_done)
{
{
std::lock_guard<std::mutex> ll(snapshots_lock);
createSnapshotInternal(s);
}
nuraft::ptr<std::exception> except(nullptr);
bool ret = true;
when_done(ret, except);
}
}

View File

@ -0,0 +1,78 @@
#pragma once
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <Core/Types.h>
#include <atomic>
#include <map>
#include <mutex>
namespace DB
{
/// Example trivial state machine.
class SummingStateMachine : public nuraft::state_machine
{
public:
SummingStateMachine();
nuraft::ptr<nuraft::buffer> pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; }
nuraft::ptr<nuraft::buffer> commit(const size_t log_idx, nuraft::buffer & data) override;
void rollback(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override {}
size_t last_commit_index() override { return last_committed_idx; }
bool apply_snapshot(nuraft::snapshot & s) override;
nuraft::ptr<nuraft::snapshot> last_snapshot() override;
void create_snapshot(
nuraft::snapshot & s,
nuraft::async_result<bool>::handler_type & when_done) override;
void save_logical_snp_obj(
nuraft::snapshot & s,
size_t & obj_id,
nuraft::buffer & data,
bool is_first_obj,
bool is_last_obj) override;
int read_logical_snp_obj(
nuraft::snapshot & s,
void* & user_snp_ctx,
size_t obj_id,
nuraft::ptr<nuraft::buffer> & data_out,
bool & is_last_obj) override;
int64_t getValue() const { return value; }
private:
struct SingleValueSnapshotContext
{
SingleValueSnapshotContext(nuraft::ptr<nuraft::snapshot> & s, int64_t v)
: snapshot(s)
, value(v)
{}
nuraft::ptr<nuraft::snapshot> snapshot;
int64_t value;
};
void createSnapshotInternal(nuraft::snapshot & s);
// State machine's current value.
std::atomic<int64_t> value;
// Last committed Raft log number.
std::atomic<uint64_t> last_committed_idx;
// Keeps the last 3 snapshots, by their Raft log numbers.
std::map<uint64_t, nuraft::ptr<SingleValueSnapshotContext>> snapshots;
// Mutex for `snapshots_`.
std::mutex snapshots_lock;
};
}

View File

@ -0,0 +1,45 @@
#pragma once
#include <queue>
#include <mutex>
namespace DB
{
/// Queue with mutex and condvar. As simple as possible.
template <typename T>
class ThreadSafeQueue
{
private:
mutable std::mutex queue_mutex;
std::condition_variable cv;
std::queue<T> queue;
public:
void push(const T & response)
{
std::lock_guard lock(queue_mutex);
queue.push(response);
cv.notify_one();
}
bool tryPop(T & response, int64_t timeout_ms = 0)
{
std::unique_lock lock(queue_mutex);
if (!cv.wait_for(lock,
std::chrono::milliseconds(timeout_ms), [this] { return !queue.empty(); }))
return false;
response = queue.front();
queue.pop();
return true;
}
size_t size() const
{
std::lock_guard lock(queue_mutex);
return queue.size();
}
};
}

View File

@ -0,0 +1,71 @@
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_WRITE_AFTER_END_OF_BUFFER;
}
void WriteBufferFromNuraftBuffer::nextImpl()
{
if (is_finished)
throw Exception("WriteBufferFromNuraftBuffer is finished", ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER);
/// pos may not be equal to vector.data() + old_size, because WriteBuffer::next() can be used to flush data
size_t pos_offset = pos - reinterpret_cast<Position>(buffer->data_begin());
size_t old_size = buffer->size();
if (pos_offset == old_size)
{
nuraft::ptr<nuraft::buffer> new_buffer = nuraft::buffer::alloc(old_size * size_multiplier);
memcpy(new_buffer->data_begin(), buffer->data_begin(), buffer->size());
buffer = new_buffer;
}
internal_buffer = Buffer(reinterpret_cast<Position>(buffer->data_begin() + pos_offset), reinterpret_cast<Position>(buffer->data_begin() + buffer->size()));
working_buffer = internal_buffer;
}
WriteBufferFromNuraftBuffer::WriteBufferFromNuraftBuffer()
: WriteBuffer(nullptr, 0)
{
buffer = nuraft::buffer::alloc(initial_size);
set(reinterpret_cast<Position>(buffer->data_begin()), buffer->size());
}
void WriteBufferFromNuraftBuffer::finalize()
{
if (is_finished)
return;
is_finished = true;
size_t real_size = pos - reinterpret_cast<Position>(buffer->data_begin());
nuraft::ptr<nuraft::buffer> new_buffer = nuraft::buffer::alloc(real_size);
memcpy(new_buffer->data_begin(), buffer->data_begin(), real_size);
buffer = new_buffer;
/// Prevent further writes.
set(nullptr, 0);
}
nuraft::ptr<nuraft::buffer> WriteBufferFromNuraftBuffer::getBuffer()
{
finalize();
return buffer;
}
WriteBufferFromNuraftBuffer::~WriteBufferFromNuraftBuffer()
{
try
{
finalize();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <IO/WriteBuffer.h>
#include <libnuraft/nuraft.hxx> // Y_IGNORE
namespace DB
{
class WriteBufferFromNuraftBuffer : public WriteBuffer
{
private:
nuraft::ptr<nuraft::buffer> buffer;
bool is_finished = false;
static constexpr size_t initial_size = 32;
static constexpr size_t size_multiplier = 2;
void nextImpl() override;
public:
WriteBufferFromNuraftBuffer();
void finalize() override final;
nuraft::ptr<nuraft::buffer> getBuffer();
bool isFinished() const { return is_finished; }
~WriteBufferFromNuraftBuffer() override;
};
}

View File

@ -0,0 +1,336 @@
#include <gtest/gtest.h>
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
# include "config_core.h"
#endif
#if USE_NURAFT
#include <Coordination/InMemoryLogStore.h>
#include <Coordination/InMemoryStateManager.h>
#include <Coordination/NuKeeperStorageSerializer.h>
#include <Coordination/SummingStateMachine.h>
#include <Coordination/NuKeeperStateMachine.h>
#include <Coordination/LoggerWrapper.h>
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <Coordination/ReadBufferFromNuraftBuffer.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <Common/Exception.h>
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <thread>
TEST(CoordinationTest, BuildTest)
{
DB::InMemoryLogStore store;
DB::SummingStateMachine machine;
EXPECT_EQ(1, 1);
}
TEST(CoordinationTest, BufferSerde)
{
Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Get);
request->xid = 3;
dynamic_cast<Coordination::ZooKeeperGetRequest *>(request.get())->path = "/path/value";
DB::WriteBufferFromNuraftBuffer wbuf;
request->write(wbuf);
auto nuraft_buffer = wbuf.getBuffer();
EXPECT_EQ(nuraft_buffer->size(), 28);
DB::ReadBufferFromNuraftBuffer rbuf(nuraft_buffer);
int32_t length;
Coordination::read(length, rbuf);
EXPECT_EQ(length + sizeof(length), nuraft_buffer->size());
int32_t xid;
Coordination::read(xid, rbuf);
EXPECT_EQ(xid, request->xid);
Coordination::OpNum opnum;
Coordination::read(opnum, rbuf);
Coordination::ZooKeeperRequestPtr request_read = Coordination::ZooKeeperRequestFactory::instance().get(opnum);
request_read->xid = xid;
request_read->readImpl(rbuf);
EXPECT_EQ(request_read->getOpNum(), Coordination::OpNum::Get);
EXPECT_EQ(request_read->xid, 3);
EXPECT_EQ(dynamic_cast<Coordination::ZooKeeperGetRequest *>(request_read.get())->path, "/path/value");
}
template <typename StateMachine>
struct SimpliestRaftServer
{
SimpliestRaftServer(int server_id_, const std::string & hostname_, int port_)
: server_id(server_id_)
, hostname(hostname_)
, port(port_)
, endpoint(hostname + ":" + std::to_string(port))
, state_machine(nuraft::cs_new<StateMachine>())
, state_manager(nuraft::cs_new<DB::InMemoryStateManager>(server_id, hostname, port))
{
nuraft::raft_params params;
params.heart_beat_interval_ = 100;
params.election_timeout_lower_bound_ = 200;
params.election_timeout_upper_bound_ = 400;
params.reserved_log_items_ = 5;
params.snapshot_distance_ = 1; /// forcefully send snapshots
params.client_req_timeout_ = 3000;
params.return_method_ = nuraft::raft_params::blocking;
raft_instance = launcher.init(
state_machine, state_manager, nuraft::cs_new<DB::LoggerWrapper>("ToyRaftLogger", DB::LogsLevel::trace), port,
nuraft::asio_service::options{}, params);
if (!raft_instance)
{
std::cerr << "Failed to initialize launcher (see the message "
"in the log file)." << std::endl;
exit(-1);
}
std::cout << "init Raft instance " << server_id;
for (size_t ii = 0; ii < 20; ++ii)
{
if (raft_instance->is_initialized())
{
std::cout << " done" << std::endl;
break;
}
std::cout << ".";
fflush(stdout);
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
}
// Server ID.
int server_id;
// Server address.
std::string hostname;
// Server port.
int port;
std::string endpoint;
// State machine.
nuraft::ptr<StateMachine> state_machine;
// State manager.
nuraft::ptr<nuraft::state_mgr> state_manager;
// Raft launcher.
nuraft::raft_launcher launcher;
// Raft server instance.
nuraft::ptr<nuraft::raft_server> raft_instance;
};
using SummingRaftServer = SimpliestRaftServer<DB::SummingStateMachine>;
nuraft::ptr<nuraft::buffer> getLogEntry(int64_t number)
{
nuraft::ptr<nuraft::buffer> ret = nuraft::buffer::alloc(sizeof(number));
nuraft::buffer_serializer bs(ret);
// WARNING: We don't consider endian-safety in this example.
bs.put_raw(&number, sizeof(number));
return ret;
}
TEST(CoordinationTest, TestSummingRaft1)
{
SummingRaftServer s1(1, "localhost", 44444);
/// Single node is leader
EXPECT_EQ(s1.raft_instance->get_leader(), 1);
auto entry1 = getLogEntry(143);
auto ret = s1.raft_instance->append_entries({entry1});
EXPECT_TRUE(ret->get_accepted()) << "failed to replicate: entry 1" << ret->get_result_code();
EXPECT_EQ(ret->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 1" << ret->get_result_code();
while (s1.state_machine->getValue() != 143)
{
std::cout << "Waiting s1 to apply entry\n";
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
EXPECT_EQ(s1.state_machine->getValue(), 143);
s1.launcher.shutdown(5);
}
TEST(CoordinationTest, TestSummingRaft3)
{
SummingRaftServer s1(1, "localhost", 44444);
SummingRaftServer s2(2, "localhost", 44445);
SummingRaftServer s3(3, "localhost", 44446);
nuraft::srv_config first_config(1, "localhost:44444");
auto ret1 = s2.raft_instance->add_srv(first_config);
if (!ret1->get_accepted())
{
std::cout << "failed to add server: "
<< ret1->get_result_str() << std::endl;
EXPECT_TRUE(false);
}
while (s1.raft_instance->get_leader() != 2)
{
std::cout << "Waiting s1 to join to s2 quorum\n";
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
nuraft::srv_config third_config(3, "localhost:44446");
auto ret3 = s2.raft_instance->add_srv(third_config);
if (!ret3->get_accepted())
{
std::cout << "failed to add server: "
<< ret3->get_result_str() << std::endl;
EXPECT_TRUE(false);
}
while (s3.raft_instance->get_leader() != 2)
{
std::cout << "Waiting s3 to join to s2 quorum\n";
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
/// S2 is leader
EXPECT_EQ(s1.raft_instance->get_leader(), 2);
EXPECT_EQ(s2.raft_instance->get_leader(), 2);
EXPECT_EQ(s3.raft_instance->get_leader(), 2);
std::cerr << "Starting to add entries\n";
auto entry = getLogEntry(1);
auto ret = s2.raft_instance->append_entries({entry});
EXPECT_TRUE(ret->get_accepted()) << "failed to replicate: entry 1" << ret->get_result_code();
EXPECT_EQ(ret->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 1" << ret->get_result_code();
while (s1.state_machine->getValue() != 1)
{
std::cout << "Waiting s1 to apply entry\n";
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
while (s2.state_machine->getValue() != 1)
{
std::cout << "Waiting s2 to apply entry\n";
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
while (s3.state_machine->getValue() != 1)
{
std::cout << "Waiting s3 to apply entry\n";
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
EXPECT_EQ(s1.state_machine->getValue(), 1);
EXPECT_EQ(s2.state_machine->getValue(), 1);
EXPECT_EQ(s3.state_machine->getValue(), 1);
auto non_leader_entry = getLogEntry(3);
auto ret_non_leader1 = s1.raft_instance->append_entries({non_leader_entry});
EXPECT_FALSE(ret_non_leader1->get_accepted());
auto ret_non_leader3 = s3.raft_instance->append_entries({non_leader_entry});
EXPECT_FALSE(ret_non_leader3->get_accepted());
auto leader_entry = getLogEntry(77);
auto ret_leader = s2.raft_instance->append_entries({leader_entry});
EXPECT_TRUE(ret_leader->get_accepted()) << "failed to replicate: entry 78" << ret_leader->get_result_code();
EXPECT_EQ(ret_leader->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 78" << ret_leader->get_result_code();
while (s1.state_machine->getValue() != 78)
{
std::cout << "Waiting s1 to apply entry\n";
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
while (s3.state_machine->getValue() != 78)
{
std::cout << "Waiting s3 to apply entry\n";
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
EXPECT_EQ(s1.state_machine->getValue(), 78);
EXPECT_EQ(s2.state_machine->getValue(), 78);
EXPECT_EQ(s3.state_machine->getValue(), 78);
s1.launcher.shutdown(5);
s2.launcher.shutdown(5);
s3.launcher.shutdown(5);
}
nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(int64_t session_id, const Coordination::ZooKeeperRequestPtr & request)
{
DB::WriteBufferFromNuraftBuffer buf;
DB::writeIntBinary(session_id, buf);
request->write(buf);
return buf.getBuffer();
}
DB::NuKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::ptr<nuraft::buffer> & buffer, const Coordination::ZooKeeperRequestPtr & request)
{
DB::NuKeeperStorage::ResponsesForSessions results;
DB::ReadBufferFromNuraftBuffer buf(buffer);
while (!buf.eof())
{
int64_t session_id;
DB::readIntBinary(session_id, buf);
int32_t length;
Coordination::XID xid;
int64_t zxid;
Coordination::Error err;
Coordination::read(length, buf);
Coordination::read(xid, buf);
Coordination::read(zxid, buf);
Coordination::read(err, buf);
auto response = request->makeResponse();
response->readImpl(buf);
results.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response});
}
return results;
}
TEST(CoordinationTest, TestStorageSerialization)
{
DB::NuKeeperStorage storage(500);
storage.container["/hello"] = DB::NuKeeperStorage::Node{.data="world"};
storage.container["/hello/somepath"] = DB::NuKeeperStorage::Node{.data="somedata"};
storage.session_id_counter = 5;
storage.zxid = 156;
storage.ephemerals[3] = {"/hello", "/"};
storage.ephemerals[1] = {"/hello/somepath"};
DB::WriteBufferFromOwnString buffer;
DB::NuKeeperStorageSerializer serializer;
serializer.serialize(storage, buffer);
std::string serialized = buffer.str();
EXPECT_NE(serialized.size(), 0);
DB::ReadBufferFromString read(serialized);
DB::NuKeeperStorage new_storage(500);
serializer.deserialize(new_storage, read);
EXPECT_EQ(new_storage.container.size(), 3);
EXPECT_EQ(new_storage.container["/hello"].data, "world");
EXPECT_EQ(new_storage.container["/hello/somepath"].data, "somedata");
EXPECT_EQ(new_storage.session_id_counter, 5);
EXPECT_EQ(new_storage.zxid, 156);
EXPECT_EQ(new_storage.ephemerals.size(), 2);
EXPECT_EQ(new_storage.ephemerals[3].size(), 2);
EXPECT_EQ(new_storage.ephemerals[1].size(), 1);
}
#endif

13
src/Coordination/ya.make Normal file
View File

@ -0,0 +1,13 @@
# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it.
OWNER(g:clickhouse)
LIBRARY()
PEERDIR(
clickhouse/src/Common
)
SRCS(
)
END()

View File

@ -0,0 +1,12 @@
OWNER(g:clickhouse)
LIBRARY()
PEERDIR(
clickhouse/src/Common
)
SRCS(
)
END()

View File

@ -13,3 +13,4 @@
#cmakedefine01 USE_LDAP
#cmakedefine01 USE_ROCKSDB
#cmakedefine01 USE_LIBPQXX
#cmakedefine01 USE_NURAFT

View File

@ -12,7 +12,7 @@
#include <Common/Stopwatch.h>
#include <Common/formatReadable.h>
#include <Common/thread_local_rng.h>
#include <Common/ZooKeeper/TestKeeperStorageDispatcher.h>
#include <Coordination/NuKeeperStorageDispatcher.h>
#include <Compression/ICompressionCodec.h>
#include <Core/BackgroundSchedulePool.h>
#include <Formats/FormatFactory.h>
@ -305,8 +305,10 @@ struct ContextShared
mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper.
ConfigurationPtr zookeeper_config; /// Stores zookeeper configs
mutable std::mutex test_keeper_storage_dispatcher_mutex;
mutable std::shared_ptr<zkutil::TestKeeperStorageDispatcher> test_keeper_storage_dispatcher;
#if USE_NURAFT
mutable std::mutex nu_keeper_storage_dispatcher_mutex;
mutable std::shared_ptr<NuKeeperStorageDispatcher> nu_keeper_storage_dispatcher;
#endif
mutable std::mutex auxiliary_zookeepers_mutex;
mutable std::map<String, zkutil::ZooKeeperPtr> auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients.
ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs
@ -449,8 +451,7 @@ struct ContextShared
trace_collector.reset();
/// Stop zookeeper connection
zookeeper.reset();
/// Stop test_keeper storage
test_keeper_storage_dispatcher.reset();
}
bool hasTraceCollector() const
@ -1583,15 +1584,48 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const
return shared->zookeeper;
}
std::shared_ptr<zkutil::TestKeeperStorageDispatcher> & Context::getTestKeeperStorageDispatcher() const
{
std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex);
if (!shared->test_keeper_storage_dispatcher)
shared->test_keeper_storage_dispatcher = std::make_shared<zkutil::TestKeeperStorageDispatcher>();
return shared->test_keeper_storage_dispatcher;
void Context::initializeNuKeeperStorageDispatcher() const
{
#if USE_NURAFT
std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex);
if (shared->nu_keeper_storage_dispatcher)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize NuKeeper multiple times");
const auto & config = getConfigRef();
if (config.has("test_keeper_server"))
{
shared->nu_keeper_storage_dispatcher = std::make_shared<NuKeeperStorageDispatcher>();
shared->nu_keeper_storage_dispatcher->initialize(config);
}
#endif
}
#if USE_NURAFT
std::shared_ptr<NuKeeperStorageDispatcher> & Context::getNuKeeperStorageDispatcher() const
{
std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex);
if (!shared->nu_keeper_storage_dispatcher)
throw Exception(ErrorCodes::LOGICAL_ERROR, "NuKeeper must be initialized before requests");
return shared->nu_keeper_storage_dispatcher;
}
#endif
void Context::shutdownNuKeeperStorageDispatcher() const
{
#if USE_NURAFT
std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex);
if (shared->nu_keeper_storage_dispatcher)
{
shared->nu_keeper_storage_dispatcher->shutdown();
shared->nu_keeper_storage_dispatcher.reset();
}
#endif
}
zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const
{
std::lock_guard lock(shared->auxiliary_zookeepers_mutex);

View File

@ -40,7 +40,6 @@ namespace Poco
namespace zkutil
{
class ZooKeeper;
class TestKeeperStorageDispatcher;
}
@ -109,6 +108,7 @@ class StoragePolicySelector;
using StoragePolicySelectorPtr = std::shared_ptr<const StoragePolicySelector>;
struct PartUUIDs;
using PartUUIDsPtr = std::shared_ptr<PartUUIDs>;
class NuKeeperStorageDispatcher;
class IOutputFormat;
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
@ -581,8 +581,11 @@ public:
/// Same as above but return a zookeeper connection from auxiliary_zookeepers configuration entry.
std::shared_ptr<zkutil::ZooKeeper> getAuxiliaryZooKeeper(const String & name) const;
std::shared_ptr<zkutil::TestKeeperStorageDispatcher> & getTestKeeperStorageDispatcher() const;
#if USE_NURAFT
std::shared_ptr<NuKeeperStorageDispatcher> & getNuKeeperStorageDispatcher() const;
#endif
void initializeNuKeeperStorageDispatcher() const;
void shutdownNuKeeperStorageDispatcher() const;
/// Set auxiliary zookeepers configuration at server starting or configuration reloading.
void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config);

View File

@ -1,4 +1,7 @@
#include <Server/TestKeeperTCPHandler.h>
#include <Server/NuKeeperTCPHandler.h>
#if USE_NURAFT
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <Core/Types.h>
#include <IO/WriteBufferFromPocoSocket.h>
@ -22,14 +25,17 @@
#include <poll.h>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int SYSTEM_ERROR;
extern const int LOGICAL_ERROR;
extern const int UNEXPECTED_PACKET_FROM_CLIENT;
extern const int TIMEOUT_EXCEEDED;
}
struct PollResult
@ -39,36 +45,6 @@ struct PollResult
bool error{false};
};
/// Queue with mutex. As simple as possible.
class ThreadSafeResponseQueue
{
private:
mutable std::mutex queue_mutex;
std::queue<Coordination::ZooKeeperResponsePtr> queue;
public:
void push(const Coordination::ZooKeeperResponsePtr & response)
{
std::lock_guard lock(queue_mutex);
queue.push(response);
}
bool tryPop(Coordination::ZooKeeperResponsePtr & response)
{
std::lock_guard lock(queue_mutex);
if (!queue.empty())
{
response = queue.front();
queue.pop();
return true;
}
return false;
}
size_t size() const
{
std::lock_guard lock(queue_mutex);
return queue.size();
}
};
struct SocketInterruptablePollWrapper
{
int sockfd;
@ -218,45 +194,47 @@ struct SocketInterruptablePollWrapper
#endif
};
TestKeeperTCPHandler::TestKeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_)
NuKeeperTCPHandler::NuKeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_)
: Poco::Net::TCPServerConnection(socket_)
, server(server_)
, log(&Poco::Logger::get("TestKeeperTCPHandler"))
, log(&Poco::Logger::get("NuKeeperTCPHandler"))
, global_context(server.context())
, test_keeper_storage_dispatcher(global_context.getTestKeeperStorageDispatcher())
, nu_keeper_storage_dispatcher(global_context.getNuKeeperStorageDispatcher())
, operation_timeout(0, global_context.getConfigRef().getUInt("test_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000)
, session_timeout(0, global_context.getConfigRef().getUInt("test_keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000)
, session_id(test_keeper_storage_dispatcher->getSessionID())
, poll_wrapper(std::make_unique<SocketInterruptablePollWrapper>(socket_))
, responses(std::make_unique<ThreadSafeResponseQueue>())
{
}
void TestKeeperTCPHandler::sendHandshake()
void NuKeeperTCPHandler::sendHandshake(bool has_leader)
{
Coordination::write(Coordination::SERVER_HANDSHAKE_LENGTH, *out);
Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out);
Coordination::write(Coordination::DEFAULT_SESSION_TIMEOUT_MS, *out);
if (has_leader)
Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out);
else /// Specially ignore connections if we are not leader, client will throw exception
Coordination::write(42, *out);
Coordination::write(static_cast<int32_t>(session_timeout.totalMilliseconds()), *out);
Coordination::write(session_id, *out);
std::array<char, Coordination::PASSWORD_LENGTH> passwd{};
Coordination::write(passwd, *out);
out->next();
}
void TestKeeperTCPHandler::run()
void NuKeeperTCPHandler::run()
{
runImpl();
}
void TestKeeperTCPHandler::receiveHandshake()
Poco::Timespan NuKeeperTCPHandler::receiveHandshake()
{
int32_t handshake_length;
int32_t protocol_version;
int64_t last_zxid_seen;
int32_t timeout;
int32_t timeout_ms;
int64_t previous_session_id = 0; /// We don't support session restore. So previous session_id is always zero.
std::array<char, Coordination::PASSWORD_LENGTH> passwd {};
Coordination::read(handshake_length, *in);
if (handshake_length != Coordination::CLIENT_HANDSHAKE_LENGTH && handshake_length != Coordination::CLIENT_HANDSHAKE_LENGTH_WITH_READONLY)
throw Exception("Unexpected handshake length received: " + toString(handshake_length), ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
@ -271,7 +249,7 @@ void TestKeeperTCPHandler::receiveHandshake()
if (last_zxid_seen != 0)
throw Exception("Non zero last_zxid_seen is not supported", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
Coordination::read(timeout, *in);
Coordination::read(timeout_ms, *in);
Coordination::read(previous_session_id, *in);
if (previous_session_id != 0)
@ -282,10 +260,12 @@ void TestKeeperTCPHandler::receiveHandshake()
int8_t readonly;
if (handshake_length == Coordination::CLIENT_HANDSHAKE_LENGTH_WITH_READONLY)
Coordination::read(readonly, *in);
return Poco::Timespan(0, timeout_ms * 1000);
}
void TestKeeperTCPHandler::runImpl()
void NuKeeperTCPHandler::runImpl()
{
setThreadName("TstKprHandler");
ThreadStatus thread_status;
@ -307,7 +287,9 @@ void TestKeeperTCPHandler::runImpl()
try
{
receiveHandshake();
auto client_timeout = receiveHandshake();
if (client_timeout != 0)
session_timeout = std::min(client_timeout, session_timeout);
}
catch (const Exception & e) /// Typical for an incorrect username, password, or address.
{
@ -315,7 +297,30 @@ void TestKeeperTCPHandler::runImpl()
return;
}
sendHandshake();
if (nu_keeper_storage_dispatcher->hasLeader())
{
try
{
LOG_INFO(log, "Requesting session ID for the new client");
session_id = nu_keeper_storage_dispatcher->getSessionID(session_timeout.totalMilliseconds());
LOG_INFO(log, "Received session ID {}", session_id);
}
catch (const Exception & e)
{
LOG_WARNING(log, "Cannot receive session id {}", e.displayText());
sendHandshake(false);
return;
}
sendHandshake(true);
}
else
{
LOG_WARNING(log, "Ignoring user request, because no alive leader exist");
sendHandshake(false);
return;
}
auto response_fd = poll_wrapper->getResponseFD();
auto response_callback = [this, response_fd] (const Coordination::ZooKeeperResponsePtr & response)
@ -324,7 +329,7 @@ void TestKeeperTCPHandler::runImpl()
UInt8 single_byte = 1;
[[maybe_unused]] int result = write(response_fd, &single_byte, sizeof(single_byte));
};
test_keeper_storage_dispatcher->registerSession(session_id, response_callback);
nu_keeper_storage_dispatcher->registerSession(session_id, response_callback);
session_stopwatch.start();
bool close_received = false;
@ -371,12 +376,13 @@ void TestKeeperTCPHandler::runImpl()
LOG_DEBUG(log, "Session #{} successfully closed", session_id);
return;
}
if (response->error == Coordination::Error::ZOK)
response->write(*out);
else if (response->xid != Coordination::WATCH_XID)
response->write(*out);
/// skipping bad response for watch
response->write(*out);
if (response->error == Coordination::Error::ZSESSIONEXPIRED)
{
LOG_DEBUG(log, "Session #{} expired because server shutting down or quorum is not alive", session_id);
nu_keeper_storage_dispatcher->finishSession(session_id);
return;
}
result.ready_responses_count--;
}
@ -386,7 +392,7 @@ void TestKeeperTCPHandler::runImpl()
if (session_stopwatch.elapsedMicroseconds() > static_cast<UInt64>(session_timeout.totalMicroseconds()))
{
LOG_DEBUG(log, "Session #{} expired", session_id);
finish();
nu_keeper_storage_dispatcher->finishSession(session_id);
break;
}
}
@ -394,22 +400,11 @@ void TestKeeperTCPHandler::runImpl()
catch (const Exception & ex)
{
LOG_INFO(log, "Got exception processing session #{}: {}", session_id, getExceptionMessage(ex, true));
finish();
nu_keeper_storage_dispatcher->finishSession(session_id);
}
}
void TestKeeperTCPHandler::finish()
{
Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close);
request->xid = close_xid;
/// Put close request (so storage will remove all info about session)
test_keeper_storage_dispatcher->putRequest(request, session_id);
/// We don't need any callbacks because session can be already dead and
/// nobody wait for response
test_keeper_storage_dispatcher->finishSession(session_id);
}
std::pair<Coordination::OpNum, Coordination::XID> TestKeeperTCPHandler::receiveRequest()
std::pair<Coordination::OpNum, Coordination::XID> NuKeeperTCPHandler::receiveRequest()
{
int32_t length;
Coordination::read(length, *in);
@ -423,8 +418,11 @@ std::pair<Coordination::OpNum, Coordination::XID> TestKeeperTCPHandler::receiveR
request->xid = xid;
request->readImpl(*in);
test_keeper_storage_dispatcher->putRequest(request, session_id);
if (!nu_keeper_storage_dispatcher->putRequest(request, session_id))
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Session {} already disconnected", session_id);
return std::make_pair(opnum, xid);
}
}
#endif

View File

@ -1,14 +1,22 @@
#pragma once
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
# include "config_core.h"
#endif
#if USE_NURAFT
#include <Poco/Net/TCPServerConnection.h>
#include "IServer.h"
#include <Common/Stopwatch.h>
#include <Interpreters/Context.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Common/ZooKeeper/ZooKeeperConstants.h>
#include <Common/ZooKeeper/TestKeeperStorageDispatcher.h>
#include <Coordination/NuKeeperStorageDispatcher.h>
#include <IO/WriteBufferFromPocoSocket.h>
#include <IO/ReadBufferFromPocoSocket.h>
#include <Coordination/ThreadSafeQueue.h>
#include <unordered_map>
namespace DB
@ -16,22 +24,24 @@ namespace DB
struct SocketInterruptablePollWrapper;
using SocketInterruptablePollWrapperPtr = std::unique_ptr<SocketInterruptablePollWrapper>;
class ThreadSafeResponseQueue;
using ThreadSafeResponseQueue = ThreadSafeQueue<Coordination::ZooKeeperResponsePtr>;
using ThreadSafeResponseQueuePtr = std::unique_ptr<ThreadSafeResponseQueue>;
class TestKeeperTCPHandler : public Poco::Net::TCPServerConnection
class NuKeeperTCPHandler : public Poco::Net::TCPServerConnection
{
public:
TestKeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_);
NuKeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_);
void run() override;
private:
IServer & server;
Poco::Logger * log;
Context global_context;
std::shared_ptr<zkutil::TestKeeperStorageDispatcher> test_keeper_storage_dispatcher;
std::shared_ptr<NuKeeperStorageDispatcher> nu_keeper_storage_dispatcher;
Poco::Timespan operation_timeout;
Poco::Timespan session_timeout;
int64_t session_id;
int64_t session_id{-1};
Stopwatch session_stopwatch;
SocketInterruptablePollWrapperPtr poll_wrapper;
@ -45,11 +55,11 @@ private:
void runImpl();
void sendHandshake();
void receiveHandshake();
void sendHandshake(bool has_leader);
Poco::Timespan receiveHandshake();
std::pair<Coordination::OpNum, Coordination::XID> receiveRequest();
void finish();
};
}
#endif

View File

@ -1,5 +1,6 @@
#pragma once
#include <Server/TestKeeperTCPHandler.h>
#include <Server/NuKeeperTCPHandler.h>
#include <Poco/Net/TCPServerConnectionFactory.h>
#include <Poco/Net/NetException.h>
#include <common/logger_useful.h>
@ -8,7 +9,7 @@
namespace DB
{
class TestKeeperTCPHandlerFactory : public Poco::Net::TCPServerConnectionFactory
class NuKeeperTCPHandlerFactory : public Poco::Net::TCPServerConnectionFactory
{
private:
IServer & server;
@ -20,9 +21,9 @@ private:
void run() override {}
};
public:
TestKeeperTCPHandlerFactory(IServer & server_)
NuKeeperTCPHandlerFactory(IServer & server_)
: server(server_)
, log(&Poco::Logger::get("TestKeeperTCPHandlerFactory"))
, log(&Poco::Logger::get("NuKeeperTCPHandlerFactory"))
{
}
@ -30,8 +31,8 @@ public:
{
try
{
LOG_TRACE(log, "Test keeper request. Address: {}", socket.peerAddress().toString());
return new TestKeeperTCPHandler(server, socket);
LOG_TRACE(log, "NuKeeper request. Address: {}", socket.peerAddress().toString());
return new NuKeeperTCPHandler(server, socket);
}
catch (const Poco::Net::NetException &)
{

View File

@ -17,6 +17,7 @@ SRCS(
MySQLHandler.cpp
MySQLHandlerFactory.cpp
NotFoundHandler.cpp
NuKeeperTCPHandler.cpp
PostgreSQLHandler.cpp
PostgreSQLHandlerFactory.cpp
PrometheusMetricsWriter.cpp
@ -25,7 +26,6 @@ SRCS(
ReplicasStatusHandler.cpp
StaticRequestHandler.cpp
TCPHandler.cpp
TestKeeperTCPHandler.cpp
WebUIRequestHandler.cpp
)

View File

@ -9,6 +9,7 @@ PEERDIR(
clickhouse/src/Columns
clickhouse/src/Common
clickhouse/src/Compression
clickhouse/src/Coordination
clickhouse/src/Core
clickhouse/src/Databases
clickhouse/src/DataStreams

View File

@ -1,7 +1,19 @@
<yandex>
<test_keeper_server>
<tcp_port>9181</tcp_port>
<operation_timeout_ms>10000</operation_timeout_ms>
<session_timeout_ms>30000</session_timeout_ms>
<server_id>1</server_id>
<coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms>
<session_timeout_ms>30000</session_timeout_ms>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>localhost</hostname>
<port>44444</port>
</server>
</raft_configuration>
</test_keeper_server>
</yandex>

View File

@ -1,7 +1,20 @@
<yandex>
<test_keeper_server>
<tcp_port>9181</tcp_port>
<operation_timeout_ms>10000</operation_timeout_ms>
<session_timeout_ms>30000</session_timeout_ms>
<server_id>1</server_id>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>localhost</hostname>
<port>44444</port>
</server>
</raft_configuration>
</test_keeper_server>
</yandex>

View File

@ -25,7 +25,7 @@ def get_fake_zk():
global _fake_zk_instance
if not _fake_zk_instance:
print("node", cluster.get_instance_ip("node"))
_fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip("node") + ":9181")
_fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip("node") + ":9181", timeout=30.0)
def reset_last_zxid_listener(state):
print("Fake zk callback called for state", state)
global _fake_zk_instance

View File

@ -0,0 +1 @@
#!/usr/bin/env python3

View File

@ -0,0 +1,38 @@
<yandex>
<test_keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<priority>3</priority>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>2</priority>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</yandex>

View File

@ -0,0 +1,38 @@
<yandex>
<test_keeper_server>
<tcp_port>9181</tcp_port>
<server_id>2</server_id>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<priority>3</priority>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>2</priority>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</yandex>

View File

@ -0,0 +1,38 @@
<yandex>
<test_keeper_server>
<tcp_port>9181</tcp_port>
<server_id>3</server_id>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<priority>3</priority>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>2</priority>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</yandex>

View File

@ -0,0 +1,12 @@
<yandex>
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/log.log</log>
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
<size>1000M</size>
<count>10</count>
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
</logger>
</yandex>

View File

@ -0,0 +1,16 @@
<yandex>
<zookeeper>
<node index="1">
<host>node1</host>
<port>9181</port>
</node>
<node index="2">
<host>node2</host>
<port>9181</port>
</node>
<node index="3">
<host>node3</host>
<port>9181</port>
</node>
</zookeeper>
</yandex>

View File

@ -0,0 +1,321 @@
import pytest
from helpers.cluster import ClickHouseCluster
import random
import string
import os
import time
from multiprocessing.dummy import Pool
from helpers.network import PartitionManager
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
node2 = cluster.add_instance('node2', main_configs=['configs/enable_test_keeper2.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
from kazoo.client import KazooClient, KazooState
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def smaller_exception(ex):
return '\n'.join(str(ex).split('\n')[0:2])
def wait_node(node):
for _ in range(100):
zk = None
try:
node.query("SELECT * FROM system.zookeeper WHERE path = '/'")
zk = get_fake_zk(node.name, timeout=30.0)
zk.create("/test", sequence=True)
print("node", node.name, "ready")
break
except Exception as ex:
time.sleep(0.2)
print("Waiting until", node.name, "will be ready, exception", ex)
finally:
if zk:
zk.stop()
zk.close()
else:
raise Exception("Can't wait node", node.name, "to become ready")
def wait_nodes():
for node in [node1, node2, node3]:
wait_node(node)
def get_fake_zk(nodename, timeout=30.0):
_fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout)
def reset_listener(state):
nonlocal _fake_zk_instance
print("Fake zk callback called for state", state)
if state != KazooState.CONNECTED:
_fake_zk_instance._reset()
_fake_zk_instance.add_listener(reset_listener)
_fake_zk_instance.start()
return _fake_zk_instance
# in extremely rare case it can take more than 5 minutes in debug build with sanitizer
@pytest.mark.timeout(600)
def test_blocade_leader(started_cluster):
wait_nodes()
for i, node in enumerate([node1, node2, node3]):
node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary")
node.query("CREATE TABLE ordinary.t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format(i + 1))
node2.query("INSERT INTO ordinary.t1 SELECT number FROM numbers(10)")
node1.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10)
node3.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10)
assert node1.query("SELECT COUNT() FROM ordinary.t1") == "10\n"
assert node2.query("SELECT COUNT() FROM ordinary.t1") == "10\n"
assert node3.query("SELECT COUNT() FROM ordinary.t1") == "10\n"
with PartitionManager() as pm:
pm.partition_instances(node2, node1)
pm.partition_instances(node3, node1)
for i in range(100):
try:
node2.query("SYSTEM RESTART REPLICA ordinary.t1")
node2.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100)")
break
except Exception as ex:
try:
node2.query("ATTACH TABLE ordinary.t1")
except Exception as attach_ex:
print("Got exception node2", smaller_exception(attach_ex))
print("Got exception node2", smaller_exception(ex))
time.sleep(0.5)
else:
for num, node in enumerate([node1, node2, node3]):
dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1))
assert False, "Cannot insert anything node2"
for i in range(100):
try:
node3.query("SYSTEM RESTART REPLICA ordinary.t1")
node3.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100)")
break
except Exception as ex:
try:
node3.query("ATTACH TABLE ordinary.t1")
except Exception as attach_ex:
print("Got exception node3", smaller_exception(attach_ex))
print("Got exception node3", smaller_exception(ex))
time.sleep(0.5)
else:
for num, node in enumerate([node1, node2, node3]):
dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1))
assert False, "Cannot insert anything node3"
for n, node in enumerate([node1, node2, node3]):
for i in range(100):
try:
node.query("SYSTEM RESTART REPLICA ordinary.t1")
break
except Exception as ex:
try:
node.query("ATTACH TABLE ordinary.t1")
except Exception as attach_ex:
print("Got exception node{}".format(n + 1), smaller_exception(attach_ex))
print("Got exception node{}".format(n + 1), smaller_exception(ex))
time.sleep(0.5)
else:
assert False, "Cannot reconnect for node{}".format(n + 1)
for i in range(100):
try:
node1.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100)")
break
except Exception as ex:
print("Got exception node1", smaller_exception(ex))
time.sleep(0.5)
else:
for num, node in enumerate([node1, node2, node3]):
dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1))
assert False, "Cannot insert anything node1"
for n, node in enumerate([node1, node2, node3]):
for i in range(100):
try:
node.query("SYSTEM RESTART REPLICA ordinary.t1")
node.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10)
break
except Exception as ex:
try:
node.query("ATTACH TABLE ordinary.t1")
except Exception as attach_ex:
print("Got exception node{}".format(n + 1), smaller_exception(attach_ex))
print("Got exception node{}".format(n + 1), smaller_exception(ex))
time.sleep(0.5)
else:
for num, node in enumerate([node1, node2, node3]):
dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1))
assert False, "Cannot sync replica node{}".format(n+1)
if node1.query("SELECT COUNT() FROM ordinary.t1") != "310\n":
for num, node in enumerate([node1, node2, node3]):
dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1))
assert node1.query("SELECT COUNT() FROM ordinary.t1") == "310\n"
assert node2.query("SELECT COUNT() FROM ordinary.t1") == "310\n"
assert node3.query("SELECT COUNT() FROM ordinary.t1") == "310\n"
def dump_zk(node, zk_path, replica_path):
print(node.query("SELECT * FROM system.replication_queue FORMAT Vertical"))
print("Replicas")
print(node.query("SELECT * FROM system.replicas FORMAT Vertical"))
print("Replica 2 info")
print(node.query("SELECT * FROM system.zookeeper WHERE path = '{}' FORMAT Vertical".format(zk_path)))
print("Queue")
print(node.query("SELECT * FROM system.zookeeper WHERE path = '{}/queue' FORMAT Vertical".format(replica_path)))
print("Log")
print(node.query("SELECT * FROM system.zookeeper WHERE path = '{}/log' FORMAT Vertical".format(zk_path)))
print("Parts")
print(node.query("SELECT name FROM system.zookeeper WHERE path = '{}/parts' FORMAT Vertical".format(replica_path)))
# in extremely rare case it can take more than 5 minutes in debug build with sanitizer
@pytest.mark.timeout(600)
def test_blocade_leader_twice(started_cluster):
wait_nodes()
for i, node in enumerate([node1, node2, node3]):
node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary")
node.query("CREATE TABLE ordinary.t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format(i + 1))
node2.query("INSERT INTO ordinary.t2 SELECT number FROM numbers(10)")
node1.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10)
node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10)
assert node1.query("SELECT COUNT() FROM ordinary.t2") == "10\n"
assert node2.query("SELECT COUNT() FROM ordinary.t2") == "10\n"
assert node3.query("SELECT COUNT() FROM ordinary.t2") == "10\n"
with PartitionManager() as pm:
pm.partition_instances(node2, node1)
pm.partition_instances(node3, node1)
for i in range(100):
try:
node2.query("SYSTEM RESTART REPLICA ordinary.t2")
node2.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)")
break
except Exception as ex:
try:
node2.query("ATTACH TABLE ordinary.t2")
except Exception as attach_ex:
print("Got exception node2", smaller_exception(attach_ex))
print("Got exception node2", smaller_exception(ex))
time.sleep(0.5)
else:
for num, node in enumerate([node1, node2, node3]):
dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1))
assert False, "Cannot reconnect for node2"
for i in range(100):
try:
node3.query("SYSTEM RESTART REPLICA ordinary.t2")
node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)")
break
except Exception as ex:
try:
node3.query("ATTACH TABLE ordinary.t2")
except Exception as attach_ex:
print("Got exception node3", smaller_exception(attach_ex))
print("Got exception node3", smaller_exception(ex))
time.sleep(0.5)
else:
for num, node in enumerate([node1, node2, node3]):
dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1))
assert False, "Cannot reconnect for node3"
# Total network partition
pm.partition_instances(node3, node2)
for i in range(10):
try:
node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)")
assert False, "Node3 became leader?"
except Exception as ex:
time.sleep(0.5)
for i in range(10):
try:
node2.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)")
assert False, "Node2 became leader?"
except Exception as ex:
time.sleep(0.5)
for n, node in enumerate([node1, node2, node3]):
for i in range(100):
try:
node.query("SYSTEM RESTART REPLICA ordinary.t2")
break
except Exception as ex:
try:
node.query("ATTACH TABLE ordinary.t2")
except Exception as attach_ex:
print("Got exception node{}".format(n + 1), smaller_exception(attach_ex))
print("Got exception node{}".format(n + 1), smaller_exception(ex))
time.sleep(0.5)
else:
for num, node in enumerate([node1, node2, node3]):
dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1))
assert False, "Cannot reconnect for node{}".format(n + 1)
for n, node in enumerate([node1, node2, node3]):
for i in range(100):
try:
node.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)")
break
except Exception as ex:
print("Got exception node{}".format(n + 1), smaller_exception(ex))
time.sleep(0.5)
else:
for num, node in enumerate([node1, node2, node3]):
dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1))
assert False, "Cannot reconnect for node{}".format(n + 1)
for n, node in enumerate([node1, node2, node3]):
for i in range(100):
try:
node.query("SYSTEM RESTART REPLICA ordinary.t2")
node.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10)
break
except Exception as ex:
try:
node.query("ATTACH TABLE ordinary.t2")
except Exception as attach_ex:
print("Got exception node{}".format(n + 1), smaller_exception(attach_ex))
print("Got exception node{}".format(n + 1), smaller_exception(ex))
time.sleep(0.5)
else:
for num, node in enumerate([node1, node2, node3]):
dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1))
assert False, "Cannot reconnect for node{}".format(n + 1)
assert node1.query("SELECT COUNT() FROM ordinary.t2") == "510\n"
if node2.query("SELECT COUNT() FROM ordinary.t2") != "510\n":
for num, node in enumerate([node1, node2, node3]):
dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1))
assert node2.query("SELECT COUNT() FROM ordinary.t2") == "510\n"
assert node3.query("SELECT COUNT() FROM ordinary.t2") == "510\n"

View File

@ -0,0 +1 @@
#!/usr/bin/env python3

View File

@ -0,0 +1,38 @@
<yandex>
<test_keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<priority>3</priority>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>2</priority>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</yandex>

View File

@ -0,0 +1,38 @@
<yandex>
<test_keeper_server>
<tcp_port>9181</tcp_port>
<server_id>2</server_id>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<priority>3</priority>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>2</priority>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</yandex>

View File

@ -0,0 +1,38 @@
<yandex>
<test_keeper_server>
<tcp_port>9181</tcp_port>
<server_id>3</server_id>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>node1</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<priority>3</priority>
</server>
<server>
<id>2</id>
<hostname>node2</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>2</priority>
</server>
<server>
<id>3</id>
<hostname>node3</hostname>
<port>44444</port>
<can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower>
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</yandex>

View File

@ -0,0 +1,12 @@
<yandex>
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/log.log</log>
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
<size>1000M</size>
<count>10</count>
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
</logger>
</yandex>

View File

@ -0,0 +1,16 @@
<yandex>
<zookeeper>
<node index="1">
<host>node1</host>
<port>9181</port>
</node>
<node index="2">
<host>node2</host>
<port>9181</port>
</node>
<node index="3">
<host>node3</host>
<port>9181</port>
</node>
</zookeeper>
</yandex>

View File

@ -0,0 +1,239 @@
import pytest
from helpers.cluster import ClickHouseCluster
import random
import string
import os
import time
from multiprocessing.dummy import Pool
from helpers.network import PartitionManager
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
node2 = cluster.add_instance('node2', main_configs=['configs/enable_test_keeper2.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
from kazoo.client import KazooClient, KazooState
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def smaller_exception(ex):
return '\n'.join(str(ex).split('\n')[0:2])
def wait_node(node):
for _ in range(100):
zk = None
try:
node.query("SELECT * FROM system.zookeeper WHERE path = '/'")
zk = get_fake_zk(node.name, timeout=30.0)
zk.create("/test", sequence=True)
print("node", node.name, "ready")
break
except Exception as ex:
time.sleep(0.2)
print("Waiting until", node.name, "will be ready, exception", ex)
finally:
if zk:
zk.stop()
zk.close()
else:
raise Exception("Can't wait node", node.name, "to become ready")
def wait_nodes():
for node in [node1, node2, node3]:
wait_node(node)
def get_fake_zk(nodename, timeout=30.0):
_fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout)
def reset_listener(state):
nonlocal _fake_zk_instance
print("Fake zk callback called for state", state)
if state != KazooState.CONNECTED:
_fake_zk_instance._reset()
_fake_zk_instance.add_listener(reset_listener)
_fake_zk_instance.start()
return _fake_zk_instance
def test_read_write_multinode(started_cluster):
try:
wait_nodes()
node1_zk = get_fake_zk("node1")
node2_zk = get_fake_zk("node2")
node3_zk = get_fake_zk("node3")
node1_zk.create("/test_read_write_multinode_node1", b"somedata1")
node2_zk.create("/test_read_write_multinode_node2", b"somedata2")
node3_zk.create("/test_read_write_multinode_node3", b"somedata3")
# stale reads are allowed
while node1_zk.exists("/test_read_write_multinode_node2") is None:
time.sleep(0.1)
while node1_zk.exists("/test_read_write_multinode_node3") is None:
time.sleep(0.1)
while node2_zk.exists("/test_read_write_multinode_node3") is None:
time.sleep(0.1)
assert node3_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1"
assert node2_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1"
assert node1_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1"
assert node3_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2"
assert node2_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2"
assert node1_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2"
assert node3_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3"
assert node2_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3"
assert node1_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3"
finally:
try:
for zk_conn in [node1_zk, node2_zk, node3_zk]:
zk_conn.stop()
zk_conn.close()
except:
pass
def test_watch_on_follower(started_cluster):
try:
wait_nodes()
node1_zk = get_fake_zk("node1")
node2_zk = get_fake_zk("node2")
node3_zk = get_fake_zk("node3")
node1_zk.create("/test_data_watches")
node2_zk.set("/test_data_watches", b"hello")
node3_zk.set("/test_data_watches", b"world")
node1_data = None
def node1_callback(event):
print("node1 data watch called")
nonlocal node1_data
node1_data = event
node1_zk.get("/test_data_watches", watch=node1_callback)
node2_data = None
def node2_callback(event):
print("node2 data watch called")
nonlocal node2_data
node2_data = event
node2_zk.get("/test_data_watches", watch=node2_callback)
node3_data = None
def node3_callback(event):
print("node3 data watch called")
nonlocal node3_data
node3_data = event
node3_zk.get("/test_data_watches", watch=node3_callback)
node1_zk.set("/test_data_watches", b"somevalue")
time.sleep(3)
print(node1_data)
print(node2_data)
print(node3_data)
assert node1_data == node2_data
assert node3_data == node2_data
finally:
try:
for zk_conn in [node1_zk, node2_zk, node3_zk]:
zk_conn.stop()
zk_conn.close()
except:
pass
def test_session_expiration(started_cluster):
try:
wait_nodes()
node1_zk = get_fake_zk("node1")
node2_zk = get_fake_zk("node2")
node3_zk = get_fake_zk("node3", timeout=3.0)
print("Node3 session id", node3_zk._session_id)
node3_zk.create("/test_ephemeral_node", b"world", ephemeral=True)
with PartitionManager() as pm:
pm.partition_instances(node3, node2)
pm.partition_instances(node3, node1)
node3_zk.stop()
node3_zk.close()
for _ in range(100):
if node1_zk.exists("/test_ephemeral_node") is None and node2_zk.exists("/test_ephemeral_node") is None:
break
print("Node1 exists", node1_zk.exists("/test_ephemeral_node"))
print("Node2 exists", node2_zk.exists("/test_ephemeral_node"))
time.sleep(0.1)
node1_zk.sync("/")
node2_zk.sync("/")
assert node1_zk.exists("/test_ephemeral_node") is None
assert node2_zk.exists("/test_ephemeral_node") is None
finally:
try:
for zk_conn in [node1_zk, node2_zk, node3_zk]:
try:
zk_conn.stop()
zk_conn.close()
except:
pass
except:
pass
def test_follower_restart(started_cluster):
try:
wait_nodes()
node1_zk = get_fake_zk("node1")
node1_zk.create("/test_restart_node", b"hello")
node3.restart_clickhouse(kill=True)
node3_zk = get_fake_zk("node3")
# got data from log
assert node3_zk.get("/test_restart_node")[0] == b"hello"
finally:
try:
for zk_conn in [node1_zk, node3_zk]:
try:
zk_conn.stop()
zk_conn.close()
except:
pass
except:
pass
def test_simple_replicated_table(started_cluster):
wait_nodes()
for i, node in enumerate([node1, node2, node3]):
node.query("CREATE TABLE t (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t', '{}') ORDER BY tuple()".format(i + 1))
node2.query("INSERT INTO t SELECT number FROM numbers(10)")
node1.query("SYSTEM SYNC REPLICA t", timeout=10)
node3.query("SYSTEM SYNC REPLICA t", timeout=10)
assert node1.query("SELECT COUNT() FROM t") == "10\n"
assert node2.query("SELECT COUNT() FROM t") == "10\n"
assert node3.query("SELECT COUNT() FROM t") == "10\n"

View File

@ -127,18 +127,22 @@ void testCreateListWatchEvent(zkutil::ZooKeeper & zk)
void testMultiRequest(zkutil::ZooKeeper & zk)
{
std::cerr << "Testing multi request\n";
Coordination::Requests requests;
requests.push_back(zkutil::makeCreateRequest("/data/multirequest", "aaa", zkutil::CreateMode::Persistent));
requests.push_back(zkutil::makeSetRequest("/data/multirequest", "bbb", -1));
zk.multi(requests);
std::cerr << "Multi executed\n";
try
{
requests.clear();
std::cerr << "Testing bad multi\n";
requests.push_back(zkutil::makeCreateRequest("/data/multirequest", "qweqwe", zkutil::CreateMode::Persistent));
requests.push_back(zkutil::makeSetRequest("/data/multirequest", "bbb", -1));
requests.push_back(zkutil::makeSetRequest("/data/multirequest", "ccc", -1));
zk.multi(requests);
std::cerr << "Bad multi executed\n";
std::terminate();
}
catch (...)
@ -147,6 +151,7 @@ void testMultiRequest(zkutil::ZooKeeper & zk)
}
checkEq(zk, "/data/multirequest", "bbb");
std::cerr << "Multi request finished\n";
}
std::mutex elements_mutex;