]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/mon/Monitor.h
update source to Ceph Pacific 16.2.2
[ceph.git] / ceph / src / mon / Monitor.h
index deda57a93dd9da2f1c77add479a57ccc80be19e1..19972c27631b192353768ac748be9f9b7cb06d3f 100644 (file)
@@ -1,4 +1,4 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
 // vim: ts=8 sw=2 smarttab
 /*
  * Ceph - scalable distributed file system
@@ -7,16 +7,16 @@
  *
  * 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 
+ * License version 2.1, as published by the Free Software
  * Foundation.  See file COPYING.
- * 
+ *
  */
 
-/* 
- * This is the top level monitor. It runs on each machine in the Monitor   
- * Cluster. The election of a leader for the paxos algorithm only happens 
- * once per machine via the elector. There is a separate paxos instance (state) 
- * kept for each of the system components: Object Store Device (OSD) Monitor, 
+/*
+ * This is the top level monitor. It runs on each machine in the Monitor
+ * Cluster. The election of a leader for the paxos algorithm only happens
+ * once per machine via the elector. There is a separate paxos instance (state)
+ * kept for each of the system components: Object Store Device (OSD) Monitor,
  * Placement Group (PG) Monitor, Metadata Server (MDS) Monitor, and Client Monitor.
  */
 
@@ -26,6 +26,7 @@
 #include <errno.h>
 #include <cmath>
 #include <string>
+#include <array>
 
 #include "include/types.h"
 #include "include/health.h"
@@ -98,7 +99,6 @@ enum {
   l_mon_last,
 };
 
-class QuorumService;
 class PaxosService;
 
 class AdminSocketHook;
@@ -114,7 +114,7 @@ public:
   const char **orig_argv = nullptr;
 
   // me
-  string name;
+  std::string name;
   int rank;
   Messenger *messenger;
   ConnectionRef con_self;
@@ -139,7 +139,7 @@ public:
   MonMap *monmap;
   uuid_d fingerprint;
 
-  set<entity_addrvec_t> extra_probe_peers;
+  std::set<entity_addrvec_t> extra_probe_peers;
 
   LogClient log_client;
   LogChannelRef clog;
@@ -152,12 +152,12 @@ public:
 
   CompatSet features;
 
-  vector<MonCommand> leader_mon_commands; // quorum leader's commands
-  vector<MonCommand> local_mon_commands;  // commands i support
-  bufferlist local_mon_commands_bl;       // encoded version of above
+  std::vector<MonCommand> leader_mon_commands; // quorum leader's commands
+  std::vector<MonCommand> local_mon_commands;  // commands i support
+  ceph::buffer::list local_mon_commands_bl;       // encoded version of above
 
-  vector<MonCommand> prenautilus_local_mon_commands;
-  bufferlist prenautilus_local_mon_commands_bl;
+  std::vector<MonCommand> prenautilus_local_mon_commands;
+  ceph::buffer::list prenautilus_local_mon_commands_bl;
 
   Messenger *mgr_messenger;
   MgrClient mgr_client;
@@ -170,8 +170,8 @@ private:
   // -- local storage --
 public:
   MonitorDBStore *store;
-  static const string MONITOR_NAME;
-  static const string MONITOR_STORE_PREFIX;
+  static const std::string MONITOR_NAME;
+  static const std::string MONITOR_STORE_PREFIX;
 
   // -- monitor state --
 private:
@@ -218,7 +218,7 @@ public:
 
   // -- elector --
 private:
-  Paxos *paxos;
+  std::unique_ptr<Paxos> paxos;
   Elector elector;
   friend class Elector;
 
@@ -226,14 +226,14 @@ private:
   uint64_t required_features;
   
   int leader;            // current leader (to best of knowledge)
