]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/mon/Monitor.cc
update sources to 12.2.2
[ceph.git] / ceph / src / mon / Monitor.cc
index 9421b4cbf14fc55619a9568a7de757979ab532bd..af3683f67e1e7a540a9c3abc76a1014c6baaf993 100644 (file)
@@ -103,28 +103,20 @@ const string Monitor::MONITOR_STORE_PREFIX = "monitor_store";
 #undef FLAG
 #undef COMMAND
 #undef COMMAND_WITH_FLAG
-MonCommand mon_commands[] = {
 #define FLAG(f) (MonCommand::FLAG_##f)
 #define COMMAND(parsesig, helptext, modulename, req_perms, avail)      \
   {parsesig, helptext, modulename, req_perms, avail, FLAG(NONE)},
 #define COMMAND_WITH_FLAG(parsesig, helptext, modulename, req_perms, avail, flags) \
   {parsesig, helptext, modulename, req_perms, avail, flags},
+MonCommand mon_commands[] = {
 #include <mon/MonCommands.h>
+};
+MonCommand pgmonitor_commands[] = {
+#include <mon/PGMonitorCommands.h>
+};
 #undef COMMAND
 #undef COMMAND_WITH_FLAG
 
-  // FIXME: slurp up the Mgr commands too
-
-#define COMMAND(parsesig, helptext, modulename, req_perms, avail)      \
-  {parsesig, helptext, modulename, req_perms, avail, FLAG(MGR)},
-#define COMMAND_WITH_FLAG(parsesig, helptext, modulename, req_perms, avail, flags) \
-  {parsesig, helptext, modulename, req_perms, avail, flags | FLAG(MGR)},
-#include <mgr/MgrCommands.h>
-#undef COMMAND
-#undef COMMAND_WITH_FLAG
-
-};
-
 
 void C_MonContext::finish(int r) {
   if (mon->is_shutdown())
@@ -154,8 +146,6 @@ Monitor::Monitor(CephContext* cct_, string nm, MonitorDBStore *s,
   auth_service_required(cct,
                        cct->_conf->auth_supported.empty() ?
                        cct->_conf->auth_service_required : cct->_conf->auth_supported ),
-  leader_supported_mon_commands(NULL),
-  leader_supported_mon_commands_size(0),
   mgr_messenger(mgr_m),
   mgr_client(cct_, mgr_m),
   pgservice(nullptr),
@@ -216,14 +206,25 @@ Monitor::Monitor(CephContext* cct_, string nm, MonitorDBStore *s,
 
   exited_quorum = ceph_clock_now();
 
+  // prepare local commands
+  local_mon_commands.resize(ARRAY_SIZE(mon_commands));
+  for (unsigned i = 0; i < ARRAY_SIZE(mon_commands); ++i) {
+    local_mon_commands[i] = mon_commands[i];
+  }
+  MonCommand::encode_vector(local_mon_commands, local_mon_commands_bl);
+
+  local_upgrading_mon_commands = local_mon_commands;
+  for (unsigned i = 0; i < ARRAY_SIZE(pgmonitor_commands); ++i) {
+    local_upgrading_mon_commands.push_back(pgmonitor_commands[i]);
+  }
+  MonCommand::encode_vector(local_upgrading_mon_commands,
+                           local_upgrading_mon_commands_bl);
+
   // assume our commands until we have an election.  this only means
   // we won't reply with EINVAL before the election; any command that
   // actually matters will wait until we have quorum etc and then
   // retry (and revalidate).
-  const MonCommand *cmds;
-  int cmdsize;
-  get_locally_supported_monitor_commands(&cmds, &cmdsize);
-  set_leader_supported_commands(cmds, cmdsize);
+  leader_mon_commands = local_mon_commands;
 
   // note: OSDMonitor may update this based on the luminous flag.
   pgservice = mgrstatmon()->get_pg_stat_service();
@@ -238,8 +239,6 @@ Monitor::~Monitor()
   delete paxos;
   assert(session_map.sessions.empty());
   delete mon_caps;
-  if (leader_supported_mon_commands != mon_commands)
-    delete[] leader_supported_mon_commands;
 }
 
 
@@ -371,6 +370,7 @@ CompatSet Monitor::get_supported_features()
   compat.incompat.insert(CEPH_MON_FEATURE_INCOMPAT_ERASURE_CODE_PLUGINS_V2);
   compat.incompat.insert(CEPH_MON_FEATURE_INCOMPAT_ERASURE_CODE_PLUGINS_V3);
   compat.incompat.insert(CEPH_MON_FEATURE_INCOMPAT_KRAKEN);
+  compat.incompat.insert(CEPH_MON_FEATURE_INCOMPAT_LUMINOUS);
   return compat;
 }
 
@@ -571,14 +571,22 @@ int Monitor::preinit()
   assert(!logger);
   {
     PerfCountersBuilder pcb(g_ceph_context, "mon", l_mon_first, l_mon_last);
-    pcb.add_u64(l_mon_num_sessions, "num_sessions", "Open sessions", "sess");
-    pcb.add_u64_counter(l_mon_session_add, "session_add", "Created sessions", "sadd");
-    pcb.add_u64_counter(l_mon_session_rm, "session_rm", "Removed sessions", "srm");
-    pcb.add_u64_counter(l_mon_session_trim, "session_trim", "Trimmed sessions");
-    pcb.add_u64_counter(l_mon_num_elections, "num_elections", "Elections participated in");
-    pcb.add_u64_counter(l_mon_election_call, "election_call", "Elections started");
-    pcb.add_u64_counter(l_mon_election_win, "election_win", "Elections won");
-    pcb.add_u64_counter(l_mon_election_lose, "election_lose", "Elections lost");
+    pcb.add_u64(l_mon_num_sessions, "num_sessions", "Open sessions", "sess",
+        PerfCountersBuilder::PRIO_USEFUL);
+    pcb.add_u64_counter(l_mon_session_add, "session_add", "Created sessions",
+        "sadd", PerfCountersBuilder::PRIO_INTERESTING);
+    pcb.add_u64_counter(l_mon_session_rm, "session_rm", "Removed sessions",
+        "srm", PerfCountersBuilder::PRIO_INTERESTING);
+    pcb.add_u64_counter(l_mon_session_trim, "session_trim", "Trimmed sessions",
+        "strm", PerfCountersBuilder::PRIO_USEFUL);
+    pcb.add_u64_counter(l_mon_num_elections, "num_elections", "Elections participated in",
+        "ecnt", PerfCountersBuilder::PRIO_USEFUL);
+    pcb.add_u64_counter(l_mon_election_call, "election_call", "Elections started",
+        "estt", PerfCountersBuilder::PRIO_INTERESTING);
+    pcb.add_u64_counter(l_mon_election_win, "election_win", "Elections won",
+        "ewon", PerfCountersBuilder::PRIO_INTERESTING);
+    pcb.add_u64_counter(l_mon_election_lose, "election_lose", "Elections lost",
+        "elst", PerfCountersBuilder::PRIO_INTERESTING);
     logger = pcb.create_perf_counters();
     cct->get_perfcounters_collection()->add(logger);
   }
@@ -797,13 +805,8 @@ int Monitor::init()
   mgr_messenger->add_dispatcher_tail(this);  // for auth ms_* calls
 
   bootstrap();
-
-  // encode command sets
-  const MonCommand *cmds;
-  int cmdsize;
-  get_locally_supported_monitor_commands(&cmds, &cmdsize);
-  MonCommand::encode_array(cmds, cmdsize, supported_commands_bl);
-
+  // add features of myself into feature_map
+  session_map.feature_map.add_mon(con_self->get_features());
   return 0;
 }
 
