Compare commits

...

2 Commits

Author SHA1 Message Date
Vitaliy Filippov 7b37ba921d Pause pool rebalance when monitor detects that it can lead to any OSD becoming full
Test / test_root_node (push) Successful in 10s Details
Test / test_rebalance_verify_ec_imm (push) Successful in 1m39s Details
Test / test_write_no_same (push) Successful in 9s Details
Test / test_switch_primary (push) Successful in 32s Details
Test / test_write (push) Successful in 32s Details
Test / test_write_xor (push) Successful in 37s Details
Test / test_heal_pg_size_2 (push) Successful in 2m17s Details
Test / test_heal_ec (push) Successful in 2m18s Details
Test / test_heal_antietcd (push) Successful in 2m17s Details
Test / test_heal_csum_32k_dmj (push) Successful in 2m20s Details
Test / test_heal_csum_32k_dj (push) Successful in 2m27s Details
Test / test_heal_csum_32k (push) Successful in 2m17s Details
Test / test_heal_csum_4k_dmj (push) Successful in 2m19s Details
Test / test_heal_csum_4k_dj (push) Successful in 2m19s Details
Test / test_resize_auto (push) Successful in 9s Details
Test / test_resize (push) Successful in 14s Details
Test / test_osd_tags (push) Successful in 8s Details
Test / test_snapshot_pool2 (push) Failing after 11s Details
Test / test_enospc (push) Successful in 11s Details
Test / test_enospc_imm (push) Successful in 11s Details
Test / test_enospc_xor (push) Successful in 13s Details
Test / test_enospc_imm_xor (push) Successful in 14s Details
Test / test_scrub (push) Successful in 14s Details
Test / test_scrub_zero_osd_2 (push) Successful in 15s Details
Test / test_scrub_xor (push) Successful in 15s Details
Test / test_scrub_pg_size_3 (push) Successful in 16s Details
Test / test_scrub_pg_size_6_pg_minsize_4_osd_count_6_ec (push) Successful in 16s Details
Test / test_scrub_ec (push) Successful in 14s Details
Test / test_nfs (push) Successful in 12s Details
Test / test_heal_csum_4k (push) Successful in 2m19s Details
2024-11-22 01:01:07 +03:00
Vitaliy Filippov 262c581400 Fix create-pool for the case of hosts split into sub-nodes
Test / test_rebalance_verify_ec (push) Successful in 1m38s Details
Test / test_rebalance_verify_ec_imm (push) Successful in 1m40s Details
Test / test_switch_primary (push) Successful in 23s Details
Test / test_write_no_same (push) Successful in 9s Details
Test / test_write (push) Successful in 31s Details
Test / test_write_xor (push) Successful in 35s Details
Test / test_heal_pg_size_2 (push) Successful in 2m18s Details
Test / test_heal_ec (push) Successful in 2m16s Details
Test / test_heal_antietcd (push) Successful in 2m17s Details
Test / test_heal_csum_32k_dmj (push) Successful in 2m19s Details
Test / test_heal_csum_32k_dj (push) Successful in 2m19s Details
Test / test_heal_csum_32k (push) Successful in 2m20s Details
Test / test_heal_csum_4k_dmj (push) Successful in 2m19s Details
Test / test_heal_csum_4k_dj (push) Successful in 2m17s Details
Test / test_resize_auto (push) Successful in 9s Details
Test / test_resize (push) Successful in 16s Details
Test / test_snapshot_pool2 (push) Failing after 11s Details
Test / test_osd_tags (push) Successful in 7s Details
Test / test_enospc (push) Successful in 12s Details
Test / test_enospc_imm (push) Successful in 11s Details
Test / test_enospc_xor (push) Successful in 14s Details
Test / test_enospc_imm_xor (push) Successful in 13s Details
Test / test_scrub (push) Successful in 13s Details
Test / test_scrub_zero_osd_2 (push) Successful in 13s Details
Test / test_scrub_xor (push) Successful in 14s Details
Test / test_scrub_pg_size_3 (push) Successful in 17s Details
Test / test_scrub_pg_size_6_pg_minsize_4_osd_count_6_ec (push) Successful in 17s Details
Test / test_scrub_ec (push) Successful in 14s Details
Test / test_nfs (push) Successful in 11s Details
Test / test_heal_csum_4k (push) Successful in 2m14s Details
2024-11-22 01:01:07 +03:00
12 changed files with 206 additions and 45 deletions

View File