-  set<int> quorum;       // current active set of monitors (if !starting)
-  mono_clock::time_point quorum_since;  // when quorum formed
+  std::set<int> quorum;       // current active set of monitors (if !starting)
+  ceph::mono_clock::time_point quorum_since;  // when quorum formed
   utime_t leader_since;  // when this monitor became the leader, if it is the leader
   utime_t exited_quorum; // time detected as not in quorum; 0 if in
 
   // map of counts of connected clients, by type and features, for
   // each quorum mon
-  map<int,FeatureMap> quorum_feature_map;
+  std::map<int,FeatureMap> quorum_feature_map;
 
   /**
    * Intersection of quorum member's connection feature bits.
@@ -246,14 +246,42 @@ private:
 
   ceph_release_t quorum_min_mon_release{ceph_release_t::unknown};
 
-  set<string> outside_quorum;
+  std::set<std::string> outside_quorum;
+
+  bool stretch_mode_engaged{false};
+  bool degraded_stretch_mode{false};
+  bool recovering_stretch_mode{false};
+  string stretch_bucket_divider;
+  map<string, set<string>> dead_mon_buckets; // bucket->mon ranks, locations with no live mons
+  set<string> up_mon_buckets; // locations with a live mon
+  void do_stretch_mode_election_work();
+
+  bool session_stretch_allowed(MonSession *s, MonOpRequestRef& op);
+  void disconnect_disallowed_stretch_sessions();
+  void set_elector_disallowed_leaders(bool allow_election);
+public:
+  bool is_stretch_mode() { return stretch_mode_engaged; }
+  bool is_degraded_stretch_mode() { return degraded_stretch_mode; }
+  bool is_recovering_stretch_mode() { return recovering_stretch_mode; }
+  void maybe_engage_stretch_mode();
+  void maybe_go_degraded_stretch_mode();
+  void trigger_degraded_stretch_mode(const set<string>& dead_mons,
+                                    const set<int>& dead_buckets);
+  void set_degraded_stretch_mode();
+  void go_recovery_stretch_mode();
+  void trigger_healthy_stretch_mode();
+  void set_healthy_stretch_mode();
+  void enable_stretch_mode();
+
+  
+private:
 
   /**
    * @defgroup Monitor_h_scrub
    * @{
    */
   version_t scrub_version;            ///< paxos version we are scrubbing
-  map<int,ScrubResult> scrub_result;  ///< results so far
+  std::map<int,ScrubResult> scrub_result;  ///< results so far
 
   /**
    * trigger a cross-mon scrub
@@ -264,13 +292,13 @@ private:
   int scrub();
   void handle_scrub(MonOpRequestRef op);
   bool _scrub(ScrubResult *r,
-              pair<string,string> *start,
+              std::pair<std::string,std::string> *start,
               int *num_keys);
   void scrub_check_results();
   void scrub_timeout();
   void scrub_finish();
   void scrub_reset();
-  void scrub_update_interval(int secs);
+  void scrub_update_interval(ceph::timespan interval);
 
   Context *scrub_event;       ///< periodic event to trigger scrub (leader)
   Context *scrub_timeout_event;  ///< scrub round timeout (leader)
@@ -280,7 +308,7 @@ private:
   void scrub_cancel_timeout();
 
   struct ScrubState {
-    pair<string,string> last_key; ///< last scrubbed key
+    std::pair<std::string,std::string> last_key; ///< last scrubbed key
     bool finished;
 
     ScrubState() : finished(false) { }
@@ -300,7 +328,7 @@ private:
     uint64_t cookie;       ///< unique cookie for this sync attempt
     utime_t timeout;       ///< when we give up and expire this attempt
     version_t last_committed; ///< last paxos version on peer
-    pair<string,string> last_key; ///< last key sent to (or on) peer
+    std::pair<std::string,std::string> last_key; ///< last key sent to (or on) peer
     bool full;             ///< full scan?
     MonitorDBStore::Synchronizer synchronizer;   ///< iterator
 
@@ -312,7 +340,7 @@ private:
     }
   };
 
-  map<uint64_t, SyncProvider> sync_providers;  ///< cookie -> SyncProvider for those syncing from us
+  std::map<std::uint64_t, SyncProvider> sync_providers;  ///< cookie -> SyncProvider for those syncing from us
   uint64_t sync_provider_count;   ///< counter for issued cookies to keep them unique
 
   /**
@@ -362,7 +390,7 @@ private:
    *
    * @returns a set of strings referring to the prefixes being synchronized
    */
