ClickHouse/src/Coordination/NuKeeperStorageDispatcher.cpp

238 lines
7.3 KiB
C++
Raw Normal View History

2021-02-01 14:14:59 +00:00
#include <Coordination/NuKeeperStorageDispatcher.h>
2021-01-19 14:22:28 +00:00
#include <Common/setThreadName.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int TIMEOUT_EXCEEDED;
}
2021-02-01 14:14:59 +00:00
NuKeeperStorageDispatcher::NuKeeperStorageDispatcher()
2021-02-09 14:47:18 +00:00
: coordination_settings(std::make_shared<CoordinationSettings>())
, log(&Poco::Logger::get("NuKeeperDispatcher"))
{
}
void NuKeeperStorageDispatcher::requestThread()
2021-01-19 14:22:28 +00:00
{
setThreadName("NuKeeperReqT");
2021-01-26 14:08:31 +00:00
while (!shutdown_called)
2021-01-19 14:22:28 +00:00
{
2021-02-01 14:14:59 +00:00
NuKeeperStorage::RequestForSession request;
2021-01-19 14:22:28 +00:00
2021-02-09 14:47:18 +00:00
UInt64 max_wait = UInt64(coordination_settings->operation_timeout_ms.totalMilliseconds());
2021-01-19 14:22:28 +00:00
2021-01-26 07:47:04 +00:00
if (requests_queue.tryPop(request, max_wait))
{
2021-01-26 14:08:31 +00:00
if (shutdown_called)
2021-01-26 07:47:04 +00:00
break;
2021-01-26 14:08:31 +00:00
2021-01-26 07:47:04 +00:00
try
2021-01-19 14:22:28 +00:00
{
server->putRequest(request);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
}
void NuKeeperStorageDispatcher::responseThread()
{
setThreadName("NuKeeperRspT");
while (!shutdown_called)
{
NuKeeperStorage::ResponseForSession response_for_session;
2021-02-09 14:47:18 +00:00
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);
2021-01-19 14:22:28 +00:00
}
2021-01-26 07:47:04 +00:00
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
2021-01-19 14:22:28 +00:00
}
}
}
2021-02-01 14:14:59 +00:00
void NuKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response)
2021-01-19 14:22:28 +00:00
{
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;
2021-01-19 14:22:28 +00:00
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);
}
2021-02-01 14:14:59 +00:00
bool NuKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id)
2021-01-19 14:22:28 +00:00
{
{
std::lock_guard lock(session_to_response_callback_mutex);
if (session_to_response_callback.count(session_id) == 0)
2021-01-25 12:29:12 +00:00
return false;
2021-01-19 14:22:28 +00:00
}
2021-02-01 14:14:59 +00:00
NuKeeperStorage::RequestForSession request_info;
2021-01-19 14:22:28 +00:00
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));
2021-02-09 14:47:18 +00:00
else if (!requests_queue.tryPush(std::move(request_info), coordination_settings->operation_timeout_ms.totalMilliseconds()))
2021-01-19 14:22:28 +00:00
throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED);
2021-01-25 12:29:12 +00:00
return true;
2021-01-19 14:22:28 +00:00
}
2021-02-01 14:14:59 +00:00
void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config)
2021-01-19 14:22:28 +00:00
{
LOG_DEBUG(log, "Initializing storage dispatcher");
2021-02-02 08:02:25 +00:00
int myid = config.getInt("test_keeper_server.server_id");
2021-01-25 12:29:12 +00:00
2021-02-11 09:17:57 +00:00
coordination_settings->loadFromConfig("test_keeper_server.coordination_settings", config);
2021-01-25 12:29:12 +00:00
2021-02-11 09:17:57 +00:00
server = std::make_unique<NuKeeperServer>(myid, coordination_settings, config, responses_queue);
try
2021-01-25 14:10:18 +00:00
{
2021-02-11 09:17:57 +00:00
LOG_DEBUG(log, "Waiting server to initialize");
server->startup(true);
2021-02-11 09:58:02 +00:00
LOG_DEBUG(log, "Server initialized, waiting for quorum");
2021-02-11 09:49:49 +00:00
2021-02-11 09:17:57 +00:00
server->waitInit();
LOG_DEBUG(log, "Quorum initialized");
2021-01-27 17:54:25 +00:00
}
catch (...)
2021-01-27 17:54:25 +00:00
{
tryLogCurrentException(__PRETTY_FUNCTION__);
throw;
2021-01-25 14:10:18 +00:00
}
2021-01-25 12:29:12 +00:00
request_thread = ThreadFromGlobalPool([this] { requestThread(); });
responses_thread = ThreadFromGlobalPool([this] { responseThread(); });
session_cleaner_thread = ThreadFromGlobalPool([this] { sessionCleanerTask(); });
2021-01-25 12:29:12 +00:00
LOG_DEBUG(log, "Dispatcher initialized");
2021-01-19 14:22:28 +00:00
}
2021-02-01 14:14:59 +00:00
void NuKeeperStorageDispatcher::shutdown()
2021-01-19 14:22:28 +00:00
{
try
{
2021-01-26 14:08:31 +00:00
{
std::lock_guard lock(push_request_mutex);
if (shutdown_called)
return;
LOG_DEBUG(log, "Shutting down storage dispatcher");
2021-01-26 14:08:31 +00:00
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();
2021-01-26 14:08:31 +00:00
}
if (server)
server->shutdown();
2021-01-26 14:08:31 +00:00
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);
2021-01-26 14:08:31 +00:00
}
session_to_response_callback.clear();
2021-01-19 14:22:28 +00:00
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
LOG_DEBUG(log, "Dispatcher shut down");
2021-01-19 14:22:28 +00:00
}
2021-02-01 14:14:59 +00:00
NuKeeperStorageDispatcher::~NuKeeperStorageDispatcher()
2021-01-26 14:08:31 +00:00
{
shutdown();
}
2021-02-01 14:14:59 +00:00
void NuKeeperStorageDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCallback callback)
2021-01-19 14:22:28 +00:00
{
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;
2021-02-09 18:29:06 +00:00
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__);
}
2021-02-09 14:47:18 +00:00
std::this_thread::sleep_for(std::chrono::milliseconds(coordination_settings->dead_session_check_period_ms.totalMilliseconds()));
}
}
2021-02-01 14:14:59 +00:00
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);
}
2021-01-19 14:22:28 +00:00
}