]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/mon/Monitor.h
update sources to 12.2.7
[ceph.git] / ceph / src / mon / Monitor.h
index 3f8b5950888e0d127191afd5cf58fee33a22b35b..008947e85be34d2203421f8d12f80e9bc00fd2cd 100644 (file)
 #include <cmath>
 
 #include "include/types.h"
+#include "include/health.h"
 #include "msg/Messenger.h"
 
 #include "common/Timer.h"
 
+#include "health_check.h"
 #include "MonMap.h"
 #include "Elector.h"
 #include "Paxos.h"
 #include "Session.h"
+#include "PGStatService.h"
+#include "MonCommand.h"
 
 #include "common/LogClient.h"
 #include "auth/cephx/CephxKeyServer.h"
@@ -110,7 +114,6 @@ struct MRoute;
 struct MForward;
 struct MTimeCheck;
 struct MMonHealth;
-struct MonCommand;
 
 #define COMPAT_SET_LOC "feature_set"
 
@@ -162,13 +165,20 @@ public:
 
   CompatSet features;
 
-  const MonCommand *leader_supported_mon_commands;
-  int leader_supported_mon_commands_size;
+  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
+
+  // for upgrading mon cluster that still uses PGMonitor
+  vector<MonCommand> local_upgrading_mon_commands;  // mixed mon cluster commands
+  bufferlist local_upgrading_mon_commands_bl;       // encoded version of above
 
   Messenger *mgr_messenger;
   MgrClient mgr_client;
   uint64_t mgr_proxy_bytes = 0;  // in-flight proxied mgr command message bytes
 
+  const MonPGStatService *pgservice;
+
 private:
   void new_tick();
 
@@ -230,6 +240,11 @@ private:
   set<int> quorum;       // current active set of monitors (if !starting)
   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;
+
   /**
    * Intersection of quorum member's connection feature bits.
    */
@@ -238,7 +253,6 @@ private:
    * Intersection of quorum members mon-specific feature bits
    */
   mon_feature_t quorum_mon_features;
-  bufferlist supported_commands_bl; // encoded MonCommands we support
 
   set<string> outside_quorum;
 
@@ -488,6 +502,7 @@ private:
   version_t timecheck_round;
   unsigned int timecheck_acks;
   utime_t timecheck_round_start;
+  friend class HealthMonitor;
   /* When we hit a skew we will start a new round based off of
    * 'mon_timecheck_skew_interval'. Each new round will be backed off
    * until we hit 'mon_timecheck_interval' -- which is the typical
@@ -548,6 +563,9 @@ 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());
+  }
   const set<int>& get_quorum() const { return quorum; }
   list<string> get_quorum_names() {
     list<string> q;
@@ -571,6 +589,8 @@ public:
   void apply_monmap_to_compatset_features();
   void calc_quorum_requirements();
 
+  void get_combined_feature_map(FeatureMap *fm);
+
 private:
   void _reset();   ///< called from bootstrap, start_, or join_election
   void wait_for_paxos_write();
@@ -584,17 +604,13 @@ public:
   void win_election(epoch_t epoch, set<int>& q,
                    uint64_t features,
                     const mon_feature_t& mon_features,
-                   const MonCommand *cmdset, int cmdsize);
+                   const map<int,Metadata>& metadata);
   void lose_election(epoch_t epoch, set<int>& q, int l,
                     uint64_t features,
                      const mon_feature_t& mon_features);
   // end election (called by Elector)
   void finish_election();
 
-  const bufferlist& get_supported_commands_bl() {
-    return supported_commands_bl;
-  }
-
   void update_logger();
 
   /**
@@ -602,8 +618,6 @@ public:
    */
   vector<PaxosService*> paxos_service;
 
-  PaxosService *get_paxos_service_by_name(const string& name);
-
   class PGMonitor *pgmon() {
     return (class PGMonitor *)paxos_service[PAXOS_PGMAP];
   }
@@ -632,6 +646,14 @@ public:
     return (class MgrMonitor*) paxos_service[PAXOS_MGR];
   }
 
+  class MgrStatMonitor *mgrstatmon() {
+    return (class MgrStatMonitor*) paxos_service[PAXOS_MGRSTAT];
+  }
+
+  class HealthMonitor *healthmon() {
+    return (class HealthMonitor*) paxos_service[PAXOS_HEALTH];
+  }
+
   friend class Paxos;
   friend class OSDMonitor;
   friend class MDSMonitor;