@@ -1066,6 +1069,7 @@ void Monitor::_reset()
   cancel_probe_timeout();
   timecheck_finish();
   health_events_cleanup();
+  health_check_log_times.clear();
   scrub_event_cancel();
 
   leader_since = utime_t();
@@ -1246,10 +1250,11 @@ void Monitor::sync_reset_timeout()
   dout(10) << __func__ << dendl;
   if (sync_timeout_event)
     timer.cancel_event(sync_timeout_event);
-  sync_timeout_event = new C_MonContext(this, [this](int) {
-      sync_timeout();
-    });
-  timer.add_event_after(g_conf->mon_sync_timeout, sync_timeout_event);
+  sync_timeout_event = timer.add_event_after(
+    g_conf->mon_sync_timeout,
+    new C_MonContext(this, [this](int) {
+       sync_timeout();
+      }));
 }
 
 void Monitor::sync_finish(version_t last_committed)
@@ -1592,8 +1597,12 @@ void Monitor::reset_probe_timeout()
       probe_timeout(r);
     });
   double t = g_conf->mon_probe_timeout;
-  timer.add_event_after(t, probe_timeout_event);
-  dout(10) << "reset_probe_timeout " << probe_timeout_event << " after " << t << " seconds" << dendl;
+  if (timer.add_event_after(t, probe_timeout_event)) {
+    dout(10) << "reset_probe_timeout " << probe_timeout_event
+            << " after " << t << " seconds" << dendl;
+  } else {
+    probe_timeout_event = nullptr;
+  }
 }
 
 void Monitor::probe_timeout(int r)
