From a60663e33d3f2e28e5c509aadb30b45c1133ff75 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 27 Oct 2021 20:26:42 +0800 Subject: [PATCH 01/49] add 4lw commands to keeper --- programs/keeper/Keeper.cpp | 3 + programs/server/Server.cpp | 4 +- src/Common/getCurrentProcessFDCount.cpp | 33 ++ src/Common/getCurrentProcessFDCount.h | 5 + src/Common/getMaxFileDescriptorCount.cpp | 27 ++ src/Common/getMaxFileDescriptorCount.h | 6 + src/Coordination/CoordinationSettings.cpp | 156 ++++++++++ src/Coordination/CoordinationSettings.h | 29 ++ src/Coordination/FourLetterCommand.cpp | 221 ++++++++++++++ src/Coordination/FourLetterCommand.h | 149 +++++++++ src/Coordination/KeeperDispatcher.cpp | 141 ++++++++- src/Coordination/KeeperDispatcher.h | 115 +++++-- src/Coordination/KeeperInfos.h | 72 +++++ src/Coordination/KeeperServer.cpp | 71 +++-- src/Coordination/KeeperServer.h | 30 +- src/Coordination/KeeperStateMachine.cpp | 34 ++- src/Coordination/KeeperStateMachine.h | 13 +- src/Coordination/KeeperStateManager.cpp | 41 +-- src/Coordination/KeeperStateManager.h | 4 +- src/Coordination/KeeperStorage.h | 46 +++ src/Server/KeeperTCPHandler.cpp | 84 +++++- src/Server/KeeperTCPHandler.h | 5 +- .../test_keeper_four_word_command/__init__.py | 1 + .../configs/enable_keeper1.xml | 41 +++ .../configs/enable_keeper2.xml | 41 +++ .../configs/enable_keeper3.xml | 41 +++ .../configs/keeper_config.xml | 24 ++ .../configs/use_keeper.xml | 16 + .../test_keeper_four_word_command/test.py | 285 ++++++++++++++++++ 29 files changed, 1628 insertions(+), 110 deletions(-) create mode 100644 src/Common/getCurrentProcessFDCount.cpp create mode 100644 src/Common/getCurrentProcessFDCount.h create mode 100644 src/Common/getMaxFileDescriptorCount.cpp create mode 100644 src/Common/getMaxFileDescriptorCount.h create mode 100644 src/Coordination/FourLetterCommand.cpp create mode 100644 src/Coordination/FourLetterCommand.h create mode 100644 src/Coordination/KeeperInfos.h create mode 100644 tests/integration/test_keeper_four_word_command/__init__.py create mode 100644 tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml create mode 100644 tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml create mode 100644 tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml create mode 100644 tests/integration/test_keeper_four_word_command/configs/keeper_config.xml create mode 100644 tests/integration/test_keeper_four_word_command/configs/use_keeper.xml create mode 100644 tests/integration/test_keeper_four_word_command/test.py diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 6e001c57e75..b06d370a2a8 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -363,6 +364,8 @@ int Keeper::main(const std::vector & /*args*/) /// Initialize keeper RAFT. Do nothing if no keeper_server in config. global_context->initializeKeeperDispatcher(/* start_async = */false); + FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher()); + for (const auto & listen_host : listen_hosts) { /// TCP Keeper diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ce7d6973f68..e988f1f8a0d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -111,7 +111,8 @@ #endif #if USE_NURAFT -# include +# include +# include #endif #if USE_BASE64 @@ -1017,6 +1018,7 @@ if (ThreadFuzzer::instance().isEffective()) } /// Initialize keeper RAFT. global_context->initializeKeeperDispatcher(can_initialize_keeper_async); + FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher()); for (const auto & listen_host : listen_hosts) { diff --git a/src/Common/getCurrentProcessFDCount.cpp b/src/Common/getCurrentProcessFDCount.cpp new file mode 100644 index 00000000000..9653b4eccd8 --- /dev/null +++ b/src/Common/getCurrentProcessFDCount.cpp @@ -0,0 +1,33 @@ +#include +#include +#include +#include +#include +#include +#include + + +int getCurrentProcessFDCount() +{ +#if defined(__linux__) || defined(__APPLE__) + using namespace DB; + + char buf[64]; + snprintf(buf, 64, "lsof -p %i | wc -l", getpid()); + + auto command = ShellCommand::execute(buf); + + WriteBufferFromOwnString out; + copyData(command->out, out); + + if(!out.str().empty()) + { + return std::stoi(out.str()); + } + + return -1; +#else + return -1; +#endif + +} diff --git a/src/Common/getCurrentProcessFDCount.h b/src/Common/getCurrentProcessFDCount.h new file mode 100644 index 00000000000..05135d3778f --- /dev/null +++ b/src/Common/getCurrentProcessFDCount.h @@ -0,0 +1,5 @@ +#pragma once + +/// Get current process file descriptor count +/// @return -1 if error occurs +int getCurrentProcessFDCount(); diff --git a/src/Common/getMaxFileDescriptorCount.cpp b/src/Common/getMaxFileDescriptorCount.cpp new file mode 100644 index 00000000000..751145e4967 --- /dev/null +++ b/src/Common/getMaxFileDescriptorCount.cpp @@ -0,0 +1,27 @@ +#include +#include +#include +#include +#include + +int getMaxFileDescriptorCount() +{ +#if defined(__linux__) || defined(__APPLE__) + using namespace DB; + + auto command = ShellCommand::execute("ulimit -n"); + + WriteBufferFromOwnString out; + copyData(command->out, out); + + if(!out.str().empty()) + { + return std::stoi(out.str()); + } + + return -1; +#else + return -1; +#endif + +} diff --git a/src/Common/getMaxFileDescriptorCount.h b/src/Common/getMaxFileDescriptorCount.h new file mode 100644 index 00000000000..779d23417ff --- /dev/null +++ b/src/Common/getMaxFileDescriptorCount.h @@ -0,0 +1,6 @@ +#pragma once + +/// Get process max file descriptor count +/// @return -1 if error occurs +int getMaxFileDescriptorCount(); + diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 6201834d17d..46e60689103 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -32,4 +33,159 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco } } +void KeeperSettings::dump(WriteBufferFromOwnString & buf) const +{ + auto write = [&buf](const String & content) { buf.write(content.data(), content.size()); }; + + auto write_int = [&buf](Int64 value) { + String str_val = std::to_string(value); + buf.write(str_val.data(), str_val.size()); + buf.write('\n'); + }; + + auto write_bool = [&buf](bool value) { + String str_val = value ? "true" : "false"; + buf.write(str_val.data(), str_val.size()); + buf.write('\n'); + }; + + write("server_id="); + write_int(server_id); + + if(tcp_port != NO_PORT) + { + write("tcp_port="); + write_int(tcp_port); + } + if(tcp_port_secure != NO_PORT) + { + write("tcp_port_secure="); + write_int(tcp_port_secure); + } + if(!super_digest.empty()) + { + write("superdigest="); + write(super_digest); + buf.write('\n'); + } + + write("log_storage_path="); + write(log_storage_path); + buf.write('\n'); + + write("snapshot_storage_path="); + write(snapshot_storage_path); + buf.write('\n'); + + /// coordination_settings + + write("max_requests_batch_size="); + write_int(coordination_settings->max_requests_batch_size); + write("session_timeout_ms="); + write_int(UInt64(coordination_settings->session_timeout_ms)); + write("operation_timeout_ms="); + write_int(UInt64(coordination_settings->operation_timeout_ms)); + write("dead_session_check_period_ms="); + write_int(UInt64(coordination_settings->dead_session_check_period_ms)); + + write("heart_beat_interval_ms="); + write_int(UInt64(coordination_settings->heart_beat_interval_ms)); + write("election_timeout_lower_bound_ms="); + write_int(UInt64(coordination_settings->election_timeout_lower_bound_ms)); + write("election_timeout_upper_bound_ms="); + write_int(UInt64(coordination_settings->election_timeout_upper_bound_ms)); + + write("reserved_log_items="); + write_int(coordination_settings->reserved_log_items); + write("snapshot_distance="); + write_int(coordination_settings->snapshot_distance); + + write("auto_forwarding="); + write_bool(coordination_settings->auto_forwarding); + write("shutdown_timeout="); + write_int(UInt64(coordination_settings->shutdown_timeout)); + write("startup_timeout="); + write_int(UInt64(coordination_settings->startup_timeout)); + + write("raft_logs_level="); + write(coordination_settings->raft_logs_level.toString()); + buf.write('\n'); + + write("snapshots_to_keep="); + write_int(coordination_settings->snapshots_to_keep); + write("rotate_log_storage_interval="); + write_int(coordination_settings->rotate_log_storage_interval); + write("stale_log_gap="); + write_int(coordination_settings->stale_log_gap); + write("fresh_log_gap="); + write_int(coordination_settings->fresh_log_gap); + + write("max_requests_batch_size="); + write_int(coordination_settings->max_requests_batch_size); + write("quorum_reads="); + write_bool(coordination_settings->quorum_reads); + write("force_sync="); + write_bool(coordination_settings->force_sync); +} + +std::shared_ptr +KeeperSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) +{ + std::shared_ptr ret = std::make_shared(); + + ret->server_id = config.getInt("keeper_server.server_id"); + ret->standalone_keeper = standalone_keeper_; + + if(config.has("keeper_server.tcp_port")) + { + ret->tcp_port = config.getInt("keeper_server.tcp_port"); + } + if(config.has("keeper_server.tcp_port_secure")) + { + ret->tcp_port_secure = config.getInt("keeper_server.tcp_port_secure"); + } + if(config.has("keeper_server.superdigest")) + { + ret->super_digest = config.getString("keeper_server.tcp_port_secure"); + } + + ret->log_storage_path = getLogsPathFromConfig(config, standalone_keeper_); + ret->snapshot_storage_path = getSnapshotsPathFromConfig(config, standalone_keeper_); + + ret->coordination_settings = std::make_shared(); + ret->coordination_settings->loadFromConfig("keeper_server.coordination_settings", config); + + return ret; +} + +String KeeperSettings::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) +{ + /// the most specialized path + if (config.has("keeper_server.log_storage_path")) + return config.getString("keeper_server.log_storage_path"); + + if (config.has("keeper_server.storage_path")) + return std::filesystem::path{config.getString("keeper_server.storage_path")} / "logs"; + + if (standalone_keeper_) + return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs"; + else + return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs"; +} + +String KeeperSettings::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) +{ + /// the most specialized path + if (config.has("keeper_server.snapshot_storage_path")) + return config.getString("keeper_server.snapshot_storage_path"); + + if (config.has("keeper_server.storage_path")) + return std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots"; + + if (standalone_keeper_) + return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots"; + else + return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots"; +} + } diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index bec5e7f1f7a..9e3a281b0e1 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -51,4 +52,32 @@ struct CoordinationSettings : public BaseSettings using CoordinationSettingsPtr = std::shared_ptr; +struct KeeperSettings +{ + static constexpr int NO_PORT = -1; + + KeeperSettings() = default; + int server_id; + + int tcp_port{NO_PORT}; + int tcp_port_secure{NO_PORT}; + + String super_digest; + + bool standalone_keeper; + CoordinationSettingsPtr coordination_settings; + + String log_storage_path; + String snapshot_storage_path; + + void dump(WriteBufferFromOwnString & buf) const; + static std::shared_ptr loadFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); + +private: + static String getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); + static String getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); +}; + +using KeeperSettingsPtr = std::shared_ptr; + } diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp new file mode 100644 index 00000000000..410f774e206 --- /dev/null +++ b/src/Coordination/FourLetterCommand.cpp @@ -0,0 +1,221 @@ +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +IFourLetterCommand::IFourLetterCommand(const KeeperDispatcher & keeper_dispatcher_) : keeper_dispatcher(keeper_dispatcher_) +{ +} + +Int32 IFourLetterCommand::code() +{ + Int32 res = *reinterpret_cast(name().data()); + /// keep consistent with Coordination::read method by changing big endian to little endian. + return __builtin_bswap32(res); +} + +void IFourLetterCommand::printSet(IFourLetterCommand::StringBuffer & buffer, std::unordered_set & set, String && prefix) +{ + for (const auto & str : set) + { + buffer.write(prefix.data(), prefix.size()); + buffer.write(str.data(), str.size()); + buffer.write('\n'); + } +} +IFourLetterCommand::~IFourLetterCommand() = default; + +FourLetterCommandFactory & FourLetterCommandFactory::instance() +{ + static FourLetterCommandFactory factory; + return factory; +} + +bool FourLetterCommandFactory::isKnown(Int32 code) +{ + if (!initialized) + { + throw Exception("Four letter command " + std::to_string(code) + " not initialized", ErrorCodes::LOGICAL_ERROR); + } + return commands.contains(code); +} + +FourLetterCommandPtr FourLetterCommandFactory::get(Int32 code) +{ + if (!initialized) + { + throw Exception("Four letter command " + std::to_string(code) + " not initialized", ErrorCodes::LOGICAL_ERROR); + } + return commands.at(code); +} + +void FourLetterCommandFactory::registerCommand(FourLetterCommandPtr & command) +{ + if (commands.contains(command->code())) + { + throw Exception("Four letter command " + std::to_string(command->code()) + " already registered", ErrorCodes::LOGICAL_ERROR); + } + auto * log = &Poco::Logger::get("FourLetterCommandFactory"); + LOG_INFO(log, "Register four letter command {}, code {}", command->name(), std::to_string(command->code())); + commands.emplace(command->code(), std::move(command)); +} + +void FourLetterCommandFactory::registerCommands(const KeeperDispatcher & keeper_dispatcher) +{ + FourLetterCommandFactory & factory = FourLetterCommandFactory::instance(); + + if (!factory.isInitialized()) + { + FourLetterCommandPtr ruok_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(ruok_command); + + FourLetterCommandPtr mntr_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(mntr_command); + + FourLetterCommandPtr srst_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(srst_command); + + FourLetterCommandPtr conf_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(conf_command); + + factory.setInitialize(true); + } +} + +String RuokCommand::name() +{ + return "ruok"; +} + +String RuokCommand::run() +{ + return "imok"; +} + +RuokCommand::~RuokCommand() = default; + +String MonitorCommand::name() +{ + return "mntr"; +} + +String MonitorCommand::run() +{ + KeeperStatsPtr stats = keeper_dispatcher.getKeeperStats(); + const IKeeperInfo & keeper_info = keeper_dispatcher.getKeeperInfo(); + + if (!keeper_info.hasLeader()) + { + return "This instance is not currently serving requests"; + } + + const IRaftInfo & raft_info = keeper_dispatcher.getRaftInfo(); + const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); + + StringBuffer ret; + print(ret, "version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); + + print(ret, "avg_latency", stats->getAvgLatency()); + print(ret, "max_latency", stats->getMaxLatency()); + print(ret, "min_latency", stats->getMinLatency()); + print(ret, "packets_received", stats->getPacketsReceived()); + print(ret, "packets_sent", stats->getPacketsSent()); + + print(ret, "num_alive_connections", keeper_info.getNumAliveConnections()); + print(ret, "outstanding_requests", keeper_info.getOutstandingRequests()); + print(ret, "server_state", keeper_info.getRole()); + + print(ret, "znode_count", state_machine.getNodeCount()); + print(ret, "watch_count", state_machine.getWatchCount()); + print(ret, "ephemerals_count", state_machine.getEphemeralCount()); + print(ret, "approximate_data_size", state_machine.getApproximateDataSize()); + +#if defined(__linux__) || defined(__APPLE__) + print(ret, "open_file_descriptor_count", getCurrentProcessFDCount()); + print(ret, "max_file_descriptor_count", getMaxFileDescriptorCount()); +#endif + + if (raft_info.isLeader()) + { + print(ret, "followers", raft_info.getFollowerCount()); + print(ret, "synced_followers", raft_info.getSyncedFollowerCount()); + /// TODO implementation + /// print(ret, "pending_syncs", 0); + } + + /// TODO Maybe the next 3 metrics are useless. + /// print(ret, "last_proposal_size", -1); + /// print(ret, "max_proposal_size", -1); + /// print(ret, "min_proposal_size", -1); + + return ret.str(); +} + +void MonitorCommand::print(IFourLetterCommand::StringBuffer & buf, const String & key, const String & value) +{ + const static String prefix = "zk_"; + const int prefix_len = prefix.size(); + + buf.write(prefix.data(), prefix_len); + buf.write(key.data(), key.size()); + + buf.write('\t'); + buf.write(value.data(), value.size()); + buf.write('\n'); +} + +void MonitorCommand::print(IFourLetterCommand::StringBuffer & buf, const String & key, UInt64 value) +{ + print(buf, key, std::to_string(value)); +} + +MonitorCommand::~MonitorCommand() = default; + +String StatResetCommand::name() +{ + return "srst"; +} + +String StatResetCommand::run() +{ + KeeperStatsPtr stats = keeper_dispatcher.getKeeperStats(); + stats->reset(); + return "Server stats reset."; +} + +StatResetCommand::~StatResetCommand() = default; + +String NopCommand::name() +{ + return "nopc"; +} + +String NopCommand::run() +{ + return DB::String(); +} + +NopCommand::~NopCommand() = default; + +String ConfCommand::name() +{ + return "conf"; +} + +String ConfCommand::run() +{ + StringBuffer buf; + keeper_dispatcher.dumpConf(buf); + return buf.str(); +} + +ConfCommand::~ConfCommand() = default; + +} diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h new file mode 100644 index 00000000000..3867a5ede86 --- /dev/null +++ b/src/Coordination/FourLetterCommand.h @@ -0,0 +1,149 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +#if !defined(ARCADIA_BUILD) +# include +#endif + +namespace DB +{ +struct IFourLetterCommand; +using FourLetterCommandPtr = std::shared_ptr; + +/// Just like zookeeper Four Letter Words commands, CH Keeper responds to a small set of commands. +/// Each command is composed of four letters, these commands are useful to monitor and issue system problems. +/// The feature is based on Zookeeper 3.5.9, details is in https://zookeeper.apache.org/doc/r3.5.9/zookeeperAdmin.html#sc_zkCommands. +struct IFourLetterCommand +{ +public: + using StringBuffer = DB::WriteBufferFromOwnString; + explicit IFourLetterCommand(const KeeperDispatcher & keeper_dispatcher_); + + virtual String name() = 0; + virtual String run() = 0; + + virtual ~IFourLetterCommand(); + Int32 code(); + + static void printSet(StringBuffer & buffer, std::unordered_set & set, String && prefix); + +protected: + const KeeperDispatcher & keeper_dispatcher; +}; + +struct FourLetterCommandFactory : private boost::noncopyable +{ +public: + using Commands = std::unordered_map; + + bool isKnown(Int32 code); + FourLetterCommandPtr get(Int32 code); + + /// There is no need to make it thread safe, because registration is no initialization and get is after startup. + void registerCommand(FourLetterCommandPtr & command); + + bool isInitialized() const { return initialized; } + + void setInitialize(bool flag) { initialized = flag; } + + static FourLetterCommandFactory & instance(); + static void registerCommands(const KeeperDispatcher & keeper_dispatcher); + +private: + volatile bool initialized = false; + Commands commands; +}; + +/**Tests if server is running in a non-error state. The server will respond with imok if it is running. + * Otherwise it will not respond at all. + * + * A response of "imok" does not necessarily indicate that the server has joined the quorum, + * just that the server process is active and bound to the specified client port. + * Use "stat" for details on state wrt quorum and client connection information. + */ +struct RuokCommand : public IFourLetterCommand +{ + explicit RuokCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override; + String run() override; + ~RuokCommand() override; +}; + +/**Outputs a list of variables that could be used for monitoring the health of the cluster. + * + * echo mntr | nc localhost 2181 + * zk_version 3.5.9 + * zk_avg_latency 0 + * zk_max_latency 0 + * zk_min_latency 0 + * zk_packets_received 70 + * zk_packets_sent 69 + * zk_outstanding_requests 0 + * zk_server_state leader + * zk_znode_count 4 + * zk_watch_count 0 + * zk_ephemerals_count 0 + * zk_approximate_data_size 27 + * zk_open_file_descriptor_count 23 - only available on Unix platforms + * zk_max_file_descriptor_count 1024 - only available on Unix platforms + * zk_followers 2 - only exposed by the Leader + * zk_synced_followers 2 - only exposed by the Leader + * zk_pending_syncs 0 - only exposed by the Leader + * zk_last_proposal_size -1 + * zk_max_proposal_size -1 + * zk_min_proposal_size -1 + */ +struct MonitorCommand : public IFourLetterCommand +{ + explicit MonitorCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override; + String run() override; + ~MonitorCommand() override; +private: + static void print(StringBuffer & buf, const String & key, const String & value); + static void print(StringBuffer & buf, const String & key, UInt64 value); +}; + +struct StatResetCommand : public IFourLetterCommand +{ + explicit StatResetCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override; + String run() override; + ~StatResetCommand() override; +}; + +/** A command that does not do anything except reply to client with predefined message. + * It is used to inform clients who execute none white listed four letter word commands. + */ +struct NopCommand : public IFourLetterCommand +{ + explicit NopCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override; + String run() override; + ~NopCommand() override; +}; + +struct ConfCommand : public IFourLetterCommand +{ + explicit ConfCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override; + String run() override; + ~ConfCommand() override; +}; + + + +} diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index c29ea4fac45..3977c36a943 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -12,11 +12,98 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int TIMEOUT_EXCEEDED; extern const int SYSTEM_ERROR; + extern const int UNKNOWN_SETTING; +} + +UInt64 KeeperDispatcher::KeeperStats::getMinLatency() const +{ + std::shared_lock lock(mutex); + return min_latency; +} + +UInt64 KeeperDispatcher::KeeperStats::getMaxLatency() const +{ + std::shared_lock lock(mutex); + return max_latency; +} + +UInt64 KeeperDispatcher::KeeperStats::getAvgLatency() const +{ + std::shared_lock lock(mutex); + if (count != 0) + { + return total_latency / count; + } + return 0; +} + +UInt64 KeeperDispatcher::KeeperStats::getPacketsReceived() const +{ + std::shared_lock lock(mutex); + return packets_received; +} + +UInt64 KeeperDispatcher::KeeperStats::getPacketsSent() const +{ + std::shared_lock lock(mutex); + return packets_sent; +} + +void KeeperDispatcher::KeeperStats::incrementPacketsReceived() +{ + std::unique_lock lock(mutex); + packets_received++; +} + +void KeeperDispatcher::KeeperStats::incrementPacketsSent() +{ + std::unique_lock lock(mutex); + packets_sent++; +} + +void KeeperDispatcher::KeeperStats::updateLatency(UInt64 latency_ms) +{ + std::unique_lock lock(mutex); + + total_latency += (latency_ms); + count++; + + if (latency_ms < min_latency) + { + min_latency = latency_ms; + } + + if (latency_ms > max_latency) + { + max_latency = latency_ms; + } +} + +void KeeperDispatcher::KeeperStats::reset() +{ + std::unique_lock lock(mutex); + resetLatency(); + resetRequestCounters(); +} + +void KeeperDispatcher::KeeperStats::resetLatency() +{ + total_latency = 0; + count = 0; + max_latency = 0; + min_latency = 0; +} + +void KeeperDispatcher::KeeperStats::resetRequestCounters() +{ + packets_received = 0; + packets_sent = 0; } KeeperDispatcher::KeeperDispatcher() - : coordination_settings(std::make_shared()) - , responses_queue(std::numeric_limits::max()) + : responses_queue(std::numeric_limits::max()) + , keeper_stats(std::make_shared()) + , settings(std::make_shared()) , log(&Poco::Logger::get("KeeperDispatcher")) { } @@ -36,8 +123,8 @@ void KeeperDispatcher::requestThread() { KeeperStorage::RequestForSession request; - UInt64 max_wait = UInt64(coordination_settings->operation_timeout_ms.totalMilliseconds()); - uint64_t max_batch_size = coordination_settings->max_requests_batch_size; + UInt64 max_wait = UInt64(settings->coordination_settings->operation_timeout_ms.totalMilliseconds()); + uint64_t max_batch_size = settings->coordination_settings->max_requests_batch_size; /// The code below do a very simple thing: batch all write (quorum) requests into vector until /// previous write batch is not finished or max_batch size achieved. The main complexity goes from @@ -58,7 +145,7 @@ void KeeperDispatcher::requestThread() /// If new request is not read request or we must to process it through quorum. /// Otherwise we will process it locally. - if (coordination_settings->quorum_reads || !request.request->isReadRequest()) + if (settings->coordination_settings->quorum_reads || !request.request->isReadRequest()) { current_batch.emplace_back(request); @@ -71,7 +158,7 @@ void KeeperDispatcher::requestThread() if (requests_queue->tryPop(request, 1)) { /// Don't append read request into batch, we have to process them separately - if (!coordination_settings->quorum_reads && request.request->isReadRequest()) + if (!settings->coordination_settings->quorum_reads && request.request->isReadRequest()) { has_read_request = true; break; @@ -141,7 +228,7 @@ void KeeperDispatcher::responseThread() { KeeperStorage::ResponseForSession response_for_session; - UInt64 max_wait = UInt64(coordination_settings->operation_timeout_ms.totalMilliseconds()); + UInt64 max_wait = UInt64(settings->coordination_settings->operation_timeout_ms.totalMilliseconds()); if (responses_queue.tryPop(response_for_session, max_wait)) { @@ -216,6 +303,7 @@ void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKe session_to_response_callback.erase(session_response_callback); } } + keeper_stats->incrementPacketsSent(); } bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) @@ -246,24 +334,22 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ { throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); } - + keeper_stats->incrementPacketsReceived(); return true; } void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper, bool start_async) { LOG_DEBUG(log, "Initializing storage dispatcher"); - int myid = config.getInt("keeper_server.server_id"); - coordination_settings->loadFromConfig("keeper_server.coordination_settings", config); - requests_queue = std::make_unique(coordination_settings->max_requests_batch_size); + settings = KeeperSettings::loadFromConfig(config_, standalone_keeper); + requests_queue = std::make_unique(settings->coordination_settings->max_requests_batch_size); request_thread = ThreadFromGlobalPool([this] { requestThread(); }); responses_thread = ThreadFromGlobalPool([this] { responseThread(); }); snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); }); - server = std::make_unique( - myid, coordination_settings, config, responses_queue, snapshots_queue, standalone_keeper); + server = std::make_unique(settings, config_, responses_queue, snapshots_queue); try { @@ -410,7 +496,7 @@ void KeeperDispatcher::sessionCleanerTask() tryLogCurrentException(__PRETTY_FUNCTION__); } - std::this_thread::sleep_for(std::chrono::milliseconds(coordination_settings->dead_session_check_period_ms.totalMilliseconds())); + std::this_thread::sleep_for(std::chrono::milliseconds(settings->coordination_settings->dead_session_check_period_ms.totalMilliseconds())); } } @@ -577,4 +663,31 @@ void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfigurati } } +void KeeperDispatcher::updateKeeperStat(UInt64 process_time_stopwatch) +{ + keeper_stats->updateLatency(process_time_stopwatch); +} + +String KeeperDispatcher::getRole() const +{ + return server->getRole(); +} + +UInt64 KeeperDispatcher::getOutstandingRequests() const +{ + std::lock_guard lock(push_request_mutex); + return requests_queue->size(); +} + +UInt64 KeeperDispatcher::getNumAliveConnections() const +{ + std::lock_guard lock(session_to_response_callback_mutex); + return session_to_response_callback.size(); +} + +void KeeperDispatcher::dumpConf(WriteBufferFromOwnString & buf) const +{ + settings->dump(buf); +} + } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 97152ae80b8..6ea35f420ea 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -15,22 +15,62 @@ #include #include #include - +#include namespace DB { - using ZooKeeperResponseCallback = std::function; /// Highlevel wrapper for ClickHouse Keeper. /// Process user requests via consensus and return responses. -class KeeperDispatcher +class KeeperDispatcher : public IKeeperInfo { -private: - std::mutex push_request_mutex; +public: + + /// Request statistics + class KeeperStats + { + public: + explicit KeeperStats() = default; + + UInt64 getMinLatency() const; + UInt64 getMaxLatency() const; + + UInt64 getAvgLatency() const; + + UInt64 getPacketsReceived() const; + UInt64 getPacketsSent() const; + + void incrementPacketsReceived(); + void incrementPacketsSent(); + + void updateLatency(UInt64 latency_ms); + void reset(); + + private: + void inline resetLatency(); + void inline resetRequestCounters(); + + mutable std::shared_mutex mutex; + + /// all response with watch response excluded + UInt64 packets_sent = 0; + /// All client request include ordinary requests, heart beat and session establish etc. + UInt64 packets_received = 0; + + /// For consistent with zookeeper measured by millisecond, + /// otherwise maybe microsecond is better + UInt64 total_latency = 0; + UInt64 max_latency = 0; + UInt64 min_latency = 0; + + UInt64 count = 0; + }; + +private: + mutable std::mutex push_request_mutex; - CoordinationSettingsPtr coordination_settings; using RequestsQueue = ConcurrentBoundedQueue; using SessionToResponseCallback = std::unordered_map; using UpdateConfigurationQueue = ConcurrentBoundedQueue; @@ -45,7 +85,7 @@ private: std::atomic shutdown_called{false}; - std::mutex session_to_response_callback_mutex; + mutable std::mutex session_to_response_callback_mutex; /// These two maps looks similar, but serves different purposes. /// The first map is subscription map for normal responses like /// (get, set, list, etc.). Dispatcher determines callback for each response @@ -72,6 +112,10 @@ private: /// RAFT wrapper. std::unique_ptr server; + std::shared_ptr keeper_stats; + + KeeperSettingsPtr settings; + Poco::Logger * log; /// Counter for new session_id requests. @@ -104,7 +148,7 @@ public: KeeperDispatcher(); /// Call shutdown - ~KeeperDispatcher(); + ~KeeperDispatcher() override; /// Initialization from config. /// standalone_keeper -- we are standalone keeper application (not inside clickhouse server) @@ -125,17 +169,6 @@ public: /// Put request to ClickHouse Keeper bool putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); - /// Are we leader - bool isLeader() const - { - return server->isLeader(); - } - - bool hasLeader() const - { - return server->isLeaderAlive(); - } - /// Get new session ID int64_t getSessionID(int64_t session_timeout_ms); @@ -144,8 +177,52 @@ public: /// Call if we don't need any responses for this session no more (session was expired) void finishSession(int64_t session_id); + + /// Invoked when a request completes. + void updateKeeperStat(UInt64 process_time_ms); + + /// Are we leader + bool isLeader() const override + { + return server->isLeader(); + } + + bool hasLeader() const override + { + return server->isLeaderAlive(); + } + + /// + String getRole() const override; + + UInt64 getOutstandingRequests() const override; + UInt64 getNumAliveConnections() const override; + + /// Request statistics such as qps, latency etc. + std::shared_ptr getKeeperStats() const + { + return keeper_stats; + } + + const IKeeperInfo & getKeeperInfo() const + { + return *this; + } + + IRaftInfo & getRaftInfo() const + { + return *server; + } + + const IStateMachineInfo & getStateMachineInfo() const + { + return *server->getKeeperStateMachine(); + } + + void dumpConf(WriteBufferFromOwnString & buf) const; }; +using KeeperStatsPtr = std::shared_ptr; } #endif diff --git a/src/Coordination/KeeperInfos.h b/src/Coordination/KeeperInfos.h new file mode 100644 index 00000000000..c495cbe1793 --- /dev/null +++ b/src/Coordination/KeeperInfos.h @@ -0,0 +1,72 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + + +/// Contains some useful interfaces which are helpful to get keeper information. +namespace DB +{ + +/// Keeper server related information +class IKeeperInfo +{ +public: + virtual bool isLeader() const = 0; + virtual bool hasLeader() const = 0; + + /// "leader", "follower", "observer" + virtual String getRole() const = 0; + + /// number alive connections of this node + virtual UInt64 getNumAliveConnections() const = 0; + + /// number of requests in queue + virtual UInt64 getOutstandingRequests() const = 0; + virtual ~IKeeperInfo() = default; +}; + +/// Keeper state machine related info +class IStateMachineInfo +{ +public: + /// last committed zxid + [[maybe_unused]] virtual UInt64 getLastProcessedZxid() const = 0; + + virtual UInt64 getNodeCount() const = 0; + virtual UInt64 getWatchCount() const = 0; + virtual UInt64 getEphemeralCount() const = 0; + + /// state machine approximate data size + virtual UInt64 getApproximateDataSize() const = 0; + virtual std::vector getDeadSessions() = 0; + + virtual ~IStateMachineInfo() = default; +}; + +/// Raft related info +class IRaftInfo +{ +public: + virtual bool isLeader() const = 0; + + virtual bool isLeaderAlive() const = 0; + + /// server role ignore zookeeper state "read-only" and "standalone" + virtual String getRole() const = 0; + + /// @return follower count if node is not leader return 0 + virtual UInt64 getFollowerCount() const = 0; + + /// @return synced follower count if node is not leader return 0 + virtual UInt64 getSyncedFollowerCount() const = 0; + + virtual ~IRaftInfo() = default; +}; + +} diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 840a0ee90bb..8b79341840c 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -63,27 +63,12 @@ void setSSLParams(nuraft::asio_service::options & asio_opts) } #endif -std::string getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper) + +std::string checkAndGetSuperdigest(const String & user_and_digest) { - /// the most specialized path - if (config.has("keeper_server.snapshot_storage_path")) - return config.getString("keeper_server.snapshot_storage_path"); - - if (config.has("keeper_server.storage_path")) - return std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots"; - - if (standalone_keeper) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots"; - else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots"; -} - -std::string checkAndGetSuperdigest(const Poco::Util::AbstractConfiguration & config) -{ - if (!config.has("keeper_server.superdigest")) + if (user_and_digest.empty()) return ""; - auto user_and_digest = config.getString("keeper_server.superdigest"); std::vector scheme_and_id; boost::split(scheme_and_id, user_and_digest, [](char c) { return c == ':'; }); if (scheme_and_id.size() != 2 || scheme_and_id[0] != "super") @@ -95,20 +80,18 @@ std::string checkAndGetSuperdigest(const Poco::Util::AbstractConfiguration & con } KeeperServer::KeeperServer( - int server_id_, - const CoordinationSettingsPtr & coordination_settings_, + const KeeperSettingsPtr & settings_, const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, - SnapshotsQueue & snapshots_queue_, - bool standalone_keeper) - : server_id(server_id_) - , coordination_settings(coordination_settings_) + SnapshotsQueue & snapshots_queue_) + : server_id(settings_->server_id) + , coordination_settings(settings_->coordination_settings) , state_machine(nuraft::cs_new( responses_queue_, snapshots_queue_, - getSnapshotsPathFromConfig(config, standalone_keeper), + settings_->log_storage_path, coordination_settings, - checkAndGetSuperdigest(config))) - , state_manager(nuraft::cs_new(server_id, "keeper_server", config, coordination_settings, standalone_keeper)) + checkAndGetSuperdigest(settings_->super_digest))) + , state_manager(nuraft::cs_new(server_id, "keeper_server", settings_->log_storage_path, config, coordination_settings)) , log(&Poco::Logger::get("KeeperServer")) { if (coordination_settings->quorum_reads) @@ -309,6 +292,40 @@ bool KeeperServer::isLeaderAlive() const return raft_instance->is_leader_alive(); } +String KeeperServer::getRole() const +{ + auto srv_config = state_manager->get_srv_config(); + if (srv_config->is_learner()) + { + /// zookeeper call read only node "observer" + return "observer"; + } + return isLeader() ? "leader" : "follower"; +} + +/// TODO test whether taking failed peer in count +UInt64 KeeperServer::getFollowerCount() const +{ + return raft_instance->get_peer_info_all().size(); +} + +UInt64 KeeperServer::getSyncedFollowerCount() const +{ + UInt64 last_log_idx = raft_instance->get_last_log_idx(); + auto followers = raft_instance->get_peer_info_all(); + + size_t stale_followers = 0; + + for (auto & fl : followers) + { + if (last_log_idx > fl.last_log_idx_ + raft_instance->get_current_params().stale_log_gap_) + { + stale_followers++; + } + } + return followers.size() - stale_followers; +} + nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * param) { if (initialized_flag) diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 7e4c5381f6e..3972109278a 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -8,13 +8,14 @@ #include #include #include +#include namespace DB { using RaftAppendResult = nuraft::ptr>>; -class KeeperServer +class KeeperServer : public IRaftInfo { private: const int server_id; @@ -52,12 +53,10 @@ private: public: KeeperServer( - int server_id_, - const CoordinationSettingsPtr & coordination_settings_, - const Poco::Util::AbstractConfiguration & config, + const KeeperSettingsPtr & settings_, + const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, - SnapshotsQueue & snapshots_queue_, - bool standalone_keeper); + SnapshotsQueue & snapshots_queue_); /// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings. void startup(); @@ -73,9 +72,24 @@ public: /// Return set of the non-active sessions std::vector getDeadSessions(); - bool isLeader() const; + /// Return set of the non-active sessions + nuraft::ptr getKeeperStateMachine() const + { + return state_machine; + } - bool isLeaderAlive() const; + bool isLeader() const override; + + bool isLeaderAlive() const override; + + /// server role ignore zookeeper state "read-only" and "standalone" + String getRole() const override; + + /// @return follower count if node is not leader return 0 + UInt64 getFollowerCount() const override; + + /// @return synced follower count if node is not leader return 0 + UInt64 getSyncedFollowerCount() const override; /// Wait server initialization (see callbackFunc) void waitInit(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 62820b41731..97da1e84f5c 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -326,16 +326,46 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push response with session id {} into responses queue", response.session_id); } +void KeeperStateMachine::shutdownStorage() +{ + std::lock_guard lock(storage_and_responses_lock); + storage->finalize(); +} + std::vector KeeperStateMachine::getDeadSessions() { std::lock_guard lock(storage_and_responses_lock); return storage->getDeadSessions(); } -void KeeperStateMachine::shutdownStorage() +UInt64 KeeperStateMachine::getLastProcessedZxid() const { std::lock_guard lock(storage_and_responses_lock); - storage->finalize(); + return storage->getZXID(); +} + +UInt64 KeeperStateMachine::getNodeCount() const +{ + std::lock_guard lock(storage_and_responses_lock); + return storage->getNodeCount(); +} + +UInt64 KeeperStateMachine::getWatchCount() const +{ + std::lock_guard lock(storage_and_responses_lock); + return storage->getWatchCount(); +} + +UInt64 KeeperStateMachine::getEphemeralCount() const +{ + std::lock_guard lock(storage_and_responses_lock); + return storage->getEphemeralCount(); +} + +UInt64 KeeperStateMachine::getApproximateDataSize() const +{ + std::lock_guard lock(storage_and_responses_lock); + return storage->getApproximateDataSize(); } ClusterConfigPtr KeeperStateMachine::getClusterConfig() const diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index b8085c032c9..b514493d75e 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -16,7 +17,7 @@ using SnapshotsQueue = ConcurrentBoundedQueue; /// ClickHouse Keeper state machine. Wrapper for KeeperStorage. /// Responsible for entries commit, snapshots creation and so on. -class KeeperStateMachine : public nuraft::state_machine +class KeeperStateMachine : public nuraft::state_machine, public IStateMachineInfo { public: KeeperStateMachine( @@ -67,6 +68,7 @@ public: nuraft::ptr & data_out, bool & is_last_obj) override; + /// just for test KeeperStorage & getStorage() { return *storage; @@ -76,6 +78,13 @@ public: void processReadRequest(const KeeperStorage::RequestForSession & request_for_session); std::vector getDeadSessions(); + UInt64 getLastProcessedZxid() const override; + + UInt64 getNodeCount() const override; + UInt64 getWatchCount() const override; + + UInt64 getEphemeralCount() const override; + UInt64 getApproximateDataSize() const override; void shutdownStorage(); @@ -110,7 +119,7 @@ private: /// we can get strange cases when, for example client send read request with /// watch and after that receive watch response and only receive response /// for request. - std::mutex storage_and_responses_lock; + mutable std::mutex storage_and_responses_lock; /// Last committed Raft log number. std::atomic last_committed_idx; diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 31b3a5f8b87..d56a3b84e8f 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -13,25 +13,6 @@ namespace ErrorCodes namespace { - std::string getLogsPathFromConfig( - const std::string & config_prefix, const Poco::Util::AbstractConfiguration & config, bool standalone_keeper) -{ - /// the most specialized path - if (config.has(config_prefix + ".log_storage_path")) - return config.getString(config_prefix + ".log_storage_path"); - - if (config.has(config_prefix + ".storage_path")) - return std::filesystem::path{config.getString(config_prefix + ".storage_path")} / "logs"; - - if (standalone_keeper) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs"; - else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs"; -} - -} - - KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const { KeeperConfigurationWrapper result; @@ -77,10 +58,12 @@ KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const P return result; } +} + KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path) - : my_server_id(server_id_) - , secure(false) - , log_store(nuraft::cs_new(logs_path, 5000, false, false)) +: my_server_id(server_id_) +, secure(false) +, log_store(nuraft::cs_new(logs_path, 5000, false, false)) { auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); configuration_wrapper.cluster_config = nuraft::cs_new(); @@ -90,18 +73,18 @@ KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, } KeeperStateManager::KeeperStateManager( - int server_id_, + int my_server_id_, const std::string & config_prefix_, + const std::string & log_storage_path, const Poco::Util::AbstractConfiguration & config, - const CoordinationSettingsPtr & coordination_settings, - bool standalone_keeper) - : my_server_id(server_id_) - , secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false)) + const CoordinationSettingsPtr & coordination_settings) + : my_server_id(my_server_id_) + , secure(config.getBool(config_prefix + ".raft_configuration.secure", false)) , config_prefix(config_prefix_) , configuration_wrapper(parseServersConfiguration(config, false)) , log_store(nuraft::cs_new( - getLogsPathFromConfig(config_prefix_, config, standalone_keeper), - coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, coordination_settings->compress_logs)) + log_storage_path, + coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync)) { } diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index 5f051fcfc30..8940451a9f9 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -52,9 +52,9 @@ public: KeeperStateManager( int server_id_, const std::string & config_prefix_, + const std::string & log_storage_path, const Poco::Util::AbstractConfiguration & config, - const CoordinationSettingsPtr & coordination_settings, - bool standalone_keeper); + const CoordinationSettingsPtr & coordination_settings); /// Constructor for tests KeeperStateManager( diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index bc9a81bc484..1ea57a9b77c 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -39,6 +39,17 @@ public: Coordination::Stat stat{}; int32_t seq_num = 0; ChildrenSet children{}; + + /// object memory size + UInt64 size() const + { + UInt64 child_size{0}; + for(auto & child : children) + { + child_size += child.size(); + } + return data.size() + sizeof (Node) + child_size; + } }; struct ResponseForSession @@ -176,6 +187,41 @@ public: { return session_expiry_queue.getExpiredSessions(); } + + UInt64 getNodeCount() const + { + return container.size(); + } + + UInt64 getWatchCount() const + { + return watches.size() + list_watches.size(); + } + + UInt64 getEphemeralCount() const + { + UInt64 ret{0}; + for(const auto & ephs : ephemerals) + { + ret += ephs.second.size(); + } + return ret; + } + + UInt64 getApproximateDataSize() const + { + UInt64 size{0}; + for (const auto & it : container) + { + size += (it.key.size() + it.value.size() + 1); + } + /// hash map key size + if (container.size()) + { + size += sizeof(StringRef) * container.size(); + } + return size; + } }; using KeeperStoragePtr = std::unique_ptr; diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 71099c23655..ce7073441b8 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #ifdef POCO_HAVE_FD_EPOLL #include @@ -222,16 +223,15 @@ void KeeperTCPHandler::run() runImpl(); } -Poco::Timespan KeeperTCPHandler::receiveHandshake() +Poco::Timespan KeeperTCPHandler::receiveHandshake(int32_t handshake_length) { - int32_t handshake_length; int32_t protocol_version; int64_t last_zxid_seen; int32_t timeout_ms; int64_t previous_session_id = 0; /// We don't support session restore. So previous session_id is always zero. std::array passwd {}; - Coordination::read(handshake_length, *in); - if (handshake_length != Coordination::CLIENT_HANDSHAKE_LENGTH && handshake_length != Coordination::CLIENT_HANDSHAKE_LENGTH_WITH_READONLY) + + if (!isHandShake(handshake_length)) throw Exception("Unexpected handshake length received: " + toString(handshake_length), ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT); Coordination::read(protocol_version, *in); @@ -274,9 +274,32 @@ void KeeperTCPHandler::runImpl() return; } + int32_t header; try { - auto client_timeout = receiveHandshake(); + Coordination::read(header, *in); + } + catch (const Exception & e) + { + LOG_WARNING(log, "Error while read connection header {}", e.displayText()); + return; + } + + /// All four letter word command code is larger than 2^24 or lower than 0. + /// Hand shake package length must be lower than 2^24 and larger than 0. + /// So collision never happens. + int32_t four_letter_cmd = header; + if (!isHandShake(four_letter_cmd)) + { + tryExecuteFourLetterWordCmd(four_letter_cmd); + return; + } + + try + { + int32_t handshake_length = header; + auto client_timeout = receiveHandshake(handshake_length); + if (client_timeout != 0) session_timeout = std::min(client_timeout, session_timeout); } @@ -328,6 +351,8 @@ void KeeperTCPHandler::runImpl() session_stopwatch.start(); bool close_received = false; + Stopwatch process_time_stopwatch; + try { while (true) @@ -337,6 +362,7 @@ void KeeperTCPHandler::runImpl() PollResult result = poll_wrapper->poll(session_timeout, in); if (result.has_requests && !close_received) { + process_time_stopwatch.start(); auto [received_op, received_xid] = receiveRequest(); if (received_op == Coordination::OpNum::Close) @@ -354,6 +380,13 @@ void KeeperTCPHandler::runImpl() session_stopwatch.restart(); } + /// Do request statistics, + /// not accurate when there is watch response in the channel + if(result.responses_count != 0) + { + process_time_stopwatch.stop(); + keeper_dispatcher->updateKeeperStat(process_time_stopwatch.elapsedMilliseconds()); + } /// Process exact amount of responses from pipe /// otherwise state of responses queue and signaling pipe /// became inconsistent and race condition is possible. @@ -399,6 +432,47 @@ void KeeperTCPHandler::runImpl() } } +bool KeeperTCPHandler::isHandShake(Int32 & handshake_length) +{ + return handshake_length == Coordination::CLIENT_HANDSHAKE_LENGTH + || handshake_length == Coordination::CLIENT_HANDSHAKE_LENGTH_WITH_READONLY; +} + +bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(Int32 & four_letter_cmd) +{ + if (FourLetterCommandFactory::instance().isKnown(four_letter_cmd)) + { + auto command = FourLetterCommandFactory::instance().get(four_letter_cmd); + LOG_DEBUG(log, "receive four letter command {}", command->name()); + + String res; + try + { + res = command->run(); + } + catch (...) + { + tryLogCurrentException(log, "Error when executing four letter command " + command->name()); + } + + try + { + out->write(res.data(), res.size()); + } + catch (const Exception &) + { + tryLogCurrentException(log, "Error when send 4 letter command response"); + } + + return true; + } + else + { + LOG_WARNING(log, "invalid four letter command {}", std::to_string(four_letter_cmd)); + } + return false; +} + std::pair KeeperTCPHandler::receiveRequest() { int32_t length; diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index 274fb21af63..eebb720939d 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -56,7 +56,10 @@ private: void runImpl(); void sendHandshake(bool has_leader); - Poco::Timespan receiveHandshake(); + Poco::Timespan receiveHandshake(int32_t handshake_length); + + static bool isHandShake(Int32 & handshake_length) ; + bool tryExecuteFourLetterWordCmd(Int32 & four_letter_cmd); std::pair receiveRequest(); }; diff --git a/tests/integration/test_keeper_four_word_command/__init__.py b/tests/integration/test_keeper_four_word_command/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_keeper_four_word_command/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml new file mode 100644 index 00000000000..bcbaa8780ce --- /dev/null +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml @@ -0,0 +1,41 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + 75 + trace + + + + + 1 + node1 + 44444 + true + 3 + + + 2 + node2 + 44444 + false + true + 2 + + + 3 + node3 + 44444 + false + true + 1 + + + + diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml new file mode 100644 index 00000000000..2f0ae15d542 --- /dev/null +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml @@ -0,0 +1,41 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + 75 + trace + + + + + 1 + node1 + 44444 + true + 3 + + + 2 + node2 + 44444 + false + true + 2 + + + 3 + node3 + 44444 + false + true + 1 + + + + diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml new file mode 100644 index 00000000000..a6315675df8 --- /dev/null +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml @@ -0,0 +1,41 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + 75 + trace + + + + + 1 + node1 + 44444 + true + 3 + + + 2 + node2 + 44444 + false + true + 2 + + + 3 + node3 + 44444 + false + true + 1 + + + + diff --git a/tests/integration/test_keeper_four_word_command/configs/keeper_config.xml b/tests/integration/test_keeper_four_word_command/configs/keeper_config.xml new file mode 100644 index 00000000000..2ad367b5bbf --- /dev/null +++ b/tests/integration/test_keeper_four_word_command/configs/keeper_config.xml @@ -0,0 +1,24 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + + 5000 + 10000 + trace + 75 + + + + + 1 + localhost + 44444 + + + + diff --git a/tests/integration/test_keeper_four_word_command/configs/use_keeper.xml b/tests/integration/test_keeper_four_word_command/configs/use_keeper.xml new file mode 100644 index 00000000000..b6139005d2f --- /dev/null +++ b/tests/integration/test_keeper_four_word_command/configs/use_keeper.xml @@ -0,0 +1,16 @@ + + + + node1 + 9181 + + + node2 + 9181 + + + node3 + 9181 + + + diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py new file mode 100644 index 00000000000..a11c6713bf1 --- /dev/null +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -0,0 +1,285 @@ +import socket +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 io import StringIO +import csv + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/enable_keeper1.xml', 'configs/use_keeper.xml'], + stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=['configs/enable_keeper2.xml', 'configs/use_keeper.xml'], + stay_alive=True) +node3 = cluster.add_instance('node3', main_configs=['configs/enable_keeper3.xml', 'configs/use_keeper.xml'], + stay_alive=True) + +from kazoo.client import KazooClient, KazooState + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def destroy_zk_client(zk): + try: + if zk: + zk.stop() + zk.close() + except: + pass + + +def wait_node(node): + for _ in range(100): + zk = None + try: + node.query("SELECT * FROM system.zookeeper WHERE path = '/'") + zk = get_fake_zk(node.name, timeout=30.0) + # zk.create("/test", sequence=True) + print("node", node.name, "ready") + break + except Exception as ex: + time.sleep(0.2) + print("Waiting until", node.name, "will be ready, exception", ex) + finally: + destroy_zk_client(zk) + else: + raise Exception("Can't wait node", node.name, "to become ready") + + +def wait_nodes(): + for n in [node1, node2, node3]: + wait_node(n) + + +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) + _fake_zk_instance.start() + return _fake_zk_instance + + +def get_keeper_socket(nodename): + hosts = cluster.get_instance_ip(nodename) + client = socket.socket() + client.settimeout(10) + client.connect((hosts, 9181)) + return client + + +def close_keeper_socket(cli): + if cli is not None: + print("close socket") + cli.close() + + +def test_cmd_ruok(started_cluster): + client = None + try: + wait_nodes() + client = get_keeper_socket("node1") + client.send(b'ruok') + data = client.recv(4) + print(data) + assert data.decode() == 'imok' + finally: + close_keeper_socket(client) + + +def do_some_action(zk, create_cnt=0, get_cnt=0, set_cnt=0, ephemeral_cnt=0, watch_cnt=0, delete_cnt=0): + assert create_cnt >= get_cnt + assert create_cnt >= set_cnt + assert create_cnt >= watch_cnt + assert create_cnt >= delete_cnt + + for i in range(create_cnt): + zk.create("/normal_node_" + str(i), b"") + + for i in range(get_cnt): + zk.get("/normal_node_" + str(i)) + + for i in range(set_cnt): + zk.set("/normal_node_" + str(i), b"new-value") + + for i in range(ephemeral_cnt): + zk.create("/ephemeral_node_" + str(i), ephemeral=True) + + fake_ephemeral_event = None + + def fake_ephemeral_callback(event): + print("Fake watch triggered") + nonlocal fake_ephemeral_event + fake_ephemeral_event = event + + for i in range(watch_cnt): + zk.exists("/normal_node_" + str(i), watch=fake_ephemeral_callback) + + for i in range(create_cnt - delete_cnt, create_cnt): + zk.delete("/normal_node_" + str(i)) + + +def test_cmd_mntr(started_cluster): + client = None + zk = None + try: + wait_nodes() + + # reset stat first + client = get_keeper_socket("node1") + client.send(b'srst') + data = client.recv(10_000) + client.close() + assert data.decode() == "Server stats reset." + + zk = get_fake_zk(node1.name, timeout=30.0) + do_some_action(zk, create_cnt=10, get_cnt=10, set_cnt=5, ephemeral_cnt=2, watch_cnt=2, delete_cnt=2) + + client = get_keeper_socket("node1") + client.send(b'mntr') + data = client.recv(10_000_000) + assert len(data) != 0 + + # print(data.decode()) + reader = csv.reader(data.decode().split('\n'), delimiter='\t') + result = {} + + for row in reader: + if len(row) != 0: + result[row[0]] = row[1] + + assert len(result["zk_version"]) != 0 + + assert int(result["zk_avg_latency"]) >= 0 + assert int(result["zk_max_latency"]) >= 0 + assert int(result["zk_min_latency"]) >= 0 + + assert int(result["zk_min_latency"]) <= int(result["zk_avg_latency"]) + assert int(result["zk_max_latency"]) >= int(result["zk_avg_latency"]) + + assert int(result["zk_packets_received"]) == 31 + # contains 31 user request response, 1 session establish response + assert int(result["zk_packets_sent"]) == 32 + + assert 1 <= int(result["zk_num_alive_connections"]) <= 4 + assert int(result["zk_outstanding_requests"]) == 0 + + assert result["zk_server_state"] == "leader" + + # contains: + # 10 nodes created by test + # 3 nodes created by clickhouse "/clickhouse/task_queue/ddl" + # 1 root node + assert int(result["zk_znode_count"]) == 14 + # ClickHouse may watch "/clickhouse/task_queue/ddl" + assert int(result["zk_watch_count"]) >= 2 + assert int(result["zk_ephemerals_count"]) == 2 + assert int(result["zk_approximate_data_size"]) > 0 + + assert int(result["zk_open_file_descriptor_count"]) > 0 + assert int(result["zk_max_file_descriptor_count"]) > 0 + + assert int(result["zk_followers"]) == 2 + assert int(result["zk_synced_followers"]) == 2 + + finally: + destroy_zk_client(zk) + close_keeper_socket(client) + + +def test_cmd_srst(started_cluster): + client = None + try: + wait_nodes() + + # reset stat first + client = get_keeper_socket("node1") + client.send(b'srst') + data = client.recv(10_000) + client.close() + assert data.decode() == "Server stats reset." + + client = get_keeper_socket("node1") + client.send(b'mntr') + data = client.recv(10_000_000) + assert len(data) != 0 + + # print(data.decode()) + reader = csv.reader(data.decode().split('\n'), delimiter='\t') + result = {} + + for row in reader: + if len(row) != 0: + result[row[0]] = row[1] + + assert int(result["zk_packets_received"]) == 0 + assert int(result["zk_packets_sent"]) == 0 + + finally: + close_keeper_socket(client) + + +def test_cmd_conf(started_cluster): + client = None + try: + wait_nodes() + client = get_keeper_socket("node1") + + # reset stat first + client.send(b'conf') + data = client.recv(10_000_000) + assert len(data) != 0 + + # print(data.decode()) + reader = csv.reader(data.decode().split('\n'), delimiter='=') + result = {} + + for row in reader: + if len(row) != 0: + print(row) + result[row[0]] = row[1] + + assert result["server_id"] == "1" + assert result["tcp_port"] == "9181" + assert "tcp_port_secure" not in result + assert "superdigest" not in result + + assert result["log_storage_path"] == "/var/lib/clickhouse/coordination/log" + assert result["snapshot_storage_path"] == "/var/lib/clickhouse/coordination/snapshots" + + assert result["session_timeout_ms"] == "10000" + assert result["operation_timeout_ms"] == "5000" + assert result["dead_session_check_period_ms"] == "500" + assert result["heart_beat_interval_ms"] == "500" + assert result["election_timeout_lower_bound_ms"] == "1000" + assert result["election_timeout_upper_bound_ms"] == "2000" + assert result["reserved_log_items"] == "100000" + + assert result["snapshot_distance"] == "75" + assert result["auto_forwarding"] == "true" + assert result["shutdown_timeout"] == "5000" + assert result["startup_timeout"] == "30000" + + assert result["raft_logs_level"] == "trace" + assert result["rotate_log_storage_interval"] == "100000" + assert result["snapshots_to_keep"] == "3" + assert result["stale_log_gap"] == "10000" + assert result["fresh_log_gap"] == "200" + + assert result["max_requests_batch_size"] == "100" + assert result["quorum_reads"] == "false" + assert result["force_sync"] == "true" + + finally: + close_keeper_socket(client) From e5b0eedd31923ae13def87010bf64f96c4ba8221 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 27 Oct 2021 22:26:53 +0800 Subject: [PATCH 02/49] adjust code style for keeper 4lw cmd --- src/Common/getCurrentProcessFDCount.cpp | 2 +- src/Common/getMaxFileDescriptorCount.cpp | 2 +- src/Coordination/CoordinationSettings.cpp | 18 ++++++++++-------- src/Coordination/FourLetterCommand.h | 6 ------ src/Coordination/KeeperDispatcher.cpp | 7 +++---- src/Coordination/KeeperStorage.h | 4 ++-- src/Server/KeeperTCPHandler.cpp | 2 +- 7 files changed, 18 insertions(+), 23 deletions(-) diff --git a/src/Common/getCurrentProcessFDCount.cpp b/src/Common/getCurrentProcessFDCount.cpp index 9653b4eccd8..6bcfabcdf20 100644 --- a/src/Common/getCurrentProcessFDCount.cpp +++ b/src/Common/getCurrentProcessFDCount.cpp @@ -20,7 +20,7 @@ int getCurrentProcessFDCount() WriteBufferFromOwnString out; copyData(command->out, out); - if(!out.str().empty()) + if (!out.str().empty()) { return std::stoi(out.str()); } diff --git a/src/Common/getMaxFileDescriptorCount.cpp b/src/Common/getMaxFileDescriptorCount.cpp index 751145e4967..70de1dd0c08 100644 --- a/src/Common/getMaxFileDescriptorCount.cpp +++ b/src/Common/getMaxFileDescriptorCount.cpp @@ -14,7 +14,7 @@ int getMaxFileDescriptorCount() WriteBufferFromOwnString out; copyData(command->out, out); - if(!out.str().empty()) + if (!out.str().empty()) { return std::stoi(out.str()); } diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 46e60689103..e4f2cd1e031 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -37,13 +37,15 @@ void KeeperSettings::dump(WriteBufferFromOwnString & buf) const { auto write = [&buf](const String & content) { buf.write(content.data(), content.size()); }; - auto write_int = [&buf](Int64 value) { + auto write_int = [&buf](Int64 value) + { String str_val = std::to_string(value); buf.write(str_val.data(), str_val.size()); buf.write('\n'); }; - auto write_bool = [&buf](bool value) { + auto write_bool = [&buf](bool value) + { String str_val = value ? "true" : "false"; buf.write(str_val.data(), str_val.size()); buf.write('\n'); @@ -52,17 +54,17 @@ void KeeperSettings::dump(WriteBufferFromOwnString & buf) const write("server_id="); write_int(server_id); - if(tcp_port != NO_PORT) + if (tcp_port != NO_PORT) { write("tcp_port="); write_int(tcp_port); } - if(tcp_port_secure != NO_PORT) + if (tcp_port_secure != NO_PORT) { write("tcp_port_secure="); write_int(tcp_port_secure); } - if(!super_digest.empty()) + if (!super_digest.empty()) { write("superdigest="); write(super_digest); @@ -136,15 +138,15 @@ KeeperSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, ret->server_id = config.getInt("keeper_server.server_id"); ret->standalone_keeper = standalone_keeper_; - if(config.has("keeper_server.tcp_port")) + if (config.has("keeper_server.tcp_port")) { ret->tcp_port = config.getInt("keeper_server.tcp_port"); } - if(config.has("keeper_server.tcp_port_secure")) + if (config.has("keeper_server.tcp_port_secure")) { ret->tcp_port_secure = config.getInt("keeper_server.tcp_port_secure"); } - if(config.has("keeper_server.superdigest")) + if (config.has("keeper_server.superdigest")) { ret->super_digest = config.getString("keeper_server.tcp_port_secure"); } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 3867a5ede86..1cbd880978f 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -7,7 +7,6 @@ #include #include #include -#include #if !defined(ARCADIA_BUILD) # include @@ -98,9 +97,6 @@ struct RuokCommand : public IFourLetterCommand * zk_followers 2 - only exposed by the Leader * zk_synced_followers 2 - only exposed by the Leader * zk_pending_syncs 0 - only exposed by the Leader - * zk_last_proposal_size -1 - * zk_max_proposal_size -1 - * zk_min_proposal_size -1 */ struct MonitorCommand : public IFourLetterCommand { @@ -144,6 +140,4 @@ struct ConfCommand : public IFourLetterCommand ~ConfCommand() override; }; - - } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 3977c36a943..e34693e2821 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -12,7 +12,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int TIMEOUT_EXCEEDED; extern const int SYSTEM_ERROR; - extern const int UNKNOWN_SETTING; } UInt64 KeeperDispatcher::KeeperStats::getMinLatency() const @@ -330,7 +329,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ if (!requests_queue->push(std::move(request_info))) throw Exception("Cannot push request to queue", ErrorCodes::SYSTEM_ERROR); } - else if (!requests_queue->tryPush(std::move(request_info), coordination_settings->operation_timeout_ms.totalMilliseconds())) + else if (!requests_queue->tryPush(std::move(request_info), settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) { throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); } @@ -342,14 +341,14 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf { LOG_DEBUG(log, "Initializing storage dispatcher"); - settings = KeeperSettings::loadFromConfig(config_, standalone_keeper); + settings = KeeperSettings::loadFromConfig(config, standalone_keeper); requests_queue = std::make_unique(settings->coordination_settings->max_requests_batch_size); request_thread = ThreadFromGlobalPool([this] { requestThread(); }); responses_thread = ThreadFromGlobalPool([this] { responseThread(); }); snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); }); - server = std::make_unique(settings, config_, responses_queue, snapshots_queue); + server = std::make_unique(settings, config, responses_queue, snapshots_queue); try { diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 1ea57a9b77c..ed12c448f24 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -44,7 +44,7 @@ public: UInt64 size() const { UInt64 child_size{0}; - for(auto & child : children) + for (auto & child : children) { child_size += child.size(); } @@ -201,7 +201,7 @@ public: UInt64 getEphemeralCount() const { UInt64 ret{0}; - for(const auto & ephs : ephemerals) + for (const auto & ephs : ephemerals) { ret += ephs.second.size(); } diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index ce7073441b8..ae11fa31a3a 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -382,7 +382,7 @@ void KeeperTCPHandler::runImpl() /// Do request statistics, /// not accurate when there is watch response in the channel - if(result.responses_count != 0) + if (result.responses_count != 0) { process_time_stopwatch.stop(); keeper_dispatcher->updateKeeperStat(process_time_stopwatch.elapsedMilliseconds()); From 4908f714f41b40a309b71fa762be59ab3b1e0453 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 27 Oct 2021 23:21:26 +0800 Subject: [PATCH 03/49] dump new coordination settings --- src/Coordination/CoordinationSettings.cpp | 8 ++++++++ src/Coordination/KeeperInfos.h | 1 - src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStateManager.cpp | 8 ++------ tests/integration/test_keeper_four_word_command/test.py | 4 ++++ 5 files changed, 15 insertions(+), 8 deletions(-) diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index e4f2cd1e031..949d0f71c5a 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -128,6 +129,13 @@ void KeeperSettings::dump(WriteBufferFromOwnString & buf) const write_bool(coordination_settings->quorum_reads); write("force_sync="); write_bool(coordination_settings->force_sync); + + write("compress_logs="); + write_bool(coordination_settings->compress_logs); + write("compress_snapshots_with_zstd_format="); + write_bool(coordination_settings->compress_snapshots_with_zstd_format); + write("configuration_change_tries_count="); + write_int(coordination_settings->configuration_change_tries_count); } std::shared_ptr diff --git a/src/Coordination/KeeperInfos.h b/src/Coordination/KeeperInfos.h index c495cbe1793..5a47fd9ded7 100644 --- a/src/Coordination/KeeperInfos.h +++ b/src/Coordination/KeeperInfos.h @@ -6,7 +6,6 @@ #include #include #include -#include /// Contains some useful interfaces which are helpful to get keeper information. diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index b514493d75e..5801389c6e8 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -77,7 +77,7 @@ public: /// Process local read request void processReadRequest(const KeeperStorage::RequestForSession & request_for_session); - std::vector getDeadSessions(); + std::vector getDeadSessions() override; UInt64 getLastProcessedZxid() const override; UInt64 getNodeCount() const override; diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index d56a3b84e8f..492af6be932 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -11,8 +11,6 @@ namespace ErrorCodes extern const int RAFT_ERROR; } -namespace -{ KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const { KeeperConfigurationWrapper result; @@ -58,8 +56,6 @@ KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const P return result; } -} - KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path) : my_server_id(server_id_) , secure(false) @@ -79,12 +75,12 @@ KeeperStateManager::KeeperStateManager( const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings) : my_server_id(my_server_id_) - , secure(config.getBool(config_prefix + ".raft_configuration.secure", false)) + , secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false)) , config_prefix(config_prefix_) , configuration_wrapper(parseServersConfiguration(config, false)) , log_store(nuraft::cs_new( log_storage_path, - coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync)) + coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, coordination_settings->compress_logs)) { } diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index a11c6713bf1..07de4e646f6 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -281,5 +281,9 @@ def test_cmd_conf(started_cluster): assert result["quorum_reads"] == "false" assert result["force_sync"] == "true" + assert result["compress_logs"] == "true" + assert result["compress_snapshots_with_zstd_format"] == "true" + assert result["configuration_change_tries_count"] == "20" + finally: close_keeper_socket(client) From 9f0fe34bcc35b76fc3f27cd6eb88c7470d8aed97 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 28 Oct 2021 14:19:50 +0800 Subject: [PATCH 04/49] fix test bug --- tests/integration/test_keeper_four_word_command/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 07de4e646f6..07e5cbcdc25 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -269,7 +269,7 @@ def test_cmd_conf(started_cluster): assert result["snapshot_distance"] == "75" assert result["auto_forwarding"] == "true" assert result["shutdown_timeout"] == "5000" - assert result["startup_timeout"] == "30000" + assert result["startup_timeout"] == "180000" assert result["raft_logs_level"] == "trace" assert result["rotate_log_storage_interval"] == "100000" From 6f672df326feb295621a1fe0218616099f9c64e1 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 28 Oct 2021 16:01:30 +0800 Subject: [PATCH 05/49] fix PVS check error --- src/Coordination/CoordinationSettings.cpp | 15 ++++++++++++--- src/Coordination/CoordinationSettings.h | 8 ++++---- 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 949d0f71c5a..9b316dca5ab 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -34,6 +34,16 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco } } +KeeperSettings::KeeperSettings() + : server_id(NOT_EXIST) + , tcp_port(NOT_EXIST) + , tcp_port_secure(NOT_EXIST) + , standalone_keeper(false) + , coordination_settings(std::make_shared()) +{ +} + + void KeeperSettings::dump(WriteBufferFromOwnString & buf) const { auto write = [&buf](const String & content) { buf.write(content.data(), content.size()); }; @@ -55,12 +65,12 @@ void KeeperSettings::dump(WriteBufferFromOwnString & buf) const write("server_id="); write_int(server_id); - if (tcp_port != NO_PORT) + if (tcp_port != NOT_EXIST) { write("tcp_port="); write_int(tcp_port); } - if (tcp_port_secure != NO_PORT) + if (tcp_port_secure != NOT_EXIST) { write("tcp_port_secure="); write_int(tcp_port_secure); @@ -162,7 +172,6 @@ KeeperSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, ret->log_storage_path = getLogsPathFromConfig(config, standalone_keeper_); ret->snapshot_storage_path = getSnapshotsPathFromConfig(config, standalone_keeper_); - ret->coordination_settings = std::make_shared(); ret->coordination_settings->loadFromConfig("keeper_server.coordination_settings", config); return ret; diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 9e3a281b0e1..74ca3744340 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -54,13 +54,13 @@ using CoordinationSettingsPtr = std::shared_ptr; struct KeeperSettings { - static constexpr int NO_PORT = -1; + static constexpr int NOT_EXIST = -1; - KeeperSettings() = default; + KeeperSettings(); int server_id; - int tcp_port{NO_PORT}; - int tcp_port_secure{NO_PORT}; + int tcp_port; + int tcp_port_secure; String super_digest; From 239a13ef9a8fcd771a8f2ba83f57420472e0b991 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 28 Oct 2021 18:27:36 +0800 Subject: [PATCH 06/49] fix clang13 build error --- src/Coordination/KeeperDispatcher.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index e34693e2821..ad117f216f1 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -662,9 +662,9 @@ void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfigurati } } -void KeeperDispatcher::updateKeeperStat(UInt64 process_time_stopwatch) +void KeeperDispatcher::updateKeeperStat(UInt64 process_time_ms) { - keeper_stats->updateLatency(process_time_stopwatch); + keeper_stats->updateLatency(process_time_ms); } String KeeperDispatcher::getRole() const From 92a0b949c6117813d47f6f6f42f73f5e3132af1a Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 28 Oct 2021 19:05:19 +0800 Subject: [PATCH 07/49] ignore superdigest whem dump keeper configuration --- src/Coordination/CoordinationSettings.cpp | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 9b316dca5ab..177412ea585 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -75,12 +75,6 @@ void KeeperSettings::dump(WriteBufferFromOwnString & buf) const write("tcp_port_secure="); write_int(tcp_port_secure); } - if (!super_digest.empty()) - { - write("superdigest="); - write(super_digest); - buf.write('\n'); - } write("log_storage_path="); write(log_storage_path); @@ -166,7 +160,7 @@ KeeperSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, } if (config.has("keeper_server.superdigest")) { - ret->super_digest = config.getString("keeper_server.tcp_port_secure"); + ret->super_digest = config.getString("keeper_server.superdigest"); } ret->log_storage_path = getLogsPathFromConfig(config, standalone_keeper_); From 694306ee62b98d53310546be1d632dc985c3b79f Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 28 Oct 2021 22:22:56 +0800 Subject: [PATCH 08/49] add 4lw white list to keeper --- src/Coordination/CoordinationSettings.cpp | 6 ++ src/Coordination/CoordinationSettings.h | 2 + src/Coordination/FourLetterCommand.cpp | 74 ++++++++++++-- src/Coordination/FourLetterCommand.h | 12 ++- src/Coordination/KeeperDispatcher.h | 5 + src/Server/KeeperTCPHandler.cpp | 3 +- .../configs/keeper_config.xml | 24 ----- .../configs/keeper_config_with_white_list.xml | 38 +++++++ .../keeper_config_with_white_list_all.xml | 38 +++++++ .../keeper_config_without_white_list.xml | 37 +++++++ .../test_keeper_four_word_command/test.py | 1 + .../test_white_list.py | 98 +++++++++++++++++++ 12 files changed, 304 insertions(+), 34 deletions(-) delete mode 100644 tests/integration/test_keeper_four_word_command/configs/keeper_config.xml create mode 100644 tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list.xml create mode 100644 tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list_all.xml create mode 100644 tests/integration/test_keeper_four_word_command/configs/keeper_config_without_white_list.xml create mode 100644 tests/integration/test_keeper_four_word_command/test_white_list.py diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 177412ea585..382d37e9ed8 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -76,6 +76,10 @@ void KeeperSettings::dump(WriteBufferFromOwnString & buf) const write_int(tcp_port_secure); } + write("four_letter_word_white_list="); + write(four_letter_word_white_list); + buf.write('\n'); + write("log_storage_path="); write(log_storage_path); buf.write('\n'); @@ -163,6 +167,8 @@ KeeperSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, ret->super_digest = config.getString("keeper_server.superdigest"); } + ret->four_letter_word_white_list = config.getString("keeper_server.four_letter_word_white_list", "*"); + ret->log_storage_path = getLogsPathFromConfig(config, standalone_keeper_); ret->snapshot_storage_path = getSnapshotsPathFromConfig(config, standalone_keeper_); diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 74ca3744340..29162587813 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -62,6 +62,8 @@ struct KeeperSettings int tcp_port; int tcp_port_secure; + String four_letter_word_white_list; + String super_digest; bool standalone_keeper; diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 410f774e206..ee0fcad331f 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -8,6 +9,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int INVALID_SETTING_VALUE; } IFourLetterCommand::IFourLetterCommand(const KeeperDispatcher & keeper_dispatcher_) : keeper_dispatcher(keeper_dispatcher_) @@ -16,7 +18,17 @@ IFourLetterCommand::IFourLetterCommand(const KeeperDispatcher & keeper_dispatche Int32 IFourLetterCommand::code() { - Int32 res = *reinterpret_cast(name().data()); + return toCode(name()); +} + +String IFourLetterCommand::toName(Int32 code) +{ + return String(reinterpret_cast(__builtin_bswap32(code))); +} + +Int32 IFourLetterCommand::toCode(const String & name) +{ + Int32 res = *reinterpret_cast(name.data()); /// keep consistent with Coordination::read method by changing big endian to little endian. return __builtin_bswap32(res); } @@ -30,6 +42,7 @@ void IFourLetterCommand::printSet(IFourLetterCommand::StringBuffer & buffer, std buffer.write('\n'); } } + IFourLetterCommand::~IFourLetterCommand() = default; FourLetterCommandFactory & FourLetterCommandFactory::instance() @@ -38,21 +51,23 @@ FourLetterCommandFactory & FourLetterCommandFactory::instance() return factory; } -bool FourLetterCommandFactory::isKnown(Int32 code) +void FourLetterCommandFactory::checkInitialization() const { if (!initialized) { - throw Exception("Four letter command " + std::to_string(code) + " not initialized", ErrorCodes::LOGICAL_ERROR); + throw Exception("Four letter command not initialized", ErrorCodes::LOGICAL_ERROR); } +} + +bool FourLetterCommandFactory::isKnown(Int32 code) +{ + checkInitialization(); return commands.contains(code); } FourLetterCommandPtr FourLetterCommandFactory::get(Int32 code) { - if (!initialized) - { - throw Exception("Four letter command " + std::to_string(code) + " not initialized", ErrorCodes::LOGICAL_ERROR); - } + checkInitialization(); return commands.at(code); } @@ -60,7 +75,7 @@ void FourLetterCommandFactory::registerCommand(FourLetterCommandPtr & command) { if (commands.contains(command->code())) { - throw Exception("Four letter command " + std::to_string(command->code()) + " already registered", ErrorCodes::LOGICAL_ERROR); + throw Exception("Four letter command " + command->name() + " already registered", ErrorCodes::LOGICAL_ERROR); } auto * log = &Poco::Logger::get("FourLetterCommandFactory"); LOG_INFO(log, "Register four letter command {}, code {}", command->name(), std::to_string(command->code())); @@ -85,10 +100,53 @@ void FourLetterCommandFactory::registerCommands(const KeeperDispatcher & keeper_ FourLetterCommandPtr conf_command = std::make_shared(keeper_dispatcher); factory.registerCommand(conf_command); + factory.initializeWhiteList(keeper_dispatcher); factory.setInitialize(true); } } +bool FourLetterCommandFactory::isEnabled(Int32 code) +{ + checkInitialization(); + if (!white_list.empty() && *white_list.cbegin() == WHITE_LIST_ALL) + { + return true; + } + return std::find(white_list.begin(), white_list.end(), code) != white_list.end(); +} + +void FourLetterCommandFactory::initializeWhiteList(const KeeperDispatcher & keeper_dispatcher) +{ + using Poco::StringTokenizer; + const auto & keeper_settings = keeper_dispatcher.getKeeperSettings(); + + String list_str = keeper_settings->four_letter_word_white_list; + StringTokenizer tokenizer(list_str, ",", 2); + + for (const String & token : tokenizer) + { + if (token == "*") + { + white_list.clear(); + white_list.resize(1); + white_list.push_back(WHITE_LIST_ALL); + return; + } + else + { + if (commands.contains(IFourLetterCommand::toCode(token))) + { + white_list.push_back(IFourLetterCommand::toCode(token)); + } + else + { + auto * log = &Poco::Logger::get("FourLetterCommandFactory"); + LOG_WARNING(log, "Find invalid keeper 4lw command {} when initializing, ignore it.", token); + } + } + } +} + String RuokCommand::name() { return "ruok"; diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 1cbd880978f..b3023027dab 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -32,6 +32,9 @@ public: virtual ~IFourLetterCommand(); Int32 code(); + static inline String toName(Int32 code); + static inline Int32 toCode(const String & name); + static void printSet(StringBuffer & buffer, std::unordered_set & set, String && prefix); protected: @@ -42,15 +45,21 @@ struct FourLetterCommandFactory : private boost::noncopyable { public: using Commands = std::unordered_map; + using WhiteList = std::vector; + + static constexpr Int32 WHITE_LIST_ALL = 0; bool isKnown(Int32 code); + bool isEnabled(Int32 code); + FourLetterCommandPtr get(Int32 code); /// There is no need to make it thread safe, because registration is no initialization and get is after startup. void registerCommand(FourLetterCommandPtr & command); + void initializeWhiteList(const KeeperDispatcher & keeper_dispatcher); + void checkInitialization() const; bool isInitialized() const { return initialized; } - void setInitialize(bool flag) { initialized = flag; } static FourLetterCommandFactory & instance(); @@ -59,6 +68,7 @@ public: private: volatile bool initialized = false; Commands commands; + WhiteList white_list; }; /**Tests if server is running in a non-error state. The server will respond with imok if it is running. diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 6ea35f420ea..87f8d23aafc 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -219,6 +219,11 @@ public: return *server->getKeeperStateMachine(); } + const KeeperSettingsPtr & getKeeperSettings() const + { + return settings; + } + void dumpConf(WriteBufferFromOwnString & buf) const; }; diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index ae11fa31a3a..602f9b03c88 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -440,7 +440,8 @@ bool KeeperTCPHandler::isHandShake(Int32 & handshake_length) bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(Int32 & four_letter_cmd) { - if (FourLetterCommandFactory::instance().isKnown(four_letter_cmd)) + if (FourLetterCommandFactory::instance().isKnown(four_letter_cmd) + && FourLetterCommandFactory::instance().isEnabled(four_letter_cmd)) { auto command = FourLetterCommandFactory::instance().get(four_letter_cmd); LOG_DEBUG(log, "receive four letter command {}", command->name()); diff --git a/tests/integration/test_keeper_four_word_command/configs/keeper_config.xml b/tests/integration/test_keeper_four_word_command/configs/keeper_config.xml deleted file mode 100644 index 2ad367b5bbf..00000000000 --- a/tests/integration/test_keeper_four_word_command/configs/keeper_config.xml +++ /dev/null @@ -1,24 +0,0 @@ - - - 9181 - 1 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - - - - 5000 - 10000 - trace - 75 - - - - - 1 - localhost - 44444 - - - - diff --git a/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list.xml b/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list.xml new file mode 100644 index 00000000000..4263cdb38c9 --- /dev/null +++ b/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list.xml @@ -0,0 +1,38 @@ + + + 9181 + 1 + ruok, conf + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + 3 + node3 + 44444 + + + + + + node1 + 9181 + + + node2 + 9181 + + + node3 + 9181 + + + diff --git a/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list_all.xml b/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list_all.xml new file mode 100644 index 00000000000..cb2b1dbd5b4 --- /dev/null +++ b/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list_all.xml @@ -0,0 +1,38 @@ + + + 9181 + 3 + * + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + 3 + node3 + 44444 + + + + + + node1 + 9181 + + + node2 + 9181 + + + node3 + 9181 + + + \ No newline at end of file diff --git a/tests/integration/test_keeper_four_word_command/configs/keeper_config_without_white_list.xml b/tests/integration/test_keeper_four_word_command/configs/keeper_config_without_white_list.xml new file mode 100644 index 00000000000..9abc1d2f6a7 --- /dev/null +++ b/tests/integration/test_keeper_four_word_command/configs/keeper_config_without_white_list.xml @@ -0,0 +1,37 @@ + + + 9181 + 2 + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + 3 + node3 + 44444 + + + + + + node1 + 9181 + + + node2 + 9181 + + + node3 + 9181 + + + \ No newline at end of file diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 07e5cbcdc25..751011f3594 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -255,6 +255,7 @@ def test_cmd_conf(started_cluster): assert "tcp_port_secure" not in result assert "superdigest" not in result + assert result["four_letter_word_white_list"] == "*" assert result["log_storage_path"] == "/var/lib/clickhouse/coordination/log" assert result["snapshot_storage_path"] == "/var/lib/clickhouse/coordination/snapshots" diff --git a/tests/integration/test_keeper_four_word_command/test_white_list.py b/tests/integration/test_keeper_four_word_command/test_white_list.py new file mode 100644 index 00000000000..edb30c25746 --- /dev/null +++ b/tests/integration/test_keeper_four_word_command/test_white_list.py @@ -0,0 +1,98 @@ +import socket +import pytest +from helpers.cluster import ClickHouseCluster +import time + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/keeper_config_with_white_list.xml'], stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=['configs/keeper_config_without_white_list.xml'], stay_alive=True) +node3 = cluster.add_instance('node3', main_configs=['configs/keeper_config_with_white_list_all.xml'], stay_alive=True) + +from kazoo.client import KazooClient, KazooState + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def destroy_zk_client(zk): + try: + if zk: + zk.stop() + zk.close() + except: + pass + + +def wait_node(node): + for _ in range(100): + zk = None + try: + node.query("SELECT * FROM system.zookeeper WHERE path = '/'") + zk = get_fake_zk(node.name, timeout=30.0) + # zk.create("/test", sequence=True) + print("node", node.name, "ready") + break + except Exception as ex: + time.sleep(0.2) + print("Waiting until", node.name, "will be ready, exception", ex) + finally: + destroy_zk_client(zk) + else: + raise Exception("Can't wait node", node.name, "to become ready") + + +def wait_nodes(): + for n in [node1, node2, node3]: + wait_node(n) + + +def get_keeper_socket(nodename): + hosts = cluster.get_instance_ip(nodename) + client = socket.socket() + client.settimeout(10) + client.connect((hosts, 9181)) + return client + + +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) + _fake_zk_instance.start() + return _fake_zk_instance + + +def close_keeper_socket(cli): + if cli is not None: + print("close socket") + cli.close() + + +def send_cmd(node_name, command = "ruok"): + client = None + try: + wait_nodes() + client = get_keeper_socket(node_name) + client.send(command.encode()) + data = client.recv(4) + return data.decode() + finally: + close_keeper_socket(client) + + +def test_white_list(started_cluster): + client = None + try: + wait_nodes() + assert send_cmd(node1.name) == 'imok' + assert send_cmd(node1.name, command = 'mntr') == '' + assert send_cmd(node2.name) == 'imok' + assert send_cmd(node3.name) == 'imok' + finally: + close_keeper_socket(client) From 715f98209674cc19d64086c2418593be4b0eaca4 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 29 Oct 2021 18:00:38 +0800 Subject: [PATCH 09/49] remove not used error code in FourLetterCommand.cpp --- src/Coordination/FourLetterCommand.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index ee0fcad331f..1b723ab2c6c 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -9,7 +9,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int INVALID_SETTING_VALUE; } IFourLetterCommand::IFourLetterCommand(const KeeperDispatcher & keeper_dispatcher_) : keeper_dispatcher(keeper_dispatcher_) From f24d4307737efcc70d46a3772638cf649f2dd222 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 1 Nov 2021 13:20:42 +0800 Subject: [PATCH 10/49] add sizeOf util function to SnapshotableHashTable --- src/Coordination/KeeperStorage.h | 14 +- src/Coordination/SnapshotableHashTable.h | 130 +++++++++++++++++- src/Coordination/tests/gtest_coordination.cpp | 64 +++++++++ 3 files changed, 192 insertions(+), 16 deletions(-) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index ed12c448f24..8774ba80a13 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -44,7 +44,7 @@ public: UInt64 size() const { UInt64 child_size{0}; - for (auto & child : children) + for (const auto & child : children) { child_size += child.size(); } @@ -210,17 +210,7 @@ public: UInt64 getApproximateDataSize() const { - UInt64 size{0}; - for (const auto & it : container) - { - size += (it.key.size() + it.value.size() + 1); - } - /// hash map key size - if (container.size()) - { - size += sizeof(StringRef) * container.size(); - } - return size; + return container.getApproximateSataSize(); } }; diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 454d1110196..499164ccd83 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -15,6 +15,19 @@ struct ListNode bool active_in_map; }; +template +struct HasSizeMethod +{ +private: + template + static auto check(int) -> decltype(std::declval().size(), std::true_type()); + template + static std::false_type check(...); + +public: + static constexpr bool value = std::is_same(0)), std::true_type>::value; +}; + template class SnapshotableHashTable { @@ -28,6 +41,96 @@ private: IndexMap map; bool snapshot_mode{false}; + UInt64 approximate_data_size{0}; + + enum OperationType + { + INSERT = 0, + INSERT_OR_REPLACE = 1, + ERASE = 2, + UPDATE_VALUE = 3, + GET_VALUE = 4, + FIND = 5, + CONTAINS = 6, + CLEAR = 7, + CLEAR_OUTDATED_NODES = 8 + }; + + /// Update hash table approximate data size + /// op_type: operation type + /// key_size: key size + /// value_size: size of value to add + /// old_value_size: size of value to minus + /// old_value_size=0 means there is no old value with the same key. + void inline updateDataSize(OperationType op_type, UInt64 key_size, UInt64 value_size, UInt64 old_value_size) + { + switch (op_type) + { + case INSERT: + approximate_data_size += key_size; + approximate_data_size += value_size; + break; + case INSERT_OR_REPLACE: + /// replace + if (old_value_size) + { + approximate_data_size += key_size; + approximate_data_size += value_size; + if(!snapshot_mode) + { + approximate_data_size += key_size; + approximate_data_size -= old_value_size; + } + } + /// inseert + else + { + approximate_data_size += key_size; + approximate_data_size += value_size; + } + break; + case UPDATE_VALUE: + approximate_data_size += key_size; + approximate_data_size += value_size; + if(!snapshot_mode) + { + approximate_data_size -= key_size; + approximate_data_size -= old_value_size; + } + break; + case ERASE: + if(!snapshot_mode) + { + approximate_data_size -= key_size; + approximate_data_size -= old_value_size; + } + break; + case CLEAR: + approximate_data_size = 0; + break; + case CLEAR_OUTDATED_NODES: + approximate_data_size -= key_size; + approximate_data_size -= value_size; + break; + default: + break; + } + } + + /// Calculate object memory size. + /// @return size(), if T has method size(), otherwise return sizeof(T) + template + inline UInt64 sizeOf(const typename std::enable_if::value, T>::type * obj) + { + return obj->size(); + } + + template + inline UInt64 sizeOf(const typename std::enable_if::value, T>::type *) + { + return sizeof(T); + } + public: using iterator = typename List::iterator; @@ -44,6 +147,7 @@ public: ListElem elem{key, value, true}; auto itr = list.insert(list.end(), elem); map.emplace(itr->key, itr); + updateDataSize(INSERT, sizeOf(&key), sizeOf(&value), 0); return true; } @@ -54,6 +158,8 @@ public: void insertOrReplace(const std::string & key, const V & value) { auto it = map.find(key); + UInt64 old_value_size = it == map.end() ? 0 : sizeOf(&it->second->value); + if (it == map.end()) { ListElem elem{key, value, true}; @@ -76,6 +182,7 @@ public: list_itr->value = value; } } + updateDataSize(INSERT_OR_REPLACE, sizeOf(&key), sizeOf(&value), old_value_size); } bool erase(const std::string & key) @@ -96,6 +203,7 @@ public: list.erase(list_itr); } + updateDataSize(ERASE, sizeOf(&key), 0, sizeOf(&list_itr->value)); return true; } @@ -108,23 +216,29 @@ public: { auto it = map.find(key); assert(it != map.end()); + + auto list_itr = it->second; + UInt64 old_value_size = sizeOf(&list_itr->value); + + const_iterator ret; + if (snapshot_mode) { - auto list_itr = it->second; auto elem_copy = *(list_itr); list_itr->active_in_map = false; map.erase(it); updater(elem_copy.value); auto itr = list.insert(list.end(), elem_copy); map.emplace(itr->key, itr); - return itr; + ret = itr; } else { - auto list_itr = it->second; updater(list_itr->value); - return list_itr; + ret = list_itr; } + updateDataSize(UPDATE_VALUE, sizeOf(&key), sizeOf(&ret->value), old_value_size); + return ret; } const_iterator find(const std::string & key) const @@ -149,7 +263,10 @@ public: for (auto itr = start; itr != end;) { if (!itr->active_in_map) + { + updateDataSize(CLEAR_OUTDATED_NODES, sizeOf(&itr->key), sizeOf(&itr->value), 0); itr = list.erase(itr); + } else itr++; } @@ -159,6 +276,7 @@ public: { list.clear(); map.clear(); + updateDataSize(CLEAR, 0, 0, 0); } void enableSnapshotMode() @@ -181,6 +299,10 @@ public: return list.size(); } + UInt64 getApproximateSataSize() const + { + return approximate_data_size; + } iterator begin() { return list.begin(); } const_iterator begin() const { return list.cbegin(); } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 12902252328..22a0dfcdbaa 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -923,6 +923,70 @@ TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot) map_snp.disableSnapshotMode(); } +TEST_P(CoordinationTest, SnapshotableHashMapDataSize) +{ + /// int + DB::SnapshotableHashTable hello; + hello.disableSnapshotMode(); + EXPECT_EQ(hello.getApproximateSataSize(), 0); + + hello.insert("hello", 1); + EXPECT_EQ(hello.getApproximateSataSize(), 9); + hello.updateValue("hello", [](int & value) { value = 2; }); + EXPECT_EQ(hello.getApproximateSataSize(), 9); + + hello.erase("hello"); + EXPECT_EQ(hello.getApproximateSataSize(), 0); + + hello.clear(); + EXPECT_EQ(hello.getApproximateSataSize(), 0); + + hello.enableSnapshotMode(); + hello.insert("hello", 1); + EXPECT_EQ(hello.getApproximateSataSize(), 9); + hello.updateValue("hello", [](int & value) { value = 2; }); + EXPECT_EQ(hello.getApproximateSataSize(), 18); + + hello.clearOutdatedNodes(); + EXPECT_EQ(hello.getApproximateSataSize(), 9); + + hello.erase("hello"); + EXPECT_EQ(hello.getApproximateSataSize(), 9); + + hello.clearOutdatedNodes(); + EXPECT_EQ(hello.getApproximateSataSize(), 0); + + /// Node + using Node = DB::KeeperStorage::Node; + DB::SnapshotableHashTable world; + Node n1; + n1.data = "1234"; + Node n2; + n2.data = "123456"; + + world.disableSnapshotMode(); + world.insert("world", n1); + /// 169 = 160 + 5 + 4 + /// 160 = sizeof Node + /// 5 = sizeof key + /// 4 = sizeof value + EXPECT_EQ(world.getApproximateSataSize(), 169); + world.updateValue("world", [&](Node & value) { value = n2; }); + EXPECT_EQ(world.getApproximateSataSize(), 171); + + world.clear(); + EXPECT_EQ(world.getApproximateSataSize(), 0); + + world.enableSnapshotMode(); + world.insert("world", n1); + EXPECT_EQ(world.getApproximateSataSize(), 169); + world.updateValue("world", [&](Node & value) { value = n2; }); + EXPECT_EQ(world.getApproximateSataSize(), 340); + + world.clearOutdatedNodes(); + EXPECT_EQ(world.getApproximateSataSize(), 171); +} + void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner=0) { using Node = DB::KeeperStorage::Node; From 89f7c7eef6135a82379408d22a618f966a2c17da Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 1 Nov 2021 13:26:25 +0800 Subject: [PATCH 11/49] fix check style error --- src/Coordination/SnapshotableHashTable.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 499164ccd83..d1ed7d822aa 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -76,7 +76,7 @@ private: { approximate_data_size += key_size; approximate_data_size += value_size; - if(!snapshot_mode) + if (!snapshot_mode) { approximate_data_size += key_size; approximate_data_size -= old_value_size; @@ -92,14 +92,14 @@ private: case UPDATE_VALUE: approximate_data_size += key_size; approximate_data_size += value_size; - if(!snapshot_mode) + if (!snapshot_mode) { approximate_data_size -= key_size; approximate_data_size -= old_value_size; } break; case ERASE: - if(!snapshot_mode) + if (!snapshot_mode) { approximate_data_size -= key_size; approximate_data_size -= old_value_size; From ec1ad60e9de3f4edf4ce64d886dcf6068716b9f4 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 2 Nov 2021 17:13:35 +0800 Subject: [PATCH 12/49] fix seg fault and add test --- src/Coordination/SnapshotableHashTable.h | 3 ++- src/Coordination/tests/gtest_coordination.cpp | 9 ++++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index d1ed7d822aa..42d1c297efb 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -192,6 +192,7 @@ public: return false; auto list_itr = it->second; + UInt64 old_data_size = sizeOf(&list_itr->value); if (snapshot_mode) { list_itr->active_in_map = false; @@ -203,7 +204,7 @@ public: list.erase(list_itr); } - updateDataSize(ERASE, sizeOf(&key), 0, sizeOf(&list_itr->value)); + updateDataSize(ERASE, sizeOf(&key), 0, old_data_size); return true; } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 22a0dfcdbaa..0b8c9809085 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -963,6 +963,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) n1.data = "1234"; Node n2; n2.data = "123456"; + n2.children.insert(""); world.disableSnapshotMode(); world.insert("world", n1); @@ -974,7 +975,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) world.updateValue("world", [&](Node & value) { value = n2; }); EXPECT_EQ(world.getApproximateSataSize(), 171); - world.clear(); + world.erase("world"); EXPECT_EQ(world.getApproximateSataSize(), 0); world.enableSnapshotMode(); @@ -985,6 +986,12 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) world.clearOutdatedNodes(); EXPECT_EQ(world.getApproximateSataSize(), 171); + + world.erase("world"); + EXPECT_EQ(world.getApproximateSataSize(), 171); + + world.clear(); + EXPECT_EQ(world.getApproximateSataSize(), 0); } void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner=0) From cc00c6993ef6c7f0c37b07c2045f460b5cac6933 Mon Sep 17 00:00:00 2001 From: nautaa <870284156@qq.com> Date: Sat, 6 Nov 2021 21:00:24 +0800 Subject: [PATCH 13/49] remove unnecessary assert --- src/Storages/FileLog/StorageFileLog.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 1ba16fc821c..c33ca7e1451 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -470,7 +470,6 @@ void StorageFileLog::openFilesAndSetPos() void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) { - assert(start >= 0); assert(start < end); assert(end <= file_infos.file_names.size()); @@ -491,7 +490,6 @@ void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) void StorageFileLog::storeMetas(size_t start, size_t end) { - assert(start >= 0); assert(start < end); assert(end <= file_infos.file_names.size()); From c71fb3337fcc701c4186fe076c9af835f12ea50d Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 5 Nov 2021 18:21:34 +0800 Subject: [PATCH 14/49] add more 4lwd commands --- src/Coordination/CoordinationSettings.cpp | 5 +- src/Coordination/CoordinationSettings.h | 1 + src/Coordination/FourLetterCommand.cpp | 248 ++++++++-- src/Coordination/FourLetterCommand.h | 143 +++++- src/Coordination/KeeperDispatcher.cpp | 119 ++--- src/Coordination/KeeperDispatcher.h | 58 +-- src/Coordination/KeeperInfos.h | 73 ++- src/Coordination/KeeperStateMachine.cpp | 30 ++ src/Coordination/KeeperStateMachine.h | 6 + src/Coordination/KeeperStats.cpp | 97 ++++ src/Coordination/KeeperStats.h | 55 +++ src/Coordination/KeeperStorage.cpp | 59 +++ src/Coordination/KeeperStorage.h | 23 + src/Server/KeeperTCPHandler.cpp | 183 +++++++- src/Server/KeeperTCPHandler.h | 48 +- src/Server/KeeperTCPHandlers.cpp | 25 ++ src/Server/KeeperTCPHandlers.h | 18 + .../configs/enable_keeper1.xml | 3 +- .../configs/enable_keeper2.xml | 3 +- .../configs/enable_keeper3.xml | 3 +- .../configs/use_keeper.xml | 4 - .../test_keeper_four_word_command/test.py | 422 +++++++++++++++--- 22 files changed, 1378 insertions(+), 248 deletions(-) create mode 100644 src/Coordination/KeeperStats.cpp create mode 100644 src/Coordination/KeeperStats.h create mode 100644 src/Server/KeeperTCPHandlers.cpp create mode 100644 src/Server/KeeperTCPHandlers.h diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 382d37e9ed8..2370e16c794 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -34,6 +34,9 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco } } + +const String KeeperSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro"; + KeeperSettings::KeeperSettings() : server_id(NOT_EXIST) , tcp_port(NOT_EXIST) @@ -167,7 +170,7 @@ KeeperSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, ret->super_digest = config.getString("keeper_server.superdigest"); } - ret->four_letter_word_white_list = config.getString("keeper_server.four_letter_word_white_list", "*"); + ret->four_letter_word_white_list = config.getString("keeper_server.four_letter_word_white_list", DEFAULT_FOUR_LETTER_WORD_CMD); ret->log_storage_path = getLogsPathFromConfig(config, standalone_keeper_); ret->snapshot_storage_path = getSnapshotsPathFromConfig(config, standalone_keeper_); diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 29162587813..9289f7aaeba 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -55,6 +55,7 @@ using CoordinationSettingsPtr = std::shared_ptr; struct KeeperSettings { static constexpr int NOT_EXIST = -1; + static const String DEFAULT_FOUR_LETTER_WORD_CMD; KeeperSettings(); int server_id; diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 1b723ab2c6c..1043d1ecffc 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -1,8 +1,14 @@ +#include #include #include +#include +#include +#include +#include #include #include #include +#include namespace DB { @@ -93,12 +99,45 @@ void FourLetterCommandFactory::registerCommands(const KeeperDispatcher & keeper_ FourLetterCommandPtr mntr_command = std::make_shared(keeper_dispatcher); factory.registerCommand(mntr_command); - FourLetterCommandPtr srst_command = std::make_shared(keeper_dispatcher); - factory.registerCommand(srst_command); - FourLetterCommandPtr conf_command = std::make_shared(keeper_dispatcher); factory.registerCommand(conf_command); + FourLetterCommandPtr cons_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(cons_command); + + FourLetterCommandPtr brief_watch_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(brief_watch_command); + + FourLetterCommandPtr data_size_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(data_size_command); + + FourLetterCommandPtr dump_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(dump_command); + + FourLetterCommandPtr envi_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(envi_command); + + FourLetterCommandPtr is_rad_only_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(is_rad_only_command); + + FourLetterCommandPtr rest_conn_stats_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(rest_conn_stats_command); + + FourLetterCommandPtr server_stat_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(server_stat_command); + + FourLetterCommandPtr stat_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(stat_command); + + FourLetterCommandPtr stat_reset_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(stat_reset_command); + + FourLetterCommandPtr watch_by_path_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(watch_by_path_command); + + FourLetterCommandPtr watch_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(watch_command); + factory.initializeWhiteList(keeper_dispatcher); factory.setInitialize(true); } @@ -146,23 +185,11 @@ void FourLetterCommandFactory::initializeWhiteList(const KeeperDispatcher & keep } } -String RuokCommand::name() -{ - return "ruok"; -} - String RuokCommand::run() { return "imok"; } -RuokCommand::~RuokCommand() = default; - -String MonitorCommand::name() -{ - return "mntr"; -} - String MonitorCommand::run() { KeeperStatsPtr stats = keeper_dispatcher.getKeeperStats(); @@ -191,7 +218,7 @@ String MonitorCommand::run() print(ret, "znode_count", state_machine.getNodeCount()); print(ret, "watch_count", state_machine.getWatchCount()); - print(ret, "ephemerals_count", state_machine.getEphemeralCount()); + print(ret, "ephemerals_count", state_machine.getEphemeralNodeCount()); print(ret, "approximate_data_size", state_machine.getApproximateDataSize()); #if defined(__linux__) || defined(__APPLE__) @@ -203,11 +230,9 @@ String MonitorCommand::run() { print(ret, "followers", raft_info.getFollowerCount()); print(ret, "synced_followers", raft_info.getSyncedFollowerCount()); - /// TODO implementation /// print(ret, "pending_syncs", 0); } - /// TODO Maybe the next 3 metrics are useless. /// print(ret, "last_proposal_size", -1); /// print(ret, "max_proposal_size", -1); /// print(ret, "min_proposal_size", -1); @@ -233,13 +258,6 @@ void MonitorCommand::print(IFourLetterCommand::StringBuffer & buf, const String print(buf, key, std::to_string(value)); } -MonitorCommand::~MonitorCommand() = default; - -String StatResetCommand::name() -{ - return "srst"; -} - String StatResetCommand::run() { KeeperStatsPtr stats = keeper_dispatcher.getKeeperStats(); @@ -247,23 +265,9 @@ String StatResetCommand::run() return "Server stats reset."; } -StatResetCommand::~StatResetCommand() = default; - -String NopCommand::name() -{ - return "nopc"; -} - String NopCommand::run() { - return DB::String(); -} - -NopCommand::~NopCommand() = default; - -String ConfCommand::name() -{ - return "conf"; + return ""; } String ConfCommand::run() @@ -273,6 +277,168 @@ String ConfCommand::run() return buf.str(); } -ConfCommand::~ConfCommand() = default; +String ConsCommand::run() +{ + StringBuffer buf; + KeeperTCPHandler::dumpConnections(buf, false); + return buf.str(); +} + +String RestConnStatsCommand::run() +{ + KeeperTCPHandler::resetConnsStats(); + return "Connection stats reset."; +} + +String ServerStatCommand::run() +{ + using std::to_string; + StringBuffer buf; + + auto write = [&buf](const String & key, const String & value) { + buf.write(key.data(), key.size()); + buf.write(": ", 2); + buf.write(value.data(), value.size()); + buf.write('\n'); + }; + + KeeperStatsPtr stats = keeper_dispatcher.getKeeperStats(); + const IKeeperInfo & keeper_info = keeper_dispatcher.getKeeperInfo(); + const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); + + write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); + + StringBuffer latency; + latency << stats->getMinLatency() << "/" << stats->getAvgLatency() << "/" << stats->getMaxLatency() << "\n"; + write("Latency min/avg/max", latency.str()); + + write("Received", to_string(stats->getPacketsReceived())); + write("Sent ", to_string(stats->getPacketsSent())); + write("Connections", to_string(keeper_info.getNumAliveConnections())); + write("Outstanding", to_string(keeper_info.getOutstandingRequests())); + write("Zxid", to_string(state_machine.getLastProcessedZxid())); + write("Mode", keeper_info.getRole()); + write("Node count", to_string(state_machine.getNodeCount())); + + return buf.str(); +} + +String StatCommand::run() +{ + using std::to_string; + StringBuffer buf; + + auto write = [&buf](const String & key, const String & value) { buf << key << ": " << value << '\n'; }; + + KeeperStatsPtr stats = keeper_dispatcher.getKeeperStats(); + const IKeeperInfo & keeper_info = keeper_dispatcher.getKeeperInfo(); + const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); + + write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); + + buf << "Clients:\n"; + KeeperTCPHandler::dumpConnections(buf, true); + buf << '\n'; + + StringBuffer latency; + latency << stats->getMinLatency() << "/" << stats->getAvgLatency() << "/" << stats->getMaxLatency() << "\n"; + write("Latency min/avg/max", latency.str()); + + write("Received", to_string(stats->getPacketsReceived())); + write("Sent ", to_string(stats->getPacketsSent())); + write("Connections", to_string(keeper_info.getNumAliveConnections())); + write("Outstanding", to_string(keeper_info.getOutstandingRequests())); + write("Zxid", to_string(state_machine.getLastProcessedZxid())); + write("Mode", keeper_info.getRole()); + write("Node count", to_string(state_machine.getNodeCount())); + + return buf.str(); +} + +String BriefWatchCommand::run() +{ + StringBuffer buf; + const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); + buf << keeper_dispatcher.getNumAliveConnections() << " connections watching " << state_machine.getWatchPathCount() << " paths\n"; + buf << "Total watches:" << state_machine.getWatchCount(); + return buf.str(); +} + +String WatchCommand::run() +{ + StringBuffer buf; + const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); + state_machine.dumpWatches(buf); + return buf.str(); +} + +String WatchByPathCommand::run() +{ + StringBuffer buf; + const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); + state_machine.dumpWatchesByPath(buf); + return buf.str(); +} + +String DataSizeCommand::run() +{ + StringBuffer buf; + buf << "datadir_size: " << keeper_dispatcher.getSnapDirSize() << '\n'; + buf << "logdir_size: " << keeper_dispatcher.getDataDirSize() << '\n'; + return buf.str(); +} + +String DumpCommand::run() +{ + StringBuffer buf; + const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); + keeper_dispatcher.dumpSessions(buf); + state_machine.dumpEphemerals(buf); + return buf.str(); +} + +String EnviCommand::run() +{ + using Poco::Environment; + using Poco::Path; + + StringBuffer buf; + buf << "Environment:\n"; + buf << "clickhouse.keeper.version=" << (String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH) << '\n'; + + buf << "host.name" << Environment::nodeName() << '\n'; + buf << "os.name" << Environment::osDisplayName() << '\n'; + buf << "os.arch" << Environment::osArchitecture() << '\n'; + buf << "os.version" << Environment::osVersion() << '\n'; + buf << "cpu.count" << Environment::processorCount() << '\n'; + + try + { + buf << "ip.address" << Environment::nodeId() << '\n'; + } + catch (...) + { + Poco::Logger * log = &Poco::Logger::get("EnviCommand"); + LOG_WARNING(log, "Can not get server ip address."); + } + + char user_name[128]; + getlogin_r(user_name, 128); + buf << "user.name" << user_name << '\n'; + + buf << "user.home" << Path::home() << '\n'; + buf << "user.dir" << Path::current() << '\n'; + buf << "user.tmp" << Path::temp() << '\n'; + + return buf.str(); +} + +String IsReadOnlyCommand::run() +{ + if (keeper_dispatcher.getRole() == "observer") + return "ro"; + else + return "rw"; +} } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index b3023027dab..861c9d54619 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -82,9 +82,9 @@ struct RuokCommand : public IFourLetterCommand { explicit RuokCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } - String name() override; + String name() override { return "ruok"; } String run() override; - ~RuokCommand() override; + ~RuokCommand() override = default; }; /**Outputs a list of variables that could be used for monitoring the health of the cluster. @@ -112,9 +112,10 @@ struct MonitorCommand : public IFourLetterCommand { explicit MonitorCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } - String name() override; + String name() override { return "mntr"; } String run() override; - ~MonitorCommand() override; + ~MonitorCommand() override = default; + private: static void print(StringBuffer & buf, const String & key, const String & value); static void print(StringBuffer & buf, const String & key, UInt64 value); @@ -124,30 +125,146 @@ struct StatResetCommand : public IFourLetterCommand { explicit StatResetCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } - String name() override; + String name() override { return "srst"; } String run() override; - ~StatResetCommand() override; + ~StatResetCommand() override = default; }; -/** A command that does not do anything except reply to client with predefined message. - * It is used to inform clients who execute none white listed four letter word commands. - */ +/// A command that does not do anything except reply to client with predefined message. +///It is used to inform clients who execute none white listed four letter word commands. struct NopCommand : public IFourLetterCommand { explicit NopCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } - String name() override; + String name() override { return "nopc"; } String run() override; - ~NopCommand() override; + ~NopCommand() override = default; }; struct ConfCommand : public IFourLetterCommand { explicit ConfCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } - String name() override; + String name() override { return "conf"; } String run() override; - ~ConfCommand() override; + ~ConfCommand() override = default; }; +/// List full connection/session details for all clients connected to this server. +/// Includes information on numbers of packets received/sent, session id, operation latencies, last operation performed, etc... +struct ConsCommand : public IFourLetterCommand +{ + explicit ConsCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override { return "cons"; } + String run() override; + ~ConsCommand() override = default; +}; + +/// Reset connection/session statistics for all connections. +struct RestConnStatsCommand : public IFourLetterCommand +{ + explicit RestConnStatsCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override { return "crst"; } + String run() override; + ~RestConnStatsCommand() override = default; +}; + +/// Lists full details for the server. +struct ServerStatCommand : public IFourLetterCommand +{ + explicit ServerStatCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override { return "srvr"; } + String run() override; + ~ServerStatCommand() override = default; +}; + +/// Lists brief details for the server and connected clients. +struct StatCommand : public IFourLetterCommand +{ + explicit StatCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override { return "stat"; } + String run() override; + ~StatCommand() override = default; +}; + +/// Lists brief information on watches for the server. +struct BriefWatchCommand : public IFourLetterCommand +{ + explicit BriefWatchCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override { return "wchs"; } + String run() override; + ~BriefWatchCommand() override = default; +}; + +/// Lists detailed information on watches for the server, by session. +/// This outputs a list of sessions(connections) with associated watches (paths). +/// Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. +struct WatchCommand : public IFourLetterCommand +{ + explicit WatchCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override { return "wchc"; } + String run() override; + ~WatchCommand() override = default; +}; + +/// Lists detailed information on watches for the server, by path. +/// This outputs a list of paths (znodes) with associated sessions. +/// Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. +struct WatchByPathCommand : public IFourLetterCommand +{ + explicit WatchByPathCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override { return "wchp"; } + String run() override; + ~WatchByPathCommand() override = default; +}; + +/// Lists the outstanding sessions and ephemeral nodes. This only works on the leader. +struct DumpCommand : public IFourLetterCommand +{ + explicit DumpCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override { return "dump"; } + String run() override; + ~DumpCommand() override = default; +}; + +/// Print details about serving environment +struct EnviCommand : public IFourLetterCommand +{ + explicit EnviCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override { return "envi"; } + String run() override; + ~EnviCommand() override = default; +}; + +/// Shows the total size of snapshot and log files in bytes +struct DataSizeCommand : public IFourLetterCommand +{ + explicit DataSizeCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override { return "dirs"; } + String run() override; + ~DataSizeCommand() override = default; +}; + +/// Tests if server is running in read-only mode. +/// The server will respond with "ro" if in read-only mode or "rw" if not in read-only mode. +struct IsReadOnlyCommand : public IFourLetterCommand +{ + explicit IsReadOnlyCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + + String name() override { return "isro"; } + String run() override; + ~IsReadOnlyCommand() override = default; +}; + + } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index ad117f216f1..6f3531471d7 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -3,6 +3,10 @@ #include #include #include +#include +#include +#include +#include namespace DB { @@ -14,89 +18,25 @@ namespace ErrorCodes extern const int SYSTEM_ERROR; } -UInt64 KeeperDispatcher::KeeperStats::getMinLatency() const -{ - std::shared_lock lock(mutex); - return min_latency; -} +using Poco::Path; +using Poco::File; +using Poco::DirectoryIterator; -UInt64 KeeperDispatcher::KeeperStats::getMaxLatency() const +UInt64 getDirSize(Path dir) { - std::shared_lock lock(mutex); - return max_latency; -} + DirectoryIterator it(dir); + DirectoryIterator end; -UInt64 KeeperDispatcher::KeeperStats::getAvgLatency() const -{ - std::shared_lock lock(mutex); - if (count != 0) + UInt64 size{0}; + while (it != end) { - return total_latency / count; + if (it->isFile()) + size += it->getSize(); + else + size += getDirSize(it->path()); + ++it; } - return 0; -} - -UInt64 KeeperDispatcher::KeeperStats::getPacketsReceived() const -{ - std::shared_lock lock(mutex); - return packets_received; -} - -UInt64 KeeperDispatcher::KeeperStats::getPacketsSent() const -{ - std::shared_lock lock(mutex); - return packets_sent; -} - -void KeeperDispatcher::KeeperStats::incrementPacketsReceived() -{ - std::unique_lock lock(mutex); - packets_received++; -} - -void KeeperDispatcher::KeeperStats::incrementPacketsSent() -{ - std::unique_lock lock(mutex); - packets_sent++; -} - -void KeeperDispatcher::KeeperStats::updateLatency(UInt64 latency_ms) -{ - std::unique_lock lock(mutex); - - total_latency += (latency_ms); - count++; - - if (latency_ms < min_latency) - { - min_latency = latency_ms; - } - - if (latency_ms > max_latency) - { - max_latency = latency_ms; - } -} - -void KeeperDispatcher::KeeperStats::reset() -{ - std::unique_lock lock(mutex); - resetLatency(); - resetRequestCounters(); -} - -void KeeperDispatcher::KeeperStats::resetLatency() -{ - total_latency = 0; - count = 0; - max_latency = 0; - min_latency = 0; -} - -void KeeperDispatcher::KeeperStats::resetRequestCounters() -{ - packets_received = 0; - packets_sent = 0; + return size; } KeeperDispatcher::KeeperDispatcher() @@ -302,7 +242,6 @@ void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKe session_to_response_callback.erase(session_response_callback); } } - keeper_stats->incrementPacketsSent(); } bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) @@ -333,7 +272,6 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ { throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); } - keeper_stats->incrementPacketsReceived(); return true; } @@ -684,9 +622,30 @@ UInt64 KeeperDispatcher::getNumAliveConnections() const return session_to_response_callback.size(); } +UInt64 KeeperDispatcher::getDataDirSize() const +{ + return getDirSize(Path(settings->log_storage_path)); +} + +UInt64 KeeperDispatcher::getSnapDirSize() const +{ + return getDirSize(Path(settings->snapshot_storage_path)); +} + void KeeperDispatcher::dumpConf(WriteBufferFromOwnString & buf) const { settings->dump(buf); } +void KeeperDispatcher::dumpSessions(WriteBufferFromOwnString & buf) const +{ + std::lock_guard lock(session_to_response_callback_mutex); + buf << "Sessions dump (" << session_to_response_callback.size() << "):\n"; + + for (const auto & e : session_to_response_callback) + { + buf << "0x" << getHexUIntLowercase(e.first) << "\n"; + } +} + } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 87f8d23aafc..7ac5af2ebc4 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -25,49 +25,6 @@ using ZooKeeperResponseCallback = std::function getKeeperStats() const { @@ -224,10 +184,20 @@ public: return settings; } + inline void incrementPacketsSent() + { + keeper_stats->incrementPacketsSent(); + } + + inline void incrementPacketsReceived() + { + keeper_stats->incrementPacketsReceived(); + } + void dumpConf(WriteBufferFromOwnString & buf) const; + void dumpSessions(WriteBufferFromOwnString & buf) const override; }; -using KeeperStatsPtr = std::shared_ptr; } #endif diff --git a/src/Coordination/KeeperInfos.h b/src/Coordination/KeeperInfos.h index 5a47fd9ded7..f809692c4fc 100644 --- a/src/Coordination/KeeperInfos.h +++ b/src/Coordination/KeeperInfos.h @@ -1,10 +1,11 @@ #pragma once -#include #include #include #include +#include #include +#include #include @@ -12,6 +13,58 @@ namespace DB { +struct LastOp +{ +private: + String last_op{"NA"}; + Int64 last_cxid{-1}; + Int64 last_zxid{-1}; + Int64 last_response_time{0}; + Int64 last_latency{0}; + +public: + inline void update(String last_op_, Int64 last_cxid_, Int64 last_zxid_, Int64 last_response_time_, Int64 last_latency_) + { + last_op = last_op_; + last_cxid = last_cxid_; + last_zxid = last_zxid_; + last_response_time = last_response_time_; + last_latency = last_latency_; + } + + inline LastOp clone() { return *this; } + + inline void reset() + { + last_op = "NA"; + last_cxid = -1; + last_zxid = -1; + last_response_time = 0; + last_latency = 0; + } + + inline const String & getLastOp() const { return last_op; } + inline Int64 getLastCxid() const { return last_cxid; } + inline Int64 getLastZxid() const { return last_zxid; } + inline Int64 getLastResponseTime() const { return last_response_time; } + inline Int64 getLastLatency() const { return last_latency; } +}; + +class IConnectionInfo +{ +public: + virtual Int64 getPacketsReceived() const = 0; + virtual Int64 getPacketsSent() const = 0; + virtual Int64 getSessionId() const = 0; + virtual Int64 getSessionTimeout() const = 0; + /// startup time + virtual Poco::Timestamp getEstablished() const = 0; + virtual LastOp getLastOp() const = 0; + virtual KeeperStatsPtr getSessionStats() const = 0; + + virtual ~IConnectionInfo() = default; +}; + /// Keeper server related information class IKeeperInfo { @@ -27,6 +80,15 @@ public: /// number of requests in queue virtual UInt64 getOutstandingRequests() const = 0; + + /// log dir size + virtual UInt64 getDataDirSize() const = 0; + /// snapshot dir size + virtual UInt64 getSnapDirSize() const = 0; + + /// dump session list connection to the node + virtual void dumpSessions(WriteBufferFromOwnString & buf) const = 0; + virtual ~IKeeperInfo() = default; }; @@ -35,16 +97,23 @@ class IStateMachineInfo { public: /// last committed zxid - [[maybe_unused]] virtual UInt64 getLastProcessedZxid() const = 0; + virtual UInt64 getLastProcessedZxid() const = 0; virtual UInt64 getNodeCount() const = 0; virtual UInt64 getWatchCount() const = 0; + virtual UInt64 getWatchPathCount() const = 0; + /// session count who has ephemeral nodes virtual UInt64 getEphemeralCount() const = 0; + virtual UInt64 getEphemeralNodeCount() const = 0; /// state machine approximate data size virtual UInt64 getApproximateDataSize() const = 0; virtual std::vector getDeadSessions() = 0; + virtual void dumpWatches(WriteBufferFromOwnString & buf) const = 0; + virtual void dumpWatchesByPath(WriteBufferFromOwnString & buf) const = 0; + virtual void dumpEphemerals(WriteBufferFromOwnString & buf) const = 0; + virtual ~IStateMachineInfo() = default; }; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 97da1e84f5c..0938b60a58d 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -356,12 +356,42 @@ UInt64 KeeperStateMachine::getWatchCount() const return storage->getWatchCount(); } +UInt64 KeeperStateMachine::getWatchPathCount() const +{ + std::lock_guard lock(storage_and_responses_lock); + return storage->getWatchPathCount(); +} + UInt64 KeeperStateMachine::getEphemeralCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getEphemeralCount(); } +UInt64 KeeperStateMachine::getEphemeralNodeCount() const +{ + std::lock_guard lock(storage_and_responses_lock); + return storage->getEphemeralNodeCount(); +} + +void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const +{ + std::lock_guard lock(storage_and_responses_lock); + storage->dumpWatches(buf); +} + +void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const +{ + std::lock_guard lock(storage_and_responses_lock); + storage->dumpWatchesByPath(buf); +} + +void KeeperStateMachine::dumpEphemerals(WriteBufferFromOwnString & buf) const +{ + std::lock_guard lock(storage_and_responses_lock); + storage->dumpEphemerals(buf); +} + UInt64 KeeperStateMachine::getApproximateDataSize() const { std::lock_guard lock(storage_and_responses_lock); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 5801389c6e8..f353ad8c8be 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -82,8 +82,14 @@ public: UInt64 getNodeCount() const override; UInt64 getWatchCount() const override; + UInt64 getWatchPathCount() const override; + + void dumpWatches(WriteBufferFromOwnString & buf) const override; + void dumpWatchesByPath(WriteBufferFromOwnString & buf) const override; + void dumpEphemerals(WriteBufferFromOwnString & buf) const override; UInt64 getEphemeralCount() const override; + UInt64 getEphemeralNodeCount() const override; UInt64 getApproximateDataSize() const override; void shutdownStorage(); diff --git a/src/Coordination/KeeperStats.cpp b/src/Coordination/KeeperStats.cpp new file mode 100644 index 00000000000..e676eedf77a --- /dev/null +++ b/src/Coordination/KeeperStats.cpp @@ -0,0 +1,97 @@ +#include + +namespace DB +{ +UInt64 KeeperStats::getMinLatency() const +{ + std::shared_lock lock(mutex); + return min_latency; +} + +UInt64 KeeperStats::getMaxLatency() const +{ + std::shared_lock lock(mutex); + return max_latency; +} + +UInt64 KeeperStats::getAvgLatency() const +{ + std::shared_lock lock(mutex); + if (count != 0) + { + return total_latency / count; + } + return 0; +} + +UInt64 KeeperStats::getLastLatency() const +{ + std::shared_lock lock(mutex); + return last_latency; +} + +UInt64 KeeperStats::getPacketsReceived() const +{ + std::shared_lock lock(mutex); + return packets_received; +} + +UInt64 KeeperStats::getPacketsSent() const +{ + std::shared_lock lock(mutex); + return packets_sent; +} + +void KeeperStats::incrementPacketsReceived() +{ + std::unique_lock lock(mutex); + packets_received++; +} + +void KeeperStats::incrementPacketsSent() +{ + std::unique_lock lock(mutex); + packets_sent++; +} + +void KeeperStats::updateLatency(UInt64 latency_ms) +{ + std::unique_lock lock(mutex); + + last_latency = latency_ms; + total_latency += (latency_ms); + count++; + + if (latency_ms < min_latency) + { + min_latency = latency_ms; + } + + if (latency_ms > max_latency) + { + max_latency = latency_ms; + } +} + +void KeeperStats::reset() +{ + std::unique_lock lock(mutex); + resetLatency(); + resetRequestCounters(); +} + +void KeeperStats::resetLatency() +{ + total_latency = 0; + count = 0; + max_latency = 0; + min_latency = 0; +} + +void KeeperStats::resetRequestCounters() +{ + packets_received = 0; + packets_sent = 0; +} + +} diff --git a/src/Coordination/KeeperStats.h b/src/Coordination/KeeperStats.h new file mode 100644 index 00000000000..3cf4af72029 --- /dev/null +++ b/src/Coordination/KeeperStats.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include + +namespace DB +{ +class KeeperStats; +using KeeperStatsPtr = std::shared_ptr; + +/// Request statistics for connection or dispatcher +class KeeperStats +{ +public: + explicit KeeperStats() = default; + + UInt64 getMinLatency() const; + UInt64 getMaxLatency() const; + + UInt64 getAvgLatency() const; + UInt64 getLastLatency() const; + + UInt64 getPacketsReceived() const; + UInt64 getPacketsSent() const; + + void incrementPacketsReceived(); + void incrementPacketsSent(); + + void updateLatency(UInt64 latency_ms); + void reset(); + +private: + void inline resetLatency(); + void inline resetRequestCounters(); + + mutable std::shared_mutex mutex; + + /// all response with watch response included + UInt64 packets_sent = 0; + /// All user requests + UInt64 packets_received = 0; + + /// For consistent with zookeeper measured by millisecond, + /// otherwise maybe microsecond is better + UInt64 total_latency = 0; + UInt64 max_latency = 0; + UInt64 min_latency = 0; + + /// last operation latency + UInt64 last_latency = 0; + + UInt64 count = 0; +}; + +} diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index b6938da958b..b3dffc89dda 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB { @@ -1220,4 +1221,62 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) } } +void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const +{ + auto write_str_set = [&buf](const std::unordered_set & objs) + { + for (const String & obj : objs) + { + buf << "\t" << obj << "\n"; + } + }; + + for (const auto & e : sessions_and_watchers) + { + buf << "0x" << getHexUIntLowercase(e.first) << "\n"; + write_str_set(e.second); + } +} + +void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const +{ + auto write_int_vec = [&buf](const std::vector & objs) + { + for (Int64 obj : objs) + { + buf << "\t0x" << getHexUIntLowercase(obj) << "\n"; + } + }; + + for (const auto & e : watches) + { + buf << e.first << "\n"; + write_int_vec(e.second); + } + + for (const auto & e : list_watches) + { + buf << e.first << "\n"; + write_int_vec(e.second); + } +} + +void KeeperStorage::dumpEphemerals(WriteBufferFromOwnString & buf) const +{ + auto write_str_set = [&buf](const std::unordered_set & objs) + { + for (const String & obj : objs) + { + buf << "\t" << obj << "\n"; + } + }; + + buf << "Sessions with Ephemerals (" << getEphemeralCount() << "):\n"; + for (const auto & e : ephemerals) + { + buf << "0x" << getHexUIntLowercase(e.first) << "\n"; + write_str_set(e.second); + } +} + } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 8774ba80a13..8c2f04e90e9 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -198,7 +198,26 @@ public: return watches.size() + list_watches.size(); } + UInt64 getWatchPathCount() const + { + UInt64 ret{0}; + for (const auto & watch : watches) + { + ret += watch.second.size(); + } + for (const auto & watch : list_watches) + { + ret += watch.second.size(); + } + return ret; + } + UInt64 getEphemeralCount() const + { + return ephemerals.size(); + } + + UInt64 getEphemeralNodeCount() const { UInt64 ret{0}; for (const auto & ephs : ephemerals) @@ -208,6 +227,10 @@ public: return ret; } + void dumpWatches(WriteBufferFromOwnString & buf) const; + void dumpWatchesByPath(WriteBufferFromOwnString & buf) const; + void dumpEphemerals(WriteBufferFromOwnString & buf) const; + UInt64 getApproximateDataSize() const { return container.getApproximateSataSize(); diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 602f9b03c88..5b066d47e71 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -19,6 +19,8 @@ #include #include #include +#include + #ifdef POCO_HAVE_FD_EPOLL #include @@ -200,7 +202,10 @@ KeeperTCPHandler::KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSoc , session_timeout(0, global_context->getConfigRef().getUInt("keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) , poll_wrapper(std::make_unique(socket_)) , responses(std::make_unique(std::numeric_limits::max())) + , established(Poco::Timestamp()) + , conn_stats(std::make_shared()) { + KeeperTCPHandler::registerConnection(this); } void KeeperTCPHandler::sendHandshake(bool has_leader) @@ -351,7 +356,6 @@ void KeeperTCPHandler::runImpl() session_stopwatch.start(); bool close_received = false; - Stopwatch process_time_stopwatch; try { @@ -362,8 +366,8 @@ void KeeperTCPHandler::runImpl() PollResult result = poll_wrapper->poll(session_timeout, in); if (result.has_requests && !close_received) { - process_time_stopwatch.start(); auto [received_op, received_xid] = receiveRequest(); + packageReceived(); if (received_op == Coordination::OpNum::Close) { @@ -375,18 +379,13 @@ void KeeperTCPHandler::runImpl() { LOG_TRACE(log, "Received heartbeat for session #{}", session_id); } + else + operations[received_xid] = Poco::Timestamp(); /// Each request restarts session stopwatch session_stopwatch.restart(); } - /// Do request statistics, - /// not accurate when there is watch response in the channel - if (result.responses_count != 0) - { - process_time_stopwatch.stop(); - keeper_dispatcher->updateKeeperStat(process_time_stopwatch.elapsedMilliseconds()); - } /// Process exact amount of responses from pipe /// otherwise state of responses queue and signaling pipe /// became inconsistent and race condition is possible. @@ -403,6 +402,9 @@ void KeeperTCPHandler::runImpl() return; } + updateStats(response); + packageSent(); + response->write(*out); if (response->error == Coordination::Error::ZSESSIONEXPIRED) { @@ -493,6 +495,169 @@ std::pair KeeperTCPHandler::receiveReque return std::make_pair(opnum, xid); } +void KeeperTCPHandler::packageSent() +{ + conn_stats->incrementPacketsSent(); + keeper_dispatcher->incrementPacketsSent(); +} + +void KeeperTCPHandler::packageReceived() +{ + conn_stats->incrementPacketsReceived(); + keeper_dispatcher->incrementPacketsReceived(); +} + +void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response) +{ + /// update statistics ignoring watch response and heartbeat. + if (response->xid != Coordination::WATCH_XID && response->getOpNum() != Coordination::OpNum::Heartbeat) + { + Int64 elapsed = (Poco::Timestamp() - operations[response->xid]) / 1000; + conn_stats->updateLatency(elapsed); + keeper_dispatcher->updateKeeperStat(elapsed); + + { + std::lock_guard lock(last_op_mutex); + last_op.update( + Coordination::toString(response->getOpNum()), + response->xid, + response->zxid, + Poco::Timestamp().epochMicroseconds() / 1000, + elapsed); + } + } +} + +void KeeperTCPHandler::dumpStats(WriteBufferFromOwnString & buf, bool brief) +{ + auto write_str = [&buf](const String & str) { buf.write(str.data(), str.size()); }; + using std::to_string; + + buf.write(' '); + write_str(socket().peerAddress().toString()); + write_str("(recved="); + write_str(to_string(conn_stats->getPacketsReceived())); + write_str(",sent="); + write_str(to_string(conn_stats->getPacketsSent())); + if (!brief) + { + if (session_id != 0) + { + write_str(",sid=0x"); + write_str(getHexUIntLowercase(getSessionId())); + + write_str(",lop="); + LastOp op; + { + std::lock_guard lock(last_op_mutex); + op = last_op.clone(); + } + write_str(op.getLastOp()); + write_str(",est="); + write_str(to_string(getEstablished().epochMicroseconds() / 1000)); + write_str(",to="); + write_str(to_string(getSessionTimeout())); + Int64 last_cxid = op.getLastCxid(); + if (last_cxid >= 0) + { + write_str(",lcxid=0x"); + write_str(getHexUIntLowercase(last_cxid)); + } + write_str(",lzxid=0x"); + write_str(getHexUIntLowercase(op.getLastZxid())); + write_str(",lresp="); + write_str(to_string(op.getLastResponseTime())); + + write_str(",llat="); + write_str(to_string(conn_stats->getLastLatency())); + write_str(",minlat="); + write_str(to_string(conn_stats->getMinLatency())); + write_str(",avglat="); + write_str(to_string(conn_stats->getAvgLatency())); + write_str(",maxlat="); + write_str(to_string(conn_stats->getMaxLatency())); + } + } + buf.write(')'); + buf.write('\n'); +} + +void KeeperTCPHandler::resetStats() +{ + conn_stats->reset(); + std::lock_guard lock(last_op_mutex); + { + last_op.reset(); + } +} + +Int64 KeeperTCPHandler::getPacketsReceived() const +{ + return conn_stats->getPacketsReceived(); +} +Int64 KeeperTCPHandler::getPacketsSent() const +{ + return conn_stats->getPacketsReceived(); +} +Int64 KeeperTCPHandler::getSessionId() const +{ + return session_id; +} +Int64 KeeperTCPHandler::getSessionTimeout() const +{ + return session_timeout.totalMilliseconds(); +} +Poco::Timestamp KeeperTCPHandler::getEstablished() const +{ + return established; +} +LastOp KeeperTCPHandler::getLastOp() const +{ + return last_op; +} + +KeeperStatsPtr KeeperTCPHandler::getSessionStats() const +{ + return conn_stats; +} +KeeperTCPHandler::~KeeperTCPHandler() +{ + KeeperTCPHandler::unregisterConnection(this); +} + +std::mutex KeeperTCPHandler::conns_mutex; +std::unordered_set KeeperTCPHandler::connections; + +void KeeperTCPHandler::registerConnection(KeeperTCPHandler * conn) +{ + std::lock_guard lock(conns_mutex); + connections.insert(conn); +} + +void KeeperTCPHandler::unregisterConnection(KeeperTCPHandler * conn) +{ + std::lock_guard lock(conns_mutex); + connections.erase(conn); +} + +void KeeperTCPHandler::dumpConnections(WriteBufferFromOwnString & buf, bool brief) +{ + std::lock_guard lock(conns_mutex); + for (auto * conn : connections) + { + conn->dumpStats(buf, brief); + } +} + +void KeeperTCPHandler::resetConnsStats() +{ + std::lock_guard lock(conns_mutex); + for (auto * conn : connections) + { + conn->resetStats(); + } +} + } #endif diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index eebb720939d..142344f9e8b 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -18,6 +18,9 @@ #include #include #include +#include +#include +#include namespace DB { @@ -29,11 +32,38 @@ using ThreadSafeResponseQueue = ConcurrentBoundedQueue; -class KeeperTCPHandler : public Poco::Net::TCPServerConnection +class KeeperTCPHandler : public Poco::Net::TCPServerConnection, IConnectionInfo { +public: + static void registerConnection(KeeperTCPHandler * conn); + static void unregisterConnection(KeeperTCPHandler * conn); + /// dump all connections statistics + static void dumpConnections(WriteBufferFromOwnString & buf, bool brief); + static void resetConnsStats(); + +private: + static std::mutex conns_mutex; + /// all connections + static std::unordered_set connections; + public: KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_); void run() override; + + void dumpStats(WriteBufferFromOwnString & buf, bool brief); + void resetStats(); + + /// statistics methods + Int64 getPacketsReceived() const override; + Int64 getPacketsSent() const override; + Int64 getSessionId() const override; + Int64 getSessionTimeout() const override; + Poco::Timestamp getEstablished() const override; + LastOp getLastOp() const override; + KeeperStatsPtr getSessionStats() const override; + + ~KeeperTCPHandler() override; + private: IServer & server; Poco::Logger * log; @@ -62,6 +92,22 @@ private: bool tryExecuteFourLetterWordCmd(Int32 & four_letter_cmd); std::pair receiveRequest(); + + void packageSent(); + void packageReceived(); + + void updateStats(Coordination::ZooKeeperResponsePtr & response); + + Poco::Timestamp established; + + using Operations = std::map; + Operations operations; + + std::mutex last_op_mutex; + LastOp last_op; + + KeeperStatsPtr conn_stats; + }; } diff --git a/src/Server/KeeperTCPHandlers.cpp b/src/Server/KeeperTCPHandlers.cpp new file mode 100644 index 00000000000..f55c9aeb69a --- /dev/null +++ b/src/Server/KeeperTCPHandlers.cpp @@ -0,0 +1,25 @@ +#include + +namespace DB +{ +void KeeperTCPHandlers::registerConnection(KeeperTCPHandler * conn) +{ + std::lock_guard lock(conns_mutex); + connections.insert(conn); +} + +void KeeperTCPHandlers::unregisterConnection(KeeperTCPHandler * conn) +{ + std::lock_guard lock(conns_mutex); + connections.erase(conn); +} + +void KeeperTCPHandlers::dumpConnections(WriteBufferFromOwnString & buf, bool brief) +{ + std::lock_guard lock(conns_mutex); + for (auto * conn : connections) + { + conn->dumpStats(buf, brief); + } +} +} diff --git a/src/Server/KeeperTCPHandlers.h b/src/Server/KeeperTCPHandlers.h new file mode 100644 index 00000000000..dfcdb2279de --- /dev/null +++ b/src/Server/KeeperTCPHandlers.h @@ -0,0 +1,18 @@ +#pragma once + +#include + +namespace DB +{ +class KeeperTCPHandlers +{ +public: + static void registerConnection(KeeperTCPHandler * conn); + static void unregisterConnection(KeeperTCPHandler * conn); + static void dumpConnections(WriteBufferFromOwnString & buf, bool brief); + +private: + static std::mutex conns_mutex; + static std::unordered_set connections; +}; +} diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml index bcbaa8780ce..258e58b558e 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml @@ -4,10 +4,11 @@ 1 /var/lib/clickhouse/coordination/log /var/lib/clickhouse/coordination/snapshots + * 5000 - 10000 + 30000 75 trace diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml index 2f0ae15d542..f49fb819256 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml @@ -4,10 +4,11 @@ 2 /var/lib/clickhouse/coordination/log /var/lib/clickhouse/coordination/snapshots + * 5000 - 10000 + 30000 75 trace diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml index a6315675df8..4606dce367f 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml @@ -4,10 +4,11 @@ 3 /var/lib/clickhouse/coordination/log /var/lib/clickhouse/coordination/snapshots + * 5000 - 10000 + 30000 75 trace diff --git a/tests/integration/test_keeper_four_word_command/configs/use_keeper.xml b/tests/integration/test_keeper_four_word_command/configs/use_keeper.xml index b6139005d2f..1e444182899 100644 --- a/tests/integration/test_keeper_four_word_command/configs/use_keeper.xml +++ b/tests/integration/test_keeper_four_word_command/configs/use_keeper.xml @@ -1,9 +1,5 @@ - - node1 - 9181 - node2 9181 diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 751011f3594..ee7f5abb4e4 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -10,6 +10,7 @@ from helpers.network import PartitionManager from helpers.test_tools import assert_eq_with_retry from io import StringIO import csv +import re cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', main_configs=['configs/enable_keeper1.xml', 'configs/use_keeper.xml'], @@ -42,6 +43,17 @@ def destroy_zk_client(zk): pass +def clear_znodes(): + zk = None + try: + zk = get_fake_zk(node3.name, timeout=30.0) + nodes = zk.get_children('/') + for node in [n for n in nodes if 'test_4lw_' in n]: + zk.delete('/' + node) + finally: + destroy_zk_client(zk) + + def wait_node(node): for _ in range(100): zk = None @@ -71,8 +83,8 @@ def get_fake_zk(nodename, timeout=30.0): return _fake_zk_instance -def get_keeper_socket(nodename): - hosts = cluster.get_instance_ip(nodename) +def get_keeper_socket(node_name): + hosts = cluster.get_instance_ip(node_name) client = socket.socket() client.settimeout(10) client.connect((hosts, 9181)) @@ -81,19 +93,50 @@ def get_keeper_socket(nodename): def close_keeper_socket(cli): if cli is not None: - print("close socket") cli.close() +def reset_node_stats(node_name=node1.name): + client = None + try: + client = get_keeper_socket(node_name) + client.send(b'srst') + client.recv(10) + finally: + if client is not None: + client.close() + + +def send_4lw_cmd(node_name=node1.name, cmd='ruok'): + client = None + try: + client = get_keeper_socket(node_name) + client.send(cmd.encode()) + data = client.recv(100_000) + data = data.decode() + return data + finally: + if client is not None: + client.close() + + +def reset_conn_stats(node_name=node1.name): + client = None + try: + client = get_keeper_socket(node_name) + client.send(b'crst') + client.recv(10_000) + finally: + if client is not None: + client.close() + + def test_cmd_ruok(started_cluster): client = None try: wait_nodes() - client = get_keeper_socket("node1") - client.send(b'ruok') - data = client.recv(4) - print(data) - assert data.decode() == 'imok' + data = send_4lw_cmd(cmd='ruok') + assert data == 'imok' finally: close_keeper_socket(client) @@ -103,18 +146,20 @@ def do_some_action(zk, create_cnt=0, get_cnt=0, set_cnt=0, ephemeral_cnt=0, watc assert create_cnt >= set_cnt assert create_cnt >= watch_cnt assert create_cnt >= delete_cnt + # ensure not delete watched node + assert create_cnt >= (delete_cnt + watch_cnt) for i in range(create_cnt): - zk.create("/normal_node_" + str(i), b"") + zk.create("/test_4lw_normal_node_" + str(i), b"") for i in range(get_cnt): - zk.get("/normal_node_" + str(i)) + zk.get("/test_4lw_normal_node_" + str(i)) for i in range(set_cnt): - zk.set("/normal_node_" + str(i), b"new-value") + zk.set("/test_4lw_normal_node_" + str(i), b"new-value") for i in range(ephemeral_cnt): - zk.create("/ephemeral_node_" + str(i), ephemeral=True) + zk.create("/test_4lw_ephemeral_node_" + str(i), ephemeral=True) fake_ephemeral_event = None @@ -124,35 +169,28 @@ def do_some_action(zk, create_cnt=0, get_cnt=0, set_cnt=0, ephemeral_cnt=0, watc fake_ephemeral_event = event for i in range(watch_cnt): - zk.exists("/normal_node_" + str(i), watch=fake_ephemeral_callback) + zk.exists("/test_4lw_normal_node_" + str(i), watch=fake_ephemeral_callback) for i in range(create_cnt - delete_cnt, create_cnt): - zk.delete("/normal_node_" + str(i)) + zk.delete("/test_4lw_normal_node_" + str(i)) def test_cmd_mntr(started_cluster): - client = None zk = None try: wait_nodes() + clear_znodes() # reset stat first - client = get_keeper_socket("node1") - client.send(b'srst') - data = client.recv(10_000) - client.close() - assert data.decode() == "Server stats reset." + reset_node_stats(node1.name) zk = get_fake_zk(node1.name, timeout=30.0) do_some_action(zk, create_cnt=10, get_cnt=10, set_cnt=5, ephemeral_cnt=2, watch_cnt=2, delete_cnt=2) - client = get_keeper_socket("node1") - client.send(b'mntr') - data = client.recv(10_000_000) - assert len(data) != 0 + data = send_4lw_cmd(cmd='mntr') # print(data.decode()) - reader = csv.reader(data.decode().split('\n'), delimiter='\t') + reader = csv.reader(data.split('\n'), delimiter='\t') result = {} for row in reader: @@ -169,10 +207,10 @@ def test_cmd_mntr(started_cluster): assert int(result["zk_max_latency"]) >= int(result["zk_avg_latency"]) assert int(result["zk_packets_received"]) == 31 - # contains 31 user request response, 1 session establish response - assert int(result["zk_packets_sent"]) == 32 + # contains 31 user request response + assert int(result["zk_packets_sent"]) == 31 - assert 1 <= int(result["zk_num_alive_connections"]) <= 4 + assert int(result["zk_num_alive_connections"]) == 1 assert int(result["zk_outstanding_requests"]) == 0 assert result["zk_server_state"] == "leader" @@ -182,8 +220,7 @@ def test_cmd_mntr(started_cluster): # 3 nodes created by clickhouse "/clickhouse/task_queue/ddl" # 1 root node assert int(result["zk_znode_count"]) == 14 - # ClickHouse may watch "/clickhouse/task_queue/ddl" - assert int(result["zk_watch_count"]) >= 2 + assert int(result["zk_watch_count"]) == 2 assert int(result["zk_ephemerals_count"]) == 2 assert int(result["zk_approximate_data_size"]) > 0 @@ -195,28 +232,22 @@ def test_cmd_mntr(started_cluster): finally: destroy_zk_client(zk) - close_keeper_socket(client) def test_cmd_srst(started_cluster): client = None try: wait_nodes() + clear_znodes() - # reset stat first - client = get_keeper_socket("node1") - client.send(b'srst') - data = client.recv(10_000) - client.close() - assert data.decode() == "Server stats reset." + data = send_4lw_cmd(cmd='srst') + assert data == "Server stats reset." - client = get_keeper_socket("node1") - client.send(b'mntr') - data = client.recv(10_000_000) + data = send_4lw_cmd(cmd='mntr') assert len(data) != 0 - # print(data.decode()) - reader = csv.reader(data.decode().split('\n'), delimiter='\t') + # print(data) + reader = csv.reader(data.split('\n'), delimiter='\t') result = {} for row in reader: @@ -234,15 +265,11 @@ def test_cmd_conf(started_cluster): client = None try: wait_nodes() - client = get_keeper_socket("node1") + clear_znodes() - # reset stat first - client.send(b'conf') - data = client.recv(10_000_000) - assert len(data) != 0 + data = send_4lw_cmd(cmd='conf') - # print(data.decode()) - reader = csv.reader(data.decode().split('\n'), delimiter='=') + reader = csv.reader(data.split('\n'), delimiter='=') result = {} for row in reader: @@ -259,7 +286,7 @@ def test_cmd_conf(started_cluster): assert result["log_storage_path"] == "/var/lib/clickhouse/coordination/log" assert result["snapshot_storage_path"] == "/var/lib/clickhouse/coordination/snapshots" - assert result["session_timeout_ms"] == "10000" + assert result["session_timeout_ms"] == "30000" assert result["operation_timeout_ms"] == "5000" assert result["dead_session_check_period_ms"] == "500" assert result["heart_beat_interval_ms"] == "500" @@ -288,3 +315,298 @@ def test_cmd_conf(started_cluster): finally: close_keeper_socket(client) + + +def test_cmd_isro(started_cluster): + wait_nodes() + assert send_4lw_cmd(node1.name, 'isro') == 'rw' + assert send_4lw_cmd(node2.name, 'isro') == 'ro' + + +def test_cmd_srvr(started_cluster): + zk = None + try: + wait_nodes() + clear_znodes() + + reset_node_stats(node1.name) + + zk = get_fake_zk(node1.name, timeout=30.0) + do_some_action(zk, create_cnt=10) + + data = send_4lw_cmd(cmd='srvr') + + print("srvr output -------------------------------------") + print(data) + + reader = csv.reader(data.split('\n'), delimiter=':') + result = {} + + for row in reader: + if len(row) != 0: + result[row[0].strip()] = row[1].strip() + + assert 'ClickHouse Keeper version' in result + assert 'Latency min/avg/max' in result + assert result['Received'] == '10' + assert result['Sent'] == '10' + assert int(result['Connections']) == 1 + assert int(result['Zxid']) > 14 + assert result['Mode'] == 'leader' + assert result['Node count'] == '14' + + finally: + destroy_zk_client(zk) + + +def test_cmd_stat(started_cluster): + zk = None + try: + wait_nodes() + clear_znodes() + reset_node_stats(node1.name) + reset_conn_stats(node1.name) + + zk = get_fake_zk(node1.name, timeout=30.0) + do_some_action(zk, create_cnt=10) + + data = send_4lw_cmd(cmd='stat') + + print("stat output -------------------------------------") + print(data) + + # keeper statistics + stats = [n for n in data.split('\n') if '=' not in n] + reader = csv.reader(stats, delimiter=':') + result = {} + + for row in reader: + if len(row) != 0: + result[row[0].strip()] = row[1].strip() + + assert 'ClickHouse Keeper version' in result + assert 'Latency min/avg/max' in result + assert result['Received'] == '10' + assert result['Sent'] == '10' + assert int(result['Connections']) == 1 + assert int(result['Zxid']) > 14 + assert result['Mode'] == 'leader' + assert result['Node count'] == '14' + + # filter connection statistics + cons = [n for n in data.split('\n') if '=' in n] + # filter connection created by 'cons' + cons = [n for n in cons if 'recved=0' not in n and len(n) > 0] + assert len(cons) == 1 + + conn_stat = re.match(r'(.*?)[:].*[(](.*?)[)].*', cons[0].strip(), re.S).group(2) + assert conn_stat is not None + + result = {} + for col in conn_stat.split(','): + col = col.strip().split('=') + result[col[0]] = col[1] + + assert result['recved'] == '10' + assert result['sent'] == '10' + + finally: + destroy_zk_client(zk) + + +def test_cmd_cons(started_cluster): + zk = None + try: + wait_nodes() + clear_znodes() + reset_conn_stats() + + zk = get_fake_zk(node1.name, timeout=30.0) + do_some_action(zk, create_cnt=10) + + data = send_4lw_cmd(cmd='cons') + + print("cons output -------------------------------------") + print(data) + + # filter connection created by 'cons' + cons = [n for n in data.split('\n') if 'recved=0' not in n and len(n) > 0] + assert len(cons) == 1 + + conn_stat = re.match(r'(.*?)[:].*[(](.*?)[)].*', cons[0].strip(), re.S).group(2) + assert conn_stat is not None + + result = {} + for col in conn_stat.split(','): + col = col.strip().split('=') + result[col[0]] = col[1] + + assert result['recved'] == '10' + assert result['sent'] == '10' + assert 'sid' in result + assert result['lop'] == 'Create' + assert 'est' in result + assert result['to'] == '30000' + assert result['lcxid'] == '0x000000000000000a' + assert 'lzxid' in result + assert 'lresp' in result + assert int(result['llat']) >= 0 + assert int(result['minlat']) >= 0 + assert int(result['avglat']) >= 0 + assert int(result['maxlat']) >= 0 + + finally: + destroy_zk_client(zk) + + +def test_cmd_crst(started_cluster): + zk = None + try: + wait_nodes() + clear_znodes() + reset_conn_stats() + + zk = get_fake_zk(node1.name, timeout=30.0) + do_some_action(zk, create_cnt=10) + + data = send_4lw_cmd(cmd='crst') + + print("crst output -------------------------------------") + print(data) + + data = send_4lw_cmd(cmd='cons') + + # 2 connections, 1 for 'cons' command, 1 for zk + cons = [n for n in data.split('\n') if len(n) > 0] + assert len(cons) == 2 + + conn_stat = re.match(r'(.*?)[:].*[(](.*?)[)].*', cons[0].strip(), re.S).group(2) + assert conn_stat is not None + + result = {} + for col in conn_stat.split(','): + col = col.strip().split('=') + result[col[0]] = col[1] + + assert result['recved'] == '0' + assert result['sent'] == '0' + assert 'sid' in result + assert result['lop'] == 'NA' + assert 'est' in result + assert result['to'] == '30000' + assert 'lcxid' not in result + assert result['lzxid'] == '0xffffffffffffffff' + assert result['lresp'] == '0' + assert int(result['llat']) == 0 + assert int(result['minlat']) == 0 + assert int(result['avglat']) == 0 + assert int(result['maxlat']) == 0 + + finally: + destroy_zk_client(zk) + + +def test_cmd_dump(started_cluster): + zk = None + try: + wait_nodes() + clear_znodes() + reset_node_stats() + + zk = get_fake_zk(node1.name, timeout=30.0) + do_some_action(zk, ephemeral_cnt=2) + + data = send_4lw_cmd(cmd='dump') + + print("dump output -------------------------------------") + print(data) + + list_data = data.split('\n') + + session_count = int(re.match(r'.*[(](.*?)[)].*', list_data[0], re.S).group(1)) + assert session_count == 1 + + assert '\t' + '/test_4lw_ephemeral_node_0' in list_data + assert '\t' + '/test_4lw_ephemeral_node_1' in list_data + finally: + destroy_zk_client(zk) + + +def test_cmd_wchs(started_cluster): + zk = None + try: + wait_nodes() + clear_znodes() + reset_node_stats() + + zk = get_fake_zk(node1.name, timeout=30.0) + do_some_action(zk, create_cnt=2, watch_cnt=2) + + data = send_4lw_cmd(cmd='wchs') + + print("wchs output -------------------------------------") + print(data) + + list_data = [n for n in data.split('\n') if len(n.strip()) > 0] + + # 37 connections watching 632141 paths + # Total watches:632141 + matcher = re.match(r'([0-9].*) connections watching ([0-9].*) paths', list_data[0], re.S) + conn_count = int(matcher.group(1)) + watch_path_count = int(matcher.group(2)) + watch_count = int(re.match(r'Total watches:([0-9].*)', list_data[1], re.S).group(1)) + + assert conn_count == 1 + assert watch_path_count == 2 + assert watch_count == 2 + finally: + destroy_zk_client(zk) + + +def test_cmd_wchc(started_cluster): + zk = None + try: + wait_nodes() + clear_znodes() + reset_node_stats() + + zk = get_fake_zk(node1.name, timeout=30.0) + do_some_action(zk, create_cnt=2, watch_cnt=2) + + data = send_4lw_cmd(cmd='wchc') + + print("wchc output -------------------------------------") + print(data) + + list_data = [n for n in data.split('\n') if len(n.strip()) > 0] + + assert len(list_data) == 3 + assert '\t' + '/test_4lw_normal_node_0' in list_data + assert '\t' + '/test_4lw_normal_node_1' in list_data + finally: + destroy_zk_client(zk) + + +def test_cmd_wchp(started_cluster): + zk = None + try: + wait_nodes() + clear_znodes() + reset_node_stats() + + zk = get_fake_zk(node1.name, timeout=30.0) + do_some_action(zk, create_cnt=2, watch_cnt=2) + + data = send_4lw_cmd(cmd='wchp') + + print("wchp output -------------------------------------") + print(data) + + list_data = [n for n in data.split('\n') if len(n.strip()) > 0] + + assert len(list_data) == 4 + assert '/test_4lw_normal_node_0' in list_data + assert '/test_4lw_normal_node_1' in list_data + finally: + destroy_zk_client(zk) + From 094f79c47b526118a0273c5a7468ea5735fe6a7c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 9 Nov 2021 17:49:42 +0800 Subject: [PATCH 15/49] fix code style error --- src/Coordination/FourLetterCommand.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 1043d1ecffc..bb2fca6bc4c 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -295,7 +295,8 @@ String ServerStatCommand::run() using std::to_string; StringBuffer buf; - auto write = [&buf](const String & key, const String & value) { + auto write = [&buf](const String & key, const String & value) + { buf.write(key.data(), key.size()); buf.write(": ", 2); buf.write(value.data(), value.size()); From b480b40ff751416a78a85db3b6b635ecb4052fef Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 9 Nov 2021 19:40:35 +0800 Subject: [PATCH 16/49] add docs --- docs/en/operations/clickhouse-keeper.md | 189 ++++++++++++++++++++++++ src/Coordination/FourLetterCommand.cpp | 32 ++-- src/Coordination/KeeperDispatcher.cpp | 5 + 3 files changed, 205 insertions(+), 21 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 58c59ce9f79..6c9ae29907a 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -100,6 +100,195 @@ ClickHouse Keeper is bundled into the ClickHouse server package, just add config clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon ``` +## Four Latter Word Commands + +ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as 'mntr', 'stat' etc. There are some more interesting commands: "stat" "stat" gives some general information about the server and connected clients, while "srvr" and "cons" give extended details on server and connections respectively. + +Four Letter Words has a white list configuration `four_letter_word_white_list` which has default value "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro". + +You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. +``` +echo mntr | nc localhost 9181 +``` + +Bellow are the detailed 4lw commands: + +- ruok : Tests if server is running in a non-error state. The server will respond with imok if it is running. Otherwise it will not respond at all. A response of "imok" does not necessarily indicate that the server has joined the quorum, just that the server process is active and bound to the specified client port. Use "stat" for details on state wrt quorum and client connection information. + +``` +imok +``` + +- mntr : Outputs a list of variables that could be used for monitoring the health of the cluster. + +``` +zk_version v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +zk_avg_latency 0 +zk_max_latency 0 +zk_min_latency 0 +zk_packets_received 68 +zk_packets_sent 68 +zk_num_alive_connections 1 +zk_outstanding_requests 0 +zk_server_state leader +zk_znode_count 4 +zk_watch_count 1 +zk_ephemerals_count 0 +zk_approximate_data_size 723 +zk_open_file_descriptor_count 310 +zk_max_file_descriptor_count 10240 +zk_followers 0 +zk_synced_followers 0 +``` + +- srvr : Lists full details for the server. + +``` +ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +Latency min/avg/max: 0/0/0 + +Received: 2 +Sent : 2 +Connections: 1 +Outstanding: 0 +Zxid: 34 +Mode: leader +Node count: 4 +``` + +- stat : Lists brief details for the server and connected clients. + +``` +ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +Clients: + 192.168.1.1:52852(recved=0,sent=0) + 192.168.1.1:52042(recved=24,sent=48) + +Latency min/avg/max: 0/0/0 + +Received: 4 +Sent : 4 +Connections: 1 +Outstanding: 0 +Zxid: 36 +Mode: leader +Node count: 4 + +``` + +- conf : Print details about serving configuration. + +``` +server_id=1 +tcp_port=2181 +four_letter_word_white_list=* +log_storage_path=./coordination/logs +snapshot_storage_path=./coordination/snapshots +max_requests_batch_size=100 +session_timeout_ms=30000 +operation_timeout_ms=10000 +dead_session_check_period_ms=500 +heart_beat_interval_ms=500 +election_timeout_lower_bound_ms=1000 +election_timeout_upper_bound_ms=2000 +reserved_log_items=1000000000000000 +snapshot_distance=10000 +auto_forwarding=true +shutdown_timeout=5000 +startup_timeout=240000 +raft_logs_level=information +snapshots_to_keep=3 +rotate_log_storage_interval=100000 +stale_log_gap=10000 +fresh_log_gap=200 +max_requests_batch_size=100 +quorum_reads=false +force_sync=false +compress_logs=true +compress_snapshots_with_zstd_format=true +configuration_change_tries_count=20 +``` + +- cons : List full connection/session details for all clients connected to this server. Includes information on numbers of packets received/sent, session id, operation latencies, last operation performed, etc... + +``` + 192.168.1.1:52163(recved=0,sent=0,sid=0xffffffffffffffff,lop=NA,est=1636454787393,to=30000,lzxid=0xffffffffffffffff,lresp=0,llat=0,minlat=0,avglat=0,maxlat=0) + 192.168.1.1:52042(recved=9,sent=18,sid=0x0000000000000001,lop=List,est=1636454739887,to=30000,lcxid=0x0000000000000005,lzxid=0x0000000000000005,lresp=1636454739892,llat=0,minlat=0,avglat=0,maxlat=0) +``` + +- crst : Reset connection/session statistics for all connections. + +``` +Connection stats reset. +``` + +- envi : Print details about serving environment + +``` +Environment: +clickhouse.keeper.version=v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +host.name=ZBMAC-C02D4054M.local +os.name=Darwin +os.arch=x86_64 +os.version=19.6.0 +cpu.count=12 +user.name=root +user.home=/Users/wujianchao5/ +user.dir=/Users/wujianchao5/project/jd/clickhouse/cmake-build-debug/programs/ +user.tmp=/var/folders/b4/smbq5mfj7578f2jzwn602tt40000gn/T/ +``` + +- srst : Reset server statistics. + +``` +Server stats reset. +``` + +- dirs : Shows the total size of snapshot and log files in bytes + +``` +snapshot_dir_size: 0 +log_dir_size: 3875 +``` + +- isro: Tests if server is running in read-only mode. The server will respond with "ro" if in read-only mode or "rw" if not in read-only mode. + +``` +rw +``` + +- wchs : Lists brief information on watches for the server. + +``` +1 connections watching 1 paths +Total watches:1 +``` + +- wchc : Lists detailed information on watches for the server, by session. This outputs a list of sessions(connections) with associated watches (paths). Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. + +``` +/clickhouse/task_queue/ddl + 0x0000000000000001 +``` + +- wchp : Lists detailed information on watches for the server, by path. This outputs a list of paths (znodes) with associated sessions. Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. + +``` +0x0000000000000001 + /clickhouse/task_queue/ddl +``` + +- dump : Lists the outstanding sessions and ephemeral nodes. This only works on the leader. + +``` +Sessions dump (2): +0x0000000000000001 +0x0000000000000002 +Sessions with Ephemerals (1): +0x0000000000000001 + /watch_path +``` + ## [experimental] Migration from ZooKeeper Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration: diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index bb2fca6bc4c..c4838c3bde0 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -384,8 +384,8 @@ String WatchByPathCommand::run() String DataSizeCommand::run() { StringBuffer buf; - buf << "datadir_size: " << keeper_dispatcher.getSnapDirSize() << '\n'; - buf << "logdir_size: " << keeper_dispatcher.getDataDirSize() << '\n'; + buf << "snapshot_dir_size: " << keeper_dispatcher.getSnapDirSize() << '\n'; + buf << "log_dir_size: " << keeper_dispatcher.getDataDirSize() << '\n'; return buf.str(); } @@ -407,29 +407,19 @@ String EnviCommand::run() buf << "Environment:\n"; buf << "clickhouse.keeper.version=" << (String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH) << '\n'; - buf << "host.name" << Environment::nodeName() << '\n'; - buf << "os.name" << Environment::osDisplayName() << '\n'; - buf << "os.arch" << Environment::osArchitecture() << '\n'; - buf << "os.version" << Environment::osVersion() << '\n'; - buf << "cpu.count" << Environment::processorCount() << '\n'; - - try - { - buf << "ip.address" << Environment::nodeId() << '\n'; - } - catch (...) - { - Poco::Logger * log = &Poco::Logger::get("EnviCommand"); - LOG_WARNING(log, "Can not get server ip address."); - } + buf << "host.name=" << Environment::nodeName() << '\n'; + buf << "os.name=" << Environment::osDisplayName() << '\n'; + buf << "os.arch=" << Environment::osArchitecture() << '\n'; + buf << "os.version=" << Environment::osVersion() << '\n'; + buf << "cpu.count=" << Environment::processorCount() << '\n'; char user_name[128]; getlogin_r(user_name, 128); - buf << "user.name" << user_name << '\n'; + buf << "user.name=" << user_name << '\n'; - buf << "user.home" << Path::home() << '\n'; - buf << "user.dir" << Path::current() << '\n'; - buf << "user.tmp" << Path::temp() << '\n'; + buf << "user.home=" << Path::home() << '\n'; + buf << "user.dir=" << Path::current() << '\n'; + buf << "user.tmp=" << Path::temp() << '\n'; return buf.str(); } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 6f3531471d7..5c4d5849d5e 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -24,6 +24,11 @@ using Poco::DirectoryIterator; UInt64 getDirSize(Path dir) { + if (!File(dir).exists()) + { + return 0; + } + DirectoryIterator it(dir); DirectoryIterator end; From 108d14b46860eb2790da5be6aea49f5e2491e562 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 10 Nov 2021 10:34:04 +0800 Subject: [PATCH 17/49] docs adjustment --- docs/en/operations/clickhouse-keeper.md | 32 +++++++++++++------------ 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 6c9ae29907a..8608ab23986 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -49,6 +49,7 @@ Internal coordination settings are located in `..` section and contain servers description. The only parameter for the whole quorum is `secure`, which enables encrypted connection for communication between quorum participants. The main parameters for each `` are: @@ -102,16 +103,16 @@ clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon ## Four Latter Word Commands -ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as 'mntr', 'stat' etc. There are some more interesting commands: "stat" "stat" gives some general information about the server and connected clients, while "srvr" and "cons" give extended details on server and connections respectively. +ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. -Four Letter Words has a white list configuration `four_letter_word_white_list` which has default value "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro". +The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro". You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. ``` echo mntr | nc localhost 9181 ``` -Bellow are the detailed 4lw commands: +Bellow is the detailed 4lw commands: - ruok : Tests if server is running in a non-error state. The server will respond with imok if it is running. Otherwise it will not respond at all. A response of "imok" does not necessarily indicate that the server has joined the quorum, just that the server process is active and bound to the specified client port. Use "stat" for details on state wrt quorum and client connection information. @@ -176,6 +177,12 @@ Node count: 4 ``` +- srst : Reset server statistics. The command will affect the result of `srvr`, `mntr` and `stat`. + +``` +Server stats reset. +``` + - conf : Print details about serving configuration. ``` @@ -233,16 +240,11 @@ os.arch=x86_64 os.version=19.6.0 cpu.count=12 user.name=root -user.home=/Users/wujianchao5/ -user.dir=/Users/wujianchao5/project/jd/clickhouse/cmake-build-debug/programs/ +user.home=/Users/JackyWoo/ +user.dir=/Users/JackyWoo/project/jd/clickhouse/cmake-build-debug/programs/ user.tmp=/var/folders/b4/smbq5mfj7578f2jzwn602tt40000gn/T/ ``` -- srst : Reset server statistics. - -``` -Server stats reset. -``` - dirs : Shows the total size of snapshot and log files in bytes @@ -267,15 +269,15 @@ Total watches:1 - wchc : Lists detailed information on watches for the server, by session. This outputs a list of sessions(connections) with associated watches (paths). Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. ``` -/clickhouse/task_queue/ddl - 0x0000000000000001 +0x0000000000000001 + /clickhouse/task_queue/ddl ``` - wchp : Lists detailed information on watches for the server, by path. This outputs a list of paths (znodes) with associated sessions. Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. ``` -0x0000000000000001 - /clickhouse/task_queue/ddl +/clickhouse/task_queue/ddl + 0x0000000000000001 ``` - dump : Lists the outstanding sessions and ephemeral nodes. This only works on the leader. @@ -286,7 +288,7 @@ Sessions dump (2): 0x0000000000000002 Sessions with Ephemerals (1): 0x0000000000000001 - /watch_path + /clickhouse/task_queue/ddl ``` ## [experimental] Migration from ZooKeeper From 1cbe9f602414566fe196001d3bec4b094597d276 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 10 Nov 2021 10:50:12 +0800 Subject: [PATCH 18/49] fix build error --- src/Server/KeeperTCPHandler.cpp | 1 - src/Server/KeeperTCPHandlers.cpp | 25 ------------------------- src/Server/KeeperTCPHandlers.h | 18 ------------------ 3 files changed, 44 deletions(-) delete mode 100644 src/Server/KeeperTCPHandlers.cpp delete mode 100644 src/Server/KeeperTCPHandlers.h diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 5b066d47e71..e4095aa411a 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -202,7 +202,6 @@ KeeperTCPHandler::KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSoc , session_timeout(0, global_context->getConfigRef().getUInt("keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) , poll_wrapper(std::make_unique(socket_)) , responses(std::make_unique(std::numeric_limits::max())) - , established(Poco::Timestamp()) , conn_stats(std::make_shared()) { KeeperTCPHandler::registerConnection(this); diff --git a/src/Server/KeeperTCPHandlers.cpp b/src/Server/KeeperTCPHandlers.cpp deleted file mode 100644 index f55c9aeb69a..00000000000 --- a/src/Server/KeeperTCPHandlers.cpp +++ /dev/null @@ -1,25 +0,0 @@ -#include - -namespace DB -{ -void KeeperTCPHandlers::registerConnection(KeeperTCPHandler * conn) -{ - std::lock_guard lock(conns_mutex); - connections.insert(conn); -} - -void KeeperTCPHandlers::unregisterConnection(KeeperTCPHandler * conn) -{ - std::lock_guard lock(conns_mutex); - connections.erase(conn); -} - -void KeeperTCPHandlers::dumpConnections(WriteBufferFromOwnString & buf, bool brief) -{ - std::lock_guard lock(conns_mutex); - for (auto * conn : connections) - { - conn->dumpStats(buf, brief); - } -} -} diff --git a/src/Server/KeeperTCPHandlers.h b/src/Server/KeeperTCPHandlers.h deleted file mode 100644 index dfcdb2279de..00000000000 --- a/src/Server/KeeperTCPHandlers.h +++ /dev/null @@ -1,18 +0,0 @@ -#pragma once - -#include - -namespace DB -{ -class KeeperTCPHandlers -{ -public: - static void registerConnection(KeeperTCPHandler * conn); - static void unregisterConnection(KeeperTCPHandler * conn); - static void dumpConnections(WriteBufferFromOwnString & buf, bool brief); - -private: - static std::mutex conns_mutex; - static std::unordered_set connections; -}; -} From a2f3337ca1b84e8d667d8ee68ac2b15525b76241 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 12 Nov 2021 20:48:42 +0800 Subject: [PATCH 19/49] code style change --- src/Common/getCurrentProcessFDCount.cpp | 34 ++++++++-- src/Common/getCurrentProcessFDCount.h | 2 +- src/Common/getMaxFileDescriptorCount.cpp | 19 ++++-- src/Common/getMaxFileDescriptorCount.h | 2 +- src/Coordination/CoordinationSettings.cpp | 75 +++++++++++------------ src/Coordination/CoordinationSettings.h | 1 + src/Coordination/FourLetterCommand.cpp | 67 +++++++------------- src/Coordination/FourLetterCommand.h | 2 - src/Coordination/KeeperDispatcher.cpp | 16 ++--- src/Coordination/KeeperStorage.cpp | 13 ++-- src/Coordination/KeeperStorage.h | 2 +- src/Coordination/SnapshotableHashTable.h | 24 ++++---- src/Server/KeeperTCPHandler.cpp | 62 +++++++++---------- 13 files changed, 158 insertions(+), 161 deletions(-) diff --git a/src/Common/getCurrentProcessFDCount.cpp b/src/Common/getCurrentProcessFDCount.cpp index 6bcfabcdf20..a78cc9fb1a0 100644 --- a/src/Common/getCurrentProcessFDCount.cpp +++ b/src/Common/getCurrentProcessFDCount.cpp @@ -2,9 +2,9 @@ #include #include #include -#include -#include #include +#include +#include int getCurrentProcessFDCount() @@ -12,17 +12,39 @@ int getCurrentProcessFDCount() #if defined(__linux__) || defined(__APPLE__) using namespace DB; - char buf[64]; - snprintf(buf, 64, "lsof -p %i | wc -l", getpid()); + Int32 pid = getpid(); + std::unique_ptr command; - auto command = ShellCommand::execute(buf); + /// First try procfs + String by_procfs = fmt::format("ls /proc/{}/fd | wc -l", pid); + command = ShellCommand::execute(by_procfs); + + try + { + command->wait(); + } + catch (...) + { + /// Then try lsof command + String by_lsof = fmt::format("lsof -p {} | wc -l", pid); + command = ShellCommand::execute(by_procfs); + + try + { + command->wait(); + } + catch (...) + { + return -1; + } + } WriteBufferFromOwnString out; copyData(command->out, out); if (!out.str().empty()) { - return std::stoi(out.str()); + return parse(out.str()); } return -1; diff --git a/src/Common/getCurrentProcessFDCount.h b/src/Common/getCurrentProcessFDCount.h index 05135d3778f..583b99f6c13 100644 --- a/src/Common/getCurrentProcessFDCount.h +++ b/src/Common/getCurrentProcessFDCount.h @@ -1,5 +1,5 @@ #pragma once /// Get current process file descriptor count -/// @return -1 if error occurs +/// @return -1 os doesn't support "lsof" command or some error occurs. int getCurrentProcessFDCount(); diff --git a/src/Common/getMaxFileDescriptorCount.cpp b/src/Common/getMaxFileDescriptorCount.cpp index 70de1dd0c08..853a2c327c3 100644 --- a/src/Common/getMaxFileDescriptorCount.cpp +++ b/src/Common/getMaxFileDescriptorCount.cpp @@ -1,27 +1,34 @@ -#include -#include +#include #include #include -#include +#include +#include int getMaxFileDescriptorCount() { -#if defined(__linux__) || defined(__APPLE__) +#if defined(__linux__) || defined(__APPLE__) using namespace DB; auto command = ShellCommand::execute("ulimit -n"); + try + { + command->wait(); + } + catch (...) + { + return -1; + } WriteBufferFromOwnString out; copyData(command->out, out); if (!out.str().empty()) { - return std::stoi(out.str()); + return parse(out.str()); } return -1; #else return -1; #endif - } diff --git a/src/Common/getMaxFileDescriptorCount.h b/src/Common/getMaxFileDescriptorCount.h index 779d23417ff..6cc5df0d14c 100644 --- a/src/Common/getMaxFileDescriptorCount.h +++ b/src/Common/getMaxFileDescriptorCount.h @@ -1,6 +1,6 @@ #pragma once /// Get process max file descriptor count -/// @return -1 if error occurs +/// @return -1 if os does not support ulimit command or some error occurs int getMaxFileDescriptorCount(); diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 2370e16c794..21e42c0be8e 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace DB { @@ -49,103 +51,100 @@ KeeperSettings::KeeperSettings() void KeeperSettings::dump(WriteBufferFromOwnString & buf) const { - auto write = [&buf](const String & content) { buf.write(content.data(), content.size()); }; - auto write_int = [&buf](Int64 value) { - String str_val = std::to_string(value); - buf.write(str_val.data(), str_val.size()); + writeIntText(value, buf); buf.write('\n'); }; auto write_bool = [&buf](bool value) { String str_val = value ? "true" : "false"; - buf.write(str_val.data(), str_val.size()); + writeText(str_val, buf); buf.write('\n'); }; - write("server_id="); + writeText("server_id=", buf); write_int(server_id); if (tcp_port != NOT_EXIST) { - write("tcp_port="); + writeText("tcp_port=", buf); write_int(tcp_port); } if (tcp_port_secure != NOT_EXIST) { - write("tcp_port_secure="); + writeText("tcp_port_secure=", buf); write_int(tcp_port_secure); } - write("four_letter_word_white_list="); - write(four_letter_word_white_list); + writeText("four_letter_word_white_list=", buf); + writeText(four_letter_word_white_list, buf); buf.write('\n'); - write("log_storage_path="); - write(log_storage_path); + writeText("log_storage_path=", buf); + writeText(log_storage_path, buf); buf.write('\n'); - write("snapshot_storage_path="); - write(snapshot_storage_path); + writeText("snapshot_storage_path=", buf); + writeText(snapshot_storage_path, buf); buf.write('\n'); /// coordination_settings - write("max_requests_batch_size="); + writeText("max_requests_batch_size=", buf); write_int(coordination_settings->max_requests_batch_size); - write("session_timeout_ms="); + writeText("session_timeout_ms=", buf); write_int(UInt64(coordination_settings->session_timeout_ms)); - write("operation_timeout_ms="); + writeText("operation_timeout_ms=", buf); write_int(UInt64(coordination_settings->operation_timeout_ms)); - write("dead_session_check_period_ms="); + writeText("dead_session_check_period_ms=", buf); write_int(UInt64(coordination_settings->dead_session_check_period_ms)); - write("heart_beat_interval_ms="); + writeText("heart_beat_interval_ms=", buf); write_int(UInt64(coordination_settings->heart_beat_interval_ms)); - write("election_timeout_lower_bound_ms="); + writeText("election_timeout_lower_bound_ms=", buf); write_int(UInt64(coordination_settings->election_timeout_lower_bound_ms)); - write("election_timeout_upper_bound_ms="); + writeText("election_timeout_upper_bound_ms=", buf); write_int(UInt64(coordination_settings->election_timeout_upper_bound_ms)); - write("reserved_log_items="); + writeText("reserved_log_items=", buf); write_int(coordination_settings->reserved_log_items); - write("snapshot_distance="); + writeText("snapshot_distance=", buf); write_int(coordination_settings->snapshot_distance); - write("auto_forwarding="); + writeText("auto_forwarding=", buf); write_bool(coordination_settings->auto_forwarding); - write("shutdown_timeout="); + writeText("shutdown_timeout=", buf); write_int(UInt64(coordination_settings->shutdown_timeout)); - write("startup_timeout="); + writeText("startup_timeout=", buf); write_int(UInt64(coordination_settings->startup_timeout)); - write("raft_logs_level="); - write(coordination_settings->raft_logs_level.toString()); + writeText("raft_logs_level=", buf); + writeText(coordination_settings->raft_logs_level.toString(), buf); buf.write('\n'); - write("snapshots_to_keep="); + writeText("snapshots_to_keep=", buf); write_int(coordination_settings->snapshots_to_keep); - write("rotate_log_storage_interval="); + writeText("rotate_log_storage_interval=", buf); write_int(coordination_settings->rotate_log_storage_interval); - write("stale_log_gap="); + writeText("stale_log_gap=", buf); write_int(coordination_settings->stale_log_gap); - write("fresh_log_gap="); + writeText("fresh_log_gap=", buf); write_int(coordination_settings->fresh_log_gap); - write("max_requests_batch_size="); + writeText("max_requests_batch_size=", buf); write_int(coordination_settings->max_requests_batch_size); - write("quorum_reads="); + writeText("quorum_reads=", buf); write_bool(coordination_settings->quorum_reads); - write("force_sync="); + writeText("force_sync=", buf); write_bool(coordination_settings->force_sync); - write("compress_logs="); + writeText("compress_logs=", buf); write_bool(coordination_settings->compress_logs); - write("compress_snapshots_with_zstd_format="); + writeText("compress_snapshots_with_zstd_format=", buf); write_bool(coordination_settings->compress_snapshots_with_zstd_format); - write("configuration_change_tries_count="); + writeText("configuration_change_tries_count=", buf); write_int(coordination_settings->configuration_change_tries_count); } diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 9289f7aaeba..0e4784439a6 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -52,6 +52,7 @@ struct CoordinationSettings : public BaseSettings using CoordinationSettingsPtr = std::shared_ptr; +/// encapsulation of keeper settings from keeper_server struct KeeperSettings { static constexpr int NOT_EXIST = -1; diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index c4838c3bde0..1332575f84e 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -38,16 +38,6 @@ Int32 IFourLetterCommand::toCode(const String & name) return __builtin_bswap32(res); } -void IFourLetterCommand::printSet(IFourLetterCommand::StringBuffer & buffer, std::unordered_set & set, String && prefix) -{ - for (const auto & str : set) - { - buffer.write(prefix.data(), prefix.size()); - buffer.write(str.data(), str.size()); - buffer.write('\n'); - } -} - IFourLetterCommand::~IFourLetterCommand() = default; FourLetterCommandFactory & FourLetterCommandFactory::instance() @@ -82,8 +72,6 @@ void FourLetterCommandFactory::registerCommand(FourLetterCommandPtr & command) { throw Exception("Four letter command " + command->name() + " already registered", ErrorCodes::LOGICAL_ERROR); } - auto * log = &Poco::Logger::get("FourLetterCommandFactory"); - LOG_INFO(log, "Register four letter command {}, code {}", command->name(), std::to_string(command->code())); commands.emplace(command->code(), std::move(command)); } @@ -230,32 +218,23 @@ String MonitorCommand::run() { print(ret, "followers", raft_info.getFollowerCount()); print(ret, "synced_followers", raft_info.getSyncedFollowerCount()); - /// print(ret, "pending_syncs", 0); } - /// print(ret, "last_proposal_size", -1); - /// print(ret, "max_proposal_size", -1); - /// print(ret, "min_proposal_size", -1); - return ret.str(); } void MonitorCommand::print(IFourLetterCommand::StringBuffer & buf, const String & key, const String & value) { - const static String prefix = "zk_"; - const int prefix_len = prefix.size(); - - buf.write(prefix.data(), prefix_len); - buf.write(key.data(), key.size()); - - buf.write('\t'); - buf.write(value.data(), value.size()); - buf.write('\n'); + writeText("zk_", buf); + writeText(key, buf); + writeText('\t', buf); + writeText(value, buf); + writeText('\n', buf); } void MonitorCommand::print(IFourLetterCommand::StringBuffer & buf, const String & key, UInt64 value) { - print(buf, key, std::to_string(value)); + print(buf, key, toString(value)); } String StatResetCommand::run() @@ -292,15 +271,14 @@ String RestConnStatsCommand::run() String ServerStatCommand::run() { - using std::to_string; StringBuffer buf; auto write = [&buf](const String & key, const String & value) { - buf.write(key.data(), key.size()); - buf.write(": ", 2); - buf.write(value.data(), value.size()); - buf.write('\n'); + writeText(key, buf); + writeText(": ", buf); + writeText(value, buf); + writeText('\n', buf); }; KeeperStatsPtr stats = keeper_dispatcher.getKeeperStats(); @@ -313,20 +291,19 @@ String ServerStatCommand::run() latency << stats->getMinLatency() << "/" << stats->getAvgLatency() << "/" << stats->getMaxLatency() << "\n"; write("Latency min/avg/max", latency.str()); - write("Received", to_string(stats->getPacketsReceived())); - write("Sent ", to_string(stats->getPacketsSent())); - write("Connections", to_string(keeper_info.getNumAliveConnections())); - write("Outstanding", to_string(keeper_info.getOutstandingRequests())); - write("Zxid", to_string(state_machine.getLastProcessedZxid())); + write("Received", toString(stats->getPacketsReceived())); + write("Sent ", toString(stats->getPacketsSent())); + write("Connections", toString(keeper_info.getNumAliveConnections())); + write("Outstanding", toString(keeper_info.getOutstandingRequests())); + write("Zxid", toString(state_machine.getLastProcessedZxid())); write("Mode", keeper_info.getRole()); - write("Node count", to_string(state_machine.getNodeCount())); + write("Node count", toString(state_machine.getNodeCount())); return buf.str(); } String StatCommand::run() { - using std::to_string; StringBuffer buf; auto write = [&buf](const String & key, const String & value) { buf << key << ": " << value << '\n'; }; @@ -345,13 +322,13 @@ String StatCommand::run() latency << stats->getMinLatency() << "/" << stats->getAvgLatency() << "/" << stats->getMaxLatency() << "\n"; write("Latency min/avg/max", latency.str()); - write("Received", to_string(stats->getPacketsReceived())); - write("Sent ", to_string(stats->getPacketsSent())); - write("Connections", to_string(keeper_info.getNumAliveConnections())); - write("Outstanding", to_string(keeper_info.getOutstandingRequests())); - write("Zxid", to_string(state_machine.getLastProcessedZxid())); + write("Received", toString(stats->getPacketsReceived())); + write("Sent ", toString(stats->getPacketsSent())); + write("Connections", toString(keeper_info.getNumAliveConnections())); + write("Outstanding", toString(keeper_info.getOutstandingRequests())); + write("Zxid", toString(state_machine.getLastProcessedZxid())); write("Mode", keeper_info.getRole()); - write("Node count", to_string(state_machine.getNodeCount())); + write("Node count", toString(state_machine.getNodeCount())); return buf.str(); } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 861c9d54619..85df8e95fd9 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -35,8 +35,6 @@ public: static inline String toName(Int32 code); static inline Int32 toCode(const String & name); - static void printSet(StringBuffer & buffer, std::unordered_set & set, String && prefix); - protected: const KeeperDispatcher & keeper_dispatcher; }; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 5c4d5849d5e..f6a4ef7f369 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -4,9 +4,10 @@ #include #include #include -#include -#include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -18,13 +19,12 @@ namespace ErrorCodes extern const int SYSTEM_ERROR; } -using Poco::Path; -using Poco::File; -using Poco::DirectoryIterator; +using Path = fs::path; +using DirectoryIterator = fs::directory_iterator; UInt64 getDirSize(Path dir) { - if (!File(dir).exists()) + if (!fs::exists(dir)) { return 0; } @@ -35,8 +35,8 @@ UInt64 getDirSize(Path dir) UInt64 size{0}; while (it != end) { - if (it->isFile()) - size += it->getSize(); + if (!it->is_regular_file()) + size += fs::file_size(*it); else size += getDirSize(it->path()); ++it; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index b3dffc89dda..466e653fa2f 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1223,18 +1223,13 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const { - auto write_str_set = [&buf](const std::unordered_set & objs) - { - for (const String & obj : objs) - { - buf << "\t" << obj << "\n"; - } - }; - for (const auto & e : sessions_and_watchers) { buf << "0x" << getHexUIntLowercase(e.first) << "\n"; - write_str_set(e.second); + for (const String & path : e.second) + { + buf << "\t" << path << "\n"; + } } } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 8c2f04e90e9..05f975d9213 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -41,7 +41,7 @@ public: ChildrenSet children{}; /// object memory size - UInt64 size() const + UInt64 sizeInBytes() const { UInt64 child_size{0}; for (const auto & child : children) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 42d1c297efb..952d5908e9f 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -16,11 +16,11 @@ struct ListNode }; template -struct HasSizeMethod +struct CanCalculateSize { private: template - static auto check(int) -> decltype(std::declval().size(), std::true_type()); + static auto check(int) -> decltype(std::declval().sizeInBytes(), std::true_type()); template static std::false_type check(...); @@ -82,7 +82,7 @@ private: approximate_data_size -= old_value_size; } } - /// inseert + /// insert else { approximate_data_size += key_size; @@ -118,15 +118,15 @@ private: } /// Calculate object memory size. - /// @return size(), if T has method size(), otherwise return sizeof(T) + /// @return sizeInBytes(), if T has method sizeInBytes, otherwise return sizeof(T) template - inline UInt64 sizeOf(const typename std::enable_if::value, T>::type * obj) + inline UInt64 sizeOf(const typename std::enable_if::value, T>::type * obj) { - return obj->size(); + return obj->sizeInBytes(); } template - inline UInt64 sizeOf(const typename std::enable_if::value, T>::type *) + inline UInt64 sizeOf(const typename std::enable_if::value, T>::type *) { return sizeof(T); } @@ -147,7 +147,7 @@ public: ListElem elem{key, value, true}; auto itr = list.insert(list.end(), elem); map.emplace(itr->key, itr); - updateDataSize(INSERT, sizeOf(&key), sizeOf(&value), 0); + updateDataSize(INSERT, key.size(), sizeOf(&value), 0); return true; } @@ -182,7 +182,7 @@ public: list_itr->value = value; } } - updateDataSize(INSERT_OR_REPLACE, sizeOf(&key), sizeOf(&value), old_value_size); + updateDataSize(INSERT_OR_REPLACE, key.size(), sizeOf(&value), old_value_size); } bool erase(const std::string & key) @@ -204,7 +204,7 @@ public: list.erase(list_itr); } - updateDataSize(ERASE, sizeOf(&key), 0, old_data_size); + updateDataSize(ERASE, key.size(), 0, old_data_size); return true; } @@ -238,7 +238,7 @@ public: updater(list_itr->value); ret = list_itr; } - updateDataSize(UPDATE_VALUE, sizeOf(&key), sizeOf(&ret->value), old_value_size); + updateDataSize(UPDATE_VALUE, key.size(), sizeOf(&ret->value), old_value_size); return ret; } @@ -265,7 +265,7 @@ public: { if (!itr->active_in_map) { - updateDataSize(CLEAR_OUTDATED_NODES, sizeOf(&itr->key), sizeOf(&itr->value), 0); + updateDataSize(CLEAR_OUTDATED_NODES, itr->key.size(), sizeOf(&itr->value), 0); itr = list.erase(itr); } else diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index e4095aa411a..832c02478ff 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -529,56 +529,54 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response void KeeperTCPHandler::dumpStats(WriteBufferFromOwnString & buf, bool brief) { - auto write_str = [&buf](const String & str) { buf.write(str.data(), str.size()); }; - using std::to_string; - buf.write(' '); - write_str(socket().peerAddress().toString()); - write_str("(recved="); - write_str(to_string(conn_stats->getPacketsReceived())); - write_str(",sent="); - write_str(to_string(conn_stats->getPacketsSent())); + writeText(' ', buf); + writeText(socket().peerAddress().toString(), buf); + writeText("(recved=", buf); + writeIntText(conn_stats->getPacketsReceived(), buf); + writeText(",sent=", buf); + writeIntText(conn_stats->getPacketsSent(), buf); if (!brief) { if (session_id != 0) { - write_str(",sid=0x"); - write_str(getHexUIntLowercase(getSessionId())); + writeText(",sid=0x", buf); + writeText(getHexUIntLowercase(getSessionId()), buf); - write_str(",lop="); + writeText(",lop=", buf); LastOp op; { std::lock_guard lock(last_op_mutex); op = last_op.clone(); } - write_str(op.getLastOp()); - write_str(",est="); - write_str(to_string(getEstablished().epochMicroseconds() / 1000)); - write_str(",to="); - write_str(to_string(getSessionTimeout())); + writeText(op.getLastOp(), buf); + writeText(",est=", buf); + writeIntText(getEstablished().epochMicroseconds() / 1000, buf); + writeText(",to=", buf); + writeIntText(getSessionTimeout(), buf); Int64 last_cxid = op.getLastCxid(); if (last_cxid >= 0) { - write_str(",lcxid=0x"); - write_str(getHexUIntLowercase(last_cxid)); + writeText(",lcxid=0x", buf); + writeText(getHexUIntLowercase(last_cxid), buf); } - write_str(",lzxid=0x"); - write_str(getHexUIntLowercase(op.getLastZxid())); - write_str(",lresp="); - write_str(to_string(op.getLastResponseTime())); + writeText(",lzxid=0x", buf); + writeText(getHexUIntLowercase(op.getLastZxid()), buf); + writeText(",lresp=", buf); + writeIntText(op.getLastResponseTime(), buf); - write_str(",llat="); - write_str(to_string(conn_stats->getLastLatency())); - write_str(",minlat="); - write_str(to_string(conn_stats->getMinLatency())); - write_str(",avglat="); - write_str(to_string(conn_stats->getAvgLatency())); - write_str(",maxlat="); - write_str(to_string(conn_stats->getMaxLatency())); + writeText(",llat=", buf); + writeIntText(conn_stats->getLastLatency(), buf); + writeText(",minlat=", buf); + writeIntText(conn_stats->getMinLatency(), buf); + writeText(",avglat=", buf); + writeIntText(conn_stats->getAvgLatency(), buf); + writeText(",maxlat=", buf); + writeIntText(conn_stats->getMaxLatency(), buf); } } - buf.write(')'); - buf.write('\n'); + writeText(')', buf); + writeText('\n', buf); } void KeeperTCPHandler::resetStats() From 0991eddcd400a946f19308155543465c1e34216f Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 15 Nov 2021 10:16:11 +0800 Subject: [PATCH 20/49] fix build error --- src/Coordination/KeeperStats.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Coordination/KeeperStats.h b/src/Coordination/KeeperStats.h index 3cf4af72029..34e0f78e517 100644 --- a/src/Coordination/KeeperStats.h +++ b/src/Coordination/KeeperStats.h @@ -1,7 +1,9 @@ #pragma once #include +#include #include +#include namespace DB { From 45cd53583982f0f0f41ce57a032a0f6c15fbf885 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Nov 2021 15:44:07 +0300 Subject: [PATCH 21/49] Fix my own wrong command usage --- src/Common/getCurrentProcessFDCount.cpp | 18 ++++-------------- src/Common/getMaxFileDescriptorCount.cpp | 16 ++++------------ 2 files changed, 8 insertions(+), 26 deletions(-) diff --git a/src/Common/getCurrentProcessFDCount.cpp b/src/Common/getCurrentProcessFDCount.cpp index a78cc9fb1a0..b85256fabb4 100644 --- a/src/Common/getCurrentProcessFDCount.cpp +++ b/src/Common/getCurrentProcessFDCount.cpp @@ -9,6 +9,7 @@ int getCurrentProcessFDCount() { + int result = -1; #if defined(__linux__) || defined(__APPLE__) using namespace DB; @@ -21,6 +22,7 @@ int getCurrentProcessFDCount() try { + readIntText(result, command->out); command->wait(); } catch (...) @@ -31,25 +33,13 @@ int getCurrentProcessFDCount() try { + readIntText(result, command->out); command->wait(); } catch (...) { - return -1; } } - - WriteBufferFromOwnString out; - copyData(command->out, out); - - if (!out.str().empty()) - { - return parse(out.str()); - } - - return -1; -#else - return -1; #endif - + return result; } diff --git a/src/Common/getMaxFileDescriptorCount.cpp b/src/Common/getMaxFileDescriptorCount.cpp index 853a2c327c3..053ac0f812d 100644 --- a/src/Common/getMaxFileDescriptorCount.cpp +++ b/src/Common/getMaxFileDescriptorCount.cpp @@ -6,29 +6,21 @@ int getMaxFileDescriptorCount() { + int result = -1; #if defined(__linux__) || defined(__APPLE__) using namespace DB; auto command = ShellCommand::execute("ulimit -n"); try { + readIntText(result, command->out); command->wait(); } catch (...) { - return -1; } - WriteBufferFromOwnString out; - copyData(command->out, out); - - if (!out.str().empty()) - { - return parse(out.str()); - } - - return -1; -#else - return -1; #endif + + return result; } From 4438e4c7783128f6844770217f9a5e27edc4d30b Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 17 Nov 2021 21:21:12 +0800 Subject: [PATCH 22/49] fix stress test error --- src/Coordination/KeeperServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 8b79341840c..8bea23d30c7 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -88,7 +88,7 @@ KeeperServer::KeeperServer( , coordination_settings(settings_->coordination_settings) , state_machine(nuraft::cs_new( responses_queue_, snapshots_queue_, - settings_->log_storage_path, + settings_->snapshot_storage_path, coordination_settings, checkAndGetSuperdigest(settings_->super_digest))) , state_manager(nuraft::cs_new(server_id, "keeper_server", settings_->log_storage_path, config, coordination_settings)) From 007366b5067a9dd09b8cfa1d43d308f3dcf392ad Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 18 Nov 2021 11:39:09 +0800 Subject: [PATCH 23/49] remove sizeOf method from SnapshotableHashTable --- src/Coordination/FourLetterCommand.cpp | 3 +- src/Coordination/FourLetterCommand.h | 3 +- src/Coordination/SnapshotableHashTable.h | 40 ++++--------------- src/Coordination/tests/gtest_coordination.cpp | 28 +++++++++---- src/Server/KeeperTCPHandler.cpp | 30 ++++++++------ src/Server/KeeperTCPHandler.h | 2 +- 6 files changed, 50 insertions(+), 56 deletions(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 1332575f84e..a0beb078067 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -28,7 +28,8 @@ Int32 IFourLetterCommand::code() String IFourLetterCommand::toName(Int32 code) { - return String(reinterpret_cast(__builtin_bswap32(code))); + int reverted_code = __builtin_bswap32(code); + return String(reinterpret_cast(&reverted_code)); } Int32 IFourLetterCommand::toCode(const String & name) diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 85df8e95fd9..59ce62c0772 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -32,7 +32,7 @@ public: virtual ~IFourLetterCommand(); Int32 code(); - static inline String toName(Int32 code); + static String toName(Int32 code); static inline Int32 toCode(const String & name); protected: @@ -45,6 +45,7 @@ public: using Commands = std::unordered_map; using WhiteList = std::vector; + ///represent '*' which is used in white list static constexpr Int32 WHITE_LIST_ALL = 0; bool isKnown(Int32 code); diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 952d5908e9f..c88aa4598fe 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -15,18 +15,6 @@ struct ListNode bool active_in_map; }; -template -struct CanCalculateSize -{ -private: - template - static auto check(int) -> decltype(std::declval().sizeInBytes(), std::true_type()); - template - static std::false_type check(...); - -public: - static constexpr bool value = std::is_same(0)), std::true_type>::value; -}; template class SnapshotableHashTable @@ -117,20 +105,6 @@ private: } } - /// Calculate object memory size. - /// @return sizeInBytes(), if T has method sizeInBytes, otherwise return sizeof(T) - template - inline UInt64 sizeOf(const typename std::enable_if::value, T>::type * obj) - { - return obj->sizeInBytes(); - } - - template - inline UInt64 sizeOf(const typename std::enable_if::value, T>::type *) - { - return sizeof(T); - } - public: using iterator = typename List::iterator; @@ -147,7 +121,7 @@ public: ListElem elem{key, value, true}; auto itr = list.insert(list.end(), elem); map.emplace(itr->key, itr); - updateDataSize(INSERT, key.size(), sizeOf(&value), 0); + updateDataSize(INSERT, key.size(), value.sizeInBytes(), 0); return true; } @@ -158,7 +132,7 @@ public: void insertOrReplace(const std::string & key, const V & value) { auto it = map.find(key); - UInt64 old_value_size = it == map.end() ? 0 : sizeOf(&it->second->value); + UInt64 old_value_size = it == map.end() ? 0 : it->second->value.sizeInBytes(); if (it == map.end()) { @@ -182,7 +156,7 @@ public: list_itr->value = value; } } - updateDataSize(INSERT_OR_REPLACE, key.size(), sizeOf(&value), old_value_size); + updateDataSize(INSERT_OR_REPLACE, key.size(), value.sizeInBytes(), old_value_size); } bool erase(const std::string & key) @@ -192,7 +166,7 @@ public: return false; auto list_itr = it->second; - UInt64 old_data_size = sizeOf(&list_itr->value); + UInt64 old_data_size = list_itr->value.sizeInBytes(); if (snapshot_mode) { list_itr->active_in_map = false; @@ -219,7 +193,7 @@ public: assert(it != map.end()); auto list_itr = it->second; - UInt64 old_value_size = sizeOf(&list_itr->value); + UInt64 old_value_size = list_itr->value.sizeInBytes(); const_iterator ret; @@ -238,7 +212,7 @@ public: updater(list_itr->value); ret = list_itr; } - updateDataSize(UPDATE_VALUE, key.size(), sizeOf(&ret->value), old_value_size); + updateDataSize(UPDATE_VALUE, key.size(), ret->value.sizeInBytes(), old_value_size); return ret; } @@ -265,7 +239,7 @@ public: { if (!itr->active_in_map) { - updateDataSize(CLEAR_OUTDATED_NODES, itr->key.size(), sizeOf(&itr->value), 0); + updateDataSize(CLEAR_OUTDATED_NODES, itr->key.size(), itr->value.sizeInBytes(), 0); itr = list.erase(itr); } else diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 87874c95a2f..f0c2bc1f60e 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -829,15 +829,29 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) EXPECT_FALSE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); } +struct IntNode +{ + int value; + IntNode(int value_) : value(value_) { } + UInt64 sizeInBytes() const { return sizeof value; } + IntNode & operator=(int rhs) + { + this->value = rhs; + return *this; + } + bool operator==(const int & rhs) const { return value == rhs; } + bool operator!=(const int & rhs) const { return rhs != this->value; } +}; + TEST_P(CoordinationTest, SnapshotableHashMapSimple) { - DB::SnapshotableHashTable hello; + DB::SnapshotableHashTable hello; EXPECT_TRUE(hello.insert("hello", 5)); EXPECT_TRUE(hello.contains("hello")); EXPECT_EQ(hello.getValue("hello"), 5); EXPECT_FALSE(hello.insert("hello", 145)); EXPECT_EQ(hello.getValue("hello"), 5); - hello.updateValue("hello", [](int & value) { value = 7; }); + hello.updateValue("hello", [](IntNode & value) { value = 7; }); EXPECT_EQ(hello.getValue("hello"), 7); EXPECT_EQ(hello.size(), 1); EXPECT_TRUE(hello.erase("hello")); @@ -846,12 +860,12 @@ TEST_P(CoordinationTest, SnapshotableHashMapSimple) TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot) { - DB::SnapshotableHashTable map_snp; + DB::SnapshotableHashTable map_snp; EXPECT_TRUE(map_snp.insert("/hello", 7)); EXPECT_FALSE(map_snp.insert("/hello", 145)); map_snp.enableSnapshotMode(); EXPECT_FALSE(map_snp.insert("/hello", 145)); - map_snp.updateValue("/hello", [](int & value) { value = 554; }); + map_snp.updateValue("/hello", [](IntNode & value) { value = 554; }); EXPECT_EQ(map_snp.getValue("/hello"), 554); EXPECT_EQ(map_snp.snapshotSize(), 2); EXPECT_EQ(map_snp.size(), 1); @@ -924,13 +938,13 @@ TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot) TEST_P(CoordinationTest, SnapshotableHashMapDataSize) { /// int - DB::SnapshotableHashTable hello; + DB::SnapshotableHashTable hello; hello.disableSnapshotMode(); EXPECT_EQ(hello.getApproximateSataSize(), 0); hello.insert("hello", 1); EXPECT_EQ(hello.getApproximateSataSize(), 9); - hello.updateValue("hello", [](int & value) { value = 2; }); + hello.updateValue("hello", [](IntNode & value) { value = 2; }); EXPECT_EQ(hello.getApproximateSataSize(), 9); hello.erase("hello"); @@ -942,7 +956,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) hello.enableSnapshotMode(); hello.insert("hello", 1); EXPECT_EQ(hello.getApproximateSataSize(), 9); - hello.updateValue("hello", [](int & value) { value = 2; }); + hello.updateValue("hello", [](IntNode & value) { value = 2; }); EXPECT_EQ(hello.getApproximateSataSize(), 18); hello.clearOutdatedNodes(); diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 4852c2f8cea..c63ddc4d663 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -446,22 +446,31 @@ bool KeeperTCPHandler::isHandShake(Int32 & handshake_length) || handshake_length == Coordination::CLIENT_HANDSHAKE_LENGTH_WITH_READONLY; } -bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(Int32 & four_letter_cmd) +bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(Int32 & command) { - if (FourLetterCommandFactory::instance().isKnown(four_letter_cmd) - && FourLetterCommandFactory::instance().isEnabled(four_letter_cmd)) + if (!FourLetterCommandFactory::instance().isKnown(command)) { - auto command = FourLetterCommandFactory::instance().get(four_letter_cmd); - LOG_DEBUG(log, "receive four letter command {}", command->name()); + LOG_WARNING(log, "invalid four letter command {}", std::to_string(command)); + return false; + } + else if (!FourLetterCommandFactory::instance().isEnabled(command)) + { + LOG_WARNING(log, "four letter command {} not enabled", IFourLetterCommand::toName(command)); + return false; + } + else + { + auto command_ptr = FourLetterCommandFactory::instance().get(command); + LOG_DEBUG(log, "receive four letter command {}", command_ptr->name()); String res; try { - res = command->run(); + res = command_ptr->run(); } catch (...) { - tryLogCurrentException(log, "Error when executing four letter command " + command->name()); + tryLogCurrentException(log, "Error when executing four letter command " + command_ptr->name()); } try @@ -470,16 +479,11 @@ bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(Int32 & four_letter_cmd) } catch (const Exception &) { - tryLogCurrentException(log, "Error when send 4 letter command response"); + tryLogCurrentException(log, "Error when send four letter command response"); } return true; } - else - { - LOG_WARNING(log, "invalid four letter command {}", std::to_string(four_letter_cmd)); - } - return false; } std::pair KeeperTCPHandler::receiveRequest() diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index a9c19770996..a000b476574 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -87,7 +87,7 @@ private: Poco::Timespan receiveHandshake(int32_t handshake_length); static bool isHandShake(Int32 & handshake_length) ; - bool tryExecuteFourLetterWordCmd(Int32 & four_letter_cmd); + bool tryExecuteFourLetterWordCmd(Int32 & command); std::pair receiveRequest(); From 27d50edc596cab78154ad886d251dded28037869 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 18 Nov 2021 15:49:43 +0800 Subject: [PATCH 24/49] adjust log output --- src/Coordination/FourLetterCommand.cpp | 2 +- src/Server/KeeperTCPHandler.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index a0beb078067..6eab502743d 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -29,7 +29,7 @@ Int32 IFourLetterCommand::code() String IFourLetterCommand::toName(Int32 code) { int reverted_code = __builtin_bswap32(code); - return String(reinterpret_cast(&reverted_code)); + return String(reinterpret_cast(&reverted_code), 4); } Int32 IFourLetterCommand::toCode(const String & name) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index c63ddc4d663..2d41ef31f0c 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -450,12 +450,12 @@ bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(Int32 & command) { if (!FourLetterCommandFactory::instance().isKnown(command)) { - LOG_WARNING(log, "invalid four letter command {}", std::to_string(command)); + LOG_WARNING(log, "invalid four letter command {}", IFourLetterCommand::toName(command)); return false; } else if (!FourLetterCommandFactory::instance().isEnabled(command)) { - LOG_WARNING(log, "four letter command {} not enabled", IFourLetterCommand::toName(command)); + LOG_WARNING(log, "not enabled four letter command {}", IFourLetterCommand::toName(command)); return false; } else From afd8a321b4033cfae6ccc83f92f9e49ca4bbdd2c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 18 Nov 2021 18:34:44 +0800 Subject: [PATCH 25/49] ignore explicit-constructor in gtest_coordination.cpp --- src/Coordination/tests/gtest_coordination.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index f0c2bc1f60e..e93899be708 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -832,7 +832,14 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) struct IntNode { int value; +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wgoogle-explicit-constructor" +#endif IntNode(int value_) : value(value_) { } +#ifdef __clang__ +#pragma clang diagnostic pop +#endif UInt64 sizeInBytes() const { return sizeof value; } IntNode & operator=(int rhs) { From 0d19f2a485aa312e87bf702f14beb1e6630d77db Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 18 Nov 2021 21:27:43 +0800 Subject: [PATCH 26/49] ignore clang-tidy for IntNode in gtest_coordination --- src/Coordination/tests/gtest_coordination.cpp | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index e93899be708..710670f5b16 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -832,14 +832,7 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) struct IntNode { int value; -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wgoogle-explicit-constructor" -#endif - IntNode(int value_) : value(value_) { } -#ifdef __clang__ -#pragma clang diagnostic pop -#endif + IntNode(int value_) : value(value_) { } // NOLINT(google-explicit-constructor) UInt64 sizeInBytes() const { return sizeof value; } IntNode & operator=(int rhs) { From dcec086573f76f76afa936d95c1f26349b011781 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Nov 2021 23:17:22 +0300 Subject: [PATCH 27/49] Small refactoring --- src/Coordination/Changelog.h | 2 +- src/Coordination/CoordinationSettings.cpp | 34 +-- src/Coordination/CoordinationSettings.h | 12 +- src/Coordination/FourLetterCommand.cpp | 197 +++++++++--------- src/Coordination/FourLetterCommand.h | 115 ++++++---- src/Coordination/KeeperConnectionStats.cpp | 85 ++++++++ src/Coordination/KeeperConnectionStats.h | 52 +++++ src/Coordination/KeeperDispatcher.cpp | 123 ++++++----- src/Coordination/KeeperDispatcher.h | 64 +++--- src/Coordination/KeeperInfos.h | 137 +++--------- src/Coordination/KeeperServer.cpp | 51 ++--- src/Coordination/KeeperServer.h | 18 +- src/Coordination/KeeperStateMachine.cpp | 34 +-- src/Coordination/KeeperStateMachine.h | 25 +-- src/Coordination/KeeperStateManager.cpp | 4 +- src/Coordination/KeeperStats.cpp | 97 --------- src/Coordination/KeeperStats.h | 57 ----- src/Coordination/KeeperStorage.cpp | 81 +++++-- src/Coordination/KeeperStorage.h | 66 +++--- src/Coordination/SnapshotableHashTable.h | 12 +- src/Coordination/tests/gtest_coordination.cpp | 2 +- src/Server/KeeperTCPHandler.cpp | 118 +++++------ src/Server/KeeperTCPHandler.h | 26 +-- src/Server/KeeperTCPHandlerFactory.h | 2 + .../configs/use_keeper.xml | 12 -- .../test_keeper_four_word_command/test.py | 19 +- 26 files changed, 698 insertions(+), 747 deletions(-) create mode 100644 src/Coordination/KeeperConnectionStats.cpp create mode 100644 src/Coordination/KeeperConnectionStats.h delete mode 100644 src/Coordination/KeeperStats.cpp delete mode 100644 src/Coordination/KeeperStats.h delete mode 100644 tests/integration/test_keeper_four_word_command/configs/use_keeper.xml diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index edca61a883c..e343b560aae 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -11,7 +11,7 @@ namespace DB { -using Checksum = UInt64; +using Checksum = uint64_t; using LogEntryPtr = nuraft::ptr; using LogEntries = std::vector; diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 21e42c0be8e..bb160c98402 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -37,9 +37,9 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco } -const String KeeperSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro"; +const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro"; -KeeperSettings::KeeperSettings() +KeeperConfigurationAndSettings::KeeperConfigurationAndSettings() : server_id(NOT_EXIST) , tcp_port(NOT_EXIST) , tcp_port_secure(NOT_EXIST) @@ -49,9 +49,9 @@ KeeperSettings::KeeperSettings() } -void KeeperSettings::dump(WriteBufferFromOwnString & buf) const +void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const { - auto write_int = [&buf](Int64 value) + auto write_int = [&buf](int64_t value) { writeIntText(value, buf); buf.write('\n'); @@ -95,18 +95,18 @@ void KeeperSettings::dump(WriteBufferFromOwnString & buf) const writeText("max_requests_batch_size=", buf); write_int(coordination_settings->max_requests_batch_size); writeText("session_timeout_ms=", buf); - write_int(UInt64(coordination_settings->session_timeout_ms)); + write_int(uint64_t(coordination_settings->session_timeout_ms)); writeText("operation_timeout_ms=", buf); - write_int(UInt64(coordination_settings->operation_timeout_ms)); + write_int(uint64_t(coordination_settings->operation_timeout_ms)); writeText("dead_session_check_period_ms=", buf); - write_int(UInt64(coordination_settings->dead_session_check_period_ms)); + write_int(uint64_t(coordination_settings->dead_session_check_period_ms)); writeText("heart_beat_interval_ms=", buf); - write_int(UInt64(coordination_settings->heart_beat_interval_ms)); + write_int(uint64_t(coordination_settings->heart_beat_interval_ms)); writeText("election_timeout_lower_bound_ms=", buf); - write_int(UInt64(coordination_settings->election_timeout_lower_bound_ms)); + write_int(uint64_t(coordination_settings->election_timeout_lower_bound_ms)); writeText("election_timeout_upper_bound_ms=", buf); - write_int(UInt64(coordination_settings->election_timeout_upper_bound_ms)); + write_int(uint64_t(coordination_settings->election_timeout_upper_bound_ms)); writeText("reserved_log_items=", buf); write_int(coordination_settings->reserved_log_items); @@ -116,9 +116,9 @@ void KeeperSettings::dump(WriteBufferFromOwnString & buf) const writeText("auto_forwarding=", buf); write_bool(coordination_settings->auto_forwarding); writeText("shutdown_timeout=", buf); - write_int(UInt64(coordination_settings->shutdown_timeout)); + write_int(uint64_t(coordination_settings->shutdown_timeout)); writeText("startup_timeout=", buf); - write_int(UInt64(coordination_settings->startup_timeout)); + write_int(uint64_t(coordination_settings->startup_timeout)); writeText("raft_logs_level=", buf); writeText(coordination_settings->raft_logs_level.toString(), buf); @@ -148,10 +148,10 @@ void KeeperSettings::dump(WriteBufferFromOwnString & buf) const write_int(coordination_settings->configuration_change_tries_count); } -std::shared_ptr -KeeperSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) +KeeperConfigurationAndSettingsPtr +KeeperConfigurationAndSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) { - std::shared_ptr ret = std::make_shared(); + std::shared_ptr ret = std::make_shared(); ret->server_id = config.getInt("keeper_server.server_id"); ret->standalone_keeper = standalone_keeper_; @@ -179,7 +179,7 @@ KeeperSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, return ret; } -String KeeperSettings::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) +String KeeperConfigurationAndSettings::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) { /// the most specialized path if (config.has("keeper_server.log_storage_path")) @@ -194,7 +194,7 @@ String KeeperSettings::getLogsPathFromConfig(const Poco::Util::AbstractConfigura return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs"; } -String KeeperSettings::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) +String KeeperConfigurationAndSettings::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) { /// the most specialized path if (config.has("keeper_server.snapshot_storage_path")) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 0e4784439a6..0bac31853e6 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -52,13 +52,15 @@ struct CoordinationSettings : public BaseSettings using CoordinationSettingsPtr = std::shared_ptr; -/// encapsulation of keeper settings from keeper_server -struct KeeperSettings +/// Coordination settings + some other parts of keeper configuration +/// which are not stored in settings. Allows to dump configuration +/// with four letter word commands. +struct KeeperConfigurationAndSettings { static constexpr int NOT_EXIST = -1; static const String DEFAULT_FOUR_LETTER_WORD_CMD; - KeeperSettings(); + KeeperConfigurationAndSettings(); int server_id; int tcp_port; @@ -75,13 +77,13 @@ struct KeeperSettings String snapshot_storage_path; void dump(WriteBufferFromOwnString & buf) const; - static std::shared_ptr loadFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); + static std::shared_ptr loadFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); private: static String getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); static String getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); }; -using KeeperSettingsPtr = std::shared_ptr; +using KeeperConfigurationAndSettingsPtr = std::shared_ptr; } diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 6eab502743d..3250ab197f3 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -17,24 +17,25 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -IFourLetterCommand::IFourLetterCommand(const KeeperDispatcher & keeper_dispatcher_) : keeper_dispatcher(keeper_dispatcher_) +IFourLetterCommand::IFourLetterCommand(KeeperDispatcher & keeper_dispatcher_) + : keeper_dispatcher(keeper_dispatcher_) { } -Int32 IFourLetterCommand::code() +int32_t IFourLetterCommand::code() { return toCode(name()); } -String IFourLetterCommand::toName(Int32 code) +String IFourLetterCommand::toName(int32_t code) { int reverted_code = __builtin_bswap32(code); return String(reinterpret_cast(&reverted_code), 4); } -Int32 IFourLetterCommand::toCode(const String & name) +int32_t IFourLetterCommand::toCode(const String & name) { - Int32 res = *reinterpret_cast(name.data()); + int32_t res = *reinterpret_cast(name.data()); /// keep consistent with Coordination::read method by changing big endian to little endian. return __builtin_bswap32(res); } @@ -55,13 +56,13 @@ void FourLetterCommandFactory::checkInitialization() const } } -bool FourLetterCommandFactory::isKnown(Int32 code) +bool FourLetterCommandFactory::isKnown(int32_t code) { checkInitialization(); return commands.contains(code); } -FourLetterCommandPtr FourLetterCommandFactory::get(Int32 code) +FourLetterCommandPtr FourLetterCommandFactory::get(int32_t code) { checkInitialization(); return commands.at(code); @@ -70,13 +71,12 @@ FourLetterCommandPtr FourLetterCommandFactory::get(Int32 code) void FourLetterCommandFactory::registerCommand(FourLetterCommandPtr & command) { if (commands.contains(command->code())) - { throw Exception("Four letter command " + command->name() + " already registered", ErrorCodes::LOGICAL_ERROR); - } + commands.emplace(command->code(), std::move(command)); } -void FourLetterCommandFactory::registerCommands(const KeeperDispatcher & keeper_dispatcher) +void FourLetterCommandFactory::registerCommands(KeeperDispatcher & keeper_dispatcher) { FourLetterCommandFactory & factory = FourLetterCommandFactory::instance(); @@ -132,20 +132,19 @@ void FourLetterCommandFactory::registerCommands(const KeeperDispatcher & keeper_ } } -bool FourLetterCommandFactory::isEnabled(Int32 code) +bool FourLetterCommandFactory::isEnabled(int32_t code) { checkInitialization(); if (!white_list.empty() && *white_list.cbegin() == WHITE_LIST_ALL) - { return true; - } + return std::find(white_list.begin(), white_list.end(), code) != white_list.end(); } -void FourLetterCommandFactory::initializeWhiteList(const KeeperDispatcher & keeper_dispatcher) +void FourLetterCommandFactory::initializeWhiteList(KeeperDispatcher & keeper_dispatcher) { using Poco::StringTokenizer; - const auto & keeper_settings = keeper_dispatcher.getKeeperSettings(); + const auto & keeper_settings = keeper_dispatcher.getKeeperConfigurationAndSettings(); String list_str = keeper_settings->four_letter_word_white_list; StringTokenizer tokenizer(list_str, ",", 2); @@ -179,52 +178,10 @@ String RuokCommand::run() return "imok"; } -String MonitorCommand::run() +namespace { - KeeperStatsPtr stats = keeper_dispatcher.getKeeperStats(); - const IKeeperInfo & keeper_info = keeper_dispatcher.getKeeperInfo(); - if (!keeper_info.hasLeader()) - { - return "This instance is not currently serving requests"; - } - - const IRaftInfo & raft_info = keeper_dispatcher.getRaftInfo(); - const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); - - StringBuffer ret; - print(ret, "version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); - - print(ret, "avg_latency", stats->getAvgLatency()); - print(ret, "max_latency", stats->getMaxLatency()); - print(ret, "min_latency", stats->getMinLatency()); - print(ret, "packets_received", stats->getPacketsReceived()); - print(ret, "packets_sent", stats->getPacketsSent()); - - print(ret, "num_alive_connections", keeper_info.getNumAliveConnections()); - print(ret, "outstanding_requests", keeper_info.getOutstandingRequests()); - print(ret, "server_state", keeper_info.getRole()); - - print(ret, "znode_count", state_machine.getNodeCount()); - print(ret, "watch_count", state_machine.getWatchCount()); - print(ret, "ephemerals_count", state_machine.getEphemeralNodeCount()); - print(ret, "approximate_data_size", state_machine.getApproximateDataSize()); - -#if defined(__linux__) || defined(__APPLE__) - print(ret, "open_file_descriptor_count", getCurrentProcessFDCount()); - print(ret, "max_file_descriptor_count", getMaxFileDescriptorCount()); -#endif - - if (raft_info.isLeader()) - { - print(ret, "followers", raft_info.getFollowerCount()); - print(ret, "synced_followers", raft_info.getSyncedFollowerCount()); - } - - return ret.str(); -} - -void MonitorCommand::print(IFourLetterCommand::StringBuffer & buf, const String & key, const String & value) +void print(IFourLetterCommand::StringBuffer & buf, const String & key, const String & value) { writeText("zk_", buf); writeText(key, buf); @@ -233,15 +190,60 @@ void MonitorCommand::print(IFourLetterCommand::StringBuffer & buf, const String writeText('\n', buf); } -void MonitorCommand::print(IFourLetterCommand::StringBuffer & buf, const String & key, UInt64 value) +void print(IFourLetterCommand::StringBuffer & buf, const String & key, uint64_t value) { print(buf, key, toString(value)); } +} + +String MonitorCommand::run() +{ + KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats(); + KeeperInfo keeper_info = keeper_dispatcher.getKeeperInfo(); + + if (!keeper_info.has_leader) + return "This instance is not currently serving requests"; + + const auto & state_machine = keeper_dispatcher.getStateMachine(); + + StringBuffer ret; + print(ret, "version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); + + print(ret, "avg_latency", stats.getAvgLatency()); + print(ret, "max_latency", stats.getMaxLatency()); + print(ret, "min_latency", stats.getMinLatency()); + print(ret, "packets_received", stats.getPacketsReceived()); + print(ret, "packets_sent", stats.getPacketsSent()); + + print(ret, "num_alive_connections", keeper_info.alive_connections_count); + print(ret, "outstanding_requests", keeper_info.outstanding_requests_count); + + print(ret, "server_state", keeper_info.getRole()); + + print(ret, "znode_count", state_machine.getNodesCount()); + print(ret, "watch_count", state_machine.getTotalWatchesCount()); + print(ret, "ephemerals_count", state_machine.getTotalEphemeralNodesCount()); + print(ret, "approximate_data_size", state_machine.getApproximateDataSize()); + +#if defined(__linux__) || defined(__APPLE__) + print(ret, "open_file_descriptor_count", getCurrentProcessFDCount()); + print(ret, "max_file_descriptor_count", getMaxFileDescriptorCount()); +#endif + + if (keeper_info.is_leader) + { + print(ret, "followers", keeper_info.follower_count); + print(ret, "synced_followers", keeper_info.synced_follower_count); + } + + return ret.str(); +} + + String StatResetCommand::run() { - KeeperStatsPtr stats = keeper_dispatcher.getKeeperStats(); - stats->reset(); + keeper_dispatcher.resetConnectionStats(); return "Server stats reset."; } @@ -282,23 +284,22 @@ String ServerStatCommand::run() writeText('\n', buf); }; - KeeperStatsPtr stats = keeper_dispatcher.getKeeperStats(); - const IKeeperInfo & keeper_info = keeper_dispatcher.getKeeperInfo(); - const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); + KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats(); + KeeperInfo keeper_info = keeper_dispatcher.getKeeperInfo(); write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); StringBuffer latency; - latency << stats->getMinLatency() << "/" << stats->getAvgLatency() << "/" << stats->getMaxLatency() << "\n"; + latency << stats.getMinLatency() << "/" << stats.getAvgLatency() << "/" << stats.getMaxLatency() << "\n"; write("Latency min/avg/max", latency.str()); - write("Received", toString(stats->getPacketsReceived())); - write("Sent ", toString(stats->getPacketsSent())); - write("Connections", toString(keeper_info.getNumAliveConnections())); - write("Outstanding", toString(keeper_info.getOutstandingRequests())); - write("Zxid", toString(state_machine.getLastProcessedZxid())); + write("Received", toString(stats.getPacketsReceived())); + write("Sent ", toString(stats.getPacketsSent())); + write("Connections", toString(keeper_info.alive_connections_count)); + write("Outstanding", toString(keeper_info.outstanding_requests_count)); + write("Zxid", toString(keeper_info.last_zxid)); write("Mode", keeper_info.getRole()); - write("Node count", toString(state_machine.getNodeCount())); + write("Node count", toString(keeper_info.total_nodes_count)); return buf.str(); } @@ -307,11 +308,10 @@ String StatCommand::run() { StringBuffer buf; - auto write = [&buf](const String & key, const String & value) { buf << key << ": " << value << '\n'; }; + auto write = [&buf] (const String & key, const String & value) { buf << key << ": " << value << '\n'; }; - KeeperStatsPtr stats = keeper_dispatcher.getKeeperStats(); - const IKeeperInfo & keeper_info = keeper_dispatcher.getKeeperInfo(); - const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); + KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats(); + KeeperInfo keeper_info = keeper_dispatcher.getKeeperInfo(); write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); @@ -320,16 +320,16 @@ String StatCommand::run() buf << '\n'; StringBuffer latency; - latency << stats->getMinLatency() << "/" << stats->getAvgLatency() << "/" << stats->getMaxLatency() << "\n"; + latency << stats.getMinLatency() << "/" << stats.getAvgLatency() << "/" << stats.getMaxLatency() << "\n"; write("Latency min/avg/max", latency.str()); - write("Received", toString(stats->getPacketsReceived())); - write("Sent ", toString(stats->getPacketsSent())); - write("Connections", toString(keeper_info.getNumAliveConnections())); - write("Outstanding", toString(keeper_info.getOutstandingRequests())); - write("Zxid", toString(state_machine.getLastProcessedZxid())); + write("Received", toString(stats.getPacketsReceived())); + write("Sent ", toString(stats.getPacketsSent())); + write("Connections", toString(keeper_info.alive_connections_count)); + write("Outstanding", toString(keeper_info.outstanding_requests_count)); + write("Zxid", toString(keeper_info.last_zxid)); write("Mode", keeper_info.getRole()); - write("Node count", toString(state_machine.getNodeCount())); + write("Node count", toString(keeper_info.total_nodes_count)); return buf.str(); } @@ -337,16 +337,17 @@ String StatCommand::run() String BriefWatchCommand::run() { StringBuffer buf; - const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); - buf << keeper_dispatcher.getNumAliveConnections() << " connections watching " << state_machine.getWatchPathCount() << " paths\n"; - buf << "Total watches:" << state_machine.getWatchCount(); + const auto & state_machine = keeper_dispatcher.getStateMachine(); + buf << state_machine.getSessionsWithWatchesCount() << " connections watching " + << state_machine.getWatchedPathsCount() << " paths\n"; + buf << "Total watches:" << state_machine.getTotalWatchesCount(); return buf.str(); } String WatchCommand::run() { StringBuffer buf; - const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); + const auto & state_machine = keeper_dispatcher.getStateMachine(); state_machine.dumpWatches(buf); return buf.str(); } @@ -354,7 +355,7 @@ String WatchCommand::run() String WatchByPathCommand::run() { StringBuffer buf; - const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); + const auto & state_machine = keeper_dispatcher.getStateMachine(); state_machine.dumpWatchesByPath(buf); return buf.str(); } @@ -363,16 +364,15 @@ String DataSizeCommand::run() { StringBuffer buf; buf << "snapshot_dir_size: " << keeper_dispatcher.getSnapDirSize() << '\n'; - buf << "log_dir_size: " << keeper_dispatcher.getDataDirSize() << '\n'; + buf << "log_dir_size: " << keeper_dispatcher.getLogDirSize() << '\n'; return buf.str(); } String DumpCommand::run() { StringBuffer buf; - const IStateMachineInfo & state_machine = keeper_dispatcher.getStateMachineInfo(); - keeper_dispatcher.dumpSessions(buf); - state_machine.dumpEphemerals(buf); + const auto & state_machine = keeper_dispatcher.getStateMachine(); + state_machine.dumpSessionsAndEphemerals(buf); return buf.str(); } @@ -391,9 +391,14 @@ String EnviCommand::run() buf << "os.version=" << Environment::osVersion() << '\n'; buf << "cpu.count=" << Environment::processorCount() << '\n'; - char user_name[128]; - getlogin_r(user_name, 128); - buf << "user.name=" << user_name << '\n'; + String os_user; + os_user.resize(256, '\0'); + if (0 == getlogin_r(os_user.data(), os_user.size() - 1)) + os_user.resize(strlen(os_user.c_str())); + else + os_user.clear(); /// Don't mind if we cannot determine user login. + + buf << "user.name=" << os_user << '\n'; buf << "user.home=" << Path::home() << '\n'; buf << "user.dir=" << Path::current() << '\n'; @@ -404,7 +409,7 @@ String EnviCommand::run() String IsReadOnlyCommand::run() { - if (keeper_dispatcher.getRole() == "observer") + if (keeper_dispatcher.isObserver()) return "ro"; else return "rw"; diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 59ce62c0772..545e20c4f54 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -3,7 +3,7 @@ #include #include #include -#include + #include #include #include @@ -24,48 +24,48 @@ struct IFourLetterCommand { public: using StringBuffer = DB::WriteBufferFromOwnString; - explicit IFourLetterCommand(const KeeperDispatcher & keeper_dispatcher_); + explicit IFourLetterCommand(KeeperDispatcher & keeper_dispatcher_); virtual String name() = 0; virtual String run() = 0; virtual ~IFourLetterCommand(); - Int32 code(); + int32_t code(); - static String toName(Int32 code); - static inline Int32 toCode(const String & name); + static String toName(int32_t code); + static inline int32_t toCode(const String & name); protected: - const KeeperDispatcher & keeper_dispatcher; + KeeperDispatcher & keeper_dispatcher; }; struct FourLetterCommandFactory : private boost::noncopyable { public: - using Commands = std::unordered_map; - using WhiteList = std::vector; + using Commands = std::unordered_map; + using WhiteList = std::vector; ///represent '*' which is used in white list - static constexpr Int32 WHITE_LIST_ALL = 0; + static constexpr int32_t WHITE_LIST_ALL = 0; - bool isKnown(Int32 code); - bool isEnabled(Int32 code); + bool isKnown(int32_t code); + bool isEnabled(int32_t code); - FourLetterCommandPtr get(Int32 code); + FourLetterCommandPtr get(int32_t code); /// There is no need to make it thread safe, because registration is no initialization and get is after startup. void registerCommand(FourLetterCommandPtr & command); - void initializeWhiteList(const KeeperDispatcher & keeper_dispatcher); + void initializeWhiteList(KeeperDispatcher & keeper_dispatcher); void checkInitialization() const; bool isInitialized() const { return initialized; } void setInitialize(bool flag) { initialized = flag; } static FourLetterCommandFactory & instance(); - static void registerCommands(const KeeperDispatcher & keeper_dispatcher); + static void registerCommands(KeeperDispatcher & keeper_dispatcher); private: - volatile bool initialized = false; + std::atomic initialized = false; Commands commands; WhiteList white_list; }; @@ -79,14 +79,15 @@ private: */ struct RuokCommand : public IFourLetterCommand { - explicit RuokCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit RuokCommand(KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } String name() override { return "ruok"; } String run() override; ~RuokCommand() override = default; }; -/**Outputs a list of variables that could be used for monitoring the health of the cluster. +/** + * Outputs a list of variables that could be used for monitoring the health of the cluster. * * echo mntr | nc localhost 2181 * zk_version 3.5.9 @@ -109,20 +110,22 @@ struct RuokCommand : public IFourLetterCommand */ struct MonitorCommand : public IFourLetterCommand { - explicit MonitorCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit MonitorCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "mntr"; } String run() override; ~MonitorCommand() override = default; - -private: - static void print(StringBuffer & buf, const String & key, const String & value); - static void print(StringBuffer & buf, const String & key, UInt64 value); }; struct StatResetCommand : public IFourLetterCommand { - explicit StatResetCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit StatResetCommand(KeeperDispatcher & keeper_dispatcher_) : + IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "srst"; } String run() override; @@ -133,7 +136,10 @@ struct StatResetCommand : public IFourLetterCommand ///It is used to inform clients who execute none white listed four letter word commands. struct NopCommand : public IFourLetterCommand { - explicit NopCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit NopCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "nopc"; } String run() override; @@ -142,7 +148,10 @@ struct NopCommand : public IFourLetterCommand struct ConfCommand : public IFourLetterCommand { - explicit ConfCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit ConfCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "conf"; } String run() override; @@ -153,7 +162,10 @@ struct ConfCommand : public IFourLetterCommand /// Includes information on numbers of packets received/sent, session id, operation latencies, last operation performed, etc... struct ConsCommand : public IFourLetterCommand { - explicit ConsCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit ConsCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "cons"; } String run() override; @@ -163,7 +175,10 @@ struct ConsCommand : public IFourLetterCommand /// Reset connection/session statistics for all connections. struct RestConnStatsCommand : public IFourLetterCommand { - explicit RestConnStatsCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit RestConnStatsCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "crst"; } String run() override; @@ -173,7 +188,10 @@ struct RestConnStatsCommand : public IFourLetterCommand /// Lists full details for the server. struct ServerStatCommand : public IFourLetterCommand { - explicit ServerStatCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit ServerStatCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "srvr"; } String run() override; @@ -183,7 +201,10 @@ struct ServerStatCommand : public IFourLetterCommand /// Lists brief details for the server and connected clients. struct StatCommand : public IFourLetterCommand { - explicit StatCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit StatCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "stat"; } String run() override; @@ -193,7 +214,10 @@ struct StatCommand : public IFourLetterCommand /// Lists brief information on watches for the server. struct BriefWatchCommand : public IFourLetterCommand { - explicit BriefWatchCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit BriefWatchCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "wchs"; } String run() override; @@ -205,7 +229,10 @@ struct BriefWatchCommand : public IFourLetterCommand /// Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. struct WatchCommand : public IFourLetterCommand { - explicit WatchCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit WatchCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "wchc"; } String run() override; @@ -217,7 +244,10 @@ struct WatchCommand : public IFourLetterCommand /// Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. struct WatchByPathCommand : public IFourLetterCommand { - explicit WatchByPathCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit WatchByPathCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "wchp"; } String run() override; @@ -227,7 +257,10 @@ struct WatchByPathCommand : public IFourLetterCommand /// Lists the outstanding sessions and ephemeral nodes. This only works on the leader. struct DumpCommand : public IFourLetterCommand { - explicit DumpCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit DumpCommand(KeeperDispatcher & keeper_dispatcher_): + IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "dump"; } String run() override; @@ -237,7 +270,10 @@ struct DumpCommand : public IFourLetterCommand /// Print details about serving environment struct EnviCommand : public IFourLetterCommand { - explicit EnviCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit EnviCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "envi"; } String run() override; @@ -247,7 +283,10 @@ struct EnviCommand : public IFourLetterCommand /// Shows the total size of snapshot and log files in bytes struct DataSizeCommand : public IFourLetterCommand { - explicit DataSizeCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit DataSizeCommand(KeeperDispatcher & keeper_dispatcher_): + IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "dirs"; } String run() override; @@ -258,12 +297,14 @@ struct DataSizeCommand : public IFourLetterCommand /// The server will respond with "ro" if in read-only mode or "rw" if not in read-only mode. struct IsReadOnlyCommand : public IFourLetterCommand { - explicit IsReadOnlyCommand(const KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } + explicit IsReadOnlyCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } String name() override { return "isro"; } String run() override; ~IsReadOnlyCommand() override = default; }; - } diff --git a/src/Coordination/KeeperConnectionStats.cpp b/src/Coordination/KeeperConnectionStats.cpp new file mode 100644 index 00000000000..f3938b395b5 --- /dev/null +++ b/src/Coordination/KeeperConnectionStats.cpp @@ -0,0 +1,85 @@ +#include + +namespace DB +{ + +uint64_t KeeperConnectionStats::getMinLatency() const +{ + return min_latency; +} + +uint64_t KeeperConnectionStats::getMaxLatency() const +{ + return max_latency; +} + +uint64_t KeeperConnectionStats::getAvgLatency() const +{ + if (count != 0) + return total_latency / count; + return 0; +} + +uint64_t KeeperConnectionStats::getLastLatency() const +{ + return last_latency; +} + +uint64_t KeeperConnectionStats::getPacketsReceived() const +{ + return packets_received; +} + +uint64_t KeeperConnectionStats::getPacketsSent() const +{ + return packets_sent; +} + +void KeeperConnectionStats::incrementPacketsReceived() +{ + packets_received++; +} + +void KeeperConnectionStats::incrementPacketsSent() +{ + packets_sent++; +} + +void KeeperConnectionStats::updateLatency(uint64_t latency_ms) +{ + last_latency = latency_ms; + total_latency += (latency_ms); + count++; + + if (latency_ms < min_latency) + { + min_latency = latency_ms; + } + + if (latency_ms > max_latency) + { + max_latency = latency_ms; + } +} + +void KeeperConnectionStats::reset() +{ + resetLatency(); + resetRequestCounters(); +} + +void KeeperConnectionStats::resetLatency() +{ + total_latency = 0; + count = 0; + max_latency = 0; + min_latency = 0; +} + +void KeeperConnectionStats::resetRequestCounters() +{ + packets_received = 0; + packets_sent = 0; +} + +} diff --git a/src/Coordination/KeeperConnectionStats.h b/src/Coordination/KeeperConnectionStats.h new file mode 100644 index 00000000000..3cd881d553b --- /dev/null +++ b/src/Coordination/KeeperConnectionStats.h @@ -0,0 +1,52 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// Request statistics for connection or dispatcher +class KeeperConnectionStats +{ +public: + KeeperConnectionStats() = default; + + uint64_t getMinLatency() const; + uint64_t getMaxLatency() const; + + uint64_t getAvgLatency() const; + uint64_t getLastLatency() const; + + uint64_t getPacketsReceived() const; + uint64_t getPacketsSent() const; + + void incrementPacketsReceived(); + void incrementPacketsSent(); + + void updateLatency(uint64_t latency_ms); + void reset(); + +private: + void resetLatency(); + void resetRequestCounters(); + + /// all response with watch response included + uint64_t packets_sent = 0; + /// All user requests + uint64_t packets_received = 0; + + /// For consistent with zookeeper measured by millisecond, + /// otherwise maybe microsecond is better + uint64_t total_latency = 0; + uint64_t max_latency = 0; + uint64_t min_latency = 0; + + /// last operation latency + uint64_t last_latency = 0; + + uint64_t count = 0; +}; + +} diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index f6a4ef7f369..fbbd5222a75 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -19,35 +20,10 @@ namespace ErrorCodes extern const int SYSTEM_ERROR; } -using Path = fs::path; -using DirectoryIterator = fs::directory_iterator; - -UInt64 getDirSize(Path dir) -{ - if (!fs::exists(dir)) - { - return 0; - } - - DirectoryIterator it(dir); - DirectoryIterator end; - - UInt64 size{0}; - while (it != end) - { - if (!it->is_regular_file()) - size += fs::file_size(*it); - else - size += getDirSize(it->path()); - ++it; - } - return size; -} KeeperDispatcher::KeeperDispatcher() : responses_queue(std::numeric_limits::max()) - , keeper_stats(std::make_shared()) - , settings(std::make_shared()) + , configuration_and_settings(std::make_shared()) , log(&Poco::Logger::get("KeeperDispatcher")) { } @@ -67,8 +43,9 @@ void KeeperDispatcher::requestThread() { KeeperStorage::RequestForSession request; - UInt64 max_wait = UInt64(settings->coordination_settings->operation_timeout_ms.totalMilliseconds()); - uint64_t max_batch_size = settings->coordination_settings->max_requests_batch_size; + auto coordination_settings = configuration_and_settings->coordination_settings; + uint64_t max_wait = coordination_settings->operation_timeout_ms.totalMilliseconds(); + uint64_t max_batch_size = coordination_settings->max_requests_batch_size; /// The code below do a very simple thing: batch all write (quorum) requests into vector until /// previous write batch is not finished or max_batch size achieved. The main complexity goes from @@ -89,7 +66,7 @@ void KeeperDispatcher::requestThread() /// If new request is not read request or we must to process it through quorum. /// Otherwise we will process it locally. - if (settings->coordination_settings->quorum_reads || !request.request->isReadRequest()) + if (coordination_settings->quorum_reads || !request.request->isReadRequest()) { current_batch.emplace_back(request); @@ -102,7 +79,7 @@ void KeeperDispatcher::requestThread() if (requests_queue->tryPop(request, 1)) { /// Don't append read request into batch, we have to process them separately - if (!settings->coordination_settings->quorum_reads && request.request->isReadRequest()) + if (!coordination_settings->quorum_reads && request.request->isReadRequest()) { has_read_request = true; break; @@ -172,7 +149,7 @@ void KeeperDispatcher::responseThread() { KeeperStorage::ResponseForSession response_for_session; - UInt64 max_wait = UInt64(settings->coordination_settings->operation_timeout_ms.totalMilliseconds()); + uint64_t max_wait = configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds(); if (responses_queue.tryPop(response_for_session, max_wait)) { @@ -273,7 +250,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ if (!requests_queue->push(std::move(request_info))) throw Exception("Cannot push request to queue", ErrorCodes::SYSTEM_ERROR); } - else if (!requests_queue->tryPush(std::move(request_info), settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) + else if (!requests_queue->tryPush(std::move(request_info), configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) { throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); } @@ -284,14 +261,14 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf { LOG_DEBUG(log, "Initializing storage dispatcher"); - settings = KeeperSettings::loadFromConfig(config, standalone_keeper); - requests_queue = std::make_unique(settings->coordination_settings->max_requests_batch_size); + configuration_and_settings = KeeperConfigurationAndSettings::loadFromConfig(config, standalone_keeper); + requests_queue = std::make_unique(configuration_and_settings->coordination_settings->max_requests_batch_size); request_thread = ThreadFromGlobalPool([this] { requestThread(); }); responses_thread = ThreadFromGlobalPool([this] { responseThread(); }); snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); }); - server = std::make_unique(settings, config, responses_queue, snapshots_queue); + server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue); try { @@ -438,7 +415,8 @@ void KeeperDispatcher::sessionCleanerTask() tryLogCurrentException(__PRETTY_FUNCTION__); } - std::this_thread::sleep_for(std::chrono::milliseconds(settings->coordination_settings->dead_session_check_period_ms.totalMilliseconds())); + auto time_to_sleep = configuration_and_settings->coordination_settings->dead_session_check_period_ms.totalMilliseconds(); + std::this_thread::sleep_for(std::chrono::milliseconds(time_to_sleep)); } } @@ -605,52 +583,71 @@ void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfigurati } } -void KeeperDispatcher::updateKeeperStat(UInt64 process_time_ms) +void KeeperDispatcher::updateKeeperStatLatency(uint64_t process_time_ms) { - keeper_stats->updateLatency(process_time_ms); + std::lock_guard lock(keeper_stats_mutex); + keeper_stats.updateLatency(process_time_ms); } -String KeeperDispatcher::getRole() const +static uint64_t getDirSize(const fs::path & dir) { - return server->getRole(); + checkStackSize(); + if (!fs::exists(dir)) + return 0; + + fs::directory_iterator it(dir); + fs::directory_iterator end; + + uint64_t size{0}; + while (it != end) + { + if (it->is_regular_file()) + size += fs::file_size(*it); + else + size += getDirSize(it->path()); + ++it; + } + return size; } -UInt64 KeeperDispatcher::getOutstandingRequests() const +uint64_t KeeperDispatcher::getLogDirSize() const { - std::lock_guard lock(push_request_mutex); - return requests_queue->size(); + return getDirSize(configuration_and_settings->log_storage_path); } -UInt64 KeeperDispatcher::getNumAliveConnections() const +uint64_t KeeperDispatcher::getSnapDirSize() const { - std::lock_guard lock(session_to_response_callback_mutex); - return session_to_response_callback.size(); -} - -UInt64 KeeperDispatcher::getDataDirSize() const -{ - return getDirSize(Path(settings->log_storage_path)); -} - -UInt64 KeeperDispatcher::getSnapDirSize() const -{ - return getDirSize(Path(settings->snapshot_storage_path)); + return getDirSize(configuration_and_settings->snapshot_storage_path); } void KeeperDispatcher::dumpConf(WriteBufferFromOwnString & buf) const { - settings->dump(buf); + configuration_and_settings->dump(buf); } -void KeeperDispatcher::dumpSessions(WriteBufferFromOwnString & buf) const +KeeperInfo KeeperDispatcher::getKeeperInfo() const { - std::lock_guard lock(session_to_response_callback_mutex); - buf << "Sessions dump (" << session_to_response_callback.size() << "):\n"; - - for (const auto & e : session_to_response_callback) + KeeperInfo result; + result.is_leader = isLeader(); + result.is_observer = server->isObserver(); + result.is_follower = server->isFollower(); + result.has_leader = hasLeader(); { - buf << "0x" << getHexUIntLowercase(e.first) << "\n"; + std::lock_guard lock(push_request_mutex); + result.outstanding_requests_count = requests_queue->size(); } + { + std::lock_guard lock(session_to_response_callback_mutex); + result.alive_connections_count = session_to_response_callback.size(); + } + if (result.is_leader) + { + result.follower_count = server->getFollowerCount(); + result.synced_follower_count = server->getSyncedFollowerCount(); + } + result.total_nodes_count = server->getKeeperStateMachine()->getNodesCount(); + result.last_zxid = server->getKeeperStateMachine()->getLastProcessedZxid(); + return result; } } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 9e5a08a6009..e16475a238d 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -14,6 +14,7 @@ #include #include #include +#include namespace DB { @@ -21,7 +22,7 @@ using ZooKeeperResponseCallback = std::function server; - std::shared_ptr keeper_stats; + mutable std::mutex keeper_stats_mutex; + KeeperConnectionStats keeper_stats; - KeeperSettingsPtr settings; + KeeperConfigurationAndSettingsPtr configuration_and_settings; Poco::Logger * log; @@ -103,7 +105,7 @@ public: KeeperDispatcher(); /// Call shutdown - ~KeeperDispatcher() override; + ~KeeperDispatcher(); /// Initialization from config. /// standalone_keeper -- we are standalone keeper application (not inside clickhouse server) @@ -134,66 +136,66 @@ public: void finishSession(int64_t session_id); /// Invoked when a request completes. - void updateKeeperStat(UInt64 process_time_ms); + void updateKeeperStatLatency(uint64_t process_time_ms); /// Are we leader - bool isLeader() const override + bool isLeader() const { return server->isLeader(); } - bool hasLeader() const override + bool hasLeader() const { return server->isLeaderAlive(); } - /// - String getRole() const override; + bool isObserver() const + { + return server->isObserver(); + } - UInt64 getOutstandingRequests() const override; - UInt64 getNumAliveConnections() const override; + uint64_t getLogDirSize() const; - UInt64 getDataDirSize() const override; - UInt64 getSnapDirSize() const override; + uint64_t getSnapDirSize() const; /// Request statistics such as qps, latency etc. - std::shared_ptr getKeeperStats() const + KeeperConnectionStats getKeeperConnectionStats() const { + std::lock_guard lock(keeper_stats_mutex); return keeper_stats; } - const IKeeperInfo & getKeeperInfo() const - { - return *this; - } + KeeperInfo getKeeperInfo() const; - IRaftInfo & getRaftInfo() const - { - return *server; - } - - const IStateMachineInfo & getStateMachineInfo() const + const KeeperStateMachine & getStateMachine() const { return *server->getKeeperStateMachine(); } - const KeeperSettingsPtr & getKeeperSettings() const + const KeeperConfigurationAndSettingsPtr & getKeeperConfigurationAndSettings() const { - return settings; + return configuration_and_settings; } - inline void incrementPacketsSent() + void incrementPacketsSent() { - keeper_stats->incrementPacketsSent(); + std::lock_guard lock(keeper_stats_mutex); + keeper_stats.incrementPacketsSent(); } - inline void incrementPacketsReceived() + void incrementPacketsReceived() { - keeper_stats->incrementPacketsReceived(); + std::lock_guard lock(keeper_stats_mutex); + keeper_stats.incrementPacketsReceived(); + } + + void resetConnectionStats() + { + std::lock_guard lock(keeper_stats_mutex); + keeper_stats.reset(); } void dumpConf(WriteBufferFromOwnString & buf) const; - void dumpSessions(WriteBufferFromOwnString & buf) const override; }; } diff --git a/src/Coordination/KeeperInfos.h b/src/Coordination/KeeperInfos.h index f809692c4fc..d4770e6ccc3 100644 --- a/src/Coordination/KeeperInfos.h +++ b/src/Coordination/KeeperInfos.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -13,128 +12,40 @@ namespace DB { -struct LastOp +namespace ErrorCodes { -private: - String last_op{"NA"}; - Int64 last_cxid{-1}; - Int64 last_zxid{-1}; - Int64 last_response_time{0}; - Int64 last_latency{0}; - -public: - inline void update(String last_op_, Int64 last_cxid_, Int64 last_zxid_, Int64 last_response_time_, Int64 last_latency_) - { - last_op = last_op_; - last_cxid = last_cxid_; - last_zxid = last_zxid_; - last_response_time = last_response_time_; - last_latency = last_latency_; - } - - inline LastOp clone() { return *this; } - - inline void reset() - { - last_op = "NA"; - last_cxid = -1; - last_zxid = -1; - last_response_time = 0; - last_latency = 0; - } - - inline const String & getLastOp() const { return last_op; } - inline Int64 getLastCxid() const { return last_cxid; } - inline Int64 getLastZxid() const { return last_zxid; } - inline Int64 getLastResponseTime() const { return last_response_time; } - inline Int64 getLastLatency() const { return last_latency; } -}; - -class IConnectionInfo -{ -public: - virtual Int64 getPacketsReceived() const = 0; - virtual Int64 getPacketsSent() const = 0; - virtual Int64 getSessionId() const = 0; - virtual Int64 getSessionTimeout() const = 0; - /// startup time - virtual Poco::Timestamp getEstablished() const = 0; - virtual LastOp getLastOp() const = 0; - virtual KeeperStatsPtr getSessionStats() const = 0; - - virtual ~IConnectionInfo() = default; -}; + extern const int LOGICAL_ERROR; +} /// Keeper server related information -class IKeeperInfo +struct KeeperInfo { -public: - virtual bool isLeader() const = 0; - virtual bool hasLeader() const = 0; + bool is_leader; + bool is_observer; + bool is_follower; - /// "leader", "follower", "observer" - virtual String getRole() const = 0; + bool has_leader; - /// number alive connections of this node - virtual UInt64 getNumAliveConnections() const = 0; + uint64_t alive_connections_count; + uint64_t outstanding_requests_count; - /// number of requests in queue - virtual UInt64 getOutstandingRequests() const = 0; + uint64_t follower_count; + uint64_t synced_follower_count; - /// log dir size - virtual UInt64 getDataDirSize() const = 0; - /// snapshot dir size - virtual UInt64 getSnapDirSize() const = 0; + uint64_t total_nodes_count; + int64_t last_zxid; - /// dump session list connection to the node - virtual void dumpSessions(WriteBufferFromOwnString & buf) const = 0; + String getRole() const + { + if (is_leader) + return "leader"; + if (is_observer) + return "observer"; + if (is_follower) + return "follower"; - virtual ~IKeeperInfo() = default; -}; - -/// Keeper state machine related info -class IStateMachineInfo -{ -public: - /// last committed zxid - virtual UInt64 getLastProcessedZxid() const = 0; - - virtual UInt64 getNodeCount() const = 0; - virtual UInt64 getWatchCount() const = 0; - virtual UInt64 getWatchPathCount() const = 0; - /// session count who has ephemeral nodes - virtual UInt64 getEphemeralCount() const = 0; - virtual UInt64 getEphemeralNodeCount() const = 0; - - /// state machine approximate data size - virtual UInt64 getApproximateDataSize() const = 0; - virtual std::vector getDeadSessions() = 0; - - virtual void dumpWatches(WriteBufferFromOwnString & buf) const = 0; - virtual void dumpWatchesByPath(WriteBufferFromOwnString & buf) const = 0; - virtual void dumpEphemerals(WriteBufferFromOwnString & buf) const = 0; - - virtual ~IStateMachineInfo() = default; -}; - -/// Raft related info -class IRaftInfo -{ -public: - virtual bool isLeader() const = 0; - - virtual bool isLeaderAlive() const = 0; - - /// server role ignore zookeeper state "read-only" and "standalone" - virtual String getRole() const = 0; - - /// @return follower count if node is not leader return 0 - virtual UInt64 getFollowerCount() const = 0; - - /// @return synced follower count if node is not leader return 0 - virtual UInt64 getSyncedFollowerCount() const = 0; - - virtual ~IRaftInfo() = default; + throw Exception(ErrorCodes::LOGICAL_ERROR, "RAFT server has undefined state state, it's a bug"); + } }; } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 368aade830e..acce76a62e7 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -78,18 +78,18 @@ std::string checkAndGetSuperdigest(const String & user_and_digest) } KeeperServer::KeeperServer( - const KeeperSettingsPtr & settings_, + const KeeperConfigurationAndSettingsPtr & configuration_and_settings_, const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_) - : server_id(settings_->server_id) - , coordination_settings(settings_->coordination_settings) + : server_id(configuration_and_settings_->server_id) + , coordination_settings(configuration_and_settings_->coordination_settings) , state_machine(nuraft::cs_new( responses_queue_, snapshots_queue_, - settings_->snapshot_storage_path, + configuration_and_settings_->snapshot_storage_path, coordination_settings, - checkAndGetSuperdigest(settings_->super_digest))) - , state_manager(nuraft::cs_new(server_id, "keeper_server", settings_->log_storage_path, config, coordination_settings)) + checkAndGetSuperdigest(configuration_and_settings_->super_digest))) + , state_manager(nuraft::cs_new(server_id, "keeper_server", configuration_and_settings_->log_storage_path, config, coordination_settings)) , log(&Poco::Logger::get("KeeperServer")) { if (coordination_settings->quorum_reads) @@ -285,41 +285,42 @@ bool KeeperServer::isLeader() const return raft_instance->is_leader(); } + +bool KeeperServer::isObserver() const +{ + auto srv_config = state_manager->get_srv_config(); + return srv_config->is_learner(); +} + + +bool KeeperServer::isFollower() const +{ + return !isLeader() && !isObserver(); +} + bool KeeperServer::isLeaderAlive() const { return raft_instance->is_leader_alive(); } -String KeeperServer::getRole() const -{ - auto srv_config = state_manager->get_srv_config(); - if (srv_config->is_learner()) - { - /// zookeeper call read only node "observer" - return "observer"; - } - return isLeader() ? "leader" : "follower"; -} - /// TODO test whether taking failed peer in count -UInt64 KeeperServer::getFollowerCount() const +uint64_t KeeperServer::getFollowerCount() const { return raft_instance->get_peer_info_all().size(); } -UInt64 KeeperServer::getSyncedFollowerCount() const +uint64_t KeeperServer::getSyncedFollowerCount() const { - UInt64 last_log_idx = raft_instance->get_last_log_idx(); - auto followers = raft_instance->get_peer_info_all(); + uint64_t last_log_idx = raft_instance->get_last_log_idx(); + const auto followers = raft_instance->get_peer_info_all(); - size_t stale_followers = 0; + uint64_t stale_followers = 0; + const uint64_t stale_follower_gap = raft_instance->get_current_params().stale_log_gap_; for (auto & fl : followers) { - if (last_log_idx > fl.last_log_idx_ + raft_instance->get_current_params().stale_log_gap_) - { + if (last_log_idx > fl.last_log_idx_ + stale_follower_gap) stale_followers++; - } } return followers.size() - stale_followers; } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 24a8934b072..14a98146b10 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -15,7 +15,7 @@ namespace DB using RaftAppendResult = nuraft::ptr>>; -class KeeperServer : public IRaftInfo +class KeeperServer { private: const int server_id; @@ -53,7 +53,7 @@ private: public: KeeperServer( - const KeeperSettingsPtr & settings_, + const KeeperConfigurationAndSettingsPtr & settings_, const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_); @@ -72,24 +72,24 @@ public: /// Return set of the non-active sessions std::vector getDeadSessions(); - /// Return set of the non-active sessions nuraft::ptr getKeeperStateMachine() const { return state_machine; } - bool isLeader() const override; + bool isLeader() const; - bool isLeaderAlive() const override; + bool isFollower() const; - /// server role ignore zookeeper state "read-only" and "standalone" - String getRole() const override; + bool isObserver() const; + + bool isLeaderAlive() const; /// @return follower count if node is not leader return 0 - UInt64 getFollowerCount() const override; + uint64_t getFollowerCount() const; /// @return synced follower count if node is not leader return 0 - UInt64 getSyncedFollowerCount() const override; + uint64_t getSyncedFollowerCount() const; /// Wait server initialization (see callbackFunc) void waitInit(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 0938b60a58d..1ac1a584451 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -338,40 +338,46 @@ std::vector KeeperStateMachine::getDeadSessions() return storage->getDeadSessions(); } -UInt64 KeeperStateMachine::getLastProcessedZxid() const +uint64_t KeeperStateMachine::getLastProcessedZxid() const { std::lock_guard lock(storage_and_responses_lock); return storage->getZXID(); } -UInt64 KeeperStateMachine::getNodeCount() const +uint64_t KeeperStateMachine::getNodesCount() const { std::lock_guard lock(storage_and_responses_lock); - return storage->getNodeCount(); + return storage->getNodesCount(); } -UInt64 KeeperStateMachine::getWatchCount() const +uint64_t KeeperStateMachine::getTotalWatchesCount() const { std::lock_guard lock(storage_and_responses_lock); - return storage->getWatchCount(); + return storage->getTotalWatchesCount(); } -UInt64 KeeperStateMachine::getWatchPathCount() const +uint64_t KeeperStateMachine::getWatchedPathsCount() const { std::lock_guard lock(storage_and_responses_lock); - return storage->getWatchPathCount(); + return storage->getWatchedPathsCount(); } -UInt64 KeeperStateMachine::getEphemeralCount() const +uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const { std::lock_guard lock(storage_and_responses_lock); - return storage->getEphemeralCount(); + return storage->getSessionsWithWatchesCount(); } -UInt64 KeeperStateMachine::getEphemeralNodeCount() const +uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const { std::lock_guard lock(storage_and_responses_lock); - return storage->getEphemeralNodeCount(); + return storage->getTotalEphemeralNodesCount(); +} + +uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const +{ + std::lock_guard lock(storage_and_responses_lock); + return storage->getSessionWithEphemeralNodesCount(); } void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const @@ -386,13 +392,13 @@ void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const storage->dumpWatchesByPath(buf); } -void KeeperStateMachine::dumpEphemerals(WriteBufferFromOwnString & buf) const +void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const { std::lock_guard lock(storage_and_responses_lock); - storage->dumpEphemerals(buf); + storage->dumpSessionsAndEphemerals(buf); } -UInt64 KeeperStateMachine::getApproximateDataSize() const +uint64_t KeeperStateMachine::getApproximateDataSize() const { std::lock_guard lock(storage_and_responses_lock); return storage->getApproximateDataSize(); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 63e167020c2..9d8a379ac1b 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -17,7 +17,7 @@ using SnapshotsQueue = ConcurrentBoundedQueue; /// ClickHouse Keeper state machine. Wrapper for KeeperStorage. /// Responsible for entries commit, snapshots creation and so on. -class KeeperStateMachine : public nuraft::state_machine, public IStateMachineInfo +class KeeperStateMachine : public nuraft::state_machine { public: KeeperStateMachine( @@ -77,20 +77,21 @@ public: /// Process local read request void processReadRequest(const KeeperStorage::RequestForSession & request_for_session); - std::vector getDeadSessions() override; - UInt64 getLastProcessedZxid() const override; + std::vector getDeadSessions(); + uint64_t getLastProcessedZxid() const; - UInt64 getNodeCount() const override; - UInt64 getWatchCount() const override; - UInt64 getWatchPathCount() const override; + uint64_t getNodesCount() const; + uint64_t getTotalWatchesCount() const; + uint64_t getWatchedPathsCount() const; + uint64_t getSessionsWithWatchesCount() const; - void dumpWatches(WriteBufferFromOwnString & buf) const override; - void dumpWatchesByPath(WriteBufferFromOwnString & buf) const override; - void dumpEphemerals(WriteBufferFromOwnString & buf) const override; + void dumpWatches(WriteBufferFromOwnString & buf) const; + void dumpWatchesByPath(WriteBufferFromOwnString & buf) const; + void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const; - UInt64 getEphemeralCount() const override; - UInt64 getEphemeralNodeCount() const override; - UInt64 getApproximateDataSize() const override; + uint64_t getSessionWithEphemeralNodesCount() const; + uint64_t getTotalEphemeralNodesCount() const; + uint64_t getApproximateDataSize() const; void shutdownStorage(); diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 492af6be932..7c61b45ba83 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -80,7 +80,9 @@ KeeperStateManager::KeeperStateManager( , configuration_wrapper(parseServersConfiguration(config, false)) , log_store(nuraft::cs_new( log_storage_path, - coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, coordination_settings->compress_logs)) + coordination_settings->rotate_log_storage_interval, + coordination_settings->force_sync, + coordination_settings->compress_logs)) { } diff --git a/src/Coordination/KeeperStats.cpp b/src/Coordination/KeeperStats.cpp deleted file mode 100644 index e676eedf77a..00000000000 --- a/src/Coordination/KeeperStats.cpp +++ /dev/null @@ -1,97 +0,0 @@ -#include - -namespace DB -{ -UInt64 KeeperStats::getMinLatency() const -{ - std::shared_lock lock(mutex); - return min_latency; -} - -UInt64 KeeperStats::getMaxLatency() const -{ - std::shared_lock lock(mutex); - return max_latency; -} - -UInt64 KeeperStats::getAvgLatency() const -{ - std::shared_lock lock(mutex); - if (count != 0) - { - return total_latency / count; - } - return 0; -} - -UInt64 KeeperStats::getLastLatency() const -{ - std::shared_lock lock(mutex); - return last_latency; -} - -UInt64 KeeperStats::getPacketsReceived() const -{ - std::shared_lock lock(mutex); - return packets_received; -} - -UInt64 KeeperStats::getPacketsSent() const -{ - std::shared_lock lock(mutex); - return packets_sent; -} - -void KeeperStats::incrementPacketsReceived() -{ - std::unique_lock lock(mutex); - packets_received++; -} - -void KeeperStats::incrementPacketsSent() -{ - std::unique_lock lock(mutex); - packets_sent++; -} - -void KeeperStats::updateLatency(UInt64 latency_ms) -{ - std::unique_lock lock(mutex); - - last_latency = latency_ms; - total_latency += (latency_ms); - count++; - - if (latency_ms < min_latency) - { - min_latency = latency_ms; - } - - if (latency_ms > max_latency) - { - max_latency = latency_ms; - } -} - -void KeeperStats::reset() -{ - std::unique_lock lock(mutex); - resetLatency(); - resetRequestCounters(); -} - -void KeeperStats::resetLatency() -{ - total_latency = 0; - count = 0; - max_latency = 0; - min_latency = 0; -} - -void KeeperStats::resetRequestCounters() -{ - packets_received = 0; - packets_sent = 0; -} - -} diff --git a/src/Coordination/KeeperStats.h b/src/Coordination/KeeperStats.h deleted file mode 100644 index 34e0f78e517..00000000000 --- a/src/Coordination/KeeperStats.h +++ /dev/null @@ -1,57 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ -class KeeperStats; -using KeeperStatsPtr = std::shared_ptr; - -/// Request statistics for connection or dispatcher -class KeeperStats -{ -public: - explicit KeeperStats() = default; - - UInt64 getMinLatency() const; - UInt64 getMaxLatency() const; - - UInt64 getAvgLatency() const; - UInt64 getLastLatency() const; - - UInt64 getPacketsReceived() const; - UInt64 getPacketsSent() const; - - void incrementPacketsReceived(); - void incrementPacketsSent(); - - void updateLatency(UInt64 latency_ms); - void reset(); - -private: - void inline resetLatency(); - void inline resetRequestCounters(); - - mutable std::shared_mutex mutex; - - /// all response with watch response included - UInt64 packets_sent = 0; - /// All user requests - UInt64 packets_received = 0; - - /// For consistent with zookeeper measured by millisecond, - /// otherwise maybe microsecond is better - UInt64 total_latency = 0; - UInt64 max_latency = 0; - UInt64 min_latency = 0; - - /// last operation latency - UInt64 last_latency = 0; - - UInt64 count = 0; -}; - -} diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 466e653fa2f..7c55c639d94 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1223,10 +1223,10 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const { - for (const auto & e : sessions_and_watchers) + for (const auto & [session_id, watches_paths] : sessions_and_watchers) { - buf << "0x" << getHexUIntLowercase(e.first) << "\n"; - for (const String & path : e.second) + buf << "0x" << getHexUIntLowercase(session_id) << "\n"; + for (const String & path : watches_paths) { buf << "\t" << path << "\n"; } @@ -1235,43 +1235,84 @@ void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const { - auto write_int_vec = [&buf](const std::vector & objs) + auto write_int_vec = [&buf](const std::vector & session_ids) { - for (Int64 obj : objs) + for (int64_t session_id : session_ids) { - buf << "\t0x" << getHexUIntLowercase(obj) << "\n"; + buf << "\t0x" << getHexUIntLowercase(session_id) << "\n"; } }; - for (const auto & e : watches) + for (const auto & [watch_path, sessions] : watches) { - buf << e.first << "\n"; - write_int_vec(e.second); + buf << watch_path << "\n"; + write_int_vec(sessions); } - for (const auto & e : list_watches) + for (const auto & [watch_path, sessions] : list_watches) { - buf << e.first << "\n"; - write_int_vec(e.second); + buf << watch_path << "\n"; + write_int_vec(sessions); } } -void KeeperStorage::dumpEphemerals(WriteBufferFromOwnString & buf) const +void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const { - auto write_str_set = [&buf](const std::unordered_set & objs) + auto write_str_set = [&buf](const std::unordered_set & ephemeral_paths) { - for (const String & obj : objs) + for (const String & path : ephemeral_paths) { - buf << "\t" << obj << "\n"; + buf << "\t" << path << "\n"; } }; - buf << "Sessions with Ephemerals (" << getEphemeralCount() << "):\n"; - for (const auto & e : ephemerals) + buf << "Sessions dump (" << session_and_timeout.size() << "):\n"; + + for (const auto & [session_id, _] : session_and_timeout) { - buf << "0x" << getHexUIntLowercase(e.first) << "\n"; - write_str_set(e.second); + buf << "0x" << getHexUIntLowercase(session_id) << "\n"; + } + + buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCount() << "):\n"; + for (const auto & [session_id, ephemeral_paths] : ephemerals) + { + buf << "0x" << getHexUIntLowercase(session_id) << "\n"; + write_str_set(ephemeral_paths); } } +uint64_t KeeperStorage::getTotalWatchesCount() const +{ + uint64_t ret = 0; + for (const auto & [path, subscribed_sessions] : watches) + ret += subscribed_sessions.size(); + + for (const auto & [path, subscribed_sessions] : list_watches) + ret += subscribed_sessions.size(); + + return ret; +} + +uint64_t KeeperStorage::getSessionsWithWatchesCount() const +{ + std::unordered_set counter; + for (const auto & [path, subscribed_sessions] : watches) + counter.insert(subscribed_sessions.begin(), subscribed_sessions.end()); + + for (const auto & [path, subscribed_sessions] : list_watches) + counter.insert(subscribed_sessions.begin(), subscribed_sessions.end()); + + return counter.size(); +} + +uint64_t KeeperStorage::getTotalEphemeralNodesCount() const +{ + uint64_t ret = 0; + for (const auto & [session_id, nodes] : ephemerals) + ret += nodes.size(); + + return ret; +} + + } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 05f975d9213..b0052e6542d 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -29,8 +29,6 @@ struct KeeperStorageSnapshot; class KeeperStorage { public: - int64_t session_id_counter{1}; - struct Node { String data; @@ -41,14 +39,13 @@ public: ChildrenSet children{}; /// object memory size - UInt64 sizeInBytes() const + uint64_t sizeInBytes() const { - UInt64 child_size{0}; + uint64_t child_size{0}; for (const auto & child : children) - { child_size += child.size(); - } - return data.size() + sizeof (Node) + child_size; + + return data.size() + sizeof(Node) + child_size; } }; @@ -57,7 +54,6 @@ public: int64_t session_id; Coordination::ZooKeeperResponsePtr response; }; - using ResponsesForSessions = std::vector; struct RequestForSession @@ -87,10 +83,13 @@ public: /// Just vector of SHA1 from user:password using AuthIDs = std::vector; using SessionAndAuth = std::unordered_map; - SessionAndAuth session_and_auth; - using Watches = std::map; +public: + int64_t session_id_counter{1}; + + SessionAndAuth session_and_auth; + /// Main hashtable with nodes. Contain all information about data. /// All other structures expect session_and_timeout can be restored from /// container. @@ -188,53 +187,36 @@ public: return session_expiry_queue.getExpiredSessions(); } - UInt64 getNodeCount() const + /// Introspection functions mostly used in 4-letter commands + + uint64_t getNodesCount() const { return container.size(); } - UInt64 getWatchCount() const + uint64_t getApproximateDataSize() const + { + return container.getApproximateSataSize(); + } + + uint64_t getTotalWatchesCount() const; + + uint64_t getWatchedPathsCount() const { return watches.size() + list_watches.size(); } - UInt64 getWatchPathCount() const - { - UInt64 ret{0}; - for (const auto & watch : watches) - { - ret += watch.second.size(); - } - for (const auto & watch : list_watches) - { - ret += watch.second.size(); - } - return ret; - } + uint64_t getSessionsWithWatchesCount() const; - UInt64 getEphemeralCount() const + uint64_t getSessionWithEphemeralNodesCount() const { return ephemerals.size(); } - - UInt64 getEphemeralNodeCount() const - { - UInt64 ret{0}; - for (const auto & ephs : ephemerals) - { - ret += ephs.second.size(); - } - return ret; - } + uint64_t getTotalEphemeralNodesCount() const; void dumpWatches(WriteBufferFromOwnString & buf) const; void dumpWatchesByPath(WriteBufferFromOwnString & buf) const; - void dumpEphemerals(WriteBufferFromOwnString & buf) const; - - UInt64 getApproximateDataSize() const - { - return container.getApproximateSataSize(); - } + void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const; }; using KeeperStoragePtr = std::unique_ptr; diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index c88aa4598fe..4fff49ed4f8 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -29,7 +29,7 @@ private: IndexMap map; bool snapshot_mode{false}; - UInt64 approximate_data_size{0}; + uint64_t approximate_data_size{0}; enum OperationType { @@ -50,7 +50,7 @@ private: /// value_size: size of value to add /// old_value_size: size of value to minus /// old_value_size=0 means there is no old value with the same key. - void inline updateDataSize(OperationType op_type, UInt64 key_size, UInt64 value_size, UInt64 old_value_size) + void inline updateDataSize(OperationType op_type, uint64_t key_size, uint64_t value_size, uint64_t old_value_size) { switch (op_type) { @@ -132,7 +132,7 @@ public: void insertOrReplace(const std::string & key, const V & value) { auto it = map.find(key); - UInt64 old_value_size = it == map.end() ? 0 : it->second->value.sizeInBytes(); + uint64_t old_value_size = it == map.end() ? 0 : it->second->value.sizeInBytes(); if (it == map.end()) { @@ -166,7 +166,7 @@ public: return false; auto list_itr = it->second; - UInt64 old_data_size = list_itr->value.sizeInBytes(); + uint64_t old_data_size = list_itr->value.sizeInBytes(); if (snapshot_mode) { list_itr->active_in_map = false; @@ -193,7 +193,7 @@ public: assert(it != map.end()); auto list_itr = it->second; - UInt64 old_value_size = list_itr->value.sizeInBytes(); + uint64_t old_value_size = list_itr->value.sizeInBytes(); const_iterator ret; @@ -274,7 +274,7 @@ public: return list.size(); } - UInt64 getApproximateSataSize() const + uint64_t getApproximateSataSize() const { return approximate_data_size; } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index f0c2bc1f60e..c922859bd2c 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -833,7 +833,7 @@ struct IntNode { int value; IntNode(int value_) : value(value_) { } - UInt64 sizeInBytes() const { return sizeof value; } + uint64_t sizeInBytes() const { return sizeof value; } IntNode & operator=(int rhs) { this->value = rhs; diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 2d41ef31f0c..658c69fdcab 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -32,6 +32,16 @@ namespace DB { +struct LastOp +{ +public: + String name{"NA"}; + int64_t last_cxid{-1}; + int64_t last_zxid{-1}; + int64_t last_response_time{0}; +}; + +static const LastOp EMPTY_LAST_OP {"NA", -1, -1, 0}; namespace ErrorCodes { @@ -202,7 +212,7 @@ KeeperTCPHandler::KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSoc , session_timeout(0, global_context->getConfigRef().getUInt("keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) , poll_wrapper(std::make_unique(socket_)) , responses(std::make_unique(std::numeric_limits::max())) - , conn_stats(std::make_shared()) + , last_op(std::make_unique(EMPTY_LAST_OP)) { KeeperTCPHandler::registerConnection(this); } @@ -507,14 +517,21 @@ std::pair KeeperTCPHandler::receiveReque void KeeperTCPHandler::packageSent() { - conn_stats->incrementPacketsSent(); + { + std::lock_guard lock(conn_stats_mutex); + conn_stats.incrementPacketsSent(); + } keeper_dispatcher->incrementPacketsSent(); } void KeeperTCPHandler::packageReceived() { - conn_stats->incrementPacketsReceived(); + { + std::lock_guard lock(conn_stats_mutex); + conn_stats.incrementPacketsReceived(); + } keeper_dispatcher->incrementPacketsReceived(); + LOG_INFO(log, "INCREMENT PACKTEDS RECEIVED VALUE {} SESSION {}", keeper_dispatcher->getKeeperConnectionStats().getPacketsReceived(), session_id); } void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response) @@ -523,67 +540,71 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response if (response->xid != Coordination::WATCH_XID && response->getOpNum() != Coordination::OpNum::Heartbeat) { Int64 elapsed = (Poco::Timestamp() - operations[response->xid]) / 1000; - conn_stats->updateLatency(elapsed); - keeper_dispatcher->updateKeeperStat(elapsed); - { - std::lock_guard lock(last_op_mutex); - last_op.update( - Coordination::toString(response->getOpNum()), - response->xid, - response->zxid, - Poco::Timestamp().epochMicroseconds() / 1000, - elapsed); + std::lock_guard lock(conn_stats_mutex); + conn_stats.updateLatency(elapsed); } + keeper_dispatcher->updateKeeperStatLatency(elapsed); + + last_op.set(std::make_unique(LastOp{ + .name = Coordination::toString(response->getOpNum()), + .last_cxid = response->xid, + .last_zxid = response->zxid, + .last_response_time = Poco::Timestamp().epochMicroseconds() / 1000, + })); } + +} + +KeeperConnectionStats KeeperTCPHandler::getConnectionStats() const +{ + std::lock_guard lock(conn_stats_mutex); + return conn_stats; } void KeeperTCPHandler::dumpStats(WriteBufferFromOwnString & buf, bool brief) { + KeeperConnectionStats stats = getConnectionStats(); writeText(' ', buf); writeText(socket().peerAddress().toString(), buf); writeText("(recved=", buf); - writeIntText(conn_stats->getPacketsReceived(), buf); + writeIntText(stats.getPacketsReceived(), buf); writeText(",sent=", buf); - writeIntText(conn_stats->getPacketsSent(), buf); + writeIntText(stats.getPacketsSent(), buf); if (!brief) { if (session_id != 0) { writeText(",sid=0x", buf); - writeText(getHexUIntLowercase(getSessionId()), buf); + writeText(getHexUIntLowercase(session_id), buf); writeText(",lop=", buf); - LastOp op; - { - std::lock_guard lock(last_op_mutex); - op = last_op.clone(); - } - writeText(op.getLastOp(), buf); + LastOpPtr op = last_op.get(); + writeText(op->name, buf); writeText(",est=", buf); - writeIntText(getEstablished().epochMicroseconds() / 1000, buf); + writeIntText(established.epochMicroseconds() / 1000, buf); writeText(",to=", buf); - writeIntText(getSessionTimeout(), buf); - Int64 last_cxid = op.getLastCxid(); + writeIntText(session_timeout.totalMilliseconds(), buf); + int64_t last_cxid = op->last_cxid; if (last_cxid >= 0) { writeText(",lcxid=0x", buf); writeText(getHexUIntLowercase(last_cxid), buf); } writeText(",lzxid=0x", buf); - writeText(getHexUIntLowercase(op.getLastZxid()), buf); + writeText(getHexUIntLowercase(op->last_zxid), buf); writeText(",lresp=", buf); - writeIntText(op.getLastResponseTime(), buf); + writeIntText(op->last_response_time, buf); writeText(",llat=", buf); - writeIntText(conn_stats->getLastLatency(), buf); + writeIntText(stats.getLastLatency(), buf); writeText(",minlat=", buf); - writeIntText(conn_stats->getMinLatency(), buf); + writeIntText(stats.getMinLatency(), buf); writeText(",avglat=", buf); - writeIntText(conn_stats->getAvgLatency(), buf); + writeIntText(stats.getAvgLatency(), buf); writeText(",maxlat=", buf); - writeIntText(conn_stats->getMaxLatency(), buf); + writeIntText(stats.getMaxLatency(), buf); } } writeText(')', buf); @@ -592,42 +613,13 @@ void KeeperTCPHandler::dumpStats(WriteBufferFromOwnString & buf, bool brief) void KeeperTCPHandler::resetStats() { - conn_stats->reset(); - std::lock_guard lock(last_op_mutex); { - last_op.reset(); + std::lock_guard lock(conn_stats_mutex); + conn_stats.reset(); } + last_op.set(std::make_unique(EMPTY_LAST_OP)); } -Int64 KeeperTCPHandler::getPacketsReceived() const -{ - return conn_stats->getPacketsReceived(); -} -Int64 KeeperTCPHandler::getPacketsSent() const -{ - return conn_stats->getPacketsReceived(); -} -Int64 KeeperTCPHandler::getSessionId() const -{ - return session_id; -} -Int64 KeeperTCPHandler::getSessionTimeout() const -{ - return session_timeout.totalMilliseconds(); -} -Poco::Timestamp KeeperTCPHandler::getEstablished() const -{ - return established; -} -LastOp KeeperTCPHandler::getLastOp() const -{ - return last_op; -} - -KeeperStatsPtr KeeperTCPHandler::getSessionStats() const -{ - return conn_stats; -} KeeperTCPHandler::~KeeperTCPHandler() { KeeperTCPHandler::unregisterConnection(this); diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index a000b476574..8d4c1dfd83b 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -6,6 +6,7 @@ #if USE_NURAFT #include +#include #include "IServer.h" #include #include @@ -17,7 +18,7 @@ #include #include #include -#include +#include #include namespace DB @@ -27,10 +28,13 @@ struct SocketInterruptablePollWrapper; using SocketInterruptablePollWrapperPtr = std::unique_ptr; using ThreadSafeResponseQueue = ConcurrentBoundedQueue; - using ThreadSafeResponseQueuePtr = std::unique_ptr; -class KeeperTCPHandler : public Poco::Net::TCPServerConnection, IConnectionInfo +struct LastOp; +using LastOpMultiVersion = MultiVersion; +using LastOpPtr = LastOpMultiVersion::Version; + +class KeeperTCPHandler : public Poco::Net::TCPServerConnection { public: static void registerConnection(KeeperTCPHandler * conn); @@ -48,18 +52,10 @@ public: KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_); void run() override; + KeeperConnectionStats getConnectionStats() const; void dumpStats(WriteBufferFromOwnString & buf, bool brief); void resetStats(); - /// statistics methods - Int64 getPacketsReceived() const override; - Int64 getPacketsSent() const override; - Int64 getSessionId() const override; - Int64 getSessionTimeout() const override; - Poco::Timestamp getEstablished() const override; - LastOp getLastOp() const override; - KeeperStatsPtr getSessionStats() const override; - ~KeeperTCPHandler() override; private: @@ -101,10 +97,10 @@ private: using Operations = std::map; Operations operations; - std::mutex last_op_mutex; - LastOp last_op; + LastOpMultiVersion last_op; - KeeperStatsPtr conn_stats; + mutable std::mutex conn_stats_mutex; + KeeperConnectionStats conn_stats; }; diff --git a/src/Server/KeeperTCPHandlerFactory.h b/src/Server/KeeperTCPHandlerFactory.h index 82ce56804d8..67bb3dab268 100644 --- a/src/Server/KeeperTCPHandlerFactory.h +++ b/src/Server/KeeperTCPHandlerFactory.h @@ -21,6 +21,7 @@ private: using Poco::Net::TCPServerConnection::TCPServerConnection; void run() override {} }; + public: KeeperTCPHandlerFactory(IServer & server_, bool secure) : server(server_) @@ -41,6 +42,7 @@ public: return new DummyTCPHandler(socket); } } + }; } diff --git a/tests/integration/test_keeper_four_word_command/configs/use_keeper.xml b/tests/integration/test_keeper_four_word_command/configs/use_keeper.xml deleted file mode 100644 index 1e444182899..00000000000 --- a/tests/integration/test_keeper_four_word_command/configs/use_keeper.xml +++ /dev/null @@ -1,12 +0,0 @@ - - - - node2 - 9181 - - - node3 - 9181 - - - diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index ee7f5abb4e4..3679bc3cc5c 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -13,11 +13,11 @@ import csv import re cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', main_configs=['configs/enable_keeper1.xml', 'configs/use_keeper.xml'], +node1 = cluster.add_instance('node1', main_configs=['configs/enable_keeper1.xml'], stay_alive=True) -node2 = cluster.add_instance('node2', main_configs=['configs/enable_keeper2.xml', 'configs/use_keeper.xml'], +node2 = cluster.add_instance('node2', main_configs=['configs/enable_keeper2.xml'], stay_alive=True) -node3 = cluster.add_instance('node3', main_configs=['configs/enable_keeper3.xml', 'configs/use_keeper.xml'], +node3 = cluster.add_instance('node3', main_configs=['configs/enable_keeper3.xml'], stay_alive=True) from kazoo.client import KazooClient, KazooState @@ -58,7 +58,6 @@ def wait_node(node): for _ in range(100): zk = None try: - node.query("SELECT * FROM system.zookeeper WHERE path = '/'") zk = get_fake_zk(node.name, timeout=30.0) # zk.create("/test", sequence=True) print("node", node.name, "ready") @@ -206,9 +205,6 @@ def test_cmd_mntr(started_cluster): assert int(result["zk_min_latency"]) <= int(result["zk_avg_latency"]) assert int(result["zk_max_latency"]) >= int(result["zk_avg_latency"]) - assert int(result["zk_packets_received"]) == 31 - # contains 31 user request response - assert int(result["zk_packets_sent"]) == 31 assert int(result["zk_num_alive_connections"]) == 1 assert int(result["zk_outstanding_requests"]) == 0 @@ -219,7 +215,7 @@ def test_cmd_mntr(started_cluster): # 10 nodes created by test # 3 nodes created by clickhouse "/clickhouse/task_queue/ddl" # 1 root node - assert int(result["zk_znode_count"]) == 14 + assert int(result["zk_znode_count"]) == 11 assert int(result["zk_watch_count"]) == 2 assert int(result["zk_ephemerals_count"]) == 2 assert int(result["zk_approximate_data_size"]) > 0 @@ -230,6 +226,9 @@ def test_cmd_mntr(started_cluster): assert int(result["zk_followers"]) == 2 assert int(result["zk_synced_followers"]) == 2 + # contains 31 user request response and some responses for server startup + assert int(result["zk_packets_sent"]) >= 31 + assert int(result["zk_packets_received"]) >= 31 finally: destroy_zk_client(zk) @@ -353,7 +352,7 @@ def test_cmd_srvr(started_cluster): assert int(result['Connections']) == 1 assert int(result['Zxid']) > 14 assert result['Mode'] == 'leader' - assert result['Node count'] == '14' + assert result['Node count'] == '11' finally: destroy_zk_client(zk) @@ -391,7 +390,7 @@ def test_cmd_stat(started_cluster): assert int(result['Connections']) == 1 assert int(result['Zxid']) > 14 assert result['Mode'] == 'leader' - assert result['Node count'] == '14' + assert result['Node count'] == '11' # filter connection statistics cons = [n for n in data.split('\n') if '=' in n] From 33396b054e130c661980444846d597cb2441d7c3 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 19 Nov 2021 15:52:35 +0800 Subject: [PATCH 28/49] fix build error --- src/Coordination/KeeperServer.cpp | 2 +- src/Coordination/KeeperStorage.h | 2 +- src/Coordination/SnapshotableHashTable.h | 2 +- src/Coordination/tests/gtest_coordination.cpp | 36 +++++++++---------- 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index acce76a62e7..171fa2986eb 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -317,7 +317,7 @@ uint64_t KeeperServer::getSyncedFollowerCount() const uint64_t stale_followers = 0; const uint64_t stale_follower_gap = raft_instance->get_current_params().stale_log_gap_; - for (auto & fl : followers) + for (const auto & fl : followers) { if (last_log_idx > fl.last_log_idx_ + stale_follower_gap) stale_followers++; diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index b0052e6542d..5ffd1b14605 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -196,7 +196,7 @@ public: uint64_t getApproximateDataSize() const { - return container.getApproximateSataSize(); + return container.getApproximateDataSize(); } uint64_t getTotalWatchesCount() const; diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 4fff49ed4f8..f97b5e36342 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -274,7 +274,7 @@ public: return list.size(); } - uint64_t getApproximateSataSize() const + uint64_t getApproximateDataSize() const { return approximate_data_size; } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 710670f5b16..5b15044ab0d 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -940,33 +940,33 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) /// int DB::SnapshotableHashTable hello; hello.disableSnapshotMode(); - EXPECT_EQ(hello.getApproximateSataSize(), 0); + EXPECT_EQ(hello.getApproximateDataSize(), 0); hello.insert("hello", 1); - EXPECT_EQ(hello.getApproximateSataSize(), 9); + EXPECT_EQ(hello.getApproximateDataSize(), 9); hello.updateValue("hello", [](IntNode & value) { value = 2; }); - EXPECT_EQ(hello.getApproximateSataSize(), 9); + EXPECT_EQ(hello.getApproximateDataSize(), 9); hello.erase("hello"); - EXPECT_EQ(hello.getApproximateSataSize(), 0); + EXPECT_EQ(hello.getApproximateDataSize(), 0); hello.clear(); - EXPECT_EQ(hello.getApproximateSataSize(), 0); + EXPECT_EQ(hello.getApproximateDataSize(), 0); hello.enableSnapshotMode(); hello.insert("hello", 1); - EXPECT_EQ(hello.getApproximateSataSize(), 9); + EXPECT_EQ(hello.getApproximateDataSize(), 9); hello.updateValue("hello", [](IntNode & value) { value = 2; }); - EXPECT_EQ(hello.getApproximateSataSize(), 18); + EXPECT_EQ(hello.getApproximateDataSize(), 18); hello.clearOutdatedNodes(); - EXPECT_EQ(hello.getApproximateSataSize(), 9); + EXPECT_EQ(hello.getApproximateDataSize(), 9); hello.erase("hello"); - EXPECT_EQ(hello.getApproximateSataSize(), 9); + EXPECT_EQ(hello.getApproximateDataSize(), 9); hello.clearOutdatedNodes(); - EXPECT_EQ(hello.getApproximateSataSize(), 0); + EXPECT_EQ(hello.getApproximateDataSize(), 0); /// Node using Node = DB::KeeperStorage::Node; @@ -983,27 +983,27 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) /// 160 = sizeof Node /// 5 = sizeof key /// 4 = sizeof value - EXPECT_EQ(world.getApproximateSataSize(), 169); + EXPECT_EQ(world.getApproximateDataSize(), 169); world.updateValue("world", [&](Node & value) { value = n2; }); - EXPECT_EQ(world.getApproximateSataSize(), 171); + EXPECT_EQ(world.getApproximateDataSize(), 171); world.erase("world"); - EXPECT_EQ(world.getApproximateSataSize(), 0); + EXPECT_EQ(world.getApproximateDataSize(), 0); world.enableSnapshotMode(); world.insert("world", n1); - EXPECT_EQ(world.getApproximateSataSize(), 169); + EXPECT_EQ(world.getApproximateDataSize(), 169); world.updateValue("world", [&](Node & value) { value = n2; }); - EXPECT_EQ(world.getApproximateSataSize(), 340); + EXPECT_EQ(world.getApproximateDataSize(), 340); world.clearOutdatedNodes(); - EXPECT_EQ(world.getApproximateSataSize(), 171); + EXPECT_EQ(world.getApproximateDataSize(), 171); world.erase("world"); - EXPECT_EQ(world.getApproximateSataSize(), 171); + EXPECT_EQ(world.getApproximateDataSize(), 171); world.clear(); - EXPECT_EQ(world.getApproximateSataSize(), 0); + EXPECT_EQ(world.getApproximateDataSize(), 0); } void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner=0) From 3f8f08e816b4b4b29b0177c69163e2474d91962e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 12:30:58 +0300 Subject: [PATCH 29/49] Slightly better --- src/Common/getCurrentProcessFDCount.cpp | 23 ++++----- src/Common/getMaxFileDescriptorCount.cpp | 22 ++++++--- src/Coordination/CoordinationSettings.h | 2 +- src/Coordination/FourLetterCommand.cpp | 26 +++++----- src/Coordination/FourLetterCommand.h | 5 +- .../{KeeperInfos.h => Keeper4LWInfo.h} | 11 +---- src/Coordination/KeeperDispatcher.cpp | 9 +--- src/Coordination/KeeperDispatcher.h | 6 +-- src/Coordination/KeeperServer.h | 2 - src/Coordination/KeeperStateMachine.h | 11 +++-- src/Coordination/KeeperStateManager.cpp | 3 +- src/Coordination/KeeperStateManager.h | 28 +++++------ src/Coordination/KeeperStorage.cpp | 16 ++++++- src/Coordination/KeeperStorage.h | 14 +----- src/Coordination/SnapshotableHashTable.h | 4 +- src/Server/KeeperTCPHandler.cpp | 5 +- src/Server/KeeperTCPHandler.h | 1 - ...2_4letter_words_stress_zookeeper.reference | 1 + .../02122_4letter_words_stress_zookeeper.sh | 47 +++++++++++++++++++ tests/queries/shell_config.sh | 2 + 20 files changed, 140 insertions(+), 98 deletions(-) rename src/Coordination/{KeeperInfos.h => Keeper4LWInfo.h} (71%) create mode 100644 tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.reference create mode 100755 tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh diff --git a/src/Common/getCurrentProcessFDCount.cpp b/src/Common/getCurrentProcessFDCount.cpp index b85256fabb4..6471324b311 100644 --- a/src/Common/getCurrentProcessFDCount.cpp +++ b/src/Common/getCurrentProcessFDCount.cpp @@ -1,35 +1,35 @@ #include #include #include -#include #include #include #include +#include int getCurrentProcessFDCount() { + namespace fs = std::filesystem; int result = -1; #if defined(__linux__) || defined(__APPLE__) using namespace DB; Int32 pid = getpid(); - std::unique_ptr command; - /// First try procfs - String by_procfs = fmt::format("ls /proc/{}/fd | wc -l", pid); - command = ShellCommand::execute(by_procfs); - - try + auto proc_fs_path = fmt::format("/proc/{}/fd", pid); + if (fs::exists(proc_fs_path)) { - readIntText(result, command->out); - command->wait(); + result = std::distance(fs::directory_iterator(proc_fs_path), fs::directory_iterator{}); } - catch (...) + else if (fs::exists("/dev/fd")) + { + result = std::distance(fs::directory_iterator("/dev/fd"), fs::directory_iterator{}); + } + else { /// Then try lsof command String by_lsof = fmt::format("lsof -p {} | wc -l", pid); - command = ShellCommand::execute(by_procfs); + auto command = ShellCommand::execute(by_lsof); try { @@ -40,6 +40,7 @@ int getCurrentProcessFDCount() { } } + #endif return result; } diff --git a/src/Common/getMaxFileDescriptorCount.cpp b/src/Common/getMaxFileDescriptorCount.cpp index 053ac0f812d..b8a2e4a0917 100644 --- a/src/Common/getMaxFileDescriptorCount.cpp +++ b/src/Common/getMaxFileDescriptorCount.cpp @@ -1,23 +1,33 @@ #include #include -#include +#include #include #include +#include int getMaxFileDescriptorCount() { + namespace fs = std::filesystem; int result = -1; #if defined(__linux__) || defined(__APPLE__) using namespace DB; - auto command = ShellCommand::execute("ulimit -n"); - try + if (fs::exists("/proc/sys/fs/file-max")) { - readIntText(result, command->out); - command->wait(); + ReadBufferFromFile reader("/proc/sys/fs/file-max"); + readIntText(result, reader); } - catch (...) + else { + auto command = ShellCommand::execute("ulimit -n"); + try + { + readIntText(result, command->out); + command->wait(); + } + catch (...) + { + } } #endif diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 0bac31853e6..9cf566f95df 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -54,7 +54,7 @@ using CoordinationSettingsPtr = std::shared_ptr; /// Coordination settings + some other parts of keeper configuration /// which are not stored in settings. Allows to dump configuration -/// with four letter word commands. +/// with 4lw commands. struct KeeperConfigurationAndSettings { static constexpr int NOT_EXIST = -1; diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 3250ab197f3..5a14ed7b97d 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -1,14 +1,16 @@ -#include #include + #include #include #include #include #include -#include +#include #include #include -#include +#include + +#include namespace DB { @@ -51,9 +53,7 @@ FourLetterCommandFactory & FourLetterCommandFactory::instance() void FourLetterCommandFactory::checkInitialization() const { if (!initialized) - { throw Exception("Four letter command not initialized", ErrorCodes::LOGICAL_ERROR); - } } bool FourLetterCommandFactory::isKnown(int32_t code) @@ -143,18 +143,17 @@ bool FourLetterCommandFactory::isEnabled(int32_t code) void FourLetterCommandFactory::initializeWhiteList(KeeperDispatcher & keeper_dispatcher) { - using Poco::StringTokenizer; const auto & keeper_settings = keeper_dispatcher.getKeeperConfigurationAndSettings(); String list_str = keeper_settings->four_letter_word_white_list; - StringTokenizer tokenizer(list_str, ",", 2); + Strings tokens; + splitInto<','>(tokens, list_str); - for (const String & token : tokenizer) + for (const String & token : tokens) { if (token == "*") { white_list.clear(); - white_list.resize(1); white_list.push_back(WHITE_LIST_ALL); return; } @@ -200,7 +199,7 @@ void print(IFourLetterCommand::StringBuffer & buf, const String & key, uint64_t String MonitorCommand::run() { KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats(); - KeeperInfo keeper_info = keeper_dispatcher.getKeeperInfo(); + Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo(); if (!keeper_info.has_leader) return "This instance is not currently serving requests"; @@ -240,7 +239,6 @@ String MonitorCommand::run() return ret.str(); } - String StatResetCommand::run() { keeper_dispatcher.resetConnectionStats(); @@ -255,7 +253,7 @@ String NopCommand::run() String ConfCommand::run() { StringBuffer buf; - keeper_dispatcher.dumpConf(buf); + keeper_dispatcher.getKeeperConfigurationAndSettings()->dump(buf); return buf.str(); } @@ -285,7 +283,7 @@ String ServerStatCommand::run() }; KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats(); - KeeperInfo keeper_info = keeper_dispatcher.getKeeperInfo(); + Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo(); write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); @@ -311,7 +309,7 @@ String StatCommand::run() auto write = [&buf] (const String & key, const String & value) { buf << key << ": " << value << '\n'; }; KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats(); - KeeperInfo keeper_info = keeper_dispatcher.getKeeperInfo(); + Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo(); write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 545e20c4f54..7f89bb1a33e 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -5,12 +5,9 @@ #include #include -#include #include -#if !defined(ARCADIA_BUILD) -# include -#endif +#include namespace DB { diff --git a/src/Coordination/KeeperInfos.h b/src/Coordination/Keeper4LWInfo.h similarity index 71% rename from src/Coordination/KeeperInfos.h rename to src/Coordination/Keeper4LWInfo.h index d4770e6ccc3..58d05da32d9 100644 --- a/src/Coordination/KeeperInfos.h +++ b/src/Coordination/Keeper4LWInfo.h @@ -1,14 +1,7 @@ #pragma once #include -#include -#include -#include -#include -#include - -/// Contains some useful interfaces which are helpful to get keeper information. namespace DB { @@ -17,8 +10,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -/// Keeper server related information -struct KeeperInfo +/// Keeper server related information for different 4lw commands +struct Keeper4LWInfo { bool is_leader; bool is_observer; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index fbbd5222a75..dd03cec5d21 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -620,14 +620,9 @@ uint64_t KeeperDispatcher::getSnapDirSize() const return getDirSize(configuration_and_settings->snapshot_storage_path); } -void KeeperDispatcher::dumpConf(WriteBufferFromOwnString & buf) const +Keeper4LWInfo KeeperDispatcher::getKeeper4LWInfo() const { - configuration_and_settings->dump(buf); -} - -KeeperInfo KeeperDispatcher::getKeeperInfo() const -{ - KeeperInfo result; + Keeper4LWInfo result; result.is_leader = isLeader(); result.is_observer = server->isObserver(); result.is_follower = server->isFollower(); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index e16475a238d..646fab5d80f 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include namespace DB @@ -165,7 +165,7 @@ public: return keeper_stats; } - KeeperInfo getKeeperInfo() const; + Keeper4LWInfo getKeeper4LWInfo() const; const KeeperStateMachine & getStateMachine() const { @@ -194,8 +194,6 @@ public: std::lock_guard lock(keeper_stats_mutex); keeper_stats.reset(); } - - void dumpConf(WriteBufferFromOwnString & buf) const; }; } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 14a98146b10..376fe111f15 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -6,9 +6,7 @@ #include #include #include -#include #include -#include namespace DB { diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 9d8a379ac1b..2803f4b9027 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB @@ -74,10 +73,16 @@ public: return *storage; } + void shutdownStorage(); + + ClusterConfigPtr getClusterConfig() const; + /// Process local read request void processReadRequest(const KeeperStorage::RequestForSession & request_for_session); std::vector getDeadSessions(); + + /// Introspection functions for 4lw commands uint64_t getLastProcessedZxid() const; uint64_t getNodesCount() const; @@ -93,10 +98,6 @@ public: uint64_t getTotalEphemeralNodesCount() const; uint64_t getApproximateDataSize() const; - void shutdownStorage(); - - ClusterConfigPtr getClusterConfig() const; - private: /// In our state machine we always have a single snapshot which is stored diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 7c61b45ba83..a40cd160863 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -1,4 +1,5 @@ #include + #include #include #include @@ -11,7 +12,7 @@ namespace ErrorCodes extern const int RAFT_ERROR; } -KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const +KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const { KeeperConfigurationWrapper result; result.cluster_config = std::make_shared(); diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index bc27d94e7cd..fad76c89503 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -13,20 +13,6 @@ namespace DB using KeeperServerConfigPtr = nuraft::ptr; -/// Wrapper struct for Keeper cluster config. We parse this -/// info from XML files. -struct KeeperConfigurationWrapper -{ - /// Our port - int port; - /// Our config - KeeperServerConfigPtr config; - /// Servers id's to start as followers - std::unordered_set servers_start_as_followers; - /// Cluster config - ClusterConfigPtr cluster_config; -}; - /// When our configuration changes the following action types /// can happen enum class ConfigUpdateActionType @@ -121,6 +107,20 @@ public: ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; private: + /// Wrapper struct for Keeper cluster config. We parse this + /// info from XML files. + struct KeeperConfigurationWrapper + { + /// Our port + int port; + /// Our config + KeeperServerConfigPtr config; + /// Servers id's to start as followers + std::unordered_set servers_start_as_followers; + /// Cluster config + ClusterConfigPtr cluster_config; + }; + int my_server_id; bool secure; std::string config_prefix; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 7c55c639d94..df40d579bc3 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -133,6 +133,20 @@ static bool fixupACL( return valid_found; } +uint64_t KeeperStorage::Node::sizeInBytes() const +{ + uint64_t total_size{0}; + for (const auto & child : children) + total_size += child.size(); + + total_size += data.size(); + total_size += sizeof(acl_id); + total_size += sizeof(is_sequental); + total_size += sizeof(stat); + total_size += sizeof(seq_num); + return total_size; +} + static KeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, Coordination::Event event_type) { KeeperStorage::ResponsesForSessions result; @@ -1227,9 +1241,7 @@ void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; for (const String & path : watches_paths) - { buf << "\t" << path << "\n"; - } } } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index b0052e6542d..0cf173e9b80 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -14,7 +13,6 @@ namespace DB { -using namespace DB; struct KeeperStorageRequestProcessor; using KeeperStorageRequestProcessorPtr = std::shared_ptr; using ResponseCallback = std::function; @@ -38,15 +36,8 @@ public: int32_t seq_num = 0; ChildrenSet children{}; - /// object memory size - uint64_t sizeInBytes() const - { - uint64_t child_size{0}; - for (const auto & child : children) - child_size += child.size(); - - return data.size() + sizeof(Node) + child_size; - } + /// Object memory size + uint64_t sizeInBytes() const; }; struct ResponseForSession @@ -188,7 +179,6 @@ public: } /// Introspection functions mostly used in 4-letter commands - uint64_t getNodesCount() const { return container.size(); diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 4fff49ed4f8..98f4c6c8e14 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -50,7 +50,7 @@ private: /// value_size: size of value to add /// old_value_size: size of value to minus /// old_value_size=0 means there is no old value with the same key. - void inline updateDataSize(OperationType op_type, uint64_t key_size, uint64_t value_size, uint64_t old_value_size) + void updateDataSize(OperationType op_type, uint64_t key_size, uint64_t value_size, uint64_t old_value_size) { switch (op_type) { @@ -60,7 +60,7 @@ private: break; case INSERT_OR_REPLACE: /// replace - if (old_value_size) + if (old_value_size != 0) { approximate_data_size += key_size; approximate_data_size += value_size; diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 658c69fdcab..5d4d0a9c79a 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -465,13 +465,13 @@ bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(Int32 & command) } else if (!FourLetterCommandFactory::instance().isEnabled(command)) { - LOG_WARNING(log, "not enabled four letter command {}", IFourLetterCommand::toName(command)); + LOG_WARNING(log, "Not enabled four letter command {}", IFourLetterCommand::toName(command)); return false; } else { auto command_ptr = FourLetterCommandFactory::instance().get(command); - LOG_DEBUG(log, "receive four letter command {}", command_ptr->name()); + LOG_DEBUG(log, "Receive four letter command {}", command_ptr->name()); String res; try @@ -531,7 +531,6 @@ void KeeperTCPHandler::packageReceived() conn_stats.incrementPacketsReceived(); } keeper_dispatcher->incrementPacketsReceived(); - LOG_INFO(log, "INCREMENT PACKTEDS RECEIVED VALUE {} SESSION {}", keeper_dispatcher->getKeeperConnectionStats().getPacketsReceived(), session_id); } void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response) diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index 8d4c1dfd83b..ce022df8b86 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -17,7 +17,6 @@ #include #include #include -#include #include #include diff --git a/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.reference b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh new file mode 100755 index 00000000000..4a94beddbba --- /dev/null +++ b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh @@ -0,0 +1,47 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +function four_letter_thread() +{ + declare -a FOUR_LETTER_COMMANDS=("conf" "cons" "crst" "envi" "ruok" "srst" "srvr" "stat" "wchc" "wchs" "dirs" "mntr" "isro") + while true; do + command=${FOUR_LETTER_COMMANDS[$RANDOM % ${#FOUR_LETTER_COMMANDS[@]} ]} + echo $command | nc ${CLICKHOUSE_HOST} ${CLICKHOUSE_PORT_KEEPER} 1>/dev/null + done + +} + +function create_drop_thread() +{ + while true; do + num=$RANDOM + $CLICKHOUSE_CLIENT --query "CREATE TABLE test_table$num (key UInt64, value1 UInt8, value2 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_table$num', '0') ORDER BY key" + sleep 0.$RANDOM + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_table$num" + done +} + +export -f four_letter_thread; +export -f create_drop_thread; + +TIMEOUT=15 + +timeout $TIMEOUT bash -c four_letter_thread 2> /dev/null & +timeout $TIMEOUT bash -c four_letter_thread 2> /dev/null & +timeout $TIMEOUT bash -c four_letter_thread 2> /dev/null & +timeout $TIMEOUT bash -c four_letter_thread 2> /dev/null & + +timeout $TIMEOUT bash -c create_drop_thread 2> /dev/null & +timeout $TIMEOUT bash -c create_drop_thread 2> /dev/null & +timeout $TIMEOUT bash -c create_drop_thread 2> /dev/null & +timeout $TIMEOUT bash -c create_drop_thread 2> /dev/null & + + +wait + +# still alive +$CLICKHOUSE_CLIENT --query "SELECT 1" diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index 1f802ddab7b..a5063fac28e 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -73,6 +73,8 @@ export CLICKHOUSE_PORT_MYSQL=${CLICKHOUSE_PORT_MYSQL:=$(${CLICKHOUSE_EXTRACT_CON export CLICKHOUSE_PORT_MYSQL=${CLICKHOUSE_PORT_MYSQL:="9004"} export CLICKHOUSE_PORT_POSTGRESQL=${CLICKHOUSE_PORT_POSTGRESQL:=$(${CLICKHOUSE_EXTRACT_CONFIG} --try --key=postgresql_port 2>/dev/null)} 2>/dev/null export CLICKHOUSE_PORT_POSTGRESQL=${CLICKHOUSE_PORT_POSTGRESQL:="9005"} +export CLICKHOUSE_PORT_POSTGRESQL=${CLICKHOUSE_PORT_POSTGRESQL:=$(${CLICKHOUSE_EXTRACT_CONFIG} --try --key=keeper_server.tcp_port 2>/dev/null)} 2>/dev/null +export CLICKHOUSE_PORT_KEEPER=${CLICKHOUSE_PORT_KEEPER:="9181"} export CLICKHOUSE_CLIENT_SECURE=${CLICKHOUSE_CLIENT_SECURE:=$(echo "${CLICKHOUSE_CLIENT}" | sed 's/'"--port=${CLICKHOUSE_PORT_TCP}"'//g; s/$/'"--secure --port=${CLICKHOUSE_PORT_TCP_SECURE}"'/g')} From 7820af7cbda9f9f37f8ab71819a79b347fd5754d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 12:48:38 +0300 Subject: [PATCH 30/49] More compatible output format --- src/Coordination/FourLetterCommand.cpp | 11 +++++------ src/Coordination/Keeper4LWInfo.h | 3 +++ src/Coordination/KeeperDispatcher.cpp | 1 + 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 5a14ed7b97d..849c60bec38 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -242,7 +241,7 @@ String MonitorCommand::run() String StatResetCommand::run() { keeper_dispatcher.resetConnectionStats(); - return "Server stats reset."; + return "Server stats reset.\n"; } String NopCommand::run() @@ -267,7 +266,7 @@ String ConsCommand::run() String RestConnStatsCommand::run() { KeeperTCPHandler::resetConnsStats(); - return "Connection stats reset."; + return "Connection stats reset.\n"; } String ServerStatCommand::run() @@ -288,7 +287,7 @@ String ServerStatCommand::run() write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); StringBuffer latency; - latency << stats.getMinLatency() << "/" << stats.getAvgLatency() << "/" << stats.getMaxLatency() << "\n"; + latency << stats.getMinLatency() << "/" << stats.getAvgLatency() << "/" << stats.getMaxLatency(); write("Latency min/avg/max", latency.str()); write("Received", toString(stats.getPacketsReceived())); @@ -318,7 +317,7 @@ String StatCommand::run() buf << '\n'; StringBuffer latency; - latency << stats.getMinLatency() << "/" << stats.getAvgLatency() << "/" << stats.getMaxLatency() << "\n"; + latency << stats.getMinLatency() << "/" << stats.getAvgLatency() << "/" << stats.getMaxLatency(); write("Latency min/avg/max", latency.str()); write("Received", toString(stats.getPacketsReceived())); @@ -338,7 +337,7 @@ String BriefWatchCommand::run() const auto & state_machine = keeper_dispatcher.getStateMachine(); buf << state_machine.getSessionsWithWatchesCount() << " connections watching " << state_machine.getWatchedPathsCount() << " paths\n"; - buf << "Total watches:" << state_machine.getTotalWatchesCount(); + buf << "Total watches:" << state_machine.getTotalWatchesCount() << "\n"; return buf.str(); } diff --git a/src/Coordination/Keeper4LWInfo.h b/src/Coordination/Keeper4LWInfo.h index 58d05da32d9..1c461582a8c 100644 --- a/src/Coordination/Keeper4LWInfo.h +++ b/src/Coordination/Keeper4LWInfo.h @@ -16,6 +16,7 @@ struct Keeper4LWInfo bool is_leader; bool is_observer; bool is_follower; + bool is_standalone; bool has_leader; @@ -30,6 +31,8 @@ struct Keeper4LWInfo String getRole() const { + if (is_standalone) + return "standalone"; if (is_leader) return "leader"; if (is_observer) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index dd03cec5d21..48016d8c511 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -623,6 +623,7 @@ uint64_t KeeperDispatcher::getSnapDirSize() const Keeper4LWInfo KeeperDispatcher::getKeeper4LWInfo() const { Keeper4LWInfo result; + result.is_standalone = !result.is_follower && server->getFollowerCount() == 0; result.is_leader = isLeader(); result.is_observer = server->isObserver(); result.is_follower = server->isFollower(); From 7b7df05e337fb0255c5b471dcedd22742ab97b86 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 13:49:04 +0300 Subject: [PATCH 31/49] Update shell_config.sh --- tests/queries/shell_config.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index a5063fac28e..cca349490ad 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -73,7 +73,7 @@ export CLICKHOUSE_PORT_MYSQL=${CLICKHOUSE_PORT_MYSQL:=$(${CLICKHOUSE_EXTRACT_CON export CLICKHOUSE_PORT_MYSQL=${CLICKHOUSE_PORT_MYSQL:="9004"} export CLICKHOUSE_PORT_POSTGRESQL=${CLICKHOUSE_PORT_POSTGRESQL:=$(${CLICKHOUSE_EXTRACT_CONFIG} --try --key=postgresql_port 2>/dev/null)} 2>/dev/null export CLICKHOUSE_PORT_POSTGRESQL=${CLICKHOUSE_PORT_POSTGRESQL:="9005"} -export CLICKHOUSE_PORT_POSTGRESQL=${CLICKHOUSE_PORT_POSTGRESQL:=$(${CLICKHOUSE_EXTRACT_CONFIG} --try --key=keeper_server.tcp_port 2>/dev/null)} 2>/dev/null +export CLICKHOUSE_PORT_KEEPER=${CLICKHOUSE_PORT_POSTGRESQL:=$(${CLICKHOUSE_EXTRACT_CONFIG} --try --key=keeper_server.tcp_port 2>/dev/null)} 2>/dev/null export CLICKHOUSE_PORT_KEEPER=${CLICKHOUSE_PORT_KEEPER:="9181"} export CLICKHOUSE_CLIENT_SECURE=${CLICKHOUSE_CLIENT_SECURE:=$(echo "${CLICKHOUSE_CLIENT}" | sed 's/'"--port=${CLICKHOUSE_PORT_TCP}"'//g; s/$/'"--secure --port=${CLICKHOUSE_PORT_TCP_SECURE}"'/g')} From d1eba74c041fd30cd97182fdc0d3b3651efbc237 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 13:52:03 +0300 Subject: [PATCH 32/49] Trim string --- src/Coordination/FourLetterCommand.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 849c60bec38..294e623d803 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -148,8 +149,10 @@ void FourLetterCommandFactory::initializeWhiteList(KeeperDispatcher & keeper_dis Strings tokens; splitInto<','>(tokens, list_str); - for (const String & token : tokens) + for (String token: tokens) { + trim(token); + if (token == "*") { white_list.clear(); From 0c0aba5ed2ed7d48f05a6138094ba85e99626f50 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 19 Nov 2021 19:04:54 +0800 Subject: [PATCH 33/49] add new line to new xml file tail --- .../configs/keeper_config_with_white_list_all.xml | 2 +- .../configs/keeper_config_without_white_list.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list_all.xml b/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list_all.xml index cb2b1dbd5b4..635afa70b85 100644 --- a/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list_all.xml +++ b/tests/integration/test_keeper_four_word_command/configs/keeper_config_with_white_list_all.xml @@ -35,4 +35,4 @@ 9181 - \ No newline at end of file + diff --git a/tests/integration/test_keeper_four_word_command/configs/keeper_config_without_white_list.xml b/tests/integration/test_keeper_four_word_command/configs/keeper_config_without_white_list.xml index 9abc1d2f6a7..084b9a63654 100644 --- a/tests/integration/test_keeper_four_word_command/configs/keeper_config_without_white_list.xml +++ b/tests/integration/test_keeper_four_word_command/configs/keeper_config_without_white_list.xml @@ -34,4 +34,4 @@ 9181 - \ No newline at end of file + From 587a1a622d9b5824472700d7566ab2382585249e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 16:03:01 +0300 Subject: [PATCH 34/49] Fix tests --- src/Coordination/KeeperStorage.cpp | 4 ++++ src/Coordination/tests/gtest_coordination.cpp | 12 ++++++------ .../test_keeper_four_word_command/test.py | 2 +- 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index df40d579bc3..7519e8bcfab 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -140,6 +140,10 @@ uint64_t KeeperStorage::Node::sizeInBytes() const total_size += child.size(); total_size += data.size(); + /// Overhead for stroing empty containers + total_size += sizeof(data); + total_size += sizeof(children); + total_size += sizeof(acl_id); total_size += sizeof(is_sequental); total_size += sizeof(stat); diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 5b15044ab0d..36a03e7c7f0 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -983,24 +983,24 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) /// 160 = sizeof Node /// 5 = sizeof key /// 4 = sizeof value - EXPECT_EQ(world.getApproximateDataSize(), 169); + EXPECT_EQ(world.getApproximateDataSize(), 158); world.updateValue("world", [&](Node & value) { value = n2; }); - EXPECT_EQ(world.getApproximateDataSize(), 171); + EXPECT_EQ(world.getApproximateDataSize(), 160); world.erase("world"); EXPECT_EQ(world.getApproximateDataSize(), 0); world.enableSnapshotMode(); world.insert("world", n1); - EXPECT_EQ(world.getApproximateDataSize(), 169); + EXPECT_EQ(world.getApproximateDataSize(), 158); world.updateValue("world", [&](Node & value) { value = n2; }); - EXPECT_EQ(world.getApproximateDataSize(), 340); + EXPECT_EQ(world.getApproximateDataSize(), 318); world.clearOutdatedNodes(); - EXPECT_EQ(world.getApproximateDataSize(), 171); + EXPECT_EQ(world.getApproximateDataSize(), 160); world.erase("world"); - EXPECT_EQ(world.getApproximateDataSize(), 171); + EXPECT_EQ(world.getApproximateDataSize(), 160); world.clear(); EXPECT_EQ(world.getApproximateDataSize(), 0); diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 3679bc3cc5c..6ea12342044 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -240,7 +240,7 @@ def test_cmd_srst(started_cluster): clear_znodes() data = send_4lw_cmd(cmd='srst') - assert data == "Server stats reset." + assert data.strip() == "Server stats reset." data = send_4lw_cmd(cmd='mntr') assert len(data) != 0 From 583c9a5720bbe82e913d51af8000819747d1b832 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 16:36:13 +0300 Subject: [PATCH 35/49] Update shell_config.sh --- tests/queries/shell_config.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index cca349490ad..e95f8433636 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -73,7 +73,7 @@ export CLICKHOUSE_PORT_MYSQL=${CLICKHOUSE_PORT_MYSQL:=$(${CLICKHOUSE_EXTRACT_CON export CLICKHOUSE_PORT_MYSQL=${CLICKHOUSE_PORT_MYSQL:="9004"} export CLICKHOUSE_PORT_POSTGRESQL=${CLICKHOUSE_PORT_POSTGRESQL:=$(${CLICKHOUSE_EXTRACT_CONFIG} --try --key=postgresql_port 2>/dev/null)} 2>/dev/null export CLICKHOUSE_PORT_POSTGRESQL=${CLICKHOUSE_PORT_POSTGRESQL:="9005"} -export CLICKHOUSE_PORT_KEEPER=${CLICKHOUSE_PORT_POSTGRESQL:=$(${CLICKHOUSE_EXTRACT_CONFIG} --try --key=keeper_server.tcp_port 2>/dev/null)} 2>/dev/null +export CLICKHOUSE_PORT_KEEPER=${CLICKHOUSE_PORT_KEEPER:=$(${CLICKHOUSE_EXTRACT_CONFIG} --try --key=keeper_server.tcp_port 2>/dev/null)} 2>/dev/null export CLICKHOUSE_PORT_KEEPER=${CLICKHOUSE_PORT_KEEPER:="9181"} export CLICKHOUSE_CLIENT_SECURE=${CLICKHOUSE_CLIENT_SECURE:=$(echo "${CLICKHOUSE_CLIENT}" | sed 's/'"--port=${CLICKHOUSE_PORT_TCP}"'//g; s/$/'"--secure --port=${CLICKHOUSE_PORT_TCP_SECURE}"'/g')} From 1afef1733b0764856b29b1c388bf8536dfbe8064 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 18:22:46 +0300 Subject: [PATCH 36/49] Ignore overhead --- src/Coordination/KeeperDispatcher.cpp | 2 +- src/Coordination/KeeperStorage.cpp | 3 --- src/Coordination/tests/gtest_coordination.cpp | 16 ++++++---------- 3 files changed, 7 insertions(+), 14 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 48016d8c511..67dc21d58d9 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -623,10 +623,10 @@ uint64_t KeeperDispatcher::getSnapDirSize() const Keeper4LWInfo KeeperDispatcher::getKeeper4LWInfo() const { Keeper4LWInfo result; + result.is_follower = server->isFollower(); result.is_standalone = !result.is_follower && server->getFollowerCount() == 0; result.is_leader = isLeader(); result.is_observer = server->isObserver(); - result.is_follower = server->isFollower(); result.has_leader = hasLeader(); { std::lock_guard lock(push_request_mutex); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 7519e8bcfab..81bb3d0dd7d 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -140,9 +140,6 @@ uint64_t KeeperStorage::Node::sizeInBytes() const total_size += child.size(); total_size += data.size(); - /// Overhead for stroing empty containers - total_size += sizeof(data); - total_size += sizeof(children); total_size += sizeof(acl_id); total_size += sizeof(is_sequental); diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 36a03e7c7f0..d5498a1bc13 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -979,28 +979,24 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) world.disableSnapshotMode(); world.insert("world", n1); - /// 169 = 160 + 5 + 4 - /// 160 = sizeof Node - /// 5 = sizeof key - /// 4 = sizeof value - EXPECT_EQ(world.getApproximateDataSize(), 158); + EXPECT_EQ(world.getApproximateDataSize(), 94); world.updateValue("world", [&](Node & value) { value = n2; }); - EXPECT_EQ(world.getApproximateDataSize(), 160); + EXPECT_EQ(world.getApproximateDataSize(), 96); world.erase("world"); EXPECT_EQ(world.getApproximateDataSize(), 0); world.enableSnapshotMode(); world.insert("world", n1); - EXPECT_EQ(world.getApproximateDataSize(), 158); + EXPECT_EQ(world.getApproximateDataSize(), 94); world.updateValue("world", [&](Node & value) { value = n2; }); - EXPECT_EQ(world.getApproximateDataSize(), 318); + EXPECT_EQ(world.getApproximateDataSize(), 190); world.clearOutdatedNodes(); - EXPECT_EQ(world.getApproximateDataSize(), 160); + EXPECT_EQ(world.getApproximateDataSize(), 96); world.erase("world"); - EXPECT_EQ(world.getApproximateDataSize(), 160); + EXPECT_EQ(world.getApproximateDataSize(), 96); world.clear(); EXPECT_EQ(world.getApproximateDataSize(), 0); From 1cf204c581913181fa0892dbcc7b8a308523a248 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 21 Nov 2021 15:28:45 +0300 Subject: [PATCH 37/49] Small changes --- src/Server/KeeperTCPHandler.cpp | 17 ++++------------- src/Server/KeeperTCPHandler.h | 4 ++-- 2 files changed, 6 insertions(+), 15 deletions(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 5d4d0a9c79a..c8b99b61c3a 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -450,13 +450,13 @@ void KeeperTCPHandler::runImpl() } } -bool KeeperTCPHandler::isHandShake(Int32 & handshake_length) +bool KeeperTCPHandler::isHandShake(int32_t handshake_length) { return handshake_length == Coordination::CLIENT_HANDSHAKE_LENGTH || handshake_length == Coordination::CLIENT_HANDSHAKE_LENGTH_WITH_READONLY; } -bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(Int32 & command) +bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(int32_t command) { if (!FourLetterCommandFactory::instance().isKnown(command)) { @@ -473,25 +473,16 @@ bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(Int32 & command) auto command_ptr = FourLetterCommandFactory::instance().get(command); LOG_DEBUG(log, "Receive four letter command {}", command_ptr->name()); - String res; try { - res = command_ptr->run(); + String res = command_ptr->run(); + out->write(res.data(), res.size()); } catch (...) { tryLogCurrentException(log, "Error when executing four letter command " + command_ptr->name()); } - try - { - out->write(res.data(), res.size()); - } - catch (const Exception &) - { - tryLogCurrentException(log, "Error when send four letter command response"); - } - return true; } } diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index ce022df8b86..fb6541d1f53 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -81,8 +81,8 @@ private: void sendHandshake(bool has_leader); Poco::Timespan receiveHandshake(int32_t handshake_length); - static bool isHandShake(Int32 & handshake_length) ; - bool tryExecuteFourLetterWordCmd(Int32 & command); + static bool isHandShake(int32_t handshake_length); + bool tryExecuteFourLetterWordCmd(int32_t command); std::pair receiveRequest(); From 203477c02a5307cbaff8a6ce9b52bc1daaf050e9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Nov 2021 19:19:20 +0300 Subject: [PATCH 38/49] Bump From de0ff97242592a011d34516eeed52d6bcd22a03c Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Mon, 22 Nov 2021 11:08:55 -0800 Subject: [PATCH 39/49] Updated adopters: added 4 new adopters, sorted alphabetically, and minor editing --- docs/en/introduction/adopters.md | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index fd0f3a12b81..de3ac4ea0e9 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -11,6 +11,7 @@ toc_title: Adopters | Company | Industry | Usecase | Cluster Size | (Un)Compressed Data Size\* | Reference | |---------|----------|---------|--------------|------------------------------------------------------------------------------|-----------| | 2gis | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) | +| Adapty | Subscription Analytics | Main product | — | — | [Tweet, November 2021](https://twitter.com/iwitaly/status/1462698148061659139) | | Admiral | Martech | Engagement Management | — | — | [Webinar Slides, June 2020](https://altinity.com/presentations/2020/06/16/big-data-in-real-time-how-clickhouse-powers-admirals-visitor-relationships-for-publishers) | | AdScribe | Ads | TV Analytics | — | — | [A quote from CTO](https://altinity.com/24x7-support/) | | Ahrefs | SEO | Analytics | — | — | [Job listing](https://ahrefs.com/jobs/data-scientist-search) | @@ -19,7 +20,7 @@ toc_title: Adopters | Aloha Browser | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.com/meetup22/aloha.pdf) | | Altinity | Cloud, SaaS | Main product | — | — | [Official Website](https://altinity.com/) | | Amadeus | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | -| ApiRoad | API marketplace | Analytics | — | — | [Blog post, Nov 2018, Mar 2020](https://pixeljets.com/blog/clickhouse-vs-elasticsearch/) | +| ApiRoad | API marketplace | Analytics | — | — | [Blog post, November 2018, March 2020](https://pixeljets.com/blog/clickhouse-vs-elasticsearch/) | | Appsflyer | Mobile analytics | Main product | — | — | [Talk in Russian, July 2019](https://www.youtube.com/watch?v=M3wbRlcpBbY) | | ArenaData | Data Platform | Main product | — | — | [Slides in Russian, December 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup38/indexes.pdf) | | Argedor | ClickHouse support | — | — | — | [Official website](https://www.argedor.com/en/clickhouse/) | @@ -50,6 +51,7 @@ toc_title: Adopters | Cryptology | Digital Assets Trading Platform | — | — | — | [Job advertisement, March 2021](https://career.habr.com/companies/cryptology/vacancies) | | Dataliance for China Telecom | Telecom | Analytics | — | — | [Slides in Chinese, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/telecom.pdf) | | Deutsche Bank | Finance | BI Analytics | — | — | [Slides in English, October 2019](https://bigdatadays.ru/wp-content/uploads/2019/10/D2-H3-3_Yakunin-Goihburg.pdf) | +| Deepl | Machine Learning | — | — | — | [Video, October 2021](https://www.youtube.com/watch?v=WIYJiPwxXdM&t=1182s) | | Deeplay | Gaming Analytics | — | — | — | [Job advertisement, 2020](https://career.habr.com/vacancies/1000062568) | | Diva-e | Digital consulting | Main Product | — | — | [Slides in English, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup29/ClickHouse-MeetUp-Unusual-Applications-sd-2019-09-17.pdf) | | Ecommpay | Payment Processing | Logs | — | — | [Video, Nov 2019](https://www.youtube.com/watch?v=d3GdZTOWGLk) | @@ -65,6 +67,7 @@ toc_title: Adopters | Gigapipe | Managed ClickHouse | Main product | — | — | [Official website](https://gigapipe.com/) | | Glaber | Monitoring | Main product | — | — | [Website](https://glaber.io/) | | GraphCDN | CDN | Traffic Analytics | — | — | [Blog Post in English, August 2021](https://altinity.com/blog/delivering-insight-on-graphql-apis-with-clickhouse-at-graphcdn/) | +| Grouparoo | Data Warehouse Integrations | Main product | — | — | [Official Website, November 2021](https://www.grouparoo.com/integrations) | | HUYA | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) | | Hydrolix | Cloud data platform | Main product | — | — | [Documentation](https://docs.hydrolix.io/guide/query) | | ICA | FinTech | Risk Management | — | — | [Blog Post in English, Sep 2020](https://altinity.com/blog/clickhouse-vs-redshift-performance-for-fintech-risk-management?utm_campaign=ClickHouse%20vs%20RedShift&utm_content=143520807&utm_medium=social&utm_source=twitter&hss_channel=tw-3894792263) | @@ -79,7 +82,7 @@ toc_title: Adopters | Ivi | Online Cinema | Analytics, Monitoring | — | — | [Article in Russian, Jan 2018](https://habr.com/en/company/ivi/blog/347408/) | | Jinshuju 金数据 | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | | Jitsu | Cloud Software | Data Pipeline | — | — | [Documentation](https://jitsu.com/docs/destinations-configuration/clickhouse-destination), [Hacker News](https://news.ycombinator.com/item?id=29106082) | -| kakaocorp | Internet company | — | — | — | [if(kakao)2020](https://tv.kakao.com/channel/3693125/cliplink/414129353), [if(kakao)2021](https://if.kakao.com/session/24) | +| kakaocorp | Internet company | — | — | — | [if(kakao)2020 conference](https://if.kakao.com/session/117) | | Kodiak Data | Clouds | Main product | — | — | [Slides in Engish, April 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup13/kodiak_data.pdf) | | Kontur | Software Development | Metrics | — | — | [Talk in Russian, November 2018](https://www.youtube.com/watch?v=U4u4Bd0FtrY) | | Kuaishou | Video | — | — | — | [ClickHouse Meetup, October 2018](https://clickhouse.com/blog/en/2018/clickhouse-community-meetup-in-beijing-on-october-28-2018/) | @@ -89,7 +92,7 @@ toc_title: Adopters | Mail.ru Cloud Solutions | Cloud services | Main product | — | — | [Article in Russian](https://mcs.mail.ru/help/db-create/clickhouse#) | | MAXILECT | Ad Tech, Blockchain, ML, AI | — | — | — | [Job advertisement, 2021](https://www.linkedin.com/feed/update/urn:li:activity:6780842017229430784/) | | Marilyn | Advertising | Statistics | — | — | [Talk in Russian, June 2017](https://www.youtube.com/watch?v=iXlIgx2khwc) | -| Mello | Marketing | Analytics | 1 server | — | [Article, Oct 2020](https://vc.ru/marketing/166180-razrabotka-tipovogo-otcheta-skvoznoy-analitiki) | +| Mello | Marketing | Analytics | 1 server | — | [Article, October 2020](https://vc.ru/marketing/166180-razrabotka-tipovogo-otcheta-skvoznoy-analitiki) | | MessageBird | Telecommunications | Statistics | — | — | [Slides in English, November 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) | | Microsoft | Web Analytics | Clarity (Main Product) | — | — | [A question on GitHub](https://github.com/ClickHouse/ClickHouse/issues/21556) | | MindsDB | Machine Learning | Main Product | — | — | [Official Website](https://www.mindsdb.com/blog/machine-learning-models-as-tables-in-ch) | @@ -100,17 +103,16 @@ toc_title: Adopters | NOC Project | Network Monitoring | Analytics | Main Product | — | [Official Website](https://getnoc.com/features/big-data/) | | Noction | Network Technology | Main Product | — | — | [Official Website](https://www.noction.com/news/irp-3-11-remote-triggered-blackholing-capability) | Nuna Inc. | Health Data Analytics | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=170) | -| Ok.ru | Social Network | — | 72 servers | 810 TB compressed, 50bn rows/day, 1.5 TB/day | [SmartData conference, Oct 2021](https://assets.ctfassets.net/oxjq45e8ilak/4JPHkbJenLgZhBGGyyonFP/57472ec6987003ec4078d0941740703b/____________________ClickHouse_______________________.pdf) | -| Omnicomm | Transportation Monitoring | — | — | — | [Facebook post, Oct 2021](https://www.facebook.com/OmnicommTeam/posts/2824479777774500) | +| Ok.ru | Social Network | — | 72 servers | 810 TB compressed, 50bn rows/day, 1.5 TB/day | [SmartData conference, October 2021](https://assets.ctfassets.net/oxjq45e8ilak/4JPHkbJenLgZhBGGyyonFP/57472ec6987003ec4078d0941740703b/____________________ClickHouse_______________________.pdf) | +| Omnicomm | Transportation Monitoring | — | — | — | [Facebook post, October 2021](https://www.facebook.com/OmnicommTeam/posts/2824479777774500) | | OneAPM | Monitoring and Data Analysis | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) | -| Open Targets | Genome Research | Genome Search | — | — | [Tweet, Oct 2021](https://twitter.com/OpenTargets/status/1452570865342758913?s=20), [Blog](https://blog.opentargets.org/graphql/) | +| Open Targets | Genome Research | Genome Search | — | — | [Tweet, October 2021](https://twitter.com/OpenTargets/status/1452570865342758913?s=20), [Blog](https://blog.opentargets.org/graphql/) | | OZON | E-commerce | — | — | — | [Official website](https://job.ozon.ru/vacancy/razrabotchik-clickhouse-ekspluatatsiya-40991870/) | | Panelbear | Analytics | Monitoring and Analytics | — | — | [Tech Stack, November 2020](https://panelbear.com/blog/tech-stack/) | | Percent 百分点 | Analytics | Main Product | — | — | [Slides in Chinese, June 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/4.%20ClickHouse万亿数据双中心的设计与实践%20.pdf) | | Percona | Performance analysis | Percona Monitoring and Management | — | — | [Official website, Mar 2020](https://www.percona.com/blog/2020/03/30/advanced-query-analysis-in-percona-monitoring-and-management-with-direct-clickhouse-access/) | -| Piwik PRO | Web Analytics | Main Product | — | — | [Official website, Dec 2018](https://piwik.pro/blog/piwik-pro-clickhouse-faster-efficient-reports/) | | Plausible | Analytics | Main Product | — | — | [Blog post, June 2020](https://twitter.com/PlausibleHQ/status/1273889629087969280) | -| PostHog | Product Analytics | Main Product | — | — | [Release Notes, Oct 2020](https://posthog.com/blog/the-posthog-array-1-15-0) | +| PostHog | Product Analytics | Main Product | — | — | [Release Notes, October 2020](https://posthog.com/blog/the-posthog-array-1-15-0), [Blog, November 2021](https://posthog.com/blog/how-we-turned-clickhouse-into-our-eventmansion) | | Postmates | Delivery | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=188) | | Pragma Innovation | Telemetry and Big Data Analysis | Main product | — | — | [Slides in English, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/4_pragma_innovation.pdf) | | PRANA | Industrial predictive analytics | Main product | — | — | [News (russian), Feb 2021](https://habr.com/en/news/t/541392/) | @@ -152,6 +154,7 @@ toc_title: Adopters | Traffic Stars | AD network | — | 300 servers in Europe/US | 1.8 PiB, 700 000 insert rps (as of 2021) | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | | Uber | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.com/meetup40/uber.pdf) | | UTMSTAT | Analytics | Main product | — | — | [Blog post, June 2020](https://vc.ru/tribuna/133956-striming-dannyh-iz-servisa-skvoznoy-analitiki-v-clickhouse) | +| Vercel | Traffic and Performance Analytics | — | — | — | Direct reference, October 2021 | | VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | | VMware | Cloud | VeloCloud, SDN | — | — | [Product documentation](https://docs.vmware.com/en/vRealize-Operations-Manager/8.3/com.vmware.vcom.metrics.doc/GUID-A9AD72E1-C948-4CA2-971B-919385AB3CA8.html) | | Walmart Labs | Internet, Retail | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=144) | @@ -167,6 +170,7 @@ toc_title: Adopters | Yandex Market | e-Commerce | Metrics, Logging | — | — | [Talk in Russian, January 2019](https://youtu.be/_l1qP0DyBcA?t=478) | | Yandex Metrica | Web analytics | Main product | 630 servers in one cluster, 360 servers in another cluster, 1862 servers in one department | 133 PiB / 8.31 PiB / 120 trillion records | [Slides, February 2020](https://presentations.clickhouse.com/meetup40/introduction/#13) | | Yotascale | Cloud | Data pipeline | — | 2 bn records/day | [LinkedIn (Accomplishments)](https://www.linkedin.com/in/adilsaleem/) | +| Your Analytics | Product Analytics | Main Product | — | - | [Tweet, November 2021](https://twitter.com/mikenikles/status/1459737241165565953) | | Zagrava Trading | — | — | — | — | [Job offer, May 2021](https://twitter.com/datastackjobs/status/1394707267082063874) | | ЦВТ | Software Development | Metrics, Logging | — | — | [Blog Post, March 2019, in Russian](https://vc.ru/dev/62715-kak-my-stroili-monitoring-na-prometheus-clickhouse-i-elk) | | МКБ | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) | @@ -174,8 +178,5 @@ toc_title: Adopters | Цифровой Рабочий | Industrial IoT, Analytics | — | — | — | [Blog post in Russian, March 2021](https://habr.com/en/company/croc/blog/548018/) | | ООО «МПЗ Богородский» | Agriculture | — | — | — | [Article in Russian, November 2020](https://cloud.yandex.ru/cases/okraina) | | ДомКлик | Real Estate | — | — | — | [Article in Russian, October 2021](https://habr.com/ru/company/domclick/blog/585936/) | -| Deepl | Machine Learning | — | — | — | [Video, October 2021](https://www.youtube.com/watch?v=WIYJiPwxXdM&t=1182s) | -| Vercel | Traffic and Performance Analytics | — | — | — | Direct reference, October 2021 | -| YourAnalytics | Web Analytics | — | — | — | [Tweet, Nov 2021](https://twitter.com/mikenikles/status/1460860140249235461) | [Original article](https://clickhouse.com/docs/en/introduction/adopters/) From 16f034fe4cb7fb110c0e015b8515a3604246f222 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 23 Nov 2021 13:31:10 +0300 Subject: [PATCH 40/49] Flush bytes after 4 letter commands --- src/Server/KeeperTCPHandler.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 188369b009a..6478ece1536 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -493,6 +493,7 @@ bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(int32_t command) { String res = command_ptr->run(); out->write(res.data(), res.size()); + out->next(); } catch (...) { From 3869ef4182b976e0aa89eee600d99e103721ceca Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 24 Nov 2021 00:36:13 +0300 Subject: [PATCH 41/49] Update adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index de3ac4ea0e9..325cd1ff825 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -82,7 +82,7 @@ toc_title: Adopters | Ivi | Online Cinema | Analytics, Monitoring | — | — | [Article in Russian, Jan 2018](https://habr.com/en/company/ivi/blog/347408/) | | Jinshuju 金数据 | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | | Jitsu | Cloud Software | Data Pipeline | — | — | [Documentation](https://jitsu.com/docs/destinations-configuration/clickhouse-destination), [Hacker News](https://news.ycombinator.com/item?id=29106082) | -| kakaocorp | Internet company | — | — | — | [if(kakao)2020 conference](https://if.kakao.com/session/117) | +| kakaocorp | Internet company | — | — | — | [if(kakao)2020](https://tv.kakao.com/channel/3693125/cliplink/414129353), [if(kakao)2021](https://if.kakao.com/session/24) | | Kodiak Data | Clouds | Main product | — | — | [Slides in Engish, April 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup13/kodiak_data.pdf) | | Kontur | Software Development | Metrics | — | — | [Talk in Russian, November 2018](https://www.youtube.com/watch?v=U4u4Bd0FtrY) | | Kuaishou | Video | — | — | — | [ClickHouse Meetup, October 2018](https://clickhouse.com/blog/en/2018/clickhouse-community-meetup-in-beijing-on-october-28-2018/) | From 1acbafdffe7461ca69cfb38e48665dfc8f1b8cfb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Nov 2021 00:48:57 +0300 Subject: [PATCH 42/49] Remove outstandingly wrong code --- src/IO/WriteBufferValidUTF8.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/IO/WriteBufferValidUTF8.cpp b/src/IO/WriteBufferValidUTF8.cpp index 1100e7ae836..f711db0f1e7 100644 --- a/src/IO/WriteBufferValidUTF8.cpp +++ b/src/IO/WriteBufferValidUTF8.cpp @@ -117,9 +117,6 @@ void WriteBufferValidUTF8::nextImpl() memory[i] = p[i]; working_buffer = Buffer(&memory[cnt], memory.data() + memory.size()); - - /// Propagate next() to the output buffer - output_buffer.next(); } WriteBufferValidUTF8::~WriteBufferValidUTF8() From 1cfa52f92193aa1aad0338dc64d8aba7b3c28b51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Nov 2021 02:01:31 +0300 Subject: [PATCH 43/49] Always re-render prompt while navigating history --- .gitmodules | 2 +- contrib/replxx | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index c973717d20e..5672b89b4d2 100644 --- a/.gitmodules +++ b/.gitmodules @@ -140,7 +140,7 @@ url = https://github.com/ClickHouse-Extras/libc-headers.git [submodule "contrib/replxx"] path = contrib/replxx - url = https://github.com/AmokHuginnsson/replxx.git + url = https://github.com/ClickHouse-Extras/replxx.git [submodule "contrib/avro"] path = contrib/avro url = https://github.com/ClickHouse-Extras/avro.git diff --git a/contrib/replxx b/contrib/replxx index 68410ac01df..f019cba7ea1 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit 68410ac01dfb4f09ea76120ac5a2cecda3943aaf +Subproject commit f019cba7ea1bcd1b4feb7826f28ed57fb581b04c From 4b0b659ee84fa1150e85117beef726f4cf184919 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Nov 2021 02:10:35 +0300 Subject: [PATCH 44/49] Add two more benchmark results from Vy Nguyen Tan --- .../benchmark/hardware/results/gcp_n2d.json | 52 ++++++++++++++++++ .../hardware/results/upcloud_8cpu_32gb.json | 54 +++++++++++++++++++ 2 files changed, 106 insertions(+) create mode 100644 website/benchmark/hardware/results/upcloud_8cpu_32gb.json diff --git a/website/benchmark/hardware/results/gcp_n2d.json b/website/benchmark/hardware/results/gcp_n2d.json index 1ce0524f0cf..068eba114eb 100644 --- a/website/benchmark/hardware/results/gcp_n2d.json +++ b/website/benchmark/hardware/results/gcp_n2d.json @@ -50,5 +50,57 @@ [0.021, 0.015, 0.015], [0.006, 0.008, 0.006] ] + }, + { + "system": "GCP n2d-standard-8", + "system_full": "GCP n2d-standard-8, AMD EPYC 7B12, 8vCPU, 64 GiB RAM", + "time": "2021-11-23 00:00:00", + "kind": "cloud", + "result": + [ +[0.002, 0.002, 0.002], +[0.054, 0.019, 0.023], +[0.109, 0.053, 0.061], +[0.282, 0.102, 0.098], +[0.261, 0.183, 0.185], +[0.812, 0.697, 0.699], +[0.058, 0.034, 0.036], +[0.048, 0.023, 0.020], +[1.164, 1.191, 1.206], +[1.332, 1.223, 1.354], +[0.378, 0.388, 0.314], +[0.448, 0.464, 0.376], +[1.593, 1.719, 1.699], +[2.137, 1.940, 2.271], +[1.957, 2.048, 1.895], +[1.797, 1.711, 1.998], +[6.016, 5.934, 5.659], +[3.573, 3.571, 3.396], +[11.690, 11.552, 11.673], +[0.251, 0.124, 0.125], +[3.106, 2.289, 2.330], +[3.490, 2.253, 2.563], +[6.666, 4.699, 5.062], +[8.078, 2.300, 2.726], +[0.883, 0.639, 0.604], +[0.513, 0.534, 0.507], +[0.879, 0.626, 0.625], +[3.068, 1.940, 1.656], +[3.389, 3.633, 3.075], +[2.576, 2.445, 2.623], +[1.431, 1.464, 1.441], +[2.365, 2.076, 2.067], +[10.904, 11.111, 13.093], +[8.407, 7.915, 8.475], +[8.005, 8.124, 9.201], +[2.701, 3.199, 3.036], +[0.244, 0.239, 0.204], +[0.118, 0.092, 0.089], +[0.103, 0.093, 0.060], +[0.539, 0.457, 0.445], +[0.048, 0.022, 0.022], +[0.038, 0.067, 0.016], +[0.011, 0.004, 0.004] + ] } ] diff --git a/website/benchmark/hardware/results/upcloud_8cpu_32gb.json b/website/benchmark/hardware/results/upcloud_8cpu_32gb.json new file mode 100644 index 00000000000..7eb1434a143 --- /dev/null +++ b/website/benchmark/hardware/results/upcloud_8cpu_32gb.json @@ -0,0 +1,54 @@ +[ + { + "system": "UpCloud 8CPU 32GiB", + "system_full": "UpCloud 8CPU 32GiB, AMD EPYC 7542", + "time": "2021-11-23 00:00:00", + "kind": "cloud", + "result": + [ +[0.002, 0.001, 0.001], +[0.024, 0.020, 0.019], +[0.123, 0.040, 0.048], +[0.448, 0.061, 0.058], +[0.517, 0.125, 0.119], +[0.849, 0.357, 0.350], +[0.037, 0.025, 0.033], +[0.038, 0.018, 0.018], +[0.804, 0.549, 0.551], +[0.920, 0.602, 0.612], +[0.542, 0.234, 0.231], +[0.573, 0.272, 0.274], +[1.032, 0.764, 0.777], +[1.617, 0.991, 1.001], +[1.219, 1.010, 0.994], +[1.056, 0.999, 0.992], +[2.643, 2.488, 2.498], +[1.659, 1.583, 1.533], +[4.957, 4.929, 4.960], +[0.453, 0.056, 0.055], +[6.116, 0.994, 0.951], +[6.914, 1.035, 1.028], +[12.878, 2.481, 2.450], +[9.791, 1.195, 1.139], +[1.682, 0.318, 0.319], +[0.784, 0.286, 0.273], +[1.706, 0.322, 0.318], +[6.060, 0.968, 0.948], +[5.184, 1.488, 1.450], +[1.151, 1.206, 1.219], +[1.520, 0.715, 0.723], +[3.469, 0.910, 0.911], +[5.128, 4.690, 4.771], +[7.003, 3.448, 3.492], +[7.022, 3.478, 3.430], +[1.266, 1.246, 1.257], +[0.225, 0.200, 0.199], +[0.087, 0.079, 0.070], +[0.076, 0.064, 0.063], +[0.450, 0.458, 0.411], +[0.034, 0.024, 0.026], +[0.024, 0.020, 0.021], +[0.006, 0.005, 0.006] + ] + } +] From 2d7ff774fd7cea79b9f47ee8e79cbf1215ebb979 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 24 Nov 2021 02:48:18 +0300 Subject: [PATCH 45/49] Update ontime.md --- docs/en/getting-started/example-datasets/ontime.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/ontime.md b/docs/en/getting-started/example-datasets/ontime.md index bcba4cfb77d..efc807b75fa 100644 --- a/docs/en/getting-started/example-datasets/ontime.md +++ b/docs/en/getting-started/example-datasets/ontime.md @@ -15,7 +15,7 @@ This dataset can be obtained in two ways: Downloading data: ``` bash -echo https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_{1987..2021}_{1..12}.zip | xargs -P10 wget --no-check-certificate --continue +wget --no-check-certificate --continue https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_{1987..2021}_{1..12}.zip ``` Creating a table: From 6859be2b3df9dca6a06739b9c0de2555cf21dd3e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Nov 2021 03:17:11 +0300 Subject: [PATCH 46/49] Update base64 library --- contrib/base64 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/base64 b/contrib/base64 index af9b331f2b4..9499e0c4945 160000 --- a/contrib/base64 +++ b/contrib/base64 @@ -1 +1 @@ -Subproject commit af9b331f2b4f30b41c70f3a571ff904a8251c1d3 +Subproject commit 9499e0c4945589973b9ea1bc927377cfbc84aa46 From 4f7f9795a1813c24238ec5019c2365cb5a6b07b0 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 24 Nov 2021 09:17:43 +0300 Subject: [PATCH 47/49] Fixed null pointer exception in `MATERIALIZE COLUMN`. --- src/Interpreters/MutationsInterpreter.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 471ad67d4e7..15d9010da08 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -569,6 +569,11 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) stages.emplace_back(context); const auto & column = columns_desc.get(command.column_name); + if (column.default_desc.kind != ColumnDefaultKind::Materialized) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Column {} could not be materialized", column.name); + } + stages.back().column_to_updated.emplace(column.name, column.default_desc.expression->clone()); } else if (command.type == MutationCommand::MATERIALIZE_INDEX) From 6ae12c0a6205ef199d7df7ad22e4c2595c7e261e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Nov 2021 13:20:42 +0300 Subject: [PATCH 48/49] Fix workflow in master --- .github/workflows/master.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index fc2e78a21e3..9a0da9c3172 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -181,7 +181,7 @@ jobs: REPO_COPY: ${{runner.temp}}/build_check/ClickHouse CACHES_PATH: ${{runner.temp}}/../ccaches CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NUMBER: 9 + BUILD_NUMBER: 8 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH From 04b98ee1d78897fb9d2bc600cdb44b2e4d92ece3 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 24 Nov 2021 14:56:12 +0300 Subject: [PATCH 49/49] Revert "Fixed null pointer exception in `MATERIALIZE COLUMN`" --- src/Interpreters/MutationsInterpreter.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 15d9010da08..471ad67d4e7 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -569,11 +569,6 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) stages.emplace_back(context); const auto & column = columns_desc.get(command.column_name); - if (column.default_desc.kind != ColumnDefaultKind::Materialized) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Column {} could not be materialized", column.name); - } - stages.back().column_to_updated.emplace(column.name, column.default_desc.expression->clone()); } else if (command.type == MutationCommand::MATERIALIZE_INDEX)