@@ -662,8 +684,9 @@ public:
 
   static void _generate_command_map(map<string,cmd_vartype>& cmdmap,
                                     map<string,string> &param_str_map);
-  static const MonCommand *_get_moncommand(const string &cmd_prefix,
-                                           MonCommand *cmds, int cmds_size);
+  static const MonCommand *_get_moncommand(
+    const string &cmd_prefix,
+    const vector<MonCommand>& cmds);
   bool _allowed_command(MonSession *s, string &module, string &prefix,
                         const map<string,cmd_vartype>& cmdmap,
                         const map<string,string>& param_str_map,
@@ -679,6 +702,7 @@ public:
   int print_nodes(Formatter *f, ostream& err);
 
   // Accumulate metadata across calls to update_mon_metadata
+  map<int, Metadata> mon_metadata;
   map<int, Metadata> pending_metadata;
 
   /**
@@ -720,6 +744,37 @@ public:
    */
   health_status_t get_health(list<string>& status, bufferlist *detailbl,
                              Formatter *f);
+
+  health_status_t get_health_status(
+    bool want_detail,
+    Formatter *f,
+    std::string *plain,
+    const char *sep1 = " ",
+    const char *sep2 = "; ");
+  void log_health(
+    const health_check_map_t& updated,
+    const health_check_map_t& previous,
+    MonitorDBStore::TransactionRef t);
+
+protected:
+
+  class HealthCheckLogStatus {
+    public:
+    health_status_t severity;
+    std::string last_message;
+    utime_t updated_at = 0;
+    HealthCheckLogStatus(health_status_t severity_,
+                         const std::string &last_message_,
+                         utime_t updated_at_)
+      : severity(severity_),
+        last_message(last_message_),
+        updated_at(updated_at_)
+    {}
+  };
+  std::map<std::string, HealthCheckLogStatus> health_check_log_times;
+
+public:
+
   void get_cluster_status(stringstream &ss, Formatter *f);
 
   void reply_command(MonOpRequestRef op, int rc, const string &rs, version_t version);
@@ -851,7 +906,8 @@ public:
   bool ms_get_authorizer(int dest_type, AuthAuthorizer **authorizer, bool force_new) override;
   bool ms_verify_authorizer(Connection *con, int peer_type,
                            int protocol, bufferlist& authorizer_data, bufferlist& authorizer_reply,
-                           bool& isvalid, CryptoKey& session_key) override;
+                           bool& isvalid, CryptoKey& session_key,
+                           std::unique_ptr<AuthAuthorizerChallenge> *challenge) override;
   bool ms_handle_reset(Connection *con) override;
   void ms_handle_remote_reset(Connection *con) override {}
   bool ms_handle_refused(Connection *con) override;
@@ -859,8 +915,11 @@ public:
   int write_default_keyring(bufferlist& bl);
   void extract_save_mon_key(KeyRing& keyring);
 
+  void collect_metadata(Metadata *m);
   void update_mon_metadata(int from, Metadata&& m);
-  int load_metadata(map<int, Metadata>& m);
+  int load_metadata();
+  void count_metadata(const string& field, Formatter *f);
+  void count_metadata(const string& field, map<string,int> *out);
 
   // features
   static CompatSet get_initial_supported_features();
@@ -922,14 +981,27 @@ private:
   Monitor& operator=(const Monitor &rhs);
 
 public:
-  static void format_command_descriptions(const MonCommand *commands,
-                                         unsigned commands_size,
+  static void format_command_descriptions(const std::vector<MonCommand> &commands,
                                          Formatter *f,
                                          bufferlist *rdata,
                                          bool hide_mgr_flag=false);
-  void get_locally_supported_monitor_commands(const MonCommand **cmds, int *count);
-  /// the Monitor owns this pointer once you pass it in
-  void set_leader_supported_commands(const MonCommand *cmds, int size);
+
+  const std::vector<MonCommand> &get_local_commands(mon_feature_t f) {
+    if (f.contains_all(ceph::features::mon::FEATURE_LUMINOUS))
+      return local_mon_commands;
+    else
+      return local_upgrading_mon_commands;
+  }
+  const bufferlist& get_local_commands_bl(mon_feature_t f) {
+    if (f.contains_all(ceph::features::mon::FEATURE_LUMINOUS))
+      return local_mon_commands_bl;
+    else
+      return local_upgrading_mon_commands_bl;
+  }
+  void set_leader_commands(const std::vector<MonCommand>& cmds) {
+    leader_mon_commands = cmds;
+  }
+
   static bool is_keyring_required();
 };
 