@@ -1872,14 +1881,10 @@ void Monitor::win_standalone_election()
   map<int,Metadata> metadata;
   collect_metadata(&metadata[0]);
 
-  const MonCommand *my_cmds = nullptr;
-  int cmdsize = 0;
-  get_locally_supported_monitor_commands(&my_cmds, &cmdsize);
   win_election(elector.get_epoch(), q,
                CEPH_FEATURES_ALL,
                ceph::features::mon::get_supported(),
-              metadata,
-               my_cmds, cmdsize);
+              metadata);
 }
 
 const utime_t& Monitor::get_leader_since() const
@@ -1907,8 +1912,7 @@ void Monitor::_finish_svc_election()
 
 void Monitor::win_election(epoch_t epoch, set<int>& active, uint64_t features,
                            const mon_feature_t& mon_features,
-                          const map<int,Metadata>& metadata,
-                           const MonCommand *cmdset, int cmdsize)
+                          const map<int,Metadata>& metadata)
 {
   dout(10) << __func__ << " epoch " << epoch << " quorum " << active
           << " features " << features
@@ -1927,7 +1931,7 @@ void Monitor::win_election(epoch_t epoch, set<int>& active, uint64_t features,
   clog->info() << "mon." << name << "@" << rank
                << " won leader election with quorum " << quorum;
 
-  set_leader_supported_commands(cmdset, cmdsize);
+  set_leader_commands(get_local_commands(mon_features));
 
   paxos->leader_init();
   // NOTE: tell monmap monitor first.  This is important for the
@@ -2089,6 +2093,13 @@ void Monitor::apply_monmap_to_compatset_features()
     assert(HAVE_FEATURE(quorum_con_features, SERVER_KRAKEN));
     new_features.incompat.insert(CEPH_MON_FEATURE_INCOMPAT_KRAKEN);
   }
+  if (monmap_features.contains_all(ceph::features::mon::FEATURE_LUMINOUS)) {
+    assert(ceph::features::mon::get_persistent().contains_all(
+           ceph::features::mon::FEATURE_LUMINOUS));
+    // this feature should only ever be set if the quorum supports it.
+    assert(HAVE_FEATURE(quorum_con_features, SERVER_LUMINOUS));
+    new_features.incompat.insert(CEPH_MON_FEATURE_INCOMPAT_LUMINOUS);
+  }
 
   dout(5) << __func__ << dendl;
   _apply_compatset_features(new_features);
@@ -2114,6 +2125,9 @@ void Monitor::calc_quorum_requirements()
   if (features.incompat.contains(CEPH_MON_FEATURE_INCOMPAT_KRAKEN)) {
     required_features |= CEPH_FEATUREMASK_SERVER_KRAKEN;
   }
+  if (features.incompat.contains(CEPH_MON_FEATURE_INCOMPAT_LUMINOUS)) {
+    required_features |= CEPH_FEATUREMASK_SERVER_LUMINOUS;
+  }
 
   // monmap
   if (monmap->get_required_features().contains_all(
@@ -2290,14 +2304,14 @@ void Monitor::health_tick_start()
   dout(15) << __func__ << dendl;
 
   health_tick_stop();
-  health_tick_event = new C_MonContext(this, [this](int r) {
-      if (r < 0)
-        return;
-      do_health_to_clog();
-      health_tick_start();
-    });
-  timer.add_event_after(cct->_conf->mon_health_to_clog_tick_interval,
-                        health_tick_event);
+  health_tick_event = timer.add_event_after(
+    cct->_conf->mon_health_to_clog_tick_interval,
+    new C_MonContext(this, [this](int r) {
+       if (r < 0)
+         return;
+       do_health_to_clog();
+       health_tick_start();
+      }));
 }
 
 void Monitor::health_tick_stop()
@@ -2344,7 +2358,9 @@ void Monitor::health_interval_start()
         return;
       do_health_to_clog_interval();
     });
-  timer.add_event_at(next, health_interval_event);
+  if (!timer.add_event_at(next, health_interval_event)) {
+    health_interval_event = nullptr;
+  }
 }
 
 void Monitor::health_interval_stop()