@ -288,6 +288,24 @@ jobs:
echo ""
done
test_create_halfhost:
runs-on: ubuntu-latest
needs: build
container: ${{env.TEST_IMAGE}}:${{github.sha}}
steps:
- name: Run test
id: test
timeout-minutes: 3
run: /root/vitastor/tests/test_create_halfhost.sh
- name: Print logs
if: always() && steps.test.outcome == 'failure'
run: |
for i in /root/vitastor/testdata/*.log /root/vitastor/testdata/*.txt; do
echo "-------- $i --------"
cat $i
echo ""
done
test_failure_domain:
runs-on: ubuntu-latest
needs: build

View File

@ -56,6 +56,7 @@ const etcd_tree = {
osd_out_time: 600, // seconds. min: 0
placement_levels: { datacenter: 1, rack: 2, host: 3, osd: 4, ... },
use_old_pg_combinator: false,
osd_backfillfull_ratio: 0.99,
// client and osd
tcp_header_buffer_size: 65536,
use_sync_send_recv: false,

View File

@ -74,6 +74,7 @@ class Mon
this.state = JSON.parse(JSON.stringify(etcd_tree));
this.prev_stats = { osd_stats: {}, osd_diff: {} };
this.recheck_pgs_active = false;
this.updating_total_stats = false;
this.watcher_active = false;
this.old_pg_config = false;
this.old_pg_stats_seen = false;
@ -658,7 +659,13 @@ class Mon
this.etcd_watch_revision, pool_id, up_osds, osd_tree, real_prev_pgs, pool_res.pgs, pg_history);
}
new_pg_config.hash = tree_hash;
return await this.save_pg_config(new_pg_config, etcd_request);
const { backfillfull_pools } = sum_object_counts({ ...this.state, pg: { ...this.state.pg, config: new_pg_config } }, this.config);
new_pg_config.backfillfull_pools = backfillfull_pools.length ? backfillfull_pools : undefined;
if (!await this.save_pg_config(new_pg_config, etcd_request))
{
return false;
}
return true;
}
async save_pg_config(new_pg_config, etcd_request = { compare: [], success: [] })
@ -730,7 +737,7 @@ class Mon
async update_total_stats()
{
const txn = [];
const { object_counts, object_bytes } = sum_object_counts(this.state, this.config);
const { object_counts, object_bytes, backfillfull_pools } = sum_object_counts(this.state, this.config);
let stats = sum_op_stats(this.state.osd, this.prev_stats);
let { inode_stats, seen_pools } = sum_inode_stats(this.state, this.prev_stats);
stats.object_counts = object_counts;
@ -783,6 +790,16 @@ class Mon
{
await this.etcd.etcd_call('/kv/txn', { success: txn }, this.config.etcd_mon_timeout, 0);
}
if (!this.recheck_pgs_active &&
backfillfull_pools.join(',') != ((this.state.pg.config||{}).no_rebalance_pools||[]).join(','))
{
console.log(
(backfillfull_pools.length ? 'Pool(s) '+backfillfull_pools.join(', ') : 'No pools')+
' are backfillfull, applying rebalance configuration'
);
const new_pg_config = { ...this.state.pg.config, backfillfull_pools: backfillfull_pools.length ? backfillfull_pools : undefined };
await this.save_pg_config(new_pg_config);
}
}
schedule_update_stats()
@ -794,7 +811,21 @@ class Mon
this.stats_timer = setTimeout(() =>
{
this.stats_timer = null;
if (this.updating_total_stats)
{
this.schedule_update_stats();
return;
}
this.updating_total_stats = true;
try
{
this.update_total_stats().catch(console.error);
}
catch (e)
{
console.error(e);
}
this.updating_total_stats = false;
}, this.config.mon_stats_timeout);
}

View File

@ -109,6 +109,8 @@ function sum_object_counts(state, global_config)
pgstats[pool_id] = { ...(state.pg.stats[pool_id] || {}), ...(pgstats[pool_id] || {}) };
}
}
const pool_per_osd = {};
const clean_per_osd = {};
for (const pool_id in pgstats)
{
let object_size = 0;
@ -143,10 +145,38 @@ function sum_object_counts(state, global_config)
object_bytes[k] += BigInt(st[k+'_count']) * object_size;
}
}
if (st.object_count)
{
for (const pg_osd in (((state.pg.config.items||{})[pool_id]||{})[pg_num]||{}).osd_set||[])
{
if (!(pg_osd in clean_per_osd))
{
clean_per_osd[pg_osd] = 0n;
}
clean_per_osd[pg_osd] += BigInt(st.object_count);
pool_per_osd[pg_osd] = pool_per_osd[pg_osd]||{};
pool_per_osd[pg_osd][pool_id] = true;
}
}
}
return { object_counts, object_bytes };
}
}
// If clean_per_osd[osd] is larger than osd capacity then it will fill up during rebalance
let backfillfull_pools = {};
for (const osd in clean_per_osd)
{
const st = state.osd.stats[osd];
if (st && st.size && st.data_block_size && (BigInt(st.size)/BigInt(st.data_block_size)*
BigInt((global_config.osd_backfillfull_ratio||0.99)*1000000)/1000000n) < clean_per_osd[osd])
{
for (const pool_id in pool_per_osd[osd])
{
backfillfull_pools[pool_id] = true;
}
}
}
backfillfull_pools = Object.keys(backfillfull_pools).sort();
return { object_counts, object_bytes, backfillfull_pools };
}
// sum_inode_stats(this.state, this.prev_stats)

View File

@ -785,7 +785,7 @@ void etcd_state_client_t::parse_state(const etcd_kv_t & kv)
}
for (auto & pool_item: value.object_items())
{
pool_config_t pc;
pool_config_t pc = {};
// ID
pool_id_t pool_id;
char null_byte = 0;
@ -931,12 +931,28 @@ void etcd_state_client_t::parse_state(const etcd_kv_t & kv)
// Ignore old key if the new one is present
return;
}
for (auto & pool_id_json: value["backfillfull_pools"].array_items())
{
auto pool_id = pool_id_json.uint64_value();
auto pool_it = this->pool_config.find(pool_id);
if (pool_it != this->pool_config.end())
{
pool_it->second.backfillfull |= 2;
}
}
for (auto & pool_item: this->pool_config)
{
for (auto & pg_item: pool_item.second.pg_config)
{
pg_item.second.config_exists = false;
}
// 3 = was 1 and became 1, 0 = was 0 and became 0
if (pool_item.second.backfillfull == 2 || pool_item.second.backfillfull == 1)
{
if (on_change_backfillfull_hook)
on_change_backfillfull_hook(pool_item.first);
}
pool_item.second.backfillfull = pool_item.second.backfillfull >> 1;
}
for (auto & pool_item: value["items"].object_items())
{

View File

@ -62,6 +62,7 @@ struct pool_config_t
std::map<pg_num_t, pg_config_t> pg_config;
uint64_t scrub_interval;
std::string used_for_fs;
int backfillfull;
};
struct inode_config_t
@ -131,6 +132,7 @@ public:
std::function<json11::Json()> load_pgs_checks_hook;
std::function<void(bool)> on_load_pgs_hook;
std::function<void()> on_change_pool_config_hook;
std::function<void(pool_id_t)> on_change_backfillfull_hook;
std::function<void(pool_id_t, pg_num_t, osd_num_t)> on_change_pg_state_hook;
std::function<void(pool_id_t, pg_num_t)> on_change_pg_history_hook;
std::function<void(osd_num_t)> on_change_osd_state_hook;

View File

@ -90,13 +90,19 @@ resume_1:
// If not forced, check that we have enough osds for pg_size
if (!force)
{
// Get node_placement configuration from etcd
// Get node_placement configuration from etcd and OSD stats
parent->etcd_txn(json11::Json::object {
{ "success", json11::Json::array {
json11::Json::object {
{ "request_range", json11::Json::object {
{ "key", base64_encode(parent->cli->st_cli.etcd_prefix+"/config/node_placement") },
} }
} },
},
json11::Json::object {
{ "request_range", json11::Json::object {
{ "key", base64_encode(parent->cli->st_cli.etcd_prefix+"/osd/stats/") },
{ "range_end", base64_encode(parent->cli->st_cli.etcd_prefix+"/osd/stats0") },
} },
},
} },
});
@ -112,10 +118,21 @@ resume_2:
return;
}
// Get state_node_tree based on node_placement and osd peer states
// Get state_node_tree based on node_placement and osd stats
{
auto kv = parent->cli->st_cli.parse_etcd_kv(parent->etcd_result["responses"][0]["response_range"]["kvs"][0]);
state_node_tree = get_state_node_tree(kv.value.object_items());
auto node_placement_kv = parent->cli->st_cli.parse_etcd_kv(parent->etcd_result["responses"][0]["response_range"]["kvs"][0]);
std::map<osd_num_t, json11::Json> osd_stats;
timespec tv_now;
clock_gettime(CLOCK_REALTIME, &tv_now);
uint64_t osd_out_time = parent->cli->config["osd_out_time"].uint64_value();
if (!osd_out_time)
osd_out_time = 600;
parent->iterate_kvs_1(parent->etcd_result["responses"][1]["response_range"]["kvs"], "/osd/stats/", [&](uint64_t cur_osd, json11::Json value)
{
if (value["time"].uint64_value()+osd_out_time >= tv_now.tv_sec)
osd_stats[cur_osd] = value;
});
state_node_tree = get_state_node_tree(node_placement_kv.value.object_items(), osd_stats);
}
// Skip tag checks, if pool has none
@ -358,42 +375,35 @@ resume_8:
// Returns a JSON object of form {"nodes": {...}, "osds": [...]} that
// contains: all nodes (osds, hosts, ...) based on node_placement config
// and current peer state, and a list of active peer osds.
json11::Json get_state_node_tree(json11::Json::object node_placement)
// and current osd stats.
json11::Json get_state_node_tree(json11::Json::object node_placement, std::map<osd_num_t, json11::Json> & osd_stats)
{
// Erase non-peer osd nodes from node_placement
// Erase non-existing osd nodes from node_placement
for (auto np_it = node_placement.begin(); np_it != node_placement.end();)
{
// Numeric nodes are osds
osd_num_t osd_num = stoull_full(np_it->first);
// If node is osd and it is not in peer states, erase it
if (osd_num > 0 &&
parent->cli->st_cli.peer_states.find(osd_num) == parent->cli->st_cli.peer_states.end())
{
// If node is osd and its stats do not exist, erase it
if (osd_num > 0 && osd_stats.find(osd_num) == osd_stats.end())
node_placement.erase(np_it++);
}
else
np_it++;
}
// List of peer osds
std::vector<std::string> peer_osds;
// List of osds
std::vector<std::string> existing_osds;
// Record peer osds and add missing osds/hosts to np
for (auto & ps: parent->cli->st_cli.peer_states)
// Record osds and add missing osds/hosts to np
for (auto & ps: osd_stats)
{
std::string osd_num = std::to_string(ps.first);
// Record peer osd
peer_osds.push_back(osd_num);
// Record osd
existing_osds.push_back(osd_num);
// Add osd, if necessary
if (node_placement.find(osd_num) == node_placement.end())
{
// Add host if necessary
std::string osd_host = ps.second["host"].as_string();
// Add host, if necessary
if (node_placement.find(osd_host) == node_placement.end())
{
node_placement[osd_host] = json11::Json::object {
@ -401,13 +411,16 @@ resume_8:
};
}
// Add osd if necessary
if (node_placement.find(osd_num) == node_placement.end())
{
node_placement[osd_num] = json11::Json::object {
{ "parent", osd_host }
};
}
}
return json11::Json::object { { "osds", peer_osds }, { "nodes", node_placement } };
return json11::Json::object { { "osds", existing_osds }, { "nodes", node_placement } };
}
// Returns new state_node_tree based on given state_node_tree with osds
@ -592,13 +605,10 @@ resume_8:
// If parent node given, ...
else if (parent_node != "")
{
// ... look for children nodes of this parent
// ... look for child nodes of this parent
for (auto & sn: node_tree)
{
auto & props = sn.second.object_items();
auto parent_prop = props.find("parent");
if (parent_prop != props.end() && (parent_prop->second.as_string() == parent_node))
if (sn.second["parent"] == parent_node)
{
nodes.push_back(sn.first);
@ -615,10 +625,7 @@ resume_8:
// ... look for all level nodes
for (auto & sn: node_tree)
{
auto & props = sn.second.object_items();
auto level_prop = props.find("level");
if (level_prop != props.end() && (level_prop->second.as_string() == level))
if (sn.second["level"] == level)
{
nodes.push_back(sn.first);
}

View File

@ -226,6 +226,7 @@ class osd_t
void parse_config(bool init);
void init_cluster();
void on_change_osd_state_hook(osd_num_t peer_osd);
void on_change_backfillfull_hook(pool_id_t pool_id);
void on_change_pg_history_hook(pool_id_t pool_id, pg_num_t pg_num);
void on_change_etcd_state_hook(std::map<std::string, etcd_kv_t> & changes);
void on_load_config_hook(json11::Json::object & changes);

View File

@ -65,6 +65,7 @@ void osd_t::init_cluster()
st_cli.tfd = tfd;
st_cli.log_level = log_level;
st_cli.on_change_osd_state_hook = [this](osd_num_t peer_osd) { on_change_osd_state_hook(peer_osd); };
st_cli.on_change_backfillfull_hook = [this](pool_id_t pool_id) { on_change_backfillfull_hook(pool_id); };
st_cli.on_change_pg_history_hook = [this](pool_id_t pool_id, pg_num_t pg_num) { on_change_pg_history_hook(pool_id, pg_num); };
st_cli.on_change_hook = [this](std::map<std::string, etcd_kv_t> & changes) { on_change_etcd_state_hook(changes); };
st_cli.on_load_config_hook = [this](json11::Json::object & cfg) { on_load_config_hook(cfg); };
@ -414,6 +415,14 @@ void osd_t::on_change_osd_state_hook(osd_num_t peer_osd)
}
}
void osd_t::on_change_backfillfull_hook(pool_id_t pool_id)
{
if (!(peering_state & (OSD_RECOVERING | OSD_FLUSHING_PGS)))
{
peering_state = peering_state | OSD_RECOVERING;
}
}
void osd_t::on_change_etcd_state_hook(std::map<std::string, etcd_kv_t> & changes)
{
if (changes.find(st_cli.etcd_prefix+"/config/global") != changes.end())

View File

@ -252,10 +252,18 @@ bool osd_t::pick_next_recovery(osd_recovery_op_t &op)
auto mask = recovery_last_degraded ? (PG_ACTIVE | PG_HAS_DEGRADED) : (PG_ACTIVE | PG_DEGRADED | PG_HAS_MISPLACED);
auto check = recovery_last_degraded ? (PG_ACTIVE | PG_HAS_DEGRADED) : (PG_ACTIVE | PG_HAS_MISPLACED);
// Restart scanning from the same PG as the last time
restart:
for (auto pg_it = pgs.lower_bound(recovery_last_pg); pg_it != pgs.end(); pg_it++)
{
if ((pg_it->second.state & mask) == check)
{
auto pool_it = st_cli.pool_config.find(pg_it->first.pool_id);
if (pool_it != st_cli.pool_config.end() && pool_it->second.backfillfull)
{
// Skip the pool
recovery_last_pg.pool_id++;
goto restart;
}
auto & src = recovery_last_degraded ? pg_it->second.degraded_objects : pg_it->second.misplaced_objects;
assert(src.size() > 0);
// Restart scanning from the next object

View File

@ -19,9 +19,12 @@ ANTIETCD=1 ./test_etcd_fail.sh
./test_interrupted_rebalance.sh
IMMEDIATE_COMMIT=1 ./test_interrupted_rebalance.sh
SCHEME=ec ./test_interrupted_rebalance.sh
SCHEME=ec IMMEDIATE_COMMIT=1 ./test_interrupted_rebalance.sh
./test_create_halfhost.sh
./test_failure_domain.sh
./test_snapshot.sh

35
tests/test_create_halfhost.sh Executable file
View File

@ -0,0 +1,35 @@
#!/bin/bash -ex
. `dirname $0`/common.sh
node mon/mon-main.js $MON_PARAMS --etcd_address $ETCD_URL --etcd_prefix "/vitastor" >>./testdata/mon.log 2>&1 &
MON_PID=$!
wait_etcd
TIME=$(date '+%s')
$ETCDCTL put /vitastor/config/global '{"placement_levels":{"dc":10,"host":100,"half":105,"osd":110}}'
$ETCDCTL put /vitastor/config/node_placement '{
"h11":{"level":"half","parent":"host1"},
"h12":{"level":"half","parent":"host1"},
"h21":{"level":"half","parent":"host2"},
"h22":{"level":"half","parent":"host2"},
"h31":{"level":"half","parent":"host3"},
"h32":{"level":"half","parent":"host3"},
"1":{"parent":"h11"},
"2":{"parent":"h12"},
"3":{"parent":"h21"},
"4":{"parent":"h22"},
"5":{"parent":"h31"},
"6":{"parent":"h32"}
}'
$ETCDCTL put /vitastor/osd/stats/1 '{"host":"host1","size":1073741824,"time":"'$TIME'"}'
$ETCDCTL put /vitastor/osd/stats/2 '{"host":"host1","size":1073741824,"time":"'$TIME'"}'
$ETCDCTL put /vitastor/osd/stats/3 '{"host":"host2","size":1073741824,"time":"'$TIME'"}'
$ETCDCTL put /vitastor/osd/stats/4 '{"host":"host2","size":1073741824,"time":"'$TIME'"}'
$ETCDCTL put /vitastor/osd/stats/5 '{"host":"host3","size":1073741824,"time":"'$TIME'"}'
$ETCDCTL put /vitastor/osd/stats/6 '{"host":"host3","size":1073741824,"time":"'$TIME'"}'
build/src/cmd/vitastor-cli --etcd_address $ETCD_URL osd-tree
# check that it doesn't fail
build/src/cmd/vitastor-cli --etcd_address $ETCD_URL create-pool testpool --ec 2+1 -n 32
format_green OK