]> git.proxmox.com Git - ceph.git/blame - ceph/src/mon/OSDMonitor.h
bump version to 18.2.2-pve1
[ceph.git] / ceph / src / mon / OSDMonitor.h
CommitLineData
7c673cae
FG
1// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
2// vim: ts=8 sw=2 smarttab
3/*
4 * Ceph - scalable distributed file system
5 *
6 * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
7 * Copyright (C) 2013,2014 Cloudwatt <libre.licensing@cloudwatt.com>
8 *
9 * Author: Loic Dachary <loic@dachary.org>
10 *
11 * This is free software; you can redistribute it and/or
12 * modify it under the terms of the GNU Lesser General Public
13 * License version 2.1, as published by the Free Software
14 * Foundation. See file COPYING.
15 *
16 */
17
18/* Object Store Device (OSD) Monitor
19 */
20
21#ifndef CEPH_OSDMONITOR_H
22#define CEPH_OSDMONITOR_H
23
24#include <map>
25#include <set>
f67539c2 26#include <utility>
1e59de90 27#include <sstream>
7c673cae
FG
28
29#include "include/types.h"
11fdf7f2 30#include "include/encoding.h"
7c673cae 31#include "common/simple_cache.hpp"
eafe8130 32#include "common/PriorityCache.h"
7c673cae
FG
33#include "msg/Messenger.h"
34
35#include "osd/OSDMap.h"
36#include "osd/OSDMapMapping.h"
37
38#include "CreatingPGs.h"
39#include "PaxosService.h"
40
f67539c2
TL
41#include "erasure-code/ErasureCodeInterface.h"
42#include "mon/MonOpRequest.h"
43#include <boost/functional/hash.hpp>
44
7c673cae
FG
45class Monitor;
46class PGMap;
f67539c2 47struct MonSession;
7c673cae
FG
48class MOSDMap;
49
7c673cae 50
7c673cae
FG
51/// information about a particular peer's failure reports for one osd
52struct failure_reporter_t {
53 utime_t failed_since; ///< when they think it failed
54 MonOpRequestRef op; ///< failure op request
55
56 failure_reporter_t() {}
b3b6e05e
TL
57 failure_reporter_t(utime_t s, MonOpRequestRef op)
58 : failed_since(s), op(op) {}
7c673cae
FG
59 ~failure_reporter_t() { }
60};
61
62/// information about all failure reports for one osd
63struct failure_info_t {
f67539c2 64 std::map<int, failure_reporter_t> reporters; ///< reporter -> failed_since etc
7c673cae
FG
65 utime_t max_failed_since; ///< most recent failed_since
66
67 failure_info_t() {}
68
69 utime_t get_failed_since() {
70 if (max_failed_since == utime_t() && !reporters.empty()) {
71 // the old max must have canceled; recalculate.
f67539c2 72 for (auto p = reporters.begin(); p != reporters.end(); ++p)
7c673cae
FG
73 if (p->second.failed_since > max_failed_since)
74 max_failed_since = p->second.failed_since;
75 }
76 return max_failed_since;
77 }
78
b3b6e05e
TL
79 // set the message for the latest report.
80 void add_report(int who, utime_t failed_since, MonOpRequestRef op) {
81 [[maybe_unused]] auto [it, new_reporter] =
82 reporters.insert_or_assign(who, failure_reporter_t{failed_since, op});
83 if (new_reporter) {
84 if (max_failed_since != utime_t() && max_failed_since < failed_since) {
7c673cae 85 max_failed_since = failed_since;
b3b6e05e 86 }
7c673cae 87 }
7c673cae
FG
88 }
89
f67539c2
TL
90 void take_report_messages(std::list<MonOpRequestRef>& ls) {
91 for (auto p = reporters.begin(); p != reporters.end(); ++p) {
7c673cae
FG
92 if (p->second.op) {
93 ls.push_back(p->second.op);
94 p->second.op.reset();
95 }
96 }
97 }
98
b3b6e05e
TL
99 void cancel_report(int who) {
100 reporters.erase(who);
91327a77 101 max_failed_since = utime_t();
7c673cae
FG
102 }
103};
104
105
106class LastEpochClean {
107 struct Lec {
f67539c2 108 std::vector<epoch_t> epoch_by_pg;
7c673cae
FG
109 ps_t next_missing = 0;
110 epoch_t floor = std::numeric_limits<epoch_t>::max();
522d829b 111 void report(unsigned pg_num, ps_t pg, epoch_t last_epoch_clean);
7c673cae
FG
112 };
113 std::map<uint64_t, Lec> report_by_pool;
114public:
522d829b 115 void report(unsigned pg_num, const pg_t& pg, epoch_t last_epoch_clean);
7c673cae
FG
116 void remove_pool(uint64_t pool);
117 epoch_t get_lower_bound(const OSDMap& latest) const;
1911f103
TL
118
119 void dump(Formatter *f) const;
7c673cae
FG
120};
121
122
11fdf7f2
TL
123struct osdmap_manifest_t {
124 // all the maps we have pinned -- i.e., won't be removed unless
125 // they are inside a trim interval.
f67539c2 126 std::set<version_t> pinned;
11fdf7f2
TL
127
128 osdmap_manifest_t() {}
129
130 version_t get_last_pinned() const
131 {
f67539c2 132 auto it = pinned.crbegin();
11fdf7f2
TL
133 if (it == pinned.crend()) {
134 return 0;
135 }
136 return *it;
137 }
138
139 version_t get_first_pinned() const
140 {
f67539c2 141 auto it = pinned.cbegin();
11fdf7f2
TL
142 if (it == pinned.cend()) {
143 return 0;
144 }
145 return *it;
146 }
147
148 bool is_pinned(version_t v) const
149 {
150 return pinned.find(v) != pinned.end();
151 }
152
153 void pin(version_t v)
154 {
155 pinned.insert(v);
156 }
157
158 version_t get_lower_closest_pinned(version_t v) const {
f67539c2 159 auto p = pinned.lower_bound(v);
11fdf7f2
TL
160 if (p == pinned.cend()) {
161 return 0;
162 } else if (*p > v) {
163 if (p == pinned.cbegin()) {
164 return 0;
165 }
166 --p;
167 }
168 return *p;
169 }
170
f67539c2 171 void encode(ceph::buffer::list& bl) const
11fdf7f2
TL
172 {
173 ENCODE_START(1, 1, bl);
174 encode(pinned, bl);
175 ENCODE_FINISH(bl);
176 }
177
f67539c2 178 void decode(ceph::buffer::list::const_iterator& bl)
11fdf7f2
TL
179 {
180 DECODE_START(1, bl);
181 decode(pinned, bl);
182 DECODE_FINISH(bl);
183 }
184
f67539c2 185 void decode(ceph::buffer::list& bl) {
11fdf7f2
TL
186 auto p = bl.cbegin();
187 decode(p);
188 }
189
f67539c2 190 void dump(ceph::Formatter *f) {
11fdf7f2
TL
191 f->dump_unsigned("first_pinned", get_first_pinned());
192 f->dump_unsigned("last_pinned", get_last_pinned());
193 f->open_array_section("pinned_maps");
194 for (auto& i : pinned) {
195 f->dump_unsigned("epoch", i);
196 }
197 f->close_section();
198 }
199};
200WRITE_CLASS_ENCODER(osdmap_manifest_t);
201
eafe8130
TL
202class OSDMonitor : public PaxosService,
203 public md_config_obs_t {
7c673cae
FG
204 CephContext *cct;
205
206public:
207 OSDMap osdmap;
208
eafe8130
TL
209 // config observer
210 const char** get_tracked_conf_keys() const override;
211 void handle_conf_change(const ConfigProxy& conf,
212 const std::set<std::string> &changed) override;
7c673cae
FG
213 // [leader]
214 OSDMap::Incremental pending_inc;
f67539c2
TL
215 std::map<int, ceph::buffer::list> pending_metadata;
216 std::set<int> pending_metadata_rm;
217 std::map<int, failure_info_t> failure_info;
218 std::map<int,utime_t> down_pending_out; // osd down -> out
81eedcae 219 bool priority_convert = false;
f67539c2 220 std::map<int64_t,std::set<snapid_t>> pending_pseudo_purged_snaps;
eafe8130
TL
221 std::shared_ptr<PriorityCache::PriCache> rocksdb_binned_kv_cache = nullptr;
222 std::shared_ptr<PriorityCache::Manager> pcm = nullptr;
223 ceph::mutex balancer_lock = ceph::make_mutex("OSDMonitor::balancer_lock");
7c673cae 224
f67539c2 225 std::map<int,double> osd_weight;
20effc67 226 std::set<int32_t> filestore_osds;
7c673cae 227
28e407b8
AA
228 using osdmap_key_t = std::pair<version_t, uint64_t>;
229 using osdmap_cache_t = SimpleLRU<osdmap_key_t,
f67539c2 230 ceph::buffer::list,
28e407b8
AA
231 std::less<osdmap_key_t>,
232 boost::hash<osdmap_key_t>>;
233 osdmap_cache_t inc_osd_cache;
234 osdmap_cache_t full_osd_cache;
7c673cae 235
11fdf7f2
TL
236 bool has_osdmap_manifest;
237 osdmap_manifest_t osdmap_manifest;
238
7c673cae
FG
239 bool check_failures(utime_t now);
240 bool check_failure(utime_t now, int target_osd, failure_info_t& fi);
b3b6e05e
TL
241 utime_t get_grace_time(utime_t now, int target_osd, failure_info_t& fi) const;
242 bool is_failure_stale(utime_t now, failure_info_t& fi) const;
224ce89b 243 void force_failure(int target_osd, int by);
7c673cae 244
7c673cae
FG
245 bool _have_pending_crush();
246 CrushWrapper &_get_stable_crush();
20effc67 247 CrushWrapper _get_pending_crush();
7c673cae
FG
248
249 enum FastReadType {
250 FAST_READ_OFF,
251 FAST_READ_ON,
252 FAST_READ_DEFAULT
253 };
254
494da23a
TL
255 struct CleanUpmapJob : public ParallelPGMapper::Job {
256 CephContext *cct;
257 const OSDMap& osdmap;
258 OSDMap::Incremental& pending_inc;
259 // lock to protect pending_inc form changing
260 // when checking is done
9f95a23c
TL
261 ceph::mutex pending_inc_lock =
262 ceph::make_mutex("CleanUpmapJob::pending_inc_lock");
494da23a
TL
263
264 CleanUpmapJob(CephContext *cct, const OSDMap& om, OSDMap::Incremental& pi)
265 : ParallelPGMapper::Job(&om),
266 cct(cct),
267 osdmap(om),
268 pending_inc(pi) {}
269
f67539c2
TL
270 void process(const std::vector<pg_t>& to_check) override {
271 std::vector<pg_t> to_cancel;
272 std::map<pg_t, mempool::osdmap::vector<std::pair<int,int>>> to_remap;
494da23a
TL
273 osdmap.check_pg_upmaps(cct, to_check, &to_cancel, &to_remap);
274 // don't bother taking lock if nothing changes
275 if (!to_cancel.empty() || !to_remap.empty()) {
276 std::lock_guard l(pending_inc_lock);
277 osdmap.clean_pg_upmaps(cct, &pending_inc, to_cancel, to_remap);
278 }
279 }
280
281 void process(int64_t poolid, unsigned ps_begin, unsigned ps_end) override {}
282 void complete() override {}
283 }; // public as this will need to be accessible from TestTestOSDMap.cc
284
7c673cae 285 // svc
11fdf7f2 286public:
7c673cae 287 void create_initial() override;
f67539c2 288 void get_store_prefixes(std::set<std::string>& s) const override;
7c673cae
FG
289
290private:
291 void update_from_paxos(bool *need_bootstrap) override;
292 void create_pending() override; // prepare a new pending
293 void encode_pending(MonitorDBStore::TransactionRef t) override;
294 void on_active() override;
295 void on_restart() override;
296 void on_shutdown() override;
11fdf7f2
TL
297
298 /* osdmap full map prune */
299 void load_osdmap_manifest();
300 bool should_prune() const;
301 void _prune_update_trimmed(
302 MonitorDBStore::TransactionRef tx,
303 version_t first);
304 void prune_init(osdmap_manifest_t& manifest);
305 bool _prune_sanitize_options() const;
306 bool is_prune_enabled() const;
307 bool is_prune_supported() const;
308 bool do_prune(MonitorDBStore::TransactionRef tx);
309
eafe8130
TL
310 // Priority cache control
311 uint32_t mon_osd_cache_size = 0; ///< Number of cached OSDMaps
312 uint64_t rocksdb_cache_size = 0; ///< Cache for kv Db
313 double cache_kv_ratio = 0; ///< Cache ratio dedicated to kv
314 double cache_inc_ratio = 0; ///< Cache ratio dedicated to inc
315 double cache_full_ratio = 0; ///< Cache ratio dedicated to full
316 uint64_t mon_memory_base = 0; ///< Mon base memory for cache autotuning
317 double mon_memory_fragmentation = 0; ///< Expected memory fragmentation
318 uint64_t mon_memory_target = 0; ///< Mon target memory for cache autotuning
319 uint64_t mon_memory_min = 0; ///< Min memory to cache osdmaps
320 bool mon_memory_autotune = false; ///< Cache auto tune setting
321 int register_cache_with_pcm();
322 int _set_cache_sizes();
323 int _set_cache_ratios();
324 void _set_new_cache_sizes();
325 void _set_cache_autotuning();
326 int _update_mon_cache_settings();
327
328 friend struct OSDMemCache;
329 friend struct IncCache;
330 friend struct FullCache;
331
7c673cae
FG
332 /**
333 * we haven't delegated full version stashing to paxosservice for some time
334 * now, making this function useless in current context.
335 */
336 void encode_full(MonitorDBStore::TransactionRef t) override { }
337 /**
338 * do not let paxosservice periodically stash full osdmaps, or we will break our
339 * locally-managed full maps. (update_from_paxos loads the latest and writes them
340 * out going forward from there, but if we just synced that may mean we skip some.)
341 */
342 bool should_stash_full() override {
343 return false;
344 }
345
346 /**
347 * hook into trim to include the oldest full map in the trim transaction
348 *
349 * This ensures that anyone post-sync will have enough to rebuild their
350 * full osdmaps.
351 */
352 void encode_trim_extra(MonitorDBStore::TransactionRef tx, version_t first) override;
353
354 void update_msgr_features();
7c673cae
FG
355 /**
356 * check if the cluster supports the features required by the
357 * given crush map. Outputs the daemons which don't support it
358 * to the stringstream.
359 *
360 * @returns true if the map is passable, false otherwise
361 */
362 bool validate_crush_against_features(const CrushWrapper *newcrush,
f67539c2 363 std::stringstream &ss);
7c673cae
FG
364 void check_osdmap_subs();
365 void share_map_with_random_osd();
366
9f95a23c
TL
367 ceph::mutex prime_pg_temp_lock =
368 ceph::make_mutex("OSDMonitor::prime_pg_temp_lock");
7c673cae
FG
369 struct PrimeTempJob : public ParallelPGMapper::Job {
370 OSDMonitor *osdmon;
371 PrimeTempJob(const OSDMap& om, OSDMonitor *m)
372 : ParallelPGMapper::Job(&om), osdmon(m) {}
373 void process(int64_t pool, unsigned ps_begin, unsigned ps_end) override {
374 for (unsigned ps = ps_begin; ps < ps_end; ++ps) {
375 pg_t pgid(ps, pool);
376 osdmon->prime_pg_temp(*osdmap, pgid);
377 }
378 }
f67539c2 379 void process(const std::vector<pg_t>& pgs) override {}
7c673cae
FG
380 void complete() override {}
381 };
382 void maybe_prime_pg_temp();
383 void prime_pg_temp(const OSDMap& next, pg_t pgid);
384
385 ParallelPGMapper mapper; ///< for background pg work
386 OSDMapMapping mapping; ///< pg <-> osd mappings
f67539c2 387 std::unique_ptr<ParallelPGMapper::Job> mapping_job; ///< background mapping job
7c673cae
FG
388 void start_mapping();
389
390 void update_logger();
391
392 void handle_query(PaxosServiceMessage *m);
393 bool preprocess_query(MonOpRequestRef op) override; // true if processed.
394 bool prepare_update(MonOpRequestRef op) override;
395 bool should_propose(double &delay) override;
396
11fdf7f2 397 version_t get_trim_to() const override;
7c673cae
FG
398
399 bool can_mark_down(int o);
400 bool can_mark_up(int o);
401 bool can_mark_out(int o);
402 bool can_mark_in(int o);
403
404 // ...
28e407b8
AA
405 MOSDMap *build_latest_full(uint64_t features);
406 MOSDMap *build_incremental(epoch_t first, epoch_t last, uint64_t features);
7c673cae
FG
407 void send_full(MonOpRequestRef op);
408 void send_incremental(MonOpRequestRef op, epoch_t first);
409public:
f67539c2
TL
410 /**
411 * Make sure the existing (up) OSDs support the given features
412 * @return 0 on success, or an error code if any OSDs re missing features.
413 * @param ss Filled in with ane explanation of failure, if any
414 */
415 int check_cluster_features(uint64_t features, std::stringstream &ss);
7c673cae
FG
416 // @param req an optional op request, if the osdmaps are replies to it. so
417 // @c Monitor::send_reply() can mark_event with it.
418 void send_incremental(epoch_t first, MonSession *session, bool onetime,
419 MonOpRequestRef req = MonOpRequestRef());
420
421private:
f67539c2 422 void print_utilization(std::ostream &out, ceph::Formatter *f, bool tree) const;
7c673cae 423
11fdf7f2 424 bool check_source(MonOpRequestRef op, uuid_d fsid);
7c673cae
FG
425
426 bool preprocess_get_osdmap(MonOpRequestRef op);
427
428 bool preprocess_mark_me_down(MonOpRequestRef op);
429
430 friend class C_AckMarkedDown;
431 bool preprocess_failure(MonOpRequestRef op);
432 bool prepare_failure(MonOpRequestRef op);
433 bool prepare_mark_me_down(MonOpRequestRef op);
434 void process_failures();
f67539c2 435 void take_all_failures(std::list<MonOpRequestRef>& ls);
7c673cae 436
9f95a23c
TL
437 bool preprocess_mark_me_dead(MonOpRequestRef op);
438 bool prepare_mark_me_dead(MonOpRequestRef op);
439
7c673cae
FG
440 bool preprocess_full(MonOpRequestRef op);
441 bool prepare_full(MonOpRequestRef op);
442
443 bool preprocess_boot(MonOpRequestRef op);
444 bool prepare_boot(MonOpRequestRef op);
445 void _booted(MonOpRequestRef op, bool logit);
446
447 void update_up_thru(int from, epoch_t up_thru);
448 bool preprocess_alive(MonOpRequestRef op);
449 bool prepare_alive(MonOpRequestRef op);
450 void _reply_map(MonOpRequestRef op, epoch_t e);
451
452 bool preprocess_pgtemp(MonOpRequestRef op);
453 bool prepare_pgtemp(MonOpRequestRef op);
454
455 bool preprocess_pg_created(MonOpRequestRef op);
456 bool prepare_pg_created(MonOpRequestRef op);
457
11fdf7f2
TL
458 bool preprocess_pg_ready_to_merge(MonOpRequestRef op);
459 bool prepare_pg_ready_to_merge(MonOpRequestRef op);
460
f67539c2 461 int _check_remove_pool(int64_t pool_id, const pg_pool_t &pool, std::ostream *ss);
7c673cae
FG
462 bool _check_become_tier(
463 int64_t tier_pool_id, const pg_pool_t *tier_pool,
464 int64_t base_pool_id, const pg_pool_t *base_pool,
f67539c2 465 int *err, std::ostream *ss) const;
7c673cae
FG
466 bool _check_remove_tier(
467 int64_t base_pool_id, const pg_pool_t *base_pool, const pg_pool_t *tier_pool,
f67539c2 468 int *err, std::ostream *ss) const;
7c673cae 469
f67539c2
TL
470 int _prepare_remove_pool(int64_t pool, std::ostream *ss, bool no_fake);
471 int _prepare_rename_pool(int64_t pool, std::string newname);
7c673cae 472
28e407b8 473 bool enforce_pool_op_caps(MonOpRequestRef op);
7c673cae
FG
474 bool preprocess_pool_op (MonOpRequestRef op);
475 bool preprocess_pool_op_create (MonOpRequestRef op);
476 bool prepare_pool_op (MonOpRequestRef op);
477 bool prepare_pool_op_create (MonOpRequestRef op);
478 bool prepare_pool_op_delete(MonOpRequestRef op);
f67539c2
TL
479 int crush_rename_bucket(const std::string& srcname,
480 const std::string& dstname,
481 std::ostream *ss);
482 void check_legacy_ec_plugin(const std::string& plugin,
483 const std::string& profile) const;
484 int normalize_profile(const std::string& profilename,
485 ceph::ErasureCodeProfile &profile,
7c673cae 486 bool force,
f67539c2
TL
487 std::ostream *ss);
488 int crush_rule_create_erasure(const std::string &name,
489 const std::string &profile,
31f18b77 490 int *rule,
f67539c2
TL
491 std::ostream *ss);
492 int get_crush_rule(const std::string &rule_name,
493 int *crush_rule,
494 std::ostream *ss);
495 int get_erasure_code(const std::string &erasure_code_profile,
496 ceph::ErasureCodeInterfaceRef *erasure_code,
497 std::ostream *ss) const;
31f18b77 498 int prepare_pool_crush_rule(const unsigned pool_type,
f67539c2
TL
499 const std::string &erasure_code_profile,
500 const std::string &rule_name,
501 int *crush_rule,
502 std::ostream *ss);
7c673cae
FG
503 bool erasure_code_profile_in_use(
504 const mempool::osdmap::map<int64_t, pg_pool_t> &pools,
f67539c2
TL
505 const std::string &profile,
506 std::ostream *ss);
507 int parse_erasure_code_profile(const std::vector<std::string> &erasure_code_profile,
508 std::map<std::string,std::string> *erasure_code_profile_map,
509 std::ostream *ss);
7c673cae 510 int prepare_pool_size(const unsigned pool_type,
f67539c2 511 const std::string &erasure_code_profile,
11fdf7f2 512 uint8_t repl_size,
7c673cae 513 unsigned *size, unsigned *min_size,
f67539c2 514 std::ostream *ss);
7c673cae 515 int prepare_pool_stripe_width(const unsigned pool_type,
f67539c2 516 const std::string &erasure_code_profile,
7c673cae 517 unsigned *stripe_width,
f67539c2 518 std::ostream *ss);
39ae355f 519 uint32_t get_osd_num_by_crush(int crush_rule);
20effc67 520 int check_pg_num(int64_t pool, int pg_num, int size, int crush_rule, std::ostream* ss);
f67539c2 521 int prepare_new_pool(std::string& name,
31f18b77 522 int crush_rule,
f67539c2 523 const std::string &crush_rule_name,
7c673cae 524 unsigned pg_num, unsigned pgp_num,
11fdf7f2 525 unsigned pg_num_min,
20effc67 526 unsigned pg_num_max,
11fdf7f2
TL
527 uint64_t repl_size,
528 const uint64_t target_size_bytes,
529 const float target_size_ratio,
f67539c2 530 const std::string &erasure_code_profile,
7c673cae
FG
531 const unsigned pool_type,
532 const uint64_t expected_num_objects,
533 FastReadType fast_read,
1e59de90 534 std::string pg_autoscale_mode,
20effc67 535 bool bulk,
1e59de90 536 bool crimson,
f67539c2 537 std::ostream *ss);
7c673cae
FG
538 int prepare_new_pool(MonOpRequestRef op);
539
3efd9988
FG
540 void set_pool_flags(int64_t pool_id, uint64_t flags);
541 void clear_pool_flags(int64_t pool_id, uint64_t flags);
7c673cae 542 bool update_pools_status();
7c673cae 543
9f95a23c
TL
544 bool _is_removed_snap(int64_t pool_id, snapid_t snapid);
545 bool _is_pending_removed_snap(int64_t pool_id, snapid_t snapid);
546
f67539c2
TL
547 std::string make_purged_snap_epoch_key(epoch_t epoch);
548 std::string make_purged_snap_key(int64_t pool, snapid_t snap);
549 std::string make_purged_snap_key_value(int64_t pool, snapid_t snap, snapid_t num,
550 epoch_t epoch, ceph::buffer::list *v);
9f95a23c 551
11fdf7f2 552 bool try_prune_purged_snaps();
9f95a23c 553 int lookup_purged_snap(int64_t pool, snapid_t snap,
11fdf7f2
TL
554 snapid_t *begin, snapid_t *end);
555
9f95a23c
TL
556 void insert_purged_snap_update(
557 int64_t pool,
558 snapid_t start, snapid_t end,
559 epoch_t epoch,
560 MonitorDBStore::TransactionRef t);
561
7c673cae
FG
562 bool prepare_set_flag(MonOpRequestRef op, int flag);
563 bool prepare_unset_flag(MonOpRequestRef op, int flag);
564
565 void _pool_op_reply(MonOpRequestRef op,
f67539c2 566 int ret, epoch_t epoch, ceph::buffer::list *blp=NULL);
7c673cae
FG
567
568 struct C_Booted : public C_MonOp {
569 OSDMonitor *cmon;
570 bool logit;
571 C_Booted(OSDMonitor *cm, MonOpRequestRef op_, bool l=true) :
572 C_MonOp(op_), cmon(cm), logit(l) {}
573 void _finish(int r) override {
574 if (r >= 0)
575 cmon->_booted(op, logit);
576 else if (r == -ECANCELED)
577 return;
578 else if (r == -EAGAIN)
579 cmon->dispatch(op);
580 else
11fdf7f2 581 ceph_abort_msg("bad C_Booted return value");
7c673cae
FG
582 }
583 };
584
585 struct C_ReplyMap : public C_MonOp {
586 OSDMonitor *osdmon;
587 epoch_t e;
588 C_ReplyMap(OSDMonitor *o, MonOpRequestRef op_, epoch_t ee)
589 : C_MonOp(op_), osdmon(o), e(ee) {}
590 void _finish(int r) override {
591 if (r >= 0)
592 osdmon->_reply_map(op, e);
593 else if (r == -ECANCELED)
594 return;
595 else if (r == -EAGAIN)
596 osdmon->dispatch(op);
597 else
11fdf7f2 598 ceph_abort_msg("bad C_ReplyMap return value");
7c673cae
FG
599 }
600 };
601 struct C_PoolOp : public C_MonOp {
602 OSDMonitor *osdmon;
603 int replyCode;
604 int epoch;
f67539c2
TL
605 ceph::buffer::list reply_data;
606 C_PoolOp(OSDMonitor * osd, MonOpRequestRef op_, int rc, int e, ceph::buffer::list *rd=NULL) :
7c673cae
FG
607 C_MonOp(op_), osdmon(osd), replyCode(rc), epoch(e) {
608 if (rd)
609 reply_data = *rd;
610 }
611 void _finish(int r) override {
612 if (r >= 0)
613 osdmon->_pool_op_reply(op, replyCode, epoch, &reply_data);
614 else if (r == -ECANCELED)
615 return;
616 else if (r == -EAGAIN)
617 osdmon->dispatch(op);
618 else
11fdf7f2 619 ceph_abort_msg("bad C_PoolOp return value");
7c673cae
FG
620 }
621 };
622
623 bool preprocess_remove_snaps(MonOpRequestRef op);
624 bool prepare_remove_snaps(MonOpRequestRef op);
625
9f95a23c
TL
626 bool preprocess_get_purged_snaps(MonOpRequestRef op);
627
f67539c2
TL
628 int load_metadata(int osd, std::map<std::string, std::string>& m,
629 std::ostream *err);
630 void count_metadata(const std::string& field, ceph::Formatter *f);
28e407b8 631
f67539c2
TL
632 void reencode_incremental_map(ceph::buffer::list& bl, uint64_t features);
633 void reencode_full_map(ceph::buffer::list& bl, uint64_t features);
c07f9fc5 634public:
f67539c2
TL
635 void count_metadata(const std::string& field, std::map<std::string,int> *out);
636 void get_versions(std::map<std::string, std::list<std::string>> &versions);
20effc67
TL
637 void get_filestore_osd_list();
638 void check_for_filestore_osds(health_check_map_t *checks);
c07f9fc5 639protected:
7c673cae
FG
640 int get_osd_objectstore_type(int osd, std::string *type);
641 bool is_pool_currently_all_bluestore(int64_t pool_id, const pg_pool_t &pool,
f67539c2 642 std::ostream *err);
7c673cae 643
f67539c2
TL
644 // when we last received PG stats from each osd and the osd's osd_beacon_report_interval
645 std::map<int, std::pair<utime_t, int>> last_osd_report;
7c673cae
FG
646 // TODO: use last_osd_report to store the osd report epochs, once we don't
647 // need to upgrade from pre-luminous releases.
f67539c2 648 std::map<int,epoch_t> osd_epochs;
7c673cae
FG
649 LastEpochClean last_epoch_clean;
650 bool preprocess_beacon(MonOpRequestRef op);
651 bool prepare_beacon(MonOpRequestRef op);
652 epoch_t get_min_last_epoch_clean() const;
653
654 friend class C_UpdateCreatingPGs;
11fdf7f2 655 std::map<int, std::map<epoch_t, std::set<spg_t>>> creating_pgs_by_osd_epoch;
7c673cae
FG
656 std::vector<pg_t> pending_created_pgs;
657 // the epoch when the pg mapping was calculated
658 epoch_t creating_pgs_epoch = 0;
659 creating_pgs_t creating_pgs;
c07f9fc5 660 mutable std::mutex creating_pgs_lock;
7c673cae 661
94b18763
FG
662 creating_pgs_t update_pending_pgs(const OSDMap::Incremental& inc,
663 const OSDMap& nextmap);
31f18b77 664 unsigned scan_for_creating_pgs(
7c673cae
FG
665 const mempool::osdmap::map<int64_t,pg_pool_t>& pools,
666 const mempool::osdmap::set<int64_t>& removed_pools,
667 utime_t modified,
668 creating_pgs_t* creating_pgs) const;
f67539c2 669 std::pair<int32_t, pg_t> get_parent_pg(pg_t pgid) const;
7c673cae
FG
670 void update_creating_pgs();
671 void check_pg_creates_subs();
c07f9fc5 672 epoch_t send_pg_creates(int osd, Connection *con, epoch_t next) const;
7c673cae 673
31f18b77
FG
674 int32_t _allocate_osd_id(int32_t* existing_id);
675
f6b5b4d7
TL
676 int get_grace_interval_threshold();
677 bool grace_interval_threshold_exceeded(int last_failed);
678 void set_default_laggy_params(int target_osd);
679
1e59de90
TL
680 int parse_pgid(const cmdmap_t& cmdmap, std::stringstream &ss,
681 pg_t &pgid, std::optional<std::string> pgidstr = std::nullopt);
7c673cae 682public:
f67539c2 683 OSDMonitor(CephContext *cct, Monitor &mn, Paxos &p, const std::string& service_name);
7c673cae
FG
684
685 void tick() override; // check state, take actions
686
7c673cae
FG
687 bool preprocess_command(MonOpRequestRef op);
688 bool prepare_command(MonOpRequestRef op);
11fdf7f2 689 bool prepare_command_impl(MonOpRequestRef op, const cmdmap_t& cmdmap);
7c673cae 690
31f18b77
FG
691 int validate_osd_create(
692 const int32_t id,
693 const uuid_d& uuid,
694 const bool check_osd_exists,
695 int32_t* existing_id,
f67539c2 696 std::stringstream& ss);
31f18b77
FG
697 int prepare_command_osd_create(
698 const int32_t id,
699 const uuid_d& uuid,
700 int32_t* existing_id,
f67539c2 701 std::stringstream& ss);
3a9019d9 702 void do_osd_create(const int32_t id, const uuid_d& uuid,
f67539c2 703 const std::string& device_class,
3a9019d9 704 int32_t* new_id);
f67539c2
TL
705 int prepare_command_osd_purge(int32_t id, std::stringstream& ss);
706 int prepare_command_osd_destroy(int32_t id, std::stringstream& ss);
31f18b77
FG
707 int _prepare_command_osd_crush_remove(
708 CrushWrapper &newcrush,
709 int32_t id,
710 int32_t ancestor,
711 bool has_ancestor,
712 bool unlink_only);
713 void do_osd_crush_remove(CrushWrapper& newcrush);
714 int prepare_command_osd_crush_remove(
715 CrushWrapper &newcrush,
716 int32_t id,
717 int32_t ancestor,
718 bool has_ancestor,
719 bool unlink_only);
720 int prepare_command_osd_remove(int32_t id);
721 int prepare_command_osd_new(
722 MonOpRequestRef op,
11fdf7f2 723 const cmdmap_t& cmdmap,
f67539c2
TL
724 const std::map<std::string,std::string>& secrets,
725 std::stringstream &ss,
726 ceph::Formatter *f);
31f18b77 727
11fdf7f2 728 int prepare_command_pool_set(const cmdmap_t& cmdmap,
f67539c2 729 std::stringstream& ss);
11fdf7f2 730
f67539c2 731 int prepare_command_pool_application(const std::string &prefix,
11fdf7f2 732 const cmdmap_t& cmdmap,
f67539c2
TL
733 std::stringstream& ss);
734 int preprocess_command_pool_application(const std::string &prefix,
11fdf7f2 735 const cmdmap_t& cmdmap,
f67539c2 736 std::stringstream& ss,
11fdf7f2 737 bool *modified);
f67539c2
TL
738 int _command_pool_application(const std::string &prefix,
739 const cmdmap_t& cmdmap,
740 std::stringstream& ss,
741 bool *modified,
742 bool preparing);
7c673cae
FG
743
744 bool handle_osd_timeouts(const utime_t &now,
f67539c2 745 std::map<int, std::pair<utime_t, int>> &last_osd_report);
7c673cae
FG
746
747 void send_latest(MonOpRequestRef op, epoch_t start=0);
748 void send_latest_now_nodelete(MonOpRequestRef op, epoch_t start=0) {
749 op->mark_osdmon_event(__func__);
750 send_incremental(op, start);
751 }
752
f67539c2
TL
753 int get_version(version_t ver, ceph::buffer::list& bl) override;
754 int get_version(version_t ver, uint64_t feature, ceph::buffer::list& bl);
28e407b8 755
f67539c2
TL
756 int get_version_full(version_t ver, uint64_t feature, ceph::buffer::list& bl);
757 int get_version_full(version_t ver, ceph::buffer::list& bl) override;
11fdf7f2 758 int get_inc(version_t ver, OSDMap::Incremental& inc);
f67539c2 759 int get_full_from_pinned_map(version_t ver, ceph::buffer::list& bl);
7c673cae 760
f67539c2
TL
761 epoch_t blocklist(const entity_addrvec_t& av, utime_t until);
762 epoch_t blocklist(entity_addr_t a, utime_t until);
7c673cae 763
f67539c2
TL
764 void dump_info(ceph::Formatter *f);
765 int dump_osd_metadata(int osd, ceph::Formatter *f, std::ostream *err);
766 void print_nodes(ceph::Formatter *f);
7c673cae
FG
767
768 void check_osdmap_sub(Subscription *sub);
769 void check_pg_creates_sub(Subscription *sub);
770
11fdf7f2
TL
771 void do_application_enable(int64_t pool_id, const std::string &app_name,
772 const std::string &app_key="",
1911f103
TL
773 const std::string &app_value="",
774 bool force=false);
494da23a
TL
775 void do_set_pool_opt(int64_t pool_id, pool_opts_t::key_t opt,
776 pool_opts_t::value_t);
c07f9fc5 777
7c673cae
FG
778 void add_flag(int flag) {
779 if (!(osdmap.flags & flag)) {
780 if (pending_inc.new_flags < 0)
781 pending_inc.new_flags = osdmap.flags;
782 pending_inc.new_flags |= flag;
783 }
784 }
785
786 void remove_flag(int flag) {
787 if(osdmap.flags & flag) {
788 if (pending_inc.new_flags < 0)
789 pending_inc.new_flags = osdmap.flags;
790 pending_inc.new_flags &= ~flag;
791 }
792 }
81eedcae 793 void convert_pool_priorities(void);
f67539c2
TL
794 /**
795 * Find the pools which are requested to be put into stretch mode,
796 * validate that they are allowed to be in stretch mode (eg, are replicated)
797 * and place copies of them in the pools set.
798 * This does not make any changes to the pools or state; it's just
799 * a safety-check-and-collect function.
800 */
20effc67 801 void try_enable_stretch_mode_pools(std::stringstream& ss, bool *okay,
f67539c2 802 int *errcode,
20effc67
TL
803 std::set<pg_pool_t*>* pools,
804 const std::string& new_crush_rule);
f67539c2
TL
805 /**
806 * Check validity of inputs and OSD/CRUSH state to
807 * engage stretch mode. Designed to be used with
808 * MonmapMonitor::try_enable_stretch_mode() where we call both twice,
809 * first with commit=false to validate.
810 * @param ss: a stringstream to write errors into
811 * @param okay: Filled to true if okay, false if validation fails
812 * @param errcode: filled with -errno if there's a problem
813 * @param commit: true if we should commit the change, false if just testing
814 * @param dividing_bucket: the bucket type (eg 'dc') that divides the cluster
815 * @param bucket_count: The number of buckets required in peering.
816 * Currently must be 2.
817 * @param pools: The pg_pool_ts which are being set to stretch mode (obtained
818 * from try_enable_stretch_mode_pools()).
819 * @param new_crush_rule: The crush rule to set the pools to.
820 */
20effc67 821 void try_enable_stretch_mode(std::stringstream& ss, bool *okay,
f67539c2 822 int *errcode, bool commit,
20effc67 823 const std::string& dividing_bucket,
f67539c2 824 uint32_t bucket_count,
20effc67
TL
825 const std::set<pg_pool_t*>& pools,
826 const std::string& new_crush_rule);
f67539c2
TL
827 /**
828 * Check the input dead_buckets mapping (buckets->dead monitors) to see
829 * if the OSDs are also down. If so, fill in really_down_buckets and
830 * really_down_mons and return true; else return false.
831 */
20effc67
TL
832 bool check_for_dead_crush_zones(const std::map<std::string,std::set<std::string>>& dead_buckets,
833 std::set<int> *really_down_buckets,
834 std::set<std::string> *really_down_mons);
f67539c2
TL
835 /**
836 * Set degraded mode in the OSDMap, adding the given dead buckets to the dead set
837 * and using the live_zones (should presently be size 1)
838 */
20effc67
TL
839 void trigger_degraded_stretch_mode(const std::set<int>& dead_buckets,
840 const std::set<std::string>& live_zones);
b3b6e05e
TL
841 /**
842 * This is just to maintain stretch_recovery_triggered; below
843 */
844 void set_degraded_stretch_mode();
f67539c2
TL
845 /**
846 * Set recovery stretch mode in the OSDMap, resetting pool size back to normal
847 */
848 void trigger_recovery_stretch_mode();
b3b6e05e
TL
849 /**
850 * This is just to maintain stretch_recovery_triggered; below
851 */
852 void set_recovery_stretch_mode();
853 /**
854 * This is just to maintain stretch_recovery_triggered; below
855 */
856 void set_healthy_stretch_mode();
f67539c2
TL
857 /**
858 * Tells the OSD there's a new pg digest, in case it's interested.
859 * (It's interested when in recovering stretch mode.)
860 */
861 void notify_new_pg_digest();
862 /**
863 * Check if we can exit recovery stretch mode and go back to normal.
864 * @param force If true, we will force the exit through once it is legal,
865 * without regard to the reported PG status.
866 */
867 void try_end_recovery_stretch_mode(bool force);
868 /**
869 * Sets the osdmap and pg_pool_t values back to healthy stretch mode status.
870 */
871 void trigger_healthy_stretch_mode();
522d829b
TL
872 /**
873 * Obtain the crush rule being used for stretch pools.
874 * Note that right now this is heuristic and simply selects the
875 * most-used rule on replicated stretch pools.
876 * @return the crush rule ID, or a negative errno
877 */
878 int get_replicated_stretch_crush_rule();
f67539c2
TL
879private:
880 utime_t stretch_recovery_triggered; // what time we committed a switch to recovery mode
7c673cae
FG
881};
882
883#endif