Trying to make testkeeper better in single server mode

This commit is contained in:
alesapin 2021-02-25 23:30:55 +03:00
parent 36c178c023
commit 84102a22e7
4 changed files with 50 additions and 5 deletions

View File

@ -81,6 +81,8 @@ clickhouse-client --query "SHOW TABLES FROM test"
./stress --hung-check --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" && echo "OK" > /test_output/script_exit_code.txt || echo "FAIL" > /test_output/script_exit_code.txt
stop
# TODO remove me when persistent snapshots will be ready
rm -fr /var/lib/clickhouse/coordination :||
start
clickhouse-client --query "SELECT 'Server successfuly started'" > /test_output/alive_check.txt || echo 'Server failed to start' > /test_output/alive_check.txt

View File

@ -35,11 +35,23 @@ void NuKeeperServer::startup()
{
state_manager->loadLogStore(state_machine->last_commit_index());
bool single_server = state_manager->getTotalServers() == 1;
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();
if (single_server)
{
/// Don't make sence in single server mode
params.heart_beat_interval_ = 0;
params.election_timeout_lower_bound_ = 0;
params.election_timeout_upper_bound_ = 0;
}
else
{
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();
@ -161,13 +173,38 @@ bool NuKeeperServer::isLeaderAlive() const
nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * /* param */)
{
if ((type == nuraft::cb_func::InitialBatchCommited && isLeader()) || type == nuraft::cb_func::BecomeFresh)
/// Only initial record
bool empty_store = state_manager->getLogStore()->size() == 1;
auto set_initialized = [this] ()
{
std::unique_lock lock(initialized_mutex);
initialized_flag = true;
initialized_cv.notify_all();
};
switch (type)
{
case nuraft::cb_func::BecomeLeader:
{
if (empty_store) /// We become leader and store is empty, ready to serve requests
set_initialized();
return nuraft::cb_func::ReturnCode::Ok;
}
case nuraft::cb_func::BecomeFresh:
{
set_initialized(); /// We are fresh follower, ready to serve requests.
return nuraft::cb_func::ReturnCode::Ok;
}
case nuraft::cb_func::InitialBatchCommited:
{
if (isLeader()) /// We have commited our log store and we are leader, ready to serve requests.
set_initialized();
return nuraft::cb_func::ReturnCode::Ok;
}
default: /// ignore other events
return nuraft::cb_func::ReturnCode::Ok;
}
return nuraft::cb_func::ReturnCode::Ok;
}
void NuKeeperServer::waitInit()

View File

@ -33,6 +33,7 @@ NuKeeperStateManager::NuKeeperStateManager(
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix + ".raft_configuration", keys);
total_servers = keys.size();
for (const auto & server_key : keys)
{

View File

@ -52,9 +52,14 @@ public:
return start_as_follower_servers.count(my_server_id);
}
nuraft::ptr<NuKeeperLogStore> getLogStore() const { return log_store; }
size_t getTotalServers() const { return total_servers; }
private:
int my_server_id;
int my_port;
size_t total_servers{0};
std::unordered_set<int> start_as_follower_servers;
nuraft::ptr<NuKeeperLogStore> log_store;
nuraft::ptr<nuraft::srv_config> my_server_config;