2021-02-09 15:39:15 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Core/Defines.h>
|
|
|
|
#include <Core/BaseSettings.h>
|
2021-02-10 09:28:53 +00:00
|
|
|
#include <Core/SettingsEnums.h>
|
2021-02-09 15:39:15 +00:00
|
|
|
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
|
|
|
#include <Poco/Util/AbstractConfiguration.h>
|
2021-10-27 12:26:42 +00:00
|
|
|
#include <IO/WriteBufferFromString.h>
|
2021-02-09 15:39:15 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
struct Settings;
|
|
|
|
|
2021-09-21 14:29:05 +00:00
|
|
|
|
2021-02-09 15:39:15 +00:00
|
|
|
/** These settings represent fine tunes for internal details of Coordination storages
|
|
|
|
* and should not be changed by the user without a reason.
|
|
|
|
*/
|
|
|
|
|
2021-09-27 11:54:04 +00:00
|
|
|
|
2021-02-09 15:39:15 +00:00
|
|
|
#define LIST_OF_COORDINATION_SETTINGS(M) \
|
2021-12-29 12:59:01 +00:00
|
|
|
M(Milliseconds, min_session_timeout_ms, Coordination::DEFAULT_MIN_SESSION_TIMEOUT_MS, "Min client session timeout", 0) \
|
2021-12-30 09:18:51 +00:00
|
|
|
M(Milliseconds, session_timeout_ms, Coordination::DEFAULT_MAX_SESSION_TIMEOUT_MS, "Max client session timeout", 0) \
|
2021-02-09 15:39:15 +00:00
|
|
|
M(Milliseconds, operation_timeout_ms, Coordination::DEFAULT_OPERATION_TIMEOUT_MS, "Default client operation timeout", 0) \
|
|
|
|
M(Milliseconds, dead_session_check_period_ms, 500, "How often leader will check sessions to consider them dead and remove", 0) \
|
|
|
|
M(Milliseconds, heart_beat_interval_ms, 500, "Heartbeat interval between quorum nodes", 0) \
|
|
|
|
M(Milliseconds, election_timeout_lower_bound_ms, 1000, "Lower bound of election timer (avoid too often leader elections)", 0) \
|
2021-07-21 19:26:45 +00:00
|
|
|
M(Milliseconds, election_timeout_upper_bound_ms, 2000, "Upper bound of election timer (avoid too often leader elections)", 0) \
|
2021-04-13 11:55:08 +00:00
|
|
|
M(UInt64, reserved_log_items, 100000, "How many log items to store (don't remove during compaction)", 0) \
|
|
|
|
M(UInt64, snapshot_distance, 100000, "How many log items we have to collect to write new snapshot", 0) \
|
2021-02-09 15:39:15 +00:00
|
|
|
M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \
|
2022-04-19 12:03:00 +00:00
|
|
|
M(Milliseconds, shutdown_timeout, 5000, "How much time we will wait until RAFT shutdown", 0) \
|
|
|
|
M(Milliseconds, startup_timeout, 180000, "How much time we will wait until RAFT to start.", 0) \
|
2021-02-16 19:02:18 +00:00
|
|
|
M(LogsLevel, raft_logs_level, LogsLevel::information, "Log internal RAFT logs into main server log level. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \
|
2021-04-13 11:55:08 +00:00
|
|
|
M(UInt64, rotate_log_storage_interval, 100000, "How many records will be stored in one log storage file", 0) \
|
2021-03-01 14:54:08 +00:00
|
|
|
M(UInt64, snapshots_to_keep, 3, "How many compressed snapshots to keep on disk", 0) \
|
2021-03-04 13:40:43 +00:00
|
|
|
M(UInt64, stale_log_gap, 10000, "When node became stale and should receive snapshots from leader", 0) \
|
2021-03-22 10:45:22 +00:00
|
|
|
M(UInt64, fresh_log_gap, 200, "When node became fresh", 0) \
|
2021-04-16 13:50:09 +00:00
|
|
|
M(UInt64, max_requests_batch_size, 100, "Max size of batch in requests count before it will be sent to RAFT", 0) \
|
2021-03-10 16:12:32 +00:00
|
|
|
M(Bool, quorum_reads, false, "Execute read requests as writes through whole RAFT consesus with similar speed", 0) \
|
2021-09-21 14:29:05 +00:00
|
|
|
M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \
|
2021-09-27 11:54:04 +00:00
|
|
|
M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) \
|
2021-10-19 12:00:26 +00:00
|
|
|
M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0) \
|
|
|
|
M(UInt64, configuration_change_tries_count, 20, "How many times we will try to apply configuration change (add/remove server) to the cluster", 0)
|
2021-02-09 15:39:15 +00:00
|
|
|
|
|
|
|
DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
|
|
|
|
|
|
|
|
|
|
|
|
struct CoordinationSettings : public BaseSettings<CoordinationSettingsTraits>
|
|
|
|
{
|
|
|
|
void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
|
|
|
|
};
|
|
|
|
|
|
|
|
using CoordinationSettingsPtr = std::shared_ptr<CoordinationSettings>;
|
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
/// Coordination settings + some other parts of keeper configuration
|
|
|
|
/// which are not stored in settings. Allows to dump configuration
|
2021-11-19 09:30:58 +00:00
|
|
|
/// with 4lw commands.
|
2021-11-18 20:17:22 +00:00
|
|
|
struct KeeperConfigurationAndSettings
|
2021-10-27 12:26:42 +00:00
|
|
|
{
|
2021-10-28 08:01:30 +00:00
|
|
|
static constexpr int NOT_EXIST = -1;
|
2021-11-05 10:21:34 +00:00
|
|
|
static const String DEFAULT_FOUR_LETTER_WORD_CMD;
|
2021-10-27 12:26:42 +00:00
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
KeeperConfigurationAndSettings();
|
2021-10-27 12:26:42 +00:00
|
|
|
int server_id;
|
|
|
|
|
2022-01-07 09:22:20 +00:00
|
|
|
bool enable_ipv6;
|
2021-10-28 08:01:30 +00:00
|
|
|
int tcp_port;
|
|
|
|
int tcp_port_secure;
|
2021-10-27 12:26:42 +00:00
|
|
|
|
2022-03-17 10:55:15 +00:00
|
|
|
String four_letter_word_allow_list;
|
2021-10-28 14:22:56 +00:00
|
|
|
|
2021-10-27 12:26:42 +00:00
|
|
|
String super_digest;
|
|
|
|
|
|
|
|
bool standalone_keeper;
|
|
|
|
CoordinationSettingsPtr coordination_settings;
|
|
|
|
|
|
|
|
String log_storage_path;
|
|
|
|
String snapshot_storage_path;
|
|
|
|
|
|
|
|
void dump(WriteBufferFromOwnString & buf) const;
|
2021-11-18 20:17:22 +00:00
|
|
|
static std::shared_ptr<KeeperConfigurationAndSettings> loadFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_);
|
2021-10-27 12:26:42 +00:00
|
|
|
|
|
|
|
private:
|
|
|
|
static String getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_);
|
|
|
|
static String getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_);
|
|
|
|
};
|
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
using KeeperConfigurationAndSettingsPtr = std::shared_ptr<KeeperConfigurationAndSettings>;
|
2021-10-27 12:26:42 +00:00
|
|
|
|
2021-02-09 15:39:15 +00:00
|
|
|
}
|