mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #28981 from JackyWoo/add_4_letter_words_commands
Add four letter commands to keeper
This commit is contained in:
commit
646cf38213
@ -49,6 +49,7 @@ Internal coordination settings are located in `<keeper_server>.<coordination_set
|
||||
- `auto_forwarding` — Allow to forward write requests from followers to the leader (default: true).
|
||||
- `shutdown_timeout` — Wait to finish internal connections and shutdown (ms) (default: 5000).
|
||||
- `startup_timeout` — If the server doesn't connect to other quorum participants in the specified timeout it will terminate (ms) (default: 30000).
|
||||
- `four_letter_word_white_list` — White list of 4lw commands (default: "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro").
|
||||
|
||||
Quorum configuration is located in `<keeper_server>.<raft_configuration>` section and contain servers description.
|
||||
|
||||
@ -104,6 +105,196 @@ 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` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively.
|
||||
|
||||
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 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.
|
||||
|
||||
```
|
||||
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
|
||||
|
||||
```
|
||||
|
||||
- srst : Reset server statistics. The command will affect the result of `srvr`, `mntr` and `stat`.
|
||||
|
||||
```
|
||||
Server stats reset.
|
||||
```
|
||||
|
||||
- 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/JackyWoo/
|
||||
user.dir=/Users/JackyWoo/project/jd/clickhouse/cmake-build-debug/programs/
|
||||
user.tmp=/var/folders/b4/smbq5mfj7578f2jzwn602tt40000gn/T/
|
||||
```
|
||||
|
||||
|
||||
- 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.
|
||||
|
||||
```
|
||||
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.
|
||||
|
||||
```
|
||||
/clickhouse/task_queue/ddl
|
||||
0x0000000000000001
|
||||
```
|
||||
|
||||
- dump : Lists the outstanding sessions and ephemeral nodes. This only works on the leader.
|
||||
|
||||
```
|
||||
Sessions dump (2):
|
||||
0x0000000000000001
|
||||
0x0000000000000002
|
||||
Sessions with Ephemerals (1):
|
||||
0x0000000000000001
|
||||
/clickhouse/task_queue/ddl
|
||||
```
|
||||
|
||||
## [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:
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Poco/Environment.h>
|
||||
#include <sys/stat.h>
|
||||
#include <pwd.h>
|
||||
#include <Coordination/FourLetterCommand.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
@ -367,6 +368,8 @@ int Keeper::main(const std::vector<std::string> & /*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
|
||||
|
@ -111,7 +111,8 @@
|
||||
#endif
|
||||
|
||||
#if USE_NURAFT
|
||||
# include <Server/KeeperTCPHandlerFactory.h>
|
||||
# include <Coordination/FourLetterCommand.h>
|
||||
# include <Server/KeeperTCPHandlerFactory.h>
|
||||
#endif
|
||||
|
||||
#if USE_BASE64
|
||||
@ -1025,6 +1026,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)
|
||||
{
|
||||
|
46
src/Common/getCurrentProcessFDCount.cpp
Normal file
46
src/Common/getCurrentProcessFDCount.cpp
Normal file
@ -0,0 +1,46 @@
|
||||
#include <Common/getCurrentProcessFDCount.h>
|
||||
#include <Common/ShellCommand.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <unistd.h>
|
||||
#include <fmt/format.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
int getCurrentProcessFDCount()
|
||||
{
|
||||
namespace fs = std::filesystem;
|
||||
int result = -1;
|
||||
#if defined(__linux__) || defined(__APPLE__)
|
||||
using namespace DB;
|
||||
|
||||
Int32 pid = getpid();
|
||||
|
||||
auto proc_fs_path = fmt::format("/proc/{}/fd", pid);
|
||||
if (fs::exists(proc_fs_path))
|
||||
{
|
||||
result = std::distance(fs::directory_iterator(proc_fs_path), fs::directory_iterator{});
|
||||
}
|
||||
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);
|
||||
auto command = ShellCommand::execute(by_lsof);
|
||||
|
||||
try
|
||||
{
|
||||
readIntText(result, command->out);
|
||||
command->wait();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
}
|
||||
}
|
||||
|
||||
#endif
|
||||
return result;
|
||||
}
|
5
src/Common/getCurrentProcessFDCount.h
Normal file
5
src/Common/getCurrentProcessFDCount.h
Normal file
@ -0,0 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
/// Get current process file descriptor count
|
||||
/// @return -1 os doesn't support "lsof" command or some error occurs.
|
||||
int getCurrentProcessFDCount();
|
36
src/Common/getMaxFileDescriptorCount.cpp
Normal file
36
src/Common/getMaxFileDescriptorCount.cpp
Normal file
@ -0,0 +1,36 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <Common/ShellCommand.h>
|
||||
#include <Common/getMaxFileDescriptorCount.h>
|
||||
#include <filesystem>
|
||||
|
||||
int getMaxFileDescriptorCount()
|
||||
{
|
||||
namespace fs = std::filesystem;
|
||||
int result = -1;
|
||||
#if defined(__linux__) || defined(__APPLE__)
|
||||
using namespace DB;
|
||||
|
||||
if (fs::exists("/proc/sys/fs/file-max"))
|
||||
{
|
||||
ReadBufferFromFile reader("/proc/sys/fs/file-max");
|
||||
readIntText(result, reader);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto command = ShellCommand::execute("ulimit -n");
|
||||
try
|
||||
{
|
||||
readIntText(result, command->out);
|
||||
command->wait();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
}
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
return result;
|
||||
}
|
6
src/Common/getMaxFileDescriptorCount.h
Normal file
6
src/Common/getMaxFileDescriptorCount.h
Normal file
@ -0,0 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
/// Get process max file descriptor count
|
||||
/// @return -1 if os does not support ulimit command or some error occurs
|
||||
int getMaxFileDescriptorCount();
|
||||
|
@ -11,7 +11,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Checksum = UInt64;
|
||||
using Checksum = uint64_t;
|
||||
|
||||
using LogEntryPtr = nuraft::ptr<nuraft::log_entry>;
|
||||
using LogEntries = std::vector<LogEntryPtr>;
|
||||
|
@ -1,6 +1,10 @@
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <base/logger_useful.h>
|
||||
#include <filesystem>
|
||||
#include <Coordination/Defines.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteIntText.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -32,4 +36,177 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro";
|
||||
|
||||
KeeperConfigurationAndSettings::KeeperConfigurationAndSettings()
|
||||
: server_id(NOT_EXIST)
|
||||
, tcp_port(NOT_EXIST)
|
||||
, tcp_port_secure(NOT_EXIST)
|
||||
, standalone_keeper(false)
|
||||
, coordination_settings(std::make_shared<CoordinationSettings>())
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
auto write_int = [&buf](int64_t value)
|
||||
{
|
||||
writeIntText(value, buf);
|
||||
buf.write('\n');
|
||||
};
|
||||
|
||||
auto write_bool = [&buf](bool value)
|
||||
{
|
||||
String str_val = value ? "true" : "false";
|
||||
writeText(str_val, buf);
|
||||
buf.write('\n');
|
||||
};
|
||||
|
||||
writeText("server_id=", buf);
|
||||
write_int(server_id);
|
||||
|
||||
if (tcp_port != NOT_EXIST)
|
||||
{
|
||||
writeText("tcp_port=", buf);
|
||||
write_int(tcp_port);
|
||||
}
|
||||
if (tcp_port_secure != NOT_EXIST)
|
||||
{
|
||||
writeText("tcp_port_secure=", buf);
|
||||
write_int(tcp_port_secure);
|
||||
}
|
||||
|
||||
writeText("four_letter_word_white_list=", buf);
|
||||
writeText(four_letter_word_white_list, buf);
|
||||
buf.write('\n');
|
||||
|
||||
writeText("log_storage_path=", buf);
|
||||
writeText(log_storage_path, buf);
|
||||
buf.write('\n');
|
||||
|
||||
writeText("snapshot_storage_path=", buf);
|
||||
writeText(snapshot_storage_path, buf);
|
||||
buf.write('\n');
|
||||
|
||||
/// coordination_settings
|
||||
|
||||
writeText("max_requests_batch_size=", buf);
|
||||
write_int(coordination_settings->max_requests_batch_size);
|
||||
writeText("session_timeout_ms=", buf);
|
||||
write_int(uint64_t(coordination_settings->session_timeout_ms));
|
||||
writeText("operation_timeout_ms=", buf);
|
||||
write_int(uint64_t(coordination_settings->operation_timeout_ms));
|
||||
writeText("dead_session_check_period_ms=", buf);
|
||||
write_int(uint64_t(coordination_settings->dead_session_check_period_ms));
|
||||
|
||||
writeText("heart_beat_interval_ms=", buf);
|
||||
write_int(uint64_t(coordination_settings->heart_beat_interval_ms));
|
||||
writeText("election_timeout_lower_bound_ms=", buf);
|
||||
write_int(uint64_t(coordination_settings->election_timeout_lower_bound_ms));
|
||||
writeText("election_timeout_upper_bound_ms=", buf);
|
||||
write_int(uint64_t(coordination_settings->election_timeout_upper_bound_ms));
|
||||
|
||||
writeText("reserved_log_items=", buf);
|
||||
write_int(coordination_settings->reserved_log_items);
|
||||
writeText("snapshot_distance=", buf);
|
||||
write_int(coordination_settings->snapshot_distance);
|
||||
|
||||
writeText("auto_forwarding=", buf);
|
||||
write_bool(coordination_settings->auto_forwarding);
|
||||
writeText("shutdown_timeout=", buf);
|
||||
write_int(uint64_t(coordination_settings->shutdown_timeout));
|
||||
writeText("startup_timeout=", buf);
|
||||
write_int(uint64_t(coordination_settings->startup_timeout));
|
||||
|
||||
writeText("raft_logs_level=", buf);
|
||||
writeText(coordination_settings->raft_logs_level.toString(), buf);
|
||||
buf.write('\n');
|
||||
|
||||
writeText("snapshots_to_keep=", buf);
|
||||
write_int(coordination_settings->snapshots_to_keep);
|
||||
writeText("rotate_log_storage_interval=", buf);
|
||||
write_int(coordination_settings->rotate_log_storage_interval);
|
||||
writeText("stale_log_gap=", buf);
|
||||
write_int(coordination_settings->stale_log_gap);
|
||||
writeText("fresh_log_gap=", buf);
|
||||
write_int(coordination_settings->fresh_log_gap);
|
||||
|
||||
writeText("max_requests_batch_size=", buf);
|
||||
write_int(coordination_settings->max_requests_batch_size);
|
||||
writeText("quorum_reads=", buf);
|
||||
write_bool(coordination_settings->quorum_reads);
|
||||
writeText("force_sync=", buf);
|
||||
write_bool(coordination_settings->force_sync);
|
||||
|
||||
writeText("compress_logs=", buf);
|
||||
write_bool(coordination_settings->compress_logs);
|
||||
writeText("compress_snapshots_with_zstd_format=", buf);
|
||||
write_bool(coordination_settings->compress_snapshots_with_zstd_format);
|
||||
writeText("configuration_change_tries_count=", buf);
|
||||
write_int(coordination_settings->configuration_change_tries_count);
|
||||
}
|
||||
|
||||
KeeperConfigurationAndSettingsPtr
|
||||
KeeperConfigurationAndSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_)
|
||||
{
|
||||
std::shared_ptr<KeeperConfigurationAndSettings> ret = std::make_shared<KeeperConfigurationAndSettings>();
|
||||
|
||||
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.superdigest");
|
||||
}
|
||||
|
||||
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_);
|
||||
|
||||
ret->coordination_settings->loadFromConfig("keeper_server.coordination_settings", config);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
String KeeperConfigurationAndSettings::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 KeeperConfigurationAndSettings::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";
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -51,4 +52,38 @@ struct CoordinationSettings : public BaseSettings<CoordinationSettingsTraits>
|
||||
|
||||
using CoordinationSettingsPtr = std::shared_ptr<CoordinationSettings>;
|
||||
|
||||
/// Coordination settings + some other parts of keeper configuration
|
||||
/// which are not stored in settings. Allows to dump configuration
|
||||
/// with 4lw commands.
|
||||
struct KeeperConfigurationAndSettings
|
||||
{
|
||||
static constexpr int NOT_EXIST = -1;
|
||||
static const String DEFAULT_FOUR_LETTER_WORD_CMD;
|
||||
|
||||
KeeperConfigurationAndSettings();
|
||||
int server_id;
|
||||
|
||||
int tcp_port;
|
||||
int tcp_port_secure;
|
||||
|
||||
String four_letter_word_white_list;
|
||||
|
||||
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<KeeperConfigurationAndSettings> 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 KeeperConfigurationAndSettingsPtr = std::shared_ptr<KeeperConfigurationAndSettings>;
|
||||
|
||||
}
|
||||
|
418
src/Coordination/FourLetterCommand.cpp
Normal file
418
src/Coordination/FourLetterCommand.cpp
Normal file
@ -0,0 +1,418 @@
|
||||
#include <Coordination/FourLetterCommand.h>
|
||||
|
||||
#include <Coordination/KeeperDispatcher.h>
|
||||
#include <Server/KeeperTCPHandler.h>
|
||||
#include <base/logger_useful.h>
|
||||
#include <Poco/Environment.h>
|
||||
#include <Poco/Path.h>
|
||||
#include <Common/getCurrentProcessFDCount.h>
|
||||
#include <Common/getMaxFileDescriptorCount.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Coordination/Keeper4LWInfo.h>
|
||||
|
||||
#include <unistd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
IFourLetterCommand::IFourLetterCommand(KeeperDispatcher & keeper_dispatcher_)
|
||||
: keeper_dispatcher(keeper_dispatcher_)
|
||||
{
|
||||
}
|
||||
|
||||
int32_t IFourLetterCommand::code()
|
||||
{
|
||||
return toCode(name());
|
||||
}
|
||||
|
||||
String IFourLetterCommand::toName(int32_t code)
|
||||
{
|
||||
int reverted_code = __builtin_bswap32(code);
|
||||
return String(reinterpret_cast<char *>(&reverted_code), 4);
|
||||
}
|
||||
|
||||
int32_t IFourLetterCommand::toCode(const String & name)
|
||||
{
|
||||
int32_t res = *reinterpret_cast<const int32_t *>(name.data());
|
||||
/// keep consistent with Coordination::read method by changing big endian to little endian.
|
||||
return __builtin_bswap32(res);
|
||||
}
|
||||
|
||||
IFourLetterCommand::~IFourLetterCommand() = default;
|
||||
|
||||
FourLetterCommandFactory & FourLetterCommandFactory::instance()
|
||||
{
|
||||
static FourLetterCommandFactory factory;
|
||||
return factory;
|
||||
}
|
||||
|
||||
void FourLetterCommandFactory::checkInitialization() const
|
||||
{
|
||||
if (!initialized)
|
||||
throw Exception("Four letter command not initialized", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
bool FourLetterCommandFactory::isKnown(int32_t code)
|
||||
{
|
||||
checkInitialization();
|
||||
return commands.contains(code);
|
||||
}
|
||||
|
||||
FourLetterCommandPtr FourLetterCommandFactory::get(int32_t code)
|
||||
{
|
||||
checkInitialization();
|
||||
return commands.at(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(KeeperDispatcher & keeper_dispatcher)
|
||||
{
|
||||
FourLetterCommandFactory & factory = FourLetterCommandFactory::instance();
|
||||
|
||||
if (!factory.isInitialized())
|
||||
{
|
||||
FourLetterCommandPtr ruok_command = std::make_shared<RuokCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(ruok_command);
|
||||
|
||||
FourLetterCommandPtr mntr_command = std::make_shared<MonitorCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(mntr_command);
|
||||
|
||||
FourLetterCommandPtr conf_command = std::make_shared<ConfCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(conf_command);
|
||||
|
||||
FourLetterCommandPtr cons_command = std::make_shared<ConsCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(cons_command);
|
||||
|
||||
FourLetterCommandPtr brief_watch_command = std::make_shared<BriefWatchCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(brief_watch_command);
|
||||
|
||||
FourLetterCommandPtr data_size_command = std::make_shared<DataSizeCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(data_size_command);
|
||||
|
||||
FourLetterCommandPtr dump_command = std::make_shared<DumpCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(dump_command);
|
||||
|
||||
FourLetterCommandPtr envi_command = std::make_shared<EnviCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(envi_command);
|
||||
|
||||
FourLetterCommandPtr is_rad_only_command = std::make_shared<IsReadOnlyCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(is_rad_only_command);
|
||||
|
||||
FourLetterCommandPtr rest_conn_stats_command = std::make_shared<RestConnStatsCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(rest_conn_stats_command);
|
||||
|
||||
FourLetterCommandPtr server_stat_command = std::make_shared<ServerStatCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(server_stat_command);
|
||||
|
||||
FourLetterCommandPtr stat_command = std::make_shared<StatCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(stat_command);
|
||||
|
||||
FourLetterCommandPtr stat_reset_command = std::make_shared<StatResetCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(stat_reset_command);
|
||||
|
||||
FourLetterCommandPtr watch_by_path_command = std::make_shared<WatchByPathCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(watch_by_path_command);
|
||||
|
||||
FourLetterCommandPtr watch_command = std::make_shared<WatchCommand>(keeper_dispatcher);
|
||||
factory.registerCommand(watch_command);
|
||||
|
||||
factory.initializeWhiteList(keeper_dispatcher);
|
||||
factory.setInitialize(true);
|
||||
}
|
||||
}
|
||||
|
||||
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(KeeperDispatcher & keeper_dispatcher)
|
||||
{
|
||||
const auto & keeper_settings = keeper_dispatcher.getKeeperConfigurationAndSettings();
|
||||
|
||||
String list_str = keeper_settings->four_letter_word_white_list;
|
||||
Strings tokens;
|
||||
splitInto<','>(tokens, list_str);
|
||||
|
||||
for (String token: tokens)
|
||||
{
|
||||
trim(token);
|
||||
|
||||
if (token == "*")
|
||||
{
|
||||
white_list.clear();
|
||||
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::run()
|
||||
{
|
||||
return "imok";
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
void print(IFourLetterCommand::StringBuffer & buf, const String & key, const String & value)
|
||||
{
|
||||
writeText("zk_", buf);
|
||||
writeText(key, buf);
|
||||
writeText('\t', buf);
|
||||
writeText(value, buf);
|
||||
writeText('\n', buf);
|
||||
}
|
||||
|
||||
void print(IFourLetterCommand::StringBuffer & buf, const String & key, uint64_t value)
|
||||
{
|
||||
print(buf, key, toString(value));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
String MonitorCommand::run()
|
||||
{
|
||||
KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats();
|
||||
Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo();
|
||||
|
||||
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()
|
||||
{
|
||||
keeper_dispatcher.resetConnectionStats();
|
||||
return "Server stats reset.\n";
|
||||
}
|
||||
|
||||
String NopCommand::run()
|
||||
{
|
||||
return "";
|
||||
}
|
||||
|
||||
String ConfCommand::run()
|
||||
{
|
||||
StringBuffer buf;
|
||||
keeper_dispatcher.getKeeperConfigurationAndSettings()->dump(buf);
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
String ConsCommand::run()
|
||||
{
|
||||
StringBuffer buf;
|
||||
KeeperTCPHandler::dumpConnections(buf, false);
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
String RestConnStatsCommand::run()
|
||||
{
|
||||
KeeperTCPHandler::resetConnsStats();
|
||||
return "Connection stats reset.\n";
|
||||
}
|
||||
|
||||
String ServerStatCommand::run()
|
||||
{
|
||||
StringBuffer buf;
|
||||
|
||||
auto write = [&buf](const String & key, const String & value)
|
||||
{
|
||||
writeText(key, buf);
|
||||
writeText(": ", buf);
|
||||
writeText(value, buf);
|
||||
writeText('\n', buf);
|
||||
};
|
||||
|
||||
KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats();
|
||||
Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo();
|
||||
|
||||
write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH);
|
||||
|
||||
StringBuffer latency;
|
||||
latency << stats.getMinLatency() << "/" << stats.getAvgLatency() << "/" << stats.getMaxLatency();
|
||||
write("Latency min/avg/max", latency.str());
|
||||
|
||||
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(keeper_info.total_nodes_count));
|
||||
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
String StatCommand::run()
|
||||
{
|
||||
StringBuffer buf;
|
||||
|
||||
auto write = [&buf] (const String & key, const String & value) { buf << key << ": " << value << '\n'; };
|
||||
|
||||
KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats();
|
||||
Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo();
|
||||
|
||||
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();
|
||||
write("Latency min/avg/max", latency.str());
|
||||
|
||||
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(keeper_info.total_nodes_count));
|
||||
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
String BriefWatchCommand::run()
|
||||
{
|
||||
StringBuffer buf;
|
||||
const auto & state_machine = keeper_dispatcher.getStateMachine();
|
||||
buf << state_machine.getSessionsWithWatchesCount() << " connections watching "
|
||||
<< state_machine.getWatchedPathsCount() << " paths\n";
|
||||
buf << "Total watches:" << state_machine.getTotalWatchesCount() << "\n";
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
String WatchCommand::run()
|
||||
{
|
||||
StringBuffer buf;
|
||||
const auto & state_machine = keeper_dispatcher.getStateMachine();
|
||||
state_machine.dumpWatches(buf);
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
String WatchByPathCommand::run()
|
||||
{
|
||||
StringBuffer buf;
|
||||
const auto & state_machine = keeper_dispatcher.getStateMachine();
|
||||
state_machine.dumpWatchesByPath(buf);
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
String DataSizeCommand::run()
|
||||
{
|
||||
StringBuffer buf;
|
||||
buf << "snapshot_dir_size: " << keeper_dispatcher.getSnapDirSize() << '\n';
|
||||
buf << "log_dir_size: " << keeper_dispatcher.getLogDirSize() << '\n';
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
String DumpCommand::run()
|
||||
{
|
||||
StringBuffer buf;
|
||||
const auto & state_machine = keeper_dispatcher.getStateMachine();
|
||||
state_machine.dumpSessionsAndEphemerals(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';
|
||||
|
||||
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';
|
||||
buf << "user.tmp=" << Path::temp() << '\n';
|
||||
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
String IsReadOnlyCommand::run()
|
||||
{
|
||||
if (keeper_dispatcher.isObserver())
|
||||
return "ro";
|
||||
else
|
||||
return "rw";
|
||||
}
|
||||
|
||||
}
|
307
src/Coordination/FourLetterCommand.h
Normal file
307
src/Coordination/FourLetterCommand.h
Normal file
@ -0,0 +1,307 @@
|
||||
#pragma once
|
||||
|
||||
#include <sstream>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
|
||||
#include <Coordination/KeeperDispatcher.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <Common/config_version.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct IFourLetterCommand;
|
||||
using FourLetterCommandPtr = std::shared_ptr<DB::IFourLetterCommand>;
|
||||
|
||||
/// 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(KeeperDispatcher & keeper_dispatcher_);
|
||||
|
||||
virtual String name() = 0;
|
||||
virtual String run() = 0;
|
||||
|
||||
virtual ~IFourLetterCommand();
|
||||
int32_t code();
|
||||
|
||||
static String toName(int32_t code);
|
||||
static inline int32_t toCode(const String & name);
|
||||
|
||||
protected:
|
||||
KeeperDispatcher & keeper_dispatcher;
|
||||
};
|
||||
|
||||
struct FourLetterCommandFactory : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
using Commands = std::unordered_map<int32_t, FourLetterCommandPtr>;
|
||||
using WhiteList = std::vector<int32_t>;
|
||||
|
||||
///represent '*' which is used in white list
|
||||
static constexpr int32_t WHITE_LIST_ALL = 0;
|
||||
|
||||
bool isKnown(int32_t code);
|
||||
bool isEnabled(int32_t 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(KeeperDispatcher & keeper_dispatcher);
|
||||
|
||||
void checkInitialization() const;
|
||||
bool isInitialized() const { return initialized; }
|
||||
void setInitialize(bool flag) { initialized = flag; }
|
||||
|
||||
static FourLetterCommandFactory & instance();
|
||||
static void registerCommands(KeeperDispatcher & keeper_dispatcher);
|
||||
|
||||
private:
|
||||
std::atomic<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.
|
||||
* 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(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.
|
||||
*
|
||||
* 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
|
||||
*/
|
||||
struct MonitorCommand : public IFourLetterCommand
|
||||
{
|
||||
explicit MonitorCommand(KeeperDispatcher & keeper_dispatcher_)
|
||||
: IFourLetterCommand(keeper_dispatcher_)
|
||||
{
|
||||
}
|
||||
|
||||
String name() override { return "mntr"; }
|
||||
String run() override;
|
||||
~MonitorCommand() override = default;
|
||||
};
|
||||
|
||||
struct StatResetCommand : public IFourLetterCommand
|
||||
{
|
||||
explicit StatResetCommand(KeeperDispatcher & keeper_dispatcher_) :
|
||||
IFourLetterCommand(keeper_dispatcher_)
|
||||
{
|
||||
}
|
||||
|
||||
String name() override { return "srst"; }
|
||||
String run() 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.
|
||||
struct NopCommand : public IFourLetterCommand
|
||||
{
|
||||
explicit NopCommand(KeeperDispatcher & keeper_dispatcher_)
|
||||
: IFourLetterCommand(keeper_dispatcher_)
|
||||
{
|
||||
}
|
||||
|
||||
String name() override { return "nopc"; }
|
||||
String run() override;
|
||||
~NopCommand() override = default;
|
||||
};
|
||||
|
||||
struct ConfCommand : public IFourLetterCommand
|
||||
{
|
||||
explicit ConfCommand(KeeperDispatcher & keeper_dispatcher_)
|
||||
: IFourLetterCommand(keeper_dispatcher_)
|
||||
{
|
||||
}
|
||||
|
||||
String name() override { return "conf"; }
|
||||
String run() 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(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(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(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(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(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(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(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(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(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(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(KeeperDispatcher & keeper_dispatcher_)
|
||||
: IFourLetterCommand(keeper_dispatcher_)
|
||||
{
|
||||
}
|
||||
|
||||
String name() override { return "isro"; }
|
||||
String run() override;
|
||||
~IsReadOnlyCommand() override = default;
|
||||
};
|
||||
|
||||
}
|
47
src/Coordination/Keeper4LWInfo.h
Normal file
47
src/Coordination/Keeper4LWInfo.h
Normal file
@ -0,0 +1,47 @@
|
||||
#pragma once
|
||||
|
||||
#include <string>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
/// Keeper server related information for different 4lw commands
|
||||
struct Keeper4LWInfo
|
||||
{
|
||||
bool is_leader;
|
||||
bool is_observer;
|
||||
bool is_follower;
|
||||
bool is_standalone;
|
||||
|
||||
bool has_leader;
|
||||
|
||||
uint64_t alive_connections_count;
|
||||
uint64_t outstanding_requests_count;
|
||||
|
||||
uint64_t follower_count;
|
||||
uint64_t synced_follower_count;
|
||||
|
||||
uint64_t total_nodes_count;
|
||||
int64_t last_zxid;
|
||||
|
||||
String getRole() const
|
||||
{
|
||||
if (is_standalone)
|
||||
return "standalone";
|
||||
if (is_leader)
|
||||
return "leader";
|
||||
if (is_observer)
|
||||
return "observer";
|
||||
if (is_follower)
|
||||
return "follower";
|
||||
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "RAFT server has undefined state state, it's a bug");
|
||||
}
|
||||
};
|
||||
|
||||
}
|
85
src/Coordination/KeeperConnectionStats.cpp
Normal file
85
src/Coordination/KeeperConnectionStats.cpp
Normal file
@ -0,0 +1,85 @@
|
||||
#include <Coordination/KeeperConnectionStats.h>
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
}
|
52
src/Coordination/KeeperConnectionStats.h
Normal file
52
src/Coordination/KeeperConnectionStats.h
Normal file
@ -0,0 +1,52 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <memory>
|
||||
#include <cstdint>
|
||||
|
||||
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;
|
||||
};
|
||||
|
||||
}
|
@ -3,6 +3,12 @@
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <future>
|
||||
#include <chrono>
|
||||
#include <Poco/Path.h>
|
||||
#include <Common/hex.h>
|
||||
#include <filesystem>
|
||||
#include <Common/checkStackSize.h>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -14,9 +20,10 @@ namespace ErrorCodes
|
||||
extern const int SYSTEM_ERROR;
|
||||
}
|
||||
|
||||
|
||||
KeeperDispatcher::KeeperDispatcher()
|
||||
: coordination_settings(std::make_shared<CoordinationSettings>())
|
||||
, responses_queue(std::numeric_limits<size_t>::max())
|
||||
: responses_queue(std::numeric_limits<size_t>::max())
|
||||
, configuration_and_settings(std::make_shared<KeeperConfigurationAndSettings>())
|
||||
, log(&Poco::Logger::get("KeeperDispatcher"))
|
||||
{
|
||||
}
|
||||
@ -36,7 +43,8 @@ void KeeperDispatcher::requestThread()
|
||||
{
|
||||
KeeperStorage::RequestForSession request;
|
||||
|
||||
UInt64 max_wait = UInt64(coordination_settings->operation_timeout_ms.totalMilliseconds());
|
||||
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
|
||||
@ -141,7 +149,7 @@ void KeeperDispatcher::responseThread()
|
||||
{
|
||||
KeeperStorage::ResponseForSession response_for_session;
|
||||
|
||||
UInt64 max_wait = UInt64(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))
|
||||
{
|
||||
@ -245,28 +253,25 @@ 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), configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds()))
|
||||
{
|
||||
throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED);
|
||||
}
|
||||
|
||||
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<RequestsQueue>(coordination_settings->max_requests_batch_size);
|
||||
configuration_and_settings = KeeperConfigurationAndSettings::loadFromConfig(config, standalone_keeper);
|
||||
requests_queue = std::make_unique<RequestsQueue>(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<KeeperServer>(
|
||||
myid, coordination_settings, config, responses_queue, snapshots_queue, standalone_keeper);
|
||||
server = std::make_unique<KeeperServer>(configuration_and_settings, config, responses_queue, snapshots_queue);
|
||||
|
||||
try
|
||||
{
|
||||
@ -413,7 +418,8 @@ void KeeperDispatcher::sessionCleanerTask()
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(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));
|
||||
}
|
||||
}
|
||||
|
||||
@ -580,4 +586,67 @@ void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfigurati
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperDispatcher::updateKeeperStatLatency(uint64_t process_time_ms)
|
||||
{
|
||||
std::lock_guard lock(keeper_stats_mutex);
|
||||
keeper_stats.updateLatency(process_time_ms);
|
||||
}
|
||||
|
||||
static uint64_t getDirSize(const fs::path & dir)
|
||||
{
|
||||
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_t KeeperDispatcher::getLogDirSize() const
|
||||
{
|
||||
return getDirSize(configuration_and_settings->log_storage_path);
|
||||
}
|
||||
|
||||
uint64_t KeeperDispatcher::getSnapDirSize() const
|
||||
{
|
||||
return getDirSize(configuration_and_settings->snapshot_storage_path);
|
||||
}
|
||||
|
||||
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.has_leader = hasLeader();
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -13,22 +13,20 @@
|
||||
#include <functional>
|
||||
#include <Coordination/KeeperServer.h>
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
|
||||
#include <Coordination/Keeper4LWInfo.h>
|
||||
#include <Coordination/KeeperConnectionStats.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using ZooKeeperResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr & response)>;
|
||||
|
||||
/// Highlevel wrapper for ClickHouse Keeper.
|
||||
/// Process user requests via consensus and return responses.
|
||||
class KeeperDispatcher
|
||||
{
|
||||
|
||||
private:
|
||||
std::mutex push_request_mutex;
|
||||
mutable std::mutex push_request_mutex;
|
||||
|
||||
CoordinationSettingsPtr coordination_settings;
|
||||
using RequestsQueue = ConcurrentBoundedQueue<KeeperStorage::RequestForSession>;
|
||||
using SessionToResponseCallback = std::unordered_map<int64_t, ZooKeeperResponseCallback>;
|
||||
using UpdateConfigurationQueue = ConcurrentBoundedQueue<ConfigUpdateAction>;
|
||||
@ -43,7 +41,7 @@ private:
|
||||
|
||||
std::atomic<bool> 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
|
||||
@ -70,6 +68,11 @@ private:
|
||||
/// RAFT wrapper.
|
||||
std::unique_ptr<KeeperServer> server;
|
||||
|
||||
mutable std::mutex keeper_stats_mutex;
|
||||
KeeperConnectionStats keeper_stats;
|
||||
|
||||
KeeperConfigurationAndSettingsPtr configuration_and_settings;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
/// Counter for new session_id requests.
|
||||
@ -123,6 +126,18 @@ public:
|
||||
/// Put request to ClickHouse Keeper
|
||||
bool putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id);
|
||||
|
||||
/// Get new session ID
|
||||
int64_t getSessionID(int64_t session_timeout_ms);
|
||||
|
||||
/// Register session and subscribe for responses with callback
|
||||
void registerSession(int64_t session_id, ZooKeeperResponseCallback callback);
|
||||
|
||||
/// 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 updateKeeperStatLatency(uint64_t process_time_ms);
|
||||
|
||||
/// Are we leader
|
||||
bool isLeader() const
|
||||
{
|
||||
@ -134,14 +149,51 @@ public:
|
||||
return server->isLeaderAlive();
|
||||
}
|
||||
|
||||
/// Get new session ID
|
||||
int64_t getSessionID(int64_t session_timeout_ms);
|
||||
bool isObserver() const
|
||||
{
|
||||
return server->isObserver();
|
||||
}
|
||||
|
||||
/// Register session and subscribe for responses with callback
|
||||
void registerSession(int64_t session_id, ZooKeeperResponseCallback callback);
|
||||
uint64_t getLogDirSize() const;
|
||||
|
||||
/// Call if we don't need any responses for this session no more (session was expired)
|
||||
void finishSession(int64_t session_id);
|
||||
uint64_t getSnapDirSize() const;
|
||||
|
||||
/// Request statistics such as qps, latency etc.
|
||||
KeeperConnectionStats getKeeperConnectionStats() const
|
||||
{
|
||||
std::lock_guard lock(keeper_stats_mutex);
|
||||
return keeper_stats;
|
||||
}
|
||||
|
||||
Keeper4LWInfo getKeeper4LWInfo() const;
|
||||
|
||||
const KeeperStateMachine & getStateMachine() const
|
||||
{
|
||||
return *server->getKeeperStateMachine();
|
||||
}
|
||||
|
||||
const KeeperConfigurationAndSettingsPtr & getKeeperConfigurationAndSettings() const
|
||||
{
|
||||
return configuration_and_settings;
|
||||
}
|
||||
|
||||
void incrementPacketsSent()
|
||||
{
|
||||
std::lock_guard lock(keeper_stats_mutex);
|
||||
keeper_stats.incrementPacketsSent();
|
||||
}
|
||||
|
||||
void incrementPacketsReceived()
|
||||
{
|
||||
std::lock_guard lock(keeper_stats_mutex);
|
||||
keeper_stats.incrementPacketsReceived();
|
||||
}
|
||||
|
||||
void resetConnectionStats()
|
||||
{
|
||||
std::lock_guard lock(keeper_stats_mutex);
|
||||
keeper_stats.reset();
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -61,27 +61,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<std::string> 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")
|
||||
@ -93,20 +78,18 @@ std::string checkAndGetSuperdigest(const Poco::Util::AbstractConfiguration & con
|
||||
}
|
||||
|
||||
KeeperServer::KeeperServer(
|
||||
int server_id_,
|
||||
const CoordinationSettingsPtr & coordination_settings_,
|
||||
const KeeperConfigurationAndSettingsPtr & configuration_and_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(configuration_and_settings_->server_id)
|
||||
, coordination_settings(configuration_and_settings_->coordination_settings)
|
||||
, state_machine(nuraft::cs_new<KeeperStateMachine>(
|
||||
responses_queue_, snapshots_queue_,
|
||||
getSnapshotsPathFromConfig(config, standalone_keeper),
|
||||
configuration_and_settings_->snapshot_storage_path,
|
||||
coordination_settings,
|
||||
checkAndGetSuperdigest(config)))
|
||||
, state_manager(nuraft::cs_new<KeeperStateManager>(server_id, "keeper_server", config, coordination_settings, standalone_keeper))
|
||||
checkAndGetSuperdigest(configuration_and_settings_->super_digest)))
|
||||
, state_manager(nuraft::cs_new<KeeperStateManager>(server_id, "keeper_server", configuration_and_settings_->log_storage_path, config, coordination_settings))
|
||||
, log(&Poco::Logger::get("KeeperServer"))
|
||||
{
|
||||
if (coordination_settings->quorum_reads)
|
||||
@ -302,11 +285,46 @@ 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();
|
||||
}
|
||||
|
||||
/// TODO test whether taking failed peer in count
|
||||
uint64_t KeeperServer::getFollowerCount() const
|
||||
{
|
||||
return raft_instance->get_peer_info_all().size();
|
||||
}
|
||||
|
||||
uint64_t KeeperServer::getSyncedFollowerCount() const
|
||||
{
|
||||
uint64_t last_log_idx = raft_instance->get_last_log_idx();
|
||||
const auto followers = raft_instance->get_peer_info_all();
|
||||
|
||||
uint64_t stale_followers = 0;
|
||||
|
||||
const uint64_t stale_follower_gap = raft_instance->get_current_params().stale_log_gap_;
|
||||
for (const auto & fl : followers)
|
||||
{
|
||||
if (last_log_idx > fl.last_log_idx_ + stale_follower_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)
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Coordination/KeeperStateMachine.h>
|
||||
#include <Coordination/KeeperStorage.h>
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <unordered_map>
|
||||
#include <base/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
@ -52,12 +51,10 @@ private:
|
||||
|
||||
public:
|
||||
KeeperServer(
|
||||
int server_id_,
|
||||
const CoordinationSettingsPtr & coordination_settings_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const KeeperConfigurationAndSettingsPtr & 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,10 +70,25 @@ public:
|
||||
/// Return set of the non-active sessions
|
||||
std::vector<int64_t> getDeadSessions();
|
||||
|
||||
nuraft::ptr<KeeperStateMachine> getKeeperStateMachine() const
|
||||
{
|
||||
return state_machine;
|
||||
}
|
||||
|
||||
bool isLeader() const;
|
||||
|
||||
bool isFollower() const;
|
||||
|
||||
bool isObserver() const;
|
||||
|
||||
bool isLeaderAlive() const;
|
||||
|
||||
/// @return follower count if node is not leader return 0
|
||||
uint64_t getFollowerCount() const;
|
||||
|
||||
/// @return synced follower count if node is not leader return 0
|
||||
uint64_t getSyncedFollowerCount() const;
|
||||
|
||||
/// Wait server initialization (see callbackFunc)
|
||||
void waitInit();
|
||||
|
||||
|
@ -326,16 +326,82 @@ 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<int64_t> KeeperStateMachine::getDeadSessions()
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getDeadSessions();
|
||||
}
|
||||
|
||||
void KeeperStateMachine::shutdownStorage()
|
||||
uint64_t KeeperStateMachine::getLastProcessedZxid() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
storage->finalize();
|
||||
return storage->getZXID();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getNodesCount() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getNodesCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getTotalWatchesCount() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getTotalWatchesCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getWatchedPathsCount() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getWatchedPathsCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getSessionsWithWatchesCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
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
|
||||
{
|
||||
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::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
storage->dumpSessionsAndEphemerals(buf);
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getApproximateDataSize() const
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
return storage->getApproximateDataSize();
|
||||
}
|
||||
|
||||
ClusterConfigPtr KeeperStateMachine::getClusterConfig() const
|
||||
|
@ -67,19 +67,36 @@ public:
|
||||
nuraft::ptr<nuraft::buffer> & data_out,
|
||||
bool & is_last_obj) override;
|
||||
|
||||
/// just for test
|
||||
KeeperStorage & getStorage()
|
||||
{
|
||||
return *storage;
|
||||
}
|
||||
|
||||
void shutdownStorage();
|
||||
|
||||
ClusterConfigPtr getClusterConfig() const;
|
||||
|
||||
/// Process local read request
|
||||
void processReadRequest(const KeeperStorage::RequestForSession & request_for_session);
|
||||
|
||||
std::vector<int64_t> getDeadSessions();
|
||||
|
||||
void shutdownStorage();
|
||||
/// Introspection functions for 4lw commands
|
||||
uint64_t getLastProcessedZxid() const;
|
||||
|
||||
ClusterConfigPtr getClusterConfig() const;
|
||||
uint64_t getNodesCount() const;
|
||||
uint64_t getTotalWatchesCount() const;
|
||||
uint64_t getWatchedPathsCount() const;
|
||||
uint64_t getSessionsWithWatchesCount() const;
|
||||
|
||||
void dumpWatches(WriteBufferFromOwnString & buf) const;
|
||||
void dumpWatchesByPath(WriteBufferFromOwnString & buf) const;
|
||||
void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const;
|
||||
|
||||
uint64_t getSessionWithEphemeralNodesCount() const;
|
||||
uint64_t getTotalEphemeralNodesCount() const;
|
||||
uint64_t getApproximateDataSize() const;
|
||||
|
||||
private:
|
||||
|
||||
@ -110,7 +127,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<uint64_t> last_committed_idx;
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Coordination/KeeperStateManager.h>
|
||||
|
||||
#include <Coordination/Defines.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <filesystem>
|
||||
@ -11,28 +12,7 @@ namespace ErrorCodes
|
||||
extern const int RAFT_ERROR;
|
||||
}
|
||||
|
||||
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
|
||||
KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const
|
||||
{
|
||||
KeeperConfigurationWrapper result;
|
||||
result.cluster_config = std::make_shared<nuraft::cluster_config>();
|
||||
@ -78,9 +58,9 @@ KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const P
|
||||
}
|
||||
|
||||
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<KeeperLogStore>(logs_path, 5000, false, false))
|
||||
: my_server_id(server_id_)
|
||||
, secure(false)
|
||||
, log_store(nuraft::cs_new<KeeperLogStore>(logs_path, 5000, false, false))
|
||||
{
|
||||
auto peer_config = nuraft::cs_new<nuraft::srv_config>(my_server_id, host + ":" + std::to_string(port));
|
||||
configuration_wrapper.cluster_config = nuraft::cs_new<nuraft::cluster_config>();
|
||||
@ -90,18 +70,20 @@ 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_)
|
||||
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<KeeperLogStore>(
|
||||
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,
|
||||
coordination_settings->compress_logs))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -13,20 +13,6 @@ namespace DB
|
||||
|
||||
using KeeperServerConfigPtr = nuraft::ptr<nuraft::srv_config>;
|
||||
|
||||
/// 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<int> servers_start_as_followers;
|
||||
/// Cluster config
|
||||
ClusterConfigPtr cluster_config;
|
||||
};
|
||||
|
||||
/// When our configuration changes the following action types
|
||||
/// can happen
|
||||
enum class ConfigUpdateActionType
|
||||
@ -52,9 +38,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(
|
||||
@ -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<int> servers_start_as_followers;
|
||||
/// Cluster config
|
||||
ClusterConfigPtr cluster_config;
|
||||
};
|
||||
|
||||
int my_server_id;
|
||||
bool secure;
|
||||
std::string config_prefix;
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Poco/SHA1Engine.h>
|
||||
#include <Poco/Base64Encoder.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <Common/hex.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -132,6 +133,21 @@ 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;
|
||||
@ -1220,4 +1236,96 @@ void KeeperStorage::clearDeadWatches(int64_t session_id)
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
for (const auto & [session_id, watches_paths] : sessions_and_watchers)
|
||||
{
|
||||
buf << "0x" << getHexUIntLowercase(session_id) << "\n";
|
||||
for (const String & path : watches_paths)
|
||||
buf << "\t" << path << "\n";
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
auto write_int_vec = [&buf](const std::vector<int64_t> & session_ids)
|
||||
{
|
||||
for (int64_t session_id : session_ids)
|
||||
{
|
||||
buf << "\t0x" << getHexUIntLowercase(session_id) << "\n";
|
||||
}
|
||||
};
|
||||
|
||||
for (const auto & [watch_path, sessions] : watches)
|
||||
{
|
||||
buf << watch_path << "\n";
|
||||
write_int_vec(sessions);
|
||||
}
|
||||
|
||||
for (const auto & [watch_path, sessions] : list_watches)
|
||||
{
|
||||
buf << watch_path << "\n";
|
||||
write_int_vec(sessions);
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
auto write_str_set = [&buf](const std::unordered_set<String> & ephemeral_paths)
|
||||
{
|
||||
for (const String & path : ephemeral_paths)
|
||||
{
|
||||
buf << "\t" << path << "\n";
|
||||
}
|
||||
};
|
||||
|
||||
buf << "Sessions dump (" << session_and_timeout.size() << "):\n";
|
||||
|
||||
for (const auto & [session_id, _] : session_and_timeout)
|
||||
{
|
||||
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<int64_t> 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;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||
@ -14,7 +13,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using namespace DB;
|
||||
struct KeeperStorageRequestProcessor;
|
||||
using KeeperStorageRequestProcessorPtr = std::shared_ptr<KeeperStorageRequestProcessor>;
|
||||
using ResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr &)>;
|
||||
@ -29,8 +27,6 @@ struct KeeperStorageSnapshot;
|
||||
class KeeperStorage
|
||||
{
|
||||
public:
|
||||
int64_t session_id_counter{1};
|
||||
|
||||
struct Node
|
||||
{
|
||||
String data;
|
||||
@ -39,6 +35,9 @@ public:
|
||||
Coordination::Stat stat{};
|
||||
int32_t seq_num = 0;
|
||||
ChildrenSet children{};
|
||||
|
||||
/// Object memory size
|
||||
uint64_t sizeInBytes() const;
|
||||
};
|
||||
|
||||
struct ResponseForSession
|
||||
@ -46,7 +45,6 @@ public:
|
||||
int64_t session_id;
|
||||
Coordination::ZooKeeperResponsePtr response;
|
||||
};
|
||||
|
||||
using ResponsesForSessions = std::vector<ResponseForSession>;
|
||||
|
||||
struct RequestForSession
|
||||
@ -76,10 +74,13 @@ public:
|
||||
/// Just vector of SHA1 from user:password
|
||||
using AuthIDs = std::vector<AuthID>;
|
||||
using SessionAndAuth = std::unordered_map<int64_t, AuthIDs>;
|
||||
SessionAndAuth session_and_auth;
|
||||
|
||||
using Watches = std::map<String /* path, relative of root_path */, SessionIDs>;
|
||||
|
||||
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.
|
||||
@ -176,6 +177,36 @@ public:
|
||||
{
|
||||
return session_expiry_queue.getExpiredSessions();
|
||||
}
|
||||
|
||||
/// Introspection functions mostly used in 4-letter commands
|
||||
uint64_t getNodesCount() const
|
||||
{
|
||||
return container.size();
|
||||
}
|
||||
|
||||
uint64_t getApproximateDataSize() const
|
||||
{
|
||||
return container.getApproximateDataSize();
|
||||
}
|
||||
|
||||
uint64_t getTotalWatchesCount() const;
|
||||
|
||||
uint64_t getWatchedPathsCount() const
|
||||
{
|
||||
return watches.size() + list_watches.size();
|
||||
}
|
||||
|
||||
uint64_t getSessionsWithWatchesCount() const;
|
||||
|
||||
uint64_t getSessionWithEphemeralNodesCount() const
|
||||
{
|
||||
return ephemerals.size();
|
||||
}
|
||||
uint64_t getTotalEphemeralNodesCount() const;
|
||||
|
||||
void dumpWatches(WriteBufferFromOwnString & buf) const;
|
||||
void dumpWatchesByPath(WriteBufferFromOwnString & buf) const;
|
||||
void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const;
|
||||
};
|
||||
|
||||
using KeeperStoragePtr = std::unique_ptr<KeeperStorage>;
|
||||
|
@ -15,6 +15,7 @@ struct ListNode
|
||||
bool active_in_map;
|
||||
};
|
||||
|
||||
|
||||
template <class V>
|
||||
class SnapshotableHashTable
|
||||
{
|
||||
@ -28,6 +29,82 @@ private:
|
||||
IndexMap map;
|
||||
bool snapshot_mode{false};
|
||||
|
||||
uint64_t 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 updateDataSize(OperationType op_type, uint64_t key_size, uint64_t value_size, uint64_t 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 != 0)
|
||||
{
|
||||
approximate_data_size += key_size;
|
||||
approximate_data_size += value_size;
|
||||
if (!snapshot_mode)
|
||||
{
|
||||
approximate_data_size += key_size;
|
||||
approximate_data_size -= old_value_size;
|
||||
}
|
||||
}
|
||||
/// insert
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
|
||||
using iterator = typename List::iterator;
|
||||
@ -44,6 +121,7 @@ public:
|
||||
ListElem elem{key, value, true};
|
||||
auto itr = list.insert(list.end(), elem);
|
||||
map.emplace(itr->key, itr);
|
||||
updateDataSize(INSERT, key.size(), value.sizeInBytes(), 0);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -54,6 +132,8 @@ public:
|
||||
void insertOrReplace(const std::string & key, const V & value)
|
||||
{
|
||||
auto it = map.find(key);
|
||||
uint64_t old_value_size = it == map.end() ? 0 : it->second->value.sizeInBytes();
|
||||
|
||||
if (it == map.end())
|
||||
{
|
||||
ListElem elem{key, value, true};
|
||||
@ -76,6 +156,7 @@ public:
|
||||
list_itr->value = value;
|
||||
}
|
||||
}
|
||||
updateDataSize(INSERT_OR_REPLACE, key.size(), value.sizeInBytes(), old_value_size);
|
||||
}
|
||||
|
||||
bool erase(const std::string & key)
|
||||
@ -85,6 +166,7 @@ public:
|
||||
return false;
|
||||
|
||||
auto list_itr = it->second;
|
||||
uint64_t old_data_size = list_itr->value.sizeInBytes();
|
||||
if (snapshot_mode)
|
||||
{
|
||||
list_itr->active_in_map = false;
|
||||
@ -96,6 +178,7 @@ public:
|
||||
list.erase(list_itr);
|
||||
}
|
||||
|
||||
updateDataSize(ERASE, key.size(), 0, old_data_size);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -108,23 +191,29 @@ public:
|
||||
{
|
||||
auto it = map.find(key);
|
||||
assert(it != map.end());
|
||||
|
||||
auto list_itr = it->second;
|
||||
uint64_t old_value_size = list_itr->value.sizeInBytes();
|
||||
|
||||
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, key.size(), ret->value.sizeInBytes(), old_value_size);
|
||||
return ret;
|
||||
}
|
||||
|
||||
const_iterator find(const std::string & key) const
|
||||
@ -149,7 +238,10 @@ public:
|
||||
for (auto itr = start; itr != end;)
|
||||
{
|
||||
if (!itr->active_in_map)
|
||||
{
|
||||
updateDataSize(CLEAR_OUTDATED_NODES, itr->key.size(), itr->value.sizeInBytes(), 0);
|
||||
itr = list.erase(itr);
|
||||
}
|
||||
else
|
||||
itr++;
|
||||
}
|
||||
@ -159,6 +251,7 @@ public:
|
||||
{
|
||||
list.clear();
|
||||
map.clear();
|
||||
updateDataSize(CLEAR, 0, 0, 0);
|
||||
}
|
||||
|
||||
void enableSnapshotMode()
|
||||
@ -181,6 +274,10 @@ public:
|
||||
return list.size();
|
||||
}
|
||||
|
||||
uint64_t getApproximateDataSize() const
|
||||
{
|
||||
return approximate_data_size;
|
||||
}
|
||||
|
||||
iterator begin() { return list.begin(); }
|
||||
const_iterator begin() const { return list.cbegin(); }
|
||||
|
@ -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_) { } // NOLINT(google-explicit-constructor)
|
||||
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<int> hello;
|
||||
DB::SnapshotableHashTable<IntNode> 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<int> map_snp;
|
||||
DB::SnapshotableHashTable<IntNode> 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);
|
||||
@ -921,6 +935,73 @@ TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot)
|
||||
map_snp.disableSnapshotMode();
|
||||
}
|
||||
|
||||
TEST_P(CoordinationTest, SnapshotableHashMapDataSize)
|
||||
{
|
||||
/// int
|
||||
DB::SnapshotableHashTable<IntNode> hello;
|
||||
hello.disableSnapshotMode();
|
||||
EXPECT_EQ(hello.getApproximateDataSize(), 0);
|
||||
|
||||
hello.insert("hello", 1);
|
||||
EXPECT_EQ(hello.getApproximateDataSize(), 9);
|
||||
hello.updateValue("hello", [](IntNode & value) { value = 2; });
|
||||
EXPECT_EQ(hello.getApproximateDataSize(), 9);
|
||||
|
||||
hello.erase("hello");
|
||||
EXPECT_EQ(hello.getApproximateDataSize(), 0);
|
||||
|
||||
hello.clear();
|
||||
EXPECT_EQ(hello.getApproximateDataSize(), 0);
|
||||
|
||||
hello.enableSnapshotMode();
|
||||
hello.insert("hello", 1);
|
||||
EXPECT_EQ(hello.getApproximateDataSize(), 9);
|
||||
hello.updateValue("hello", [](IntNode & value) { value = 2; });
|
||||
EXPECT_EQ(hello.getApproximateDataSize(), 18);
|
||||
|
||||
hello.clearOutdatedNodes();
|
||||
EXPECT_EQ(hello.getApproximateDataSize(), 9);
|
||||
|
||||
hello.erase("hello");
|
||||
EXPECT_EQ(hello.getApproximateDataSize(), 9);
|
||||
|
||||
hello.clearOutdatedNodes();
|
||||
EXPECT_EQ(hello.getApproximateDataSize(), 0);
|
||||
|
||||
/// Node
|
||||
using Node = DB::KeeperStorage::Node;
|
||||
DB::SnapshotableHashTable<Node> world;
|
||||
Node n1;
|
||||
n1.data = "1234";
|
||||
Node n2;
|
||||
n2.data = "123456";
|
||||
n2.children.insert("");
|
||||
|
||||
world.disableSnapshotMode();
|
||||
world.insert("world", n1);
|
||||
EXPECT_EQ(world.getApproximateDataSize(), 94);
|
||||
world.updateValue("world", [&](Node & value) { value = n2; });
|
||||
EXPECT_EQ(world.getApproximateDataSize(), 96);
|
||||
|
||||
world.erase("world");
|
||||
EXPECT_EQ(world.getApproximateDataSize(), 0);
|
||||
|
||||
world.enableSnapshotMode();
|
||||
world.insert("world", n1);
|
||||
EXPECT_EQ(world.getApproximateDataSize(), 94);
|
||||
world.updateValue("world", [&](Node & value) { value = n2; });
|
||||
EXPECT_EQ(world.getApproximateDataSize(), 190);
|
||||
|
||||
world.clearOutdatedNodes();
|
||||
EXPECT_EQ(world.getApproximateDataSize(), 96);
|
||||
|
||||
world.erase("world");
|
||||
EXPECT_EQ(world.getApproximateDataSize(), 96);
|
||||
|
||||
world.clear();
|
||||
EXPECT_EQ(world.getApproximateDataSize(), 0);
|
||||
}
|
||||
|
||||
void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner=0)
|
||||
{
|
||||
using Node = DB::KeeperStorage::Node;
|
||||
|
@ -18,6 +18,9 @@
|
||||
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||
#include <queue>
|
||||
#include <mutex>
|
||||
#include <Coordination/FourLetterCommand.h>
|
||||
#include <Common/hex.h>
|
||||
|
||||
|
||||
#ifdef POCO_HAVE_FD_EPOLL
|
||||
#include <sys/epoll.h>
|
||||
@ -29,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
|
||||
{
|
||||
@ -199,7 +212,9 @@ 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<SocketInterruptablePollWrapper>(socket_))
|
||||
, responses(std::make_unique<ThreadSafeResponseQueue>(std::numeric_limits<size_t>::max()))
|
||||
, last_op(std::make_unique<LastOp>(EMPTY_LAST_OP))
|
||||
{
|
||||
KeeperTCPHandler::registerConnection(this);
|
||||
}
|
||||
|
||||
void KeeperTCPHandler::sendHandshake(bool has_leader)
|
||||
@ -222,16 +237,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<char, Coordination::PASSWORD_LENGTH> 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 +288,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);
|
||||
}
|
||||
@ -345,6 +382,7 @@ void KeeperTCPHandler::runImpl()
|
||||
|
||||
session_stopwatch.start();
|
||||
bool close_received = false;
|
||||
|
||||
try
|
||||
{
|
||||
while (true)
|
||||
@ -356,6 +394,7 @@ void KeeperTCPHandler::runImpl()
|
||||
if (result.has_requests && !close_received)
|
||||
{
|
||||
auto [received_op, received_xid] = receiveRequest();
|
||||
packageReceived();
|
||||
log_long_operation("Receiving request");
|
||||
|
||||
if (received_op == Coordination::OpNum::Close)
|
||||
@ -368,6 +407,8 @@ 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();
|
||||
@ -390,6 +431,9 @@ void KeeperTCPHandler::runImpl()
|
||||
return;
|
||||
}
|
||||
|
||||
updateStats(response);
|
||||
packageSent();
|
||||
|
||||
response->write(*out);
|
||||
log_long_operation("Sending response");
|
||||
if (response->error == Coordination::Error::ZSESSIONEXPIRED)
|
||||
@ -422,6 +466,44 @@ void KeeperTCPHandler::runImpl()
|
||||
}
|
||||
}
|
||||
|
||||
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_t command)
|
||||
{
|
||||
if (!FourLetterCommandFactory::instance().isKnown(command))
|
||||
{
|
||||
LOG_WARNING(log, "invalid four letter command {}", IFourLetterCommand::toName(command));
|
||||
return false;
|
||||
}
|
||||
else if (!FourLetterCommandFactory::instance().isEnabled(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());
|
||||
|
||||
try
|
||||
{
|
||||
String res = command_ptr->run();
|
||||
out->write(res.data(), res.size());
|
||||
out->next();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Error when executing four letter command " + command_ptr->name());
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
std::pair<Coordination::OpNum, Coordination::XID> KeeperTCPHandler::receiveRequest()
|
||||
{
|
||||
int32_t length;
|
||||
@ -441,6 +523,148 @@ std::pair<Coordination::OpNum, Coordination::XID> KeeperTCPHandler::receiveReque
|
||||
return std::make_pair(opnum, xid);
|
||||
}
|
||||
|
||||
void KeeperTCPHandler::packageSent()
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(conn_stats_mutex);
|
||||
conn_stats.incrementPacketsSent();
|
||||
}
|
||||
keeper_dispatcher->incrementPacketsSent();
|
||||
}
|
||||
|
||||
void KeeperTCPHandler::packageReceived()
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(conn_stats_mutex);
|
||||
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;
|
||||
{
|
||||
std::lock_guard lock(conn_stats_mutex);
|
||||
conn_stats.updateLatency(elapsed);
|
||||
}
|
||||
keeper_dispatcher->updateKeeperStatLatency(elapsed);
|
||||
|
||||
last_op.set(std::make_unique<LastOp>(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(stats.getPacketsReceived(), buf);
|
||||
writeText(",sent=", buf);
|
||||
writeIntText(stats.getPacketsSent(), buf);
|
||||
if (!brief)
|
||||
{
|
||||
if (session_id != 0)
|
||||
{
|
||||
writeText(",sid=0x", buf);
|
||||
writeText(getHexUIntLowercase(session_id), buf);
|
||||
|
||||
writeText(",lop=", buf);
|
||||
LastOpPtr op = last_op.get();
|
||||
writeText(op->name, buf);
|
||||
writeText(",est=", buf);
|
||||
writeIntText(established.epochMicroseconds() / 1000, buf);
|
||||
writeText(",to=", buf);
|
||||
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->last_zxid), buf);
|
||||
writeText(",lresp=", buf);
|
||||
writeIntText(op->last_response_time, buf);
|
||||
|
||||
writeText(",llat=", buf);
|
||||
writeIntText(stats.getLastLatency(), buf);
|
||||
writeText(",minlat=", buf);
|
||||
writeIntText(stats.getMinLatency(), buf);
|
||||
writeText(",avglat=", buf);
|
||||
writeIntText(stats.getAvgLatency(), buf);
|
||||
writeText(",maxlat=", buf);
|
||||
writeIntText(stats.getMaxLatency(), buf);
|
||||
}
|
||||
}
|
||||
writeText(')', buf);
|
||||
writeText('\n', buf);
|
||||
}
|
||||
|
||||
void KeeperTCPHandler::resetStats()
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(conn_stats_mutex);
|
||||
conn_stats.reset();
|
||||
}
|
||||
last_op.set(std::make_unique<LastOp>(EMPTY_LAST_OP));
|
||||
}
|
||||
|
||||
KeeperTCPHandler::~KeeperTCPHandler()
|
||||
{
|
||||
KeeperTCPHandler::unregisterConnection(this);
|
||||
}
|
||||
|
||||
std::mutex KeeperTCPHandler::conns_mutex;
|
||||
std::unordered_set<KeeperTCPHandler *> 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
|
||||
|
@ -6,6 +6,7 @@
|
||||
#if USE_NURAFT
|
||||
|
||||
#include <Poco/Net/TCPServerConnection.h>
|
||||
#include <Common/MultiVersion.h>
|
||||
#include "IServer.h"
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Interpreters/Context.h>
|
||||
@ -16,6 +17,8 @@
|
||||
#include <IO/WriteBufferFromPocoSocket.h>
|
||||
#include <IO/ReadBufferFromPocoSocket.h>
|
||||
#include <unordered_map>
|
||||
#include <Coordination/KeeperConnectionStats.h>
|
||||
#include <Poco/Timestamp.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -24,14 +27,36 @@ struct SocketInterruptablePollWrapper;
|
||||
using SocketInterruptablePollWrapperPtr = std::unique_ptr<SocketInterruptablePollWrapper>;
|
||||
|
||||
using ThreadSafeResponseQueue = ConcurrentBoundedQueue<Coordination::ZooKeeperResponsePtr>;
|
||||
|
||||
using ThreadSafeResponseQueuePtr = std::unique_ptr<ThreadSafeResponseQueue>;
|
||||
|
||||
struct LastOp;
|
||||
using LastOpMultiVersion = MultiVersion<LastOp>;
|
||||
using LastOpPtr = LastOpMultiVersion::Version;
|
||||
|
||||
class KeeperTCPHandler : public Poco::Net::TCPServerConnection
|
||||
{
|
||||
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<KeeperTCPHandler *> connections;
|
||||
|
||||
public:
|
||||
KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_);
|
||||
void run() override;
|
||||
|
||||
KeeperConnectionStats getConnectionStats() const;
|
||||
void dumpStats(WriteBufferFromOwnString & buf, bool brief);
|
||||
void resetStats();
|
||||
|
||||
~KeeperTCPHandler() override;
|
||||
|
||||
private:
|
||||
IServer & server;
|
||||
Poco::Logger * log;
|
||||
@ -54,9 +79,28 @@ private:
|
||||
void runImpl();
|
||||
|
||||
void sendHandshake(bool has_leader);
|
||||
Poco::Timespan receiveHandshake();
|
||||
Poco::Timespan receiveHandshake(int32_t handshake_length);
|
||||
|
||||
static bool isHandShake(int32_t handshake_length);
|
||||
bool tryExecuteFourLetterWordCmd(int32_t command);
|
||||
|
||||
std::pair<Coordination::OpNum, Coordination::XID> receiveRequest();
|
||||
|
||||
void packageSent();
|
||||
void packageReceived();
|
||||
|
||||
void updateStats(Coordination::ZooKeeperResponsePtr & response);
|
||||
|
||||
Poco::Timestamp established;
|
||||
|
||||
using Operations = std::map<Coordination::XID, Poco::Timestamp>;
|
||||
Operations operations;
|
||||
|
||||
LastOpMultiVersion last_op;
|
||||
|
||||
mutable std::mutex conn_stats_mutex;
|
||||
KeeperConnectionStats conn_stats;
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -0,0 +1 @@
|
||||
#!/usr/bin/env python3
|
@ -0,0 +1,42 @@
|
||||
<yandex>
|
||||
<keeper_server>
|
||||
<tcp_port>9181</tcp_port>
|
||||
<server_id>1</server_id>
|
||||
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
|
||||
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
|
||||
<four_letter_word_white_list>*</four_letter_word_white_list>
|
||||
|
||||
<coordination_settings>
|
||||
<operation_timeout_ms>5000</operation_timeout_ms>
|
||||
<session_timeout_ms>30000</session_timeout_ms>
|
||||
<snapshot_distance>75</snapshot_distance>
|
||||
<raft_logs_level>trace</raft_logs_level>
|
||||
</coordination_settings>
|
||||
|
||||
<raft_configuration>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>node1</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<priority>3</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>2</id>
|
||||
<hostname>node2</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>false</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>2</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>3</id>
|
||||
<hostname>node3</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>false</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>1</priority>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
</yandex>
|
@ -0,0 +1,42 @@
|
||||
<yandex>
|
||||
<keeper_server>
|
||||
<tcp_port>9181</tcp_port>
|
||||
<server_id>2</server_id>
|
||||
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
|
||||
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
|
||||
<four_letter_word_white_list>*</four_letter_word_white_list>
|
||||
|
||||
<coordination_settings>
|
||||
<operation_timeout_ms>5000</operation_timeout_ms>
|
||||
<session_timeout_ms>30000</session_timeout_ms>
|
||||
<snapshot_distance>75</snapshot_distance>
|
||||
<raft_logs_level>trace</raft_logs_level>
|
||||
</coordination_settings>
|
||||
|
||||
<raft_configuration>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>node1</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<priority>3</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>2</id>
|
||||
<hostname>node2</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>false</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>2</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>3</id>
|
||||
<hostname>node3</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>false</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>1</priority>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
</yandex>
|
@ -0,0 +1,42 @@
|
||||
<yandex>
|
||||
<keeper_server>
|
||||
<tcp_port>9181</tcp_port>
|
||||
<server_id>3</server_id>
|
||||
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
|
||||
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
|
||||
<four_letter_word_white_list>*</four_letter_word_white_list>
|
||||
|
||||
<coordination_settings>
|
||||
<operation_timeout_ms>5000</operation_timeout_ms>
|
||||
<session_timeout_ms>30000</session_timeout_ms>
|
||||
<snapshot_distance>75</snapshot_distance>
|
||||
<raft_logs_level>trace</raft_logs_level>
|
||||
</coordination_settings>
|
||||
|
||||
<raft_configuration>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>node1</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<priority>3</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>2</id>
|
||||
<hostname>node2</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>false</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>2</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>3</id>
|
||||
<hostname>node3</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>false</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>1</priority>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
</yandex>
|
@ -0,0 +1,38 @@
|
||||
<yandex>
|
||||
<keeper_server>
|
||||
<tcp_port>9181</tcp_port>
|
||||
<server_id>1</server_id>
|
||||
<four_letter_word_white_list>ruok, conf</four_letter_word_white_list>
|
||||
<raft_configuration>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>node1</hostname>
|
||||
<port>44444</port>
|
||||
</server>
|
||||
<server>
|
||||
<id>2</id>
|
||||
<hostname>node2</hostname>
|
||||
<port>44444</port>
|
||||
</server>
|
||||
<server>
|
||||
<id>3</id>
|
||||
<hostname>node3</hostname>
|
||||
<port>44444</port>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
<zookeeper>
|
||||
<node index="1">
|
||||
<host>node1</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="2">
|
||||
<host>node2</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="3">
|
||||
<host>node3</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
</zookeeper>
|
||||
</yandex>
|
@ -0,0 +1,38 @@
|
||||
<yandex>
|
||||
<keeper_server>
|
||||
<tcp_port>9181</tcp_port>
|
||||
<server_id>3</server_id>
|
||||
<four_letter_word_white_list>*</four_letter_word_white_list>
|
||||
<raft_configuration>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>node1</hostname>
|
||||
<port>44444</port>
|
||||
</server>
|
||||
<server>
|
||||
<id>2</id>
|
||||
<hostname>node2</hostname>
|
||||
<port>44444</port>
|
||||
</server>
|
||||
<server>
|
||||
<id>3</id>
|
||||
<hostname>node3</hostname>
|
||||
<port>44444</port>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
<zookeeper>
|
||||
<node index="1">
|
||||
<host>node1</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="2">
|
||||
<host>node2</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="3">
|
||||
<host>node3</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
</zookeeper>
|
||||
</yandex>
|
@ -0,0 +1,37 @@
|
||||
<yandex>
|
||||
<keeper_server>
|
||||
<tcp_port>9181</tcp_port>
|
||||
<server_id>2</server_id>
|
||||
<raft_configuration>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>node1</hostname>
|
||||
<port>44444</port>
|
||||
</server>
|
||||
<server>
|
||||
<id>2</id>
|
||||
<hostname>node2</hostname>
|
||||
<port>44444</port>
|
||||
</server>
|
||||
<server>
|
||||
<id>3</id>
|
||||
<hostname>node3</hostname>
|
||||
<port>44444</port>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
<zookeeper>
|
||||
<node index="1">
|
||||
<host>node1</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="2">
|
||||
<host>node2</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="3">
|
||||
<host>node3</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
</zookeeper>
|
||||
</yandex>
|
611
tests/integration/test_keeper_four_word_command/test.py
Normal file
611
tests/integration/test_keeper_four_word_command/test.py
Normal file
@ -0,0 +1,611 @@
|
||||
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
|
||||
import re
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
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'],
|
||||
stay_alive=True)
|
||||
node3 = cluster.add_instance('node3', main_configs=['configs/enable_keeper3.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 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
|
||||
try:
|
||||
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(node_name):
|
||||
hosts = cluster.get_instance_ip(node_name)
|
||||
client = socket.socket()
|
||||
client.settimeout(10)
|
||||
client.connect((hosts, 9181))
|
||||
return client
|
||||
|
||||
|
||||
def close_keeper_socket(cli):
|
||||
if cli is not None:
|
||||
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()
|
||||
data = send_4lw_cmd(cmd='ruok')
|
||||
assert data == '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
|
||||
# ensure not delete watched node
|
||||
assert create_cnt >= (delete_cnt + watch_cnt)
|
||||
|
||||
for i in range(create_cnt):
|
||||
zk.create("/test_4lw_normal_node_" + str(i), b"")
|
||||
|
||||
for i in range(get_cnt):
|
||||
zk.get("/test_4lw_normal_node_" + str(i))
|
||||
|
||||
for i in range(set_cnt):
|
||||
zk.set("/test_4lw_normal_node_" + str(i), b"new-value")
|
||||
|
||||
for i in range(ephemeral_cnt):
|
||||
zk.create("/test_4lw_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("/test_4lw_normal_node_" + str(i), watch=fake_ephemeral_callback)
|
||||
|
||||
for i in range(create_cnt - delete_cnt, create_cnt):
|
||||
zk.delete("/test_4lw_normal_node_" + str(i))
|
||||
|
||||
|
||||
def test_cmd_mntr(started_cluster):
|
||||
zk = None
|
||||
try:
|
||||
wait_nodes()
|
||||
clear_znodes()
|
||||
|
||||
# reset stat first
|
||||
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)
|
||||
|
||||
data = send_4lw_cmd(cmd='mntr')
|
||||
|
||||
# print(data.decode())
|
||||
reader = csv.reader(data.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_num_alive_connections"]) == 1
|
||||
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"]) == 11
|
||||
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
|
||||
|
||||
# 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)
|
||||
|
||||
|
||||
def test_cmd_srst(started_cluster):
|
||||
client = None
|
||||
try:
|
||||
wait_nodes()
|
||||
clear_znodes()
|
||||
|
||||
data = send_4lw_cmd(cmd='srst')
|
||||
assert data.strip() == "Server stats reset."
|
||||
|
||||
data = send_4lw_cmd(cmd='mntr')
|
||||
assert len(data) != 0
|
||||
|
||||
# print(data)
|
||||
reader = csv.reader(data.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()
|
||||
clear_znodes()
|
||||
|
||||
data = send_4lw_cmd(cmd='conf')
|
||||
|
||||
reader = csv.reader(data.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["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"
|
||||
|
||||
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"
|
||||
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"] == "180000"
|
||||
|
||||
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"
|
||||
|
||||
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)
|
||||
|
||||
|
||||
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'] == '11'
|
||||
|
||||
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'] == '11'
|
||||
|
||||
# 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)
|
||||
|
@ -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)
|
@ -0,0 +1 @@
|
||||
1
|
47
tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh
Executable file
47
tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh
Executable file
@ -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"
|
@ -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_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')}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user