mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Trying to make testkeeper better in single server mode
This commit is contained in:
parent
36c178c023
commit
84102a22e7
@ -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
|
||||
|
@ -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()
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user