mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Add test for leader remove
This commit is contained in:
parent
647856658f
commit
042eebd981
@ -475,7 +475,7 @@ bool KeeperServer::waitConfigurationUpdate(const ConfigUpdateAction & task)
|
||||
{
|
||||
if (raft_instance->get_srv_config(task.server->get_id()) != nullptr)
|
||||
{
|
||||
LOG_INFO(log, "Server with id {} was successfully added", task.server->get_id());
|
||||
LOG_INFO(log, "Server with id {} was successfully added by leader", task.server->get_id());
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -491,13 +491,13 @@ bool KeeperServer::waitConfigurationUpdate(const ConfigUpdateAction & task)
|
||||
}
|
||||
else if (task.action_type == ConfigUpdateActionType::RemoveServer)
|
||||
{
|
||||
LOG_INFO(log, "Will try to remove server with id {}", task.server->get_id());
|
||||
LOG_INFO(log, "Will try to wait remove of server with id {}", task.server->get_id());
|
||||
|
||||
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count; ++i)
|
||||
{
|
||||
if (raft_instance->get_srv_config(task.server->get_id()) == nullptr)
|
||||
{
|
||||
LOG_INFO(log, "Server with id {} was successfully removed", task.server->get_id());
|
||||
LOG_INFO(log, "Server with id {} was successfully removed by leader", task.server->get_id());
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -32,7 +32,7 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const
|
||||
KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const
|
||||
{
|
||||
KeeperConfigurationWrapper result;
|
||||
result.cluster_config = std::make_shared<nuraft::cluster_config>();
|
||||
@ -68,7 +68,7 @@ KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const P
|
||||
total_servers++;
|
||||
}
|
||||
|
||||
if (!result.config)
|
||||
if (!result.config && !allow_without_us)
|
||||
throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section", my_server_id);
|
||||
|
||||
if (result.servers_start_as_followers.size() == total_servers)
|
||||
@ -98,7 +98,7 @@ KeeperStateManager::KeeperStateManager(
|
||||
: my_server_id(server_id_)
|
||||
, secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false))
|
||||
, config_prefix(config_prefix_)
|
||||
, configuration_wrapper(parseServersConfiguration(config))
|
||||
, configuration_wrapper(parseServersConfiguration(config, false))
|
||||
, log_store(nuraft::cs_new<KeeperLogStore>(
|
||||
getLogsPathFromConfig(config_prefix_, config, standalone_keeper),
|
||||
coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, coordination_settings->compress_logs))
|
||||
@ -138,9 +138,7 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state)
|
||||
|
||||
ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const
|
||||
{
|
||||
auto new_configuration_wrapper = parseServersConfiguration(config);
|
||||
if (new_configuration_wrapper.port != configuration_wrapper.port)
|
||||
throw Exception(ErrorCodes::RAFT_ERROR, "Cannot change port of already running RAFT server");
|
||||
auto new_configuration_wrapper = parseServersConfiguration(config, true);
|
||||
|
||||
std::unordered_map<int, KeeperServerConfigPtr> new_ids, old_ids;
|
||||
for (auto new_server : new_configuration_wrapper.cluster_config->get_servers())
|
||||
|
@ -132,7 +132,7 @@ private:
|
||||
nuraft::ptr<nuraft::srv_state> server_state;
|
||||
|
||||
/// Parse configuration from xml config.
|
||||
KeeperConfigurationWrapper parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const;
|
||||
KeeperConfigurationWrapper parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -2,13 +2,7 @@
|
||||
|
||||
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
|
||||
from helpers.test_tools import assert_eq_with_retry
|
||||
from kazoo.client import KazooClient, KazooState
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
@ -29,10 +23,6 @@ def started_cluster():
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def start(node):
|
||||
node.start_clickhouse()
|
||||
|
||||
|
||||
def get_fake_zk(node, timeout=30.0):
|
||||
_fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout)
|
||||
_fake_zk_instance.start()
|
||||
|
Loading…
Reference in New Issue
Block a user