-  set<string> get_sync_targets_names();
+  std::set<std::string> get_sync_targets_names();
 
   /**
    * Reset the monitor's sync-related data structures for syncing *from* a peer
@@ -382,7 +410,7 @@ private:
   /**
    * Get the latest monmap for backup purposes during sync
    */
-  void sync_obtain_latest_monmap(bufferlist &bl);
+  void sync_obtain_latest_monmap(ceph::buffer::list &bl);
 
   /**
    * Start sync process
@@ -398,7 +426,7 @@ public:
   /**
    * force a sync on next mon restart
    */
-  void sync_force(Formatter *f);
+  void sync_force(ceph::Formatter *f);
 
 private:
   /**
@@ -459,8 +487,8 @@ private:
    * @} // Synchronization
    */
 
-  list<Context*> waitfor_quorum;
-  list<Context*> maybe_wait_for_quorum;
+  std::list<Context*> waitfor_quorum;
+  std::list<Context*> maybe_wait_for_quorum;
 
   /**
    * @defgroup Monitor_h_TimeCheck Monitor Clock Drift Early Warning System
@@ -486,9 +514,9 @@ private:
    *  - Once all the quorum members have pong'ed, the leader will share the
    *    clock skew and latency maps with all the monitors in the quorum.
    */
-  map<int, utime_t> timecheck_waiting;
-  map<int, double> timecheck_skews;
-  map<int, double> timecheck_latencies;
+  std::map<int, utime_t> timecheck_waiting;
+  std::map<int, double> timecheck_skews;
+  std::map<int, double> timecheck_latencies;
   // odd value means we are mid-round; even value means the round has
   // finished.
   version_t timecheck_round;
@@ -519,7 +547,7 @@ private:
   void timecheck_check_skews();
   void timecheck_report();
   void timecheck();
