]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/mon/Monitor.h
update sources to ceph Nautilus 14.2.1
[ceph.git] / ceph / src / mon / Monitor.h
index 008947e85be34d2203421f8d12f80e9bc00fd2cd..c57c2ec4936158e3fc23531d711361a0abffbd00 100644 (file)
@@ -25,6 +25,7 @@
 
 #include <errno.h>
 #include <cmath>
+#include <string>
 
 #include "include/types.h"
 #include "include/health.h"
 #include "Elector.h"
 #include "Paxos.h"
 #include "Session.h"
-#include "PGStatService.h"
 #include "MonCommand.h"
 
+
+#include "common/config_obs.h"
 #include "common/LogClient.h"
+#include "auth/AuthClient.h"
+#include "auth/AuthServer.h"
 #include "auth/cephx/CephxKeyServer.h"
 #include "auth/AuthMethodList.h"
 #include "auth/KeyRing.h"
 #include "messages/MMonCommand.h"
 #include "mon/MonitorDBStore.h"
-#include "include/memory.h"
 #include "mgr/MgrClient.h"
 
 #include "mon/MonOpRequest.h"
@@ -77,10 +80,6 @@ enum {
   l_cluster_num_object_misplaced,
   l_cluster_num_object_unfound,
   l_cluster_num_bytes,
-  l_cluster_num_mds_up,
-  l_cluster_num_mds_in,
-  l_cluster_num_mds_failed,
-  l_cluster_mds_epoch,
   l_cluster_last,
 };
 
@@ -103,18 +102,6 @@ class PaxosService;
 class PerfCounters;
 class AdminSocketHook;
 
-class MMonGetMap;
-class MMonGetVersion;
-class MMonMetadata;
-class MMonSync;
-class MMonScrub;
-class MMonProbe;
-struct MMonSubscribe;
-struct MRoute;
-struct MForward;
-struct MTimeCheck;
-struct MMonHealth;
-
 #define COMPAT_SET_LOC "feature_set"
 
 class C_MonContext final : public FunctionContext {
@@ -126,8 +113,13 @@ public:
 };
 
 class Monitor : public Dispatcher,
