Rename file

This commit is contained in:
alesapin 2021-02-19 19:05:26 +03:00
parent 12d05c2792
commit ad374ec095
5 changed files with 17 additions and 17 deletions

View File

@ -1,7 +1,7 @@
#include <Coordination/NuKeeperServer.h>
#include <Coordination/LoggerWrapper.h>
#include <Coordination/NuKeeperStateMachine.h>
#include <Coordination/InMemoryStateManager.h>
#include <Coordination/NuKeeperStateManager.h>
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <Coordination/ReadBufferFromNuraftBuffer.h>
#include <IO/ReadHelpers.h>
@ -26,7 +26,7 @@ NuKeeperServer::NuKeeperServer(
: 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", config, coordination_settings))
, state_manager(nuraft::cs_new<NuKeeperStateManager>(server_id, "test_keeper_server", config, coordination_settings))
, responses_queue(responses_queue_)
{
}

View File

@ -2,7 +2,7 @@
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <Coordination/InMemoryLogStore.h>
#include <Coordination/InMemoryStateManager.h>
#include <Coordination/NuKeeperStateManager.h>
#include <Coordination/NuKeeperStateMachine.h>
#include <Coordination/NuKeeperStorage.h>
#include <Coordination/CoordinationSettings.h>
@ -20,7 +20,7 @@ private:
nuraft::ptr<NuKeeperStateMachine> state_machine;
nuraft::ptr<InMemoryStateManager> state_manager;
nuraft::ptr<NuKeeperStateManager> state_manager;
nuraft::raft_launcher launcher;

View File

@ -1,4 +1,4 @@
#include <Coordination/InMemoryStateManager.h>
#include <Coordination/NuKeeperStateManager.h>
#include <Common/Exception.h>
namespace DB
@ -9,7 +9,7 @@ namespace ErrorCodes
extern const int RAFT_ERROR;
}
InMemoryStateManager::InMemoryStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path)
NuKeeperStateManager::NuKeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path)
: my_server_id(server_id_)
, my_port(port)
, log_store(nuraft::cs_new<NuKeeperLogStore>(logs_path, 5000, true))
@ -19,7 +19,7 @@ InMemoryStateManager::InMemoryStateManager(int server_id_, const std::string & h
cluster_config->get_servers().push_back(peer_config);
}
InMemoryStateManager::InMemoryStateManager(
NuKeeperStateManager::NuKeeperStateManager(
int my_server_id_,
const std::string & config_prefix,
const Poco::Util::AbstractConfiguration & config,
@ -63,17 +63,17 @@ InMemoryStateManager::InMemoryStateManager(
throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without <start_as_follower>)");
}
void InMemoryStateManager::loadLogStore(size_t start_log_index)
void NuKeeperStateManager::loadLogStore(size_t start_log_index)
{
log_store->init(start_log_index);
}
void InMemoryStateManager::flushLogStore()
void NuKeeperStateManager::flushLogStore()
{
log_store->flush();
}
void InMemoryStateManager::save_config(const nuraft::cluster_config & config)
void NuKeeperStateManager::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.
@ -81,7 +81,7 @@ void InMemoryStateManager::save_config(const nuraft::cluster_config & config)
cluster_config = nuraft::cluster_config::deserialize(*buf);
}
void InMemoryStateManager::save_state(const nuraft::srv_state & state)
void NuKeeperStateManager::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.

View File

@ -10,16 +10,16 @@
namespace DB
{
class InMemoryStateManager : public nuraft::state_mgr
class NuKeeperStateManager : public nuraft::state_mgr
{
public:
InMemoryStateManager(
NuKeeperStateManager(
int server_id_,
const std::string & config_prefix,
const Poco::Util::AbstractConfiguration & config,
const CoordinationSettingsPtr & coordination_settings);
InMemoryStateManager(
NuKeeperStateManager(
int server_id_,
const std::string & host,
int port,

View File

@ -9,7 +9,7 @@
#include <Poco/ConsoleChannel.h>
#include <Poco/Logger.h>
#include <Coordination/InMemoryLogStore.h>
#include <Coordination/InMemoryStateManager.h>
#include <Coordination/NuKeeperStateManager.h>
#include <Coordination/NuKeeperStorageSerializer.h>
#include <Coordination/SummingStateMachine.h>
#include <Coordination/NuKeeperStateMachine.h>
@ -100,7 +100,7 @@ struct SimpliestRaftServer
, 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, logs_path))
, state_manager(nuraft::cs_new<DB::NuKeeperStateManager>(server_id, hostname, port, logs_path))
{
state_manager->loadLogStore(1);
nuraft::raft_params params;
@ -151,7 +151,7 @@ struct SimpliestRaftServer
nuraft::ptr<StateMachine> state_machine;
// State manager.
nuraft::ptr<DB::InMemoryStateManager> state_manager;
nuraft::ptr<DB::NuKeeperStateManager> state_manager;
// Raft launcher.
nuraft::raft_launcher launcher;