Implement online config update

epoch-deletions
Vitaliy Filippov 2023-03-26 17:50:35 +03:00
parent 3bbc46543d
commit d06ed2b0e7
19 changed files with 332 additions and 121 deletions

View File

@ -51,8 +51,9 @@ const etcd_tree = {
// THIS IS JUST A POOR MAN'S CONFIG DOCUMENTATION
// etcd connection
config_path: "/etc/vitastor/vitastor.conf",
etcd_address: "10.0.115.10:2379/v3",
etcd_prefix: "/vitastor",
// etcd connection - configurable online
etcd_address: "10.0.115.10:2379/v3",
// mon
etcd_mon_ttl: 30, // min: 10
etcd_mon_timeout: 1000, // ms. min: 0
@ -73,11 +74,12 @@ const etcd_tree = {
rdma_max_send: 8,
rdma_max_recv: 16,
rdma_max_msg: 132096,
log_level: 0,
block_size: 131072,
disk_alignment: 4096,
bitmap_granularity: 4096,
immediate_commit: false, // 'all' or 'small'
// client and osd - configurable online
log_level: 0,
client_dirty_limit: 33554432,
peer_connect_interval: 5, // seconds. min: 1
peer_connect_timeout: 5, // seconds. min: 1
@ -95,18 +97,19 @@ const etcd_tree = {
osd_network: null, // "192.168.7.0/24" or an array of masks
bind_address: "0.0.0.0",
bind_port: 0,
readonly: false,
osd_memlock: false,
// osd - configurable online
autosync_interval: 5,
autosync_writes: 128,
client_queue_depth: 128, // unused
recovery_queue_depth: 4,
recovery_sync_batch: 16,
readonly: false,
no_recovery: false,
no_rebalance: false,
print_stats_interval: 3,
slow_log_interval: 10,
inode_vanish_time: 60,
osd_memlock: false,
// blockstore - fixed in superblock
block_size,
disk_alignment,
@ -125,14 +128,15 @@ const etcd_tree = {
meta_offset,
disable_meta_fsync,
disable_device_lock,
// blockstore - configurable
max_write_iodepth,
min_flusher_count: 1,
max_flusher_count: 256,
// blockstore - configurable offline
inmemory_metadata,
inmemory_journal,
journal_sector_buffer_count,
journal_no_same_sector_overwrites,
// blockstore - configurable online
max_write_iodepth,
min_flusher_count: 1,
max_flusher_count: 256,
throttle_small_writes: false,
throttle_target_iops: 100,
throttle_target_mbs: 100,

View File

@ -13,6 +13,11 @@ blockstore_t::~blockstore_t()
delete impl;
}
void blockstore_t::parse_config(blockstore_config_t & config)
{
impl->parse_config(config, false);
}
void blockstore_t::loop()
{
impl->loop();

View File

@ -165,6 +165,9 @@ public:
blockstore_t(blockstore_config_t & config, ring_loop_t *ringloop, timerfd_manager_t *tfd);
~blockstore_t();
// Update configuration
void parse_config(blockstore_config_t & config);
// Event loop
void loop();

View File

@ -11,7 +11,7 @@ blockstore_impl_t::blockstore_impl_t(blockstore_config_t & config, ring_loop_t *
ring_consumer.loop = [this]() { loop(); };
ringloop->register_consumer(&ring_consumer);
initialized = 0;
parse_config(config);
parse_config(config, true);
zero_object = (uint8_t*)memalign_or_die(MEM_ALIGNMENT, dsk.data_block_size);
try
{

View File

@ -277,7 +277,6 @@ class blockstore_impl_t
friend class journal_flusher_t;
friend class journal_flusher_co;
void parse_config(blockstore_config_t & config);
void calc_lengths();
void open_data();
void open_meta();
@ -341,6 +340,8 @@ public:
blockstore_impl_t(blockstore_config_t & config, ring_loop_t *ringloop, timerfd_manager_t *tfd);
~blockstore_impl_t();
void parse_config(blockstore_config_t & config, bool init);
// Event loop
void loop();

View File

@ -4,8 +4,54 @@
#include <sys/file.h>
#include "blockstore_impl.h"
void blockstore_impl_t::parse_config(blockstore_config_t & config)
void blockstore_impl_t::parse_config(blockstore_config_t & config, bool init)
{
// Online-configurable options:
max_flusher_count = strtoull(config["max_flusher_count"].c_str(), NULL, 10);
if (!max_flusher_count)
{
max_flusher_count = strtoull(config["flusher_count"].c_str(), NULL, 10);
}
min_flusher_count = strtoull(config["min_flusher_count"].c_str(), NULL, 10);
max_write_iodepth = strtoull(config["max_write_iodepth"].c_str(), NULL, 10);
throttle_small_writes = config["throttle_small_writes"] == "true" || config["throttle_small_writes"] == "1" || config["throttle_small_writes"] == "yes";
throttle_target_iops = strtoull(config["throttle_target_iops"].c_str(), NULL, 10);
throttle_target_mbs = strtoull(config["throttle_target_mbs"].c_str(), NULL, 10);
throttle_target_parallelism = strtoull(config["throttle_target_parallelism"].c_str(), NULL, 10);
throttle_threshold_us = strtoull(config["throttle_threshold_us"].c_str(), NULL, 10);
if (!max_flusher_count)
{
max_flusher_count = 256;
}
if (!min_flusher_count || journal.flush_journal)
{
min_flusher_count = 1;
}
if (!max_write_iodepth)
{
max_write_iodepth = 128;
}
if (!throttle_target_iops)
{
throttle_target_iops = 100;
}
if (!throttle_target_mbs)
{
throttle_target_mbs = 100;
}
if (!throttle_target_parallelism)
{
throttle_target_parallelism = 1;
}
if (!throttle_threshold_us)
{
throttle_threshold_us = 50;
}
if (!init)
{
return;
}
// Offline-configurable options:
// Common disk options
dsk.parse_config(config);
// Parse
@ -44,29 +90,7 @@ void blockstore_impl_t::parse_config(blockstore_config_t & config)
journal.no_same_sector_overwrites = config["journal_no_same_sector_overwrites"] == "true" ||
config["journal_no_same_sector_overwrites"] == "1" || config["journal_no_same_sector_overwrites"] == "yes";
journal.inmemory = config["inmemory_journal"] != "false";
max_flusher_count = strtoull(config["max_flusher_count"].c_str(), NULL, 10);
if (!max_flusher_count)
max_flusher_count = strtoull(config["flusher_count"].c_str(), NULL, 10);
min_flusher_count = strtoull(config["min_flusher_count"].c_str(), NULL, 10);
max_write_iodepth = strtoull(config["max_write_iodepth"].c_str(), NULL, 10);
throttle_small_writes = config["throttle_small_writes"] == "true" || config["throttle_small_writes"] == "1" || config["throttle_small_writes"] == "yes";
throttle_target_iops = strtoull(config["throttle_target_iops"].c_str(), NULL, 10);
throttle_target_mbs = strtoull(config["throttle_target_mbs"].c_str(), NULL, 10);
throttle_target_parallelism = strtoull(config["throttle_target_parallelism"].c_str(), NULL, 10);
throttle_threshold_us = strtoull(config["throttle_threshold_us"].c_str(), NULL, 10);
// Validate
if (!max_flusher_count)
{
max_flusher_count = 256;
}
if (!min_flusher_count || journal.flush_journal)
{
min_flusher_count = 1;
}
if (!max_write_iodepth)
{
max_write_iodepth = 128;
}
if (journal.sector_count < 2)
{
journal.sector_count = 32;
@ -91,22 +115,6 @@ void blockstore_impl_t::parse_config(blockstore_config_t & config)
{
throw std::runtime_error("immediate_commit=all requires disable_journal_fsync and disable_data_fsync");
}
if (!throttle_target_iops)
{
throttle_target_iops = 100;
}
if (!throttle_target_mbs)
{
throttle_target_mbs = 100;
}
if (!throttle_target_parallelism)
{
throttle_target_parallelism = 1;
}
if (!throttle_threshold_us)
{
throttle_threshold_us = 50;
}
// init some fields
journal.block_size = dsk.journal_block_size;
journal.next_free = dsk.journal_block_size;

View File

@ -278,7 +278,7 @@ struct rm_osd_t
if (rsp["response_delete_range"]["deleted"].uint64_value() > 0)
{
// Wait for mon_change_timeout before updating PG history, or the monitor's change will likely interfere with ours
retry_wait = parent->cli->merged_config["mon_change_timeout"].uint64_value();
retry_wait = parent->cli->config["mon_change_timeout"].uint64_value();
if (!retry_wait)
retry_wait = 1000;
retry_wait += etcd_tx_retry_ms;

View File

@ -198,9 +198,9 @@ resume_2:
}
pgs_by_state_str += std::to_string(kv.second)+" "+kv.first;
}
bool readonly = json_is_true(parent->cli->merged_config["readonly"]);
bool no_recovery = json_is_true(parent->cli->merged_config["no_recovery"]);
bool no_rebalance = json_is_true(parent->cli->merged_config["no_rebalance"]);
bool readonly = json_is_true(parent->cli->config["readonly"]);
bool no_recovery = json_is_true(parent->cli->config["no_recovery"]);
bool no_rebalance = json_is_true(parent->cli->config["no_rebalance"]);
if (parent->json_output)
{
// JSON output

View File

@ -18,11 +18,12 @@
cluster_client_t::cluster_client_t(ring_loop_t *ringloop, timerfd_manager_t *tfd, json11::Json & config)
{
config = osd_messenger_t::read_config(config);
cli_config = config.object_items();
file_config = osd_messenger_t::read_config(config);
config = osd_messenger_t::merge_configs(cli_config, file_config, etcd_global_config, {});
this->ringloop = ringloop;
this->tfd = tfd;
this->config = config;
msgr.osd_num = 0;
msgr.tfd = tfd;
@ -58,7 +59,7 @@ cluster_client_t::cluster_client_t(ring_loop_t *ringloop, timerfd_manager_t *tfd
msgr.stop_client(op->peer_fd);
delete op;
};
msgr.parse_config(this->config);
msgr.parse_config(config);
st_cli.tfd = tfd;
st_cli.on_load_config_hook = [this](json11::Json::object & cfg) { on_load_config_hook(cfg); };
@ -276,13 +277,10 @@ restart:
continuing_ops = 0;
}
void cluster_client_t::on_load_config_hook(json11::Json::object & config)
void cluster_client_t::on_load_config_hook(json11::Json::object & etcd_global_config)
{
this->merged_config = config;
for (auto & kv: this->config.object_items())
{
this->merged_config[kv.first] = kv.second;
}
this->etcd_global_config = etcd_global_config;
config = osd_messenger_t::merge_configs(cli_config, file_config, etcd_global_config, {});
if (config.find("client_max_dirty_bytes") != config.end())
{
client_max_dirty_bytes = config["client_max_dirty_bytes"].uint64_value();
@ -292,14 +290,13 @@ void cluster_client_t::on_load_config_hook(json11::Json::object & config)
// Old name
client_max_dirty_bytes = config["client_dirty_limit"].uint64_value();
}
if (config.find("client_max_dirty_ops") != config.end())
{
client_max_dirty_ops = config["client_max_dirty_ops"].uint64_value();
}
else
client_max_dirty_bytes = 0;
if (!client_max_dirty_bytes)
{
client_max_dirty_bytes = DEFAULT_CLIENT_MAX_DIRTY_BYTES;
}
client_max_dirty_ops = config["client_max_dirty_ops"].uint64_value();
if (!client_max_dirty_ops)
{
client_max_dirty_ops = DEFAULT_CLIENT_MAX_DIRTY_OPS;
@ -314,7 +311,7 @@ void cluster_client_t::on_load_config_hook(json11::Json::object & config)
up_wait_retry_interval = 50;
}
msgr.parse_config(config);
msgr.parse_config(this->config);
st_cli.parse_config(config);
st_cli.load_pgs();
}

View File

@ -112,8 +112,8 @@ public:
osd_messenger_t msgr;
void init_msgr();
json11::Json config;
json11::Json::object merged_config;
json11::Json::object cli_config, file_config, etcd_global_config;
json11::Json::object config;
cluster_client_t(ring_loop_t *ringloop, timerfd_manager_t *tfd, json11::Json & config);
~cluster_client_t();

View File

@ -18,12 +18,8 @@ etcd_state_client_t::~etcd_state_client_t()
}
watches.clear();
etcd_watches_initialised = -1;
if (ws_keepalive_timer >= 0)
{
tfd->clear_timer(ws_keepalive_timer);
ws_keepalive_timer = -1;
}
#ifndef __MOCK__
stop_ws_keepalive();
if (etcd_watch_ws)
{
http_close(etcd_watch_ws);
@ -245,6 +241,7 @@ void etcd_state_client_t::parse_config(const json11::Json & config)
if (this->etcd_keepalive_timeout < 30)
this->etcd_keepalive_timeout = 30;
}
auto old_etcd_ws_keepalive_interval = this->etcd_ws_keepalive_interval;
this->etcd_ws_keepalive_interval = config["etcd_ws_keepalive_interval"].uint64_value();
if (this->etcd_ws_keepalive_interval <= 0)
{
@ -265,6 +262,13 @@ void etcd_state_client_t::parse_config(const json11::Json & config)
{
this->etcd_quick_timeout = 1000;
}
if (this->etcd_ws_keepalive_interval != old_etcd_ws_keepalive_interval && ws_keepalive_timer >= 0)
{
#ifndef __MOCK__
stop_ws_keepalive();
start_ws_keepalive();
#endif
}
}
void etcd_state_client_t::pick_next_etcd()
@ -478,6 +482,20 @@ void etcd_state_client_t::start_etcd_watcher()
{
on_start_watcher_hook(etcd_watch_ws);
}
start_ws_keepalive();
}
void etcd_state_client_t::stop_ws_keepalive()
{
if (ws_keepalive_timer >= 0)
{
tfd->clear_timer(ws_keepalive_timer);
ws_keepalive_timer = -1;
}
}
void etcd_state_client_t::start_ws_keepalive()
{
if (ws_keepalive_timer < 0)
{
ws_keepalive_timer = tfd->set_timer(etcd_ws_keepalive_interval*1000, true, [this](int)

View File

@ -132,6 +132,8 @@ public:
void etcd_txn(json11::Json txn, int timeout, int retries, int interval, std::function<void(std::string, json11::Json)> callback);
void etcd_txn_slow(json11::Json txn, std::function<void(std::string, json11::Json)> callback);
void start_etcd_watcher();
void stop_ws_keepalive();
void start_ws_keepalive();
void load_global_config();
void load_pgs();
void parse_state(const etcd_kv_t & kv);

View File

@ -534,8 +534,9 @@ bool osd_messenger_t::is_rdma_enabled()
}
#endif
json11::Json osd_messenger_t::read_config(const json11::Json & config)
json11::Json::object osd_messenger_t::read_config(const json11::Json & config)
{
json11::Json::object file_config;
const char *config_path = config["config_path"].string_value() != ""
? config["config_path"].string_value().c_str() : VITASTOR_CONFIG_PATH;
int fd = open(config_path, O_RDONLY);
@ -543,14 +544,14 @@ json11::Json osd_messenger_t::read_config(const json11::Json & config)
{
if (errno != ENOENT)
fprintf(stderr, "Error reading %s: %s\n", config_path, strerror(errno));
return config;
return file_config;
}
struct stat st;
if (fstat(fd, &st) != 0)
{
fprintf(stderr, "Error reading %s: %s\n", config_path, strerror(errno));
close(fd);
return config;
return file_config;
}
std::string buf;
buf.resize(st.st_size);
@ -562,23 +563,125 @@ json11::Json osd_messenger_t::read_config(const json11::Json & config)
{
fprintf(stderr, "Error reading %s: %s\n", config_path, strerror(errno));
close(fd);
return config;
return file_config;
}
done += r;
}
close(fd);
std::string json_err;
json11::Json::object file_config = json11::Json::parse(buf, json_err).object_items();
file_config = json11::Json::parse(buf, json_err).object_items();
if (json_err != "")
{
fprintf(stderr, "Invalid JSON in %s: %s\n", config_path, json_err.c_str());
return config;
}
file_config.erase("config_path");
file_config.erase("osd_num");
for (auto kv: config.object_items())
{
file_config[kv.first] = kv.second;
}
return file_config;
}
static const char* cli_only_params[] = {
// The list has to be sorted
"bitmap_granularity",
"block_size",
"data_device",
"data_offset",
"data_size",
"disable_data_fsync",
"disable_device_lock",
"disable_journal_fsync",
"disable_meta_fsync",
"disk_alignment",
"flush_journal",
"immediate_commit",
"inmemory_journal",
"inmemory_metadata",
"journal_block_size",
"journal_device",
"journal_no_same_sector_overwrites",
"journal_offset",
"journal_sector_buffer_count",
"journal_size",
"meta_block_size",
"meta_buf_size",
"meta_device",
"meta_offset",
"osd_num",
"readonly",
};
static const char **cli_only_end = cli_only_params + (sizeof(cli_only_params)/sizeof(cli_only_params[0]));
static const char* local_only_params[] = {
// The list has to be sorted
"config_path",
"rdma_device",
"rdma_gid_index",
"rdma_max_msg",
"rdma_max_recv",
"rdma_max_send",
"rdma_max_sge",
"rdma_mtu",
"rdma_port_num",
"tcp_header_buffer_size",
"use_rdma",
"use_sync_send_recv",
};
static const char **local_only_end = local_only_params + (sizeof(local_only_params)/sizeof(local_only_params[0]));
// Basically could be replaced by std::lower_bound()...
static int find_str_array(const char **start, const char **end, const std::string & s)
{
int min = 0, max = end-start;
while (max-min >= 2)
{
int mid = (min+max)/2;
int r = strcmp(s.c_str(), start[mid]);
if (r < 0)
max = mid;
else if (r > 0)
min = mid;
else
return mid;
}
if (min < end-start && !strcmp(s.c_str(), start[min]))
return min;
return -1;
}
json11::Json::object osd_messenger_t::merge_configs(const json11::Json::object & cli_config,
const json11::Json::object & file_config,
const json11::Json::object & etcd_global_config,
const json11::Json::object & etcd_osd_config)
{
// Priority: most important -> less important:
// etcd_osd_config -> cli_config -> etcd_global_config -> file_config
json11::Json::object res = file_config;
for (auto & kv: file_config)
{
int cli_only = find_str_array(cli_only_params, cli_only_end, kv.first);
if (cli_only < 0)
{
res[kv.first] = kv.second;
}
}
for (auto & kv: etcd_global_config)
{
int local_only = find_str_array(local_only_params, local_only_end, kv.first);
if (local_only < 0)
{
res[kv.first] = kv.second;
}
}
for (auto & kv: cli_config)
{
res[kv.first] = kv.second;
}
for (auto & kv: etcd_osd_config)
{
int local_only = find_str_array(local_only_params, local_only_end, kv.first);
if (local_only < 0)
{
res[kv.first] = kv.second;
}
}
return res;
}

View File

@ -166,7 +166,11 @@ public:
void accept_connections(int listen_fd);
~osd_messenger_t();
static json11::Json read_config(const json11::Json & config);
static json11::Json::object read_config(const json11::Json & config);
static json11::Json::object merge_configs(const json11::Json::object & cli_config,
const json11::Json::object & file_config,
const json11::Json::object & etcd_global_config,
const json11::Json::object & etcd_osd_config);
#ifdef WITH_RDMA
bool is_rdma_enabled();

View File

@ -43,7 +43,15 @@ void osd_messenger_t::send_replies()
{
}
json11::Json osd_messenger_t::read_config(const json11::Json & config)
json11::Json::object osd_messenger_t::read_config(const json11::Json & config)
{
return config;
return json11::Json::object();
}
json11::Json::object osd_messenger_t::merge_configs(const json11::Json::object & cli_config,
const json11::Json::object & file_config,
const json11::Json::object & etcd_global_config,
const json11::Json::object & etcd_osd_config)
{
return cli_config;
}

View File

@ -35,10 +35,9 @@ osd_t::osd_t(const json11::Json & config, ring_loop_t *ringloop)
this->ringloop = ringloop;
this->config = msgr.read_config(config).object_items();
if (this->config.find("log_level") == this->config.end())
this->config["log_level"] = 1;
parse_config(this->config, true);
this->cli_config = config.object_items();
this->file_config = msgr.read_config(this->cli_config);
parse_config(true);
epmgr = new epoll_manager_t(ringloop);
// FIXME: Use timerfd_interval based directly on io_uring
@ -68,11 +67,11 @@ osd_t::osd_t(const json11::Json & config, ring_loop_t *ringloop)
}
}
this->tfd->set_timer(print_stats_interval*1000, true, [this](int timer_id)
print_stats_timer_id = this->tfd->set_timer(print_stats_interval*1000, true, [this](int timer_id)
{
print_stats();
});
this->tfd->set_timer(slow_log_interval*1000, true, [this](int timer_id)
slow_log_timer_id = this->tfd->set_timer(slow_log_interval*1000, true, [this](int timer_id)
{
print_slow();
});
@ -92,6 +91,21 @@ osd_t::osd_t(const json11::Json & config, ring_loop_t *ringloop)
osd_t::~osd_t()
{
if (slow_log_timer_id >= 0)
{
tfd->clear_timer(slow_log_timer_id);
slow_log_timer_id = -1;
}
if (print_stats_timer_id >= 0)
{
tfd->clear_timer(print_stats_timer_id);
print_stats_timer_id = -1;
}
if (autosync_timer_id >= 0)
{
tfd->clear_timer(autosync_timer_id);
autosync_timer_id = -1;
}
ringloop->unregister_consumer(&consumer);
delete epmgr;
if (bs)
@ -100,11 +114,19 @@ osd_t::~osd_t()
free(zero_buffer);
}
void osd_t::parse_config(const json11::Json & config, bool allow_disk_params)
void osd_t::parse_config(bool init)
{
config = msgr.merge_configs(cli_config, file_config, etcd_global_config, etcd_osd_config);
if (config.find("log_level") == this->config.end())
config["log_level"] = 1;
if (bs)
{
auto bs_cfg = json_to_bs(config);
bs->parse_config(bs_cfg);
}
st_cli.parse_config(config);
msgr.parse_config(config);
if (allow_disk_params)
if (init)
{
// OSD number
osd_num = config["osd_num"].uint64_value();
@ -126,24 +148,27 @@ void osd_t::parse_config(const json11::Json & config, bool allow_disk_params)
immediate_commit = IMMEDIATE_SMALL;
else
immediate_commit = IMMEDIATE_NONE;
// Bind address
bind_address = config["bind_address"].string_value();
if (bind_address == "")
bind_address = "0.0.0.0";
bind_port = config["bind_port"].uint64_value();
if (bind_port <= 0 || bind_port > 65535)
bind_port = 0;
// OSD configuration
etcd_report_interval = config["etcd_report_interval"].uint64_value();
if (etcd_report_interval <= 0)
etcd_report_interval = 5;
readonly = json_is_true(config["readonly"]);
run_primary = !json_is_false(config["run_primary"]);
allow_test_ops = json_is_true(config["allow_test_ops"]);
}
// Bind address
bind_address = config["bind_address"].string_value();
if (bind_address == "")
bind_address = "0.0.0.0";
bind_port = config["bind_port"].uint64_value();
if (bind_port <= 0 || bind_port > 65535)
bind_port = 0;
// OSD configuration
log_level = config["log_level"].uint64_value();
etcd_report_interval = config["etcd_report_interval"].uint64_value();
if (etcd_report_interval <= 0)
etcd_report_interval = 5;
readonly = json_is_true(config["readonly"]);
run_primary = !json_is_false(config["run_primary"]);
auto old_no_rebalance = no_rebalance;
no_rebalance = json_is_true(config["no_rebalance"]);
auto old_no_recovery = no_recovery;
no_recovery = json_is_true(config["no_recovery"]);
allow_test_ops = json_is_true(config["allow_test_ops"]);
auto old_autosync_interval = autosync_interval;
if (!config["autosync_interval"].is_null())
{
// Allow to set it to 0
@ -171,15 +196,46 @@ void osd_t::parse_config(const json11::Json & config, bool allow_disk_params)
recovery_sync_batch = config["recovery_sync_batch"].uint64_value();
if (recovery_sync_batch < 1 || recovery_sync_batch > MAX_RECOVERY_QUEUE)
recovery_sync_batch = DEFAULT_RECOVERY_BATCH;
auto old_print_stats_interval = print_stats_interval;
print_stats_interval = config["print_stats_interval"].uint64_value();
if (!print_stats_interval)
print_stats_interval = 3;
auto old_slow_log_interval = slow_log_interval;
slow_log_interval = config["slow_log_interval"].uint64_value();
if (!slow_log_interval)
slow_log_interval = 10;
inode_vanish_time = config["inode_vanish_time"].uint64_value();
if (!inode_vanish_time)
inode_vanish_time = 60;
if ((old_no_rebalance && !no_rebalance || old_no_recovery && !no_recovery) &&
!(peering_state & (OSD_RECOVERING | OSD_FLUSHING_PGS)))
{
peering_state = peering_state | OSD_RECOVERING;
}
if (old_autosync_interval != autosync_interval && autosync_timer_id >= 0)
{
this->tfd->clear_timer(autosync_timer_id);
autosync_timer_id = this->tfd->set_timer(autosync_interval*1000, true, [this](int timer_id)
{
autosync();
});
}
if (old_print_stats_interval != print_stats_interval && print_stats_timer_id >= 0)
{
tfd->clear_timer(print_stats_timer_id);
print_stats_timer_id = this->tfd->set_timer(print_stats_interval*1000, true, [this](int timer_id)
{
print_stats();
});
}
if (old_slow_log_interval != slow_log_interval && slow_log_timer_id >= 0)
{
tfd->clear_timer(slow_log_timer_id);
slow_log_timer_id = this->tfd->set_timer(slow_log_interval*1000, true, [this](int timer_id)
{
print_slow();
});
}
}
void osd_t::bind_socket()

View File

@ -90,7 +90,7 @@ class osd_t
{
// config
json11::Json::object config;
json11::Json::object cli_config, file_config, etcd_global_config, etcd_osd_config, config;
int etcd_report_interval = 5;
bool readonly = false;
@ -126,6 +126,7 @@ class osd_t
bool pg_config_applied = false;
bool etcd_reporting_pg_state = false;
bool etcd_reporting_stats = false;
int autosync_timer_id = -1, print_stats_timer_id = -1, slow_log_timer_id = -1;
// peers and PGs
@ -173,7 +174,7 @@ class osd_t
uint64_t recovery_stat_bytes[2][2] = {};
// cluster connection
void parse_config(const json11::Json & config, bool allow_disk_params);
void parse_config(bool init);
void init_cluster();
void on_change_osd_state_hook(osd_num_t peer_osd);
void on_change_pg_history_hook(pool_id_t pool_id, pg_num_t pg_num);

View File

@ -75,7 +75,7 @@ void osd_t::init_cluster()
}
if (run_primary && autosync_interval > 0)
{
this->tfd->set_timer(autosync_interval*1000, true, [this](int timer_id)
autosync_timer_id = this->tfd->set_timer(autosync_interval*1000, true, [this](int timer_id)
{
autosync();
});
@ -375,7 +375,11 @@ void osd_t::on_change_osd_state_hook(osd_num_t peer_osd)
void osd_t::on_change_etcd_state_hook(std::map<std::string, etcd_kv_t> & changes)
{
// FIXME apply config changes in runtime (maybe, some)
if (changes.find(st_cli.etcd_prefix+"/config/global") != changes.end())
{
etcd_global_config = changes[st_cli.etcd_prefix+"/config/global"].value.object_items();
parse_config(false);
}
if (run_primary)
{
apply_pg_count();
@ -385,11 +389,8 @@ void osd_t::on_change_etcd_state_hook(std::map<std::string, etcd_kv_t> & changes
void osd_t::on_load_config_hook(json11::Json::object & global_config)
{
json11::Json::object osd_config = this->config;
for (auto & kv: global_config)
if (osd_config.find(kv.first) == osd_config.end())
osd_config[kv.first] = kv.second;
parse_config(osd_config, false);
etcd_global_config = global_config;
parse_config(true);
bind_socket();
acquire_lease();
}

View File

@ -76,7 +76,7 @@ void osd_t::handle_peers()
peering_state = peering_state & ~OSD_FLUSHING_PGS | OSD_RECOVERING;
}
}
if ((peering_state & OSD_RECOVERING) && !readonly)
if (!(peering_state & OSD_FLUSHING_PGS) && (peering_state & OSD_RECOVERING) && !readonly)
{
if (!continue_recovery())
{