+               public AuthClient,
+               public AuthServer,
                 public md_config_obs_t {
 public:
+  int orig_argc = 0;
+  const char **orig_argv = nullptr;
+
   // me
   string name;
   int rank;
@@ -137,7 +129,9 @@ public:
   SafeTimer timer;
   Finisher finisher;
   ThreadPool cpu_tp;  ///< threadpool for CPU intensive work
-  
+
+  ceph::mutex auth_lock = ceph::make_mutex("Monitor::auth_lock");
+
   /// true if we have ever joined a quorum.  if false, we are either a
   /// new cluster, a newly joining monitor, or a just-upgraded
   /// monitor.
@@ -152,7 +146,7 @@ public:
   MonMap *monmap;
   uuid_d fingerprint;
 
-  set<entity_addr_t> extra_probe_peers;
+  set<entity_addrvec_t> extra_probe_peers;
 
   LogClient log_client;
   LogChannelRef clog;
@@ -169,15 +163,13 @@ public:
   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
+  vector<MonCommand> prenautilus_local_mon_commands;
+  bufferlist prenautilus_local_mon_commands_bl;
 
   Messenger *mgr_messenger;
   MgrClient mgr_client;
   uint64_t mgr_proxy_bytes = 0;  // in-flight proxied mgr command message bytes
-
-  const MonPGStatService *pgservice;
+  std::string gss_ktfile_client{};
 
 private:
   void new_tick();
@@ -191,14 +183,15 @@ public:
   // -- monitor state --
 private:
   enum {
-    STATE_PROBING = 1,
+    STATE_INIT = 1,
+    STATE_PROBING,
     STATE_SYNCHRONIZING,
     STATE_ELECTING,
     STATE_LEADER,
     STATE_PEON,
     STATE_SHUTDOWN
   };
-  int state;
+  int state = STATE_INIT;
 
 public:
   static const char *get_state_name(int s) {
@@ -216,6 +209,7 @@ public:
     return get_state_name(state);
   }
 
+  bool is_init() const { return state == STATE_INIT; }
   bool is_shutdown() const { return state == STATE_SHUTDOWN; }
   bool is_probing() const { return state == STATE_PROBING; }
   bool is_synchronizing() const { return state == STATE_SYNCHRONIZING; }
@@ -227,6 +221,8 @@ public:
 
   void prepare_new_fingerprint(MonitorDBStore::TransactionRef t);
 
+  std::vector<DaemonHealthMetric> get_health_metrics();
+
   // -- elector --
 private:
   Paxos *paxos;
@@ -238,6 +234,7 @@ private:
   
   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
   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
 
@@ -254,6 +251,8 @@ private:
    */
   mon_feature_t quorum_mon_features;
 
+  int quorum_min_mon_release = -1;
+
   set<string> outside_quorum;
 
   /**
@@ -294,7 +293,7 @@ private:
     ScrubState() : finished(false) { }
     virtual ~ScrubState() { }
   };
-  ceph::shared_ptr<ScrubState> scrub_state; ///< keeps track of current scrub
+  std::shared_ptr<ScrubState> scrub_state; ///< keeps track of current scrub
 
   /**
    * @defgroup Monitor_h_sync Synchronization
@@ -304,7 +303,7 @@ private:
    * @} // provider state
    */
   struct SyncProvider {
-    entity_inst_t entity;  ///< who
+    entity_addrvec_t addrs;
     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
@@ -326,7 +325,7 @@ private:
   /**
    * @} // requester state
    */
-  entity_inst_t sync_provider;   ///< who we are syncing from
+  entity_addrvec_t sync_provider;  ///< who we are syncing from
   uint64_t sync_cookie;          ///< 0 if we are starting, non-zero otherwise
   bool sync_full;                ///< true if we are a full sync, false for recent catch-up
   version_t sync_start_version;  ///< last_committed at sync start
@@ -400,7 +399,7 @@ private:
    * @param entity where to pull committed state from
    * @param full whether to do a full sync or just catch up on recent paxos
    */
-  void sync_start(entity_inst_t &entity, bool full);
+  void sync_start(entity_addrvec_t &addrs, bool full);
 
 public:
   /**
@@ -494,9 +493,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<entity_inst_t, utime_t> timecheck_waiting;
-  map<entity_inst_t, double> timecheck_skews;
-  map<entity_inst_t, double> timecheck_latencies;
+  map<int, utime_t> timecheck_waiting;
+  map<int, double> timecheck_skews;
+  map<int, double> timecheck_latencies;
   // odd value means we are mid-round; even value means the round has
   // finished.
   version_t timecheck_round;
@@ -541,7 +540,7 @@ private:
     double abs_skew = std::fabs(skew_bound);
     if (abs)
       *abs = abs_skew;
-    return (abs_skew > g_conf->mon_clock_drift_allowed);
+    return (abs_skew > g_conf()->mon_clock_drift_allowed);
   }
 
   /**
@@ -595,6 +594,7 @@ private:
   void _reset();   ///< called from bootstrap, start_, or join_election
   void wait_for_paxos_write();
   void _finish_svc_election(); ///< called by {win,lose}_election
+  void respawn();
 public:
   void bootstrap();
   void join_election();
@@ -604,10 +604,12 @@ public:
   void win_election(epoch_t epoch, set<int>& q,
                    uint64_t features,
                     const mon_feature_t& mon_features,
+                   int min_mon_release,
                    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);
+                     const mon_feature_t& mon_features,
+                    int min_mon_release);
   // end election (called by Elector)
   void finish_election();
 
@@ -616,53 +618,51 @@ public:
   /**
    * Vector holding the Services serviced by this Monitor.
    */
-  vector<PaxosService*> paxos_service;
-
-  class PGMonitor *pgmon() {
-    return (class PGMonitor *)paxos_service[PAXOS_PGMAP];
-  }
+  vector<std::unique_ptr<PaxosService>> paxos_service;
 
   class MDSMonitor *mdsmon() {
-    return (class MDSMonitor *)paxos_service[PAXOS_MDSMAP];
+    return (class MDSMonitor *)paxos_service[PAXOS_MDSMAP].get();
   }
 
   class MonmapMonitor *monmon() {
-    return (class MonmapMonitor *)paxos_service[PAXOS_MONMAP];
+    return (class MonmapMonitor *)paxos_service[PAXOS_MONMAP].get();
   }
 
   class OSDMonitor *osdmon() {
-    return (class OSDMonitor *)paxos_service[PAXOS_OSDMAP];
+    return (class OSDMonitor *)paxos_service[PAXOS_OSDMAP].get();
   }
 
   class AuthMonitor *authmon() {
-    return (class AuthMonitor *)paxos_service[PAXOS_AUTH];
+    return (class AuthMonitor *)paxos_service[PAXOS_AUTH].get();
   }
 
   class LogMonitor *logmon() {
-    return (class LogMonitor*) paxos_service[PAXOS_LOG];
+    return (class LogMonitor*) paxos_service[PAXOS_LOG].get();
   }
 
   class MgrMonitor *mgrmon() {
-    return (class MgrMonitor*) paxos_service[PAXOS_MGR];
+    return (class MgrMonitor*) paxos_service[PAXOS_MGR].get();
   }
 
   class MgrStatMonitor *mgrstatmon() {
-    return (class MgrStatMonitor*) paxos_service[PAXOS_MGRSTAT];
+    return (class MgrStatMonitor*) paxos_service[PAXOS_MGRSTAT].get();
   }
 
   class HealthMonitor *healthmon() {
-    return (class HealthMonitor*) paxos_service[PAXOS_HEALTH];
+    return (class HealthMonitor*) paxos_service[PAXOS_HEALTH].get();
+  }
+
+  class ConfigMonitor *configmon() {
+    return (class ConfigMonitor*) paxos_service[PAXOS_CONFIG].get();
   }
 
   friend class Paxos;
   friend class OSDMonitor;
   friend class MDSMonitor;
   friend class MonmapMonitor;
-  friend class PGMonitor;
   friend class LogMonitor;
   friend class ConfigKeyService;
 
-  QuorumService *health_monitor;
   QuorumService *config_key_service;
 
   // -- sessions --
@@ -672,7 +672,7 @@ public:
 
   template<typename Func, typename...Args>
   void with_session_map(Func&& func) {
-    Mutex::Locker l(session_map_lock);
+    std::lock_guard l(session_map_lock);
     std::forward<Func>(func)(session_map);
   }
   void send_latest_monmap(Connection *con);
@@ -682,18 +682,20 @@ public:
   void handle_subscribe(MonOpRequestRef op);
   void handle_mon_get_map(MonOpRequestRef op);
 
-  static void _generate_command_map(map<string,cmd_vartype>& cmdmap,
+  static void _generate_command_map(cmdmap_t& cmdmap,
                                     map<string,string> &param_str_map);
   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,
+  bool _allowed_command(MonSession *s, const string& module,
+                       const string& prefix,
+                        const cmdmap_t& cmdmap,
                         const map<string,string>& param_str_map,
                         const MonCommand *this_cmd);
   void get_mon_status(Formatter *f, ostream& ss);
   void _quorum_status(Formatter *f, ostream& ss);
-  bool _add_bootstrap_peer_hint(string cmd, cmdmap_t& cmdmap, ostream& ss);
+  bool _add_bootstrap_peer_hint(std::string_view cmd, const cmdmap_t& cmdmap,
+                               std::ostream& ss);
   void handle_command(MonOpRequestRef op);
   void handle_route(MonOpRequestRef op);
 
@@ -735,16 +737,6 @@ public:
   void do_health_to_clog_interval();
   void do_health_to_clog(bool force = false);
 
-  /**
-   * Generate health report
-   *
-   * @param status one-line status summary
-   * @param detailbl optional bufferlist* to fill with a detailed report
-   * @returns health status
-   */
-  health_status_t get_health(list<string>& status, bufferlist *detailbl,
-                             Formatter *f);
-
   health_status_t get_health_status(
     bool want_detail,
     Formatter *f,
@@ -806,7 +798,6 @@ public:
     MonSession *session;
     ConnectionRef con;
     uint64_t con_features;
-    entity_inst_t client_inst;
     MonOpRequestRef op;
 
     RoutedRequest() : tid(0), session(NULL), con_features(0) {}
@@ -820,7 +811,6 @@ public:
   
   void forward_request_leader(MonOpRequestRef op);
   void handle_forward(MonOpRequestRef op);
-  void try_send_message(Message *m, const entity_inst_t& to);
   void send_reply(MonOpRequestRef op, Message *reply);
   void no_reply(MonOpRequestRef op);
   void resend_routed_requests();
@@ -828,8 +818,7 @@ public:
   void remove_all_sessions();
   void waitlist_or_zap_client(MonOpRequestRef op);
 
-  void send_command(const entity_inst_t& inst,
-                   const vector<string>& com);
+  void send_mon_message(Message *m, int rank);
 
 public:
   struct C_Command : public C_MonOp {
@@ -854,7 +843,7 @@ public:
 
           // if client drops we may not have a session to draw information from.
           if (s) {
-            ss << "from='" << s->inst << "' "
+            ss << "from='" << s->name << " " << s->addrs << "' "
               << "entity='" << s->entity_name << "' ";
           } else {
             ss << "session dropped for command ";
@@ -870,7 +859,7 @@ public:
       else if (r == -EAGAIN)
        mon->dispatch_op(op);
       else
-       assert(0 == "bad C_Command return value");
+       ceph_abort_msg("bad C_Command return value");
     }
   };
 
@@ -887,7 +876,7 @@ public:
       else if (r == -ECANCELED)
         return;
       else
-       assert(0 == "bad C_RetryMessage return value");
+       ceph_abort_msg("bad C_RetryMessage return value");
     }
   };
 
@@ -902,16 +891,55 @@ public:
   }
   void dispatch_op(MonOpRequestRef op);
   //mon_caps is used for un-connected messages from monitors
-  MonCap * mon_caps;
-  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,
-                           std::unique_ptr<AuthAuthorizerChallenge> *challenge) override;
+  MonCap mon_caps;
+  bool ms_get_authorizer(int dest_type, AuthAuthorizer **authorizer) override;
+  KeyStore *ms_get_auth1_authorizer_keystore();
+public: // for AuthMonitor msgr1:
+  int ms_handle_authentication(Connection *con) override;
+private:
+  void ms_handle_accept(Connection *con) override;
   bool ms_handle_reset(Connection *con) override;
   void ms_handle_remote_reset(Connection *con) override {}
   bool ms_handle_refused(Connection *con) override;
 
