2006-10-05 05:39:29 +00:00
|
|
|
// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
|
2007-06-01 19:51:31 +00:00
|
|
|
// vim: ts=8 sw=2 smarttab
|
2006-10-05 05:39:29 +00:00
|
|
|
/*
|
|
|
|
* Ceph - scalable distributed file system
|
|
|
|
*
|
|
|
|
* Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
|
2014-03-16 11:03:10 +00:00
|
|
|
* Copyright (C) 2013,2014 Cloudwatt <libre.licensing@cloudwatt.com>
|
|
|
|
*
|
|
|
|
* Author: Loic Dachary <loic@dachary.org>
|
2006-10-05 05:39:29 +00:00
|
|
|
*
|
|
|
|
* This is free software; you can redistribute it and/or
|
|
|
|
* modify it under the terms of the GNU Lesser General Public
|
|
|
|
* License version 2.1, as published by the Free Software
|
|
|
|
* Foundation. See file COPYING.
|
|
|
|
*
|
|
|
|
*/
|
|
|
|
|
2008-01-16 14:06:02 +00:00
|
|
|
/* Object Store Device (OSD) Monitor
|
|
|
|
*/
|
2006-10-05 05:39:29 +00:00
|
|
|
|
2010-06-12 13:04:11 +00:00
|
|
|
#ifndef CEPH_OSDMONITOR_H
|
|
|
|
#define CEPH_OSDMONITOR_H
|
2006-10-05 05:39:29 +00:00
|
|
|
|
|
|
|
#include <map>
|
|
|
|
#include <set>
|
|
|
|
|
|
|
|
#include "include/types.h"
|
2015-08-10 11:25:03 +00:00
|
|
|
#include "common/simple_cache.hpp"
|
2006-10-05 05:39:29 +00:00
|
|
|
#include "msg/Messenger.h"
|
|
|
|
|
|
|
|
#include "osd/OSDMap.h"
|
2017-01-30 19:57:46 +00:00
|
|
|
#include "osd/OSDMapMapping.h"
|
2006-10-05 05:39:29 +00:00
|
|
|
|
2017-03-09 04:08:29 +00:00
|
|
|
#include "CreatingPGs.h"
|
2007-07-01 14:43:27 +00:00
|
|
|
#include "PaxosService.h"
|
2006-10-05 05:39:29 +00:00
|
|
|
|
2007-07-01 14:43:27 +00:00
|
|
|
class Monitor;
|
2015-01-21 02:35:13 +00:00
|
|
|
class PGMap;
|
2016-05-19 05:03:29 +00:00
|
|
|
class MonSession;
|
|
|
|
class MOSDMap;
|
2006-10-05 05:39:29 +00:00
|
|
|
|
2014-02-06 10:00:33 +00:00
|
|
|
#include "erasure-code/ErasureCodeInterface.h"
|
2015-02-12 19:41:33 +00:00
|
|
|
#include "mon/MonOpRequest.h"
|
|
|
|
|
2012-09-18 21:38:47 +00:00
|
|
|
/// information about a particular peer's failure reports for one osd
|
|
|
|
struct failure_reporter_t {
|
|
|
|
utime_t failed_since; ///< when they think it failed
|
2015-11-14 03:27:14 +00:00
|
|
|
MonOpRequestRef op; ///< failure op request
|
2012-09-18 21:38:47 +00:00
|
|
|
|
2015-11-14 03:27:14 +00:00
|
|
|
failure_reporter_t() {}
|
2016-01-28 15:36:23 +00:00
|
|
|
explicit failure_reporter_t(utime_t s) : failed_since(s) {}
|
2015-06-24 16:54:33 +00:00
|
|
|
~failure_reporter_t() { }
|
2012-09-18 21:38:47 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
/// information about all failure reports for one osd
|
|
|
|
struct failure_info_t {
|
2015-11-14 03:27:14 +00:00
|
|
|
map<int, failure_reporter_t> reporters; ///< reporter -> failed_since etc
|
2012-09-18 21:38:47 +00:00
|
|
|
utime_t max_failed_since; ///< most recent failed_since
|
|
|
|
|
2015-11-14 03:27:14 +00:00
|
|
|
failure_info_t() {}
|
2012-09-18 21:38:47 +00:00
|
|
|
|
2012-09-04 18:50:30 +00:00
|
|
|
utime_t get_failed_since() {
|
2013-02-13 13:10:21 +00:00
|
|
|
if (max_failed_since == utime_t() && !reporters.empty()) {
|
2012-09-04 18:50:30 +00:00
|
|
|
// the old max must have canceled; recalculate.
|
|
|
|
for (map<int, failure_reporter_t>::iterator p = reporters.begin();
|
|
|
|
p != reporters.end();
|
|
|
|
++p)
|
|
|
|
if (p->second.failed_since > max_failed_since)
|
|
|
|
max_failed_since = p->second.failed_since;
|
|
|
|
}
|
|
|
|
return max_failed_since;
|
|
|
|
}
|
|
|
|
|
2015-06-24 16:54:33 +00:00
|
|
|
// set the message for the latest report. return any old op request we had,
|
2012-09-04 20:04:58 +00:00
|
|
|
// if any, so we can discard it.
|
2015-06-24 16:54:33 +00:00
|
|
|
MonOpRequestRef add_report(int who, utime_t failed_since,
|
2015-11-14 03:27:14 +00:00
|
|
|
MonOpRequestRef op) {
|
2012-09-18 21:38:47 +00:00
|
|
|
map<int, failure_reporter_t>::iterator p = reporters.find(who);
|
|
|
|
if (p == reporters.end()) {
|
2016-08-11 03:39:11 +00:00
|
|
|
if (max_failed_since < failed_since)
|
2012-09-18 21:38:47 +00:00
|
|
|
max_failed_since = failed_since;
|
2012-09-04 20:04:58 +00:00
|
|
|
p = reporters.insert(map<int, failure_reporter_t>::value_type(who, failure_reporter_t(failed_since))).first;
|
2012-09-18 21:38:47 +00:00
|
|
|
}
|
2012-09-04 20:04:58 +00:00
|
|
|
|
2015-06-24 16:54:33 +00:00
|
|
|
MonOpRequestRef ret = p->second.op;
|
|
|
|
p->second.op = op;
|
2012-09-04 20:04:58 +00:00
|
|
|
return ret;
|
|
|
|
}
|
|
|
|
|
2015-06-24 16:54:33 +00:00
|
|
|
void take_report_messages(list<MonOpRequestRef>& ls) {
|
2012-09-04 20:04:58 +00:00
|
|
|
for (map<int, failure_reporter_t>::iterator p = reporters.begin();
|
|
|
|
p != reporters.end();
|
|
|
|
++p) {
|
2015-06-24 16:54:33 +00:00
|
|
|
if (p->second.op) {
|
|
|
|
ls.push_back(p->second.op);
|
|
|
|
p->second.op.reset();
|
2012-09-04 20:04:58 +00:00
|
|
|
}
|
|
|
|
}
|
2012-09-18 21:38:47 +00:00
|
|
|
}
|
|
|
|
|
2017-01-13 07:12:34 +00:00
|
|
|
MonOpRequestRef cancel_report(int who) {
|
2012-09-18 21:38:47 +00:00
|
|
|
map<int, failure_reporter_t>::iterator p = reporters.find(who);
|
|
|
|
if (p == reporters.end())
|
2017-01-13 07:12:34 +00:00
|
|
|
return MonOpRequestRef();
|
|
|
|
MonOpRequestRef ret = p->second.op;
|
2012-09-18 21:38:47 +00:00
|
|
|
reporters.erase(p);
|
2017-01-13 07:12:34 +00:00
|
|
|
return ret;
|
2012-09-18 21:38:47 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2017-04-13 11:08:45 +00:00
|
|
|
|
|
|
|
class LastEpochClean {
|
|
|
|
struct Lec {
|
|
|
|
vector<epoch_t> epoch_by_pg;
|
|
|
|
ps_t next_missing = 0;
|
|
|
|
epoch_t floor = std::numeric_limits<epoch_t>::max();
|
|
|
|
void report(ps_t pg, epoch_t last_epoch_clean);
|
|
|
|
};
|
|
|
|
std::map<uint64_t, Lec> report_by_pool;
|
|
|
|
public:
|
|
|
|
void report(const pg_t& pg, epoch_t last_epoch_clean);
|
|
|
|
void remove_pool(uint64_t pool);
|
|
|
|
epoch_t get_lower_bound(const OSDMap& latest) const;
|
|
|
|
};
|
|
|
|
|
|
|
|
|
2007-07-01 14:43:27 +00:00
|
|
|
class OSDMonitor : public PaxosService {
|
2016-04-28 23:50:12 +00:00
|
|
|
CephContext *cct;
|
2017-01-31 19:59:51 +00:00
|
|
|
|
2006-10-24 16:55:51 +00:00
|
|
|
public:
|
2006-10-11 21:36:16 +00:00
|
|
|
OSDMap osdmap;
|
2017-01-30 21:51:51 +00:00
|
|
|
|
2006-10-09 19:10:44 +00:00
|
|
|
// [leader]
|
|
|
|
OSDMap::Incremental pending_inc;
|
2013-11-09 14:05:59 +00:00
|
|
|
map<int, bufferlist> pending_metadata;
|
|
|
|
set<int> pending_metadata_rm;
|
2012-09-18 21:38:47 +00:00
|
|
|
map<int, failure_info_t> failure_info;
|
2006-10-09 19:10:44 +00:00
|
|
|
map<int,utime_t> down_pending_out; // osd down -> out
|
|
|
|
|
2007-09-28 21:07:08 +00:00
|
|
|
map<int,double> osd_weight;
|
2012-04-25 16:23:49 +00:00
|
|
|
|
2015-08-10 11:25:03 +00:00
|
|
|
SimpleLRU<version_t, bufferlist> inc_osd_cache;
|
|
|
|
SimpleLRU<version_t, bufferlist> full_osd_cache;
|
|
|
|
|
2016-04-07 08:43:17 +00:00
|
|
|
bool check_failures(utime_t now);
|
2012-09-04 20:20:32 +00:00
|
|
|
bool check_failure(utime_t now, int target_osd, failure_info_t& fi);
|
2017-07-07 07:45:21 +00:00
|
|
|
void force_failure(int target_osd, int by);
|
2012-09-04 20:20:32 +00:00
|
|
|
|
2016-11-24 06:35:58 +00:00
|
|
|
// the time of last msg(MSG_ALIVE and MSG_PGTEMP) proposed without delay
|
|
|
|
utime_t last_attempted_minwait_time;
|
|
|
|
|
2013-04-27 00:16:53 +00:00
|
|
|
bool _have_pending_crush();
|
|
|
|
CrushWrapper &_get_stable_crush();
|
2013-03-22 21:27:21 +00:00
|
|
|
void _get_pending_crush(CrushWrapper& newcrush);
|
|
|
|
|
2015-07-21 20:45:48 +00:00
|
|
|
enum FastReadType {
|
|
|
|
FAST_READ_OFF,
|
|
|
|
FAST_READ_ON,
|
|
|
|
FAST_READ_DEFAULT
|
|
|
|
};
|
|
|
|
|
2007-07-01 14:43:27 +00:00
|
|
|
// svc
|
2008-03-10 23:23:41 +00:00
|
|
|
public:
|
2017-02-22 10:07:02 +00:00
|
|
|
void create_initial() override;
|
2017-09-14 08:11:15 +00:00
|
|
|
void get_store_prefixes(std::set<string>& s) const override;
|
2017-03-09 04:08:29 +00:00
|
|
|
|
2008-03-10 23:23:41 +00:00
|
|
|
private:
|
2017-02-22 10:07:02 +00:00
|
|
|
void update_from_paxos(bool *need_bootstrap) override;
|
|
|
|
void create_pending() override; // prepare a new pending
|
|
|
|
void encode_pending(MonitorDBStore::TransactionRef t) override;
|
|
|
|
void on_active() override;
|
2017-04-19 06:55:34 +00:00
|
|
|
void on_restart() override;
|
2017-02-22 10:07:02 +00:00
|
|
|
void on_shutdown() override;
|
2013-07-23 15:34:26 +00:00
|
|
|
/**
|
|
|
|
* we haven't delegated full version stashing to paxosservice for some time
|
|
|
|
* now, making this function useless in current context.
|
|
|
|
*/
|
2017-02-22 10:07:02 +00:00
|
|
|
void encode_full(MonitorDBStore::TransactionRef t) override { }
|
2013-07-08 22:04:59 +00:00
|
|
|
/**
|
|
|
|
* do not let paxosservice periodically stash full osdmaps, or we will break our
|
|
|
|
* locally-managed full maps. (update_from_paxos loads the latest and writes them
|
|
|
|
* out going forward from there, but if we just synced that may mean we skip some.)
|
|
|
|
*/
|
2017-02-22 10:07:02 +00:00
|
|
|
bool should_stash_full() override {
|
2013-07-08 22:04:59 +00:00
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* hook into trim to include the oldest full map in the trim transaction
|
|
|
|
*
|
|
|
|
* This ensures that anyone post-sync will have enough to rebuild their
|
|
|
|
* full osdmaps.
|
|
|
|
*/
|
2017-02-22 10:07:02 +00:00
|
|
|
void encode_trim_extra(MonitorDBStore::TransactionRef tx, version_t first) override;
|
2013-07-08 22:04:59 +00:00
|
|
|
|
2012-08-13 17:46:30 +00:00
|
|
|
void update_msgr_features();
|
2014-02-04 00:31:52 +00:00
|
|
|
int check_cluster_features(uint64_t features, stringstream &ss);
|
2014-07-03 23:30:49 +00:00
|
|
|
/**
|
|
|
|
* check if the cluster supports the features required by the
|
|
|
|
* given crush map. Outputs the daemons which don't support it
|
|
|
|
* to the stringstream.
|
|
|
|
*
|
|
|
|
* @returns true if the map is passable, false otherwise
|
|
|
|
*/
|
|
|
|
bool validate_crush_against_features(const CrushWrapper *newcrush,
|
|
|
|
stringstream &ss);
|
2017-03-09 02:27:43 +00:00
|
|
|
void check_osdmap_subs();
|
2011-11-18 17:56:10 +00:00
|
|
|
void share_map_with_random_osd();
|
2007-09-09 23:47:09 +00:00
|
|
|
|
2017-01-31 20:00:28 +00:00
|
|
|
Mutex prime_pg_temp_lock = {"OSDMonitor::prime_pg_temp_lock"};
|
|
|
|
struct PrimeTempJob : public ParallelPGMapper::Job {
|
|
|
|
OSDMonitor *osdmon;
|
|
|
|
PrimeTempJob(const OSDMap& om, OSDMonitor *m)
|
|
|
|
: ParallelPGMapper::Job(&om), osdmon(m) {}
|
|
|
|
void process(int64_t pool, unsigned ps_begin, unsigned ps_end) override {
|
|
|
|
for (unsigned ps = ps_begin; ps < ps_end; ++ps) {
|
|
|
|
pg_t pgid(ps, pool);
|
|
|
|
osdmon->prime_pg_temp(*osdmap, pgid);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
void complete() override {}
|
|
|
|
};
|
2015-01-21 02:35:13 +00:00
|
|
|
void maybe_prime_pg_temp();
|
2017-01-31 20:00:28 +00:00
|
|
|
void prime_pg_temp(const OSDMap& next, pg_t pgid);
|
2015-01-21 02:35:13 +00:00
|
|
|
|
2017-03-19 06:02:15 +00:00
|
|
|
ParallelPGMapper mapper; ///< for background pg work
|
|
|
|
OSDMapMapping mapping; ///< pg <-> osd mappings
|
|
|
|
unique_ptr<ParallelPGMapper::Job> mapping_job; ///< background mapping job
|
|
|
|
void start_mapping();
|
|
|
|
|
2011-12-08 20:53:14 +00:00
|
|
|
void update_logger();
|
|
|
|
|
2009-06-23 21:03:34 +00:00
|
|
|
void handle_query(PaxosServiceMessage *m);
|
2017-02-22 10:07:02 +00:00
|
|
|
bool preprocess_query(MonOpRequestRef op) override; // true if processed.
|
|
|
|
bool prepare_update(MonOpRequestRef op) override;
|
|
|
|
bool should_propose(double &delay) override;
|
2006-10-09 19:10:44 +00:00
|
|
|
|
2017-09-14 08:11:15 +00:00
|
|
|
version_t get_trim_to() const override;
|
mon: Paxos: trim through Paxos
Instead of directly modifying the store whenever we want to trim our Paxos
state, we should do it through Paxos, proposing the trim to the quorum and
commit it once accepted.
This enforces three major invariants that we will be able to leverage later
on during the store synchronization:
1) The Leader will set the pace for trimming across the system. No one
will trim their state unless they are committing the value proposed by
the Leader;
2) Following (1), the monitors in the quorum will trim at the same time.
There will be no diverging states due to trimming on different monitors.
3) Each trim will be kept as a transaction in the Paxos' store allowing
us to obtain a consistent state during synchronization, by shipping
the Paxos versions to the other monitor and applying them. We could
incur in an inconsistent state if the trim happened without
constraints, without being logged; by going through Paxos this concern
is no longer relevant.
The trimming itself may be triggered each time a proposal finishes, which
is the time at which we know we have committed a new version on the store.
It shall be triggered iff we are sure we have enough versions on the store
to fill the gap of any monitor that might become alive and still hasn't
drifted enough to require synchronization. Roughly speaking, we will check
if the number of available versions is higher than 'paxos_max_join_drift'.
Furthermore, we added a new option, 'paxos_trim_tolerance', so we are able
to avoid trimming every single time the above condition is met -- which
would happen every time we trimmed a version, and then proposed a new one,
and then we would trim it again, etc. So, just tolerate a couple of commits
before trimming again.
Finally, we added support to enable/disable trimming, which will be
essential during the store synchronization process.
Signed-off-by: Joao Eduardo Luis <joao.luis@inktank.com>
2012-07-04 10:47:03 +00:00
|
|
|
|
2012-04-25 18:15:34 +00:00
|
|
|
bool can_mark_down(int o);
|
|
|
|
bool can_mark_up(int o);
|
|
|
|
bool can_mark_out(int o);
|
|
|
|
bool can_mark_in(int o);
|
|
|
|
|
2007-07-01 14:43:27 +00:00
|
|
|
// ...
|
2011-10-04 20:43:25 +00:00
|
|
|
MOSDMap *build_latest_full();
|
2010-09-09 18:09:57 +00:00
|
|
|
MOSDMap *build_incremental(epoch_t first, epoch_t last);
|
2015-06-24 22:48:23 +00:00
|
|
|
void send_full(MonOpRequestRef op);
|
|
|
|
void send_incremental(MonOpRequestRef op, epoch_t first);
|
2015-09-17 01:44:04 +00:00
|
|
|
public:
|
2015-07-21 05:24:11 +00:00
|
|
|
// @param req an optional op request, if the osdmaps are replies to it. so
|
|
|
|
// @c Monitor::send_reply() can mark_event with it.
|
|
|
|
void send_incremental(epoch_t first, MonSession *session, bool onetime,
|
|
|
|
MonOpRequestRef req = MonOpRequestRef());
|
2016-03-02 22:33:06 +00:00
|
|
|
|
2015-09-17 01:44:04 +00:00
|
|
|
private:
|
2015-01-09 11:14:12 +00:00
|
|
|
void print_utilization(ostream &out, Formatter *f, bool tree) const;
|
|
|
|
|
2013-03-20 21:30:29 +00:00
|
|
|
bool check_source(PaxosServiceMessage *m, uuid_d fsid);
|
2007-07-01 14:43:27 +00:00
|
|
|
|
2015-02-12 19:41:33 +00:00
|
|
|
bool preprocess_get_osdmap(MonOpRequestRef op);
|
2014-08-16 01:06:14 +00:00
|
|
|
|
2015-02-12 19:41:33 +00:00
|
|
|
bool preprocess_mark_me_down(MonOpRequestRef op);
|
2013-03-20 21:30:49 +00:00
|
|
|
|
|
|
|
friend class C_AckMarkedDown;
|
2015-02-12 19:41:33 +00:00
|
|
|
bool preprocess_failure(MonOpRequestRef op);
|
|
|
|
bool prepare_failure(MonOpRequestRef op);
|
|
|
|
bool prepare_mark_me_down(MonOpRequestRef op);
|
2012-11-12 21:39:02 +00:00
|
|
|
void process_failures();
|
2015-06-24 16:54:33 +00:00
|
|
|
void take_all_failures(list<MonOpRequestRef>& ls);
|
2007-07-01 14:43:27 +00:00
|
|
|
|
2017-02-23 20:51:37 +00:00
|
|
|
bool preprocess_full(MonOpRequestRef op);
|
|
|
|
bool prepare_full(MonOpRequestRef op);
|
|
|
|
|
2015-02-12 19:41:33 +00:00
|
|
|
bool preprocess_boot(MonOpRequestRef op);
|
|
|
|
bool prepare_boot(MonOpRequestRef op);
|
|
|
|
void _booted(MonOpRequestRef op, bool logit);
|
2007-07-01 14:43:27 +00:00
|
|
|
|
2016-06-08 00:15:05 +00:00
|
|
|
void update_up_thru(int from, epoch_t up_thru);
|
2015-02-12 19:41:33 +00:00
|
|
|
bool preprocess_alive(MonOpRequestRef op);
|
|
|
|
bool prepare_alive(MonOpRequestRef op);
|
|
|
|
void _reply_map(MonOpRequestRef op, epoch_t e);
|
2009-08-06 23:15:48 +00:00
|
|
|
|
2015-02-12 19:41:33 +00:00
|
|
|
bool preprocess_pgtemp(MonOpRequestRef op);
|
|
|
|
bool prepare_pgtemp(MonOpRequestRef op);
|
2008-05-13 19:19:05 +00:00
|
|
|
|
2017-03-13 07:50:17 +00:00
|
|
|
bool preprocess_pg_created(MonOpRequestRef op);
|
|
|
|
bool prepare_pg_created(MonOpRequestRef op);
|
|
|
|
|
2017-03-26 04:05:12 +00:00
|
|
|
int _check_remove_pool(int64_t pool_id, const pg_pool_t &pool, ostream *ss);
|
2014-09-16 09:49:15 +00:00
|
|
|
bool _check_become_tier(
|
|
|
|
int64_t tier_pool_id, const pg_pool_t *tier_pool,
|
|
|
|
int64_t base_pool_id, const pg_pool_t *base_pool,
|
|
|
|
int *err, ostream *ss) const;
|
|
|
|
bool _check_remove_tier(
|
2015-05-07 13:23:37 +00:00
|
|
|
int64_t base_pool_id, const pg_pool_t *base_pool, const pg_pool_t *tier_pool,
|
2014-09-16 09:49:15 +00:00
|
|
|
int *err, ostream *ss) const;
|
|
|
|
|
2017-04-18 19:53:50 +00:00
|
|
|
int _prepare_remove_pool(int64_t pool, ostream *ss, bool no_fake);
|
2013-10-29 15:59:06 +00:00
|
|
|
int _prepare_rename_pool(int64_t pool, string newname);
|
2011-05-26 20:17:12 +00:00
|
|
|
|
2015-02-12 19:41:33 +00:00
|
|
|
bool preprocess_pool_op (MonOpRequestRef op);
|
|
|
|
bool preprocess_pool_op_create (MonOpRequestRef op);
|
|
|
|
bool prepare_pool_op (MonOpRequestRef op);
|
|
|
|
bool prepare_pool_op_create (MonOpRequestRef op);
|
|
|
|
bool prepare_pool_op_delete(MonOpRequestRef op);
|
2014-10-16 00:14:53 +00:00
|
|
|
int crush_rename_bucket(const string& srcname,
|
|
|
|
const string& dstname,
|
|
|
|
ostream *ss);
|
2016-09-21 14:00:01 +00:00
|
|
|
void check_legacy_ec_plugin(const string& plugin,
|
|
|
|
const string& profile) const;
|
|
|
|
int normalize_profile(const string& profilename,
|
|
|
|
ErasureCodeProfile &profile,
|
2017-02-14 01:42:33 +00:00
|
|
|
bool force,
|
2016-09-21 14:00:01 +00:00
|
|
|
ostream *ss);
|
2017-03-01 16:35:55 +00:00
|
|
|
int crush_rule_create_erasure(const string &name,
|
|
|
|
const string &profile,
|
|
|
|
int *rule,
|
|
|
|
ostream *ss);
|
|
|
|
int get_crush_rule(const string &rule_name,
|
|
|
|
int *crush_rule,
|
2015-05-17 13:28:52 +00:00
|
|
|
ostream *ss);
|
2014-03-17 21:47:03 +00:00
|
|
|
int get_erasure_code(const string &erasure_code_profile,
|
2014-02-02 08:59:52 +00:00
|
|
|
ErasureCodeInterfaceRef *erasure_code,
|
2015-05-17 13:28:52 +00:00
|
|
|
ostream *ss) const;
|
2017-03-01 16:35:55 +00:00
|
|
|
int prepare_pool_crush_rule(const unsigned pool_type,
|
2014-03-17 21:47:03 +00:00
|
|
|
const string &erasure_code_profile,
|
2017-03-01 16:35:55 +00:00
|
|
|
const string &rule_name,
|
|
|
|
int *crush_rule,
|
2015-05-17 13:28:52 +00:00
|
|
|
ostream *ss);
|
2017-04-24 22:54:16 +00:00
|
|
|
bool erasure_code_profile_in_use(
|
|
|
|
const mempool::osdmap::map<int64_t, pg_pool_t> &pools,
|
|
|
|
const string &profile,
|
|
|
|
ostream *ss);
|
2014-03-16 12:00:50 +00:00
|
|
|
int parse_erasure_code_profile(const vector<string> &erasure_code_profile,
|
|
|
|
map<string,string> *erasure_code_profile_map,
|
2015-05-17 13:28:52 +00:00
|
|
|
ostream *ss);
|
2014-02-02 08:51:50 +00:00
|
|
|
int prepare_pool_size(const unsigned pool_type,
|
2014-03-17 21:47:03 +00:00
|
|
|
const string &erasure_code_profile,
|
2014-05-28 20:59:47 +00:00
|
|
|
unsigned *size, unsigned *min_size,
|
2015-05-17 13:28:52 +00:00
|
|
|
ostream *ss);
|
2014-02-05 19:40:46 +00:00
|
|
|
int prepare_pool_stripe_width(const unsigned pool_type,
|
2014-03-17 21:47:03 +00:00
|
|
|
const string &erasure_code_profile,
|
2014-02-05 19:40:46 +00:00
|
|
|
unsigned *stripe_width,
|
2015-05-17 13:28:52 +00:00
|
|
|
ostream *ss);
|
2017-09-01 18:45:12 +00:00
|
|
|
int check_pg_num(int64_t pool, int pg_num, int size, ostream* ss);
|
2014-03-16 15:24:58 +00:00
|
|
|
int prepare_new_pool(string& name, uint64_t auid,
|
2017-03-01 16:35:55 +00:00
|
|
|
int crush_rule,
|
|
|
|
const string &crush_rule_name,
|
2013-09-09 11:23:42 +00:00
|
|
|
unsigned pg_num, unsigned pgp_num,
|
2014-03-17 21:47:03 +00:00
|
|
|
const string &erasure_code_profile,
|
2013-12-20 16:23:16 +00:00
|
|
|
const unsigned pool_type,
|
2014-06-30 07:22:17 +00:00
|
|
|
const uint64_t expected_num_objects,
|
2015-07-21 20:45:48 +00:00
|
|
|
FastReadType fast_read,
|
2015-05-17 13:28:52 +00:00
|
|
|
ostream *ss);
|
2015-02-12 19:41:33 +00:00
|
|
|
int prepare_new_pool(MonOpRequestRef op);
|
2013-03-30 00:59:35 +00:00
|
|
|
|
2017-08-28 07:51:28 +00:00
|
|
|
void set_pool_flags(int64_t pool_id, uint64_t flags);
|
|
|
|
void clear_pool_flags(int64_t pool_id, uint64_t flags);
|
2013-03-30 00:59:35 +00:00
|
|
|
bool update_pools_status();
|
|
|
|
|
2015-02-12 19:41:33 +00:00
|
|
|
bool prepare_set_flag(MonOpRequestRef op, int flag);
|
|
|
|
bool prepare_unset_flag(MonOpRequestRef op, int flag);
|
2010-03-10 20:56:37 +00:00
|
|
|
|
2015-02-12 19:41:33 +00:00
|
|
|
void _pool_op_reply(MonOpRequestRef op,
|
|
|
|
int ret, epoch_t epoch, bufferlist *blp=NULL);
|
2009-06-16 21:22:32 +00:00
|
|
|
|
2015-06-17 10:00:06 +00:00
|
|
|
struct C_Booted : public C_MonOp {
|
2007-07-01 14:43:27 +00:00
|
|
|
OSDMonitor *cmon;
|
2010-01-29 21:05:44 +00:00
|
|
|
bool logit;
|
2015-06-17 10:00:06 +00:00
|
|
|
C_Booted(OSDMonitor *cm, MonOpRequestRef op_, bool l=true) :
|
|
|
|
C_MonOp(op_), cmon(cm), logit(l) {}
|
2017-02-22 10:07:02 +00:00
|
|
|
void _finish(int r) override {
|
2007-07-01 14:43:27 +00:00
|
|
|
if (r >= 0)
|
2015-02-12 19:41:33 +00:00
|
|
|
cmon->_booted(op, logit);
|
2013-02-08 06:06:14 +00:00
|
|
|
else if (r == -ECANCELED)
|
2015-02-12 19:41:33 +00:00
|
|
|
return;
|
2013-02-08 07:13:17 +00:00
|
|
|
else if (r == -EAGAIN)
|
2015-02-12 19:41:33 +00:00
|
|
|
cmon->dispatch(op);
|
2013-02-08 07:13:17 +00:00
|
|
|
else
|
|
|
|
assert(0 == "bad C_Booted return value");
|
2007-07-01 14:43:27 +00:00
|
|
|
}
|
|
|
|
};
|
2009-06-23 21:03:34 +00:00
|
|
|
|
2015-06-17 10:00:06 +00:00
|
|
|
struct C_ReplyMap : public C_MonOp {
|
2008-05-13 19:19:05 +00:00
|
|
|
OSDMonitor *osdmon;
|
2009-08-06 23:15:48 +00:00
|
|
|
epoch_t e;
|
2015-02-12 19:41:33 +00:00
|
|
|
C_ReplyMap(OSDMonitor *o, MonOpRequestRef op_, epoch_t ee)
|
2015-06-17 10:00:06 +00:00
|
|
|
: C_MonOp(op_), osdmon(o), e(ee) {}
|
2017-02-22 10:07:02 +00:00
|
|
|
void _finish(int r) override {
|
2013-02-08 07:13:17 +00:00
|
|
|
if (r >= 0)
|
2015-02-12 19:41:33 +00:00
|
|
|
osdmon->_reply_map(op, e);
|
2013-02-08 07:13:17 +00:00
|
|
|
else if (r == -ECANCELED)
|
2015-02-12 19:41:33 +00:00
|
|
|
return;
|
2013-02-08 07:13:17 +00:00
|
|
|
else if (r == -EAGAIN)
|
2015-02-12 19:41:33 +00:00
|
|
|
osdmon->dispatch(op);
|
2013-02-08 07:13:17 +00:00
|
|
|
else
|
|
|
|
assert(0 == "bad C_ReplyMap return value");
|
2008-05-13 19:19:05 +00:00
|
|
|
}
|
|
|
|
};
|
2015-06-17 10:00:06 +00:00
|
|
|
struct C_PoolOp : public C_MonOp {
|
2009-06-16 21:22:32 +00:00
|
|
|
OSDMonitor *osdmon;
|
|
|
|
int replyCode;
|
|
|
|
int epoch;
|
2012-12-09 05:44:54 +00:00
|
|
|
bufferlist reply_data;
|
2015-02-12 19:41:33 +00:00
|
|
|
C_PoolOp(OSDMonitor * osd, MonOpRequestRef op_, int rc, int e, bufferlist *rd=NULL) :
|
2015-06-17 10:00:06 +00:00
|
|
|
C_MonOp(op_), osdmon(osd), replyCode(rc), epoch(e) {
|
2012-12-09 05:44:54 +00:00
|
|
|
if (rd)
|
|
|
|
reply_data = *rd;
|
|
|
|
}
|
2017-02-22 10:07:02 +00:00
|
|
|
void _finish(int r) override {
|
2013-02-08 07:13:17 +00:00
|
|
|
if (r >= 0)
|
2015-02-12 19:41:33 +00:00
|
|
|
osdmon->_pool_op_reply(op, replyCode, epoch, &reply_data);
|
2013-02-08 07:13:17 +00:00
|
|
|
else if (r == -ECANCELED)
|
2015-02-12 19:41:33 +00:00
|
|
|
return;
|
2013-02-08 07:13:17 +00:00
|
|
|
else if (r == -EAGAIN)
|
2015-02-12 19:41:33 +00:00
|
|
|
osdmon->dispatch(op);
|
2013-02-08 07:13:17 +00:00
|
|
|
else
|
|
|
|
assert(0 == "bad C_PoolOp return value");
|
2009-06-16 21:22:32 +00:00
|
|
|
}
|
|
|
|
};
|
2007-07-01 14:43:27 +00:00
|
|
|
|
2015-02-12 19:41:33 +00:00
|
|
|
bool preprocess_remove_snaps(MonOpRequestRef op);
|
|
|
|
bool prepare_remove_snaps(MonOpRequestRef op);
|
|
|
|
|
|
|
|
OpTracker op_tracker;
|
2008-08-07 21:21:16 +00:00
|
|
|
|
2015-03-31 15:41:53 +00:00
|
|
|
int load_metadata(int osd, map<string, string>& m, ostream *err);
|
2017-06-02 18:46:12 +00:00
|
|
|
void count_metadata(const string& field, Formatter *f);
|
2017-07-20 20:25:42 +00:00
|
|
|
public:
|
|
|
|
void count_metadata(const string& field, map<string,int> *out);
|
|
|
|
protected:
|
2017-04-06 18:40:41 +00:00
|
|
|
int get_osd_objectstore_type(int osd, std::string *type);
|
|
|
|
bool is_pool_currently_all_bluestore(int64_t pool_id, const pg_pool_t &pool,
|
|
|
|
ostream *err);
|
2015-03-31 15:41:53 +00:00
|
|
|
|
2017-03-09 08:25:59 +00:00
|
|
|
// when we last received PG stats from each osd
|
|
|
|
map<int,utime_t> last_osd_report;
|
2017-04-13 11:08:45 +00:00
|
|
|
// TODO: use last_osd_report to store the osd report epochs, once we don't
|
|
|
|
// need to upgrade from pre-luminous releases.
|
|
|
|
map<int,epoch_t> osd_epochs;
|
|
|
|
LastEpochClean last_epoch_clean;
|
2017-03-09 08:25:59 +00:00
|
|
|
bool preprocess_beacon(MonOpRequestRef op);
|
|
|
|
bool prepare_beacon(MonOpRequestRef op);
|
2017-04-13 11:08:45 +00:00
|
|
|
epoch_t get_min_last_epoch_clean() const;
|
2017-03-09 08:25:59 +00:00
|
|
|
|
2017-03-09 04:08:29 +00:00
|
|
|
friend class C_UpdateCreatingPGs;
|
|
|
|
std::map<int, std::map<epoch_t, std::set<pg_t>>> creating_pgs_by_osd_epoch;
|
|
|
|
std::vector<pg_t> pending_created_pgs;
|
|
|
|
// the epoch when the pg mapping was calculated
|
|
|
|
epoch_t creating_pgs_epoch = 0;
|
|
|
|
creating_pgs_t creating_pgs;
|
2017-07-30 05:07:02 +00:00
|
|
|
mutable std::mutex creating_pgs_lock;
|
2017-03-09 04:08:29 +00:00
|
|
|
|
2017-04-13 11:08:45 +00:00
|
|
|
creating_pgs_t update_pending_pgs(const OSDMap::Incremental& inc);
|
2017-05-05 20:37:21 +00:00
|
|
|
void trim_creating_pgs(creating_pgs_t *creating_pgs,
|
|
|
|
const ceph::unordered_map<pg_t,pg_stat_t>& pgm);
|
2017-05-24 11:34:08 +00:00
|
|
|
unsigned scan_for_creating_pgs(
|
2017-04-24 22:54:16 +00:00
|
|
|
const mempool::osdmap::map<int64_t,pg_pool_t>& pools,
|
|
|
|
const mempool::osdmap::set<int64_t>& removed_pools,
|
|
|
|
utime_t modified,
|
|
|
|
creating_pgs_t* creating_pgs) const;
|
2017-03-09 04:08:29 +00:00
|
|
|
pair<int32_t, pg_t> get_parent_pg(pg_t pgid) const;
|
|
|
|
void update_creating_pgs();
|
|
|
|
void check_pg_creates_subs();
|
2017-07-30 05:07:02 +00:00
|
|
|
epoch_t send_pg_creates(int osd, Connection *con, epoch_t next) const;
|
2017-03-09 04:08:29 +00:00
|
|
|
|
2017-05-30 12:34:20 +00:00
|
|
|
int32_t _allocate_osd_id(int32_t* existing_id);
|
|
|
|
|
2017-03-09 08:25:59 +00:00
|
|
|
public:
|
2015-08-10 11:25:03 +00:00
|
|
|
OSDMonitor(CephContext *cct, Monitor *mn, Paxos *p, const string& service_name);
|
2006-10-05 05:39:29 +00:00
|
|
|
|
2017-02-22 10:07:02 +00:00
|
|
|
void tick() override; // check state, take actions
|
2006-10-05 05:39:29 +00:00
|
|
|
|
2015-02-12 19:41:33 +00:00
|
|
|
bool preprocess_command(MonOpRequestRef op);
|
|
|
|
bool prepare_command(MonOpRequestRef op);
|
|
|
|
bool prepare_command_impl(MonOpRequestRef op, map<string,cmd_vartype>& cmdmap);
|
2007-12-19 04:53:48 +00:00
|
|
|
|
2017-05-30 12:34:20 +00:00
|
|
|
int validate_osd_create(
|
|
|
|
const int32_t id,
|
|
|
|
const uuid_d& uuid,
|
|
|
|
const bool check_osd_exists,
|
|
|
|
int32_t* existing_id,
|
|
|
|
stringstream& ss);
|
2017-05-23 17:23:01 +00:00
|
|
|
int prepare_command_osd_create(
|
|
|
|
const int32_t id,
|
|
|
|
const uuid_d& uuid,
|
|
|
|
int32_t* existing_id,
|
|
|
|
stringstream& ss);
|
|
|
|
void do_osd_create(const int32_t id, const uuid_d& uuid, int32_t* new_id);
|
2017-04-12 13:29:23 +00:00
|
|
|
int prepare_command_osd_purge(int32_t id, stringstream& ss);
|
2017-05-31 08:55:36 +00:00
|
|
|
int prepare_command_osd_destroy(int32_t id, stringstream& ss);
|
2017-04-12 13:29:23 +00:00
|
|
|
int _prepare_command_osd_crush_remove(
|
|
|
|
CrushWrapper &newcrush,
|
|
|
|
int32_t id,
|
|
|
|
int32_t ancestor,
|
|
|
|
bool has_ancestor,
|
|
|
|
bool unlink_only);
|
|
|
|
void do_osd_crush_remove(CrushWrapper& newcrush);
|
2017-04-12 13:20:43 +00:00
|
|
|
int prepare_command_osd_crush_remove(
|
|
|
|
CrushWrapper &newcrush,
|
|
|
|
int32_t id,
|
|
|
|
int32_t ancestor,
|
|
|
|
bool has_ancestor,
|
|
|
|
bool unlink_only);
|
|
|
|
int prepare_command_osd_remove(int32_t id);
|
2017-05-02 22:58:51 +00:00
|
|
|
int prepare_command_osd_new(
|
|
|
|
MonOpRequestRef op,
|
|
|
|
const map<string,cmd_vartype>& cmdmap,
|
|
|
|
const map<string,string>& secrets,
|
2017-05-30 12:34:20 +00:00
|
|
|
stringstream &ss,
|
|
|
|
Formatter *f);
|
2017-05-02 22:58:51 +00:00
|
|
|
|
2013-10-10 22:34:06 +00:00
|
|
|
int prepare_command_pool_set(map<string,cmd_vartype> &cmdmap,
|
|
|
|
stringstream& ss);
|
2017-06-19 20:18:59 +00:00
|
|
|
int prepare_command_pool_application(const string &prefix,
|
|
|
|
map<string,cmd_vartype> &cmdmap,
|
|
|
|
stringstream& ss);
|
2013-10-10 22:34:06 +00:00
|
|
|
|
2017-03-09 08:25:59 +00:00
|
|
|
bool handle_osd_timeouts(const utime_t &now,
|
2012-04-24 17:55:18 +00:00
|
|
|
std::map<int,utime_t> &last_osd_report);
|
2007-02-26 00:17:32 +00:00
|
|
|
|
2015-06-24 22:48:23 +00:00
|
|
|
void send_latest(MonOpRequestRef op, epoch_t start=0);
|
|
|
|
void send_latest_now_nodelete(MonOpRequestRef op, epoch_t start=0) {
|
|
|
|
op->mark_osdmon_event(__func__);
|
|
|
|
send_incremental(op, start);
|
2009-11-06 21:54:53 +00:00
|
|
|
}
|
2007-05-16 21:53:22 +00:00
|
|
|
|
2015-08-10 11:25:03 +00:00
|
|
|
int get_version(version_t ver, bufferlist& bl) override;
|
|
|
|
int get_version_full(version_t ver, bufferlist& bl) override;
|
|
|
|
|
2012-10-21 22:31:17 +00:00
|
|
|
epoch_t blacklist(const entity_addr_t& a, utime_t until);
|
2008-12-16 18:41:55 +00:00
|
|
|
|
2012-08-21 21:22:20 +00:00
|
|
|
void dump_info(Formatter *f);
|
2013-11-30 06:26:57 +00:00
|
|
|
int dump_osd_metadata(int osd, Formatter *f, ostream *err);
|
2015-03-31 15:41:53 +00:00
|
|
|
void print_nodes(Formatter *f);
|
2012-08-21 21:22:20 +00:00
|
|
|
|
2017-03-09 02:27:43 +00:00
|
|
|
void check_osdmap_sub(Subscription *sub);
|
2017-03-09 04:08:29 +00:00
|
|
|
void check_pg_creates_sub(Subscription *sub);
|
2009-08-28 23:48:09 +00:00
|
|
|
|
2017-08-23 20:34:07 +00:00
|
|
|
void do_application_enable(int64_t pool_id, const std::string &app_name,
|
|
|
|
const std::string &app_key="",
|
|
|
|
const std::string &app_value="");
|
2017-06-20 16:13:42 +00:00
|
|
|
|
2009-10-22 05:48:12 +00:00
|
|
|
void add_flag(int flag) {
|
2009-10-23 00:15:20 +00:00
|
|
|
if (!(osdmap.flags & flag)) {
|
|
|
|
if (pending_inc.new_flags < 0)
|
|
|
|
pending_inc.new_flags = osdmap.flags;
|
|
|
|
pending_inc.new_flags |= flag;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void remove_flag(int flag) {
|
|
|
|
if(osdmap.flags & flag) {
|
|
|
|
if (pending_inc.new_flags < 0)
|
|
|
|
pending_inc.new_flags = osdmap.flags;
|
|
|
|
pending_inc.new_flags &= ~flag;
|
|
|
|
}
|
2009-10-22 05:48:12 +00:00
|
|
|
}
|
2006-10-05 05:39:29 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
#endif
|