@@ -941,101 +1013,9 @@ public:
 #define CEPH_MON_FEATURE_INCOMPAT_ERASURE_CODE_PLUGINS_V2 CompatSet::Feature(6, "support isa/lrc erasure code")
 #define CEPH_MON_FEATURE_INCOMPAT_ERASURE_CODE_PLUGINS_V3 CompatSet::Feature(7, "support shec erasure code")
 #define CEPH_MON_FEATURE_INCOMPAT_KRAKEN CompatSet::Feature(8, "support monmap features")
+#define CEPH_MON_FEATURE_INCOMPAT_LUMINOUS CompatSet::Feature(9, "luminous ondisk layout")
 // make sure you add your feature to Monitor::get_supported_features
 
-long parse_pos_long(const char *s, ostream *pss = NULL);
-
-struct MonCommand {
-  string cmdstring;
-  string helpstring;
-  string module;
-  string req_perms;
-  string availability;
-  uint64_t flags;
-
-  // MonCommand flags
-  static const uint64_t FLAG_NONE       = 0;
-  static const uint64_t FLAG_NOFORWARD  = 1 << 0;
-  static const uint64_t FLAG_OBSOLETE   = 1 << 1;
-  static const uint64_t FLAG_DEPRECATED = 1 << 2;
-  static const uint64_t FLAG_MGR        = 1 << 3;
-
-  bool has_flag(uint64_t flag) const { return (flags & flag) != 0; }
-  void set_flag(uint64_t flag) { flags |= flag; }
-  void unset_flag(uint64_t flag) { flags &= ~flag; }
-
-  void encode(bufferlist &bl) const {
-    /*
-     * very naughty: deliberately unversioned because individual commands
-     * shouldn't be encoded standalone, only as a full set (which we do
-     * version, see encode_array() below).
-     */
-    ::encode(cmdstring, bl);
-    ::encode(helpstring, bl);
-    ::encode(module, bl);
-    ::encode(req_perms, bl);
-    ::encode(availability, bl);
-  }
-  void decode(bufferlist::iterator &bl) {
-    ::decode(cmdstring, bl);
-    ::decode(helpstring, bl);
-    ::decode(module, bl);
-    ::decode(req_perms, bl);
-    ::decode(availability, bl);
-  }
-  bool is_compat(const MonCommand* o) const {
-    return cmdstring == o->cmdstring &&
-       module == o->module && req_perms == o->req_perms &&
-       availability == o->availability;
-  }
-
-  bool is_noforward() const {
-    return has_flag(MonCommand::FLAG_NOFORWARD);
-  }
-
-  bool is_obsolete() const {
-    return has_flag(MonCommand::FLAG_OBSOLETE);
-  }
-
-  bool is_deprecated() const {
-    return has_flag(MonCommand::FLAG_DEPRECATED);
-  }
-
-  bool is_mgr() const {
-    return has_flag(MonCommand::FLAG_MGR);
-  }
-
-  static void encode_array(const MonCommand *cmds, int size, bufferlist &bl) {
-    ENCODE_START(2, 1, bl);
-    uint16_t s = size;
-    ::encode(s, bl);
-    ::encode_array_nohead(cmds, size, bl);
-    for (int i = 0; i < size; i++)
-      ::encode(cmds[i].flags, bl);
-    ENCODE_FINISH(bl);
-  }
-  static void decode_array(MonCommand **cmds, int *size,
-                           bufferlist::iterator &bl) {
-    DECODE_START(2, bl);
-    uint16_t s = 0;
-    ::decode(s, bl);
-    *size = s;
-    *cmds = new MonCommand[*size];
-    ::decode_array_nohead(*cmds, *size, bl);
-    if (struct_v >= 2) {
-      for (int i = 0; i < *size; i++)
-       ::decode((*cmds)[i].flags, bl);
-    } else {
-      for (int i = 0; i < *size; i++)
-       (*cmds)[i].flags = 0;
-    }
-    DECODE_FINISH(bl);
-  }
 
-  bool requires_perm(char p) const {
-    return (req_perms.find(p) != string::npos); 
-  }
-};
-WRITE_CLASS_ENCODER(MonCommand)
 
 #endif