-  health_status_t timecheck_status(ostringstream &ss,
+  health_status_t timecheck_status(std::ostringstream &ss,
                                    const double skew_bound,
                                    const double latency);
   void handle_timecheck_leader(MonOpRequestRef op);
@@ -555,13 +583,13 @@ private:
 public:
   epoch_t get_epoch();
   int get_leader() const { return leader; }
-  string get_leader_name() {
-    return quorum.empty() ? string() : monmap->get_name(*quorum.begin());
+  std::string get_leader_name() {
+    return quorum.empty() ? std::string() : monmap->get_name(leader);
   }
-  const set<int>& get_quorum() const { return quorum; }
-  list<string> get_quorum_names() {
-    list<string> q;
-    for (set<int>::iterator p = quorum.begin(); p != quorum.end(); ++p)
+  const std::set<int>& get_quorum() const { return quorum; }
+  std::list<std::string> get_quorum_names() {
+    std::list<std::string> q;
+    for (auto p = quorum.begin(); p != quorum.end(); ++p)
       q.push_back(monmap->get_name(*p));
     return q;
   }
@@ -594,12 +622,12 @@ public:
   void start_election();
   void win_standalone_election();
   // end election (called by Elector)
-  void win_election(epoch_t epoch, const set<int>& q,
+  void win_election(epoch_t epoch, const std::set<int>& q,
                    uint64_t features,
                     const mon_feature_t& mon_features,
                    ceph_release_t min_mon_release,
-                   const map<int,Metadata>& metadata);
-  void lose_election(epoch_t epoch, set<int>& q, int l,
+                   const std::map<int,Metadata>& metadata);
+  void lose_election(epoch_t epoch, std::set<int>& q, int l,
                     uint64_t features,
                      const mon_feature_t& mon_features,
                     ceph_release_t min_mon_release);
@@ -611,7 +639,7 @@ public:
   /**
    * Vector holding the Services serviced by this Monitor.
    */
-  vector<std::unique_ptr<PaxosService>> paxos_service;
+  std::array<std::unique_ptr<PaxosService>, PAXOS_NUM> paxos_service;
 
   class MDSMonitor *mdsmon() {
     return (class MDSMonitor *)paxos_service[PAXOS_MDSMAP].get();
@@ -649,14 +677,16 @@ public:
     return (class ConfigMonitor*) paxos_service[PAXOS_CONFIG].get();
   }
 
+  class KVMonitor *kvmon() {
+    return (class KVMonitor*) paxos_service[PAXOS_KV].get();
+  }
+
   friend class Paxos;
   friend class OSDMonitor;
   friend class MDSMonitor;
   friend class MonmapMonitor;
   friend class LogMonitor;
-  friend class ConfigKeyService;
-
-  QuorumService *config_key_service;
+  friend class KVMonitor;
 
   // -- sessions --
   MonSessionMap session_map;
@@ -676,37 +706,36 @@ public:
   void handle_mon_get_map(MonOpRequestRef op);
 
   static void _generate_command_map(cmdmap_t& cmdmap,
-                                    map<string,string> &param_str_map);
+                                    std::map<std::string,std::string> &param_str_map);
   static const MonCommand *_get_moncommand(
-    const string &cmd_prefix,
-    const vector<MonCommand>& cmds);
-  bool _allowed_command(MonSession *s, const string& module,
-                       const string& prefix,
+    const std::string &cmd_prefix,
+    const std::vector<MonCommand>& cmds);
+  bool _allowed_command(MonSession *s, const std::string& module,
+                       const std::string& prefix,
                         const cmdmap_t& cmdmap,
-                        const map<string,string>& param_str_map,
+                        const std::map<std::string,std::string>& param_str_map,
                         const MonCommand *this_cmd);
-  void get_mon_status(Formatter *f);
-  void _quorum_status(Formatter *f, ostream& ss);
+  void get_mon_status(ceph::Formatter *f);
+  void _quorum_status(ceph::Formatter *f, std::ostream& ss);
   bool _add_bootstrap_peer_hint(std::string_view cmd, const cmdmap_t& cmdmap,
                                std::ostream& ss);
   void handle_tell_command(MonOpRequestRef op);
   void handle_command(MonOpRequestRef op);
   void handle_route(MonOpRequestRef op);
 
-  void handle_mon_metadata(MonOpRequestRef op);
-  int get_mon_metadata(int mon, Formatter *f, ostream& err);
-  int print_nodes(Formatter *f, ostream& err);
+  int get_mon_metadata(int mon, ceph::Formatter *f, std::ostream& err);
+  int print_nodes(ceph::Formatter *f, std::ostream& err);
 
-  // Accumulate metadata across calls to update_mon_metadata
-  map<int, Metadata> mon_metadata;
-  map<int, Metadata> pending_metadata;
+  // track metadata reported by win_election()
+  std::map<int, Metadata> mon_metadata;
+  std::map<int, Metadata> pending_metadata;
 
   /**
    *
    */
   struct health_cache_t {
     health_status_t overall;
-    string summary;
+    std::string summary;
 
     void reset() {
       // health_status_t doesn't really have a NONE value and we're not
@@ -755,12 +784,13 @@ protected:
 
 public:
 
-  void get_cluster_status(stringstream &ss, Formatter *f);
+  void get_cluster_status(std::stringstream &ss, ceph::Formatter *f,
+                         MonSession *session);
 
-  void reply_command(MonOpRequestRef op, int rc, const string &rs, version_t version);
-  void reply_command(MonOpRequestRef op, int rc, const string &rs, bufferlist& rdata, version_t version);
+  void reply_command(MonOpRequestRef op, int rc, const std::string &rs, version_t version);
+  void reply_command(MonOpRequestRef op, int rc, const std::string &rs, ceph::buffer::list& rdata, version_t version);
 
-  void reply_tell_command(MonOpRequestRef op, int rc, const string &rs);
+  void reply_tell_command(MonOpRequestRef op, int rc, const std::string &rs);
 
 
 
@@ -785,7 +815,7 @@ public:
   // request routing
   struct RoutedRequest {
     uint64_t tid;
-    bufferlist request_bl;
+    ceph::buffer::list request_bl;
     MonSession *session;
     ConnectionRef con;
     uint64_t con_features;
@@ -798,8 +828,8 @@ public:
     }
   };
   uint64_t routed_request_tid;
-  map<uint64_t, RoutedRequest*> routed_requests;
-  
+  std::map<uint64_t, RoutedRequest*> routed_requests;
+
   void forward_request_leader(MonOpRequestRef op);
   void handle_forward(MonOpRequestRef op);
   void send_reply(MonOpRequestRef op, Message *reply);
@@ -810,23 +840,24 @@ public:
   void waitlist_or_zap_client(MonOpRequestRef op);
 
   void send_mon_message(Message *m, int rank);
+  void notify_new_monmap();
 
 public:
   struct C_Command : public C_MonOp {
-    Monitor *mon;
+    Monitor &mon;
     int rc;
-    string rs;
-    bufferlist rdata;
+    std::string rs;
+    ceph::buffer::list rdata;
     version_t version;
-    C_Command(Monitor *_mm, MonOpRequestRef _op, int r, string s, version_t v) :
+    C_Command(Monitor &_mm, MonOpRequestRef _op, int r, std::string s, version_t v) :
       C_MonOp(_op), mon(_mm), rc(r), rs(s), version(v){}
-    C_Command(Monitor *_mm, MonOpRequestRef _op, int r, string s, bufferlist rd, version_t v) :
+    C_Command(Monitor &_mm, MonOpRequestRef _op, int r, std::string s, ceph::buffer::list rd, version_t v) :
       C_MonOp(_op), mon(_mm), rc(r), rs(s), rdata(rd), version(v){}
 
     void _finish(int r) override {
       auto m = op->get_req<MMonCommand>();
       if (r >= 0) {
-        ostringstream ss;
+       std::ostringstream ss;
         if (!op->get_req()->get_connection()) {
           ss << "connection dropped for command ";
         } else {
@@ -841,20 +872,20 @@ public:
           }
         }
         cmdmap_t cmdmap;
-        stringstream ds;
+        std::ostringstream ds;
         string prefix;
         cmdmap_from_json(m->cmd, &cmdmap, ds);
         cmd_getval(cmdmap, "prefix", prefix);
         if (prefix != "config set" && prefix != "config-key set")
           ss << "cmd='" << m->cmd << "': finished";
 
-        mon->audit_clog->info() << ss.str();
-       mon->reply_command(op, rc, rs, rdata, version);
+        mon.audit_clog->info() << ss.str();
+        mon.reply_command(op, rc, rs, rdata, version);
       }
       else if (r == -ECANCELED)
         return;
       else if (r == -EAGAIN)
-       mon->dispatch_op(op);
+        mon.dispatch_op(op);
       else
        ceph_abort_msg("bad C_Command return value");
     }
@@ -902,19 +933,19 @@ private:
     Connection *con,
     AuthConnectionMeta *auth_meta,
     uint32_t *method,
-    vector<uint32_t> *preferred_modes,
-    bufferlist *out) override;
+    std::vector<uint32_t> *preferred_modes,
+    ceph::buffer::list *out) override;
   int handle_auth_reply_more(
     Connection *con,
     AuthConnectionMeta *auth_meta,
-   const bufferlist& bl,
-    bufferlist *reply) override;
+   const ceph::buffer::list& bl,
+    ceph::buffer::list *reply) override;
   int handle_auth_done(
     Connection *con,
     AuthConnectionMeta *auth_meta,
     uint64_t global_id,
     uint32_t con_mode,
-    const bufferlist& bl,
+    const ceph::buffer::list& bl,
     CryptoKey *session_key,
     std::string *connection_secret) override;
   int handle_auth_bad_method(
@@ -931,18 +962,20 @@ private:
     AuthConnectionMeta *auth_meta,
     bool more,
     uint32_t auth_method,
-    const bufferlist& bl,
-    bufferlist *reply) override;
+    const ceph::buffer::list& bl,
+    ceph::buffer::list *reply) override;
   // /AuthServer
 