@@ -2431,14 +2447,7 @@ void Monitor::do_health_to_clog(bool force)
        summary == health_status_cache.summary &&
        level == health_status_cache.overall)
       return;
-    if (level == HEALTH_OK)
-      clog->info() << "overall " << summary;
-    else if (level == HEALTH_WARN)
-      clog->warn() << "overall " << summary;
-    else if (level == HEALTH_ERR)
-      clog->error() << "overall " << summary;
-    else
-      ceph_abort();
+    clog->health(level) << "overall " << summary;
     health_status_cache.summary = summary;
     health_status_cache.overall = level;
   } else {
@@ -2475,6 +2484,7 @@ health_status_t Monitor::get_health_status(
 {
   health_status_t r = HEALTH_OK;
   bool compat = g_conf->mon_health_preluminous_compat;
+  bool compat_warn = g_conf->get_val<bool>("mon_health_preluminous_compat_warning");
   if (f) {
     f->open_object_section("health");
     f->open_object_section("checks");
@@ -2500,25 +2510,42 @@ health_status_t Monitor::get_health_status(
     *plain += "\n";
   }
 
-  if (f && compat) {
+  const std::string old_fields_message = "'ceph health' JSON format has "
+    "changed in luminous. If you see this your monitoring system is "
+    "scraping the wrong fields. Disable this with 'mon health preluminous "
+    "compat warning = false'";
+
+  if (f && (compat || compat_warn)) {
+    health_status_t cr = compat_warn ? min(HEALTH_WARN, r) : r;
     f->open_array_section("summary");
-    for (auto& svc : paxos_service) {
-      svc->get_health_checks().dump_summary_compat(f);
+    if (compat_warn) {
+      f->open_object_section("item");
+      f->dump_stream("severity") << HEALTH_WARN;
+      f->dump_string("summary", old_fields_message);
+      f->close_section();
+    }
+    if (compat) {
+      for (auto& svc : paxos_service) {
+        svc->get_health_checks().dump_summary_compat(f);
+      }
     }
     f->close_section();
-    f->dump_stream("overall_status") << r;
+    f->dump_stream("overall_status") << cr;
   }
 
   if (want_detail) {
-    if (f && compat) {
+    if (f && (compat || compat_warn)) {
       f->open_array_section("detail");
+      if (compat_warn) {
+       f->dump_string("item", old_fields_message);
+      }
     }
 
     for (auto& svc : paxos_service) {
       svc->get_health_checks().dump_detail(f, plain, compat);
     }
 
-    if (f && compat) {
+    if (f && (compat || compat_warn)) {
       f->close_section();
     }
   }
@@ -2536,31 +2563,60 @@ void Monitor::log_health(
   if (!g_conf->mon_health_to_clog) {
     return;
   }
+
+  const utime_t now = ceph_clock_now();
+
   // FIXME: log atomically as part of @t instead of using clog.
   dout(10) << __func__ << " updated " << updated.checks.size()
           << " previous " << previous.checks.size()
           << dendl;
+  const auto min_log_period = g_conf->get_val<int64_t>(
+      "mon_health_log_update_period");
   for (auto& p : updated.checks) {
     auto q = previous.checks.find(p.first);
+    bool logged = false;
     if (q == previous.checks.end()) {
       // new
       ostringstream ss;
       ss << "Health check failed: " << p.second.summary << " ("
          << p.first << ")";
-      if (p.second.severity == HEALTH_WARN)
-       clog->warn() << ss.str();
-      else
-       clog->error() << ss.str();
+      clog->health(p.second.severity) << ss.str();
+
+      logged = true;
     } else {
       if (p.second.summary != q->second.summary ||
          p.second.severity != q->second.severity) {
-       // summary or severity changed (ignore detail changes at this level)
-       ostringstream ss;
+
+        auto status_iter = health_check_log_times.find(p.first);
+        if (status_iter != health_check_log_times.end()) {
+          if (p.second.severity == q->second.severity &&
+              now - status_iter->second.updated_at < min_log_period) {
+            // We already logged this recently and the severity is unchanged,
+            // so skip emitting an update of the summary string.
+            // We'll get an update out of tick() later if the check
+            // is still failing.
+            continue;
+          }
+        }
+
+        // summary or severity changed (ignore detail changes at this level)
+        ostringstream ss;
         ss << "Health check update: " << p.second.summary << " (" << p.first << ")";
-       if (p.second.severity == HEALTH_WARN)
-         clog->warn() << ss.str();
-       else
-         clog->error() << ss.str();
+        clog->health(p.second.severity) << ss.str();
+
+        logged = true;
+      }
+    }
+    // Record the time at which we last logged, so that we can check this
+    // when considering whether/when to print update messages.
+    if (logged) {
+      auto iter = health_check_log_times.find(p.first);
+      if (iter == health_check_log_times.end()) {
+        health_check_log_times.emplace(p.first, HealthCheckLogStatus(
+          p.second.severity, p.second.summary, now));
+      } else {
+        iter->second = HealthCheckLogStatus(
+          p.second.severity, p.second.summary, now);
       }
     }
   }
@@ -2576,6 +2632,10 @@ void Monitor::log_health(
         clog->info() << "Health check cleared: " << p.first << " (was: "
                      << p.second.summary << ")";
       }
+
+      if (health_check_log_times.count(p.first)) {
+        health_check_log_times.erase(p.first);
+      }
     }
   }
 
@@ -2707,7 +2767,12 @@ void Monitor::get_cluster_status(stringstream &ss, Formatter *f)
 
   if (f) {
     f->dump_stream("fsid") << monmap->get_fsid();
-    get_health_status(false, f, nullptr);
+    if (osdmon()->osdmap.require_osd_release >= CEPH_RELEASE_LUMINOUS) {
+      get_health_status(false, f, nullptr);
+    } else {
+      list<string> health_str;
+      get_health(health_str, nullptr, f);
+    }
     f->dump_unsigned("election_epoch", get_epoch());
     {
       f->open_array_section("quorum");
@@ -2821,18 +2886,16 @@ void Monitor::_generate_command_map(map<string,cmd_vartype>& cmdmap,
   }
 }
 
-const MonCommand *Monitor::_get_moncommand(const string &cmd_prefix,
-                                           MonCommand *cmds, int cmds_size)
+const MonCommand *Monitor::_get_moncommand(
+  const string &cmd_prefix,
+  const vector<MonCommand>& cmds)
 {
-  MonCommand *this_cmd = NULL;
-  for (MonCommand *cp = cmds;
-       cp < &cmds[cmds_size]; cp++) {
-    if (cp->cmdstring.compare(0, cmd_prefix.size(), cmd_prefix) == 0) {
-      this_cmd = cp;
-      break;
+  for (auto& c : cmds) {
+    if (c.cmdstring.compare(0, cmd_prefix.size(), cmd_prefix) == 0) {
+      return &c;
     }
   }
-  return this_cmd;
+  return nullptr;
 }
 
 bool Monitor::_allowed_command(MonSession *s, string &module, string &prefix,
@@ -2855,26 +2918,23 @@ bool Monitor::_allowed_command(MonSession *s, string &module, string &prefix,
   return capable;
 }
 
-void Monitor::format_command_descriptions(const MonCommand *commands,
-                                         unsigned commands_size,
+void Monitor::format_command_descriptions(const std::vector<MonCommand> &commands,
                                          Formatter *f,
                                          bufferlist *rdata,
                                          bool hide_mgr_flag)
 {
   int cmdnum = 0;
   f->open_object_section("command_descriptions");
-  for (const MonCommand *cp = commands;
-       cp < &commands[commands_size]; cp++) {
-
-    unsigned flags = cp->flags;
+  for (const auto &cmd : commands) {
+    unsigned flags = cmd.flags;
     if (hide_mgr_flag) {
       flags &= ~MonCommand::FLAG_MGR;
     }
     ostringstream secname;
     secname << "cmd" << setfill('0') << std::setw(3) << cmdnum;
     dump_cmddesc_to_json(f, secname.str(),
-                        cp->cmdstring, cp->helpstring, cp->module,
-                        cp->req_perms, cp->availability, flags);
+                        cmd.cmdstring, cmd.helpstring, cmd.module,
+                        cmd.req_perms, cmd.availability, flags);
     cmdnum++;
   }
   f->close_section();  // command_descriptions
@@ -2882,20 +2942,6 @@ void Monitor::format_command_descriptions(const MonCommand *commands,
   f->flush(*rdata);
 }
 
-void Monitor::get_locally_supported_monitor_commands(const MonCommand **cmds,
-                                                    int *count)
-{
-  *cmds = mon_commands;
-  *count = ARRAY_SIZE(mon_commands);
-}
-void Monitor::set_leader_supported_commands(const MonCommand *cmds, int size)
-{
-  if (leader_supported_mon_commands != mon_commands)
-    delete[] leader_supported_mon_commands;
-  leader_supported_mon_commands = cmds;
-  leader_supported_mon_commands_size = size;
-}
-
 bool Monitor::is_keyring_required()
 {
   string auth_cluster_required = g_conf->auth_supported.empty() ?
@@ -2985,9 +3031,21 @@ void Monitor::handle_command(MonOpRequestRef op)
     // hide mgr commands until luminous upgrade is complete
     bool hide_mgr_flag =
       osdmon()->osdmap.require_osd_release < CEPH_RELEASE_LUMINOUS;
-    format_command_descriptions(leader_supported_mon_commands,
-                               leader_supported_mon_commands_size, f, &rdata,
-                               hide_mgr_flag);
+
+    std::vector<MonCommand> commands;
+
+    // only include mgr commands once all mons are upgrade (and we've dropped
+    // the hard-coded PGMonitor commands)
+    if (quorum_mon_features.contains_all(ceph::features::mon::FEATURE_LUMINOUS)) {
+      commands = static_cast<MgrMonitor*>(
+        paxos_service[PAXOS_MGR])->get_command_descs();
+    }
+
+    for (auto& c : leader_mon_commands) {
+      commands.push_back(c);
+    }
+
+    format_command_descriptions(commands, f, &rdata, hide_mgr_flag);
     delete f;
     reply_command(op, 0, "", rdata, 0);
     return;
@@ -3017,17 +3075,26 @@ void Monitor::handle_command(MonOpRequestRef op)
   // validate command is in leader map
 
   const MonCommand *leader_cmd;
-  leader_cmd = _get_moncommand(prefix,
-                               // the boost underlying this isn't const for some reason
-                               const_cast<MonCommand*>(leader_supported_mon_commands),
-                               leader_supported_mon_commands_size);
+  const auto& mgr_cmds = mgrmon()->get_command_descs();
+  const MonCommand *mgr_cmd = nullptr;
+  if (!mgr_cmds.empty()) {
+    mgr_cmd = _get_moncommand(prefix, mgr_cmds);
+  }
+  leader_cmd = _get_moncommand(prefix, leader_mon_commands);
   if (!leader_cmd) {
-    reply_command(op, -EINVAL, "command not known", 0);
-    return;
+    leader_cmd = mgr_cmd;
+    if (!leader_cmd) {
+      reply_command(op, -EINVAL, "command not known", 0);
+      return;
+    }
   }
   // validate command is in our map & matches, or forward if it is allowed
-  const MonCommand *mon_cmd = _get_moncommand(prefix, mon_commands,
-                                              ARRAY_SIZE(mon_commands));
+  const MonCommand *mon_cmd = _get_moncommand(
+    prefix,
+    get_local_commands(quorum_mon_features));
+  if (!mon_cmd) {
+    mon_cmd = mgr_cmd;
+  }
   if (!is_leader()) {
     if (!mon_cmd) {
       if (leader_cmd->is_noforward()) {
@@ -3102,8 +3169,8 @@ void Monitor::handle_command(MonOpRequestRef op)
       osdmon()->osdmap.require_osd_release >= CEPH_RELEASE_LUMINOUS) {
     const auto& hdr = m->get_header();
     uint64_t size = hdr.front_len + hdr.middle_len + hdr.data_len;
-    uint64_t max =
-      g_conf->mon_client_bytes * g_conf->mon_mgr_proxy_client_bytes_ratio;
+    uint64_t max = g_conf->get_val<uint64_t>("mon_client_bytes")
+                 * g_conf->get_val<double>("mon_mgr_proxy_client_bytes_ratio");
     if (mgr_proxy_bytes + size > max) {
       dout(10) << __func__ << " current mgr proxy bytes " << mgr_proxy_bytes
               << " + " << size << " > max " << max << dendl;
@@ -3122,7 +3189,8 @@ void Monitor::handle_command(MonOpRequestRef op)
     return;
   }
 
-  if (module == "mds" || module == "fs") {
+  if ((module == "mds" || module == "fs")  &&
+      prefix != "fs authorize") {
     mdsmon()->dispatch(op);
     return;
   }
@@ -3151,7 +3219,7 @@ void Monitor::handle_command(MonOpRequestRef op)
     monmon()->dispatch(op);
     return;
   }
-  if (module == "auth") {
+  if (module == "auth" || prefix == "fs authorize") {
     authmon()->dispatch(op);
     return;
   }
@@ -3257,6 +3325,17 @@ void Monitor::handle_command(MonOpRequestRef op)
     f->flush(rdata);
     r = 0;
     rs = "";
+  } else if (prefix == "config set") {
+    std::string key;
+    cmd_getval(cct, cmdmap, "key", key);
+    std::string val;
+    cmd_getval(cct, cmdmap, "value", val);
+    r = g_conf->set_val(key, val, true, &ss);
+    if (r == 0) {
+      g_conf->apply_changes(nullptr);
+    }
+    rs = ss.str();
+    goto out;
   } else if (prefix == "status" ||
             prefix == "health" ||
             prefix == "df") {
@@ -3342,8 +3421,12 @@ void Monitor::handle_command(MonOpRequestRef op)
       tagstr = tagstr.substr(0, tagstr.find_last_of(' '));
     f->dump_string("tag", tagstr);
 
-    list<string> hs;
-    get_health(hs, NULL, f.get());
+    if (osdmon()->osdmap.require_osd_release >= CEPH_RELEASE_LUMINOUS) {
+      get_health_status(true, f.get(), nullptr);
+    } else {
+      list<string> health_str;
+      get_health(health_str, nullptr, f.get());
+    }
 
     monmon()->dump_info(f.get());
     osdmon()->dump_info(f.get());
@@ -3552,6 +3635,65 @@ void Monitor::handle_command(MonOpRequestRef op)
     rdata.append(ds);
     rs = "";
     r = 0;
+  } else if (prefix == "versions") {
+    if (!f)
+      f.reset(Formatter::create("json-pretty"));
+    map<string,int> overall;
+    f->open_object_section("version");
+    map<string,int> mon, mgr, osd, mds;
+
+    count_metadata("ceph_version", &mon);
+    f->open_object_section("mon");
+    for (auto& p : mon) {
+      f->dump_int(p.first.c_str(), p.second);
+      overall[p.first] += p.second;
+    }
+    f->close_section();
+
+    mgrmon()->count_metadata("ceph_version", &mgr);
+    f->open_object_section("mgr");
+    for (auto& p : mgr) {
+      f->dump_int(p.first.c_str(), p.second);
+      overall[p.first] += p.second;
+    }
+    f->close_section();
+
+    osdmon()->count_metadata("ceph_version", &osd);
+    f->open_object_section("osd");
+    for (auto& p : osd) {
+      f->dump_int(p.first.c_str(), p.second);
+      overall[p.first] += p.second;
+    }
+    f->close_section();
+
+    mdsmon()->count_metadata("ceph_version", &mds);
+    f->open_object_section("mds");
+    for (auto& p : mds) {
+      f->dump_int(p.first.c_str(), p.second);
+      overall[p.first] += p.second;
+    }
+    f->close_section();
+
+    for (auto& p : mgrstatmon()->get_service_map().services) {
+      f->open_object_section(p.first.c_str());
+      map<string,int> m;
+      p.second.count_metadata("ceph_version", &m);
+      for (auto& q : m) {
+       f->dump_int(q.first.c_str(), q.second);
+       overall[q.first] += q.second;
+      }
+      f->close_section();
+    }
+
+    f->open_object_section("overall");
+    for (auto& p : overall) {
+      f->dump_int(p.first.c_str(), p.second);
+    }
+    f->close_section();
+    f->close_section();
+    f->flush(rdata);
+    rs = "";
+    r = 0;
   }
 
  out:
@@ -4319,8 +4461,13 @@ void Monitor::handle_ping(MonOpRequestRef op)
   boost::scoped_ptr<Formatter> f(new JSONFormatter(true));
   f->open_object_section("pong");
 
-  list<string> health_str;
-  get_health(health_str, NULL, f.get());
+  if (osdmon()->osdmap.require_osd_release >= CEPH_RELEASE_LUMINOUS) {
+    get_health_status(false, f.get(), nullptr);
+  } else {
+    list<string> health_str;
+    get_health(health_str, nullptr, f.get());
+  }
+
   {
     stringstream ss;
     get_mon_status(f.get(), ss);
@@ -4451,10 +4598,11 @@ void Monitor::timecheck_reset_event()
            << " rounds_since_clean " << timecheck_rounds_since_clean
            << dendl;
 
-  timecheck_event = new C_MonContext(this, [this](int) {
-      timecheck_start_round();
-    });
-  timer.add_event_after(delay, timecheck_event);
+  timecheck_event = timer.add_event_after(
+    delay,
+    new C_MonContext(this, [this](int) {
+       timecheck_start_round();
+      }));
 }
 
 void Monitor::timecheck_check_skews()
@@ -4685,10 +4833,7 @@ void Monitor::handle_timecheck_leader(MonOpRequestRef op)
 
   ostringstream ss;
   health_status_t status = timecheck_status(ss, skew_bound, latency);
-  if (status == HEALTH_ERR)
-    clog->error() << other << " " << ss.str();
-  else if (status == HEALTH_WARN)
-    clog->warn() << other << " " << ss.str();
+  clog->health(status) << other << " " << ss.str();
 
   dout(10) << __func__ << " from " << other << " ts " << m->timestamp
           << " delta " << delta << " skew_bound " << skew_bound
@@ -4999,17 +5144,22 @@ int Monitor::get_mon_metadata(int mon, Formatter *f, ostream& err)
   return 0;
 }
 
-void Monitor::count_metadata(const string& field, Formatter *f)
+void Monitor::count_metadata(const string& field, map<string,int> *out)
 {
-  map<string,int> by_val;
   for (auto& p : mon_metadata) {
     auto q = p.second.find(field);
     if (q == p.second.end()) {
-      by_val["unknown"]++;
+      (*out)["unknown"]++;
     } else {
-      by_val[q->second]++;
+      (*out)[q->second]++;
     }
   }
+}
+
+void Monitor::count_metadata(const string& field, Formatter *f)
+{
+  map<string,int> by_val;
+  count_metadata(field, &by_val);
   f->open_object_section(field.c_str());
   for (auto& p : by_val) {
     f->dump_int(p.first.c_str(), p.second);
@@ -5240,7 +5390,7 @@ void Monitor::scrub_check_results()
     }
   }
   if (!errors)
-    clog->info() << "scrub ok on " << quorum << ": " << mine;
+    clog->debug() << "scrub ok on " << quorum << ": " << mine;
 }
 
 inline void Monitor::scrub_timeout()
@@ -5298,10 +5448,11 @@ void Monitor::scrub_event_start()
     return;
   }
 
-  scrub_event = new C_MonContext(this, [this](int) {
+  scrub_event = timer.add_event_after(
+    cct->_conf->mon_scrub_interval,
+    new C_MonContext(this, [this](int) {
       scrub_start();
-    });
-  timer.add_event_after(cct->_conf->mon_scrub_interval, scrub_event);
+      }));
 }
 
 void Monitor::scrub_event_cancel()
@@ -5325,11 +5476,11 @@ void Monitor::scrub_reset_timeout()
 {
   dout(15) << __func__ << " reset timeout event" << dendl;
   scrub_cancel_timeout();
-
-  scrub_timeout_event = new C_MonContext(this, [this](int) {
+  scrub_timeout_event = timer.add_event_after(
+    g_conf->mon_scrub_timeout,
+    new C_MonContext(this, [this](int) {
       scrub_timeout();
-    });
-  timer.add_event_after(g_conf->mon_scrub_timeout, scrub_timeout_event);
+    }));
 }
 
 /************ TICK ***************/
@@ -5344,14 +5495,49 @@ void Monitor::tick()
 {
   // ok go.
   dout(11) << "tick" << dendl;
+  const utime_t now = ceph_clock_now();
   
+  // Check if we need to emit any delayed health check updated messages
+  if (is_leader()) {
+    const auto min_period = g_conf->get_val<int64_t>(
+                              "mon_health_log_update_period");
+    for (auto& svc : paxos_service) {
+      auto health = svc->get_health_checks();
+
+      for (const auto &i : health.checks) {
+        const std::string &code = i.first;
+        const std::string &summary = i.second.summary;
+        const health_status_t severity = i.second.severity;
+
+        auto status_iter = health_check_log_times.find(code);
+        if (status_iter == health_check_log_times.end()) {
+          continue;
+        }
+
+        auto &log_status = status_iter->second;
+        bool const changed = log_status.last_message != summary
+                             || log_status.severity != severity;
+
+        if (changed && now - log_status.updated_at > min_period) {
+          log_status.last_message = summary;
+          log_status.updated_at = now;
+          log_status.severity = severity;
+
+          ostringstream ss;
+          ss << "Health check update: " << summary << " (" << code << ")";
+          clog->health(severity) << ss.str();
+        }
+      }
+    }
+  }
+
+
   for (vector<PaxosService*>::iterator p = paxos_service.begin(); p != paxos_service.end(); ++p) {
     (*p)->tick();
     (*p)->maybe_trim();
   }
   
   // trim sessions
-  utime_t now = ceph_clock_now();
   {
     Mutex::Locker l(session_map_lock);
     auto p = session_map.sessions.begin();