+  // AuthClient
+  int get_auth_request(
+    Connection *con,
+    AuthConnectionMeta *auth_meta,
+    uint32_t *method,
+    vector<uint32_t> *preferred_modes,
+    bufferlist *out) override;
+  int handle_auth_reply_more(
+    Connection *con,
+    AuthConnectionMeta *auth_meta,
+   const bufferlist& bl,
+    bufferlist *reply) override;
+  int handle_auth_done(
+    Connection *con,
+    AuthConnectionMeta *auth_meta,
+    uint64_t global_id,
+    uint32_t con_mode,
+    const bufferlist& bl,
+    CryptoKey *session_key,
+    std::string *connection_secret) override;
+  int handle_auth_bad_method(
+    Connection *con,
+    AuthConnectionMeta *auth_meta,
+    uint32_t old_auth_method,
+    int result,
+    const std::vector<uint32_t>& allowed_methods,
+    const std::vector<uint32_t>& allowed_modes) override;
+  // /AuthClient
+  // AuthServer
+  int handle_auth_request(
+    Connection *con,
+    AuthConnectionMeta *auth_meta,
+    bool more,
+    uint32_t auth_method,
+    const bufferlist& bl,
+    bufferlist *reply) override;
+  // /AuthServer
+
   int write_default_keyring(bufferlist& bl);
   void extract_save_mon_key(KeyRing& keyring);
 