-  int write_default_keyring(bufferlist& bl);
+  int write_default_keyring(ceph::buffer::list& bl);
   void extract_save_mon_key(KeyRing& keyring);
 
   void collect_metadata(Metadata *m);
-  void update_mon_metadata(int from, Metadata&& m);
   int load_metadata();
-  void count_metadata(const string& field, Formatter *f);
-  void count_metadata(const string& field, map<string,int> *out);
+  void count_metadata(const std::string& field, ceph::Formatter *f);
+  void count_metadata(const std::string& field, std::map<std::string,int> *out);
+  // get_all_versions() gathers version information from daemons for health check
+  void get_all_versions(std::map<string, std::list<std::string>> &versions);
+  void get_versions(std::map<string, std::list<std::string>> &versions);
 
   // features
   static CompatSet get_initial_supported_features();
@@ -956,7 +989,7 @@ private:
   OpTracker op_tracker;
 
  public:
-  Monitor(CephContext *cct_, string nm, MonitorDBStore *s,
+  Monitor(CephContext *cct_, std::string nm, MonitorDBStore *s,
          Messenger *m, Messenger *mgr_m, MonMap *map);
   ~Monitor() override;
 
@@ -978,7 +1011,7 @@ private:
 
   void handle_signal(int sig);
 
-  int mkfs(bufferlist& osdmapbl);
+  int mkfs(ceph::buffer::list& osdmapbl);
 
   /**
    * check cluster_fsid file
@@ -996,7 +1029,7 @@ private:
   int write_fsid(MonitorDBStore::TransactionRef t);
 
   int do_admin_command(std::string_view command, const cmdmap_t& cmdmap,
-                      Formatter *f,
+                      ceph::Formatter *f,
                       std::ostream& err,
                       std::ostream& out);
 
@@ -1007,9 +1040,9 @@ private:
 
 public:
   static void format_command_descriptions(const std::vector<MonCommand> &commands,
-                                         Formatter *f,
+                                         ceph::Formatter *f,
                                          uint64_t features,
-                                         bufferlist *rdata);
+                                         ceph::buffer::list *rdata);
 
   const std::vector<MonCommand> &get_local_commands(mon_feature_t f) {
     if (f.contains_all(ceph::features::mon::FEATURE_NAUTILUS)) {
@@ -1018,7 +1051,7 @@ public:
       return prenautilus_local_mon_commands;
     }
   }
-  const bufferlist& get_local_commands_bl(mon_feature_t f) {
+  const ceph::buffer::list& get_local_commands_bl(mon_feature_t f) {
     if (f.contains_all(ceph::features::mon::FEATURE_NAUTILUS)) {
       return local_mon_commands_bl;
     } else {
@@ -1044,6 +1077,7 @@ public:
 #define CEPH_MON_FEATURE_INCOMPAT_MIMIC CompatSet::Feature(10, "mimic ondisk layout")
 #define CEPH_MON_FEATURE_INCOMPAT_NAUTILUS CompatSet::Feature(11, "nautilus ondisk layout")
 #define CEPH_MON_FEATURE_INCOMPAT_OCTOPUS CompatSet::Feature(12, "octopus ondisk layout")
+#define CEPH_MON_FEATURE_INCOMPAT_PACIFIC CompatSet::Feature(13, "pacific ondisk layout")
 // make sure you add your feature to Monitor::get_supported_features