@@ -941,7 +969,7 @@ public:
 
   // config observer
   const char** get_tracked_conf_keys() const override;
-  void handle_conf_change(const struct md_config_t *conf,
+  void handle_conf_change(const ConfigProxy& conf,
                           const std::set<std::string> &changed) override;
 
   void update_log_clients();
@@ -972,8 +1000,8 @@ public:
   int write_fsid();
   int write_fsid(MonitorDBStore::TransactionRef t);
 
-  void do_admin_command(std::string command, cmdmap_t& cmdmap,
-                       std::string format, ostream& ss);
+  void do_admin_command(std::string_view command, const cmdmap_t& cmdmap,
+                       std::string_view format, std::ostream& ss);
 
 private:
   // don't allow copying
@@ -983,26 +1011,28 @@ private:
 public:
   static void format_command_descriptions(const std::vector<MonCommand> &commands,
                                          Formatter *f,
-                                         bufferlist *rdata,
-                                         bool hide_mgr_flag=false);
+                                         uint64_t features,
+                                         bufferlist *rdata);
 
   const std::vector<MonCommand> &get_local_commands(mon_feature_t f) {
-    if (f.contains_all(ceph::features::mon::FEATURE_LUMINOUS))
+    if (f.contains_all(ceph::features::mon::FEATURE_NAUTILUS)) {
       return local_mon_commands;
-    else
-      return local_upgrading_mon_commands;
+    } else {
+      return prenautilus_local_mon_commands;
+    }
   }
   const bufferlist& get_local_commands_bl(mon_feature_t f) {
-    if (f.contains_all(ceph::features::mon::FEATURE_LUMINOUS))
+    if (f.contains_all(ceph::features::mon::FEATURE_NAUTILUS)) {
       return local_mon_commands_bl;
-    else
-      return local_upgrading_mon_commands_bl;
+    } else {
+      return prenautilus_local_mon_commands_bl;
+    }
   }
   void set_leader_commands(const std::vector<MonCommand>& cmds) {
     leader_mon_commands = cmds;
   }
 
-  static bool is_keyring_required();
+  bool is_keyring_required();
 };
 
 #define CEPH_MON_FEATURE_INCOMPAT_BASE CompatSet::Feature (1, "initial feature set (~v.18)")
@@ -1014,6 +1044,8 @@ public:
 #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")
+#define CEPH_MON_FEATURE_INCOMPAT_MIMIC CompatSet::Feature(10, "mimic ondisk layout")
+#define CEPH_MON_FEATURE_INCOMPAT_NAUTILUS CompatSet::Feature(11, "nautilus ondisk layout")
 // make sure you add your feature to Monitor::get_supported_features