]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/osd/OSD.cc
import quincy 17.2.0
[ceph.git] / ceph / src / osd / OSD.cc
index abfdde2f30ba579bcedeae498805a71b3070c6a0..c35ac0b526746fd85d400f6d74a43e9eae9d7afe 100644 (file)
@@ -23,7 +23,8 @@
 #include <unistd.h>
 #include <sys/stat.h>
 #include <signal.h>
-#include <boost/scoped_ptr.hpp>
+#include <time.h>
+#include <boost/range/adaptor/reversed.hpp>
 
 #ifdef HAVE_SYS_PARAM_H
 #include <sys/param.h>
 #endif
 
 #include "osd/PG.h"
+#include "osd/scrubber/scrub_machine.h"
+#include "osd/scrubber/pg_scrubber.h"
 
 #include "include/types.h"
 #include "include/compat.h"
 #include "include/random.h"
+#include "include/scope_guard.h"
 
 #include "OSD.h"
 #include "OSDMap.h"
 
 #include "common/errno.h"
 #include "common/ceph_argparse.h"
+#include "common/ceph_releases.h"
 #include "common/ceph_time.h"
 #include "common/version.h"
+#include "common/async/blocked_completion.h"
 #include "common/pick_address.h"
 #include "common/blkdev.h"
 #include "common/numa.h"
@@ -70,6 +76,7 @@
 #include "messages/MOSDPing.h"
 #include "messages/MOSDFailure.h"
 #include "messages/MOSDMarkMeDown.h"
+#include "messages/MOSDMarkMeDead.h"
 #include "messages/MOSDFull.h"
 #include "messages/MOSDOp.h"
 #include "messages/MOSDOpReply.h"
 #include "messages/MOSDMap.h"
 #include "messages/MMonGetOSDMap.h"
 #include "messages/MOSDPGNotify.h"
-#include "messages/MOSDPGQuery.h"
+#include "messages/MOSDPGNotify2.h"
+#include "messages/MOSDPGQuery2.h"
 #include "messages/MOSDPGLog.h"
 #include "messages/MOSDPGRemove.h"
 #include "messages/MOSDPGInfo.h"
+#include "messages/MOSDPGInfo2.h"
 #include "messages/MOSDPGCreate.h"
 #include "messages/MOSDPGCreate2.h"
-#include "messages/MOSDPGTrim.h"
-#include "messages/MOSDPGScan.h"
 #include "messages/MBackfillReserve.h"
 #include "messages/MRecoveryReserve.h"
 #include "messages/MOSDForceRecovery.h"
 
 #include "messages/MOSDScrub.h"
 #include "messages/MOSDScrub2.h"
-#include "messages/MOSDRepScrub.h"
 
-#include "messages/MMonCommand.h"
 #include "messages/MCommand.h"
 #include "messages/MCommandReply.h"
 
 #include "messages/MPGStats.h"
-#include "messages/MPGStatsAck.h"
 
-#include "messages/MWatchNotify.h"
-#include "messages/MOSDPGPush.h"
-#include "messages/MOSDPGPushReply.h"
-#include "messages/MOSDPGPull.h"
+#include "messages/MMonGetPurgedSnaps.h"
+#include "messages/MMonGetPurgedSnapsReply.h"
 
 #include "common/perf_counters.h"
 #include "common/Timer.h"
 #include "perfglue/cpu_profiler.h"
 #include "perfglue/heap_profiler.h"
 
+#include "osd/ClassHandler.h"
 #include "osd/OpRequest.h"
 
 #include "auth/AuthAuthorizeHandler.h"
 #define tracepoint(...)
 #endif
 
+#include "osd_tracer.h"
+
+
 #define dout_context cct
 #define dout_subsys ceph_subsys_osd
 #undef dout_prefix
 #define dout_prefix _prefix(_dout, whoami, get_osdmap_epoch())
 
+using std::deque;
+using std::list;
+using std::lock_guard;
+using std::make_pair;
+using std::make_tuple;
+using std::make_unique;
+using std::map;
+using std::ostream;
+using std::ostringstream;
+using std::pair;
+using std::set;
+using std::string;
+using std::stringstream;
+using std::to_string;
+using std::unique_ptr;
+using std::vector;
+
+using ceph::bufferlist;
+using ceph::bufferptr;
+using ceph::decode;
+using ceph::encode;
+using ceph::fixed_u_to_string;
+using ceph::Formatter;
+using ceph::heartbeat_handle_d;
+using ceph::make_mutex;
+
+using namespace ceph::osd::scheduler;
+using TOPNSPC::common::cmd_getval;
+using TOPNSPC::common::cmd_getval_or;
 
 static ostream& _prefix(std::ostream* _dout, int whoami, epoch_t epoch) {
   return *_dout << "osd." << whoami << " " << epoch << " ";
 }
 
+
 //Initial features in new superblock.
 //Features here are also automatically upgraded
 CompatSet OSD::get_osd_initial_compat_set() {
@@ -197,6 +233,7 @@ CompatSet OSD::get_osd_initial_compat_set() {
   ceph_osd_feature_incompat.insert(CEPH_OSD_FEATURE_INCOMPAT_MISSING);
   ceph_osd_feature_incompat.insert(CEPH_OSD_FEATURE_INCOMPAT_FASTINFO);
   ceph_osd_feature_incompat.insert(CEPH_OSD_FEATURE_INCOMPAT_RECOVERY_DELETES);
+  ceph_osd_feature_incompat.insert(CEPH_OSD_FEATURE_INCOMPAT_SNAPMAPPER2);
   return CompatSet(ceph_osd_feature_compat, ceph_osd_feature_ro_compat,
                   ceph_osd_feature_incompat);
 }
@@ -209,10 +246,10 @@ CompatSet OSD::get_osd_compat_set() {
   return compat;
 }
 
-OSDService::OSDService(OSD *osd) :
+OSDService::OSDService(OSD *osd, ceph::async::io_context_pool& poolctx) :
   osd(osd),
   cct(osd->cct),
-  whoami(osd->whoami), store(osd->store),
+  whoami(osd->whoami), store(osd->store.get()),
   log_client(osd->log_client), clog(osd->clog),
   pg_recovery_stats(osd->pg_recovery_stats),
   cluster_messenger(osd->cluster_messenger),
@@ -220,87 +257,60 @@ OSDService::OSDService(OSD *osd) :
   logger(osd->logger),
   recoverystate_perf(osd->recoverystate_perf),
   monc(osd->monc),
-  class_handler(osd->class_handler),
   osd_max_object_size(cct->_conf, "osd_max_object_size"),
   osd_skip_data_digest(cct->_conf, "osd_skip_data_digest"),
   publish_lock{ceph::make_mutex("OSDService::publish_lock")},
   pre_publish_lock{ceph::make_mutex("OSDService::pre_publish_lock")},
   max_oldest_map(0),
-  peer_map_epoch_lock("OSDService::peer_map_epoch_lock"),
-  sched_scrub_lock("OSDService::sched_scrub_lock"), scrubs_pending(0),
-  scrubs_active(0),
-  agent_lock("OSDService::agent_lock"),
+  m_scrub_queue{cct, *this},
   agent_valid_iterator(false),
   agent_ops(0),
   flush_mode_high_count(0),
   agent_active(true),
   agent_thread(this),
   agent_stop_flag(false),
-  agent_timer_lock("OSDService::agent_timer_lock"),
   agent_timer(osd->client_messenger->cct, agent_timer_lock),
   last_recalibrate(ceph_clock_now()),
   promote_max_objects(0),
   promote_max_bytes(0),
-  objecter(new Objecter(osd->client_messenger->cct, osd->objecter_messenger, osd->monc, NULL, 0, 0)),
+  poolctx(poolctx),
+  objecter(make_unique<Objecter>(osd->client_messenger->cct,
+                                osd->objecter_messenger,
+                                osd->monc, poolctx)),
   m_objecter_finishers(cct->_conf->osd_objecter_finishers),
-  watch_lock("OSDService::watch_lock"),
   watch_timer(osd->client_messenger->cct, watch_lock),
   next_notif_id(0),
-  recovery_request_lock("OSDService::recovery_request_lock"),
   recovery_request_timer(cct, recovery_request_lock, false),
-  sleep_lock("OSDService::sleep_lock"),
   sleep_timer(cct, sleep_lock, false),
   reserver_finisher(cct),
   local_reserver(cct, &reserver_finisher, cct->_conf->osd_max_backfills,
                 cct->_conf->osd_min_recovery_priority),
   remote_reserver(cct, &reserver_finisher, cct->_conf->osd_max_backfills,
                  cct->_conf->osd_min_recovery_priority),
-  pg_temp_lock("OSDService::pg_temp_lock"),
   snap_reserver(cct, &reserver_finisher,
                cct->_conf->osd_max_trimming_pgs),
-  recovery_lock("OSDService::recovery_lock"),
   recovery_ops_active(0),
   recovery_ops_reserved(0),
   recovery_paused(false),
-  map_cache_lock("OSDService::map_cache_lock"),
   map_cache(cct, cct->_conf->osd_map_cache_size),
   map_bl_cache(cct->_conf->osd_map_cache_size),
   map_bl_inc_cache(cct->_conf->osd_map_cache_size),
-  stat_lock("OSDService::stat_lock"),
-  full_status_lock("OSDService::full_status_lock"),
   cur_state(NONE),
   cur_ratio(0), physical_ratio(0),
-  epoch_lock("OSDService::epoch_lock"),
-  boot_epoch(0), up_epoch(0), bind_epoch(0),
-  is_stopping_lock("OSDService::is_stopping_lock")
-#ifdef PG_DEBUG_REFS
-  , pgid_lock("OSDService::pgid_lock")
-#endif
+  boot_epoch(0), up_epoch(0), bind_epoch(0)
 {
   objecter->init();
 
   for (int i = 0; i < m_objecter_finishers; i++) {
     ostringstream str;
     str << "objecter-finisher-" << i;
-    Finisher *fin = new Finisher(osd->client_messenger->cct, str.str(), "finisher");
-    objecter_finishers.push_back(fin);
-  }
-}
-
-OSDService::~OSDService()
-{
-  delete objecter;
-
-  for (auto f : objecter_finishers) {
-    delete f;
-    f = NULL;
+    auto fin = make_unique<Finisher>(osd->client_messenger->cct, str.str(), "finisher");
+    objecter_finishers.push_back(std::move(fin));
   }
 }
 
-
-
 #ifdef PG_DEBUG_REFS
-void OSDService::add_pgid(spg_t pgid, PG *pg){
+void OSDService::add_pgid(spg_t pgid, PG *pg) {
   std::lock_guard l(pgid_lock);
   if (!pgid_tracker.count(pgid)) {
     live_pgs[pgid] = pg;
@@ -332,6 +342,10 @@ void OSDService::dump_live_pgids()
 #endif
 
 
+ceph::signedspan OSDService::get_mnow()
+{
+  return ceph::mono_clock::now() - osd->startup_time;
+}
 
 void OSDService::identify_splits_and_merges(
   OSDMapRef old_map,
@@ -353,9 +367,11 @@ void OSDService::identify_splits_and_merges(
           << " pg_nums " << p->second << dendl;
   deque<spg_t> queue;
   queue.push_back(pgid);
+  set<spg_t> did;
   while (!queue.empty()) {
     auto cur = queue.front();
     queue.pop_front();
+    did.insert(cur);
     unsigned pgnum = old_pgnum;
     for (auto q = p->second.lower_bound(old_map->get_epoch());
         q != p->second.end() &&
@@ -371,7 +387,8 @@ void OSDService::identify_splits_and_merges(
                     << " children " << children << dendl;
            for (auto i : children) {
              split_children->insert(make_pair(i, q->first));
-             queue.push_back(i);
+              if (!did.count(i))
+               queue.push_back(i);
            }
          }
        } else if (cur.ps() < q->second) {
@@ -401,8 +418,15 @@ void OSDService::identify_splits_and_merges(
                       << " is merge source, target " << parent
                       << ", source(s) " << children << dendl;
              merge_pgs->insert(make_pair(parent, q->first));
+              if (!did.count(parent)) {
+                // queue (and re-scan) parent in case it might not exist yet
+                // and there are some future splits pending on it
+                queue.push_back(parent);
+              }
              for (auto c : children) {
                merge_pgs->insert(make_pair(c, q->first));
+                if (!did.count(c))
+                  queue.push_back(c);
              }
            }
          } else {
@@ -418,6 +442,8 @@ void OSDService::identify_splits_and_merges(
                     << " is merge target, source " << children << dendl;
            for (auto c : children) {
              merge_pgs->insert(make_pair(c, q->first));
+              if (!did.count(c))
+                queue.push_back(c);
            }
            merge_pgs->insert(make_pair(cur, q->first));
          }
@@ -433,6 +459,28 @@ void OSDService::need_heartbeat_peer_update()
   osd->need_heartbeat_peer_update();
 }
 
+HeartbeatStampsRef OSDService::get_hb_stamps(unsigned peer)
+{
+  std::lock_guard l(hb_stamp_lock);
+  if (peer >= hb_stamps.size()) {
+    hb_stamps.resize(peer + 1);
+  }
+  if (!hb_stamps[peer]) {
+    hb_stamps[peer] = ceph::make_ref<HeartbeatStamps>(peer);
+  }
+  return hb_stamps[peer];
+}
+
+void OSDService::queue_renew_lease(epoch_t epoch, spg_t spgid)
+{
+  osd->enqueue_peering_evt(
+    spgid,
+    PGPeeringEventRef(
+      std::make_shared<PGPeeringEvent>(
+       epoch, epoch,
+       RenewLease())));
+}
+
 void OSDService::start_shutdown()
 {
   {
@@ -459,13 +507,15 @@ void OSDService::shutdown_reserver()
 
 void OSDService::shutdown()
 {
+  mono_timer.suspend();
+
   {
     std::lock_guard l(watch_lock);
     watch_timer.shutdown();
   }
 
   objecter->shutdown();
-  for (auto f : objecter_finishers) {
+  for (auto& f : objecter_finishers) {
     f->wait_for_empty();
     f->stop();
   }
@@ -477,7 +527,7 @@ void OSDService::shutdown()
 void OSDService::init()
 {
   reserver_finisher.start();
-  for (auto f : objecter_finishers) {
+  for (auto& f : objecter_finishers) {
     f->start();
   }
   objecter->set_client_incarnation(0);
@@ -487,6 +537,7 @@ void OSDService::init()
 
   watch_timer.init();
   agent_timer.init();
+  mono_timer.resume();
 
   agent_thread.create("osd_srv_agent");
 
@@ -502,12 +553,11 @@ void OSDService::final_init()
 void OSDService::activate_map()
 {
   // wake/unwake the tiering agent
-  agent_lock.Lock();
+  std::lock_guard l{agent_lock};
   agent_active =
     !osdmap->test_flag(CEPH_OSDMAP_NOTIERAGENT) &&
     osd->is_active();
-  agent_cond.Signal();
-  agent_lock.Unlock();
+  agent_cond.notify_all();
 }
 
 void OSDService::request_osdmap_update(epoch_t e)
@@ -515,6 +565,7 @@ void OSDService::request_osdmap_update(epoch_t e)
   osd->osdmap_subscribe(e, false);
 }
 
+
 class AgentTimeoutCB : public Context {
   PGRef pg;
 public:
@@ -527,12 +578,12 @@ public:
 void OSDService::agent_entry()
 {
   dout(10) << __func__ << " start" << dendl;
-  agent_lock.Lock();
+  std::unique_lock agent_locker{agent_lock};
 
   while (!agent_stop_flag) {
     if (agent_queue.empty()) {
       dout(20) << __func__ << " empty queue" << dendl;
-      agent_cond.Wait(agent_lock);
+      agent_cond.wait(agent_locker);
       continue;
     }
     uint64_t level = agent_queue.rbegin()->first;
@@ -551,7 +602,7 @@ void OSDService::agent_entry()
     if (!flush_mode_high_count)
       agent_flush_quota = cct->_conf->osd_agent_max_low_ops - agent_ops;
     if (agent_flush_quota <= 0 || top.empty() || !agent_active) {
-      agent_cond.Wait(agent_lock);
+      agent_cond.wait(agent_locker);
       continue;
     }
 
@@ -563,22 +614,20 @@ void OSDService::agent_entry()
     dout(10) << "high_count " << flush_mode_high_count
             << " agent_ops " << agent_ops
             << " flush_quota " << agent_flush_quota << dendl;
-    agent_lock.Unlock();
+    agent_locker.unlock();
     if (!pg->agent_work(max, agent_flush_quota)) {
       dout(10) << __func__ << " " << pg->pg_id
        << " no agent_work, delay for " << cct->_conf->osd_agent_delay_time
        << " seconds" << dendl;
 
-      osd->logger->inc(l_osd_tier_delay);
+      logger->inc(l_osd_tier_delay);
       // Queue a timer to call agent_choose_mode for this pg in 5 seconds
-      agent_timer_lock.Lock();
+      std::lock_guard timer_locker{agent_timer_lock};
       Context *cb = new AgentTimeoutCB(pg);
       agent_timer.add_event_after(cct->_conf->osd_agent_delay_time, cb);
-      agent_timer_lock.Unlock();
     }
-    agent_lock.Lock();
+    agent_locker.lock();
   }
-  agent_lock.Unlock();
   dout(10) << __func__ << " finish" << dendl;
 }
 
@@ -597,7 +646,7 @@ void OSDService::agent_stop()
     }
 
     agent_stop_flag = true;
-    agent_cond.Signal();
+    agent_cond.notify_all();
   }
   agent_thread.join();
 }
@@ -699,7 +748,7 @@ OSDService::s_names OSDService::recalc_full_state(float ratio, float pratio, str
   float full_ratio = std::max(osdmap->get_full_ratio(), backfillfull_ratio);
   float failsafe_ratio = std::max(get_failsafe_full_ratio(), full_ratio);
 
-  if (osdmap->require_osd_release < CEPH_RELEASE_LUMINOUS) {
+  if (osdmap->require_osd_release < ceph_release_t::luminous) {
     // use the failsafe for nearfull and full; the mon isn't using the
     // flags anyway because we're mid-upgrade.
     full_ratio = failsafe_ratio;
@@ -725,7 +774,7 @@ OSDService::s_names OSDService::recalc_full_state(float ratio, float pratio, str
     return FULL;
   } else if (ratio > backfillfull_ratio) {
     return BACKFILLFULL;
-  } else if (ratio > nearfull_ratio) {
+  } else if (pratio > nearfull_ratio) {
     return NEARFULL;
   }
    return NONE;
@@ -920,9 +969,9 @@ void OSDService::set_statfs(const struct store_statfs_t &stbuf,
     used = bytes - avail;
   }
 
-  osd->logger->set(l_osd_stat_bytes, bytes);
-  osd->logger->set(l_osd_stat_bytes_used, used);
-  osd->logger->set(l_osd_stat_bytes_avail, avail);
+  logger->set(l_osd_stat_bytes, bytes);
+  logger->set(l_osd_stat_bytes_used, used);
+  logger->set(l_osd_stat_bytes_avail, avail);
 
   std::lock_guard l(stat_lock);
   osd_stat.statfs = stbuf;
@@ -939,10 +988,24 @@ void OSDService::set_statfs(const struct store_statfs_t &stbuf,
 osd_stat_t OSDService::set_osd_stat(vector<int>& hb_peers,
                                    int num_pgs)
 {
+  utime_t now = ceph_clock_now();
+  auto stale_time = g_conf().get_val<int64_t>("osd_mon_heartbeat_stat_stale");
   std::lock_guard l(stat_lock);
   osd_stat.hb_peers.swap(hb_peers);
   osd->op_tracker.get_age_ms_histogram(&osd_stat.op_queue_age_hist);
   osd_stat.num_pgs = num_pgs;
+  // Clean entries that aren't updated
+  // This is called often enough that we can just remove 1 at a time
+  for (auto i: osd_stat.hb_pingtime) {
+    if (i.second.last_update == 0)
+      continue;
+    if (stale_time && now.sec() - i.second.last_update > stale_time) {
+      dout(20) << __func__ << " time out heartbeat for osd " << i.first
+              << " last_update " << i.second.last_update << dendl;
+      osd_stat.hb_pingtime.erase(i.first);
+      break;
+    }
+  }
   return osd_stat;
 }
 
@@ -957,7 +1020,7 @@ float OSDService::compute_adjusted_ratio(osd_stat_t new_stat, float *pratio,
                                         uint64_t adjust_used)
 {
   *pratio =
-   ((float)new_stat.statfs.get_used()) / ((float)new_stat.statfs.total);
+   ((float)new_stat.statfs.get_used_raw()) / ((float)new_stat.statfs.total);
 
   if (adjust_used) {
     dout(20) << __func__ << " Before kb_used() " << new_stat.statfs.kb_used()  << dendl;
@@ -978,17 +1041,7 @@ float OSDService::compute_adjusted_ratio(osd_stat_t new_stat, float *pratio,
   if (backfill_adjusted) {
     dout(20) << __func__ << " backfill adjusted " << new_stat << dendl;
   }
-  return ((float)new_stat.statfs.get_used()) / ((float)new_stat.statfs.total);
-}
-
-bool OSDService::check_osdmap_full(const set<pg_shard_t> &missing_on)
-{
-  OSDMapRef osdmap = get_osdmap();
-  for (auto shard : missing_on) {
-    if (osdmap->get_state(shard.osd) & CEPH_OSD_FULL)
-      return true;
-  }
-  return false;
+  return ((float)new_stat.statfs.get_used_raw()) / ((float)new_stat.statfs.total);
 }
 
 void OSDService::send_message_osd_cluster(int peer, Message *m, epoch_t from_epoch)
@@ -1003,13 +1056,42 @@ void OSDService::send_message_osd_cluster(int peer, Message *m, epoch_t from_epo
     release_map(next_map);
     return;
   }
-  ConnectionRef peer_con = osd->cluster_messenger->connect_to_osd(
-    next_map->get_cluster_addrs(peer));
-  share_map_peer(peer, peer_con.get(), next_map);
+  ConnectionRef peer_con;
+  if (peer == whoami) {
+    peer_con = osd->cluster_messenger->get_loopback_connection();
+  } else {
+    peer_con = osd->cluster_messenger->connect_to_osd(
+       next_map->get_cluster_addrs(peer), false, true);
+  }
+  maybe_share_map(peer_con.get(), next_map);
   peer_con->send_message(m);
   release_map(next_map);
 }
 
+void OSDService::send_message_osd_cluster(std::vector<std::pair<int, Message*>>& messages, epoch_t from_epoch)
+{
+  OSDMapRef next_map = get_nextmap_reserved();
+  // service map is always newer/newest
+  ceph_assert(from_epoch <= next_map->get_epoch());
+
+  for (auto& iter : messages) {
+    if (next_map->is_down(iter.first) ||
+       next_map->get_info(iter.first).up_from > from_epoch) {
+      iter.second->put();
+      continue;
+    }
+    ConnectionRef peer_con;
+    if (iter.first == whoami) {
+      peer_con = osd->cluster_messenger->get_loopback_connection();
+    } else {
+      peer_con = osd->cluster_messenger->connect_to_osd(
+         next_map->get_cluster_addrs(iter.first), false, true);
+    }
+    maybe_share_map(peer_con.get(), next_map);
+    peer_con->send_message(iter.second);
+  }
+  release_map(next_map);
+}
 ConnectionRef OSDService::get_con_osd_cluster(int peer, epoch_t from_epoch)
 {
   OSDMapRef next_map = get_nextmap_reserved();
@@ -1021,8 +1103,13 @@ ConnectionRef OSDService::get_con_osd_cluster(int peer, epoch_t from_epoch)
     release_map(next_map);
     return NULL;
   }
-  ConnectionRef con = osd->cluster_messenger->connect_to_osd(
-    next_map->get_cluster_addrs(peer));
+  ConnectionRef con;
+  if (peer == whoami) {
+    con = osd->cluster_messenger->get_loopback_connection();
+  } else {
+    con = osd->cluster_messenger->connect_to_osd(
+       next_map->get_cluster_addrs(peer), false, true);
+  }
   release_map(next_map);
   return con;
 }
@@ -1134,7 +1221,7 @@ void OSDService::send_pg_created(pg_t pgid)
   std::lock_guard l(pg_created_lock);
   dout(20) << __func__ << dendl;
   auto o = get_osdmap();
-  if (o->require_osd_release >= CEPH_RELEASE_LUMINOUS) {
+  if (o->require_osd_release >= ceph_release_t::luminous) {
     pg_created.insert(pgid);
     monc->send_mon_message(new MOSDPGCreated(pgid));
   }
@@ -1145,7 +1232,7 @@ void OSDService::send_pg_created()
   std::lock_guard l(pg_created_lock);
   dout(20) << __func__ << dendl;
   auto o = get_osdmap();
-  if (o->require_osd_release >= CEPH_RELEASE_LUMINOUS) {
+  if (o->require_osd_release >= ceph_release_t::luminous) {
     for (auto pgid : pg_created) {
       monc->send_mon_message(new MOSDPGCreated(pgid));
     }
@@ -1174,227 +1261,6 @@ void OSDService::prune_pg_created()
 // --------------------------------------
 // dispatch
 
-epoch_t OSDService::get_peer_epoch(int peer)
-{
-  std::lock_guard l(peer_map_epoch_lock);
-  map<int,epoch_t>::iterator p = peer_map_epoch.find(peer);
-  if (p == peer_map_epoch.end())
-    return 0;
-  return p->second;
-}
-
-epoch_t OSDService::note_peer_epoch(int peer, epoch_t e)
-{
-  std::lock_guard l(peer_map_epoch_lock);
-  map<int,epoch_t>::iterator p = peer_map_epoch.find(peer);
-  if (p != peer_map_epoch.end()) {
-    if (p->second < e) {
-      dout(10) << "note_peer_epoch osd." << peer << " has " << e << dendl;
-      p->second = e;
-    } else {
-      dout(30) << "note_peer_epoch osd." << peer << " has " << p->second << " >= " << e << dendl;
-    }
-    return p->second;
-  } else {
-    dout(10) << "note_peer_epoch osd." << peer << " now has " << e << dendl;
-    peer_map_epoch[peer] = e;
-    return e;
-  }
-}
-
-void OSDService::forget_peer_epoch(int peer, epoch_t as_of)
-{
-  std::lock_guard l(peer_map_epoch_lock);
-  map<int,epoch_t>::iterator p = peer_map_epoch.find(peer);
-  if (p != peer_map_epoch.end()) {
-    if (p->second <= as_of) {
-      dout(10) << "forget_peer_epoch osd." << peer << " as_of " << as_of
-              << " had " << p->second << dendl;
-      peer_map_epoch.erase(p);
-    } else {
-      dout(10) << "forget_peer_epoch osd." << peer << " as_of " << as_of
-              << " has " << p->second << " - not forgetting" << dendl;
-    }
-  }
-}
-
-bool OSDService::should_share_map(entity_name_t name, Connection *con,
-                                  epoch_t epoch, const OSDMapRef& osdmap,
-                                  const epoch_t *sent_epoch_p)
-{
-  dout(20) << "should_share_map "
-           << name << " " << con->get_peer_addr()
-           << " " << epoch << dendl;
-
-  // does client have old map?
-  if (name.is_client()) {
-    bool message_sendmap = epoch < osdmap->get_epoch();
-    if (message_sendmap && sent_epoch_p) {
-      dout(20) << "client session last_sent_epoch: "
-               << *sent_epoch_p
-               << " versus osdmap epoch " << osdmap->get_epoch() << dendl;
-      if (*sent_epoch_p < osdmap->get_epoch()) {
-        return true;
-      } // else we don't need to send it out again
-    }
-  }
-
-  if (con->get_messenger() == osd->cluster_messenger &&
-      con != osd->cluster_messenger->get_loopback_connection() &&
-      osdmap->is_up(name.num()) &&
-      (osdmap->get_cluster_addrs(name.num()) == con->get_peer_addrs() ||
-       osdmap->get_hb_back_addrs(name.num()) == con->get_peer_addrs())) {
-    // remember
-    epoch_t has = std::max(get_peer_epoch(name.num()), epoch);
-
-    // share?
-    if (has < osdmap->get_epoch()) {
-      dout(10) << name << " " << con->get_peer_addr()
-               << " has old map " << epoch << " < "
-               << osdmap->get_epoch() << dendl;
-      return true;
-    }
-  }
-
-  return false;
-}
-
-void OSDService::share_map(
-    entity_name_t name,
-    Connection *con,
-    epoch_t epoch,
-    OSDMapRef& osdmap,
-    epoch_t *sent_epoch_p)
-{
-  dout(20) << "share_map "
-          << name << " " << con->get_peer_addr()
-          << " " << epoch << dendl;
-
-  if (!osd->is_active()) {
-    /*It is safe not to proceed as OSD is not in healthy state*/
-    return;
-  }
-
-  bool want_shared = should_share_map(name, con, epoch,
-                                      osdmap, sent_epoch_p);
-
-  if (want_shared){
-    if (name.is_client()) {
-      dout(10) << name << " has old map " << epoch
-          << " < " << osdmap->get_epoch() << dendl;
-      // we know the Session is valid or we wouldn't be sending
-      if (sent_epoch_p) {
-       *sent_epoch_p = osdmap->get_epoch();
-      }
-      send_incremental_map(epoch, con, osdmap);
-    } else if (con->get_messenger() == osd->cluster_messenger &&
-        osdmap->is_up(name.num()) &&
-        (osdmap->get_cluster_addrs(name.num()) == con->get_peer_addrs() ||
-            osdmap->get_hb_back_addrs(name.num()) == con->get_peer_addrs())) {
-      dout(10) << name << " " << con->get_peer_addrs()
-                      << " has old map " << epoch << " < "
-                      << osdmap->get_epoch() << dendl;
-      note_peer_epoch(name.num(), osdmap->get_epoch());
-      send_incremental_map(epoch, con, osdmap);
-    }
-  }
-}
-
-void OSDService::share_map_peer(int peer, Connection *con, OSDMapRef map)
-{
-  if (!map)
-    map = get_osdmap();
-
-  // send map?
-  epoch_t pe = get_peer_epoch(peer);
-  if (pe) {
-    if (pe < map->get_epoch()) {
-      send_incremental_map(pe, con, map);
-      note_peer_epoch(peer, map->get_epoch());
-    } else
-      dout(20) << "share_map_peer " << con << " already has epoch " << pe << dendl;
-  } else {
-    dout(20) << "share_map_peer " << con << " don't know epoch, doing nothing" << dendl;
-    // no idea about peer's epoch.
-    // ??? send recent ???
-    // do nothing.
-  }
-}
-
-bool OSDService::can_inc_scrubs_pending()
-{
-  bool can_inc = false;
-  std::lock_guard l(sched_scrub_lock);
-
-  if (scrubs_pending + scrubs_active < cct->_conf->osd_max_scrubs) {
-    dout(20) << __func__ << " " << scrubs_pending << " -> " << (scrubs_pending+1)
-            << " (max " << cct->_conf->osd_max_scrubs << ", active " << scrubs_active
-            << ")" << dendl;
-    can_inc = true;
-  } else {
-    dout(20) << __func__ << " " << scrubs_pending << " + " << scrubs_active
-            << " active >= max " << cct->_conf->osd_max_scrubs << dendl;
-  }
-
-  return can_inc;
-}
-
-bool OSDService::inc_scrubs_pending()
-{
-  bool result = false;
-
-  sched_scrub_lock.Lock();
-  if (scrubs_pending + scrubs_active < cct->_conf->osd_max_scrubs) {
-    dout(20) << "inc_scrubs_pending " << scrubs_pending << " -> " << (scrubs_pending+1)
-            << " (max " << cct->_conf->osd_max_scrubs << ", active " << scrubs_active << ")" << dendl;
-    result = true;
-    ++scrubs_pending;
-  } else {
-    dout(20) << "inc_scrubs_pending " << scrubs_pending << " + " << scrubs_active << " active >= max " << cct->_conf->osd_max_scrubs << dendl;
-  }
-  sched_scrub_lock.Unlock();
-
-  return result;
-}
-
-void OSDService::dec_scrubs_pending()
-{
-  sched_scrub_lock.Lock();
-  dout(20) << "dec_scrubs_pending " << scrubs_pending << " -> " << (scrubs_pending-1)
-          << " (max " << cct->_conf->osd_max_scrubs << ", active " << scrubs_active << ")" << dendl;
-  --scrubs_pending;
-  ceph_assert(scrubs_pending >= 0);
-  sched_scrub_lock.Unlock();
-}
-
-void OSDService::inc_scrubs_active(bool reserved)
-{
-  sched_scrub_lock.Lock();
-  ++(scrubs_active);
-  if (reserved) {
-    --(scrubs_pending);
-    dout(20) << "inc_scrubs_active " << (scrubs_active-1) << " -> " << scrubs_active
-            << " (max " << cct->_conf->osd_max_scrubs
-            << ", pending " << (scrubs_pending+1) << " -> " << scrubs_pending << ")" << dendl;
-    ceph_assert(scrubs_pending >= 0);
-  } else {
-    dout(20) << "inc_scrubs_active " << (scrubs_active-1) << " -> " << scrubs_active
-            << " (max " << cct->_conf->osd_max_scrubs
-            << ", pending " << scrubs_pending << ")" << dendl;
-  }
-  sched_scrub_lock.Unlock();
-}
-
-void OSDService::dec_scrubs_active()
-{
-  sched_scrub_lock.Lock();
-  dout(20) << "dec_scrubs_active " << scrubs_active << " -> " << (scrubs_active-1)
-          << " (max " << cct->_conf->osd_max_scrubs << ", pending " << scrubs_pending << ")" << dendl;
-  --scrubs_active;
-  ceph_assert(scrubs_active >= 0);
-  sched_scrub_lock.Unlock();
-}
-
 void OSDService::retrieve_epochs(epoch_t *_boot_epoch, epoch_t *_up_epoch,
                                  epoch_t *_bind_epoch) const
 {
@@ -1427,13 +1293,13 @@ void OSDService::set_epochs(const epoch_t *_boot_epoch, const epoch_t *_up_epoch
 
 bool OSDService::prepare_to_stop()
 {
-  std::lock_guard l(is_stopping_lock);
+  std::unique_lock l(is_stopping_lock);
   if (get_state() != NOT_STOPPING)
     return false;
 
   OSDMapRef osdmap = get_osdmap();
   if (osdmap && osdmap->is_up(whoami)) {
-    dout(0) << __func__ << " telling mon we are shutting down" << dendl;
+    dout(0) << __func__ << " telling mon we are shutting down and dead " << dendl;
     set_state(PREPARING_TO_STOP);
     monc->send_mon_message(
       new MOSDMarkMeDown(
@@ -1441,16 +1307,14 @@ bool OSDService::prepare_to_stop()
        whoami,
        osdmap->get_addrs(whoami),
        osdmap->get_epoch(),
-       true  // request ack
+       true,  // request ack
+       true   // mark as down and dead
        ));
-    utime_t now = ceph_clock_now();
-    utime_t timeout;
-    timeout.set_from_double(now + cct->_conf->osd_mon_shutdown_timeout);
-    while ((ceph_clock_now() < timeout) &&
-       (get_state() != STOPPING)) {
-      is_stopping_cond.WaitUntil(is_stopping_lock, timeout);
-    }
+    const auto timeout = ceph::make_timespan(cct->_conf->osd_mon_shutdown_timeout);
+    is_stopping_cond.wait_for(l, timeout,
+      [this] { return get_state() == STOPPING; });
   }
+
   dout(0) << __func__ << " starting shutdown" << dendl;
   set_state(STOPPING);
   return true;
@@ -1458,11 +1322,11 @@ bool OSDService::prepare_to_stop()
 
 void OSDService::got_stop_ack()
 {
-  std::lock_guard l(is_stopping_lock);
+  std::scoped_lock l(is_stopping_lock);
   if (get_state() == PREPARING_TO_STOP) {
     dout(0) << __func__ << " starting shutdown" << dendl;
     set_state(STOPPING);
-    is_stopping_cond.Signal();
+    is_stopping_cond.notify_all();
   } else {
     dout(10) << __func__ << " ignoring msg" << dendl;
   }
@@ -1492,19 +1356,19 @@ MOSDMap *OSDService::build_incremental_map_msg(epoch_t since, epoch_t to,
     }
     max--;
     max_bytes -= bl.length();
-    m->maps[since].claim(bl);
+    m->maps[since] = std::move(bl);
   }
   for (epoch_t e = since + 1; e <= to; ++e) {
     bufferlist bl;
     if (get_inc_map_bl(e, bl)) {
-      m->incremental_maps[e].claim(bl);
+      m->incremental_maps[e] = std::move(bl);
     } else {
-      derr << __func__ << " missing incremental map " << e << dendl;
+      dout(10) << __func__ << " missing incremental map " << e << dendl;
       if (!get_map_bl(e, bl)) {
        derr << __func__ << " also missing full map " << e << dendl;
        goto panic;
       }
-      m->maps[e].claim(bl);
+      m->maps[e] = std::move(bl);
     }
     max--;
     max_bytes -= bl.length();
@@ -1523,7 +1387,7 @@ MOSDMap *OSDService::build_incremental_map_msg(epoch_t since, epoch_t to,
   // send something
   bufferlist bl;
   if (get_inc_map_bl(m->newest_map, bl)) {
-    m->incremental_maps[m->newest_map].claim(bl);
+    m->incremental_maps[m->newest_map] = std::move(bl);
   } else {
     derr << __func__ << " unable to load latest map " << m->newest_map << dendl;
     if (!get_map_bl(m->newest_map, bl)) {
@@ -1531,7 +1395,7 @@ MOSDMap *OSDService::build_incremental_map_msg(epoch_t since, epoch_t to,
           << dendl;
       ceph_abort();
     }
-    m->maps[m->newest_map].claim(bl);
+    m->maps[m->newest_map] = std::move(bl);
   }
   return m;
 }
@@ -1542,7 +1406,7 @@ void OSDService::send_map(MOSDMap *m, Connection *con)
 }
 
 void OSDService::send_incremental_map(epoch_t since, Connection *con,
-                                      OSDMapRef& osdmap)
+                                      const OSDMapRef& osdmap)
 {
   epoch_t to = osdmap->get_epoch();
   dout(10) << "send_incremental_map " << since << " -> " << to
@@ -1577,12 +1441,10 @@ bool OSDService::_get_map_bl(epoch_t e, bufferlist& bl)
 {
   bool found = map_bl_cache.lookup(e, &bl);
   if (found) {
-    if (logger)
-      logger->inc(l_osd_map_bl_cache_hit);
+    logger->inc(l_osd_map_bl_cache_hit);
     return true;
   }
-  if (logger)
-    logger->inc(l_osd_map_bl_cache_miss);
+  logger->inc(l_osd_map_bl_cache_miss);
   found = store->read(meta_ch,
                      OSD::get_osdmap_pobject_name(e), 0, 0, bl,
                      CEPH_OSD_OP_FLAG_FADVISE_WILLNEED) >= 0;
@@ -1597,12 +1459,10 @@ bool OSDService::get_inc_map_bl(epoch_t e, bufferlist& bl)
   std::lock_guard l(map_cache_lock);
   bool found = map_bl_inc_cache.lookup(e, &bl);
   if (found) {
-    if (logger)
-      logger->inc(l_osd_map_bl_cache_hit);
+    logger->inc(l_osd_map_bl_cache_hit);
     return true;
   }
-  if (logger)
-    logger->inc(l_osd_map_bl_cache_miss);
+  logger->inc(l_osd_map_bl_cache_miss);
   found = store->read(meta_ch,
                      OSD::get_inc_osdmap_pobject_name(e), 0, 0, bl,
                      CEPH_OSD_OP_FLAG_FADVISE_WILLNEED) >= 0;
@@ -1634,26 +1494,6 @@ void OSDService::_add_map_inc_bl(epoch_t e, bufferlist& bl)
   map_bl_inc_cache.add(e, bl);
 }
 
-int OSDService::get_deleted_pool_pg_num(int64_t pool)
-{
-  std::lock_guard l(map_cache_lock);
-  auto p = deleted_pool_pg_nums.find(pool);
-  if (p != deleted_pool_pg_nums.end()) {
-    return p->second;
-  }
-  dout(20) << __func__ << " " << pool << " loading" << dendl;
-  ghobject_t oid = OSD::make_final_pool_info_oid(pool);
-  bufferlist bl;
-  int r = store->read(meta_ch, oid, 0, 0, bl);
-  ceph_assert(r >= 0);
-  auto blp = bl.cbegin();
-  pg_pool_t pi;
-  ::decode(pi, blp);
-  deleted_pool_pg_nums[pool] = pi.get_pg_num();
-  dout(20) << __func__ << " " << pool << " got " << pi.get_pg_num() << dendl;
-  return pi.get_pg_num();
-}
-
 OSDMapRef OSDService::_add_map(OSDMap *o)
 {
   epoch_t e = o->get_epoch();
@@ -1679,12 +1519,10 @@ OSDMapRef OSDService::try_get_map(epoch_t epoch)
   OSDMapRef retval = map_cache.lookup(epoch);
   if (retval) {
     dout(30) << "get_map " << epoch << " -cached" << dendl;
-    if (logger) {
-      logger->inc(l_osd_map_cache_hit);
-    }
+    logger->inc(l_osd_map_cache_hit);
     return retval;
   }
-  if (logger) {
+  {
     logger->inc(l_osd_map_cache_miss);
     epoch_t lb = map_cache.cached_key_lower_bound();
     if (epoch < lb) {
@@ -1715,19 +1553,22 @@ OSDMapRef OSDService::try_get_map(epoch_t epoch)
 
 void OSDService::reply_op_error(OpRequestRef op, int err)
 {
-  reply_op_error(op, err, eversion_t(), 0);
+  reply_op_error(op, err, eversion_t(), 0, {});
 }
 
 void OSDService::reply_op_error(OpRequestRef op, int err, eversion_t v,
-                                version_t uv)
+                                version_t uv,
+                               vector<pg_log_op_return_item_t> op_returns)
 {
-  const MOSDOp *m = static_cast<const MOSDOp*>(op->get_req());
+  auto m = op->get_req<MOSDOp>();
   ceph_assert(m->get_type() == CEPH_MSG_OSD_OP);
   int flags;
   flags = m->get_flags() & (CEPH_OSD_FLAG_ACK|CEPH_OSD_FLAG_ONDISK);
 
-  MOSDOpReply *reply = new MOSDOpReply(m, err, osdmap->get_epoch(), flags, true);
+  MOSDOpReply *reply = new MOSDOpReply(m, err, osdmap->get_epoch(), flags,
+                                      !m->has_flag(CEPH_OSD_FLAG_RETURNVEC));
   reply->set_reply_versions(v, uv);
+  reply->set_op_returns(op_returns);
   m->get_connection()->send_message(reply);
 }
 
@@ -1737,7 +1578,7 @@ void OSDService::handle_misdirected_op(PG *pg, OpRequestRef op)
     return;
   }
 
-  const MOSDOp *m = static_cast<const MOSDOp*>(op->get_req());
+  auto m = op->get_req<MOSDOp>();
   ceph_assert(m->get_type() == CEPH_MSG_OSD_OP);
 
   ceph_assert(m->get_map_epoch() >= pg->get_history().same_primary_since);
@@ -1786,12 +1627,12 @@ void OSDService::handle_misdirected_op(PG *pg, OpRequestRef op)
               << " in e" << m->get_map_epoch() << "/" << osdmap->get_epoch();
 }
 
-void OSDService::enqueue_back(OpQueueItem&& qi)
+void OSDService::enqueue_back(OpSchedulerItem&& qi)
 {
   osd->op_shardedwq.queue(std::move(qi));
 }
 
-void OSDService::enqueue_front(OpQueueItem&& qi)
+void OSDService::enqueue_front(OpSchedulerItem&& qi)
 {
   osd->op_shardedwq.queue_front(std::move(qi));
 }
@@ -1802,8 +1643,8 @@ void OSDService::queue_recovery_context(
 {
   epoch_t e = get_osdmap_epoch();
   enqueue_back(
-    OpQueueItem(
-      unique_ptr<OpQueueItem::OpQueueable>(
+    OpSchedulerItem(
+      unique_ptr<OpSchedulerItem::OpQueueable>(
        new PGRecoveryContext(pg->get_pgid(), c, e)),
       cct->_conf->osd_recovery_cost,
       cct->_conf->osd_recovery_priority,
@@ -1816,8 +1657,8 @@ void OSDService::queue_for_snap_trim(PG *pg)
 {
   dout(10) << "queueing " << *pg << " for snaptrim" << dendl;
   enqueue_back(
-    OpQueueItem(
-      unique_ptr<OpQueueItem::OpQueueable>(
+    OpSchedulerItem(
+      unique_ptr<OpSchedulerItem::OpQueueable>(
        new PGSnapTrim(pg->get_pgid(), pg->get_osdmap_epoch())),
       cct->_conf->osd_snap_trim_cost,
       cct->_conf->osd_snap_trim_priority,
@@ -1826,29 +1667,158 @@ void OSDService::queue_for_snap_trim(PG *pg)
       pg->get_osdmap_epoch()));
 }
 
-void OSDService::queue_for_scrub(PG *pg, bool with_high_priority)
+template <class MSG_TYPE>
+void OSDService::queue_scrub_event_msg(PG* pg,
+                                      Scrub::scrub_prio_t with_priority,
+                                      unsigned int qu_priority,
+                                      Scrub::act_token_t act_token)
 {
-  unsigned scrub_queue_priority = pg->scrubber.priority;
-  if (with_high_priority && scrub_queue_priority < cct->_conf->osd_client_op_priority) {
-    scrub_queue_priority = cct->_conf->osd_client_op_priority;
-  }
   const auto epoch = pg->get_osdmap_epoch();
-  enqueue_back(
-    OpQueueItem(
-      unique_ptr<OpQueueItem::OpQueueable>(new PGScrub(pg->get_pgid(), epoch)),
-      cct->_conf->osd_scrub_cost,
-      scrub_queue_priority,
-      ceph_clock_now(),
-      0,
-      epoch));
+  auto msg = new MSG_TYPE(pg->get_pgid(), epoch, act_token);
+  dout(15) << "queue a scrub event (" << *msg << ") for " << *pg
+           << ". Epoch: " << epoch << " token: " << act_token << dendl;
+
+  enqueue_back(OpSchedulerItem(
+    unique_ptr<OpSchedulerItem::OpQueueable>(msg), cct->_conf->osd_scrub_cost,
+    pg->scrub_requeue_priority(with_priority, qu_priority), ceph_clock_now(), 0, epoch));
+}
+
+template <class MSG_TYPE>
+void OSDService::queue_scrub_event_msg(PG* pg,
+                                       Scrub::scrub_prio_t with_priority)
+{
+  const auto epoch = pg->get_osdmap_epoch();
+  auto msg = new MSG_TYPE(pg->get_pgid(), epoch);
+  dout(15) << "queue a scrub event (" << *msg << ") for " << *pg << ". Epoch: " << epoch << dendl;
+
+  enqueue_back(OpSchedulerItem(
+    unique_ptr<OpSchedulerItem::OpQueueable>(msg), cct->_conf->osd_scrub_cost,
+    pg->scrub_requeue_priority(with_priority), ceph_clock_now(), 0, epoch));
+}
+
+void OSDService::queue_for_scrub(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  queue_scrub_event_msg<PGScrub>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_after_repair(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  queue_scrub_event_msg<PGScrubAfterRepair>(pg, with_priority);
+}
+
+void OSDService::queue_for_rep_scrub(PG* pg,
+                                    Scrub::scrub_prio_t with_priority,
+                                    unsigned int qu_priority,
+                                    Scrub::act_token_t act_token)
+{
+  queue_scrub_event_msg<PGRepScrub>(pg, with_priority, qu_priority, act_token);
+}
+
+void OSDService::queue_for_rep_scrub_resched(PG* pg,
+                                            Scrub::scrub_prio_t with_priority,
+                                            unsigned int qu_priority,
+                                            Scrub::act_token_t act_token)
+{
+  // Resulting scrub event: 'SchedReplica'
+  queue_scrub_event_msg<PGRepScrubResched>(pg, with_priority, qu_priority,
+                                          act_token);
+}
+
+void OSDService::queue_for_scrub_granted(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'RemotesReserved'
+  queue_scrub_event_msg<PGScrubResourcesOK>(pg, with_priority);
+}
+
+void OSDService::queue_for_scrub_denied(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'ReservationFailure'
+  queue_scrub_event_msg<PGScrubDenied>(pg, with_priority);
+}
+
+void OSDService::queue_for_scrub_resched(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'InternalSchedScrub'
+  queue_scrub_event_msg<PGScrubResched>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_pushes_update(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'ActivePushesUpd'
+  queue_scrub_event_msg<PGScrubPushesUpdate>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_chunk_free(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'SelectedChunkFree'
+  queue_scrub_event_msg<PGScrubChunkIsFree>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_chunk_busy(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'ChunkIsBusy'
+  queue_scrub_event_msg<PGScrubChunkIsBusy>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_applied_update(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  queue_scrub_event_msg<PGScrubAppliedUpdate>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_unblocking(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'Unblocked'
+  queue_scrub_event_msg<PGScrubUnblocked>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_digest_update(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'DigestUpdate'
+  queue_scrub_event_msg<PGScrubDigestUpdate>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_got_local_map(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'IntLocalMapDone'
+  queue_scrub_event_msg<PGScrubGotLocalMap>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_got_repl_maps(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'GotReplicas'
+  queue_scrub_event_msg<PGScrubGotReplMaps>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_maps_compared(PG* pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'MapsCompared'
+  queue_scrub_event_msg<PGScrubMapsCompared>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_replica_pushes(PG *pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'ReplicaPushesUpd'
+  queue_scrub_event_msg<PGScrubReplicaPushes>(pg, with_priority);
+}
+
+void OSDService::queue_scrub_is_finished(PG *pg)
+{
+  // Resulting scrub event: 'ScrubFinished'
+  queue_scrub_event_msg<PGScrubScrubFinished>(pg, Scrub::scrub_prio_t::high_priority);
+}
+
+void OSDService::queue_scrub_next_chunk(PG *pg, Scrub::scrub_prio_t with_priority)
+{
+  // Resulting scrub event: 'NextChunk'
+  queue_scrub_event_msg<PGScrubGetNextChunk>(pg, with_priority);
 }
 
 void OSDService::queue_for_pg_delete(spg_t pgid, epoch_t e)
 {
   dout(10) << __func__ << " on " << pgid << " e " << e  << dendl;
   enqueue_back(
-    OpQueueItem(
-      unique_ptr<OpQueueItem::OpQueueable>(
+    OpSchedulerItem(
+      unique_ptr<OpSchedulerItem::OpQueueable>(
        new PGDelete(pgid, e)),
       cct->_conf->osd_pg_delete_cost,
       cct->_conf->osd_pg_delete_priority,
@@ -1979,7 +1949,7 @@ void OSDService::clear_sent_ready_to_merge()
   sent_ready_to_merge_source.clear();
 }
 
-void OSDService::prune_sent_ready_to_merge(OSDMapRef& osdmap)
+void OSDService::prune_sent_ready_to_merge(const OSDMapRef& osdmap)
 {
   std::lock_guard l(merge_lock);
   auto i = sent_ready_to_merge_source.begin();
@@ -1999,10 +1969,10 @@ void OSDService::_queue_for_recovery(
   std::pair<epoch_t, PGRef> p,
   uint64_t reserved_pushes)
 {
-  ceph_assert(recovery_lock.is_locked_by_me());
+  ceph_assert(ceph_mutex_is_locked_by_me(recovery_lock));
   enqueue_back(
-    OpQueueItem(
-      unique_ptr<OpQueueItem::OpQueueable>(
+    OpSchedulerItem(
+      unique_ptr<OpSchedulerItem::OpQueueable>(
        new PGRecovery(
          p.second->get_pgid(), p.first, reserved_pushes)),
       cct->_conf->osd_recovery_cost,
@@ -2019,21 +1989,22 @@ void OSDService::_queue_for_recovery(
 #define dout_prefix *_dout
 
 // Commands shared between OSD's console and admin console:
-namespace ceph { 
-namespace osd_cmds { 
+namespace ceph::osd_cmds {
 
 int heap(CephContext& cct, const cmdmap_t& cmdmap, Formatter& f, std::ostream& os);
-}} // namespace ceph::osd_cmds
 
-int OSD::mkfs(CephContext *cct, ObjectStore *store, uuid_d fsid, int whoami)
+} // namespace ceph::osd_cmds
+
+int OSD::mkfs(CephContext *cct,
+             std::unique_ptr<ObjectStore> store,
+             uuid_d fsid,
+             int whoami,
+             string osdspec_affinity)
 {
   int ret;
 
   OSDSuperblock sb;
   bufferlist sbbl;
-  ObjectStore::CollectionHandle ch;
-
   // if we are fed a uuid for this osd, use it.
   store->set_fsid(cct->_conf->osd_uuid);
 
@@ -2041,7 +2012,7 @@ int OSD::mkfs(CephContext *cct, ObjectStore *store, uuid_d fsid, int whoami)
   if (ret) {
     derr << "OSD::mkfs: ObjectStore::mkfs failed with error "
          << cpp_strerror(ret) << dendl;
-    goto free_store;
+    return ret;
   }
 
   store->set_cache_shards(1);  // doesn't matter for mkfs!
@@ -2050,15 +2021,20 @@ int OSD::mkfs(CephContext *cct, ObjectStore *store, uuid_d fsid, int whoami)
   if (ret) {
     derr << "OSD::mkfs: couldn't mount ObjectStore: error "
          << cpp_strerror(ret) << dendl;
-    goto free_store;
+    return ret;
   }
 
-  ch = store->open_collection(coll_t::meta());
+  auto umount_store = make_scope_guard([&] {
+    store->umount();
+  });
+
+  ObjectStore::CollectionHandle ch =
+    store->open_collection(coll_t::meta());
   if (ch) {
     ret = store->read(ch, OSD_SUPERBLOCK_GOBJECT, 0, 0, sbbl);
     if (ret < 0) {
       derr << "OSD::mkfs: have meta collection but no superblock" << dendl;
-      goto free_store;
+      return ret;
     }
     /* if we already have superblock, check content of superblock */
     dout(0) << " have superblock" << dendl;
@@ -2067,14 +2043,12 @@ int OSD::mkfs(CephContext *cct, ObjectStore *store, uuid_d fsid, int whoami)
     if (whoami != sb.whoami) {
       derr << "provided osd id " << whoami << " != superblock's " << sb.whoami
           << dendl;
-      ret = -EINVAL;
-      goto umount_store;
+      return -EINVAL;
     }
     if (fsid != sb.cluster_fsid) {
       derr << "provided cluster fsid " << fsid
           << " != superblock's " << sb.cluster_fsid << dendl;
-      ret = -EINVAL;
-      goto umount_store;
+      return -EINVAL;
     }
   } else {
     // create superblock
@@ -2095,28 +2069,20 @@ int OSD::mkfs(CephContext *cct, ObjectStore *store, uuid_d fsid, int whoami)
     if (ret) {
       derr << "OSD::mkfs: error while writing OSD_SUPERBLOCK_GOBJECT: "
           << "queue_transaction returned " << cpp_strerror(ret) << dendl;
-      goto umount_store;
+      return ret;
     }
+    ch->flush();
   }
 
-  ret = write_meta(cct, store, sb.cluster_fsid, sb.osd_fsid, whoami);
+  ret = write_meta(cct, store.get(), sb.cluster_fsid, sb.osd_fsid, whoami, osdspec_affinity);
   if (ret) {
     derr << "OSD::mkfs: failed to write fsid file: error "
          << cpp_strerror(ret) << dendl;
-    goto umount_store;
-  }
-
-umount_store:
-  if (ch) {
-    ch.reset();
   }
-  store->umount();
-free_store:
-  delete store;
   return ret;
 }
 
-int OSD::write_meta(CephContext *cct, ObjectStore *store, uuid_d& cluster_fsid, uuid_d& osd_fsid, int whoami)
+int OSD::write_meta(CephContext *cct, ObjectStore *store, uuid_d& cluster_fsid, uuid_d& osd_fsid, int whoami, string& osdspec_affinity)
 {
   char val[80];
   int r;
@@ -2157,6 +2123,11 @@ int OSD::write_meta(CephContext *cct, ObjectStore *store, uuid_d& cluster_fsid,
        return r;
     }
   }
+  if (!osdspec_affinity.empty()) {
+    r = store->write_meta("osdspec_affinity", osdspec_affinity.c_str());
+    if (r < 0)
+      return r;
+  }
 
   r = store->write_meta("ready", "ready");
   if (r < 0)
@@ -2170,7 +2141,7 @@ int OSD::peek_meta(ObjectStore *store,
                   uuid_d *cluster_fsid,
                   uuid_d *osd_fsid,
                   int *whoami,
-                  int *require_osd_release)
+                  ceph_release_t *require_osd_release)
 {
   string val;
 
@@ -2202,7 +2173,7 @@ int OSD::peek_meta(ObjectStore *store,
 
   r = store->read_meta("require_osd_release", &val);
   if (r >= 0) {
-    *require_osd_release = atoi(val.c_str());
+    *require_osd_release = ceph_release_from_name(val);
   }
 
   return 0;
@@ -2214,7 +2185,8 @@ int OSD::peek_meta(ObjectStore *store,
 
 // cons/des
 
-OSD::OSD(CephContext *cct_, ObjectStore *store_,
+OSD::OSD(CephContext *cct_,
+        std::unique_ptr<ObjectStore> store_,
         int id,
         Messenger *internal_messenger,
         Messenger *external_messenger,
@@ -2224,21 +2196,20 @@ OSD::OSD(CephContext *cct_, ObjectStore *store_,
         Messenger *hb_back_serverm,
         Messenger *osdc_messenger,
         MonClient *mc,
-        const std::string &dev, const std::string &jdev) :
+        const std::string &dev, const std::string &jdev,
+        ceph::async::io_context_pool& poolctx) :
   Dispatcher(cct_),
-  osd_lock("OSD::osd_lock"),
   tick_timer(cct, osd_lock),
-  tick_timer_lock("OSD::tick_timer_lock"),
   tick_timer_without_osd_lock(cct, tick_timer_lock),
   gss_ktfile_client(cct->_conf.get_val<std::string>("gss_ktab_client_file")),
   cluster_messenger(internal_messenger),
   client_messenger(external_messenger),
   objecter_messenger(osdc_messenger),
   monc(mc),
-  mgrc(cct_, client_messenger),
-  logger(NULL),
-  recoverystate_perf(NULL),
-  store(store_),
+  mgrc(cct_, client_messenger, &mc->monmap),
+  logger(create_logger()),
+  recoverystate_perf(create_recoverystate_perf()),
+  store(std::move(store_)),
   log_client(cct, client_messenger, &mc->monmap, LogClient::NO_FLAGS),
   clog(log_client.create_channel()),
   whoami(id),
@@ -2251,10 +2222,6 @@ OSD::OSD(CephContext *cct_, ObjectStore *store_,
   osd_compat(get_osd_compat_set()),
   osd_op_tp(cct, "OSD::osd_op_tp", "tp_osd_tp",
            get_num_op_threads()),
-  command_tp(cct, "OSD::command_tp", "tp_osd_cmd",  1),
-  session_waiting_lock("OSD::session_waiting_lock"),
-  osdmap_subscribe_lock("OSD::osdmap_subscribe_lock"),
-  heartbeat_lock("OSD::heartbeat_lock"),
   heartbeat_stop(false),
   heartbeat_need_update(true),
   hb_front_client_messenger(hb_client_front),
@@ -2267,31 +2234,22 @@ OSD::OSD(CephContext *cct_, ObjectStore *store_,
   op_tracker(cct, cct->_conf->osd_enable_op_tracker,
                   cct->_conf->osd_num_op_tracker_shard),
   test_ops_hook(NULL),
-  op_queue(get_io_queue()),
-  op_prio_cutoff(get_io_prio_cut()),
   op_shardedwq(
     this,
-    cct->_conf->osd_op_thread_timeout,
-    cct->_conf->osd_op_thread_suicide_timeout,
+    ceph::make_timespan(cct->_conf->osd_op_thread_timeout),
+    ceph::make_timespan(cct->_conf->osd_op_thread_suicide_timeout),
     &osd_op_tp),
-  map_lock("OSD::map_lock"),
   last_pg_create_epoch(0),
-  mon_report_lock("OSD::mon_report_lock"),
   boot_finisher(cct),
   up_thru_wanted(0),
   requested_full_first(0),
   requested_full_last(0),
-  command_wq(
-    this,
-    cct->_conf->osd_command_thread_timeout,
-    cct->_conf->osd_command_thread_suicide_timeout,
-    &command_tp),
-  service(this)
+  service(this, poolctx)
 {
 
   if (!gss_ktfile_client.empty()) {
-    // Assert we can export environment variable 
-    /* 
+    // Assert we can export environment variable
+    /*
         The default client keytab is used, if it is present and readable,
         to automatically obtain initial credentials for GSSAPI client
         applications. The principal name of the first entry in the client
@@ -2300,7 +2258,7 @@ OSD::OSD(CephContext *cct_, ObjectStore *store_,
         2. The default_client_keytab_name profile variable in [libdefaults].
         3. The hardcoded default, DEFCKTNAME.
     */
-    const int32_t set_result(setenv("KRB5_CLIENT_KTNAME", 
+    const int32_t set_result(setenv("KRB5_CLIENT_KTNAME",
                                     gss_ktfile_client.c_str(), 1));
     ceph_assert(set_result == 0);
   }
@@ -2312,6 +2270,7 @@ OSD::OSD(CephContext *cct_, ObjectStore *store_,
                                            cct->_conf->osd_op_history_duration);
   op_tracker.set_history_slow_op_size_and_threshold(cct->_conf->osd_op_history_slow_op_size,
                                                     cct->_conf->osd_op_history_slow_op_threshold);
+  ObjectCleanRegions::set_max_num_intervals(cct->_conf->osd_object_clean_region_max_num_intervals);
 #ifdef WITH_BLKIN
   std::stringstream ss;
   ss << "osd." << whoami;
@@ -2324,10 +2283,7 @@ OSD::OSD(CephContext *cct_, ObjectStore *store_,
     OSDShard *one_shard = new OSDShard(
       i,
       cct,
-      this,
-      cct->_conf->osd_op_pq_max_tokens_per_priority,
-      cct->_conf->osd_op_pq_min_cost,
-      op_queue);
+      this);
     shards.push_back(one_shard);
   }
 }
@@ -2338,12 +2294,10 @@ OSD::~OSD()
     delete shards.back();
     shards.pop_back();
   }
-  delete class_handler;
   cct->get_perfcounters_collection()->remove(recoverystate_perf);
   cct->get_perfcounters_collection()->remove(logger);
   delete recoverystate_perf;
   delete logger;
-  delete store;
 }
 
 double OSD::get_tick_interval() const
@@ -2354,8 +2308,6 @@ double OSD::get_tick_interval() const
          ceph::util::generate_random_number(1.0 - delta, 1.0 + delta));
 }
 
-void cls_initialize(ClassHandler *ch);
-
 void OSD::handle_signal(int signum)
 {
   ceph_assert(signum == SIGINT || signum == SIGTERM);
@@ -2397,11 +2349,11 @@ int OSD::set_numa_affinity()
     cct,
     cluster_messenger->get_myaddrs().front().get_sockaddr_storage());
   int r = get_iface_numa_node(front_iface, &front_node);
-  if (r >= 0) {
+  if (r >= 0 && front_node >= 0) {
     dout(1) << __func__ << " public network " << front_iface << " numa node "
-             << front_node << dendl;
+            << front_node << dendl;
     r = get_iface_numa_node(back_iface, &back_node);
-    if (r >= 0) {
+    if (r >= 0 && back_node >= 0) {
       dout(1) << __func__ << " cluster network " << back_iface << " numa node "
              << back_node << dendl;
       if (front_node == back_node &&
@@ -2410,11 +2362,23 @@ int OSD::set_numa_affinity()
        if (g_conf().get_val<bool>("osd_numa_auto_affinity")) {
          numa_node = front_node;
        }
+      } else if (front_node != back_node) {
+        dout(1) << __func__ << " public and cluster network numa nodes do not match"
+                << dendl;
       } else {
        dout(1) << __func__ << " objectstore and network numa nodes do not match"
                << dendl;
       }
+    } else if (back_node == -2) {
+      dout(1) << __func__ << " cluster network " << back_iface
+              << " ports numa nodes do not match" << dendl;
+    } else {
+      derr << __func__ << " unable to identify cluster interface '" << back_iface
+           << "' numa node: " << cpp_strerror(r) << dendl;
     }
+  } else if (front_node == -2) {
+    dout(1) << __func__ << " public network " << front_iface
+            << " ports numa nodes do not match" << dendl;
   } else {
     derr << __func__ << " unable to identify public interface '" << front_iface
         << "' numa node: " << cpp_strerror(r) << dendl;
@@ -2434,7 +2398,7 @@ int OSD::set_numa_affinity()
              << " cpus "
              << cpu_set_to_str_list(numa_cpu_set_size, &numa_cpu_set)
              << dendl;
-      r = sched_setaffinity(getpid(), numa_cpu_set_size, &numa_cpu_set);
+      r = set_cpu_affinity_all_threads(numa_cpu_set_size, &numa_cpu_set);
       if (r < 0) {
        r = -errno;
        derr << __func__ << " failed to set numa affinity: " << cpp_strerror(r)
@@ -2454,18 +2418,24 @@ class OSDSocketHook : public AdminSocketHook {
   OSD *osd;
 public:
   explicit OSDSocketHook(OSD *o) : osd(o) {}
-  bool call(std::string_view admin_command, const cmdmap_t& cmdmap,
-           std::string_view format, bufferlist& out) override {
-    stringstream ss;
-    bool r = true;
+  int call(std::string_view prefix, const cmdmap_t& cmdmap,
+          Formatter *f,
+          std::ostream& ss,
+          bufferlist& out) override {
+    ceph_abort("should use async hook");
+  }
+  void call_async(
+    std::string_view prefix,
+    const cmdmap_t& cmdmap,
+    Formatter *f,
+    const bufferlist& inbl,
+    std::function<void(int,const std::string&,bufferlist&)> on_finish) override {
     try {
-      r = osd->asok_command(admin_command, cmdmap, format, ss);
-    } catch (const bad_cmd_get& e) {
-      ss << e.what();
-      r = true;
+      osd->asok_command(prefix, cmdmap, f, inbl, on_finish);
+    } catch (const TOPNSPC::common::bad_cmd_get& e) {
+      bufferlist empty;
+      on_finish(-EINVAL, e.what(), empty);
     }
-    out.append(ss);
-    return r;
   }
 };
 
@@ -2480,76 +2450,216 @@ std::set<int64_t> OSD::get_mapped_pools()
   return pools;
 }
 
-bool OSD::asok_command(std::string_view admin_command, const cmdmap_t& cmdmap,
-                      std::string_view format, ostream& ss)
+OSD::PGRefOrError OSD::locate_asok_target(const cmdmap_t& cmdmap,
+                                    stringstream& ss,
+                                    bool only_primary)
 {
-  Formatter *f = Formatter::create(format, "json-pretty", "json-pretty");
-  if (admin_command == "status") {
-    f->open_object_section("status");
-    f->dump_stream("cluster_fsid") << superblock.cluster_fsid;
-    f->dump_stream("osd_fsid") << superblock.osd_fsid;
-    f->dump_unsigned("whoami", superblock.whoami);
-    f->dump_string("state", get_state_name(get_state()));
-    f->dump_unsigned("oldest_map", superblock.oldest_map);
-    f->dump_unsigned("newest_map", superblock.newest_map);
-    f->dump_unsigned("num_pgs", num_pgs);
-    f->close_section();
-  } else if (admin_command == "flush_journal") {
-    store->flush_journal();
-  } else if (admin_command == "dump_ops_in_flight" ||
-             admin_command == "ops" ||
-             admin_command == "dump_blocked_ops" ||
-             admin_command == "dump_historic_ops" ||
-             admin_command == "dump_historic_ops_by_duration" ||
-             admin_command == "dump_historic_slow_ops") {
+  string pgidstr;
+  if (!cmd_getval(cmdmap, "pgid", pgidstr)) {
+    ss << "no pgid specified";
+    return OSD::PGRefOrError{std::nullopt, -EINVAL};
+  }
 
-    const string error_str = "op_tracker tracking is not enabled now, so no ops are tracked currently, \
-even those get stuck. Please enable \"osd_enable_op_tracker\", and the tracker \
-will start to track new ops received afterwards.";
+  pg_t pgid;
+  if (!pgid.parse(pgidstr.c_str())) {
+    ss << "couldn't parse pgid '" << pgidstr << "'";
+    return OSD::PGRefOrError{std::nullopt, -EINVAL};
+  }
 
-    set<string> filters;
-    vector<string> filter_str;
-    if (cmd_getval(cct, cmdmap, "filterstr", filter_str)) {
-        copy(filter_str.begin(), filter_str.end(),
-           inserter(filters, filters.end()));
+  spg_t pcand;
+  PGRef pg;
+  if (get_osdmap()->get_primary_shard(pgid, &pcand) && (pg = _lookup_lock_pg(pcand))) {
+    if (pg->is_primary() || !only_primary) {
+      return OSD::PGRefOrError{pg, 0};
     }
 
-    if (admin_command == "dump_ops_in_flight" ||
-        admin_command == "ops") {
-      if (!op_tracker.dump_ops_in_flight(f, false, filters)) {
-        ss << error_str;
-      }
-    }
-    if (admin_command == "dump_blocked_ops") {
-      if (!op_tracker.dump_ops_in_flight(f, true, filters)) {
+    ss << "not primary for pgid " << pgid;
+    pg->unlock();
+    return OSD::PGRefOrError{std::nullopt, -EAGAIN};
+  } else {
+    ss << "i don't have pgid " << pgid;
+    return OSD::PGRefOrError{std::nullopt, -ENOENT};
+  }
+}
+
+// note that the cmdmap is explicitly copied into asok_route_to_pg()
+int OSD::asok_route_to_pg(
+  bool only_primary,
+  std::string_view prefix,
+  cmdmap_t cmdmap,
+  Formatter* f,
+  stringstream& ss,
+  const bufferlist& inbl,
+  bufferlist& outbl,
+  std::function<void(int, const std::string&, bufferlist&)> on_finish)
+{
+  auto [target_pg, ret] = locate_asok_target(cmdmap, ss, only_primary);
+
+  if (!target_pg.has_value()) {
+    // 'ss' and 'ret' already contain the error information
+    on_finish(ret, ss.str(), outbl);
+    return ret;
+  }
+
+  //  the PG was locked by locate_asok_target()
+  try {
+    (*target_pg)->do_command(prefix, cmdmap, inbl, on_finish);
+    (*target_pg)->unlock();
+    return 0;  // the pg handler calls on_finish directly
+  } catch (const TOPNSPC::common::bad_cmd_get& e) {
+    (*target_pg)->unlock();
+    ss << e.what();
+    on_finish(ret, ss.str(), outbl);
+    return -EINVAL;
+  }
+}
+
+void OSD::asok_command(
+  std::string_view prefix, const cmdmap_t& cmdmap,
+  Formatter *f,
+  const bufferlist& inbl,
+  std::function<void(int,const std::string&,bufferlist&)> on_finish)
+{
+  int ret = 0;
+  stringstream ss;   // stderr error message stream
+  bufferlist outbl;  // if empty at end, we'll dump formatter as output
+
+  // --- PG commands are routed here to PG::do_command ---
+  if (prefix == "pg" ||
+      prefix == "query" ||
+      prefix == "mark_unfound_lost" ||
+      prefix == "list_unfound" ||
+      prefix == "scrub" ||
+      prefix == "deep_scrub"
+    ) {
+    string pgidstr;
+    pg_t pgid;
+    if (!cmd_getval(cmdmap, "pgid", pgidstr)) {
+      ss << "no pgid specified";
+      ret = -EINVAL;
+      goto out;
+    }
+    if (!pgid.parse(pgidstr.c_str())) {
+      ss << "couldn't parse pgid '" << pgidstr << "'";
+      ret = -EINVAL;
+      goto out;
+    }
+    spg_t pcand;
+    PGRef pg;
+    if (get_osdmap()->get_primary_shard(pgid, &pcand) &&
+       (pg = _lookup_lock_pg(pcand))) {
+      if (pg->is_primary()) {
+       cmdmap_t new_cmdmap = cmdmap;
+       try {
+         pg->do_command(prefix, new_cmdmap, inbl, on_finish);
+         pg->unlock();
+         return; // the pg handler calls on_finish directly
+       } catch (const TOPNSPC::common::bad_cmd_get& e) {
+         pg->unlock();
+         ss << e.what();
+         ret = -EINVAL;
+         goto out;
+       }
+      } else {
+       ss << "not primary for pgid " << pgid;
+       // do not reply; they will get newer maps and realize they
+       // need to resend.
+       pg->unlock();
+       ret = -EAGAIN;
+       goto out;
+      }
+    } else {
+      ss << "i don't have pgid " << pgid;
+      ret = -ENOENT;
+    }
+  }
+
+  // --- PG commands that will be answered even if !primary ---
+
+  else if (prefix == "scrubdebug") {
+    asok_route_to_pg(false, prefix, cmdmap, f, ss, inbl, outbl, on_finish);
+    return;
+  }
+
+  // --- OSD commands follow ---
+
+  else if (prefix == "status") {
+    lock_guard l(osd_lock);
+    f->open_object_section("status");
+    f->dump_stream("cluster_fsid") << superblock.cluster_fsid;
+    f->dump_stream("osd_fsid") << superblock.osd_fsid;
+    f->dump_unsigned("whoami", superblock.whoami);
+    f->dump_string("state", get_state_name(get_state()));
+    f->dump_unsigned("oldest_map", superblock.oldest_map);
+    f->dump_unsigned("newest_map", superblock.newest_map);
+    f->dump_unsigned("num_pgs", num_pgs);
+    f->close_section();
+  } else if (prefix == "flush_journal") {
+    store->flush_journal();
+  } else if (prefix == "dump_ops_in_flight" ||
+             prefix == "ops" ||
+             prefix == "dump_blocked_ops" ||
+             prefix == "dump_historic_ops" ||
+             prefix == "dump_historic_ops_by_duration" ||
+             prefix == "dump_historic_slow_ops") {
+
+    const string error_str = "op_tracker tracking is not enabled now, so no ops are tracked currently, \
+even those get stuck. Please enable \"osd_enable_op_tracker\", and the tracker \
+will start to track new ops received afterwards.";
+
+    set<string> filters;
+    vector<string> filter_str;
+    if (cmd_getval(cmdmap, "filterstr", filter_str)) {
+        copy(filter_str.begin(), filter_str.end(),
+           inserter(filters, filters.end()));
+    }
+
+    if (prefix == "dump_ops_in_flight" ||
+        prefix == "ops") {
+      if (!op_tracker.dump_ops_in_flight(f, false, filters)) {
         ss << error_str;
+       ret = -EINVAL;
+       goto out;
       }
     }
-    if (admin_command == "dump_historic_ops") {
+    if (prefix == "dump_blocked_ops") {
+      if (!op_tracker.dump_ops_in_flight(f, true, filters)) {
+        ss << error_str;
+       ret = -EINVAL;
+       goto out;
+      }
+    }
+    if (prefix == "dump_historic_ops") {
       if (!op_tracker.dump_historic_ops(f, false, filters)) {
         ss << error_str;
+       ret = -EINVAL;
+       goto out;
       }
     }
-    if (admin_command == "dump_historic_ops_by_duration") {
+    if (prefix == "dump_historic_ops_by_duration") {
       if (!op_tracker.dump_historic_ops(f, true, filters)) {
         ss << error_str;
+       ret = -EINVAL;
+       goto out;
       }
     }
-    if (admin_command == "dump_historic_slow_ops") {
+    if (prefix == "dump_historic_slow_ops") {
       if (!op_tracker.dump_historic_slow_ops(f, filters)) {
         ss << error_str;
+       ret = -EINVAL;
+       goto out;
       }
     }
-  } else if (admin_command == "dump_op_pq_state") {
+  } else if (prefix == "dump_op_pq_state") {
     f->open_object_section("pq");
     op_shardedwq.dump(f);
     f->close_section();
-  } else if (admin_command == "dump_blacklist") {
+  } else if (prefix == "dump_blocklist") {
     list<pair<entity_addr_t,utime_t> > bl;
     OSDMapRef curmap = service.get_osdmap();
 
-    f->open_array_section("blacklist");
-    curmap->get_blacklist(&bl);
+    f->open_array_section("blocklist");
+    curmap->get_blocklist(&bl);
     for (list<pair<entity_addr_t,utime_t> >::iterator it = bl.begin();
        it != bl.end(); ++it) {
       f->open_object_section("entry");
@@ -2559,8 +2669,8 @@ will start to track new ops received afterwards.";
       it->second.localtime(f->dump_stream("expire_time"));
       f->close_section(); //entry
     }
-    f->close_section(); //blacklist
-  } else if (admin_command == "dump_watchers") {
+    f->close_section(); //blocklist
+  } else if (prefix == "dump_watchers") {
     list<obj_watch_item_t> watchers;
     // scan pg's
     vector<PGRef> pgs;
@@ -2595,7 +2705,7 @@ will start to track new ops received afterwards.";
     }
 
     f->close_section(); //watchers
-  } else if (admin_command == "dump_reservations") {
+  } else if (prefix == "dump_recovery_reservations") {
     f->open_object_section("reservations");
     f->open_object_section("local_reservations");
     service.local_reserver.dump(f);
@@ -2604,25 +2714,21 @@ will start to track new ops received afterwards.";
     service.remote_reserver.dump(f);
     f->close_section();
     f->close_section();
-  } else if (admin_command == "get_latest_osdmap") {
-    get_latest_osdmap();
-  } else if (admin_command == "heap") {
-    auto result = ceph::osd_cmds::heap(*cct, cmdmap, *f, ss);
-
-    // Note: Failed heap profile commands won't necessarily trigger an error:
-    f->open_object_section("result");
-    f->dump_string("error", cpp_strerror(result));
-    f->dump_bool("success", result >= 0);
+  } else if (prefix == "dump_scrub_reservations") {
+    f->open_object_section("scrub_reservations");
+    service.get_scrub_services().dump_scrub_reservations(f);
     f->close_section();
-  } else if (admin_command == "set_heap_property") {
+  } else if (prefix == "get_latest_osdmap") {
+    get_latest_osdmap();
+  } else if (prefix == "set_heap_property") {
     string property;
     int64_t value = 0;
     string error;
     bool success = false;
-    if (!cmd_getval(cct, cmdmap, "property", property)) {
+    if (!cmd_getval(cmdmap, "property", property)) {
       error = "unable to get property";
       success = false;
-    } else if (!cmd_getval(cct, cmdmap, "value", value)) {
+    } else if (!cmd_getval(cmdmap, "value", value)) {
       error = "unable to get value";
       success = false;
     } else if (value < 0) {
@@ -2638,12 +2744,12 @@ will start to track new ops received afterwards.";
     f->dump_string("error", error);
     f->dump_bool("success", success);
     f->close_section();
-  } else if (admin_command == "get_heap_property") {
+  } else if (prefix == "get_heap_property") {
     string property;
     size_t value = 0;
     string error;
     bool success = false;
-    if (!cmd_getval(cct, cmdmap, "property", property)) {
+    if (!cmd_getval(cmdmap, "property", property)) {
       error = "unable to get property";
       success = false;
     } else if (!ceph_heap_get_numeric_property(property.c_str(), &value)) {
@@ -2657,67 +2763,530 @@ will start to track new ops received afterwards.";
     f->dump_bool("success", success);
     f->dump_int("value", value);
     f->close_section();
-  } else if (admin_command == "dump_objectstore_kv_stats") {
+  } else if (prefix == "dump_objectstore_kv_stats") {
     store->get_db_statistics(f);
-  } else if (admin_command == "dump_scrubs") {
-    service.dumps_scrub(f);
-  } else if (admin_command == "calc_objectstore_db_histogram") {
+  } else if (prefix == "dump_scrubs") {
+    service.get_scrub_services().dump_scrubs(f);
+  } else if (prefix == "calc_objectstore_db_histogram") {
     store->generate_db_histogram(f);
-  } else if (admin_command == "flush_store_cache") {
+  } else if (prefix == "flush_store_cache") {
     store->flush_cache(&ss);
-  } else if (admin_command == "dump_pgstate_history") {
+  } else if (prefix == "dump_pgstate_history") {
     f->open_object_section("pgstate_history");
+    f->open_array_section("pgs");
     vector<PGRef> pgs;
     _get_pgs(&pgs);
     for (auto& pg : pgs) {
+      f->open_object_section("pg");
       f->dump_stream("pg") << pg->pg_id;
+      f->dump_string("currently", pg->get_current_state());
       pg->dump_pgstate_history(f);
+      f->close_section();
     }
     f->close_section();
-  } else if (admin_command == "compact") {
+    f->close_section();
+  } else if (prefix == "compact") {
     dout(1) << "triggering manual compaction" << dendl;
     auto start = ceph::coarse_mono_clock::now();
     store->compact();
     auto end = ceph::coarse_mono_clock::now();
     double duration = std::chrono::duration<double>(end-start).count();
-    dout(1) << "finished manual compaction in " 
+    dout(1) << "finished manual compaction in "
             << duration
             << " seconds" << dendl;
     f->open_object_section("compact_result");
     f->dump_float("elapsed_time", duration);
     f->close_section();
-  } else if (admin_command == "get_mapped_pools") {
+  } else if (prefix == "get_mapped_pools") {
     f->open_array_section("mapped_pools");
     set<int64_t> poollist = get_mapped_pools();
     for (auto pool : poollist) {
       f->dump_int("pool_id", pool);
     }
     f->close_section();
-  } else if (admin_command == "smart") {
+  } else if (prefix == "smart") {
     string devid;
-    cmd_getval(cct, cmdmap, "devid", devid);
-    probe_smart(devid, ss);
-  } else if (admin_command == "list_devices") {
+    cmd_getval(cmdmap, "devid", devid);
+    ostringstream out;
+    probe_smart(devid, out);
+    outbl.append(out.str());
+  } else if (prefix == "list_devices") {
     set<string> devnames;
     store->get_devices(&devnames);
-    f->open_object_section("list_devices");
+    f->open_array_section("list_devices");
     for (auto dev : devnames) {
       if (dev.find("dm-") == 0) {
        continue;
       }
+      string err;
+      f->open_object_section("device");
       f->dump_string("device", "/dev/" + dev);
+      f->dump_string("device_id", get_device_id(dev, &err));
+      f->close_section();
     }
     f->close_section();
-  } else if (admin_command == "send_beacon") {
+  } else if (prefix == "send_beacon") {
+    lock_guard l(osd_lock);
     if (is_active()) {
       send_beacon(ceph::coarse_mono_clock::now());
     }
+  }
+
+  else if (prefix == "cluster_log") {
+    vector<string> msg;
+    cmd_getval(cmdmap, "message", msg);
+    if (msg.empty()) {
+      ret = -EINVAL;
+      ss << "ignoring empty log message";
+      goto out;
+    }
+    string message = msg.front();
+    for (vector<string>::iterator a = ++msg.begin(); a != msg.end(); ++a)
+      message += " " + *a;
+    string lvl;
+    cmd_getval(cmdmap, "level", lvl);
+    clog_type level = string_to_clog_type(lvl);
+    if (level < 0) {
+      ret = -EINVAL;
+      ss << "unknown level '" << lvl << "'";
+      goto out;
+    }
+    clog->do_log(level, message);
+  }
+
+  else if (prefix == "bench") {
+    // default count 1G, size 4MB
+    int64_t count = cmd_getval_or<int64_t>(cmdmap, "count", 1LL << 30);
+    int64_t bsize = cmd_getval_or<int64_t>(cmdmap, "size", 4LL << 20);
+    int64_t osize = cmd_getval_or<int64_t>(cmdmap, "object_size", 0);
+    int64_t onum = cmd_getval_or<int64_t>(cmdmap, "object_num", 0);
+    double elapsed = 0.0;
+
+    ret = run_osd_bench_test(count, bsize, osize, onum, &elapsed, ss);
+    if (ret != 0) {
+      goto out;
+    }
+
+    double rate = count / elapsed;
+    double iops = rate / bsize;
+    f->open_object_section("osd_bench_results");
+    f->dump_int("bytes_written", count);
+    f->dump_int("blocksize", bsize);
+    f->dump_float("elapsed_sec", elapsed);
+    f->dump_float("bytes_per_sec", rate);
+    f->dump_float("iops", iops);
+    f->close_section();
+  }
+
+  else if (prefix == "flush_pg_stats") {
+    mgrc.send_pgstats();
+    f->dump_unsigned("stat_seq", service.get_osd_stat_seq());
+  }
+
+  else if (prefix == "heap") {
+    ret = ceph::osd_cmds::heap(*cct, cmdmap, *f, ss);
+  }
+
+  else if (prefix == "debug dump_missing") {
+    f->open_array_section("pgs");
+    vector<PGRef> pgs;
+    _get_pgs(&pgs);
+    for (auto& pg : pgs) {
+      string s = stringify(pg->pg_id);
+      f->open_array_section(s.c_str());
+      pg->lock();
+      pg->dump_missing(f);
+      pg->unlock();
+      f->close_section();
+    }
+    f->close_section();
+  }
+
+  else if (prefix == "debug kick_recovery_wq") {
+    int64_t delay;
+    cmd_getval(cmdmap, "delay", delay);
+    ostringstream oss;
+    oss << delay;
+    ret = cct->_conf.set_val("osd_recovery_delay_start", oss.str().c_str());
+    if (ret != 0) {
+      ss << "kick_recovery_wq: error setting "
+        << "osd_recovery_delay_start to '" << delay << "': error "
+        << ret;
+      goto out;
+    }
+    cct->_conf.apply_changes(nullptr);
+    ss << "kicking recovery queue. set osd_recovery_delay_start "
+       << "to " << cct->_conf->osd_recovery_delay_start;
+  }
+
+  else if (prefix == "cpu_profiler") {
+    ostringstream ds;
+    string arg;
+    cmd_getval(cmdmap, "arg", arg);
+    vector<string> argvec;
+    get_str_vec(arg, argvec);
+    cpu_profiler_handle_command(argvec, ds);
+    outbl.append(ds.str());
+  }
+
+  else if (prefix == "dump_pg_recovery_stats") {
+    lock_guard l(osd_lock);
+    pg_recovery_stats.dump_formatted(f);
+  }
+
+  else if (prefix == "reset_pg_recovery_stats") {
+    lock_guard l(osd_lock);
+    pg_recovery_stats.reset();
+  }
+
+  else if (prefix == "perf histogram dump") {
+    std::string logger;
+    std::string counter;
+    cmd_getval(cmdmap, "logger", logger);
+    cmd_getval(cmdmap, "counter", counter);
+    cct->get_perfcounters_collection()->dump_formatted_histograms(
+      f, false, logger, counter);
+  }
+
+  else if (prefix == "cache drop") {
+    lock_guard l(osd_lock);
+    dout(20) << "clearing all caches" << dendl;
+    // Clear the objectstore's cache - onode and buffer for Bluestore,
+    // system's pagecache for Filestore
+    ret = store->flush_cache(&ss);
+    if (ret < 0) {
+      ss << "Error flushing objectstore cache: " << cpp_strerror(ret);
+      goto out;
+    }
+    // Clear the objectcontext cache (per PG)
+    vector<PGRef> pgs;
+    _get_pgs(&pgs);
+    for (auto& pg: pgs) {
+      pg->clear_cache();
+    }
+  }
+
+  else if (prefix == "cache status") {
+    lock_guard l(osd_lock);
+    int obj_ctx_count = 0;
+    vector<PGRef> pgs;
+    _get_pgs(&pgs);
+    for (auto& pg: pgs) {
+      obj_ctx_count += pg->get_cache_obj_count();
+    }
+    f->open_object_section("cache_status");
+    f->dump_int("object_ctx", obj_ctx_count);
+    store->dump_cache_stats(f);
+    f->close_section();
+  }
+
+  else if (prefix == "scrub_purged_snaps") {
+    lock_guard l(osd_lock);
+    scrub_purged_snaps();
+  }
+
+  else if (prefix == "dump_osd_network") {
+    lock_guard l(osd_lock);
+    int64_t value = 0;
+    if (!(cmd_getval(cmdmap, "value", value))) {
+      // Convert milliseconds to microseconds
+      value = static_cast<double>(g_conf().get_val<double>(
+                                   "mon_warn_on_slow_ping_time")) * 1000;
+      if (value == 0) {
+       double ratio = g_conf().get_val<double>("mon_warn_on_slow_ping_ratio");
+       value = g_conf().get_val<int64_t>("osd_heartbeat_grace");
+       value *= 1000000 * ratio; // Seconds of grace to microseconds at ratio
+      }
+    } else {
+      // Convert user input to microseconds
+      value *= 1000;
+    }
+    if (value < 0) value = 0;
+
+    struct osd_ping_time_t {
+      uint32_t pingtime;
+      int to;
+      bool back;
+      std::array<uint32_t,3> times;
+      std::array<uint32_t,3> min;
+      std::array<uint32_t,3> max;
+      uint32_t last;
+      uint32_t last_update;
+
+      bool operator<(const osd_ping_time_t& rhs) const {
+       if (pingtime < rhs.pingtime)
+          return true;
+       if (pingtime > rhs.pingtime)
+         return false;
+        if (to < rhs.to)
+         return true;
+        if (to > rhs.to)
+         return false;
+       return back;
+      }
+    };
+
+    set<osd_ping_time_t> sorted;
+    // Get pingtimes under lock and not on the stack
+    map<int, osd_stat_t::Interfaces> *pingtimes = new map<int, osd_stat_t::Interfaces>;
+    service.get_hb_pingtime(pingtimes);
+    for (auto j : *pingtimes) {
+      if (j.second.last_update == 0)
+       continue;
+      osd_ping_time_t item;
+      item.pingtime = std::max(j.second.back_pingtime[0], j.second.back_pingtime[1]);
+      item.pingtime = std::max(item.pingtime, j.second.back_pingtime[2]);
+      if (item.pingtime >= value) {
+       item.to = j.first;
+       item.times[0] = j.second.back_pingtime[0];
+       item.times[1] = j.second.back_pingtime[1];
+       item.times[2] = j.second.back_pingtime[2];
+       item.min[0] = j.second.back_min[0];
+       item.min[1] = j.second.back_min[1];
+       item.min[2] = j.second.back_min[2];
+       item.max[0] = j.second.back_max[0];
+       item.max[1] = j.second.back_max[1];
+       item.max[2] = j.second.back_max[2];
+       item.last = j.second.back_last;
+       item.back = true;
+       item.last_update = j.second.last_update;
+       sorted.emplace(item);
+      }
+      if (j.second.front_last == 0)
+       continue;
+      item.pingtime = std::max(j.second.front_pingtime[0], j.second.front_pingtime[1]);
+      item.pingtime = std::max(item.pingtime, j.second.front_pingtime[2]);
+      if (item.pingtime >= value) {
+       item.to = j.first;
+       item.times[0] = j.second.front_pingtime[0];
+       item.times[1] = j.second.front_pingtime[1];
+       item.times[2] = j.second.front_pingtime[2];
+       item.min[0] = j.second.front_min[0];
+       item.min[1] = j.second.front_min[1];
+       item.min[2] = j.second.front_min[2];
+       item.max[0] = j.second.front_max[0];
+       item.max[1] = j.second.front_max[1];
+       item.max[2] = j.second.front_max[2];
+       item.last = j.second.front_last;
+       item.last_update = j.second.last_update;
+       item.back = false;
+       sorted.emplace(item);
+      }
+    }
+    delete pingtimes;
+    //
+    // Network ping times (1min 5min 15min)
+    f->open_object_section("network_ping_times");
+    f->dump_int("threshold", value / 1000);
+    f->open_array_section("entries");
+    for (auto &sitem : boost::adaptors::reverse(sorted)) {
+      ceph_assert(sitem.pingtime >= value);
+      f->open_object_section("entry");
+
+      const time_t lu(sitem.last_update);
+      char buffer[26];
+      string lustr(ctime_r(&lu, buffer));
+      lustr.pop_back();   // Remove trailing \n
+      auto stale = cct->_conf.get_val<int64_t>("osd_heartbeat_stale");
+      f->dump_string("last update", lustr);
+      f->dump_bool("stale", ceph_clock_now().sec() - sitem.last_update > stale);
+      f->dump_int("from osd", whoami);
+      f->dump_int("to osd", sitem.to);
+      f->dump_string("interface", (sitem.back ? "back" : "front"));
+      f->open_object_section("average");
+      f->dump_format_unquoted("1min", "%s", fixed_u_to_string(sitem.times[0],3).c_str());
+      f->dump_format_unquoted("5min", "%s", fixed_u_to_string(sitem.times[1],3).c_str());
+      f->dump_format_unquoted("15min", "%s", fixed_u_to_string(sitem.times[2],3).c_str());
+      f->close_section();  // average
+      f->open_object_section("min");
+      f->dump_format_unquoted("1min", "%s", fixed_u_to_string(sitem.max[0],3).c_str());
+      f->dump_format_unquoted("5min", "%s", fixed_u_to_string(sitem.max[1],3).c_str());
+      f->dump_format_unquoted("15min", "%s", fixed_u_to_string(sitem.max[2],3).c_str());
+      f->close_section();  // min
+      f->open_object_section("max");
+      f->dump_format_unquoted("1min", "%s", fixed_u_to_string(sitem.max[0],3).c_str());
+      f->dump_format_unquoted("5min", "%s", fixed_u_to_string(sitem.max[1],3).c_str());
+      f->dump_format_unquoted("15min", "%s", fixed_u_to_string(sitem.max[2],3).c_str());
+      f->close_section();  // max
+      f->dump_format_unquoted("last", "%s", fixed_u_to_string(sitem.last,3).c_str());
+      f->close_section();  // entry
+    }
+    f->close_section(); // entries
+    f->close_section(); // network_ping_times
+  } else if (prefix == "dump_pool_statfs") {
+    lock_guard l(osd_lock);
+
+    int64_t p = 0;
+    if (!(cmd_getval(cmdmap, "poolid", p))) {
+      ss << "Error dumping pool statfs: no poolid provided";
+      ret = -EINVAL;
+      goto out;
+    }
+
+    store_statfs_t st;
+    bool per_pool_omap_stats = false;
+
+    ret = store->pool_statfs(p, &st, &per_pool_omap_stats);
+    if (ret < 0) {
+      ss << "Error dumping pool statfs: " << cpp_strerror(ret);
+      goto out;
+    } else {
+      ss << "dumping pool statfs...";
+      f->open_object_section("pool_statfs");
+      f->dump_int("poolid", p);
+      st.dump(f);
+      f->close_section();
+    }
   } else {
     ceph_abort_msg("broken asok registration");
   }
-  f->flush(ss);
-  delete f;
-  return true;
+
+ out:
+  on_finish(ret, ss.str(), outbl);
+}
+
+int OSD::run_osd_bench_test(
+  int64_t count,
+  int64_t bsize,
+  int64_t osize,
+  int64_t onum,
+  double *elapsed,
+  ostream &ss)
+{
+  int ret = 0;
+  uint32_t duration = cct->_conf->osd_bench_duration;
+
+  if (bsize > (int64_t) cct->_conf->osd_bench_max_block_size) {
+    // let us limit the block size because the next checks rely on it
+    // having a sane value.  If we allow any block size to be set things
+    // can still go sideways.
+    ss << "block 'size' values are capped at "
+       << byte_u_t(cct->_conf->osd_bench_max_block_size) << ". If you wish to use"
+       << " a higher value, please adjust 'osd_bench_max_block_size'";
+    ret = -EINVAL;
+    return ret;
+  } else if (bsize < (int64_t) (1 << 20)) {
+    // entering the realm of small block sizes.
+    // limit the count to a sane value, assuming a configurable amount of
+    // IOPS and duration, so that the OSD doesn't get hung up on this,
+    // preventing timeouts from going off
+    int64_t max_count =
+      bsize * duration * cct->_conf->osd_bench_small_size_max_iops;
+    if (count > max_count) {
+      ss << "'count' values greater than " << max_count
+         << " for a block size of " << byte_u_t(bsize) << ", assuming "
+         << cct->_conf->osd_bench_small_size_max_iops << " IOPS,"
+         << " for " << duration << " seconds,"
+         << " can cause ill effects on osd. "
+         << " Please adjust 'osd_bench_small_size_max_iops' with a higher"
+         << " value if you wish to use a higher 'count'.";
+      ret = -EINVAL;
+      return ret;
+    }
+  } else {
+    // 1MB block sizes are big enough so that we get more stuff done.
+    // However, to avoid the osd from getting hung on this and having
+    // timers being triggered, we are going to limit the count assuming
+    // a configurable throughput and duration.
+    // NOTE: max_count is the total amount of bytes that we believe we
+    //       will be able to write during 'duration' for the given
+    //       throughput.  The block size hardly impacts this unless it's
+    //       way too big.  Given we already check how big the block size
+    //       is, it's safe to assume everything will check out.
+    int64_t max_count =
+      cct->_conf->osd_bench_large_size_max_throughput * duration;
+    if (count > max_count) {
+      ss << "'count' values greater than " << max_count
+         << " for a block size of " << byte_u_t(bsize) << ", assuming "
+         << byte_u_t(cct->_conf->osd_bench_large_size_max_throughput) << "/s,"
+         << " for " << duration << " seconds,"
+         << " can cause ill effects on osd. "
+         << " Please adjust 'osd_bench_large_size_max_throughput'"
+         << " with a higher value if you wish to use a higher 'count'.";
+      ret = -EINVAL;
+      return ret;
+    }
+  }
+
+  if (osize && bsize > osize) {
+    bsize = osize;
+  }
+
+  dout(1) << " bench count " << count
+          << " bsize " << byte_u_t(bsize) << dendl;
+
+  ObjectStore::Transaction cleanupt;
+
+  if (osize && onum) {
+    bufferlist bl;
+    bufferptr bp(osize);
+    memset(bp.c_str(), 'a', bp.length());
+    bl.push_back(std::move(bp));
+    bl.rebuild_page_aligned();
+    for (int i=0; i<onum; ++i) {
+      char nm[30];
+      snprintf(nm, sizeof(nm), "disk_bw_test_%d", i);
+      object_t oid(nm);
+      hobject_t soid(sobject_t(oid, 0));
+      ObjectStore::Transaction t;
+      t.write(coll_t(), ghobject_t(soid), 0, osize, bl);
+      store->queue_transaction(service.meta_ch, std::move(t), nullptr);
+      cleanupt.remove(coll_t(), ghobject_t(soid));
+    }
+  }
+
+  bufferlist bl;
+  bufferptr bp(bsize);
+  memset(bp.c_str(), 'a', bp.length());
+  bl.push_back(std::move(bp));
+  bl.rebuild_page_aligned();
+
+  {
+    C_SaferCond waiter;
+    if (!service.meta_ch->flush_commit(&waiter)) {
+      waiter.wait();
+    }
+  }
+
+  utime_t start = ceph_clock_now();
+  for (int64_t pos = 0; pos < count; pos += bsize) {
+    char nm[30];
+    unsigned offset = 0;
+    if (onum && osize) {
+      snprintf(nm, sizeof(nm), "disk_bw_test_%d", (int)(rand() % onum));
+      offset = rand() % (osize / bsize) * bsize;
+    } else {
+      snprintf(nm, sizeof(nm), "disk_bw_test_%lld", (long long)pos);
+    }
+    object_t oid(nm);
+    hobject_t soid(sobject_t(oid, 0));
+    ObjectStore::Transaction t;
+    t.write(coll_t::meta(), ghobject_t(soid), offset, bsize, bl);
+    store->queue_transaction(service.meta_ch, std::move(t), nullptr);
+    if (!onum || !osize) {
+      cleanupt.remove(coll_t::meta(), ghobject_t(soid));
+    }
+  }
+
+  {
+    C_SaferCond waiter;
+    if (!service.meta_ch->flush_commit(&waiter)) {
+      waiter.wait();
+    }
+  }
+  utime_t end = ceph_clock_now();
+  *elapsed = end - start;
+
+  // clean up
+  store->queue_transaction(service.meta_ch, std::move(cleanupt), nullptr);
+  {
+    C_SaferCond waiter;
+    if (!service.meta_ch->flush_commit(&waiter)) {
+      waiter.wait();
+    }
+  }
+
+ return ret;
 }
 
 class TestOpsSocketHook : public AdminSocketHook {
@@ -2725,16 +3294,20 @@ class TestOpsSocketHook : public AdminSocketHook {
   ObjectStore *store;
 public:
   TestOpsSocketHook(OSDService *s, ObjectStore *st) : service(s), store(st) {}
-  bool call(std::string_view command, const cmdmap_t& cmdmap,
-           std::string_view format, bufferlist& out) override {
-    stringstream ss;
+  int call(std::string_view command, const cmdmap_t& cmdmap,
+          Formatter *f,
+          std::ostream& errss,
+          bufferlist& out) override {
+    int r = 0;
+    stringstream outss;
     try {
-      test_ops(service, store, command, cmdmap, ss);
-    } catch (const bad_cmd_get& e) {
-      ss << e.what();
+      test_ops(service, store, command, cmdmap, outss);
+      out.append(outss);
+    } catch (const TOPNSPC::common::bad_cmd_get& e) {
+      errss << e.what();
+      r = -EINVAL;
     }
-    out.append(ss);
-    return true;
+    return r;
   }
   void test_ops(OSDService *service, ObjectStore *store,
                std::string_view command, const cmdmap_t& cmdmap, ostream &ss);
@@ -2788,7 +3361,7 @@ int OSD::enable_disable_fuse(bool stop)
           << cpp_strerror(r) << dendl;
       return r;
     }
-    fuse_store = new FuseStore(store, mntpath);
+    fuse_store = new FuseStore(store.get(), mntpath);
     r = fuse_store->start();
     if (r < 0) {
       derr << __func__ << " unable to start fuse: " << cpp_strerror(r) << dendl;
@@ -2801,6 +3374,11 @@ int OSD::enable_disable_fuse(bool stop)
   return 0;
 }
 
+size_t OSD::get_num_cache_shards()
+{
+  return cct->_conf.get_val<Option::size_t>("osd_num_cache_shards");
+}
+
 int OSD::get_num_op_shards()
 {
   if (cct->_conf->osd_op_num_shards)
@@ -2845,13 +3423,36 @@ float OSD::get_osd_delete_sleep()
   return cct->_conf.get_val<double>("osd_delete_sleep_hdd");
 }
 
+int OSD::get_recovery_max_active()
+{
+  if (cct->_conf->osd_recovery_max_active)
+    return cct->_conf->osd_recovery_max_active;
+  if (store_is_rotational)
+    return cct->_conf->osd_recovery_max_active_hdd;
+  else
+    return cct->_conf->osd_recovery_max_active_ssd;
+}
+
+float OSD::get_osd_snap_trim_sleep()
+{
+  float osd_snap_trim_sleep = cct->_conf.get_val<double>("osd_snap_trim_sleep");
+  if (osd_snap_trim_sleep > 0)
+    return osd_snap_trim_sleep;
+  if (!store_is_rotational && !journal_is_rotational)
+    return cct->_conf.get_val<double>("osd_snap_trim_sleep_ssd");
+  if (store_is_rotational && !journal_is_rotational)
+    return cct->_conf.get_val<double>("osd_snap_trim_sleep_hybrid");
+  return cct->_conf.get_val<double>("osd_snap_trim_sleep_hdd");
+}
+
 int OSD::init()
 {
+  OSDMapRef osdmap;
   CompatSet initial, diff;
   std::lock_guard lock(osd_lock);
   if (is_stopping())
     return 0;
-
+  tracing::osd::tracer.init("osd");
   tick_timer.init();
   tick_timer_without_osd_lock.init();
   service.recovery_request_timer.init();
@@ -2862,7 +3463,7 @@ int OSD::init()
   {
     string val;
     store->read_meta("require_osd_release", &val);
-    last_require_osd_release = atoi(val.c_str());
+    last_require_osd_release = ceph_release_from_name(val);
   }
 
   // mount.
@@ -2872,7 +3473,11 @@ int OSD::init()
   dout(2) << "journal " << journal_path << dendl;
   ceph_assert(store);  // call pre_init() first!
 
-  store->set_cache_shards(get_num_op_shards());
+  store->set_cache_shards(get_num_cache_shards());
+
+ int rotating_auth_attempts = 0;
+ auto rotating_auth_timeout =
+   g_conf().get_val<int64_t>("rotating_keys_bootstrap_timeout");
 
   int r = store->mount();
   if (r < 0) {
@@ -2888,7 +3493,12 @@ int OSD::init()
   dout(2) << "boot" << dendl;
 
   service.meta_ch = store->open_collection(coll_t::meta());
-
+  if (!service.meta_ch) {
+    derr << "OSD:init: unable to open meta collection"
+         << dendl;
+    r = -ENOENT;
+    goto out;
+  }
   // initialize the daily loadavg with current 15min loadavg
   double loadavgs[3];
   if (getloadavg(loadavgs, 3) == 3) {
@@ -2898,10 +3508,6 @@ int OSD::init()
     daily_loadavg = 1.0;
   }
 
-  int rotating_auth_attempts = 0;
-  auto rotating_auth_timeout =
-    g_conf().get_val<int64_t>("rotating_keys_bootstrap_timeout");
-
   // sanity check long object name handling
   {
     hobject_t l;
@@ -2962,14 +3568,17 @@ int OSD::init()
     goto out;
   }
 
+  startup_time = ceph::mono_clock::now();
+
   // load up "current" osdmap
-  assert_warn(!osdmap);
-  if (osdmap) {
+  assert_warn(!get_osdmap());
+  if (get_osdmap()) {
     derr << "OSD::init: unable to read current osdmap" << dendl;
     r = -EINVAL;
     goto out;
   }
   osdmap = get_map(superblock.current_epoch);
+  set_osdmap(osdmap);
 
   // make sure we don't have legacy pgs deleting
   {
@@ -2995,6 +3604,17 @@ int OSD::init()
   initial = get_osd_initial_compat_set();
   diff = superblock.compat_features.unsupported(initial);
   if (superblock.compat_features.merge(initial)) {
+    // Are we adding SNAPMAPPER2?
+    if (diff.incompat.contains(CEPH_OSD_FEATURE_INCOMPAT_SNAPMAPPER2)) {
+      dout(1) << __func__ << " upgrade snap_mapper (first start as octopus)"
+             << dendl;
+      auto ch = service.meta_ch;
+      auto hoid = make_snapmapper_oid();
+      unsigned max = cct->_conf->osd_target_transaction_size;
+      r = SnapMapper::convert_legacy(cct, store.get(), ch, hoid, max);
+      if (r < 0)
+       goto out;
+    }
     // We need to persist the new compat_set before we
     // do anything else
     dout(5) << "Upgrading superblock adding: " << diff << dendl;
@@ -3014,20 +3634,23 @@ int OSD::init()
     if (r < 0)
       goto out;
   }
-
-  class_handler = new ClassHandler(cct);
-  cls_initialize(class_handler);
+  if (!store->exists(service.meta_ch, OSD::make_purged_snaps_oid())) {
+    dout(10) << "init creating/touching purged_snaps object" << dendl;
+    ObjectStore::Transaction t;
+    t.touch(coll_t::meta(), OSD::make_purged_snaps_oid());
+    r = store->queue_transaction(service.meta_ch, std::move(t));
+    if (r < 0)
+      goto out;
+  }
 
   if (cct->_conf->osd_open_classes_on_start) {
-    int r = class_handler->open_all_classes();
+    int r = ClassHandler::get_instance().open_all_classes();
     if (r)
       dout(1) << "warning: got an error loading one or more classes: " << cpp_strerror(r) << dendl;
   }
 
   check_osdmap_features();
 
-  create_recoverystate_perf();
-
   {
     epoch_t bind_epoch = osdmap->get_epoch();
     service.set_epochs(NULL, NULL, &bind_epoch);
@@ -3045,10 +3668,11 @@ int OSD::init()
   load_pgs();
 
   dout(2) << "superblock: I am osd." << superblock.whoami << dendl;
-  dout(0) << "using " << op_queue << " op queue with priority op cut off at " <<
-    op_prio_cutoff << "." << dendl;
 
-  create_logger();
+  if (cct->_conf.get_val<bool>("osd_compact_on_start")) {
+    dout(2) << "compacting object store's omap" << dendl;
+    store->compact();
+  }
 
   // prime osd stats
   {
@@ -3059,7 +3683,7 @@ int OSD::init()
     service.set_statfs(stbuf, alerts);
   }
 
-  // client_messenger auth_client is already set up by monc.
+  // client_messenger's auth_client will be set up by monc->init() later.
   for (auto m : { cluster_messenger,
        objecter_messenger,
        hb_front_client_messenger,
@@ -3082,13 +3706,13 @@ int OSD::init()
   if (r < 0)
     goto out;
 
-  mgrc.set_pgstats_cb([this](){ return collect_pg_stats(); });
+  mgrc.set_pgstats_cb([this]() { return collect_pg_stats(); });
   mgrc.set_perf_metric_query_cb(
-    [this](const std::map<OSDPerfMetricQuery, OSDPerfMetricLimits> &queries) {
-        set_perf_queries(queries);
+    [this](const ConfigPayload &config_payload) {
+        set_perf_queries(config_payload);
       },
-      [this](std::map<OSDPerfMetricQuery, OSDPerfMetricReport> *reports) {
-        get_perf_reports(reports);
+      [this] {
+        return get_perf_reports();
       });
   mgrc.init();
 
@@ -3106,7 +3730,7 @@ int OSD::init()
   hb_front_server_messenger->add_dispatcher_head(&heartbeat_dispatcher);
   hb_back_server_messenger->add_dispatcher_head(&heartbeat_dispatcher);
 
-  objecter_messenger->add_dispatcher_head(service.objecter);
+  objecter_messenger->add_dispatcher_head(service.objecter.get());
 
   service.init();
   service.publish_map(osdmap);
@@ -3125,7 +3749,7 @@ int OSD::init()
       pgs.insert(pg);
     }
     for (auto pg : pgs) {
-      pg->lock();
+      std::scoped_lock l{*pg};
       set<pair<spg_t,epoch_t>> new_children;
       set<pair<spg_t,epoch_t>> merge_pgs;
       service.identify_splits_and_merges(pg->get_osdmap(), osdmap, pg->pg_id,
@@ -3142,12 +3766,10 @@ int OSD::init()
        }
        assert(merge_pgs.empty());
       }
-      pg->unlock();
     }
   }
 
   osd_op_tp.start();
-  command_tp.start();
 
   // start the heartbeat
   heartbeat_thread.create("osd_srv_heartbt");
@@ -3161,7 +3783,7 @@ int OSD::init()
                                                new C_Tick_WithoutOSDLock(this));
   }
 
-  osd_lock.Unlock();
+  osd_lock.unlock();
 
   r = monc->authenticate();
   if (r < 0) {
@@ -3193,7 +3815,7 @@ int OSD::init()
     exit(1);
   }
 
-  osd_lock.Lock();
+  osd_lock.lock();
   if (is_stopping())
     return 0;
 
@@ -3221,13 +3843,16 @@ int OSD::init()
 
   start_boot();
 
+  // Override a few options if mclock scheduler is enabled.
+  maybe_override_max_osd_capacity_for_qos();
+  maybe_override_options_for_qos();
+
   return 0;
 
 out:
   enable_disable_fuse(true);
   store->umount();
-  delete store;
-  store = NULL;
+  store.reset();
   return r;
 }
 
@@ -3235,154 +3860,146 @@ void OSD::final_init()
 {
   AdminSocket *admin_socket = cct->get_admin_socket();
   asok_hook = new OSDSocketHook(this);
-  int r = admin_socket->register_command("status", "status", asok_hook,
+  int r = admin_socket->register_command("status", asok_hook,
                                         "high-level status of OSD");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("flush_journal", "flush_journal",
+  r = admin_socket->register_command("flush_journal",
                                      asok_hook,
                                      "flush the journal to permanent store");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("dump_ops_in_flight",
-                                    "dump_ops_in_flight " \
+  r = admin_socket->register_command("dump_ops_in_flight " \
                                     "name=filterstr,type=CephString,n=N,req=false",
                                     asok_hook,
                                     "show the ops currently in flight");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("ops",
-                                    "ops " \
+  r = admin_socket->register_command("ops " \
                                     "name=filterstr,type=CephString,n=N,req=false",
                                     asok_hook,
                                     "show the ops currently in flight");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("dump_blocked_ops",
-                                    "dump_blocked_ops " \
+  r = admin_socket->register_command("dump_blocked_ops " \
                                     "name=filterstr,type=CephString,n=N,req=false",
                                     asok_hook,
                                     "show the blocked ops currently in flight");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("dump_historic_ops",
-                                     "dump_historic_ops " \
+  r = admin_socket->register_command("dump_historic_ops " \
                                      "name=filterstr,type=CephString,n=N,req=false",
                                     asok_hook,
                                     "show recent ops");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("dump_historic_slow_ops",
-                                     "dump_historic_slow_ops " \
+  r = admin_socket->register_command("dump_historic_slow_ops " \
                                      "name=filterstr,type=CephString,n=N,req=false",
                                     asok_hook,
                                     "show slowest recent ops");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("dump_historic_ops_by_duration",
-                                     "dump_historic_ops_by_duration " \
+  r = admin_socket->register_command("dump_historic_ops_by_duration " \
                                      "name=filterstr,type=CephString,n=N,req=false",
                                     asok_hook,
                                     "show slowest recent ops, sorted by duration");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("dump_op_pq_state", "dump_op_pq_state",
+  r = admin_socket->register_command("dump_op_pq_state",
                                     asok_hook,
-                                    "dump op priority queue state");
+                                    "dump op queue state");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("dump_blacklist", "dump_blacklist",
+  r = admin_socket->register_command("dump_blocklist",
                                     asok_hook,
-                                    "dump blacklisted clients and times");
+                                    "dump blocklisted clients and times");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("dump_watchers", "dump_watchers",
+  r = admin_socket->register_command("dump_watchers",
                                     asok_hook,
                                     "show clients which have active watches,"
                                     " and on which objects");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("dump_reservations", "dump_reservations",
+  r = admin_socket->register_command("dump_recovery_reservations",
                                     asok_hook,
                                     "show recovery reservations");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("get_latest_osdmap", "get_latest_osdmap",
+  r = admin_socket->register_command("dump_scrub_reservations",
+                                    asok_hook,
+                                    "show scrub reservations");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command("get_latest_osdmap",
                                     asok_hook,
                                     "force osd to update the latest map from "
                                     "the mon");
   ceph_assert(r == 0);
 
-  r = admin_socket->register_command( "heap",
-                                      "heap " \
-                                      "name=heapcmd,type=CephString " \
-                                      "name=value,type=CephString,req=false",
-                                      asok_hook,
-                                      "show heap usage info (available only if "
-                                      "compiled with tcmalloc)");
-  ceph_assert(r == 0);
-
-  r = admin_socket->register_command("set_heap_property",
-                                    "set_heap_property " \
+  r = admin_socket->register_command("set_heap_property " \
                                     "name=property,type=CephString " \
                                     "name=value,type=CephInt",
                                     asok_hook,
                                     "update malloc extension heap property");
   ceph_assert(r == 0);
 
-  r = admin_socket->register_command("get_heap_property",
-                                    "get_heap_property " \
+  r = admin_socket->register_command("get_heap_property " \
                                     "name=property,type=CephString",
                                     asok_hook,
                                     "get malloc extension heap property");
   ceph_assert(r == 0);
 
   r = admin_socket->register_command("dump_objectstore_kv_stats",
-                                    "dump_objectstore_kv_stats",
                                     asok_hook,
                                     "print statistics of kvdb which used by bluestore");
   ceph_assert(r == 0);
 
   r = admin_socket->register_command("dump_scrubs",
-                                    "dump_scrubs",
                                     asok_hook,
                                     "print scheduled scrubs");
   ceph_assert(r == 0);
 
   r = admin_socket->register_command("calc_objectstore_db_histogram",
-                                     "calc_objectstore_db_histogram",
                                      asok_hook,
                                      "Generate key value histogram of kvdb(rocksdb) which used by bluestore");
   ceph_assert(r == 0);
 
   r = admin_socket->register_command("flush_store_cache",
-                                     "flush_store_cache",
                                      asok_hook,
                                      "Flush bluestore internal cache");
   ceph_assert(r == 0);
-  r = admin_socket->register_command("dump_pgstate_history", "dump_pgstate_history",
+  r = admin_socket->register_command("dump_pgstate_history",
                                     asok_hook,
                                     "show recent state history");
   ceph_assert(r == 0);
 
-  r = admin_socket->register_command("compact", "compact",
+  r = admin_socket->register_command("compact",
                                     asok_hook,
                                     "Commpact object store's omap."
                                      " WARNING: Compaction probably slows your requests");
   ceph_assert(r == 0);
 
-  r = admin_socket->register_command("get_mapped_pools", "get_mapped_pools",
+  r = admin_socket->register_command("get_mapped_pools",
                                      asok_hook,
                                      "dump pools whose PG(s) are mapped to this OSD.");
 
   ceph_assert(r == 0);
 
-  r = admin_socket->register_command("smart", "smart name=devid,type=CephString,req=False",
+  r = admin_socket->register_command("smart name=devid,type=CephString,req=false",
                                      asok_hook,
                                      "probe OSD devices for SMART data.");
 
   ceph_assert(r == 0);
 
-  r = admin_socket->register_command("list_devices", "list_devices",
+  r = admin_socket->register_command("list_devices",
                                      asok_hook,
                                      "list OSD devices.");
-  r = admin_socket->register_command("send_beacon", "send_beacon",
+  r = admin_socket->register_command("send_beacon",
                                      asok_hook,
                                      "send OSD beacon to mon immediately");
 
-  test_ops_hook = new TestOpsSocketHook(&(this->service), this->store);
+  r = admin_socket->register_command(
+    "dump_osd_network name=value,type=CephInt,req=false", asok_hook,
+    "Dump osd heartbeat network ping times");
+  ceph_assert(r == 0);
+
+  r = admin_socket->register_command(
+    "dump_pool_statfs name=poolid,type=CephInt,req=true", asok_hook,
+    "Dump store's statistics for the given pool");
+  ceph_assert(r == 0);
+
+  test_ops_hook = new TestOpsSocketHook(&(this->service), this->store.get());
   // Note: pools are CephString instead of CephPoolname because
   // these commands traditionally support both pool names and numbers
   r = admin_socket->register_command(
-   "setomapval",
    "setomapval " \
    "name=pool,type=CephString " \
    "name=objname,type=CephObjectname " \
@@ -3392,7 +4009,6 @@ void OSD::final_init()
    "set omap key");
   ceph_assert(r == 0);
   r = admin_socket->register_command(
-    "rmomapkey",
     "rmomapkey " \
     "name=pool,type=CephString " \
     "name=objname,type=CephObjectname " \
@@ -3401,7 +4017,6 @@ void OSD::final_init()
     "remove omap key");
   ceph_assert(r == 0);
   r = admin_socket->register_command(
-    "setomapheader",
     "setomapheader " \
     "name=pool,type=CephString " \
     "name=objname,type=CephObjectname " \
@@ -3411,7 +4026,6 @@ void OSD::final_init()
   ceph_assert(r == 0);
 
   r = admin_socket->register_command(
-    "getomap",
     "getomap " \
     "name=pool,type=CephString " \
     "name=objname,type=CephObjectname",
@@ -3420,7 +4034,6 @@ void OSD::final_init()
   ceph_assert(r == 0);
 
   r = admin_socket->register_command(
-    "truncobj",
     "truncobj " \
     "name=pool,type=CephString " \
     "name=objname,type=CephObjectname " \
@@ -3430,7 +4043,6 @@ void OSD::final_init()
   ceph_assert(r == 0);
 
   r = admin_socket->register_command(
-    "injectdataerr",
     "injectdataerr " \
     "name=pool,type=CephString " \
     "name=objname,type=CephObjectname " \
@@ -3440,7 +4052,6 @@ void OSD::final_init()
   ceph_assert(r == 0);
 
   r = admin_socket->register_command(
-    "injectmdataerr",
     "injectmdataerr " \
     "name=pool,type=CephString " \
     "name=objname,type=CephObjectname " \
@@ -3449,375 +4060,231 @@ void OSD::final_init()
     "inject metadata error to an object");
   ceph_assert(r == 0);
   r = admin_socket->register_command(
-    "set_recovery_delay",
     "set_recovery_delay " \
     "name=utime,type=CephInt,req=false",
     test_ops_hook,
      "Delay osd recovery by specified seconds");
   ceph_assert(r == 0);
   r = admin_socket->register_command(
-   "trigger_scrub",
-   "trigger_scrub " \
-   "name=pgid,type=CephString " \
-   "name=time,type=CephInt,req=false",
-   test_ops_hook,
-   "Trigger a scheduled scrub ");
-  ceph_assert(r == 0);
-  r = admin_socket->register_command(
-   "trigger_deep_scrub",
-   "trigger_deep_scrub " \
-   "name=pgid,type=CephString " \
-   "name=time,type=CephInt,req=false",
-   test_ops_hook,
-   "Trigger a scheduled deep scrub ");
-  ceph_assert(r == 0);
-  r = admin_socket->register_command(
-   "injectfull",
    "injectfull " \
    "name=type,type=CephString,req=false " \
    "name=count,type=CephInt,req=false ",
    test_ops_hook,
    "Inject a full disk (optional count times)");
   ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "bench " \
+    "name=count,type=CephInt,req=false "    \
+    "name=size,type=CephInt,req=false "                   \
+    "name=object_size,type=CephInt,req=false "    \
+    "name=object_num,type=CephInt,req=false ",
+    asok_hook,
+    "OSD benchmark: write <count> <size>-byte objects(with <obj_size> <obj_num>), " \
+    "(default count=1G default size=4MB). Results in log.");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "cluster_log " \
+    "name=level,type=CephChoices,strings=error,warning,info,debug "    \
+    "name=message,type=CephString,n=N",
+    asok_hook,
+    "log a message to the cluster log");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "flush_pg_stats",
+    asok_hook,
+    "flush pg stats");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "heap " \
+    "name=heapcmd,type=CephChoices,strings="                           \
+    "dump|start_profiler|stop_profiler|release|get_release_rate|set_release_rate|stats " \
+    "name=value,type=CephString,req=false",
+    asok_hook,
+    "show heap usage info (available only if compiled with tcmalloc)");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "debug dump_missing "                      \
+    "name=filename,type=CephFilepath",
+    asok_hook,
+    "dump missing objects to a named file");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "debug kick_recovery_wq "                                          \
+    "name=delay,type=CephInt,range=0",
+    asok_hook,
+    "set osd_recovery_delay_start to <val>");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "cpu_profiler "                                            \
+    "name=arg,type=CephChoices,strings=status|flush",
+    asok_hook,
+    "run cpu profiling on daemon");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "dump_pg_recovery_stats",
+    asok_hook,
+    "dump pg recovery statistics");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "reset_pg_recovery_stats",
+    asok_hook,
+    "reset pg recovery statistics");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "cache drop",
+    asok_hook,
+    "Drop all OSD caches");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "cache status",
+    asok_hook,
+    "Get OSD caches statistics");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "scrub_purged_snaps",
+    asok_hook,
+    "Scrub purged_snaps vs snapmapper index");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "scrubdebug "                                              \
+    "name=pgid,type=CephPgid "                                 \
+    "name=cmd,type=CephChoices,strings=block|unblock|set|unset " \
+    "name=value,type=CephString,req=false",
+    asok_hook,
+    "debug the scrubber");
+  ceph_assert(r == 0);
+
+  // -- pg commands --
+  // old form: ceph pg <pgid> command ...
+  r = admin_socket->register_command(
+    "pg "                         \
+    "name=pgid,type=CephPgid "    \
+    "name=cmd,type=CephChoices,strings=query",
+    asok_hook,
+    "");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "pg "                         \
+    "name=pgid,type=CephPgid "    \
+    "name=cmd,type=CephChoices,strings=mark_unfound_lost " \
+    "name=mulcmd,type=CephChoices,strings=revert|delete",
+    asok_hook,
+    "");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "pg "                         \
+    "name=pgid,type=CephPgid "    \
+    "name=cmd,type=CephChoices,strings=list_unfound " \
+    "name=offset,type=CephString,req=false",
+    asok_hook,
+    "");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "pg "                         \
+    "name=pgid,type=CephPgid "    \
+    "name=cmd,type=CephChoices,strings=scrub " \
+    "name=time,type=CephInt,req=false",
+    asok_hook,
+    "");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "pg "                         \
+    "name=pgid,type=CephPgid "    \
+    "name=cmd,type=CephChoices,strings=deep_scrub " \
+    "name=time,type=CephInt,req=false",
+    asok_hook,
+    "");
+  ceph_assert(r == 0);
+  // new form: tell <pgid> <cmd> for both cli and rest
+  r = admin_socket->register_command(
+    "query",
+    asok_hook,
+    "show details of a specific pg");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "mark_unfound_lost "                                       \
+    "name=pgid,type=CephPgid,req=false "                       \
+    "name=mulcmd,type=CephChoices,strings=revert|delete",
+    asok_hook,
+    "mark all unfound objects in this pg as lost, either removing or reverting to a prior version if one is available");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "list_unfound "                                    \
+    "name=pgid,type=CephPgid,req=false "               \
+    "name=offset,type=CephString,req=false",
+    asok_hook,
+    "list unfound objects on this pg, perhaps starting at an offset given in JSON");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "scrub "                           \
+    "name=pgid,type=CephPgid,req=false "       \
+    "name=time,type=CephInt,req=false",
+    asok_hook,
+    "Trigger a scheduled scrub ");
+  ceph_assert(r == 0);
+  r = admin_socket->register_command(
+    "deep_scrub "                      \
+    "name=pgid,type=CephPgid,req=false "       \
+    "name=time,type=CephInt,req=false",
+    asok_hook,
+    "Trigger a scheduled deep scrub ");
+  ceph_assert(r == 0);
 }
 
-void OSD::create_logger()
+PerfCounters* OSD::create_logger()
 {
-  dout(10) << "create_logger" << dendl;
-
-  PerfCountersBuilder osd_plb(cct, "osd", l_osd_first, l_osd_last);
-
-  // Latency axis configuration for op histograms, values are in nanoseconds
-  PerfHistogramCommon::axis_config_d op_hist_x_axis_config{
-    "Latency (usec)",
-    PerfHistogramCommon::SCALE_LOG2, ///< Latency in logarithmic scale
-    0,                               ///< Start at 0
-    100000,                          ///< Quantization unit is 100usec
-    32,                              ///< Enough to cover much longer than slow requests
-  };
-
-  // Op size axis configuration for op histograms, values are in bytes
-  PerfHistogramCommon::axis_config_d op_hist_y_axis_config{
-    "Request size (bytes)",
-    PerfHistogramCommon::SCALE_LOG2, ///< Request size in logarithmic scale
-    0,                               ///< Start at 0
-    512,                             ///< Quantization unit is 512 bytes
-    32,                              ///< Enough to cover requests larger than GB
-  };
-
-
-  // All the basic OSD operation stats are to be considered useful
-  osd_plb.set_prio_default(PerfCountersBuilder::PRIO_USEFUL);
-
-  osd_plb.add_u64(
-    l_osd_op_wip, "op_wip",
-    "Replication operations currently being processed (primary)");
-  osd_plb.add_u64_counter(
-    l_osd_op, "op",
-    "Client operations",
-    "ops", PerfCountersBuilder::PRIO_CRITICAL);
-  osd_plb.add_u64_counter(
-    l_osd_op_inb,   "op_in_bytes",
-    "Client operations total write size",
-    "wr", PerfCountersBuilder::PRIO_INTERESTING, unit_t(UNIT_BYTES));
-  osd_plb.add_u64_counter(
-    l_osd_op_outb,  "op_out_bytes",
-    "Client operations total read size",
-    "rd", PerfCountersBuilder::PRIO_INTERESTING, unit_t(UNIT_BYTES));
-  osd_plb.add_time_avg(
-    l_osd_op_lat,   "op_latency",
-    "Latency of client operations (including queue time)",
-    "l", 9);
-  osd_plb.add_time_avg(
-    l_osd_op_process_lat, "op_process_latency",
-    "Latency of client operations (excluding queue time)");
-  osd_plb.add_time_avg(
-    l_osd_op_prepare_lat, "op_prepare_latency",
-    "Latency of client operations (excluding queue time and wait for finished)");
-
-  osd_plb.add_u64_counter(
-    l_osd_op_r, "op_r", "Client read operations");
-  osd_plb.add_u64_counter(
-    l_osd_op_r_outb, "op_r_out_bytes", "Client data read", NULL, PerfCountersBuilder::PRIO_USEFUL, unit_t(UNIT_BYTES));
-  osd_plb.add_time_avg(
-    l_osd_op_r_lat, "op_r_latency",
-    "Latency of read operation (including queue time)");
-  osd_plb.add_u64_counter_histogram(
-    l_osd_op_r_lat_outb_hist, "op_r_latency_out_bytes_histogram",
-    op_hist_x_axis_config, op_hist_y_axis_config,
-    "Histogram of operation latency (including queue time) + data read");
-  osd_plb.add_time_avg(
-    l_osd_op_r_process_lat, "op_r_process_latency",
-    "Latency of read operation (excluding queue time)");
-  osd_plb.add_time_avg(
-    l_osd_op_r_prepare_lat, "op_r_prepare_latency",
-    "Latency of read operations (excluding queue time and wait for finished)");
-  osd_plb.add_u64_counter(
-    l_osd_op_w, "op_w", "Client write operations");
-  osd_plb.add_u64_counter(
-    l_osd_op_w_inb, "op_w_in_bytes", "Client data written");
-  osd_plb.add_time_avg(
-    l_osd_op_w_lat,  "op_w_latency",
-    "Latency of write operation (including queue time)");
-  osd_plb.add_u64_counter_histogram(
-    l_osd_op_w_lat_inb_hist, "op_w_latency_in_bytes_histogram",
-    op_hist_x_axis_config, op_hist_y_axis_config,
-    "Histogram of operation latency (including queue time) + data written");
-  osd_plb.add_time_avg(
-    l_osd_op_w_process_lat, "op_w_process_latency",
-    "Latency of write operation (excluding queue time)");
-  osd_plb.add_time_avg(
-    l_osd_op_w_prepare_lat, "op_w_prepare_latency",
-    "Latency of write operations (excluding queue time and wait for finished)");
-  osd_plb.add_u64_counter(
-    l_osd_op_rw, "op_rw",
-    "Client read-modify-write operations");
-  osd_plb.add_u64_counter(
-    l_osd_op_rw_inb, "op_rw_in_bytes",
-    "Client read-modify-write operations write in", NULL, PerfCountersBuilder::PRIO_USEFUL, unit_t(UNIT_BYTES));
-  osd_plb.add_u64_counter(
-    l_osd_op_rw_outb,"op_rw_out_bytes",
-    "Client read-modify-write operations read out ", NULL, PerfCountersBuilder::PRIO_USEFUL, unit_t(UNIT_BYTES));
-  osd_plb.add_time_avg(
-    l_osd_op_rw_lat, "op_rw_latency",
-    "Latency of read-modify-write operation (including queue time)");
-  osd_plb.add_u64_counter_histogram(
-    l_osd_op_rw_lat_inb_hist, "op_rw_latency_in_bytes_histogram",
-    op_hist_x_axis_config, op_hist_y_axis_config,
-    "Histogram of rw operation latency (including queue time) + data written");
-  osd_plb.add_u64_counter_histogram(
-    l_osd_op_rw_lat_outb_hist, "op_rw_latency_out_bytes_histogram",
-    op_hist_x_axis_config, op_hist_y_axis_config,
-    "Histogram of rw operation latency (including queue time) + data read");
-  osd_plb.add_time_avg(
-    l_osd_op_rw_process_lat, "op_rw_process_latency",
-    "Latency of read-modify-write operation (excluding queue time)");
-  osd_plb.add_time_avg(
-    l_osd_op_rw_prepare_lat, "op_rw_prepare_latency",
-    "Latency of read-modify-write operations (excluding queue time and wait for finished)");
-
-  // Now we move on to some more obscure stats, revert to assuming things
-  // are low priority unless otherwise specified.
-  osd_plb.set_prio_default(PerfCountersBuilder::PRIO_DEBUGONLY);
-
-  osd_plb.add_time_avg(l_osd_op_before_queue_op_lat, "op_before_queue_op_lat",
-    "Latency of IO before calling queue(before really queue into ShardedOpWq)"); // client io before queue op_wq latency
-  osd_plb.add_time_avg(l_osd_op_before_dequeue_op_lat, "op_before_dequeue_op_lat",
-    "Latency of IO before calling dequeue_op(already dequeued and get PG lock)"); // client io before dequeue_op latency
-
-  osd_plb.add_u64_counter(
-    l_osd_sop, "subop", "Suboperations");
-  osd_plb.add_u64_counter(
-    l_osd_sop_inb, "subop_in_bytes", "Suboperations total size", NULL, 0, unit_t(UNIT_BYTES));
-  osd_plb.add_time_avg(l_osd_sop_lat, "subop_latency", "Suboperations latency");
-
-  osd_plb.add_u64_counter(l_osd_sop_w, "subop_w", "Replicated writes");
-  osd_plb.add_u64_counter(
-    l_osd_sop_w_inb, "subop_w_in_bytes", "Replicated written data size", NULL, 0, unit_t(UNIT_BYTES));
-  osd_plb.add_time_avg(
-    l_osd_sop_w_lat, "subop_w_latency", "Replicated writes latency");
-  osd_plb.add_u64_counter(
-    l_osd_sop_pull, "subop_pull", "Suboperations pull requests");
-  osd_plb.add_time_avg(
-    l_osd_sop_pull_lat, "subop_pull_latency", "Suboperations pull latency");
-  osd_plb.add_u64_counter(
-    l_osd_sop_push, "subop_push", "Suboperations push messages");
-  osd_plb.add_u64_counter(
-    l_osd_sop_push_inb, "subop_push_in_bytes", "Suboperations pushed size", NULL, 0, unit_t(UNIT_BYTES));
-  osd_plb.add_time_avg(
-    l_osd_sop_push_lat, "subop_push_latency", "Suboperations push latency");
-
-  osd_plb.add_u64_counter(l_osd_pull, "pull", "Pull requests sent");
-  osd_plb.add_u64_counter(l_osd_push, "push", "Push messages sent");
-  osd_plb.add_u64_counter(l_osd_push_outb, "push_out_bytes", "Pushed size", NULL, 0, unit_t(UNIT_BYTES));
-
-  osd_plb.add_u64_counter(
-    l_osd_rop, "recovery_ops",
-    "Started recovery operations",
-    "rop", PerfCountersBuilder::PRIO_INTERESTING);
-
-  osd_plb.add_u64_counter(
-   l_osd_rbytes, "recovery_bytes",
-   "recovery bytes",
-   "rbt", PerfCountersBuilder::PRIO_INTERESTING);
-
-  osd_plb.add_u64(l_osd_loadavg, "loadavg", "CPU load");
-  osd_plb.add_u64(
-    l_osd_cached_crc, "cached_crc", "Total number getting crc from crc_cache");
-  osd_plb.add_u64(
-    l_osd_cached_crc_adjusted, "cached_crc_adjusted",
-    "Total number getting crc from crc_cache with adjusting");
-  osd_plb.add_u64(l_osd_missed_crc, "missed_crc", 
-    "Total number of crc cache misses");
-
-  osd_plb.add_u64(l_osd_pg, "numpg", "Placement groups",
-                 "pgs", PerfCountersBuilder::PRIO_USEFUL);
-  osd_plb.add_u64(
-    l_osd_pg_primary, "numpg_primary",
-    "Placement groups for which this osd is primary");
-  osd_plb.add_u64(
-    l_osd_pg_replica, "numpg_replica",
-    "Placement groups for which this osd is replica");
-  osd_plb.add_u64(
-    l_osd_pg_stray, "numpg_stray",
-    "Placement groups ready to be deleted from this osd");
-  osd_plb.add_u64(
-    l_osd_pg_removing, "numpg_removing",
-    "Placement groups queued for local deletion", "pgsr",
-    PerfCountersBuilder::PRIO_USEFUL);
-  osd_plb.add_u64(
-    l_osd_hb_to, "heartbeat_to_peers", "Heartbeat (ping) peers we send to");
-  osd_plb.add_u64_counter(l_osd_map, "map_messages", "OSD map messages");
-  osd_plb.add_u64_counter(l_osd_mape, "map_message_epochs", "OSD map epochs");
-  osd_plb.add_u64_counter(
-    l_osd_mape_dup, "map_message_epoch_dups", "OSD map duplicates");
-  osd_plb.add_u64_counter(
-    l_osd_waiting_for_map, "messages_delayed_for_map",
-    "Operations waiting for OSD map");
-
-  osd_plb.add_u64_counter(
-    l_osd_map_cache_hit, "osd_map_cache_hit", "osdmap cache hit");
-  osd_plb.add_u64_counter(
-    l_osd_map_cache_miss, "osd_map_cache_miss", "osdmap cache miss");
-  osd_plb.add_u64_counter(
-    l_osd_map_cache_miss_low, "osd_map_cache_miss_low",
-    "osdmap cache miss below cache lower bound");
-  osd_plb.add_u64_avg(
-    l_osd_map_cache_miss_low_avg, "osd_map_cache_miss_low_avg",
-    "osdmap cache miss, avg distance below cache lower bound");
-  osd_plb.add_u64_counter(
-    l_osd_map_bl_cache_hit, "osd_map_bl_cache_hit",
-    "OSDMap buffer cache hits");
-  osd_plb.add_u64_counter(
-    l_osd_map_bl_cache_miss, "osd_map_bl_cache_miss",
-    "OSDMap buffer cache misses");
-
-  osd_plb.add_u64(
-    l_osd_stat_bytes, "stat_bytes", "OSD size", "size",
-    PerfCountersBuilder::PRIO_USEFUL, unit_t(UNIT_BYTES));
-  osd_plb.add_u64(
-    l_osd_stat_bytes_used, "stat_bytes_used", "Used space", "used",
-    PerfCountersBuilder::PRIO_USEFUL, unit_t(UNIT_BYTES));
-  osd_plb.add_u64(l_osd_stat_bytes_avail, "stat_bytes_avail", "Available space", NULL, 0, unit_t(UNIT_BYTES));
-
-  osd_plb.add_u64_counter(
-    l_osd_copyfrom, "copyfrom", "Rados \"copy-from\" operations");
-
-  osd_plb.add_u64_counter(l_osd_tier_promote, "tier_promote", "Tier promotions");
-  osd_plb.add_u64_counter(l_osd_tier_flush, "tier_flush", "Tier flushes");
-  osd_plb.add_u64_counter(
-    l_osd_tier_flush_fail, "tier_flush_fail", "Failed tier flushes");
-  osd_plb.add_u64_counter(
-    l_osd_tier_try_flush, "tier_try_flush", "Tier flush attempts");
-  osd_plb.add_u64_counter(
-    l_osd_tier_try_flush_fail, "tier_try_flush_fail",
-    "Failed tier flush attempts");
-  osd_plb.add_u64_counter(
-    l_osd_tier_evict, "tier_evict", "Tier evictions");
-  osd_plb.add_u64_counter(
-    l_osd_tier_whiteout, "tier_whiteout", "Tier whiteouts");
-  osd_plb.add_u64_counter(
-    l_osd_tier_dirty, "tier_dirty", "Dirty tier flag set");
-  osd_plb.add_u64_counter(
-    l_osd_tier_clean, "tier_clean", "Dirty tier flag cleaned");
-  osd_plb.add_u64_counter(
-    l_osd_tier_delay, "tier_delay", "Tier delays (agent waiting)");
-  osd_plb.add_u64_counter(
-    l_osd_tier_proxy_read, "tier_proxy_read", "Tier proxy reads");
-  osd_plb.add_u64_counter(
-    l_osd_tier_proxy_write, "tier_proxy_write", "Tier proxy writes");
-
-  osd_plb.add_u64_counter(
-    l_osd_agent_wake, "agent_wake", "Tiering agent wake up");
-  osd_plb.add_u64_counter(
-    l_osd_agent_skip, "agent_skip", "Objects skipped by agent");
-  osd_plb.add_u64_counter(
-    l_osd_agent_flush, "agent_flush", "Tiering agent flushes");
-  osd_plb.add_u64_counter(
-    l_osd_agent_evict, "agent_evict", "Tiering agent evictions");
-
-  osd_plb.add_u64_counter(
-    l_osd_object_ctx_cache_hit, "object_ctx_cache_hit", "Object context cache hits");
-  osd_plb.add_u64_counter(
-    l_osd_object_ctx_cache_total, "object_ctx_cache_total", "Object context cache lookups");
-
-  osd_plb.add_u64_counter(l_osd_op_cache_hit, "op_cache_hit");
-  osd_plb.add_time_avg(
-    l_osd_tier_flush_lat, "osd_tier_flush_lat", "Object flush latency");
-  osd_plb.add_time_avg(
-    l_osd_tier_promote_lat, "osd_tier_promote_lat", "Object promote latency");
-  osd_plb.add_time_avg(
-    l_osd_tier_r_lat, "osd_tier_r_lat", "Object proxy read latency");
-
-  osd_plb.add_u64_counter(
-    l_osd_pg_info, "osd_pg_info", "PG updated its info (using any method)");
-  osd_plb.add_u64_counter(
-    l_osd_pg_fastinfo, "osd_pg_fastinfo",
-    "PG updated its info using fastinfo attr");
-  osd_plb.add_u64_counter(
-    l_osd_pg_biginfo, "osd_pg_biginfo", "PG updated its biginfo attr");
-
-  logger = osd_plb.create_perf_counters();
+  PerfCounters* logger = build_osd_logger(cct);
   cct->get_perfcounters_collection()->add(logger);
+  return logger;
 }
 
-void OSD::create_recoverystate_perf()
-{
-  dout(10) << "create_recoverystate_perf" << dendl;
-
-  PerfCountersBuilder rs_perf(cct, "recoverystate_perf", rs_first, rs_last);
-
-  rs_perf.add_time_avg(rs_initial_latency, "initial_latency", "Initial recovery state latency");
-  rs_perf.add_time_avg(rs_started_latency, "started_latency", "Started recovery state latency");
-  rs_perf.add_time_avg(rs_reset_latency, "reset_latency", "Reset recovery state latency");
-  rs_perf.add_time_avg(rs_start_latency, "start_latency", "Start recovery state latency");
-  rs_perf.add_time_avg(rs_primary_latency, "primary_latency", "Primary recovery state latency");
-  rs_perf.add_time_avg(rs_peering_latency, "peering_latency", "Peering recovery state latency");
-  rs_perf.add_time_avg(rs_backfilling_latency, "backfilling_latency", "Backfilling recovery state latency");
-  rs_perf.add_time_avg(rs_waitremotebackfillreserved_latency, "waitremotebackfillreserved_latency", "Wait remote backfill reserved recovery state latency");
-  rs_perf.add_time_avg(rs_waitlocalbackfillreserved_latency, "waitlocalbackfillreserved_latency", "Wait local backfill reserved recovery state latency");
-  rs_perf.add_time_avg(rs_notbackfilling_latency, "notbackfilling_latency", "Notbackfilling recovery state latency");
-  rs_perf.add_time_avg(rs_repnotrecovering_latency, "repnotrecovering_latency", "Repnotrecovering recovery state latency");
-  rs_perf.add_time_avg(rs_repwaitrecoveryreserved_latency, "repwaitrecoveryreserved_latency", "Rep wait recovery reserved recovery state latency");
-  rs_perf.add_time_avg(rs_repwaitbackfillreserved_latency, "repwaitbackfillreserved_latency", "Rep wait backfill reserved recovery state latency");
-  rs_perf.add_time_avg(rs_reprecovering_latency, "reprecovering_latency", "RepRecovering recovery state latency");
-  rs_perf.add_time_avg(rs_activating_latency, "activating_latency", "Activating recovery state latency");
-  rs_perf.add_time_avg(rs_waitlocalrecoveryreserved_latency, "waitlocalrecoveryreserved_latency", "Wait local recovery reserved recovery state latency");
-  rs_perf.add_time_avg(rs_waitremoterecoveryreserved_latency, "waitremoterecoveryreserved_latency", "Wait remote recovery reserved recovery state latency");
-  rs_perf.add_time_avg(rs_recovering_latency, "recovering_latency", "Recovering recovery state latency");
-  rs_perf.add_time_avg(rs_recovered_latency, "recovered_latency", "Recovered recovery state latency");
-  rs_perf.add_time_avg(rs_clean_latency, "clean_latency", "Clean recovery state latency");
-  rs_perf.add_time_avg(rs_active_latency, "active_latency", "Active recovery state latency");
-  rs_perf.add_time_avg(rs_replicaactive_latency, "replicaactive_latency", "Replicaactive recovery state latency");
-  rs_perf.add_time_avg(rs_stray_latency, "stray_latency", "Stray recovery state latency");
-  rs_perf.add_time_avg(rs_getinfo_latency, "getinfo_latency", "Getinfo recovery state latency");
-  rs_perf.add_time_avg(rs_getlog_latency, "getlog_latency", "Getlog recovery state latency");
-  rs_perf.add_time_avg(rs_waitactingchange_latency, "waitactingchange_latency", "Waitactingchange recovery state latency");
-  rs_perf.add_time_avg(rs_incomplete_latency, "incomplete_latency", "Incomplete recovery state latency");
-  rs_perf.add_time_avg(rs_down_latency, "down_latency", "Down recovery state latency");
-  rs_perf.add_time_avg(rs_getmissing_latency, "getmissing_latency", "Getmissing recovery state latency");
-  rs_perf.add_time_avg(rs_waitupthru_latency, "waitupthru_latency", "Waitupthru recovery state latency");
-  rs_perf.add_time_avg(rs_notrecovering_latency, "notrecovering_latency", "Notrecovering recovery state latency");
-
-  recoverystate_perf = rs_perf.create_perf_counters();
+PerfCounters* OSD::create_recoverystate_perf()
+{
+  PerfCounters* recoverystate_perf = build_recoverystate_perf(cct);
   cct->get_perfcounters_collection()->add(recoverystate_perf);
+  return recoverystate_perf;
 }
 
 int OSD::shutdown()
 {
-  if (!service.prepare_to_stop())
+  // vstart overwrites osd_fast_shutdown value in the conf file -> force the value here!
+  //cct->_conf->osd_fast_shutdown = true;
+
+  dout(0) << "Fast Shutdown: - cct->_conf->osd_fast_shutdown = "
+         << cct->_conf->osd_fast_shutdown
+         << ", null-fm = " << store->has_null_manager() << dendl;
+
+  utime_t  start_time_func = ceph_clock_now();
+
+  if (cct->_conf->osd_fast_shutdown) {
+    derr << "*** Immediate shutdown (osd_fast_shutdown=true) ***" << dendl;
+    if (cct->_conf->osd_fast_shutdown_notify_mon)
+      service.prepare_to_stop();
+
+    // There is no state we need to keep wehn running in NULL-FM moode
+    if (!store->has_null_manager()) {
+      cct->_log->flush();
+      _exit(0);
+    }
+  } else if (!service.prepare_to_stop()) {
     return 0; // already shutting down
-  osd_lock.Lock();
+  }
+
+  osd_lock.lock();
   if (is_stopping()) {
-    osd_lock.Unlock();
+    osd_lock.unlock();
     return 0;
   }
-  dout(0) << "shutdown" << dendl;
 
+  if (!cct->_conf->osd_fast_shutdown) {
+    dout(0) << "shutdown" << dendl;
+  }
+
+  // don't accept new task for this OSD
   set_state(STATE_STOPPING);
 
-  // Debugging
-  if (cct->_conf.get_val<bool>("osd_debug_shutdown")) {
+  // Disabled debugging during fast-shutdown
+  if (!cct->_conf->osd_fast_shutdown && cct->_conf.get_val<bool>("osd_debug_shutdown")) {
     cct->_conf.set_val("debug_osd", "100");
     cct->_conf.set_val("debug_journal", "100");
     cct->_conf.set_val("debug_filestore", "100");
@@ -3826,6 +4293,45 @@ int OSD::shutdown()
     cct->_conf.apply_changes(nullptr);
   }
 
+  if (cct->_conf->osd_fast_shutdown) {
+    // first, stop new task from being taken from op_shardedwq
+    // and clear all pending tasks
+    op_shardedwq.stop_for_fast_shutdown();
+
+    utime_t  start_time_timer = ceph_clock_now();
+    tick_timer.shutdown();
+    {
+      std::lock_guard l(tick_timer_lock);
+      tick_timer_without_osd_lock.shutdown();
+    }
+
+    osd_lock.unlock();
+    utime_t  start_time_osd_drain = ceph_clock_now();
+
+    // then, wait on osd_op_tp to drain (TBD: should probably add a timeout)
+    osd_op_tp.drain();
+    osd_op_tp.stop();
+
+    utime_t  start_time_umount = ceph_clock_now();
+    store->prepare_for_fast_shutdown();
+    std::lock_guard lock(osd_lock);
+    // TBD: assert in allocator that nothing is being add
+    store->umount();
+
+    utime_t end_time = ceph_clock_now();
+    if (cct->_conf->osd_fast_shutdown_timeout) {
+      ceph_assert(end_time - start_time_func < cct->_conf->osd_fast_shutdown_timeout);
+    }
+    dout(0) <<"Fast Shutdown duration total     :" << end_time              - start_time_func       << " seconds" << dendl;
+    dout(0) <<"Fast Shutdown duration osd_drain :" << start_time_umount     - start_time_osd_drain  << " seconds" << dendl;
+    dout(0) <<"Fast Shutdown duration umount    :" << end_time              - start_time_umount     << " seconds" << dendl;
+    dout(0) <<"Fast Shutdown duration timer     :" << start_time_osd_drain  - start_time_timer      << " seconds" << dendl;
+    cct->_log->flush();
+
+    // now it is safe to exit
+    _exit(0);
+  }
+
   // stop MgrClient earlier as it's more like an internal consumer of OSD
   mgrc.shutdown();
 
@@ -3860,31 +4366,34 @@ int OSD::shutdown()
   delete test_ops_hook;
   test_ops_hook = NULL;
 
-  osd_lock.Unlock();
+  osd_lock.unlock();
 
-  heartbeat_lock.Lock();
-  heartbeat_stop = true;
-  heartbeat_cond.Signal();
-  heartbeat_lock.Unlock();
+  {
+    std::lock_guard l{heartbeat_lock};
+    heartbeat_stop = true;
+    heartbeat_cond.notify_all();
+    heartbeat_peers.clear();
+  }
   heartbeat_thread.join();
 
+  hb_back_server_messenger->mark_down_all();
+  hb_front_server_messenger->mark_down_all();
+  hb_front_client_messenger->mark_down_all();
+  hb_back_client_messenger->mark_down_all();
+
   osd_op_tp.drain();
   osd_op_tp.stop();
   dout(10) << "op sharded tp stopped" << dendl;
 
-  command_tp.drain();
-  command_tp.stop();
-  dout(10) << "command tp stopped" << dendl;
-
   dout(10) << "stopping agent" << dendl;
   service.agent_stop();
 
   boot_finisher.wait_for_empty();
 
-  osd_lock.Lock();
+  osd_lock.lock();
 
   boot_finisher.stop();
-  reset_heartbeat_peers();
+  reset_heartbeat_peers(true);
 
   tick_timer.shutdown();
 
@@ -3894,9 +4403,9 @@ int OSD::shutdown()
   }
 
   // note unmount epoch
-  dout(10) << "noting clean unmount in epoch " << osdmap->get_epoch() << dendl;
+  dout(10) << "noting clean unmount in epoch " << get_osdmap_epoch() << dendl;
   superblock.mounted = service.get_boot_epoch();
-  superblock.clean_thru = osdmap->get_epoch();
+  superblock.clean_thru = get_osdmap_epoch();
   ObjectStore::Transaction t;
   write_superblock(t);
   int r = store->queue_transaction(service.meta_ch, std::move(t));
@@ -3942,9 +4451,9 @@ int OSD::shutdown()
   service.dump_live_pgids();
 #endif
 
-  osd_lock.Unlock();
+  osd_lock.unlock();
   cct->_conf.remove_observer(this);
-  osd_lock.Lock();
+  osd_lock.lock();
 
   service.meta_ch.reset();
 
@@ -3957,12 +4466,11 @@ int OSD::shutdown()
   }
 
   monc->shutdown();
-  osd_lock.Unlock();
-
-  map_lock.get_write();
-  osdmap = OSDMapRef();
-  map_lock.put_write();
-
+  osd_lock.unlock();
+  {
+    std::unique_lock l{map_lock};
+    set_osdmap(OSDMapRef());
+  }
   for (auto s : shards) {
     std::lock_guard l(s->osdmap_lock);
     s->shard_osdmap = OSDMapRef();
@@ -3971,13 +4479,12 @@ int OSD::shutdown()
 
   std::lock_guard lock(osd_lock);
   store->umount();
-  delete store;
-  store = nullptr;
+  store.reset();
   dout(10) << "Store synced" << dendl;
 
   op_tracker.on_shutdown();
 
-  class_handler->shutdown();
+  ClassHandler::get_instance().shutdown();
   client_messenger->shutdown();
   cluster_messenger->shutdown();
   hb_front_client_messenger->shutdown();
@@ -3986,6 +4493,11 @@ int OSD::shutdown()
   hb_front_server_messenger->shutdown();
   hb_back_server_messenger->shutdown();
 
+  utime_t duration = ceph_clock_now() - start_time_func;
+  dout(0) <<"Slow Shutdown duration:" << duration << " seconds" << dendl;
+
+  tracing::osd::tracer.shutdown();
+
   return r;
 }
 
@@ -4051,21 +4563,13 @@ int OSD::update_crush_location()
                      double(1ull << 40 /* TB */)));
   }
 
-  std::multimap<string,string> loc = cct->crush_location.get_location();
-  dout(10) << __func__ << " crush location is " << loc << dendl;
+  dout(10) << __func__ << " crush location is " << cct->crush_location << dendl;
 
   string cmd =
     string("{\"prefix\": \"osd crush create-or-move\", ") +
-    string("\"id\": ") + stringify(whoami) + string(", ") +
-    string("\"weight\":") + weight + string(", ") +
-    string("\"args\": [");
-  for (multimap<string,string>::iterator p = loc.begin(); p != loc.end(); ++p) {
-    if (p != loc.begin())
-      cmd += ", ";
-    cmd += "\"" + p->first + "=" + p->second + "\"";
-  }
-  cmd += "]}";
-
+    string("\"id\": ") + stringify(whoami) + ", " +
+    string("\"weight\":") + weight + ", " +
+    string("\"args\": [") + stringify(cct->crush_location) + "]}";
   return mon_cmd_maybe_osd_create(cmd);
 }
 
@@ -4270,7 +4774,7 @@ PG* OSD::_make_pg(
     }
     decode(ec_profile, p);
   }
-  PGPool pool(cct, createmap, pgid.pool(), pi, name);
+  PGPool pool(createmap, pgid.pool(), pi, name);
   PG *pg;
   if (pi.type == pg_pool_t::TYPE_REPLICATED ||
       pi.type == pg_pool_t::TYPE_ERASURE)
@@ -4353,8 +4857,8 @@ bool OSD::try_finish_pg_delete(PG *pg, unsigned old_pg_num)
   // update pg count now since we might not get an osdmap any time soon.
   if (pg->is_primary())
     service.logger->dec(l_osd_pg_primary);
-  else if (pg->is_replica())
-    service.logger->dec(l_osd_pg_replica);
+  else if (pg->is_nonprimary())
+    service.logger->dec(l_osd_pg_replica); // misnomver
   else
     service.logger->dec(l_osd_pg_stray);
 
@@ -4394,7 +4898,7 @@ PGRef OSD::lookup_lock_pg(spg_t pgid)
 
 void OSD::load_pgs()
 {
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
   dout(0) << "load_pgs" << dendl;
 
   {
@@ -4420,10 +4924,10 @@ void OSD::load_pgs()
        ++it) {
     spg_t pgid;
     if (it->is_temp(&pgid) ||
-       (it->is_pg(&pgid) && PG::_has_removal_flag(store, pgid))) {
+       (it->is_pg(&pgid) && PG::_has_removal_flag(store.get(), pgid))) {
       dout(10) << "load_pgs " << *it
               << " removing, legacy or flagged for removal pg" << dendl;
-      recursive_remove_collection(cct, store, pgid, *it);
+      recursive_remove_collection(cct, store.get(), pgid, *it);
       continue;
     }
 
@@ -4434,7 +4938,7 @@ void OSD::load_pgs()
 
     dout(10) << "pgid " << pgid << " coll " << coll_t(pgid) << dendl;
     epoch_t map_epoch = 0;
-    int r = PG::peek_map_epoch(store, pgid, &map_epoch);
+    int r = PG::peek_map_epoch(store.get(), pgid, &map_epoch);
     if (r < 0) {
       derr << __func__ << " unable to peek at " << pgid << " metadata, skipping"
           << dendl;
@@ -4445,7 +4949,7 @@ void OSD::load_pgs()
     if (map_epoch > 0) {
       OSDMapRef pgosdmap = service.try_get_map(map_epoch);
       if (!pgosdmap) {
-       if (!osdmap->have_pg_pool(pgid.pool())) {
+       if (!get_osdmap()->have_pg_pool(pgid.pool())) {
          derr << __func__ << ": could not find map for epoch " << map_epoch
               << " on pg " << pgid << ", but the pool is not present in the "
               << "current map, so this is probably a result of bug 10617.  "
@@ -4461,10 +4965,10 @@ void OSD::load_pgs()
       }
       pg = _make_pg(pgosdmap, pgid);
     } else {
-      pg = _make_pg(osdmap, pgid);
+      pg = _make_pg(get_osdmap(), pgid);
     }
     if (!pg) {
-      recursive_remove_collection(cct, store, pgid, *it);
+      recursive_remove_collection(cct, store.get(), pgid, *it);
       continue;
     }
 
@@ -4474,13 +4978,13 @@ void OSD::load_pgs()
     pg->ch = store->open_collection(pg->coll);
 
     // read pg state, log
-    pg->read_state(store);
+    pg->read_state(store.get());
 
     if (pg->dne())  {
       dout(10) << "load_pgs " << *it << " deleting dne" << dendl;
       pg->ch = nullptr;
       pg->unlock();
-      recursive_remove_collection(cct, store, pgid, *it);
+      recursive_remove_collection(cct, store.get(), pgid, *it);
       continue;
     }
     {
@@ -4489,8 +4993,6 @@ void OSD::load_pgs()
       store->set_collection_commit_queue(pg->coll, &(shards[shard_index]->context_queue));
     }
 
-    pg->reg_next_scrub();
-
     dout(10) << __func__ << " loaded " << *pg << dendl;
     pg->unlock();
 
@@ -4511,8 +5013,6 @@ PGRef OSD::handle_pg_create_info(const OSDMapRef& osdmap,
     return nullptr;
   }
 
-  PG::RecoveryCtx rctx = create_context();
-
   OSDMapRef startmap = get_map(info->epoch);
 
   if (info->by_mon) {
@@ -4522,7 +5022,7 @@ PGRef OSD::handle_pg_create_info(const OSDMapRef& osdmap,
       dout(10) << __func__ << " ignoring " << pgid << ", pool dne" << dendl;
       return nullptr;
     }
-    if (osdmap->require_osd_release >= CEPH_RELEASE_NAUTILUS &&
+    if (osdmap->require_osd_release >= ceph_release_t::nautilus &&
        !pool->has_flag(pg_pool_t::FLAG_CREATING)) {
       // this ensures we do not process old creating messages after the
       // pool's initial pgs have been created (and pg are subsequently
@@ -4546,13 +5046,12 @@ PGRef OSD::handle_pg_create_info(const OSDMapRef& osdmap,
                 << "the pool allows ec overwrites but is not stored in "
                 << "bluestore, so deep scrubbing will not detect bitrot";
   }
-  PG::_create(*rctx.transaction, pgid, pgid.get_split_bits(pp->get_pg_num()));
-  PG::_init(*rctx.transaction, pgid, pp);
+  PeeringCtx rctx;
+  create_pg_collection(
+    rctx.transaction, pgid, pgid.get_split_bits(pp->get_pg_num()));
+  init_pg_ondisk(rctx.transaction, pgid, pp);
 
-  int role = startmap->calc_pg_role(whoami, acting, acting.size());
-  if (!pp->is_replicated() && role != pgid.shard) {
-    role = -1;
-  }
+  int role = startmap->calc_pg_role(pg_shard_t(whoami, pgid.shard), acting);
 
   PGRef pg = _make_pg(startmap, pgid);
   pg->ch = store->create_new_collection(pg->coll);
@@ -4576,16 +5075,17 @@ PGRef OSD::handle_pg_create_info(const OSDMapRef& osdmap,
     acting_primary,
     info->history,
     info->past_intervals,
-    false,
     rctx.transaction);
 
+  pg->init_collection_pool_opts();
+
   if (pg->is_primary()) {
-    Mutex::Locker locker(m_perf_queries_lock);
+    std::lock_guard locker{m_perf_queries_lock};
     pg->set_dynamic_perf_stats_queries(m_perf_queries);
   }
 
-  pg->handle_initialize(&rctx);
-  pg->handle_activate_map(&rctx);
+  pg->handle_initialize(rctx);
+  pg->handle_activate_map(rctx);
 
   dispatch_context(rctx, pg.get(), osdmap, nullptr);
 
@@ -4609,8 +5109,8 @@ bool OSD::maybe_wait_for_max_pg(const OSDMapRef& osdmap,
   if (is_mon_create) {
     pending_creates_from_mon++;
   } else {
-    bool is_primary = osdmap->get_pg_acting_rank(pgid.pgid, whoami) == 0;
-    pending_creates_from_osd.emplace(pgid.pgid, is_primary);
+    bool is_primary = osdmap->get_pg_acting_role(pgid, whoami) == 0;
+    pending_creates_from_osd.emplace(pgid, is_primary);
   }
   dout(1) << __func__ << " withhold creation of pg " << pgid
          << ": " << num_pgs << " >= "<< max_pgs_per_osd << dendl;
@@ -4659,8 +5159,8 @@ void OSD::resume_creating_pg()
     while (spare_pgs > 0 && pg != pending_creates_from_osd.cend()) {
       dout(20) << __func__ << " pg " << pg->first << dendl;
       vector<int> acting;
-      osdmap->pg_to_up_acting_osds(pg->first, nullptr, nullptr, &acting, nullptr);
-      service.queue_want_pg_temp(pg->first, twiddle(acting), true);
+      get_osdmap()->pg_to_up_acting_osds(pg->first.pgid, nullptr, nullptr, &acting, nullptr);
+      service.queue_want_pg_temp(pg->first.pgid, twiddle(acting), true);
       pg = pending_creates_from_osd.erase(pg);
       do_sub_pg_creates = true;
       spare_pgs--;
@@ -4677,7 +5177,7 @@ void OSD::resume_creating_pg()
       do_renew_subs = true;
     }
   }
-  version_t start = osdmap->get_epoch() + 1;
+  version_t start = get_osdmap_epoch() + 1;
   if (have_pending_creates) {
     // don't miss any new osdmap deleting PGs
     if (monc->sub_want("osdmap", start, 0)) {
@@ -4710,14 +5210,7 @@ void OSD::build_initial_pg_history(
   PastIntervals *pi)
 {
   dout(10) << __func__ << " " << pgid << " created " << created << dendl;
-  h->epoch_created = created;
-  h->epoch_pool_created = created;
-  h->same_interval_since = created;
-  h->same_up_since = created;
-  h->same_primary_since = created;
-  h->last_scrub_stamp = created_stamp;
-  h->last_deep_scrub_stamp = created_stamp;
-  h->last_clean_scrub_stamp = created_stamp;
+  *h = pg_history_t(created, created_stamp);
 
   OSDMapRef lastmap = service.get_map(created);
   int up_primary, acting_primary;
@@ -4726,7 +5219,7 @@ void OSD::build_initial_pg_history(
     pgid.pgid, &up, &up_primary, &acting, &acting_primary);
 
   ostringstream debug;
-  for (epoch_t e = created + 1; e <= osdmap->get_epoch(); ++e) {
+  for (epoch_t e = created + 1; e <= get_osdmap_epoch(); ++e) {
     OSDMapRef osdmap = service.get_map(e);
     int new_up_primary, new_acting_primary;
     vector<int> new_up, new_acting;
@@ -4751,10 +5244,10 @@ void OSD::build_initial_pg_history(
       up, new_up,
       h->same_interval_since,
       h->last_epoch_clean,
-      osdmap,
-      lastmap,
+      osdmap.get(),
+      lastmap.get(),
       pgid.pgid,
-      &min_size_predicate,
+      min_size_predicate,
       pi,
       &debug);
     if (new_interval) {
@@ -4792,31 +5285,37 @@ void OSD::_add_heartbeat_peer(int p)
 
   map<int,HeartbeatInfo>::iterator i = heartbeat_peers.find(p);
   if (i == heartbeat_peers.end()) {
-    pair<ConnectionRef,ConnectionRef> cons = service.get_con_osd_hb(p, osdmap->get_epoch());
+    pair<ConnectionRef,ConnectionRef> cons = service.get_con_osd_hb(p, get_osdmap_epoch());
     if (!cons.first)
       return;
+    assert(cons.second);
+
     hi = &heartbeat_peers[p];
     hi->peer = p;
-    RefCountedPtr s{new HeartbeatSession{p}, false};
+
+    auto stamps = service.get_hb_stamps(p);
+
+    auto sb = ceph::make_ref<Session>(cct, cons.first.get());
+    sb->peer = p;
+    sb->stamps = stamps;
+    hi->hb_interval_start = ceph_clock_now();
     hi->con_back = cons.first.get();
-    hi->con_back->set_priv(s);
-    if (cons.second) {
-      hi->con_front = cons.second.get();
-      hi->con_front->set_priv(s);
-      dout(10) << "_add_heartbeat_peer: new peer osd." << p
-              << " " << hi->con_back->get_peer_addr()
-              << " " << hi->con_front->get_peer_addr()
-              << dendl;
-    } else {
-      hi->con_front.reset(NULL);
-      dout(10) << "_add_heartbeat_peer: new peer osd." << p
-              << " " << hi->con_back->get_peer_addr()
-              << dendl;
-    }
+    hi->con_back->set_priv(sb);
+
+    auto sf = ceph::make_ref<Session>(cct, cons.second.get());
+    sf->peer = p;
+    sf->stamps = stamps;
+    hi->con_front = cons.second.get();
+    hi->con_front->set_priv(sf);
+
+    dout(10) << "_add_heartbeat_peer: new peer osd." << p
+            << " " << hi->con_back->get_peer_addr()
+            << " " << hi->con_front->get_peer_addr()
+            << dendl;
   } else {
     hi = &i->second;
   }
-  hi->epoch = osdmap->get_epoch();
+  hi->epoch = get_osdmap_epoch();
 }
 
 void OSD::_remove_heartbeat_peer(int n)
@@ -4827,10 +5326,7 @@ void OSD::_remove_heartbeat_peer(int n)
           << " " << q->second.con_back->get_peer_addr()
           << " " << (q->second.con_front ? q->second.con_front->get_peer_addr() : entity_addr_t())
           << dendl;
-  q->second.con_back->mark_down();
-  if (q->second.con_front) {
-    q->second.con_front->mark_down();
-  }
+  q->second.clear_mark_down();
   heartbeat_peers.erase(q);
 }
 
@@ -4844,7 +5340,7 @@ void OSD::need_heartbeat_peer_update()
 
 void OSD::maybe_update_heartbeat_peers()
 {
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
 
   if (is_waiting_for_healthy() || is_active()) {
     utime_t now = ceph_clock_now();
@@ -4857,9 +5353,9 @@ void OSD::maybe_update_heartbeat_peers()
        dout(10) << "maybe_update_heartbeat_peers forcing update after " << dur << " seconds" << dendl;
        heartbeat_set_peers_need_update();
        last_heartbeat_resample = now;
-        if (is_waiting_for_healthy()) {
-         reset_heartbeat_peers();   // we want *new* peers!
-        }
+       // automatically clean up any stale heartbeat peers
+       // if we are unhealthy, then clean all
+       reset_heartbeat_peers(is_waiting_for_healthy());
       }
     }
   }
@@ -4879,7 +5375,7 @@ void OSD::maybe_update_heartbeat_peers()
     _get_pgs(&pgs);
     for (auto& pg : pgs) {
       pg->with_heartbeat_peers([&](int peer) {
-         if (osdmap->is_up(peer)) {
+         if (get_osdmap()->is_up(peer)) {
            _add_heartbeat_peer(peer);
          }
        });
@@ -4888,10 +5384,10 @@ void OSD::maybe_update_heartbeat_peers()
 
   // include next and previous up osds to ensure we have a fully-connected set
   set<int> want, extras;
-  const int next = osdmap->get_next_up_osd_after(whoami);
+  const int next = get_osdmap()->get_next_up_osd_after(whoami);
   if (next >= 0)
     want.insert(next);
-  int prev = osdmap->get_previous_up_osd_before(whoami);
+  int prev = get_osdmap()->get_previous_up_osd_before(whoami);
   if (prev >= 0 && prev != next)
     want.insert(prev);
 
@@ -4899,8 +5395,9 @@ void OSD::maybe_update_heartbeat_peers()
   // subtree level (e.g., hosts) for fast failure detection.
   auto min_down = cct->_conf.get_val<uint64_t>("mon_osd_min_down_reporters");
   auto subtree = cct->_conf.get_val<string>("mon_osd_reporter_subtree_level");
-  osdmap->get_random_up_osds_by_subtree(
-    whoami, subtree, min_down, want, &want);
+  auto limit = std::max(min_down, (uint64_t)cct->_conf->osd_heartbeat_min_peers);
+  get_osdmap()->get_random_up_osds_by_subtree(
+    whoami, subtree, limit, want, &want);
 
   for (set<int>::iterator p = want.begin(); p != want.end(); ++p) {
     dout(10) << " adding neighbor peer osd." << *p << dendl;
@@ -4911,13 +5408,13 @@ void OSD::maybe_update_heartbeat_peers()
   // remove down peers; enumerate extras
   map<int,HeartbeatInfo>::iterator p = heartbeat_peers.begin();
   while (p != heartbeat_peers.end()) {
-    if (!osdmap->is_up(p->first)) {
+    if (!get_osdmap()->is_up(p->first)) {
       int o = p->first;
       ++p;
       _remove_heartbeat_peer(o);
       continue;
     }
-    if (p->second.epoch < osdmap->get_epoch()) {
+    if (p->second.epoch < get_osdmap_epoch()) {
       extras.insert(p->first);
     }
     ++p;
@@ -4932,7 +5429,7 @@ void OSD::maybe_update_heartbeat_peers()
       extras.insert(n);
       _add_heartbeat_peer(n);
     }
-    n = osdmap->get_next_up_osd_after(n);
+    n = get_osdmap()->get_next_up_osd_after(n);
     if (n == next)
       break;  // came full circle; stop
   }
@@ -4947,48 +5444,79 @@ void OSD::maybe_update_heartbeat_peers()
   }
 
   dout(10) << "maybe_update_heartbeat_peers " << heartbeat_peers.size() << " peers, extras " << extras << dendl;
+
+  // clean up stale failure pending
+  for (auto it = failure_pending.begin(); it != failure_pending.end();) {
+    if (heartbeat_peers.count(it->first) == 0) {
+      send_still_alive(get_osdmap_epoch(), it->first, it->second.second);
+      failure_pending.erase(it++);
+    } else {
+      it++;
+    }
+  }
 }
 
-void OSD::reset_heartbeat_peers()
+void OSD::reset_heartbeat_peers(bool all)
 {
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
   dout(10) << "reset_heartbeat_peers" << dendl;
+  utime_t stale = ceph_clock_now();
+  stale -= cct->_conf.get_val<int64_t>("osd_heartbeat_stale");
   std::lock_guard l(heartbeat_lock);
-  while (!heartbeat_peers.empty()) {
-    HeartbeatInfo& hi = heartbeat_peers.begin()->second;
-    hi.con_back->mark_down();
-    if (hi.con_front) {
-      hi.con_front->mark_down();
+  for (auto it = heartbeat_peers.begin(); it != heartbeat_peers.end();) {
+    auto& [peer, hi] = *it;
+    if (all || hi.is_stale(stale)) {
+      hi.clear_mark_down();
+      // stop sending failure_report to mon too
+      failure_queue.erase(peer);
+      failure_pending.erase(peer);
+      it = heartbeat_peers.erase(it);
+    } else {
+      ++it;
     }
-    heartbeat_peers.erase(heartbeat_peers.begin());
   }
-  failure_queue.clear();
 }
 
 void OSD::handle_osd_ping(MOSDPing *m)
 {
   if (superblock.cluster_fsid != m->fsid) {
     dout(20) << "handle_osd_ping from " << m->get_source_inst()
-            << " bad fsid " << m->fsid << " != " << superblock.cluster_fsid << dendl;
+            << " bad fsid " << m->fsid << " != " << superblock.cluster_fsid
+            << dendl;
     m->put();
     return;
   }
 
   int from = m->get_source().num();
 
-  heartbeat_lock.Lock();
+  heartbeat_lock.lock();
   if (is_stopping()) {
-    heartbeat_lock.Unlock();
+    heartbeat_lock.unlock();
     m->put();
     return;
   }
 
+  utime_t now = ceph_clock_now();
+  auto mnow = service.get_mnow();
+  ConnectionRef con(m->get_connection());
   OSDMapRef curmap = service.get_osdmap();
   if (!curmap) {
-    heartbeat_lock.Unlock();
+    heartbeat_lock.unlock();
+    m->put();
+    return;
+  }
+
+  auto sref = con->get_priv();
+  Session *s = static_cast<Session*>(sref.get());
+  if (!s) {
+    heartbeat_lock.unlock();
     m->put();
     return;
   }
+  if (!s->stamps) {
+    s->peer = from;
+    s->stamps = service.get_hb_stamps(from);
+  }
 
   switch (m->op) {
 
@@ -5018,23 +5546,38 @@ void OSD::handle_osd_ping(MOSDPing *m)
        }
       }
 
+      ceph::signedspan sender_delta_ub{};
+      s->stamps->got_ping(
+       m->up_from,
+       mnow,
+       m->mono_send_stamp,
+       m->delta_ub,
+       &sender_delta_ub);
+      dout(20) << __func__ << " new stamps " << *s->stamps << dendl;
+
       if (!cct->get_heartbeat_map()->is_healthy()) {
-       dout(10) << "internal heartbeat not healthy, dropping ping request" << dendl;
+       dout(10) << "internal heartbeat not healthy, dropping ping request"
+                << dendl;
        break;
       }
 
       Message *r = new MOSDPing(monc->get_fsid(),
                                curmap->get_epoch(),
-                               MOSDPing::PING_REPLY, m->stamp,
-                               cct->_conf->osd_heartbeat_min_size);
-      m->get_connection()->send_message(r);
+                               MOSDPing::PING_REPLY,
+                               m->ping_stamp,
+                               m->mono_ping_stamp,
+                               mnow,
+                               service.get_up_epoch(),
+                               cct->_conf->osd_heartbeat_min_size,
+                               sender_delta_ub);
+      con->send_message(r);
 
       if (curmap->is_up(from)) {
-       service.note_peer_epoch(from, m->map_epoch);
        if (is_active()) {
-         ConnectionRef con = service.get_con_osd_cluster(from, curmap->get_epoch());
-         if (con) {
-           service.share_map_peer(from, con.get());
+         ConnectionRef cluster_con = service.get_con_osd_cluster(
+           from, curmap->get_epoch());
+         if (cluster_con) {
+           service.maybe_share_map(cluster_con.get(), curmap, m->map_epoch);
          }
        }
       } else if (!curmap->exists(from) ||
@@ -5043,9 +5586,12 @@ void OSD::handle_osd_ping(MOSDPing *m)
        Message *r = new MOSDPing(monc->get_fsid(),
                                  curmap->get_epoch(),
                                  MOSDPing::YOU_DIED,
-                                 m->stamp,
+                                 m->ping_stamp,
+                                 m->mono_ping_stamp,
+                                 mnow,
+                                 service.get_up_epoch(),
                                  cct->_conf->osd_heartbeat_min_size);
-       m->get_connection()->send_message(r);
+       con->send_message(r);
       }
     }
     break;
@@ -5054,15 +5600,15 @@ void OSD::handle_osd_ping(MOSDPing *m)
     {
       map<int,HeartbeatInfo>::iterator i = heartbeat_peers.find(from);
       if (i != heartbeat_peers.end()) {
-        auto acked = i->second.ping_history.find(m->stamp);
+        auto acked = i->second.ping_history.find(m->ping_stamp);
         if (acked != i->second.ping_history.end()) {
-          utime_t now = ceph_clock_now();
           int &unacknowledged = acked->second.second;
-          if (m->get_connection() == i->second.con_back) {
+          if (con == i->second.con_back) {
             dout(25) << "handle_osd_ping got reply from osd." << from
                      << " first_tx " << i->second.first_tx
                      << " last_tx " << i->second.last_tx
-                     << " last_rx_back " << i->second.last_rx_back << " -> " << now
+                     << " last_rx_back " << i->second.last_rx_back
+                    << " -> " << now
                      << " last_rx_front " << i->second.last_rx_front
                      << dendl;
             i->second.last_rx_back = now;
@@ -5074,12 +5620,13 @@ void OSD::handle_osd_ping(MOSDPing *m)
               ceph_assert(unacknowledged > 0);
               --unacknowledged;
             }
-          } else if (m->get_connection() == i->second.con_front) {
+          } else if (con == i->second.con_front) {
             dout(25) << "handle_osd_ping got reply from osd." << from
                      << " first_tx " << i->second.first_tx
                      << " last_tx " << i->second.last_tx
                      << " last_rx_back " << i->second.last_rx_back
-                     << " last_rx_front " << i->second.last_rx_front << " -> " << now
+                     << " last_rx_front " << i->second.last_rx_front
+                    << " -> " << now
                      << dendl;
             i->second.last_rx_front = now;
             ceph_assert(unacknowledged > 0);
@@ -5089,9 +5636,134 @@ void OSD::handle_osd_ping(MOSDPing *m)
           if (unacknowledged == 0) {
             // succeeded in getting all replies
             dout(25) << "handle_osd_ping got all replies from osd." << from
-                     << " , erase pending ping(sent at " << m->stamp << ")"
+                     << " , erase pending ping(sent at " << m->ping_stamp << ")"
                      << " and older pending ping(s)"
                      << dendl;
+
+#define ROUND_S_TO_USEC(sec) (uint32_t)((sec) * 1000 * 1000 + 0.5)
+           ++i->second.hb_average_count;
+           uint32_t back_pingtime = ROUND_S_TO_USEC(i->second.last_rx_back - m->ping_stamp);
+           i->second.hb_total_back += back_pingtime;
+           if (back_pingtime < i->second.hb_min_back)
+             i->second.hb_min_back = back_pingtime;
+           if (back_pingtime > i->second.hb_max_back)
+             i->second.hb_max_back = back_pingtime;
+           uint32_t front_pingtime = ROUND_S_TO_USEC(i->second.last_rx_front - m->ping_stamp);
+           i->second.hb_total_front += front_pingtime;
+           if (front_pingtime < i->second.hb_min_front)
+             i->second.hb_min_front = front_pingtime;
+           if (front_pingtime > i->second.hb_max_front)
+             i->second.hb_max_front = front_pingtime;
+
+           ceph_assert(i->second.hb_interval_start != utime_t());
+           if (i->second.hb_interval_start == utime_t())
+             i->second.hb_interval_start = now;
+           int64_t hb_avg_time_period = 60;
+           if (cct->_conf.get_val<int64_t>("debug_heartbeat_testing_span")) {
+             hb_avg_time_period = cct->_conf.get_val<int64_t>("debug_heartbeat_testing_span");
+           }
+           if (now - i->second.hb_interval_start >=  utime_t(hb_avg_time_period, 0)) {
+              uint32_t back_avg = i->second.hb_total_back / i->second.hb_average_count;
+              uint32_t back_min = i->second.hb_min_back;
+              uint32_t back_max = i->second.hb_max_back;
+              uint32_t front_avg = i->second.hb_total_front / i->second.hb_average_count;
+              uint32_t front_min = i->second.hb_min_front;
+              uint32_t front_max = i->second.hb_max_front;
+
+             // Reset for new interval
+             i->second.hb_average_count = 0;
+             i->second.hb_interval_start = now;
+             i->second.hb_total_back = i->second.hb_max_back = 0;
+             i->second.hb_min_back =  UINT_MAX;
+             i->second.hb_total_front = i->second.hb_max_front = 0;
+             i->second.hb_min_front = UINT_MAX;
+
+             // Record per osd interace ping times
+             // Based on osd_heartbeat_interval ignoring that it is randomly short than this interval
+             if (i->second.hb_back_pingtime.size() == 0) {
+               ceph_assert(i->second.hb_front_pingtime.size() == 0);
+               for (unsigned k = 0 ; k < hb_vector_size; ++k) {
+                 i->second.hb_back_pingtime.push_back(back_avg);
+                 i->second.hb_back_min.push_back(back_min);
+                 i->second.hb_back_max.push_back(back_max);
+                 i->second.hb_front_pingtime.push_back(front_avg);
+                 i->second.hb_front_min.push_back(front_min);
+                 i->second.hb_front_max.push_back(front_max);
+                 ++i->second.hb_index;
+               }
+             } else {
+               int index = i->second.hb_index & (hb_vector_size - 1);
+               i->second.hb_back_pingtime[index] = back_avg;
+               i->second.hb_back_min[index] = back_min;
+               i->second.hb_back_max[index] = back_max;
+               i->second.hb_front_pingtime[index] = front_avg;
+               i->second.hb_front_min[index] = front_min;
+               i->second.hb_front_max[index] = front_max;
+               ++i->second.hb_index;
+             }
+
+             {
+               std::lock_guard l(service.stat_lock);
+               service.osd_stat.hb_pingtime[from].last_update = now.sec();
+               service.osd_stat.hb_pingtime[from].back_last =  back_pingtime;
+
+               uint32_t total = 0;
+               uint32_t min = UINT_MAX;
+               uint32_t max = 0;
+               uint32_t count = 0;
+               uint32_t which = 0;
+               uint32_t size = (uint32_t)i->second.hb_back_pingtime.size();
+               for (int32_t k = size - 1 ; k >= 0; --k) {
+                 ++count;
+                 int index = (i->second.hb_index + k) % size;
+                 total += i->second.hb_back_pingtime[index];
+                 if (i->second.hb_back_min[index] < min)
+                   min = i->second.hb_back_min[index];
+                 if (i->second.hb_back_max[index] > max)
+                   max = i->second.hb_back_max[index];
+                 if (count == 1 || count == 5 || count == 15) {
+                   service.osd_stat.hb_pingtime[from].back_pingtime[which] = total / count;
+                   service.osd_stat.hb_pingtime[from].back_min[which] = min;
+                   service.osd_stat.hb_pingtime[from].back_max[which] = max;
+                   which++;
+                   if (count == 15)
+                     break;
+                 }
+               }
+
+                if (i->second.con_front != NULL) {
+                 service.osd_stat.hb_pingtime[from].front_last = front_pingtime;
+
+                 total = 0;
+                 min = UINT_MAX;
+                 max = 0;
+                 count = 0;
+                 which = 0;
+                 for (int32_t k = size - 1 ; k >= 0; --k) {
+                   ++count;
+                   int index = (i->second.hb_index + k) % size;
+                   total += i->second.hb_front_pingtime[index];
+                   if (i->second.hb_front_min[index] < min)
+                     min = i->second.hb_front_min[index];
+                   if (i->second.hb_front_max[index] > max)
+                     max = i->second.hb_front_max[index];
+                   if (count == 1 || count == 5 || count == 15) {
+                     service.osd_stat.hb_pingtime[from].front_pingtime[which] = total / count;
+                     service.osd_stat.hb_pingtime[from].front_min[which] = min;
+                     service.osd_stat.hb_pingtime[from].front_max[which] = max;
+                     which++;
+                     if (count == 15)
+                       break;
+                   }
+                 }
+               }
+             }
+           } else {
+               std::lock_guard l(service.stat_lock);
+               service.osd_stat.hb_pingtime[from].back_last =  back_pingtime;
+                if (i->second.con_front != NULL)
+                 service.osd_stat.hb_pingtime[from].front_last = front_pingtime;
+           }
             i->second.ping_history.erase(i->second.ping_history.begin(), ++acked);
           }
 
@@ -5116,7 +5788,7 @@ void OSD::handle_osd_ping(MOSDPing *m)
           }
         } else {
           // old replies, deprecated by newly sent pings.
-          dout(10) << "handle_osd_ping no pending ping(sent at " << m->stamp
+          dout(10) << "handle_osd_ping no pending ping(sent at " << m->ping_stamp
                    << ") is found, treat as covered by newly sent pings "
                    << "and ignore"
                    << dendl;
@@ -5125,14 +5797,20 @@ void OSD::handle_osd_ping(MOSDPing *m)
 
       if (m->map_epoch &&
          curmap->is_up(from)) {
-       service.note_peer_epoch(from, m->map_epoch);
        if (is_active()) {
-         ConnectionRef con = service.get_con_osd_cluster(from, curmap->get_epoch());
-         if (con) {
-           service.share_map_peer(from, con.get());
+         ConnectionRef cluster_con = service.get_con_osd_cluster(
+           from, curmap->get_epoch());
+         if (cluster_con) {
+           service.maybe_share_map(cluster_con.get(), curmap, m->map_epoch);
          }
        }
       }
+
+      s->stamps->got_ping_reply(
+       mnow,
+       m->mono_send_stamp,
+       m->delta_ub);
+      dout(20) << __func__ << " new stamps " << *s->stamps << dendl;
     }
     break;
 
@@ -5143,23 +5821,27 @@ void OSD::handle_osd_ping(MOSDPing *m)
     break;
   }
 
-  heartbeat_lock.Unlock();
+  heartbeat_lock.unlock();
   m->put();
 }
 
 void OSD::heartbeat_entry()
 {
-  std::lock_guard l(heartbeat_lock);
+  std::unique_lock l(heartbeat_lock);
   if (is_stopping())
     return;
   while (!heartbeat_stop) {
     heartbeat();
 
-    double wait = .5 + ((float)(rand() % 10)/10.0) * (float)cct->_conf->osd_heartbeat_interval;
-    utime_t w;
-    w.set_from_double(wait);
+    double wait;
+    if (cct->_conf.get_val<bool>("debug_disable_randomized_ping")) {
+      wait = (float)cct->_conf->osd_heartbeat_interval;
+    } else {
+      wait = .5 + ((float)(rand() % 10)/10.0) * (float)cct->_conf->osd_heartbeat_interval;
+    }
+    auto w = ceph::make_timespan(wait);
     dout(30) << "heartbeat_entry sleeping for " << wait << dendl;
-    heartbeat_cond.WaitInterval(heartbeat_lock, w);
+    heartbeat_cond.wait_for(l, w);
     if (is_stopping())
       return;
     dout(30) << "heartbeat_entry woke up" << dendl;
@@ -5168,7 +5850,7 @@ void OSD::heartbeat_entry()
 
 void OSD::heartbeat_check()
 {
-  ceph_assert(heartbeat_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(heartbeat_lock));
   utime_t now = ceph_clock_now();
 
   // check for incoming heartbeats (move me elsewhere?)
@@ -5217,25 +5899,13 @@ void OSD::heartbeat_check()
 
 void OSD::heartbeat()
 {
-  ceph_assert(heartbeat_lock.is_locked_by_me());
+  ceph_assert(ceph_mutex_is_locked_by_me(heartbeat_lock));
   dout(30) << "heartbeat" << dendl;
 
-  // get CPU load avg
-  double loadavgs[1];
-  int hb_interval = cct->_conf->osd_heartbeat_interval;
-  int n_samples = 86400;
-  if (hb_interval > 1) {
-    n_samples /= hb_interval;
-    if (n_samples < 1)
-      n_samples = 1;
-  }
-
-  if (getloadavg(loadavgs, 1) == 1) {
-    logger->set(l_osd_loadavg, 100 * loadavgs[0]);
-    daily_loadavg = (daily_loadavg * (n_samples - 1) + loadavgs[0]) / n_samples;
-    dout(30) << "heartbeat: daily_loadavg " << daily_loadavg << dendl;
+  auto load_for_logger = service.get_scrub_services().update_load_average();
+  if (load_for_logger) {
+    logger->set(l_osd_loadavg, load_for_logger.value());
   }
-
   dout(30) << "heartbeat checking stats" << dendl;
 
   // refresh peer list and osd stats
@@ -5255,6 +5925,7 @@ void OSD::heartbeat()
   service.check_full_status(ratio, pratio);
 
   utime_t now = ceph_clock_now();
+  auto mnow = service.get_mnow();
   utime_t deadline = now;
   deadline += cct->_conf->osd_heartbeat_grace;
 
@@ -5263,22 +5934,46 @@ void OSD::heartbeat()
        i != heartbeat_peers.end();
        ++i) {
     int peer = i->first;
+    Session *s = static_cast<Session*>(i->second.con_back->get_priv().get());
+    if (!s) {
+      dout(30) << "heartbeat osd." << peer << " has no open con" << dendl;
+      continue;
+    }
+    dout(30) << "heartbeat sending ping to osd." << peer << dendl;
+
     i->second.last_tx = now;
     if (i->second.first_tx == utime_t())
       i->second.first_tx = now;
     i->second.ping_history[now] = make_pair(deadline,
       HeartbeatInfo::HEARTBEAT_MAX_CONN);
-    dout(30) << "heartbeat sending ping to osd." << peer << dendl;
-    i->second.con_back->send_message(new MOSDPing(monc->get_fsid(),
-                                         service.get_osdmap_epoch(),
-                                         MOSDPing::PING, now,
-                                         cct->_conf->osd_heartbeat_min_size));
+    if (i->second.hb_interval_start == utime_t())
+      i->second.hb_interval_start = now;
+
+    std::optional<ceph::signedspan> delta_ub;
+    s->stamps->sent_ping(&delta_ub);
+
+    i->second.con_back->send_message(
+      new MOSDPing(monc->get_fsid(),
+                  service.get_osdmap_epoch(),
+                  MOSDPing::PING,
+                  now,
+                  mnow,
+                  mnow,
+                  service.get_up_epoch(),
+                  cct->_conf->osd_heartbeat_min_size,
+                  delta_ub));
 
     if (i->second.con_front)
-      i->second.con_front->send_message(new MOSDPing(monc->get_fsid(),
-                                            service.get_osdmap_epoch(),
-                                            MOSDPing::PING, now,
-                                         cct->_conf->osd_heartbeat_min_size));
+      i->second.con_front->send_message(
+       new MOSDPing(monc->get_fsid(),
+                    service.get_osdmap_epoch(),
+                    MOSDPing::PING,
+                    now,
+                    mnow,
+                    mnow,
+                    service.get_up_epoch(),
+                    cct->_conf->osd_heartbeat_min_size,
+                    delta_ub));
   }
 
   logger->set(l_osd_hb_to, heartbeat_peers.size());
@@ -5289,7 +5984,7 @@ void OSD::heartbeat()
     if (now - last_mon_heartbeat > cct->_conf->osd_mon_heartbeat_interval && is_active()) {
       last_mon_heartbeat = now;
       dout(10) << "i have no heartbeat peers; checking mon for new map" << dendl;
-      osdmap_subscribe(osdmap->get_epoch() + 1, false);
+      osdmap_subscribe(get_osdmap_epoch() + 1, false);
     }
   }
 
@@ -5300,26 +5995,20 @@ bool OSD::heartbeat_reset(Connection *con)
 {
   std::lock_guard l(heartbeat_lock);
   auto s = con->get_priv();
+  dout(20) << __func__ << " con " << con << " s " << s.get() << dendl;
   con->set_priv(nullptr);
   if (s) {
     if (is_stopping()) {
       return true;
     }
-    auto heartbeat_session = static_cast<HeartbeatSession*>(s.get());
-    auto p = heartbeat_peers.find(heartbeat_session->peer);
+    auto session = static_cast<Session*>(s.get());
+    auto p = heartbeat_peers.find(session->peer);
     if (p != heartbeat_peers.end() &&
        (p->second.con_back == con ||
         p->second.con_front == con)) {
       dout(10) << "heartbeat_reset failed hb con " << con << " for osd." << p->second.peer
               << ", reopening" << dendl;
-      if (con != p->second.con_back) {
-       p->second.con_back->mark_down();
-      }
-      p->second.con_back.reset(NULL);
-      if (p->second.con_front && con != p->second.con_front) {
-       p->second.con_front->mark_down();
-      }
-      p->second.con_front.reset(NULL);
+      p->second.clear_mark_down(con);
       pair<ConnectionRef,ConnectionRef> newcon = service.get_con_osd_hb(p->second.peer, p->second.epoch);
       if (newcon.first) {
        p->second.con_back = newcon.first.get();
@@ -5347,40 +6036,69 @@ bool OSD::heartbeat_reset(Connection *con)
 
 void OSD::tick()
 {
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
   dout(10) << "tick" << dendl;
 
+  utime_t now = ceph_clock_now();
+  // throw out any obsolete markdown log
+  utime_t grace = utime_t(cct->_conf->osd_max_markdown_period, 0);
+  while (!osd_markdown_log.empty() &&
+          osd_markdown_log.front() + grace < now)
+    osd_markdown_log.pop_front();
+
   if (is_active() || is_waiting_for_healthy()) {
     maybe_update_heartbeat_peers();
   }
 
   if (is_waiting_for_healthy()) {
     start_boot();
-    if (is_waiting_for_healthy()) {
-      // failed to boot
-      std::lock_guard l(heartbeat_lock);
-      utime_t now = ceph_clock_now();
-      if (now - last_mon_heartbeat > cct->_conf->osd_mon_heartbeat_interval) {
-        last_mon_heartbeat = now;
-        dout(1) << __func__ << " checking mon for new map" << dendl;
-        osdmap_subscribe(osdmap->get_epoch() + 1, false);
-      }
+  }
+
+  if (is_waiting_for_healthy() || is_booting()) {
+    std::lock_guard l(heartbeat_lock);
+    if (now - last_mon_heartbeat > cct->_conf->osd_mon_heartbeat_interval) {
+      last_mon_heartbeat = now;
+      dout(1) << __func__ << " checking mon for new map" << dendl;
+      osdmap_subscribe(get_osdmap_epoch() + 1, false);
     }
   }
 
   do_waiters();
 
+  // scrub purged_snaps every deep scrub interval
+  {
+    const utime_t last = superblock.last_purged_snaps_scrub;
+    utime_t next = last;
+    next += cct->_conf->osd_scrub_min_interval;
+    std::mt19937 rng;
+    // use a seed that is stable for each scrub interval, but varies
+    // by OSD to avoid any herds.
+    rng.seed(whoami + superblock.last_purged_snaps_scrub.sec());
+    double r = (rng() % 1024) / 1024.0;
+    next +=
+      cct->_conf->osd_scrub_min_interval *
+      cct->_conf->osd_scrub_interval_randomize_ratio * r;
+    if (next < ceph_clock_now()) {
+      dout(20) << __func__ << " last_purged_snaps_scrub " << last
+              << " next " << next << " ... now" << dendl;
+      scrub_purged_snaps();
+    } else {
+      dout(20) << __func__ << " last_purged_snaps_scrub " << last
+              << " next " << next << dendl;
+    }
+  }
+
   tick_timer.add_event_after(get_tick_interval(), new C_Tick(this));
 }
 
 void OSD::tick_without_osd_lock()
 {
-  ceph_assert(tick_timer_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(tick_timer_lock));
   dout(10) << "tick_without_osd_lock" << dendl;
 
-  logger->set(l_osd_cached_crc, buffer::get_cached_crc());
-  logger->set(l_osd_cached_crc_adjusted, buffer::get_cached_crc_adjusted());
-  logger->set(l_osd_missed_crc, buffer::get_missed_crc());
+  logger->set(l_osd_cached_crc, ceph::buffer::get_cached_crc());
+  logger->set(l_osd_cached_crc_adjusted, ceph::buffer::get_cached_crc_adjusted());
+  logger->set(l_osd_missed_crc, ceph::buffer::get_missed_crc());
 
   // refresh osd stats
   struct store_statfs_t stbuf;
@@ -5392,11 +6110,11 @@ void OSD::tick_without_osd_lock()
   // osd_lock is not being held, which means the OSD state
   // might change when doing the monitor report
   if (is_active() || is_waiting_for_healthy()) {
-    heartbeat_lock.Lock();
-    heartbeat_check();
-    heartbeat_lock.Unlock();
-
-    map_lock.get_read();
+    {
+      std::lock_guard l{heartbeat_lock};
+      heartbeat_check();
+    }
+    map_lock.lock_shared();
     std::lock_guard l(mon_report_lock);
 
     // mon report?
@@ -5407,7 +6125,7 @@ void OSD::tick_without_osd_lock()
       send_full_update();
       send_failures();
     }
-    map_lock.put_read();
+    map_lock.unlock_shared();
 
     epoch_t max_waiting_epoch = 0;
     for (auto s : shards) {
@@ -5433,7 +6151,7 @@ void OSD::tick_without_osd_lock()
       // borrow lec lock to pretect last_sent_beacon from changing
       std::lock_guard l{min_last_epoch_clean_lock};
       const auto elapsed = now - last_sent_beacon;
-      if (chrono::duration_cast<chrono::seconds>(elapsed).count() >
+      if (std::chrono::duration_cast<std::chrono::seconds>(elapsed).count() >
         cct->_conf->osd_beacon_report_interval) {
         need_send_beacon = true;
       }
@@ -5478,7 +6196,7 @@ void TestOpsSocketHook::test_ops(OSDService *service, ObjectStore *store,
 
     string poolstr;
 
-    cmd_getval(service->cct, cmdmap, "pool", poolstr);
+    cmd_getval(cmdmap, "pool", poolstr);
     pool = curmap->lookup_pg_pool_name(poolstr);
     //If we can't find it by name then maybe id specified
     if (pool < 0 && isdigit(poolstr[0]))
@@ -5489,7 +6207,7 @@ void TestOpsSocketHook::test_ops(OSDService *service, ObjectStore *store,
     }
 
     string objname, nspace;
-    cmd_getval(service->cct, cmdmap, "objname", objname);
+    cmd_getval(cmdmap, "objname", objname);
     std::size_t found = objname.find_first_of('/');
     if (found != string::npos) {
       nspace = objname.substr(0, found);
@@ -5503,8 +6221,7 @@ void TestOpsSocketHook::test_ops(OSDService *service, ObjectStore *store,
       return;
     }
 
-    int64_t shardid;
-    cmd_getval(service->cct, cmdmap, "shardid", shardid, int64_t(shard_id_t::NO_SHARD));
+    int64_t shardid = cmd_getval_or<int64_t>(cmdmap, "shardid", shard_id_t::NO_SHARD);
     hobject_t obj(object_t(objname), string(""), CEPH_NOSNAP, rawpg.ps(), pool, nspace);
     ghobject_t gobj(obj, ghobject_t::NO_GEN, shard_id_t(uint8_t(shardid)));
     spg_t pgid(curmap->raw_pg_to_pg(rawpg), shard_id_t(shardid));
@@ -5521,8 +6238,8 @@ void TestOpsSocketHook::test_ops(OSDService *service, ObjectStore *store,
       map<string, bufferlist> newattrs;
       bufferlist val;
       string key, valstr;
-      cmd_getval(service->cct, cmdmap, "key", key);
-      cmd_getval(service->cct, cmdmap, "val", valstr);
+      cmd_getval(cmdmap, "key", key);
+      cmd_getval(cmdmap, "val", valstr);
 
       val.append(valstr);
       newattrs[key] = val;
@@ -5534,11 +6251,9 @@ void TestOpsSocketHook::test_ops(OSDService *service, ObjectStore *store,
         ss << "ok";
     } else if (command == "rmomapkey") {
       string key;
-      set<string> keys;
-      cmd_getval(service->cct, cmdmap, "key", key);
+      cmd_getval(cmdmap, "key", key);
 
-      keys.insert(key);
-      t.omap_rmkeys(coll_t(pgid), ghobject_t(obj), keys);
+      t.omap_rmkey(coll_t(pgid), ghobject_t(obj), key);
       r = store->queue_transaction(service->meta_ch, std::move(t));
       if (r < 0)
         ss << "error=" << r;
@@ -5548,7 +6263,7 @@ void TestOpsSocketHook::test_ops(OSDService *service, ObjectStore *store,
       bufferlist newheader;
       string headerstr;
 
-      cmd_getval(service->cct, cmdmap, "header", headerstr);
+      cmd_getval(cmdmap, "header", headerstr);
       newheader.append(headerstr);
       t.omap_setheader(coll_t(pgid), ghobject_t(obj), newheader);
       r = store->queue_transaction(service->meta_ch, std::move(t));
@@ -5578,7 +6293,7 @@ void TestOpsSocketHook::test_ops(OSDService *service, ObjectStore *store,
       }
     } else if (command == "truncobj") {
       int64_t trunclen;
-      cmd_getval(service->cct, cmdmap, "len", trunclen);
+      cmd_getval(cmdmap, "len", trunclen);
       t.truncate(coll_t(pgid), ghobject_t(obj), trunclen);
       r = store->queue_transaction(service->meta_ch, std::move(t));
       if (r < 0)
@@ -5595,8 +6310,7 @@ void TestOpsSocketHook::test_ops(OSDService *service, ObjectStore *store,
     return;
   }
   if (command == "set_recovery_delay") {
-    int64_t delay;
-    cmd_getval(service->cct, cmdmap, "utime", delay, (int64_t)0);
+    int64_t delay = cmd_getval_or<int64_t>(cmdmap, "utime", 0);
     ostringstream oss;
     oss << delay;
     int r = service->cct->_conf.set_val("osd_recovery_delay_start",
@@ -5612,69 +6326,11 @@ void TestOpsSocketHook::test_ops(OSDService *service, ObjectStore *store,
        << "to " << service->cct->_conf->osd_recovery_delay_start;
     return;
   }
-  if (command ==  "trigger_scrub" || command == "trigger_deep_scrub") {
-    spg_t pgid;
-    bool deep = (command == "trigger_deep_scrub");
-    OSDMapRef curmap = service->get_osdmap();
-
-    string pgidstr;
-
-    cmd_getval(service->cct, cmdmap, "pgid", pgidstr);
-    if (!pgid.parse(pgidstr.c_str())) {
-      ss << "Invalid pgid specified";
-      return;
-    }
-
-    int64_t time;
-    cmd_getval(service->cct, cmdmap, "time", time, (int64_t)0);
-
-    PGRef pg = service->osd->_lookup_lock_pg(pgid);
-    if (pg == nullptr) {
-      ss << "Can't find pg " << pgid;
-      return;
-    }
-
-    if (pg->is_primary()) {
-      pg->unreg_next_scrub();
-      const pg_pool_t *p = curmap->get_pg_pool(pgid.pool());
-      double pool_scrub_max_interval = 0;
-      double scrub_max_interval;
-      if (deep) {
-        p->opts.get(pool_opts_t::DEEP_SCRUB_INTERVAL, &pool_scrub_max_interval);
-        scrub_max_interval = pool_scrub_max_interval > 0 ?
-          pool_scrub_max_interval : g_conf()->osd_deep_scrub_interval;
-      } else {
-        p->opts.get(pool_opts_t::SCRUB_MAX_INTERVAL, &pool_scrub_max_interval);
-        scrub_max_interval = pool_scrub_max_interval > 0 ?
-          pool_scrub_max_interval : g_conf()->osd_scrub_max_interval;
-      }
-      // Instead of marking must_scrub force a schedule scrub
-      utime_t stamp = ceph_clock_now();
-      if (time == 0)
-        stamp -= scrub_max_interval;
-      else
-        stamp -=  (float)time;
-      stamp -= 100.0;  // push back last scrub more for good measure
-      if (deep) {
-        pg->set_last_deep_scrub_stamp(stamp);
-      } else {
-        pg->set_last_scrub_stamp(stamp);
-      }
-      pg->reg_next_scrub();
-      pg->publish_stats_to_osd();
-      ss << "ok - set" << (deep ? " deep" : "" ) << " stamp " << stamp;
-    } else {
-      ss << "Not primary";
-    }
-    pg->unlock();
-    return;
-  }
   if (command == "injectfull") {
-    int64_t count;
-    string type;
+    int64_t count = cmd_getval_or<int64_t>(cmdmap, "count", -1);
+    string type = cmd_getval_or<string>(cmdmap, "type", "full");
     OSDService::s_names state;
-    cmd_getval(service->cct, cmdmap, "type", type, string("full"));
-    cmd_getval(service->cct, cmdmap, "count", count, (int64_t)-1);
+
     if (type == "none" || count == 0) {
       type = "none";
       count = 0;
@@ -5706,7 +6362,7 @@ void OSD::ms_handle_connect(Connection *con)
     } else if (is_booting()) {
       _send_boot();       // resend boot message
     } else {
-      map_lock.get_read();
+      map_lock.lock_shared();
       std::lock_guard l2(mon_report_lock);
 
       utime_t now = ceph_clock_now();
@@ -5723,7 +6379,7 @@ void OSD::ms_handle_connect(Connection *con)
       requeue_failures();
       send_failures();
 
-      map_lock.put_read();
+      map_lock.unlock_shared();
       if (is_active()) {
        send_beacon(ceph::coarse_mono_clock::now());
       }
@@ -5740,11 +6396,9 @@ void OSD::ms_handle_fast_connect(Connection *con)
 {
   if (con->get_peer_type() != CEPH_ENTITY_TYPE_MON &&
       con->get_peer_type() != CEPH_ENTITY_TYPE_MGR) {
-    auto priv = con->get_priv();
-    auto s = static_cast<Session*>(priv.get());
-    if (!s) {
-      s = new Session{cct, con};
-      con->set_priv(RefCountedPtr{s, false});
+    if (auto s = ceph::ref_cast<Session>(con->get_priv()); !s) {
+      s = ceph::make_ref<Session>(cct, con);
+      con->set_priv(s);
       dout(10) << " new session (outgoing) " << s << " con=" << s->con
           << " addr=" << s->con->get_peer_addr() << dendl;
       // we don't connect to clients
@@ -5758,11 +6412,9 @@ void OSD::ms_handle_fast_accept(Connection *con)
 {
   if (con->get_peer_type() != CEPH_ENTITY_TYPE_MON &&
       con->get_peer_type() != CEPH_ENTITY_TYPE_MGR) {
-    auto priv = con->get_priv();
-    auto s = static_cast<Session*>(priv.get());
-    if (!s) {
-      s = new Session{cct, con};
-      con->set_priv(RefCountedPtr{s, false});
+    if (auto s = ceph::ref_cast<Session>(con->get_priv()); !s) {
+      s = ceph::make_ref<Session>(cct, con);
+      con->set_priv(s);
       dout(10) << "new session (incoming)" << s << " con=" << con
           << " addr=" << con->get_peer_addr()
           << " must have raced with connect" << dendl;
@@ -5774,9 +6426,8 @@ void OSD::ms_handle_fast_accept(Connection *con)
 
 bool OSD::ms_handle_reset(Connection *con)
 {
-  auto s = con->get_priv();
-  auto session = static_cast<Session*>(s.get());
-  dout(2) << "ms_handle_reset con " << con << " session " << session << dendl;
+  auto session = ceph::ref_cast<Session>(con->get_priv());
+  dout(2) << "ms_handle_reset con " << con << " session " << session.get() << dendl;
   if (!session)
     return false;
   session->wstate.reset(con);
@@ -5785,7 +6436,7 @@ bool OSD::ms_handle_reset(Connection *con)
   // note that we break session->con *before* the session_handle_reset
   // cleanup below.  this avoids a race between us and
   // PG::add_backoff, Session::check_backoff, etc.
-  session_handle_reset(SessionRef{session});
+  session_handle_reset(session);
   return true;
 }
 
@@ -5794,9 +6445,8 @@ bool OSD::ms_handle_refused(Connection *con)
   if (!cct->_conf->osd_fast_fail_on_connection_refused)
     return false;
 
-  auto priv = con->get_priv();
-  auto session = static_cast<Session*>(priv.get());
-  dout(2) << "ms_handle_refused con " << con << " session " << session << dendl;
+  auto session = ceph::ref_cast<Session>(con->get_priv());
+  dout(2) << "ms_handle_refused con " << con << " session " << session.get() << dendl;
   if (!session)
     return false;
   int type = con->get_peer_type();
@@ -5823,12 +6473,12 @@ bool OSD::ms_handle_refused(Connection *con)
   return true;
 }
 
-struct C_OSD_GetVersion : public Context {
+struct CB_OSD_GetVersion {
   OSD *osd;
-  uint64_t oldest, newest;
-  explicit C_OSD_GetVersion(OSD *o) : osd(o), oldest(0), newest(0) {}
-  void finish(int r) override {
-    if (r >= 0)
+  explicit CB_OSD_GetVersion(OSD *o) : osd(o) {}
+  void operator ()(boost::system::error_code ec, version_t newest,
+                  version_t oldest) {
+    if (!ec)
       osd->_got_mon_epochs(oldest, newest);
   }
 };
@@ -5848,8 +6498,7 @@ void OSD::start_boot()
   set_state(STATE_PREBOOT);
   dout(10) << "start_boot - have maps " << superblock.oldest_map
           << ".." << superblock.newest_map << dendl;
-  C_OSD_GetVersion *c = new C_OSD_GetVersion(this);
-  monc->get_version("osdmap", &c->newest, &c->oldest, c);
+  monc->get_version("osdmap", CB_OSD_GetVersion(this));
 }
 
 void OSD::_got_mon_epochs(epoch_t oldest, epoch_t newest)
@@ -5872,6 +6521,8 @@ void OSD::_preboot(epoch_t oldest, epoch_t newest)
     heartbeat();
   }
 
+  const auto& monmap = monc->monmap;
+  const auto osdmap = get_osdmap();
   // if our map within recent history, try to add ourselves to the osdmap.
   if (osdmap->get_epoch() == 0) {
     derr << "waiting for initial osdmap" << dendl;
@@ -5887,12 +6538,14 @@ void OSD::_preboot(epoch_t oldest, epoch_t newest)
   } else if (!osdmap->test_flag(CEPH_OSDMAP_SORTBITWISE)) {
     derr << "osdmap SORTBITWISE OSDMap flag is NOT set; please set it"
         << dendl;
-  } else if (osdmap->require_osd_release < CEPH_RELEASE_LUMINOUS) {
-    derr << "osdmap require_osd_release < luminous; please upgrade to luminous"
-        << dendl;
   } else if (service.need_fullness_update()) {
     derr << "osdmap fullness state needs update" << dendl;
     send_full_update();
+  } else if (monmap.min_mon_release >= ceph_release_t::octopus &&
+            superblock.purged_snaps_last < superblock.current_epoch) {
+    dout(10) << __func__ << " purged_snaps_last " << superblock.purged_snaps_last
+            << " < newest_map " << superblock.current_epoch << dendl;
+    _get_purged_snaps();
   } else if (osdmap->get_epoch() >= oldest - 1 &&
             osdmap->get_epoch() + cct->_conf->osd_map_message_max > newest) {
 
@@ -5900,17 +6553,17 @@ void OSD::_preboot(epoch_t oldest, epoch_t newest)
     // this thread might be required for splitting and merging PGs to
     // make progress.
     boot_finisher.queue(
-      new FunctionContext(
+      new LambdaContext(
        [this](int r) {
-         std::lock_guard l(osd_lock);
+         std::unique_lock l(osd_lock);
          if (is_preboot()) {
            dout(10) << __func__ << " waiting for peering work to drain"
                     << dendl;
-           osd_lock.Unlock();
+           l.unlock();
            for (auto shard : shards) {
-             shard->wait_min_pg_epoch(osdmap->get_epoch());
+             shard->wait_min_pg_epoch(get_osdmap_epoch());
            }
-           osd_lock.Lock();
+           l.lock();
          }
          if (is_preboot()) {
            _send_boot();
@@ -5918,12 +6571,51 @@ void OSD::_preboot(epoch_t oldest, epoch_t newest)
        }));
     return;
   }
-
-  // get all the latest maps
-  if (osdmap->get_epoch() + 1 >= oldest)
-    osdmap_subscribe(osdmap->get_epoch() + 1, false);
-  else
-    osdmap_subscribe(oldest - 1, true);
+
+  // get all the latest maps
+  if (osdmap->get_epoch() + 1 >= oldest)
+    osdmap_subscribe(osdmap->get_epoch() + 1, false);
+  else
+    osdmap_subscribe(oldest - 1, true);
+}
+
+void OSD::_get_purged_snaps()
+{
+  // NOTE: this is a naive, stateless implementaiton.  it may send multiple
+  // overlapping requests to the mon, which will be somewhat inefficient, but
+  // it should be reliable.
+  dout(10) << __func__ << " purged_snaps_last " << superblock.purged_snaps_last
+          << ", newest_map " << superblock.current_epoch << dendl;
+  MMonGetPurgedSnaps *m = new MMonGetPurgedSnaps(
+    superblock.purged_snaps_last + 1,
+    superblock.current_epoch + 1);
+  monc->send_mon_message(m);
+}
+
+void OSD::handle_get_purged_snaps_reply(MMonGetPurgedSnapsReply *m)
+{
+  dout(10) << __func__ << " " << *m << dendl;
+  ObjectStore::Transaction t;
+  if (!is_preboot() ||
+      m->last < superblock.purged_snaps_last) {
+    goto out;
+  }
+  SnapMapper::record_purged_snaps(cct, store.get(), service.meta_ch,
+                                 make_purged_snaps_oid(), &t,
+                                 m->purged_snaps);
+  superblock.purged_snaps_last = m->last;
+  write_superblock(t);
+  store->queue_transaction(
+    service.meta_ch,
+    std::move(t));
+  service.publish_superblock(superblock);
+  if (m->last < superblock.current_epoch) {
+    _get_purged_snaps();
+  } else {
+    start_boot();
+  }
+out:
+  m->put();
 }
 
 void OSD::send_full_update()
@@ -5941,7 +6633,7 @@ void OSD::send_full_update()
   set<string> s;
   OSDMap::calc_state_set(state, s);
   dout(10) << __func__ << " want state " << s << dendl;
-  monc->send_mon_message(new MOSDFull(osdmap->get_epoch(), state));
+  monc->send_mon_message(new MOSDFull(get_osdmap_epoch(), state));
 }
 
 void OSD::start_waiting_for_healthy()
@@ -5951,7 +6643,7 @@ void OSD::start_waiting_for_healthy()
   last_heartbeat_resample = utime_t();
 
   // subscribe to osdmap updates, in case our peers really are known to be dead
-  osdmap_subscribe(osdmap->get_epoch() + 1, false);
+  osdmap_subscribe(get_osdmap_epoch() + 1, false);
 }
 
 bool OSD::_is_healthy()
@@ -5963,13 +6655,10 @@ bool OSD::_is_healthy()
 
   if (is_waiting_for_healthy()) {
      utime_t now = ceph_clock_now();
-     utime_t grace = utime_t(cct->_conf->osd_max_markdown_period, 0);
-     while (!osd_markdown_log.empty() &&
-             osd_markdown_log.front() + grace < now)
-       osd_markdown_log.pop_front();
-     if (osd_markdown_log.size() <= 1) {
-       dout(5) << __func__ << " first time marked as down,"
-               << " try reboot unconditionally" << dendl;
+     if (osd_markdown_log.empty()) {
+       dout(5) << __func__ << " force returning true since last markdown"
+               << " was " << cct->_conf->osd_max_markdown_period
+               << "s ago" << dendl;
        return true;
     }
     std::lock_guard l(heartbeat_lock);
@@ -6072,6 +6761,12 @@ void OSD::_collect_metadata(map<string,string> *pm)
   (*pm)["rotational"] = store_is_rotational ? "1" : "0";
   (*pm)["journal_rotational"] = journal_is_rotational ? "1" : "0";
   (*pm)["default_device_class"] = store->get_default_device_class();
+  string osdspec_affinity;
+  int r = store->read_meta("osdspec_affinity", &osdspec_affinity);
+  if (r < 0 || osdspec_affinity.empty()) {
+    osdspec_affinity = "";
+  }
+  (*pm)["osdspec_affinity"] = osdspec_affinity;
   store->collect_metadata(pm);
 
   collect_sys_info(pm, cct);
@@ -6123,31 +6818,19 @@ void OSD::_collect_metadata(map<string,string> *pm)
 
   set<string> devnames;
   store->get_devices(&devnames);
-  (*pm)["devices"] = stringify(devnames);
-  string devids;
-  for (auto& dev : devnames) {
-    string err;
-    string id = get_device_id(dev, &err);
-    if (id.size()) {
-      if (!devids.empty()) {
-       devids += ",";
-      }
-      devids += dev + "=" + id;
-    } else {
-      dout(10) << __func__ << " no unique device id for " << dev << ": "
-              << err << dendl;
-    }
+  map<string,string> errs;
+  get_device_metadata(devnames, pm, &errs);
+  for (auto& i : errs) {
+    dout(1) << __func__ << " " << i.first << ": " << i.second << dendl;
   }
-  (*pm)["device_ids"] = devids;
-
   dout(10) << __func__ << " " << *pm << dendl;
 }
 
 void OSD::queue_want_up_thru(epoch_t want)
 {
-  map_lock.get_read();
-  epoch_t cur = osdmap->get_up_thru(whoami);
-  std::lock_guard l(mon_report_lock);
+  std::shared_lock map_locker{map_lock};
+  epoch_t cur = get_osdmap()->get_up_thru(whoami);
+  std::lock_guard report_locker(mon_report_lock);
   if (want > up_thru_wanted) {
     dout(10) << "queue_want_up_thru now " << want << " (was " << up_thru_wanted << ")"
             << ", currently " << cur
@@ -6159,12 +6842,12 @@ void OSD::queue_want_up_thru(epoch_t want)
             << ", currently " << cur
             << dendl;
   }
-  map_lock.put_read();
 }
 
 void OSD::send_alive()
 {
-  ceph_assert(mon_report_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(mon_report_lock));
+  const auto osdmap = get_osdmap();
   if (!osdmap->exists(whoami))
     return;
   epoch_t up_thru = osdmap->get_up_thru(whoami);
@@ -6180,7 +6863,7 @@ void OSD::request_full_map(epoch_t first, epoch_t last)
   dout(10) << __func__ << " " << first << ".." << last
           << ", previously requested "
           << requested_full_first << ".." << requested_full_last << dendl;
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
   ceph_assert(first > 0 && last > 0);
   ceph_assert(first <= last);
   ceph_assert(first >= requested_full_first);  // we shouldn't ever ask for older maps
@@ -6204,7 +6887,7 @@ void OSD::request_full_map(epoch_t first, epoch_t last)
 void OSD::got_full_map(epoch_t e)
 {
   ceph_assert(requested_full_first <= requested_full_last);
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
   if (requested_full_first == 0) {
     dout(20) << __func__ << " " << e << ", nothing requested" << dendl;
     return;
@@ -6221,7 +6904,7 @@ void OSD::got_full_map(epoch_t e)
     requested_full_first = requested_full_last = 0;
     return;
   }
-  
+
   requested_full_first = e + 1;
 
   dout(10) << __func__ << " " << e << ", requested " << requested_full_first
@@ -6244,10 +6927,11 @@ void OSD::requeue_failures()
 
 void OSD::send_failures()
 {
-  ceph_assert(map_lock.is_locked());
-  ceph_assert(mon_report_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(map_lock));
+  ceph_assert(ceph_mutex_is_locked(mon_report_lock));
   std::lock_guard l(heartbeat_lock);
   utime_t now = ceph_clock_now();
+  const auto osdmap = get_osdmap();
   while (!failure_queue.empty()) {
     int osd = failure_queue.begin()->first;
     if (!failure_pending.count(osd)) {
@@ -6280,7 +6964,7 @@ void OSD::cancel_pending_failures()
   while (it != failure_pending.end()) {
     dout(10) << __func__ << " canceling in-flight failure report for osd."
              << it->first << dendl;
-    send_still_alive(osdmap->get_epoch(), it->first, it->second.second);
+    send_still_alive(get_osdmap_epoch(), it->first, it->second.second);
     failure_pending.erase(it++);
   }
 }
@@ -6297,8 +6981,11 @@ void OSD::send_beacon(const ceph::coarse_mono_clock::time_point& now)
     MOSDBeacon* beacon = nullptr;
     {
       std::lock_guard l{min_last_epoch_clean_lock};
-      beacon = new MOSDBeacon(osdmap->get_epoch(), min_last_epoch_clean);
-      std::swap(beacon->pgs, min_last_epoch_clean_pgs);
+      beacon = new MOSDBeacon(get_osdmap_epoch(),
+                             min_last_epoch_clean,
+                             superblock.last_purged_snaps_scrub,
+                             cct->_conf->osd_beacon_report_interval);
+      beacon->pgs = min_last_epoch_clean_pgs;
       last_sent_beacon = now;
     }
     monc->send_mon_message(beacon);
@@ -6307,698 +6994,95 @@ void OSD::send_beacon(const ceph::coarse_mono_clock::time_point& now)
   }
 }
 
-void OSD::handle_command(MMonCommand *m)
-{
-  if (!require_mon_peer(m)) {
-    m->put();
-    return;
-  }
-
-  Command *c = new Command(m->cmd, m->get_tid(), m->get_data(), NULL);
-  command_wq.queue(c);
-  m->put();
-}
-
 void OSD::handle_command(MCommand *m)
 {
   ConnectionRef con = m->get_connection();
-  auto priv = con->get_priv();
-  auto session = static_cast<Session *>(priv.get());
+  auto session = ceph::ref_cast<Session>(con->get_priv());
   if (!session) {
-    con->send_message(new MCommandReply(m, -EPERM));
+    con->send_message(new MCommandReply(m, -EACCES));
     m->put();
     return;
   }
-
-  OSDCap& caps = session->caps;
-  priv.reset();
-
-  if (!caps.allow_all() || m->get_source().is_mon()) {
-    con->send_message(new MCommandReply(m, -EPERM));
+  if (!session->caps.allow_all()) {
+    con->send_message(new MCommandReply(m, -EACCES));
     m->put();
-    return;
-  }
-
-  Command *c = new Command(m->cmd, m->get_tid(), m->get_data(), con.get());
-  command_wq.queue(c);
-
-  m->put();
-}
-
-struct OSDCommand {
-  string cmdstring;
-  string helpstring;
-  string module;
-  string perm;
-} osd_commands[] = {
-
-#define COMMAND(parsesig, helptext, module, perm) \
-  {parsesig, helptext, module, perm},
-
-// yes, these are really pg commands, but there's a limit to how
-// much work it's worth.  The OSD returns all of them.  Make this
-// form (pg <pgid> <cmd>) valid only for the cli.
-// Rest uses "tell <pgid> <cmd>"
-
-COMMAND("pg " \
-       "name=pgid,type=CephPgid " \
-       "name=cmd,type=CephChoices,strings=query", \
-       "show details of a specific pg", "osd", "r")
-COMMAND("pg " \
-       "name=pgid,type=CephPgid " \
-       "name=cmd,type=CephChoices,strings=mark_unfound_lost " \
-       "name=mulcmd,type=CephChoices,strings=revert|delete", \
-       "mark all unfound objects in this pg as lost, either removing or reverting to a prior version if one is available",
-       "osd", "rw")
-COMMAND("pg " \
-       "name=pgid,type=CephPgid " \
-       "name=cmd,type=CephChoices,strings=list_unfound " \
-       "name=offset,type=CephString,req=false",
-       "list unfound objects on this pg, perhaps starting at an offset given in JSON",
-       "osd", "r")
-
-// new form: tell <pgid> <cmd> for both cli and rest
-
-COMMAND("query",
-       "show details of a specific pg", "osd", "r")
-COMMAND("mark_unfound_lost " \
-       "name=mulcmd,type=CephChoices,strings=revert|delete", \
-       "mark all unfound objects in this pg as lost, either removing or reverting to a prior version if one is available",
-       "osd", "rw")
-COMMAND("list_unfound " \
-       "name=offset,type=CephString,req=false",
-       "list unfound objects on this pg, perhaps starting at an offset given in JSON",
-       "osd", "r")
-COMMAND("perf histogram dump "
-        "name=logger,type=CephString,req=false "
-        "name=counter,type=CephString,req=false",
-       "Get histogram data",
-       "osd", "r")
-
-// tell <osd.n> commands.  Validation of osd.n must be special-cased in client
-COMMAND("version", "report version of OSD", "osd", "r")
-COMMAND("get_command_descriptions", "list commands descriptions", "osd", "r")
-COMMAND("injectargs " \
-       "name=injected_args,type=CephString,n=N",
-       "inject configuration arguments into running OSD",
-       "osd", "rw")
-COMMAND("config set " \
-       "name=key,type=CephString name=value,type=CephString",
-       "Set a configuration option at runtime (not persistent)",
-       "osd", "rw")
-COMMAND("config get " \
-       "name=key,type=CephString",
-       "Get a configuration option at runtime",
-       "osd", "r")
-COMMAND("config unset " \
-       "name=key,type=CephString",
-       "Unset a configuration option at runtime (not persistent)",
-       "osd", "rw")
-COMMAND("cluster_log " \
-       "name=level,type=CephChoices,strings=error,warning,info,debug " \
-       "name=message,type=CephString,n=N",
-       "log a message to the cluster log",
-       "osd", "rw")
-COMMAND("bench " \
-       "name=count,type=CephInt,req=false " \
-       "name=size,type=CephInt,req=false " \
-       "name=object_size,type=CephInt,req=false " \
-       "name=object_num,type=CephInt,req=false ", \
-       "OSD benchmark: write <count> <size>-byte objects(with <obj_size> <obj_num>), " \
-       "(default count=1G default size=4MB). Results in log.",
-       "osd", "rw")
-COMMAND("flush_pg_stats", "flush pg stats", "osd", "rw")
-COMMAND("heap " \
-       "name=heapcmd,type=CephChoices,strings="\
-           "dump|start_profiler|stop_profiler|release|get_release_rate|set_release_rate|stats " \
-       "name=value,type=CephString,req=false",
-       "show heap usage info (available only if compiled with tcmalloc)",
-       "osd", "rw")
-COMMAND("debug dump_missing " \
-       "name=filename,type=CephFilepath",
-       "dump missing objects to a named file", "osd", "r")
-COMMAND("debug kick_recovery_wq " \
-       "name=delay,type=CephInt,range=0",
-       "set osd_recovery_delay_start to <val>", "osd", "rw")
-COMMAND("cpu_profiler " \
-       "name=arg,type=CephChoices,strings=status|flush",
-       "run cpu profiling on daemon", "osd", "rw")
-COMMAND("dump_pg_recovery_stats", "dump pg recovery statistics",
-       "osd", "r")
-COMMAND("reset_pg_recovery_stats", "reset pg recovery statistics",
-       "osd", "rw")
-COMMAND("compact",
-        "compact object store's omap. "
-        "WARNING: Compaction probably slows your requests",
-        "osd", "rw")
-COMMAND("smart name=devid,type=CephString,req=False",
-        "runs smartctl on this osd devices.  ",
-        "osd", "rw")
-COMMAND("cache drop",
-        "Drop all OSD caches",
-        "osd", "rwx")
-COMMAND("cache status",
-        "Get OSD caches statistics",
-        "osd", "r")
-COMMAND("send_beacon",
-        "Send OSD beacon to mon immediately",
-        "osd", "r")
-};
-
-void OSD::do_command(
-  Connection *con, ceph_tid_t tid, vector<string>& cmd, bufferlist& data)
-{
-  dout(20) << "do_command tid " << tid << " " << cmd << dendl;
-
-  int r = 0;
-  stringstream ss, ds;
-  bufferlist odata;
-  cmdmap_t cmdmap;
-  if (cmd.empty()) {
-    ss << "no command given";
-    goto out;
-  }
-  if (!cmdmap_from_json(cmd, &cmdmap, ss)) {
-    r = -EINVAL;
-    goto out;
-  }
-
-  try {
-    r = _do_command(con, cmdmap, tid, data, odata, ss, ds);
-  } catch (const bad_cmd_get& e) {
-    r = -EINVAL;
-    ss << e.what();
-  }
-  if (r == -EAGAIN) {
-    return;
-  }
- out:
-  string rs = ss.str();
-  odata.append(ds);
-  dout(0) << "do_command r=" << r << " " << rs << dendl;
-  clog->info() << rs;
-  if (con) {
-    MCommandReply *reply = new MCommandReply(r, rs);
-    reply->set_tid(tid);
-    reply->set_data(odata);
-    con->send_message(reply);
-  }
-}
-
-namespace {
-  class unlock_guard {
-    Mutex& m;
-  public:
-    explicit unlock_guard(Mutex& mutex)
-      : m(mutex)
-    {
-      m.unlock();
-    }
-    unlock_guard(unlock_guard&) = delete;
-    ~unlock_guard() {
-      m.lock();
-    }
-  };
-}
-
-int OSD::_do_command(
-  Connection *con, cmdmap_t& cmdmap, ceph_tid_t tid, bufferlist& data,
-  bufferlist& odata, stringstream& ss, stringstream& ds)
-{
-  int r = 0;
-  string prefix;
-  string format;
-  string pgidstr;
-  boost::scoped_ptr<Formatter> f;
-
-  cmd_getval(cct, cmdmap, "prefix", prefix);
-
-  if (prefix == "get_command_descriptions") {
-    int cmdnum = 0;
-    JSONFormatter *f = new JSONFormatter();
-    f->open_object_section("command_descriptions");
-    for (OSDCommand *cp = osd_commands;
-        cp < &osd_commands[std::size(osd_commands)]; cp++) {
-
-      ostringstream secname;
-      secname << "cmd" << setfill('0') << std::setw(3) << cmdnum;
-      dump_cmddesc_to_json(f, con->get_features(),
-                           secname.str(), cp->cmdstring, cp->helpstring,
-                          cp->module, cp->perm, 0);
-      cmdnum++;
-    }
-    f->close_section();        // command_descriptions
-
-    f->flush(ds);
-    delete f;
-    goto out;
-  }
-
-  cmd_getval(cct, cmdmap, "format", format);
-  f.reset(Formatter::create(format));
-
-  if (prefix == "version") {
-    if (f) {
-      f->open_object_section("version");
-      f->dump_string("version", pretty_version_to_str());
-      f->close_section();
-      f->flush(ds);
-    } else {
-      ds << pretty_version_to_str();
-    }
-    goto out;
-  }
-  else if (prefix == "injectargs") {
-    vector<string> argsvec;
-    cmd_getval(cct, cmdmap, "injected_args", argsvec);
-
-    if (argsvec.empty()) {
-      r = -EINVAL;
-      ss << "ignoring empty injectargs";
-      goto out;
-    }
-    string args = argsvec.front();
-    for (vector<string>::iterator a = ++argsvec.begin(); a != argsvec.end(); ++a)
-      args += " " + *a;
-    unlock_guard unlock{osd_lock};
-    r = cct->_conf.injectargs(args, &ss);
-  }
-  else if (prefix == "config set") {
-    std::string key;
-    std::string val;
-    cmd_getval(cct, cmdmap, "key", key);
-    cmd_getval(cct, cmdmap, "value", val);
-    unlock_guard unlock{osd_lock};
-    r = cct->_conf.set_val(key, val, &ss);
-    if (r == 0) {
-      cct->_conf.apply_changes(nullptr);
-    }
-  }
-  else if (prefix == "config get") {
-    std::string key;
-    cmd_getval(cct, cmdmap, "key", key);
-    unlock_guard unlock{osd_lock};
-    std::string val;
-    r = cct->_conf.get_val(key, &val);
-    if (r == 0) {
-      ds << val;
-    }
-  }
-  else if (prefix == "config unset") {
-    std::string key;
-    cmd_getval(cct, cmdmap, "key", key);
-    unlock_guard unlock{osd_lock};
-    r = cct->_conf.rm_val(key);
-    if (r == 0) {
-      cct->_conf.apply_changes(nullptr);
-    }
-    if (r == -ENOENT) {
-      r = 0;  // make command idempotent
-    }
-  }
-  else if (prefix == "cluster_log") {
-    vector<string> msg;
-    cmd_getval(cct, cmdmap, "message", msg);
-    if (msg.empty()) {
-      r = -EINVAL;
-      ss << "ignoring empty log message";
-      goto out;
-    }
-    string message = msg.front();
-    for (vector<string>::iterator a = ++msg.begin(); a != msg.end(); ++a)
-      message += " " + *a;
-    string lvl;
-    cmd_getval(cct, cmdmap, "level", lvl);
-    clog_type level = string_to_clog_type(lvl);
-    if (level < 0) {
-      r = -EINVAL;
-      ss << "unknown level '" << lvl << "'";
-      goto out;
-    }
-    clog->do_log(level, message);
-  }
-
-  // either 'pg <pgid> <command>' or
-  // 'tell <pgid>' (which comes in without any of that prefix)?
-
-  else if (prefix == "pg" ||
-           prefix == "query" ||
-           prefix == "mark_unfound_lost" ||
-           prefix == "list_unfound"
-          ) {
-    pg_t pgid;
-
-    if (!cmd_getval(cct, cmdmap, "pgid", pgidstr)) {
-      ss << "no pgid specified";
-      r = -EINVAL;
-    } else if (!pgid.parse(pgidstr.c_str())) {
-      ss << "couldn't parse pgid '" << pgidstr << "'";
-      r = -EINVAL;
-    } else {
-      spg_t pcand;
-      PGRef pg;
-      if (osdmap->get_primary_shard(pgid, &pcand) &&
-         (pg = _lookup_lock_pg(pcand))) {
-       if (pg->is_primary()) {
-         // simulate pg <pgid> cmd= for pg->do-command
-         if (prefix != "pg")
-           cmd_putval(cct, cmdmap, "cmd", prefix);
-         try {
-           r = pg->do_command(cmdmap, ss, data, odata, con, tid);
-         } catch (const bad_cmd_get& e) {
-           pg->unlock();
-           ss << e.what();
-           return -EINVAL;
-         }
-         if (r == -EAGAIN) {
-           pg->unlock();
-           // don't reply, pg will do so async
-           return -EAGAIN;
-         }
-       } else {
-         ss << "not primary for pgid " << pgid;
-
-         // send them the latest diff to ensure they realize the mapping
-         // has changed.
-         service.send_incremental_map(osdmap->get_epoch() - 1, con, osdmap);
-
-         // do not reply; they will get newer maps and realize they
-         // need to resend.
-         pg->unlock();
-         return -EAGAIN;
-       }
-       pg->unlock();
-      } else {
-       ss << "i don't have pgid " << pgid;
-       r = -ENOENT;
-      }
-    }
-  }
-
-  else if (prefix == "bench") {
-    int64_t count;
-    int64_t bsize;
-    int64_t osize, onum;
-    // default count 1G, size 4MB
-    cmd_getval(cct, cmdmap, "count", count, (int64_t)1 << 30);
-    cmd_getval(cct, cmdmap, "size", bsize, (int64_t)4 << 20);
-    cmd_getval(cct, cmdmap, "object_size", osize, (int64_t)0);
-    cmd_getval(cct, cmdmap, "object_num", onum, (int64_t)0);
-
-    uint32_t duration = cct->_conf->osd_bench_duration;
-
-    if (bsize > (int64_t) cct->_conf->osd_bench_max_block_size) {
-      // let us limit the block size because the next checks rely on it
-      // having a sane value.  If we allow any block size to be set things
-      // can still go sideways.
-      ss << "block 'size' values are capped at "
-         << byte_u_t(cct->_conf->osd_bench_max_block_size) << ". If you wish to use"
-         << " a higher value, please adjust 'osd_bench_max_block_size'";
-      r = -EINVAL;
-      goto out;
-    } else if (bsize < (int64_t) (1 << 20)) {
-      // entering the realm of small block sizes.
-      // limit the count to a sane value, assuming a configurable amount of
-      // IOPS and duration, so that the OSD doesn't get hung up on this,
-      // preventing timeouts from going off
-      int64_t max_count =
-        bsize * duration * cct->_conf->osd_bench_small_size_max_iops;
-      if (count > max_count) {
-        ss << "'count' values greater than " << max_count
-           << " for a block size of " << byte_u_t(bsize) << ", assuming "
-           << cct->_conf->osd_bench_small_size_max_iops << " IOPS,"
-           << " for " << duration << " seconds,"
-           << " can cause ill effects on osd. "
-           << " Please adjust 'osd_bench_small_size_max_iops' with a higher"
-           << " value if you wish to use a higher 'count'.";
-        r = -EINVAL;
-        goto out;
-      }
-    } else {
-      // 1MB block sizes are big enough so that we get more stuff done.
-      // However, to avoid the osd from getting hung on this and having
-      // timers being triggered, we are going to limit the count assuming
-      // a configurable throughput and duration.
-      // NOTE: max_count is the total amount of bytes that we believe we
-      //       will be able to write during 'duration' for the given
-      //       throughput.  The block size hardly impacts this unless it's
-      //       way too big.  Given we already check how big the block size
-      //       is, it's safe to assume everything will check out.
-      int64_t max_count =
-        cct->_conf->osd_bench_large_size_max_throughput * duration;
-      if (count > max_count) {
-        ss << "'count' values greater than " << max_count
-           << " for a block size of " << byte_u_t(bsize) << ", assuming "
-           << byte_u_t(cct->_conf->osd_bench_large_size_max_throughput) << "/s,"
-           << " for " << duration << " seconds,"
-           << " can cause ill effects on osd. "
-           << " Please adjust 'osd_bench_large_size_max_throughput'"
-           << " with a higher value if you wish to use a higher 'count'.";
-        r = -EINVAL;
-        goto out;
-      }
-    }
-
-    if (osize && bsize > osize)
-      bsize = osize;
-
-    dout(1) << " bench count " << count
-            << " bsize " << byte_u_t(bsize) << dendl;
-
-    ObjectStore::Transaction cleanupt;
-
-    if (osize && onum) {
-      bufferlist bl;
-      bufferptr bp(osize);
-      bp.zero();
-      bl.push_back(std::move(bp));
-      bl.rebuild_page_aligned();
-      for (int i=0; i<onum; ++i) {
-       char nm[30];
-       snprintf(nm, sizeof(nm), "disk_bw_test_%d", i);
-       object_t oid(nm);
-       hobject_t soid(sobject_t(oid, 0));
-       ObjectStore::Transaction t;
-       t.write(coll_t(), ghobject_t(soid), 0, osize, bl);
-       store->queue_transaction(service.meta_ch, std::move(t), NULL);
-       cleanupt.remove(coll_t(), ghobject_t(soid));
-      }
-    }
-
-    bufferlist bl;
-    bufferptr bp(bsize);
-    bp.zero();
-    bl.push_back(std::move(bp));
-    bl.rebuild_page_aligned();
-
-    {
-      C_SaferCond waiter;
-      if (!service.meta_ch->flush_commit(&waiter)) {
-       waiter.wait();
-      }
-    }
-
-    utime_t start = ceph_clock_now();
-    for (int64_t pos = 0; pos < count; pos += bsize) {
-      char nm[30];
-      unsigned offset = 0;
-      if (onum && osize) {
-       snprintf(nm, sizeof(nm), "disk_bw_test_%d", (int)(rand() % onum));
-       offset = rand() % (osize / bsize) * bsize;
-      } else {
-       snprintf(nm, sizeof(nm), "disk_bw_test_%lld", (long long)pos);
-      }
-      object_t oid(nm);
-      hobject_t soid(sobject_t(oid, 0));
-      ObjectStore::Transaction t;
-      t.write(coll_t::meta(), ghobject_t(soid), offset, bsize, bl);
-      store->queue_transaction(service.meta_ch, std::move(t), NULL);
-      if (!onum || !osize)
-       cleanupt.remove(coll_t::meta(), ghobject_t(soid));
-    }
-
-    {
-      C_SaferCond waiter;
-      if (!service.meta_ch->flush_commit(&waiter)) {
-       waiter.wait();
-      }
-    }
-    utime_t end = ceph_clock_now();
-
-    // clean up
-    store->queue_transaction(service.meta_ch, std::move(cleanupt), NULL);
-    {
-      C_SaferCond waiter;
-      if (!service.meta_ch->flush_commit(&waiter)) {
-       waiter.wait();
-      }
-    }
-
-    double elapsed = end - start;
-    double rate = count / elapsed;
-    double iops = rate / bsize;
-    if (f) {
-      f->open_object_section("osd_bench_results");
-      f->dump_int("bytes_written", count);
-      f->dump_int("blocksize", bsize);
-      f->dump_float("elapsed_sec", elapsed);
-      f->dump_float("bytes_per_sec", rate);
-      f->dump_float("iops", iops);
-      f->close_section();
-      f->flush(ds);
-    } else {
-      ds << "bench: wrote " << byte_u_t(count)
-        << " in blocks of " << byte_u_t(bsize) << " in "
-        << elapsed << " sec at " << byte_u_t(rate) << "/sec "
-        << si_u_t(iops) << " IOPS";
-    }
-  }
-
-  else if (prefix == "flush_pg_stats") {
-    mgrc.send_pgstats();
-    ds << service.get_osd_stat_seq() << "\n";
-  }
-
-  else if (prefix == "heap") {
-    r = ceph::osd_cmds::heap(*cct, cmdmap, *f, ds);
-  }
-
-  else if (prefix == "debug dump_missing") {
-    if (!f) {
-      f.reset(new JSONFormatter(true));
-    }
-    f->open_array_section("pgs");
-    vector<PGRef> pgs;
-    _get_pgs(&pgs);
-    for (auto& pg : pgs) {
-      string s = stringify(pg->pg_id);
-      f->open_array_section(s.c_str());
-      pg->lock();
-      pg->dump_missing(f.get());
-      pg->unlock();
-      f->close_section();
-    }
-    f->close_section();
-    f->flush(ds);
-  }
-  else if (prefix == "debug kick_recovery_wq") {
-    int64_t delay;
-    cmd_getval(cct, cmdmap, "delay", delay);
-    ostringstream oss;
-    oss << delay;
-    unlock_guard unlock{osd_lock};
-    r = cct->_conf.set_val("osd_recovery_delay_start", oss.str().c_str());
-    if (r != 0) {
-      ss << "kick_recovery_wq: error setting "
-        << "osd_recovery_delay_start to '" << delay << "': error "
-        << r;
-      goto out;
-    }
-    cct->_conf.apply_changes(nullptr);
-    ss << "kicking recovery queue. set osd_recovery_delay_start "
-       << "to " << cct->_conf->osd_recovery_delay_start;
-  }
-
-  else if (prefix == "cpu_profiler") {
-    string arg;
-    cmd_getval(cct, cmdmap, "arg", arg);
-    vector<string> argvec;
-    get_str_vec(arg, argvec);
-    cpu_profiler_handle_command(argvec, ds);
-  }
-
-  else if (prefix == "dump_pg_recovery_stats") {
-    stringstream s;
-    if (f) {
-      pg_recovery_stats.dump_formatted(f.get());
-      f->flush(ds);
-    } else {
-      pg_recovery_stats.dump(s);
-      ds << "dump pg recovery stats: " << s.str();
-    }
-  }
-
-  else if (prefix == "reset_pg_recovery_stats") {
-    ss << "reset pg recovery stats";
-    pg_recovery_stats.reset();
-  }
-
-  else if (prefix == "perf histogram dump") {
-    std::string logger;
-    std::string counter;
-    cmd_getval(cct, cmdmap, "logger", logger);
-    cmd_getval(cct, cmdmap, "counter", counter);
-    if (f) {
-      cct->get_perfcounters_collection()->dump_formatted_histograms(
-          f.get(), false, logger, counter);
-      f->flush(ds);
-    }
-  }
-
-  else if (prefix == "compact") {
-    dout(1) << "triggering manual compaction" << dendl;
-    auto start = ceph::coarse_mono_clock::now();
-    store->compact();
-    auto end = ceph::coarse_mono_clock::now();
-    double duration = std::chrono::duration<double>(end-start).count();
-    dout(1) << "finished manual compaction in "
-            << duration
-            << " seconds" << dendl;
-    ss << "compacted omap in " << duration << " seconds";
-  }
-
-  else if (prefix == "smart") {
-    string devid;
-    cmd_getval(cct, cmdmap, "devid", devid);
-    probe_smart(devid, ds);
+    return;
   }
+  cct->get_admin_socket()->queue_tell_command(m);
+  m->put();
+}
 
-  else if (prefix == "cache drop") {
-    dout(20) << "clearing all caches" << dendl;
-    // Clear the objectstore's cache - onode and buffer for Bluestore,
-    // system's pagecache for Filestore
-    r = store->flush_cache(&ss);
-    if (r < 0) {
-      ds << "Error flushing objectstore cache: " << cpp_strerror(r);
-      goto out;
+namespace {
+  class unlock_guard {
+    ceph::mutex& m;
+  public:
+    explicit unlock_guard(ceph::mutex& mutex)
+      : m(mutex)
+    {
+      m.unlock();
     }
-    // Clear the objectcontext cache (per PG)
-    vector<PGRef> pgs;
-    _get_pgs(&pgs);
-    for (auto& pg: pgs) {
-      pg->clear_cache();
+    unlock_guard(unlock_guard&) = delete;
+    ~unlock_guard() {
+      m.lock();
     }
-  }
+  };
+}
 
-  else if (prefix == "cache status") {
-    int obj_ctx_count = 0;
-    vector<PGRef> pgs;
-    _get_pgs(&pgs);
-    for (auto& pg: pgs) {
-      obj_ctx_count += pg->get_cache_obj_count();
+void OSD::scrub_purged_snaps()
+{
+  dout(10) << __func__ << dendl;
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
+  SnapMapper::Scrubber s(cct, store.get(), service.meta_ch,
+                        make_snapmapper_oid(),
+                        make_purged_snaps_oid());
+  clog->debug() << "purged_snaps scrub starts";
+  osd_lock.unlock();
+  s.run();
+  if (s.stray.size()) {
+    clog->debug() << "purged_snaps scrub found " << s.stray.size() << " strays";
+  } else {
+    clog->debug() << "purged_snaps scrub ok";
+  }
+  set<pair<spg_t,snapid_t>> queued;
+  for (auto& [pool, snap, hash, shard] : s.stray) {
+    const pg_pool_t *pi = get_osdmap()->get_pg_pool(pool);
+    if (!pi) {
+      dout(20) << __func__ << " pool " << pool << " dne" << dendl;
+      continue;
     }
-    if (f) {
-      f->open_object_section("cache_status");
-      f->dump_int("object_ctx", obj_ctx_count);
-      store->dump_cache_stats(f.get());
-      f->close_section();
-      f->flush(ds);
-    } else {
-      ds << "object_ctx: " << obj_ctx_count;
-      store->dump_cache_stats(ds);
+    pg_t pgid(pi->raw_hash_to_pg(hash), pool);
+    spg_t spgid(pgid, shard);
+    pair<spg_t,snapid_t> p(spgid, snap);
+    if (queued.count(p)) {
+      dout(20) << __func__ << " pg " << spgid << " snap " << snap
+              << " already queued" << dendl;
+      continue;
     }
-  }
-  else if (prefix == "send_beacon") {
-    if (is_active()) {
-      send_beacon(ceph::coarse_mono_clock::now());
+    PGRef pg = lookup_lock_pg(spgid);
+    if (!pg) {
+      dout(20) << __func__ << " pg " << spgid << " not found" << dendl;
+      continue;
     }
-  } else {
-    ss << "unrecognized command '" << prefix << "'";
-    r = -EINVAL;
+    queued.insert(p);
+    dout(10) << __func__ << " requeue pg " << spgid << " " << pg << " snap "
+            << snap << dendl;
+    pg->queue_snap_retrim(snap);
+    pg->unlock();
   }
-
- out:
-  return r;
+  osd_lock.lock();
+  if (is_stopping()) {
+    return;
+  }
+  dout(10) << __func__ << " done queueing pgs, updating superblock" << dendl;
+  ObjectStore::Transaction t;
+  superblock.last_purged_snaps_scrub = ceph_clock_now();
+  write_superblock(t);
+  int tr = store->queue_transaction(service.meta_ch, std::move(t), nullptr);
+  ceph_assert(tr == 0);
+  if (is_active()) {
+    send_beacon(ceph::coarse_mono_clock::now());
+  }
+  dout(10) << __func__ << " done" << dendl;
 }
 
 void OSD::probe_smart(const string& only_devid, ostream& ss)
@@ -7072,9 +7156,9 @@ bool OSD::ms_dispatch(Message *m)
 
   // lock!
 
-  osd_lock.Lock();
+  osd_lock.lock();
   if (is_stopping()) {
-    osd_lock.Unlock();
+    osd_lock.unlock();
     m->put();
     return true;
   }
@@ -7082,56 +7166,63 @@ bool OSD::ms_dispatch(Message *m)
   do_waiters();
   _dispatch(m);
 
-  osd_lock.Unlock();
+  osd_lock.unlock();
 
   return true;
 }
 
-void OSD::maybe_share_map(
-  Session *session,
-  OpRequestRef op,
-  OSDMapRef osdmap)
+void OSDService::maybe_share_map(
+  Connection *con,
+  const OSDMapRef& osdmap,
+  epoch_t peer_epoch_lb)
 {
-  if (!op->check_send_map) {
+  // NOTE: we assume caller hold something that keeps the Connection itself
+  // pinned (e.g., an OpRequest's MessageRef).
+  auto session = ceph::ref_cast<Session>(con->get_priv());
+  if (!session) {
     return;
   }
-  epoch_t last_sent_epoch = 0;
 
+  // assume the peer has the newer of the op's sent_epoch and what
+  // we think we sent them.
   session->sent_epoch_lock.lock();
-  last_sent_epoch = session->last_sent_epoch;
+  if (peer_epoch_lb > session->last_sent_epoch) {
+    dout(10) << __func__ << " con " << con
+            << " " << con->get_peer_addr()
+            << " map epoch " << session->last_sent_epoch
+            << " -> " << peer_epoch_lb << " (as per caller)" << dendl;
+    session->last_sent_epoch = peer_epoch_lb;
+  }
+  epoch_t last_sent_epoch = session->last_sent_epoch;
   session->sent_epoch_lock.unlock();
 
-  // assume the peer has the newer of the op's sent_epoch and what
-  // we think we sent them.
-  epoch_t from = std::max(last_sent_epoch, op->sent_epoch);
+  if (osdmap->get_epoch() <= last_sent_epoch) {
+    return;
+  }
 
-  const Message *m = op->get_req();
-  service.share_map(
-    m->get_source(),
-    m->get_connection().get(),
-    from,
-    osdmap,
-    session ? &last_sent_epoch : NULL);
+  send_incremental_map(last_sent_epoch, con, osdmap);
+  last_sent_epoch = osdmap->get_epoch();
 
   session->sent_epoch_lock.lock();
   if (session->last_sent_epoch < last_sent_epoch) {
+    dout(10) << __func__ << " con " << con
+            << " " << con->get_peer_addr()
+            << " map epoch " << session->last_sent_epoch
+            << " -> " << last_sent_epoch << " (shared)" << dendl;
     session->last_sent_epoch = last_sent_epoch;
   }
   session->sent_epoch_lock.unlock();
-
-  op->check_send_map = false;
 }
 
-void OSD::dispatch_session_waiting(SessionRef session, OSDMapRef osdmap)
+void OSD::dispatch_session_waiting(const ceph::ref_t<Session>& session, OSDMapRef osdmap)
 {
-  ceph_assert(session->session_dispatch_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(session->session_dispatch_lock));
 
   auto i = session->waiting_on_map.begin();
   while (i != session->waiting_on_map.end()) {
     OpRequestRef op = &(*i);
     ceph_assert(ms_can_fast_dispatch(op->get_req()));
-    const MOSDFastDispatchOp *m = static_cast<const MOSDFastDispatchOp*>(
-      op->get_req());
+    auto m = op->get_req<MOSDFastDispatchOp>();
     if (m->get_min_epoch() > osdmap->get_epoch()) {
       break;
     }
@@ -7160,44 +7251,42 @@ void OSD::dispatch_session_waiting(SessionRef session, OSDMapRef osdmap)
 
 void OSD::ms_fast_dispatch(Message *m)
 {
+  auto dispatch_span = tracing::osd::tracer.start_trace(__func__);
   FUNCTRACE(cct);
   if (service.is_stopping()) {
     m->put();
     return;
   }
-
   // peering event?
   switch (m->get_type()) {
   case CEPH_MSG_PING:
     dout(10) << "ping from " << m->get_source() << dendl;
     m->put();
     return;
-  case MSG_MON_COMMAND:
-    handle_command(static_cast<MMonCommand*>(m));
-    return;
   case MSG_OSD_FORCE_RECOVERY:
     handle_fast_force_recovery(static_cast<MOSDForceRecovery*>(m));
     return;
   case MSG_OSD_SCRUB2:
     handle_fast_scrub(static_cast<MOSDScrub2*>(m));
     return;
-
   case MSG_OSD_PG_CREATE2:
     return handle_fast_pg_create(static_cast<MOSDPGCreate2*>(m));
-  case MSG_OSD_PG_QUERY:
-    return handle_fast_pg_query(static_cast<MOSDPGQuery*>(m));
   case MSG_OSD_PG_NOTIFY:
     return handle_fast_pg_notify(static_cast<MOSDPGNotify*>(m));
   case MSG_OSD_PG_INFO:
     return handle_fast_pg_info(static_cast<MOSDPGInfo*>(m));
   case MSG_OSD_PG_REMOVE:
     return handle_fast_pg_remove(static_cast<MOSDPGRemove*>(m));
-
     // these are single-pg messages that handle themselves
   case MSG_OSD_PG_LOG:
   case MSG_OSD_PG_TRIM:
+  case MSG_OSD_PG_NOTIFY2:
+  case MSG_OSD_PG_QUERY2:
+  case MSG_OSD_PG_INFO2:
   case MSG_OSD_BACKFILL_RESERVE:
   case MSG_OSD_RECOVERY_RESERVE:
+  case MSG_OSD_PG_LEASE:
+  case MSG_OSD_PG_LEASE_ACK:
     {
       MOSDPeeringOp *pm = static_cast<MOSDPeeringOp*>(m);
       if (require_osd_peer(pm)) {
@@ -7218,6 +7307,7 @@ void OSD::ms_fast_dispatch(Message *m)
     tracepoint(osd, ms_fast_dispatch, reqid.name._type,
         reqid.name._num, reqid.tid, reqid.inc);
   }
+  op->osd_parent_span = tracing::osd::tracer.add_span("op-request-created", dispatch_span);
 
   if (m->trace)
     op->osd_trace.init("osd op", &trace_endpoint, &m->trace);
@@ -7250,38 +7340,16 @@ void OSD::ms_fast_dispatch(Message *m)
       service.release_map(nextmap);
     }
   }
-  OID_EVENT_TRACE_WITH_MSG(m, "MS_FAST_DISPATCH_END", false); 
-}
-
-bool OSD::ms_get_authorizer(int dest_type, AuthAuthorizer **authorizer)
-{
-  dout(10) << "OSD::ms_get_authorizer type=" << ceph_entity_type_name(dest_type) << dendl;
-
-  if (is_stopping()) {
-    dout(10) << __func__ << " bailing, we are shutting down" << dendl;
-    return false;
-  }
-
-  if (dest_type == CEPH_ENTITY_TYPE_MON)
-    return true;
-
-  *authorizer = monc->build_authorizer(dest_type);
-  return *authorizer != NULL;
-}
-
-KeyStore *OSD::ms_get_auth1_authorizer_keystore()
-{
-  return monc->rotating_secrets.get();
+  OID_EVENT_TRACE_WITH_MSG(m, "MS_FAST_DISPATCH_END", false);
 }
 
 int OSD::ms_handle_authentication(Connection *con)
 {
   int ret = 0;
-  auto priv = con->get_priv();
-  Session *s = static_cast<Session*>(priv.get());
+  auto s = ceph::ref_cast<Session>(con->get_priv());
   if (!s) {
-    s = new Session(cct, con);
-    con->set_priv(RefCountedPtr{s, false});
+    s = ceph::make_ref<Session>(cct, con);
+    con->set_priv(s);
     s->entity_name = con->get_peer_entity_name();
     dout(10) << __func__ << " new session " << s << " con " << s->con
             << " entity " << s->entity_name
@@ -7293,19 +7361,18 @@ int OSD::ms_handle_authentication(Connection *con)
   }
 
   AuthCapsInfo &caps_info = con->get_peer_caps_info();
-  if (caps_info.allow_all)
+  if (caps_info.allow_all) {
     s->caps.set_allow_all();
-
-  if (caps_info.caps.length() > 0) {
+  } else if (caps_info.caps.length() > 0) {
     bufferlist::const_iterator p = caps_info.caps.cbegin();
     string str;
     try {
       decode(str, p);
     }
-    catch (buffer::error& e) {
+    catch (ceph::buffer::error& e) {
       dout(10) << __func__ << " session " << s << " " << s->entity_name
               << " failed to decode caps string" << dendl;
-      ret = -EPERM;
+      ret = -EACCES;
     }
     if (!ret) {
       bool success = s->caps.parse(str);
@@ -7317,7 +7384,7 @@ int OSD::ms_handle_authentication(Connection *con)
       } else {
        dout(10) << __func__ << " session " << s << " " << s->entity_name
                 << " failed to parse caps '" << str << "'" << dendl;
-       ret = -EPERM;
+       ret = -EACCES;
       }
     }
   }
@@ -7326,7 +7393,7 @@ int OSD::ms_handle_authentication(Connection *con)
 
 void OSD::do_waiters()
 {
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
 
   dout(10) << "do_waiters -- start" << dendl;
   while (!finished.empty()) {
@@ -7349,7 +7416,7 @@ void OSD::dispatch_op(OpRequestRef op)
 
 void OSD::_dispatch(Message *m)
 {
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
   dout(20) << "_dispatch " << m << " " << *m << dendl;
 
   switch (m->get_type()) {
@@ -7359,6 +7426,9 @@ void OSD::_dispatch(Message *m)
   case CEPH_MSG_OSD_MAP:
     handle_osd_map(static_cast<MOSDMap*>(m));
     break;
+  case MSG_MON_GET_PURGED_SNAPS_REPLY:
+    handle_get_purged_snaps_reply(static_cast<MMonGetPurgedSnapsReply*>(m));
+    break;
 
     // osd
   case MSG_OSD_SCRUB:
@@ -7377,7 +7447,7 @@ void OSD::_dispatch(Message *m)
       if (m->trace)
         op->osd_trace.init("osd op", &trace_endpoint, &m->trace);
       // no map?  starting up?
-      if (!osdmap) {
+      if (!get_osdmap()) {
         dout(7) << "no OSDMap, not booted" << dendl;
        logger->inc(l_osd_waiting_for_map);
         waiting_for_osdmap.push_back(op);
@@ -7413,7 +7483,7 @@ void OSD::handle_scrub(MOSDScrub *m)
     vector<spg_t> v;
     for (auto pgid : m->scrub_pgs) {
       spg_t pcand;
-      if (osdmap->get_primary_shard(pgid, &pcand) &&
+      if (get_osdmap()->get_primary_shard(pgid, &pcand) &&
          std::find(spgs.begin(), spgs.end(), pcand) != spgs.end()) {
        v.push_back(pcand);
       }
@@ -7428,7 +7498,7 @@ void OSD::handle_scrub(MOSDScrub *m)
        std::make_shared<PGPeeringEvent>(
          get_osdmap_epoch(),
          get_osdmap_epoch(),
-         PG::RequestScrub(m->deep, m->repair))));
+         PeeringState::RequestScrub(m->deep, m->repair))));
   }
 
   m->put();
@@ -7454,7 +7524,7 @@ void OSD::handle_fast_scrub(MOSDScrub2 *m)
        std::make_shared<PGPeeringEvent>(
          m->epoch,
          m->epoch,
-         PG::RequestScrub(m->deep, m->repair))));
+         PeeringState::RequestScrub(m->deep, m->repair))));
   }
   m->put();
 }
@@ -7470,193 +7540,125 @@ bool OSD::scrub_random_backoff()
   return false;
 }
 
-OSDService::ScrubJob::ScrubJob(CephContext* cct,
-                              const spg_t& pg, const utime_t& timestamp,
-                              double pool_scrub_min_interval,
-                              double pool_scrub_max_interval, bool must)
-  : cct(cct),
-    pgid(pg),
-    sched_time(timestamp),
-    deadline(timestamp)
-{
-  // if not explicitly requested, postpone the scrub with a random delay
-  if (!must) {
-    double scrub_min_interval = pool_scrub_min_interval > 0 ?
-      pool_scrub_min_interval : cct->_conf->osd_scrub_min_interval;
-    double scrub_max_interval = pool_scrub_max_interval > 0 ?
-      pool_scrub_max_interval : cct->_conf->osd_scrub_max_interval;
-
-    sched_time += scrub_min_interval;
-    double r = rand() / (double)RAND_MAX;
-    sched_time +=
-      scrub_min_interval * cct->_conf->osd_scrub_interval_randomize_ratio * r;
-    if (scrub_max_interval == 0) {
-      deadline = utime_t();
-    } else {
-      deadline += scrub_max_interval;
-    }
-
-  }
-}
-
-bool OSDService::ScrubJob::ScrubJob::operator<(const OSDService::ScrubJob& rhs) const {
-  if (sched_time < rhs.sched_time)
-    return true;
-  if (sched_time > rhs.sched_time)
-    return false;
-  return pgid < rhs.pgid;
-}
 
-bool OSD::scrub_time_permit(utime_t now)
+void OSD::sched_scrub()
 {
-  struct tm bdt;
-  time_t tt = now.sec();
-  localtime_r(&tt, &bdt);
+  auto& scrub_scheduler = service.get_scrub_services();
 
-  bool day_permit = false;
-  if (cct->_conf->osd_scrub_begin_week_day < cct->_conf->osd_scrub_end_week_day) {
-    if (bdt.tm_wday >= cct->_conf->osd_scrub_begin_week_day && bdt.tm_wday < cct->_conf->osd_scrub_end_week_day) {
-      day_permit = true;
-    }
-  } else {
-    if (bdt.tm_wday >= cct->_conf->osd_scrub_begin_week_day || bdt.tm_wday < cct->_conf->osd_scrub_end_week_day) {
-      day_permit = true;
-    }
+  // fail fast if no resources are available
+  if (!scrub_scheduler.can_inc_scrubs()) {
+    dout(20) << __func__ << ": OSD cannot inc scrubs" << dendl;
+    return;
   }
 
-  if (!day_permit) {
-    dout(20) << __func__ << " should run between week day " << cct->_conf->osd_scrub_begin_week_day
-            << " - " << cct->_conf->osd_scrub_end_week_day
-            << " now " << bdt.tm_wday << " = no" << dendl;
-    return false;
+  // if there is a PG that is just now trying to reserve scrub replica resources -
+  // we should wait and not initiate a new scrub
+  if (scrub_scheduler.is_reserving_now()) {
+    dout(20) << __func__ << ": scrub resources reservation in progress" << dendl;
+    return;
   }
 
-  bool time_permit = false;
-  if (cct->_conf->osd_scrub_begin_hour < cct->_conf->osd_scrub_end_hour) {
-    if (bdt.tm_hour >= cct->_conf->osd_scrub_begin_hour && bdt.tm_hour < cct->_conf->osd_scrub_end_hour) {
-      time_permit = true;
-    }
-  } else {
-    if (bdt.tm_hour >= cct->_conf->osd_scrub_begin_hour || bdt.tm_hour < cct->_conf->osd_scrub_end_hour) {
-      time_permit = true;
+  Scrub::ScrubPreconds env_conditions;
+
+  if (service.is_recovery_active() && !cct->_conf->osd_scrub_during_recovery) {
+    if (!cct->_conf->osd_repair_during_recovery) {
+      dout(15) << __func__ << ": not scheduling scrubs due to active recovery"
+              << dendl;
+      return;
     }
+    dout(10) << __func__
+      << " will only schedule explicitly requested repair due to active recovery"
+      << dendl;
+    env_conditions.allow_requested_repair_only = true;
   }
-  if (!time_permit) {
-    dout(20) << __func__ << " should run between " << cct->_conf->osd_scrub_begin_hour
-            << " - " << cct->_conf->osd_scrub_end_hour
-            << " now " << bdt.tm_hour << " = no" << dendl;
-  } else {
-    dout(20) << __func__ << " should run between " << cct->_conf->osd_scrub_begin_hour
-            << " - " << cct->_conf->osd_scrub_end_hour
-            << " now " << bdt.tm_hour << " = yes" << dendl;
+
+  if (g_conf()->subsys.should_gather<ceph_subsys_osd, 20>()) {
+    dout(20) << __func__ << " sched_scrub starts" << dendl;
+    auto all_jobs = scrub_scheduler.list_registered_jobs();
+    for (const auto& sj : all_jobs) {
+      dout(20) << "sched_scrub scrub-queue jobs: " << *sj << dendl;
+    }
   }
-  return time_permit;
+
+  auto was_started = scrub_scheduler.select_pg_and_scrub(env_conditions);
+  dout(20) << "sched_scrub done (" << ScrubQueue::attempt_res_text(was_started)
+          << ")" << dendl;
 }
 
-bool OSD::scrub_load_below_threshold()
+Scrub::schedule_result_t OSDService::initiate_a_scrub(spg_t pgid,
+                                                     bool allow_requested_repair_only)
 {
-  double loadavgs[3];
-  if (getloadavg(loadavgs, 3) != 3) {
-    dout(10) << __func__ << " couldn't read loadavgs\n" << dendl;
-    return false;
-  }
+  dout(20) << __func__ << " trying " << pgid << dendl;
 
-  // allow scrub if below configured threshold
-  long cpus = sysconf(_SC_NPROCESSORS_ONLN);
-  double loadavg_per_cpu = cpus > 0 ? loadavgs[0] / cpus : loadavgs[0];
-  if (loadavg_per_cpu < cct->_conf->osd_scrub_load_threshold) {
-    dout(20) << __func__ << " loadavg per cpu " << loadavg_per_cpu
-            << " < max " << cct->_conf->osd_scrub_load_threshold
-            << " = yes" << dendl;
-    return true;
+  // we have a candidate to scrub. We need some PG information to know if scrubbing is
+  // allowed
+
+  PGRef pg = osd->lookup_lock_pg(pgid);
+  if (!pg) {
+    // the PG was dequeued in the short timespan between creating the candidates list
+    // (collect_ripe_jobs()) and here
+    dout(5) << __func__ << " pg  " << pgid << " not found" << dendl;
+    return Scrub::schedule_result_t::no_such_pg;
   }
 
-  // allow scrub if below daily avg and currently decreasing
-  if (loadavgs[0] < daily_loadavg && loadavgs[0] < loadavgs[2]) {
-    dout(20) << __func__ << " loadavg " << loadavgs[0]
-            << " < daily_loadavg " << daily_loadavg
-            << " and < 15m avg " << loadavgs[2]
-            << " = yes" << dendl;
-    return true;
+  // This has already started, so go on to the next scrub job
+  if (pg->is_scrub_queued_or_active()) {
+    pg->unlock();
+    dout(20) << __func__ << ": already in progress pgid " << pgid << dendl;
+    return Scrub::schedule_result_t::already_started;
+  }
+  // Skip other kinds of scrubbing if only explicitly requested repairing is allowed
+  if (allow_requested_repair_only && !pg->m_planned_scrub.must_repair) {
+    pg->unlock();
+    dout(10) << __func__ << " skip " << pgid
+            << " because repairing is not explicitly requested on it" << dendl;
+    return Scrub::schedule_result_t::preconditions;
   }
 
-  dout(20) << __func__ << " loadavg " << loadavgs[0]
-          << " >= max " << cct->_conf->osd_scrub_load_threshold
-          << " and ( >= daily_loadavg " << daily_loadavg
-          << " or >= 15m avg " << loadavgs[2]
-          << ") = no" << dendl;
-  return false;
+  auto scrub_attempt = pg->sched_scrub();
+  pg->unlock();
+  return scrub_attempt;
 }
 
-void OSD::sched_scrub()
+void OSD::resched_all_scrubs()
 {
-  // if not permitted, fail fast
-  if (!service.can_inc_scrubs_pending()) {
-    return;
-  }
-  if (!cct->_conf->osd_scrub_during_recovery && service.is_recovery_active()) {
-    dout(20) << __func__ << " not scheduling scrubs due to active recovery" << dendl;
-    return;
-  }
+  dout(10) << __func__ << ": start" << dendl;
+  auto all_jobs = service.get_scrub_services().list_registered_jobs();
+  for (auto& e : all_jobs) {
 
+    auto& job = *e;
+    dout(20) << __func__ << ": examine " << job.pgid << dendl;
 
-  utime_t now = ceph_clock_now();
-  bool time_permit = scrub_time_permit(now);
-  bool load_is_low = scrub_load_below_threshold();
-  dout(20) << "sched_scrub load_is_low=" << (int)load_is_low << dendl;
-
-  OSDService::ScrubJob scrub;
-  if (service.first_scrub_stamp(&scrub)) {
-    do {
-      dout(30) << "sched_scrub examine " << scrub.pgid << " at " << scrub.sched_time << dendl;
-
-      if (scrub.sched_time > now) {
-       // save ourselves some effort
-       dout(10) << "sched_scrub " << scrub.pgid << " scheduled at " << scrub.sched_time
-                << " > " << now << dendl;
-       break;
-      }
-
-      if ((scrub.deadline.is_zero() || scrub.deadline >= now) && !(time_permit && load_is_low)) {
-        dout(10) << __func__ << " not scheduling scrub for " << scrub.pgid << " due to "
-                 << (!time_permit ? "time not permit" : "high load") << dendl;
-        continue;
-      }
+    PGRef pg = _lookup_lock_pg(job.pgid);
+    if (!pg)
+      continue;
 
-      PGRef pg = _lookup_lock_pg(scrub.pgid);
-      if (!pg)
-       continue;
-      dout(10) << "sched_scrub scrubbing " << scrub.pgid << " at " << scrub.sched_time
-              << (pg->get_must_scrub() ? ", explicitly requested" :
-                  (load_is_low ? ", load_is_low" : " deadline < now"))
-              << dendl;
-      if (pg->sched_scrub()) {
-       pg->unlock();
-       break;
-      }
-      pg->unlock();
-    } while (service.next_scrub_stamp(scrub, &scrub));
+    if (!pg->m_planned_scrub.must_scrub && !pg->m_planned_scrub.need_auto) {
+      dout(15) << __func__ << ": reschedule " << job.pgid << dendl;
+      pg->reschedule_scrub();
+    }
+    pg->unlock();
   }
-  dout(20) << "sched_scrub done" << dendl;
+  dout(10) << __func__ << ": done" << dendl;
 }
 
 MPGStats* OSD::collect_pg_stats()
 {
+  dout(15) << __func__ << dendl;
   // This implementation unconditionally sends every is_primary PG's
   // stats every time we're called.  This has equivalent cost to the
   // previous implementation's worst case where all PGs are busy and
   // their stats are always enqueued for sending.
-  RWLock::RLocker l(map_lock);
+  std::shared_lock l{map_lock};
 
-  utime_t had_for = ceph_clock_now() - had_map_since;
   osd_stat_t cur_stat = service.get_osd_stat();
   cur_stat.os_perf_stat = store->get_cur_stats();
 
-  auto m = new MPGStats(monc->get_fsid(), osdmap->get_epoch(), had_for);
+  auto m = new MPGStats(monc->get_fsid(), get_osdmap_epoch());
   m->osd_stat = cur_stat;
 
   std::lock_guard lec{min_last_epoch_clean_lock};
-  min_last_epoch_clean = osdmap->get_epoch();
+  min_last_epoch_clean = get_osdmap_epoch();
   min_last_epoch_clean_pgs.clear();
 
   std::set<int64_t> pool_set;
@@ -7668,16 +7670,17 @@ MPGStats* OSD::collect_pg_stats()
     if (!pg->is_primary()) {
       continue;
     }
-    pg->get_pg_stats([&](const pg_stat_t& s, epoch_t lec) {
+    pg->with_pg_stats([&](const pg_stat_t& s, epoch_t lec) {
        m->pg_stat[pg->pg_id.pgid] = s;
-       min_last_epoch_clean = min(min_last_epoch_clean, lec);
+       min_last_epoch_clean = std::min(min_last_epoch_clean, lec);
        min_last_epoch_clean_pgs.push_back(pg->pg_id.pgid);
       });
   }
   store_statfs_t st;
   bool per_pool_stats = false;
+  bool per_pool_omap_stats = false;
   for (auto p : pool_set) {
-    int r = store->pool_statfs(p, &st);
+    int r = store->pool_statfs(p, &st, &per_pool_omap_stats);
     if (r == -ENOTSUP) {
       break;
     } else {
@@ -7690,6 +7693,7 @@ MPGStats* OSD::collect_pg_stats()
   // indicate whether we are reporting per-pool stats
   m->osd_stat.num_osds = 1;
   m->osd_stat.num_per_pool_osds = per_pool_stats ? 1 : 0;
+  m->osd_stat.num_per_pool_omap_osds = per_pool_omap_stats ? 1 : 0;
 
   return m;
 }
@@ -7704,11 +7708,37 @@ vector<DaemonHealthMetric> OSD::get_health_metrics()
     too_old -= cct->_conf.get_val<double>("osd_op_complaint_time");
     int slow = 0;
     TrackedOpRef oldest_op;
+    OSDMapRef osdmap = get_osdmap();
+    // map of slow op counts by slow op event type for an aggregated logging to
+    // the cluster log.
+    map<uint8_t, int> slow_op_types;
+    // map of slow op counts by pool for reporting a pool name with highest
+    // slow ops.
+    map<uint64_t, int> slow_op_pools;
+    bool log_aggregated_slow_op =
+           cct->_conf.get_val<bool>("osd_aggregated_slow_ops_logging");
     auto count_slow_ops = [&](TrackedOp& op) {
       if (op.get_initiated() < too_old) {
-       lgeneric_subdout(cct,osd,20) << "slow op " << op.get_desc()
-                                    << " initiated "
-                                    << op.get_initiated() << dendl;
+        stringstream ss;
+        ss << "slow request " << op.get_desc()
+           << " initiated "
+           << op.get_initiated()
+           << " currently "
+           << op.state_string();
+        lgeneric_subdout(cct,osd,20) << ss.str() << dendl;
+        if (log_aggregated_slow_op) {
+          if (const OpRequest *req = dynamic_cast<const OpRequest *>(&op)) {
+            uint8_t op_type = req->state_flag();
+            auto m = req->get_req<MOSDFastDispatchOp>();
+            uint64_t poolid = m->get_spg().pgid.m_pool;
+            slow_op_types[op_type]++;
+            if (poolid > 0 && poolid <= (uint64_t) osdmap->get_pool_max()) {
+              slow_op_pools[poolid]++;
+            }
+          }
+        } else {
+          clog->warn() << ss.str();
+        }
        slow++;
        if (!oldest_op || op.get_initiated() < oldest_op->get_initiated()) {
          oldest_op = &op;
@@ -7722,6 +7752,32 @@ vector<DaemonHealthMetric> OSD::get_health_metrics()
       if (slow) {
        derr << __func__ << " reporting " << slow << " slow ops, oldest is "
             << oldest_op->get_desc() << dendl;
+        if (log_aggregated_slow_op &&
+             slow_op_types.size() > 0) {
+          stringstream ss;
+          ss << slow << " slow requests (by type [ ";
+          for (const auto& [op_type, count] : slow_op_types) {
+            ss << "'" << OpRequest::get_state_string(op_type)
+               << "' : " << count
+               << " ";
+          }
+          auto slow_pool_it = std::max_element(slow_op_pools.begin(), slow_op_pools.end(),
+                                 [](std::pair<uint64_t, int> p1, std::pair<uint64_t, int> p2) {
+                                   return p1.second < p2.second;
+                                 });
+          if (osdmap->get_pools().find(slow_pool_it->first) != osdmap->get_pools().end()) {
+            string pool_name = osdmap->get_pool_name(slow_pool_it->first);
+            ss << "] most affected pool [ '"
+               << pool_name
+               << "' : "
+               << slow_pool_it->second
+               << " ])";
+          } else {
+            ss << "])";
+          }
+          lgeneric_subdout(cct,osd,20) << ss.str() << dendl;
+          clog->warn() << ss.str();
+        }
       }
       metrics.emplace_back(daemon_metric::SLOW_OPS, slow, oldest_secs);
     } else {
@@ -7749,7 +7805,7 @@ void OSD::wait_for_new_map(OpRequestRef op)
 {
   // ask?
   if (waiting_for_osdmap.empty()) {
-    osdmap_subscribe(osdmap->get_epoch() + 1, false);
+    osdmap_subscribe(get_osdmap_epoch() + 1, false);
   }
 
   logger->inc(l_osd_waiting_for_map);
@@ -7764,26 +7820,21 @@ void OSD::wait_for_new_map(OpRequestRef op)
 
 void OSD::note_down_osd(int peer)
 {
-  ceph_assert(osd_lock.is_locked());
-  cluster_messenger->mark_down_addrs(osdmap->get_cluster_addrs(peer));
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
+  cluster_messenger->mark_down_addrs(get_osdmap()->get_cluster_addrs(peer));
 
-  heartbeat_lock.Lock();
+  std::lock_guard l{heartbeat_lock};
   failure_queue.erase(peer);
   failure_pending.erase(peer);
   map<int,HeartbeatInfo>::iterator p = heartbeat_peers.find(peer);
   if (p != heartbeat_peers.end()) {
-    p->second.con_back->mark_down();
-    if (p->second.con_front) {
-      p->second.con_front->mark_down();
-    }
+    p->second.clear_mark_down();
     heartbeat_peers.erase(p);
   }
-  heartbeat_lock.Unlock();
 }
 
 void OSD::note_up_osd(int peer)
 {
-  service.forget_peer_epoch(peer, osdmap->get_epoch() - 1);
   heartbeat_set_peers_need_update();
 }
 
@@ -7871,10 +7922,11 @@ void OSD::handle_osd_map(MOSDMap *m)
        osd_min = min;
       }
     }
+    epoch_t osdmap_epoch = get_osdmap_epoch();
     if (osd_min > 0 &&
-       osdmap->get_epoch() > max_lag &&
-       osdmap->get_epoch() - max_lag > osd_min) {
-      epoch_t need = osdmap->get_epoch() - max_lag;
+       osdmap_epoch > max_lag &&
+       osdmap_epoch - max_lag > osd_min) {
+      epoch_t need = osdmap_epoch - max_lag;
       dout(10) << __func__ << " waiting for pgs to catch up (need " << need
               << " max_lag " << max_lag << ")" << dendl;
       for (auto shard : shards) {
@@ -7892,7 +7944,7 @@ void OSD::handle_osd_map(MOSDMap *m)
     }
   }
 
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
   map<epoch_t,OSDMapRef> added_maps;
   map<epoch_t,bufferlist> added_maps_bl;
   if (m->fsid != monc->get_fsid()) {
@@ -7907,9 +7959,8 @@ void OSD::handle_osd_map(MOSDMap *m)
     return;
   }
 
-  auto priv = m->get_connection()->get_priv();
-  if (auto session = static_cast<Session *>(priv.get());
-      session && !(session->entity_name.is_mon() ||
+  auto session = ceph::ref_cast<Session>(m->get_connection()->get_priv());
+  if (session && !(session->entity_name.is_mon() ||
                   session->entity_name.is_osd())) {
     //not enough perms!
     dout(10) << "got osd map from Session " << session
@@ -7971,6 +8022,8 @@ void OSD::handle_osd_map(MOSDMap *m)
   ObjectStore::Transaction t;
   uint64_t txn_size = 0;
 
+  map<epoch_t,mempool::osdmap::map<int64_t,snap_interval_set_t>> purged_snaps;
+
   // store new maps: queue for disk and put in the osdmap cache
   epoch_t start = std::max(superblock.newest_map + 1, first);
   for (epoch_t e = start; e <= last; e++) {
@@ -7988,6 +8041,8 @@ void OSD::handle_osd_map(MOSDMap *m)
 
       o->decode(bl);
 
+      purged_snaps[e] = o->get_new_purged_snaps();
+
       ghobject_t fulloid = get_osdmap_pobject_name(e);
       t.write(coll_t::meta(), fulloid, 0, bl.length(), bl);
       added_maps[e] = add_map(o);
@@ -8018,8 +8073,9 @@ void OSD::handle_osd_map(MOSDMap *m)
       OSDMap::Incremental inc;
       auto p = bl.cbegin();
       inc.decode(p);
+
       if (o->apply_incremental(inc) < 0) {
-       derr << "ERROR: bad fsid?  i have " << osdmap->get_fsid() << " and inc has " << inc.fsid << dendl;
+       derr << "ERROR: bad fsid?  i have " << get_osdmap()->get_fsid() << " and inc has " << inc.fsid << dendl;
        ceph_abort_msg("bad fsid");
       }
 
@@ -8044,9 +8100,17 @@ void OSD::handle_osd_map(MOSDMap *m)
        delete o;
        request_full_map(e, last);
        last = e - 1;
+
+       // don't continue committing if we failed to enc the first inc map
+       if (last < start) {
+         dout(10) << __func__ << " bailing because last < start (" << last << "<" << start << ")" << dendl;
+         m->put();
+         return;
+       }
        break;
       }
       got_full_map(e);
+      purged_snaps[e] = o->get_new_purged_snaps();
 
       ghobject_t fulloid = get_osdmap_pobject_name(e);
       t.write(coll_t::meta(), fulloid, 0, fbl.length(), fbl);
@@ -8115,7 +8179,6 @@ void OSD::handle_osd_map(MOSDMap *m)
        }
        encode(profile, bl);
        t.write(coll_t::meta(), obj, 0, bl.length(), bl);
-       service.store_deleted_pool_pg_num(j.first, j.second.get_pg_num());
       } else if (unsigned new_pg_num = i.second->get_pg_num(j.first);
                 new_pg_num != j.second.get_pg_num()) {
        dout(10) << __func__ << " recording pool " << j.first << " pg_num "
@@ -8141,6 +8204,18 @@ void OSD::handle_osd_map(MOSDMap *m)
     dout(20) << __func__ << " pg_num_history " << pg_num_history << dendl;
   }
 
+  // record new purged_snaps
+  if (superblock.purged_snaps_last == start - 1) {
+    SnapMapper::record_purged_snaps(cct, store.get(), service.meta_ch,
+                                   make_purged_snaps_oid(), &t,
+                                   purged_snaps);
+    superblock.purged_snaps_last = last;
+  } else {
+    dout(10) << __func__ << " superblock purged_snaps_last is "
+            << superblock.purged_snaps_last
+            << ", not recording new purged_snaps" << dendl;
+  }
+
   // superblock and commit
   write_superblock(t);
   t.register_on_commit(new C_OnMapCommit(this, start, last, m));
@@ -8162,11 +8237,14 @@ void OSD::_committed_osd_maps(epoch_t first, epoch_t last, MOSDMap *m)
     dout(10) << __func__ << " bailing, we are shutting down" << dendl;
     return;
   }
-  map_lock.get_write();
+  map_lock.lock();
+
+  ceph_assert(first <= last);
 
   bool do_shutdown = false;
   bool do_restart = false;
   bool network_error = false;
+  OSDMapRef osdmap = get_osdmap();
 
   // advance through the new maps
   for (epoch_t cur = first; cur <= last; cur++) {
@@ -8178,12 +8256,13 @@ void OSD::_committed_osd_maps(epoch_t first, epoch_t last, MOSDMap *m)
     OSDMapRef newmap = get_map(cur);
     ceph_assert(newmap);  // we just cached it above!
 
-    // start blacklisting messages sent to peers that go down.
+    // start blocklisting messages sent to peers that go down.
     service.pre_publish_map(newmap);
 
     // kill connections to newly down osds
     bool waited_for_reservations = false;
     set<int> old;
+    osdmap = get_osdmap();
     osdmap->get_all_osds(old);
     for (set<int>::iterator p = old.begin(); p != old.end(); ++p) {
       if (*p != whoami &&
@@ -8215,7 +8294,8 @@ void OSD::_committed_osd_maps(epoch_t first, epoch_t last, MOSDMap *m)
       }
     }
 
-    osdmap = newmap;
+    osdmap = std::move(newmap);
+    set_osdmap(osdmap);
     epoch_t up_epoch;
     epoch_t boot_epoch;
     service.retrieve_epochs(&boot_epoch, &up_epoch, NULL);
@@ -8232,8 +8312,6 @@ void OSD::_committed_osd_maps(epoch_t first, epoch_t last, MOSDMap *m)
     }
   }
 
-  had_map_since = ceph_clock_now();
-
   epoch_t _bind_epoch = service.get_bind_epoch();
   if (osdmap->is_up(whoami) &&
       osdmap->get_addrs(whoami).legacy_equals(
@@ -8255,9 +8333,12 @@ void OSD::_committed_osd_maps(epoch_t first, epoch_t last, MOSDMap *m)
   if (osdmap->get_epoch() > 0 &&
       is_active()) {
     if (!osdmap->exists(whoami)) {
-      dout(0) << "map says i do not exist.  shutting down." << dendl;
+      derr << "map says i do not exist.  shutting down." << dendl;
       do_shutdown = true;   // don't call shutdown() while we have
                            // everything paused
+    } else if (osdmap->is_stop(whoami)) {
+      derr << "map says i am stopped by admin. shutting down." << dendl;
+      do_shutdown = true;
     } else if (!osdmap->is_up(whoami) ||
               !osdmap->get_addrs(whoami).legacy_equals(
                 client_messenger->get_myaddrs()) ||
@@ -8277,6 +8358,14 @@ void OSD::_committed_osd_maps(epoch_t first, epoch_t last, MOSDMap *m)
                         << " wrongly marked me down at e"
                         << osdmap->get_down_at(whoami);
        }
+       if (monc->monmap.min_mon_release >= ceph_release_t::octopus) {
+         // note that this is best-effort...
+         monc->send_mon_message(
+           new MOSDMarkMeDead(
+             monc->get_fsid(),
+             whoami,
+             osdmap->get_epoch()));
+       }
       } else if (!osdmap->get_addrs(whoami).legacy_equals(
                   client_messenger->get_myaddrs())) {
        clog->error() << "map e" << osdmap->get_epoch()
@@ -8314,17 +8403,13 @@ void OSD::_committed_osd_maps(epoch_t first, epoch_t last, MOSDMap *m)
        utime_t now = ceph_clock_now();
        utime_t grace = utime_t(cct->_conf->osd_max_markdown_period, 0);
        osd_markdown_log.push_back(now);
-       //clear all out-of-date log
-       while (!osd_markdown_log.empty() &&
-              osd_markdown_log.front() + grace < now)
-         osd_markdown_log.pop_front();
        if ((int)osd_markdown_log.size() > cct->_conf->osd_max_markdown_count) {
-         dout(0) << __func__ << " marked down "
-                 << osd_markdown_log.size()
-                 << " > osd_max_markdown_count "
-                 << cct->_conf->osd_max_markdown_count
-                 << " in last " << grace << " seconds, shutting down"
-                 << dendl;
+         derr << __func__ << " marked down "
+              << osd_markdown_log.size()
+              << " > osd_max_markdown_count "
+              << cct->_conf->osd_max_markdown_count
+              << " in last " << grace << " seconds, shutting down"
+              << dendl;
          do_restart = false;
          do_shutdown = true;
        }
@@ -8334,47 +8419,34 @@ void OSD::_committed_osd_maps(epoch_t first, epoch_t last, MOSDMap *m)
        set<int> avoid_ports;
 #if defined(__FreeBSD__)
         // prevent FreeBSD from grabbing the client_messenger port during
-        // rebinding. In which case a cluster_meesneger will connect also 
+        // rebinding. In which case a cluster_meesneger will connect also
        // to the same port
        client_messenger->get_myaddrs().get_ports(&avoid_ports);
 #endif
        cluster_messenger->get_myaddrs().get_ports(&avoid_ports);
-       hb_back_server_messenger->get_myaddrs().get_ports(&avoid_ports);
-       hb_front_server_messenger->get_myaddrs().get_ports(&avoid_ports);
 
        int r = cluster_messenger->rebind(avoid_ports);
        if (r != 0) {
          do_shutdown = true;  // FIXME: do_restart?
           network_error = true;
-          dout(0) << __func__ << " marked down:"
-                  << " rebind cluster_messenger failed" << dendl;
-        }
-
-       r = hb_back_server_messenger->rebind(avoid_ports);
-       if (r != 0) {
-         do_shutdown = true;  // FIXME: do_restart?
-          network_error = true;
-          dout(0) << __func__ << " marked down:"
-                  << " rebind hb_back_server_messenger failed" << dendl;
-        }
-
-       r = hb_front_server_messenger->rebind(avoid_ports);
-       if (r != 0) {
-         do_shutdown = true;  // FIXME: do_restart?
-          network_error = true;
-          dout(0) << __func__ << " marked down:" 
-                  << " rebind hb_front_server_messenger failed" << dendl;
+          derr << __func__ << " marked down:"
+              << " rebind cluster_messenger failed" << dendl;
         }
 
+       hb_back_server_messenger->mark_down_all();
+       hb_front_server_messenger->mark_down_all();
        hb_front_client_messenger->mark_down_all();
        hb_back_client_messenger->mark_down_all();
 
-       reset_heartbeat_peers();
+       reset_heartbeat_peers(true);
       }
     }
+  } else if (osdmap->get_epoch() > 0 && osdmap->is_stop(whoami)) {
+    derr << "map says i am stopped by admin. shutting down." << dendl;
+    do_shutdown = true;
   }
 
-  map_lock.put_write();
+  map_lock.unlock();
 
   check_osdmap_features();
 
@@ -8423,6 +8495,7 @@ void OSD::check_osdmap_features()
   // current memory location, and setting or clearing bits in integer
   // fields, and we are the only writer, this is not a problem.
 
+  const auto osdmap = get_osdmap();
   {
     Messenger::Policy p = client_messenger->get_default_policy();
     uint64_t mask;
@@ -8468,8 +8541,12 @@ void OSD::check_osdmap_features()
     }
   }
 
-  if (osdmap->require_osd_release < CEPH_RELEASE_NAUTILUS) {
-    heartbeat_dispatcher.ms_set_require_authorizer(false);
+  if (osdmap->require_osd_release < ceph_release_t::nautilus) {
+    hb_front_server_messenger->set_require_authorizer(false);
+    hb_back_server_messenger->set_require_authorizer(false);
+  } else {
+    hb_front_server_messenger->set_require_authorizer(true);
+    hb_back_server_messenger->set_require_authorizer(true);
   }
 
   if (osdmap->require_osd_release != last_require_osd_release) {
@@ -8496,25 +8573,23 @@ void OSD::_finish_splits(set<PGRef>& pgs)
   dout(10) << __func__ << " " << pgs << dendl;
   if (is_stopping())
     return;
-  PG::RecoveryCtx rctx = create_context();
   for (set<PGRef>::iterator i = pgs.begin();
        i != pgs.end();
        ++i) {
     PG *pg = i->get();
 
+    PeeringCtx rctx;
     pg->lock();
     dout(10) << __func__ << " " << *pg << dendl;
     epoch_t e = pg->get_osdmap_epoch();
-    pg->handle_initialize(&rctx);
+    pg->handle_initialize(rctx);
     pg->queue_null(e, e);
-    dispatch_context_transaction(rctx, pg);
+    dispatch_context(rctx, pg, service.get_osdmap());
     pg->unlock();
 
     unsigned shard_index = pg->pg_id.hash_to_shard(num_shards);
     shards[shard_index]->register_and_wake_split_child(pg);
   }
-
-  dispatch_context(rctx, 0, service.get_osdmap());
 };
 
 bool OSD::add_merge_waiter(OSDMapRef nextmap, spg_t target, PGRef src,
@@ -8533,14 +8608,13 @@ bool OSD::advance_pg(
   epoch_t osd_epoch,
   PG *pg,
   ThreadPool::TPHandle &handle,
-  PG::RecoveryCtx *rctx)
+  PeeringCtx &rctx)
 {
   if (osd_epoch <= pg->get_osdmap_epoch()) {
     return true;
   }
   ceph_assert(pg->is_locked());
   OSDMapRef lastmap = pg->get_osdmap();
-  ceph_assert(lastmap->get_epoch() < osd_epoch);
   set<PGRef> new_pgs;  // any split children
   bool ret = true;
 
@@ -8572,8 +8646,18 @@ bool OSD::advance_pg(
                  << " is merge source, target is " << parent
                   << dendl;
          pg->write_if_dirty(rctx);
-         dispatch_context_transaction(*rctx, pg, &handle);
+         if (!new_pgs.empty()) {
+           rctx.transaction.register_on_applied(new C_FinishSplits(this,
+                                                                   new_pgs));
+           new_pgs.clear();
+         }
+         dispatch_context(rctx, pg, pg->get_osdmap(), &handle);
          pg->ch->flush();
+         // release backoffs explicitly, since the on_shutdown path
+         // aggressively tears down backoff state.
+         if (pg->is_primary()) {
+           pg->release_pg_backoffs();
+         }
          pg->on_shutdown();
          OSDShard *sdata = pg->osd_shard;
          {
@@ -8584,8 +8668,8 @@ bool OSD::advance_pg(
              // any time soon.
              if (pg->is_primary())
                logger->dec(l_osd_pg_primary);
-             else if (pg->is_replica())
-               logger->dec(l_osd_pg_replica);
+             else if (pg->is_nonprimary())
+               logger->dec(l_osd_pg_replica); // misnomer
              else
                logger->dec(l_osd_pg_stray);
            }
@@ -8639,6 +8723,12 @@ bool OSD::advance_pg(
          } else {
            dout(20) << __func__ << " not ready to merge yet" << dendl;
            pg->write_if_dirty(rctx);
+           if (!new_pgs.empty()) {
+             rctx.transaction.register_on_applied(new C_FinishSplits(this,
+                                                                     new_pgs));
+             new_pgs.clear();
+           }
+           dispatch_context(rctx, pg, pg->get_osdmap(), &handle);
            pg->unlock();
            // kick source(s) to get them ready
            for (auto& i : children) {
@@ -8668,6 +8758,31 @@ bool OSD::advance_pg(
       nextmap, lastmap, newup, up_primary,
       newacting, acting_primary, rctx);
 
+    auto oldpool = lastmap->get_pools().find(pg->pg_id.pool());
+    auto newpool = nextmap->get_pools().find(pg->pg_id.pool());
+    if (oldpool != lastmap->get_pools().end()
+        && newpool != nextmap->get_pools().end()) {
+      dout(20) << __func__
+              << " new pool opts " << newpool->second.opts
+              << " old pool opts " << oldpool->second.opts
+              << dendl;
+
+      double old_min_interval = 0, new_min_interval = 0;
+      oldpool->second.opts.get(pool_opts_t::SCRUB_MIN_INTERVAL, &old_min_interval);
+      newpool->second.opts.get(pool_opts_t::SCRUB_MIN_INTERVAL, &new_min_interval);
+
+      double old_max_interval = 0, new_max_interval = 0;
+      oldpool->second.opts.get(pool_opts_t::SCRUB_MAX_INTERVAL, &old_max_interval);
+      newpool->second.opts.get(pool_opts_t::SCRUB_MAX_INTERVAL, &new_max_interval);
+
+      // Assume if an interval is change from set to unset or vice versa the actual config
+      // is different.  Keep it simple even if it is possible to call resched_all_scrub()
+      // unnecessarily.
+      if (old_min_interval != new_min_interval || old_max_interval != new_max_interval) {
+       pg->on_info_history_change();
+      }
+    }
+
     if (new_pg_num && old_pg_num != new_pg_num) {
       // check for split
       set<spg_t> children;
@@ -8690,14 +8805,15 @@ bool OSD::advance_pg(
   ret = true;
  out:
   if (!new_pgs.empty()) {
-    rctx->transaction->register_on_applied(new C_FinishSplits(this, new_pgs));
+    rctx.transaction.register_on_applied(new C_FinishSplits(this, new_pgs));
   }
   return ret;
 }
 
 void OSD::consume_map()
 {
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
+  auto osdmap = get_osdmap();
   dout(7) << "consume_map version " << osdmap->get_epoch() << dendl;
 
   /** make sure the cluster is speaking in SORTBITWISE, because we don't
@@ -8767,8 +8883,8 @@ void OSD::consume_map()
     // racy, but we don't want to take pg lock here.
     if (pg->is_primary())
       num_pg_primary++;
-    else if (pg->is_replica())
-      num_pg_replica++;
+    else if (pg->is_nonprimary())
+      num_pg_replica++;  // misnomer
     else
       num_pg_stray++;
   }
@@ -8778,7 +8894,7 @@ void OSD::consume_map()
     std::lock_guard l(pending_creates_lock);
     for (auto pg = pending_creates_from_osd.begin();
         pg != pending_creates_from_osd.end();) {
-      if (osdmap->get_pg_acting_rank(pg->first, whoami) < 0) {
+      if (osdmap->get_pg_acting_role(pg->first, whoami) < 0) {
        dout(10) << __func__ << " pg " << pg->first << " doesn't map here, "
                 << "discarding pending_create_from_osd" << dendl;
        pg = pending_creates_from_osd.erase(pg);
@@ -8814,15 +8930,11 @@ void OSD::consume_map()
 
 void OSD::activate_map()
 {
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
+  auto osdmap = get_osdmap();
 
   dout(7) << "activate_map version " << osdmap->get_epoch() << dendl;
 
-  if (osdmap->test_flag(CEPH_OSDMAP_FULL)) {
-    dout(10) << " osdmap flagged full, doing onetime osdmap subscribe" << dendl;
-    osdmap_subscribe(osdmap->get_epoch() + 1, false);
-  }
-
   // norecover?
   if (osdmap->test_flag(CEPH_OSDMAP_NORECOVER)) {
     if (!service.recovery_is_paused()) {
@@ -8892,7 +9004,7 @@ bool OSD::require_self_aliveness(const Message *m, epoch_t epoch)
   return true;
 }
 
-bool OSD::require_same_peer_instance(const Message *m, OSDMapRef& map,
+bool OSD::require_same_peer_instance(const Message *m, const OSDMapRef& map,
                                     bool is_fast_dispatch)
 {
   int from = m->get_source().num();
@@ -8907,15 +9019,14 @@ bool OSD::require_same_peer_instance(const Message *m, OSDMapRef& map,
            << dendl;
     ConnectionRef con = m->get_connection();
     con->mark_down();
-    auto priv = con->get_priv();
-    if (auto s = static_cast<Session*>(priv.get()); s) {
+    if (auto s = ceph::ref_cast<Session>(con->get_priv()); s) {
       if (!is_fast_dispatch)
-       s->session_dispatch_lock.Lock();
+       s->session_dispatch_lock.lock();
       clear_session_waiting_on_map(s);
       con->set_priv(nullptr);   // break ref <-> session cycle, if any
       s->con.reset();
       if (!is_fast_dispatch)
-       s->session_dispatch_lock.Unlock();
+       s->session_dispatch_lock.unlock();
     }
     return false;
   }
@@ -8931,10 +9042,11 @@ bool OSD::require_same_or_newer_map(OpRequestRef& op, epoch_t epoch,
                                    bool is_fast_dispatch)
 {
   const Message *m = op->get_req();
+  const auto osdmap = get_osdmap();
   dout(15) << "require_same_or_newer_map " << epoch
           << " (i am " << osdmap->get_epoch() << ") " << m << dendl;
 
-  ceph_assert(osd_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(osd_lock));
 
   // do they have a newer map?
   if (epoch > osdmap->get_epoch()) {
@@ -8969,7 +9081,7 @@ void OSD::split_pgs(
   const set<spg_t> &childpgids, set<PGRef> *out_pgs,
   OSDMapRef curmap,
   OSDMapRef nextmap,
-  PG::RecoveryCtx *rctx)
+  PeeringCtx &rctx)
 {
   unsigned pg_num = nextmap->get_pg_num(parent->pg_id.pool());
   parent->update_snap_mapper_bits(parent->get_pgid().get_split_bits(pg_num));
@@ -9003,17 +9115,19 @@ void OSD::split_pgs(
       split_bits,
       i->ps(),
       &child->get_pool().info,
-      rctx->transaction);
+      rctx.transaction);
     parent->split_into(
       i->pgid,
       child,
       split_bits);
 
-    child->finish_split_stats(*stat_iter, rctx->transaction);
+    child->init_collection_pool_opts();
+
+    child->finish_split_stats(*stat_iter, rctx.transaction);
     child->unlock();
   }
   ceph_assert(stat_iter != updated_stats.end());
-  parent->finish_split_stats(*stat_iter, rctx->transaction);
+  parent->finish_split_stats(*stat_iter, rctx.transaction);
 }
 
 /*
@@ -9021,7 +9135,10 @@ void OSD::split_pgs(
  */
 void OSD::handle_pg_create(OpRequestRef op)
 {
-  const MOSDPGCreate *m = static_cast<const MOSDPGCreate*>(op->get_req());
+  // NOTE: this can be removed in P release (mimic is the last version to
+  // send MOSDPGCreate messages).
+
+  auto m = op->get_req<MOSDPGCreate>();
   ceph_assert(m->get_type() == MSG_OSD_PG_CREATE);
 
   dout(10) << "handle_pg_create " << *m << dendl;
@@ -9035,6 +9152,7 @@ void OSD::handle_pg_create(OpRequestRef op)
 
   op->mark_started();
 
+  const auto osdmap = get_osdmap();
   map<pg_t,utime_t>::const_iterator ci = m->ctimes.begin();
   for (map<pg_t,pg_create_t>::const_iterator p = m->mkpg.begin();
        p != m->mkpg.end();
@@ -9052,12 +9170,16 @@ void OSD::handle_pg_create(OpRequestRef op)
 
     dout(20) << "mkpg " << on << " e" << created << "@" << ci->second << dendl;
 
+    spg_t pgid;
+    bool mapped = osdmap->get_primary_shard(on, &pgid);
+    ceph_assert(mapped);
+
     // is it still ours?
     vector<int> up, acting;
     int up_primary = -1;
     int acting_primary = -1;
     osdmap->pg_to_up_acting_osds(on, &up, &up_primary, &acting, &acting_primary);
-    int role = osdmap->calc_pg_role(whoami, acting, acting.size());
+    int role = osdmap->calc_pg_role(pg_shard_t(whoami, pgid.shard), acting);
 
     if (acting_primary != whoami) {
       dout(10) << "mkpg " << on << "  not acting_primary (" << acting_primary
@@ -9065,9 +9187,6 @@ void OSD::handle_pg_create(OpRequestRef op)
       continue;
     }
 
-    spg_t pgid;
-    bool mapped = osdmap->get_primary_shard(on, &pgid);
-    ceph_assert(mapped);
 
     PastIntervals pi;
     pg_history_t history;
@@ -9114,33 +9233,7 @@ void OSD::handle_pg_create(OpRequestRef op)
 // ----------------------------------------
 // peering and recovery
 
-PG::RecoveryCtx OSD::create_context()
-{
-  ObjectStore::Transaction *t = new ObjectStore::Transaction;
-  map<int, map<spg_t,pg_query_t> > *query_map =
-    new map<int, map<spg_t, pg_query_t> >;
-  map<int,vector<pair<pg_notify_t, PastIntervals> > > *notify_list =
-    new map<int, vector<pair<pg_notify_t, PastIntervals> > >;
-  map<int,vector<pair<pg_notify_t, PastIntervals> > > *info_map =
-    new map<int,vector<pair<pg_notify_t, PastIntervals> > >;
-  PG::RecoveryCtx rctx(query_map, info_map, notify_list, t);
-  return rctx;
-}
-
-void OSD::dispatch_context_transaction(PG::RecoveryCtx &ctx, PG *pg,
-                                       ThreadPool::TPHandle *handle)
-{
-  if (!ctx.transaction->empty() || ctx.transaction->has_contexts()) {
-    int tr = store->queue_transaction(
-      pg->ch,
-      std::move(*ctx.transaction), TrackedOpRef(), handle);
-    ceph_assert(tr == 0);
-    delete (ctx.transaction);
-    ctx.transaction = new ObjectStore::Transaction;
-  }
-}
-
-void OSD::dispatch_context(PG::RecoveryCtx &ctx, PG *pg, OSDMapRef curmap,
+void OSD::dispatch_context(PeeringCtx &ctx, PG *pg, OSDMapRef curmap,
                            ThreadPool::TPHandle *handle)
 {
   if (!service.get_osdmap()->is_up(whoami)) {
@@ -9148,128 +9241,32 @@ void OSD::dispatch_context(PG::RecoveryCtx &ctx, PG *pg, OSDMapRef curmap,
   } else if (!is_active()) {
     dout(20) << __func__ << " not active" << dendl;
   } else {
-    do_notifies(*ctx.notify_list, curmap);
-    do_queries(*ctx.query_map, curmap);
-    do_infos(*ctx.info_map, curmap);
-  }
-  if ((!ctx.transaction->empty() || ctx.transaction->has_contexts()) && pg) {
-    int tr = store->queue_transaction(
-      pg->ch,
-      std::move(*ctx.transaction), TrackedOpRef(),
-      handle);
-    ceph_assert(tr == 0);
-  }
-  delete ctx.notify_list;
-  delete ctx.query_map;
-  delete ctx.info_map;
-  delete ctx.transaction;
-}
-
-void OSD::discard_context(PG::RecoveryCtx& ctx)
-{
-  delete ctx.notify_list;
-  delete ctx.query_map;
-  delete ctx.info_map;
-  delete ctx.transaction;
-}
-
-
-/** do_notifies
- * Send an MOSDPGNotify to a primary, with a list of PGs that I have
- * content for, and they are primary for.
- */
-
-void OSD::do_notifies(
-  map<int,vector<pair<pg_notify_t,PastIntervals> > >& notify_list,
-  OSDMapRef curmap)
-{
-  for (map<int,
-          vector<pair<pg_notify_t,PastIntervals> > >::iterator it =
-        notify_list.begin();
-       it != notify_list.end();
-       ++it) {
-    if (!curmap->is_up(it->first)) {
-      dout(20) << __func__ << " skipping down osd." << it->first << dendl;
-      continue;
-    }
-    ConnectionRef con = service.get_con_osd_cluster(
-      it->first, curmap->get_epoch());
-    if (!con) {
-      dout(20) << __func__ << " skipping osd." << it->first
-              << " (NULL con)" << dendl;
-      continue;
-    }
-    service.share_map_peer(it->first, con.get(), curmap);
-    dout(7) << __func__ << " osd." << it->first
-           << " on " << it->second.size() << " PGs" << dendl;
-    MOSDPGNotify *m = new MOSDPGNotify(curmap->get_epoch(),
-                                      it->second);
-    con->send_message(m);
-  }
-}
-
-
-/** do_queries
- * send out pending queries for info | summaries
- */
-void OSD::do_queries(map<int, map<spg_t,pg_query_t> >& query_map,
-                    OSDMapRef curmap)
-{
-  for (map<int, map<spg_t,pg_query_t> >::iterator pit = query_map.begin();
-       pit != query_map.end();
-       ++pit) {
-    if (!curmap->is_up(pit->first)) {
-      dout(20) << __func__ << " skipping down osd." << pit->first << dendl;
-      continue;
-    }
-    int who = pit->first;
-    ConnectionRef con = service.get_con_osd_cluster(who, curmap->get_epoch());
-    if (!con) {
-      dout(20) << __func__ << " skipping osd." << who
-              << " (NULL con)" << dendl;
-      continue;
-    }
-    service.share_map_peer(who, con.get(), curmap);
-    dout(7) << __func__ << " querying osd." << who
-           << " on " << pit->second.size() << " PGs" << dendl;
-    MOSDPGQuery *m = new MOSDPGQuery(curmap->get_epoch(), pit->second);
-    con->send_message(m);
-  }
-}
-
-
-void OSD::do_infos(map<int,
-                      vector<pair<pg_notify_t, PastIntervals> > >& info_map,
-                  OSDMapRef curmap)
-{
-  for (map<int,
-          vector<pair<pg_notify_t, PastIntervals> > >::iterator p =
-        info_map.begin();
-       p != info_map.end();
-       ++p) {
-    if (!curmap->is_up(p->first)) {
-      dout(20) << __func__ << " skipping down osd." << p->first << dendl;
-      continue;
-    }
-    for (vector<pair<pg_notify_t,PastIntervals> >::iterator i = p->second.begin();
-        i != p->second.end();
-        ++i) {
-      dout(20) << __func__ << " sending info " << i->first.info
-              << " to shard " << p->first << dendl;
-    }
-    ConnectionRef con = service.get_con_osd_cluster(
-      p->first, curmap->get_epoch());
-    if (!con) {
-      dout(20) << __func__ << " skipping osd." << p->first
-              << " (NULL con)" << dendl;
-      continue;
+    for (auto& [osd, ls] : ctx.message_map) {
+      if (!curmap->is_up(osd)) {
+       dout(20) << __func__ << " skipping down osd." << osd << dendl;
+       continue;
+      }
+      ConnectionRef con = service.get_con_osd_cluster(
+       osd, curmap->get_epoch());
+      if (!con) {
+       dout(20) << __func__ << " skipping osd." << osd << " (NULL con)"
+                << dendl;
+       continue;
+      }
+      service.maybe_share_map(con.get(), curmap);
+      for (auto m : ls) {
+       con->send_message2(m);
+      }
+      ls.clear();
     }
-    service.share_map_peer(p->first, con.get(), curmap);
-    MOSDPGInfo *m = new MOSDPGInfo(curmap->get_epoch());
-    m->pg_list = p->second;
-    con->send_message(m);
   }
-  info_map.clear();
+  if ((!ctx.transaction.empty() || ctx.transaction.has_contexts()) && pg) {
+    int tr = store->queue_transaction(
+      pg->ch,
+      std::move(ctx.transaction), TrackedOpRef(),
+      handle);
+    ceph_assert(tr == 0);
+  }
 }
 
 void OSD::handle_fast_pg_create(MOSDPGCreate2 *m)
@@ -9283,33 +9280,55 @@ void OSD::handle_fast_pg_create(MOSDPGCreate2 *m)
     spg_t pgid = p.first;
     epoch_t created = p.second.first;
     utime_t created_stamp = p.second.second;
-    dout(20) << __func__ << " " << pgid << " e" << created
-            << "@" << created_stamp << dendl;
-    pg_history_t h;
-    h.epoch_created = created;
-    h.epoch_pool_created = created;
-    h.same_up_since = created;
-    h.same_interval_since = created;
-    h.same_primary_since = created;
-    h.last_scrub_stamp = created_stamp;
-    h.last_deep_scrub_stamp = created_stamp;
-    h.last_clean_scrub_stamp = created_stamp;
-
-    enqueue_peering_evt(
-      pgid,
-      PGPeeringEventRef(
-       std::make_shared<PGPeeringEvent>(
-         m->epoch,
-         m->epoch,
-         NullEvt(),
-         true,
-         new PGCreateInfo(
-           pgid,
-           created,
-           h,
-           PastIntervals(),
-           true)
-         )));
+    auto q = m->pg_extra.find(pgid);
+    if (q == m->pg_extra.end()) {
+      dout(20) << __func__ << " " << pgid << " e" << created
+              << "@" << created_stamp
+              << " (no history or past_intervals)" << dendl;
+      // pre-octopus ... no pg history.  this can be removed in Q release.
+      enqueue_peering_evt(
+       pgid,
+       PGPeeringEventRef(
+         std::make_shared<PGPeeringEvent>(
+           m->epoch,
+           m->epoch,
+           NullEvt(),
+           true,
+           new PGCreateInfo(
+             pgid,
+             created,
+             pg_history_t(created, created_stamp),
+             PastIntervals(),
+             true)
+           )));
+    } else {
+      dout(20) << __func__ << " " << pgid << " e" << created
+              << "@" << created_stamp
+              << " history " << q->second.first
+              << " pi " << q->second.second << dendl;
+      if (!q->second.second.empty() &&
+         m->epoch < q->second.second.get_bounds().second) {
+       clog->error() << "got pg_create on " << pgid << " epoch " << m->epoch
+                     << " and unmatched past_intervals " << q->second.second
+                     << " (history " << q->second.first << ")";
+      } else {
+       enqueue_peering_evt(
+         pgid,
+         PGPeeringEventRef(
+           std::make_shared<PGPeeringEvent>(
+             m->epoch,
+             m->epoch,
+             NullEvt(),
+             true,
+             new PGCreateInfo(
+               pgid,
+               m->epoch,
+               q->second.first,
+               q->second.second,
+               true)
+             )));
+      }
+    }
   }
 
   {
@@ -9322,31 +9341,6 @@ void OSD::handle_fast_pg_create(MOSDPGCreate2 *m)
   m->put();
 }
 
-void OSD::handle_fast_pg_query(MOSDPGQuery *m)
-{
-  dout(7) << __func__ << " " << *m << " from " << m->get_source() << dendl;
-  if (!require_osd_peer(m)) {
-    m->put();
-    return;
-  }
-  int from = m->get_source().num();
-  for (auto& p : m->pg_list) {
-    enqueue_peering_evt(
-      p.first,
-      PGPeeringEventRef(
-       std::make_shared<PGPeeringEvent>(
-         p.second.epoch_sent, p.second.epoch_sent,
-         MQuery(
-           p.first,
-           pg_shard_t(from, p.second.from),
-           p.second,
-           p.second.epoch_sent),
-         false))
-      );
-  }
-  m->put();
-}
-
 void OSD::handle_fast_pg_notify(MOSDPGNotify* m)
 {
   dout(7) << __func__ << " " << *m << " from " << m->get_source() << dendl;
@@ -9356,24 +9350,23 @@ void OSD::handle_fast_pg_notify(MOSDPGNotify* m)
   }
   int from = m->get_source().num();
   for (auto& p : m->get_pg_list()) {
-    spg_t pgid(p.first.info.pgid.pgid, p.first.to);
+    spg_t pgid(p.info.pgid.pgid, p.to);
     enqueue_peering_evt(
       pgid,
       PGPeeringEventRef(
        std::make_shared<PGPeeringEvent>(
-         p.first.epoch_sent,
-         p.first.query_epoch,
+         p.epoch_sent,
+         p.query_epoch,
          MNotifyRec(
-           pgid, pg_shard_t(from, p.first.from),
-           p.first,
-           m->get_connection()->get_features(),
-           p.second),
+           pgid, pg_shard_t(from, p.from),
+           p,
+           m->get_connection()->get_features()),
          true,
          new PGCreateInfo(
            pgid,
-           p.first.query_epoch,
-           p.first.info.history,
-           p.second,
+           p.query_epoch,
+           p.info.history,
+           p.past_intervals,
            false)
          )));
   }
@@ -9390,14 +9383,14 @@ void OSD::handle_fast_pg_info(MOSDPGInfo* m)
   int from = m->get_source().num();
   for (auto& p : m->pg_list) {
     enqueue_peering_evt(
-      spg_t(p.first.info.pgid.pgid, p.first.to),
+      spg_t(p.info.pgid.pgid, p.to),
       PGPeeringEventRef(
-       std::make_shared<PGPeeringEvent>(
-         p.first.epoch_sent, p.first.query_epoch,
-         MInfoRec(
-           pg_shard_t(from, p.first.from),
-           p.first.info,
-           p.first.epoch_sent)))
+       std::make_shared<PGPeeringEvent>(
+         p.epoch_sent, p.query_epoch,
+         MInfoRec(
+           pg_shard_t(from, p.from),
+           p.info,
+           p.epoch_sent)))
       );
   }
   m->put();
@@ -9416,7 +9409,7 @@ void OSD::handle_fast_pg_remove(MOSDPGRemove *m)
       PGPeeringEventRef(
        std::make_shared<PGPeeringEvent>(
          m->get_epoch(), m->get_epoch(),
-         PG::DeleteStart())));
+         PeeringState::DeleteStart())));
   }
   m->put();
 }
@@ -9437,14 +9430,14 @@ void OSD::handle_fast_force_recovery(MOSDForceRecovery *m)
          PGPeeringEventRef(
            std::make_shared<PGPeeringEvent>(
              epoch, epoch,
-             PG::UnsetForceBackfill())));
+             PeeringState::UnsetForceBackfill())));
       } else {
        enqueue_peering_evt(
          pgid,
          PGPeeringEventRef(
            std::make_shared<PGPeeringEvent>(
              epoch, epoch,
-             PG::SetForceBackfill())));
+             PeeringState::SetForceBackfill())));
       }
     } else if (m->options & OFR_RECOVERY) {
       if (m->options & OFR_CANCEL) {
@@ -9453,14 +9446,14 @@ void OSD::handle_fast_force_recovery(MOSDForceRecovery *m)
          PGPeeringEventRef(
            std::make_shared<PGPeeringEvent>(
              epoch, epoch,
-             PG::UnsetForceRecovery())));
+             PeeringState::UnsetForceRecovery())));
       } else {
        enqueue_peering_evt(
          pgid,
          PGPeeringEventRef(
            std::make_shared<PGPeeringEvent>(
              epoch, epoch,
-             PG::SetForceRecovery())));
+             PeeringState::SetForceRecovery())));
       }
     }
   }
@@ -9488,28 +9481,45 @@ void OSD::handle_pg_query_nopg(const MQuery& q)
        osdmap->get_epoch(), empty,
        q.query.epoch_sent);
     } else {
-      vector<pair<pg_notify_t,PastIntervals>> ls;
-      ls.push_back(
-       make_pair(
-         pg_notify_t(
-           q.query.from, q.query.to,
-           q.query.epoch_sent,
-           osdmap->get_epoch(),
-           empty),
-         PastIntervals()));
-      m = new MOSDPGNotify(osdmap->get_epoch(), ls);
-    }
-    service.share_map_peer(q.from.osd, con.get(), osdmap);
+      pg_notify_t notify{q.query.from, q.query.to,
+                        q.query.epoch_sent,
+                        osdmap->get_epoch(),
+                        empty,
+                        PastIntervals()};
+      m = new MOSDPGNotify2(spg_t{pgid.pgid, q.query.from},
+                           std::move(notify));
+    }
+    service.maybe_share_map(con.get(), osdmap);
     con->send_message(m);
   }
 }
 
+void OSDService::queue_check_readable(spg_t spgid,
+                                     epoch_t lpr,
+                                     ceph::signedspan delay)
+{
+  if (delay == ceph::signedspan::zero()) {
+    osd->enqueue_peering_evt(
+      spgid,
+      PGPeeringEventRef(
+       std::make_shared<PGPeeringEvent>(
+         lpr, lpr,
+         PeeringState::CheckReadable())));
+  } else {
+    mono_timer.add_event(
+      delay,
+      [this, spgid, lpr]() {
+       queue_check_readable(spgid, lpr);
+      });
+  }
+}
+
 
 // =========================================================
 // RECOVERY
 
 void OSDService::_maybe_queue_recovery() {
-  ceph_assert(recovery_lock.is_locked_by_me());
+  ceph_assert(ceph_mutex_is_locked_by_me(recovery_lock));
   uint64_t available_pushes;
   while (!awaiting_throttle.empty() &&
         _recover_now(&available_pushes)) {
@@ -9540,7 +9550,7 @@ bool OSDService::_recover_now(uint64_t *available_pushes)
     return false;
   }
 
-  uint64_t max = cct->_conf->osd_recovery_max_active;
+  uint64_t max = osd->get_recovery_max_active();
   if (max <= recovery_ops_active + recovery_ops_reserved) {
     dout(15) << __func__ << " active " << recovery_ops_active
             << " + reserved " << recovery_ops_reserved
@@ -9554,6 +9564,26 @@ bool OSDService::_recover_now(uint64_t *available_pushes)
   return true;
 }
 
+unsigned OSDService::get_target_pg_log_entries() const
+{
+  auto num_pgs = osd->get_num_pgs();
+  auto target = cct->_conf->osd_target_pg_log_entries_per_osd;
+  if (num_pgs > 0 && target > 0) {
+    // target an even spread of our budgeted log entries across all
+    // PGs.  note that while we only get to control the entry count
+    // for primary PGs, we'll normally be responsible for a mix of
+    // primary and replica PGs (for the same pool(s) even), so this
+    // will work out.
+    return std::max<unsigned>(
+      std::min<unsigned>(target / num_pgs,
+                        cct->_conf->osd_max_pg_log_entries),
+      cct->_conf->osd_min_pg_log_entries);
+  } else {
+    // fall back to a per-pg value.
+    return cct->_conf->osd_min_pg_log_entries;
+  }
+}
+
 void OSD::do_recovery(
   PG *pg, epoch_t queued, uint64_t reserved_pushes,
   ThreadPool::TPHandle &handle)
@@ -9572,7 +9602,7 @@ void OSD::do_recovery(
     std::lock_guard l(service.sleep_lock);
     if (recovery_sleep > 0 && service.recovery_needs_sleep) {
       PGRef pgref(pg);
-      auto recovery_requeue_callback = new FunctionContext([this, pgref, queued, reserved_pushes](int r) {
+      auto recovery_requeue_callback = new LambdaContext([this, pgref, queued, reserved_pushes](int r) {
         dout(20) << "do_recovery wake up at "
                  << ceph_clock_now()
                 << ", re-queuing recovery" << dendl;
@@ -9584,12 +9614,14 @@ void OSD::do_recovery(
       // This is true for the first recovery op and when the previous recovery op
       // has been scheduled in the past. The next recovery op is scheduled after
       // completing the sleep from now.
-      if (service.recovery_schedule_time < ceph_clock_now()) {
-        service.recovery_schedule_time = ceph_clock_now();
+
+      if (auto now = ceph::real_clock::now();
+         service.recovery_schedule_time < now) {
+        service.recovery_schedule_time = now;
       }
-      service.recovery_schedule_time += recovery_sleep;
+      service.recovery_schedule_time += ceph::make_timespan(recovery_sleep);
       service.sleep_timer.add_event_at(service.recovery_schedule_time,
-                                               recovery_requeue_callback);
+                                      recovery_requeue_callback);
       dout(20) << "Recovery event scheduled at "
                << service.recovery_schedule_time << dendl;
       return;
@@ -9612,13 +9644,13 @@ void OSD::do_recovery(
 #endif
 
     bool do_unfound = pg->start_recovery_ops(reserved_pushes, handle, &started);
-    dout(10) << "do_recovery started " << started << "/" << reserved_pushes 
+    dout(10) << "do_recovery started " << started << "/" << reserved_pushes
             << " on " << *pg << dendl;
 
     if (do_unfound) {
-      PG::RecoveryCtx rctx = create_context();
+      PeeringCtx rctx;
       rctx.handle = &handle;
-      pg->find_unfound(queued, &rctx);
+      pg->find_unfound(queued, rctx);
       dispatch_context(rctx, pg, pg->get_osdmap());
     }
   }
@@ -9633,7 +9665,7 @@ void OSDService::start_recovery_op(PG *pg, const hobject_t& soid)
   std::lock_guard l(recovery_lock);
   dout(10) << "start_recovery_op " << *pg << " " << soid
           << " (" << recovery_ops_active << "/"
-          << cct->_conf->osd_recovery_max_active << " rops)"
+          << osd->get_recovery_max_active() << " rops)"
           << dendl;
   recovery_ops_active++;
 
@@ -9649,7 +9681,8 @@ void OSDService::finish_recovery_op(PG *pg, const hobject_t& soid, bool dequeue)
   std::lock_guard l(recovery_lock);
   dout(10) << "finish_recovery_op " << *pg << " " << soid
           << " dequeue=" << dequeue
-          << " (" << recovery_ops_active << "/" << cct->_conf->osd_recovery_max_active << " rops)"
+          << " (" << recovery_ops_active << "/"
+          << osd->get_recovery_max_active() << " rops)"
           << dendl;
 
   // adjust count
@@ -9667,6 +9700,9 @@ void OSDService::finish_recovery_op(PG *pg, const hobject_t& soid, bool dequeue)
 
 bool OSDService::is_recovery_active()
 {
+  if (cct->_conf->osd_debug_pretend_recovery_active) {
+    return true;
+  }
   return local_reserver.has_reservation() || remote_reserver.has_reservation();
 }
 
@@ -9701,8 +9737,10 @@ void OSD::enqueue_op(spg_t pg, OpRequestRef&& op, epoch_t epoch)
   const unsigned priority = op->get_req()->get_priority();
   const int cost = op->get_req()->get_cost();
   const uint64_t owner = op->get_req()->get_source().num();
+  const int type = op->get_req()->get_type();
 
   dout(15) << "enqueue_op " << op << " prio " << priority
+           << " type " << type
           << " cost " << cost
           << " latency " << latency
           << " epoch " << epoch
@@ -9710,33 +9748,38 @@ void OSD::enqueue_op(spg_t pg, OpRequestRef&& op, epoch_t epoch)
   op->osd_trace.event("enqueue op");
   op->osd_trace.keyval("priority", priority);
   op->osd_trace.keyval("cost", cost);
+
+  auto enqueue_span = tracing::osd::tracer.add_span(__func__, op->osd_parent_span);
+  enqueue_span->AddEvent(__func__, {
+    {"priority", priority},
+    {"cost", cost},
+    {"epoch", epoch},
+    {"owner", owner},
+    {"type", type}
+    });
+
   op->mark_queued_for_pg();
   logger->tinc(l_osd_op_before_queue_op_lat, latency);
-  op_shardedwq.queue(
-    OpQueueItem(
-      unique_ptr<OpQueueItem::OpQueueable>(new PGOpItem(pg, std::move(op))),
-      cost, priority, stamp, owner, epoch));
+  if (type == MSG_OSD_PG_PUSH ||
+      type == MSG_OSD_PG_PUSH_REPLY) {
+    op_shardedwq.queue(
+      OpSchedulerItem(
+        unique_ptr<OpSchedulerItem::OpQueueable>(new PGRecoveryMsg(pg, std::move(op))),
+        cost, priority, stamp, owner, epoch));
+  } else {
+    op_shardedwq.queue(
+      OpSchedulerItem(
+        unique_ptr<OpSchedulerItem::OpQueueable>(new PGOpItem(pg, std::move(op))),
+        cost, priority, stamp, owner, epoch));
+  }
 }
 
 void OSD::enqueue_peering_evt(spg_t pgid, PGPeeringEventRef evt)
 {
   dout(15) << __func__ << " " << pgid << " " << evt->get_desc() << dendl;
   op_shardedwq.queue(
-    OpQueueItem(
-      unique_ptr<OpQueueItem::OpQueueable>(new PGPeeringItem(pgid, evt)),
-      10,
-      cct->_conf->osd_peering_op_priority,
-      utime_t(),
-      0,
-      evt->get_epoch_sent()));
-}
-
-void OSD::enqueue_peering_evt_front(spg_t pgid, PGPeeringEventRef evt)
-{
-  dout(15) << __func__ << " " << pgid << " " << evt->get_desc() << dendl;
-  op_shardedwq.queue_front(
-    OpQueueItem(
-      unique_ptr<OpQueueItem::OpQueueable>(new PGPeeringItem(pgid, evt)),
+    OpSchedulerItem(
+      unique_ptr<OpSchedulerItem::OpQueueable>(new PGPeeringItem(pgid, evt)),
       10,
       cct->_conf->osd_peering_op_priority,
       utime_t(),
@@ -9751,24 +9794,26 @@ void OSD::dequeue_op(
   PGRef pg, OpRequestRef op,
   ThreadPool::TPHandle &handle)
 {
+  const Message *m = op->get_req();
+
   FUNCTRACE(cct);
-  OID_EVENT_TRACE_WITH_MSG(op->get_req(), "DEQUEUE_OP_BEGIN", false);
+  OID_EVENT_TRACE_WITH_MSG(m, "DEQUEUE_OP_BEGIN", false);
 
   utime_t now = ceph_clock_now();
   op->set_dequeued_time(now);
-  utime_t latency = now - op->get_req()->get_recv_stamp();
-  dout(10) << "dequeue_op " << op << " prio " << op->get_req()->get_priority()
-          << " cost " << op->get_req()->get_cost()
+
+  utime_t latency = now - m->get_recv_stamp();
+  dout(10) << "dequeue_op " << op << " prio " << m->get_priority()
+          << " cost " << m->get_cost()
           << " latency " << latency
-          << " " << *(op->get_req())
+          << " " << *m
           << " pg " << *pg << dendl;
 
   logger->tinc(l_osd_op_before_dequeue_op_lat, latency);
 
-  auto priv = op->get_req()->get_connection()->get_priv();
-  if (auto session = static_cast<Session *>(priv.get()); session) {
-    maybe_share_map(session, op, pg->get_osdmap());
-  }
+  service.maybe_share_map(m->get_connection().get(),
+                         pg->get_osdmap(),
+                         op->sent_epoch);
 
   if (pg->is_deleting())
     return;
@@ -9780,7 +9825,7 @@ void OSD::dequeue_op(
 
   // finish
   dout(10) << "dequeue_op " << op << " finish" << dendl;
-  OID_EVENT_TRACE_WITH_MSG(op->get_req(), "DEQUEUE_OP_END", false);
+  OID_EVENT_TRACE_WITH_MSG(m, "DEQUEUE_OP_END", false);
 }
 
 
@@ -9790,9 +9835,9 @@ void OSD::dequeue_peering_evt(
   PGPeeringEventRef evt,
   ThreadPool::TPHandle& handle)
 {
-  PG::RecoveryCtx rctx = create_context();
   auto curmap = sdata->get_osdmap();
-  epoch_t need_up_thru = 0, same_interval_since = 0;
+  bool need_up_thru = false;
+  epoch_t same_interval_since = 0;
   if (!pg) {
     if (const MQuery *q = dynamic_cast<const MQuery*>(evt->evt.get())) {
       handle_pg_query_nopg(*q);
@@ -9800,15 +9845,14 @@ void OSD::dequeue_peering_evt(
       derr << __func__ << " unrecognized pg-less event " << evt->get_desc() << dendl;
       ceph_abort();
     }
-  } else if (advance_pg(curmap->get_epoch(), pg, handle, &rctx)) {
-    pg->do_peering_event(evt, &rctx);
+  } else if (PeeringCtx rctx;
+            advance_pg(curmap->get_epoch(), pg, handle, rctx)) {
+    pg->do_peering_event(evt, rctx);
     if (pg->is_deleted()) {
-      // do not dispatch rctx; the final _delete_some already did it.
-      discard_context(rctx);
       pg->unlock();
       return;
     }
-    dispatch_context_transaction(rctx, pg, &handle);
+    dispatch_context(rctx, pg, curmap, &handle);
     need_up_thru = pg->get_need_up_thru();
     same_interval_since = pg->get_same_interval_since();
     pg->unlock();
@@ -9817,7 +9861,6 @@ void OSD::dequeue_peering_evt(
   if (need_up_thru) {
     queue_want_up_thru(same_interval_since);
   }
-  dispatch_context(rctx, pg, curmap, &handle);
 
   service.send_pg_temp();
 }
@@ -9834,7 +9877,7 @@ void OSD::dequeue_delete(
     PGPeeringEventRef(
       std::make_shared<PGPeeringEvent>(
        e, e,
-       PG::DeleteSome())),
+       PeeringState::DeleteSome())),
     handle);
 }
 
@@ -9858,6 +9901,22 @@ const char** OSD::get_tracked_conf_keys() const
     "osd_map_cache_size",
     "osd_pg_epoch_max_lag_factor",
     "osd_pg_epoch_persisted_max_stale",
+    "osd_recovery_sleep",
+    "osd_recovery_sleep_hdd",
+    "osd_recovery_sleep_ssd",
+    "osd_recovery_sleep_hybrid",
+    "osd_delete_sleep",
+    "osd_delete_sleep_hdd",
+    "osd_delete_sleep_ssd",
+    "osd_delete_sleep_hybrid",
+    "osd_snap_trim_sleep",
+    "osd_snap_trim_sleep_hdd",
+    "osd_snap_trim_sleep_ssd",
+    "osd_snap_trim_sleep_hybrid",
+    "osd_scrub_sleep",
+    "osd_recovery_max_active",
+    "osd_recovery_max_active_hdd",
+    "osd_recovery_max_active_ssd",
     // clog & admin clog
     "clog_to_monitors",
     "clog_to_syslog",
@@ -9874,6 +9933,9 @@ const char** OSD::get_tracked_conf_keys() const
     "osd_client_message_cap",
     "osd_heartbeat_min_size",
     "osd_heartbeat_interval",
+    "osd_object_clean_region_max_num_intervals",
+    "osd_scrub_min_interval",
+    "osd_scrub_max_interval",
     NULL
   };
   return KEYS;
@@ -9882,10 +9944,31 @@ const char** OSD::get_tracked_conf_keys() const
 void OSD::handle_conf_change(const ConfigProxy& conf,
                             const std::set <std::string> &changed)
 {
-  Mutex::Locker l(osd_lock);
-  if (changed.count("osd_max_backfills")) {
-    service.local_reserver.set_max(cct->_conf->osd_max_backfills);
-    service.remote_reserver.set_max(cct->_conf->osd_max_backfills);
+  std::lock_guard l{osd_lock};
+
+  if (changed.count("osd_max_backfills") ||
+      changed.count("osd_delete_sleep") ||
+      changed.count("osd_delete_sleep_hdd") ||
+      changed.count("osd_delete_sleep_ssd") ||
+      changed.count("osd_delete_sleep_hybrid") ||
+      changed.count("osd_snap_trim_sleep") ||
+      changed.count("osd_snap_trim_sleep_hdd") ||
+      changed.count("osd_snap_trim_sleep_ssd") ||
+      changed.count("osd_snap_trim_sleep_hybrid") ||
+      changed.count("osd_scrub_sleep") ||
+      changed.count("osd_recovery_sleep") ||
+      changed.count("osd_recovery_sleep_hdd") ||
+      changed.count("osd_recovery_sleep_ssd") ||
+      changed.count("osd_recovery_sleep_hybrid") ||
+      changed.count("osd_recovery_max_active") ||
+      changed.count("osd_recovery_max_active_hdd") ||
+      changed.count("osd_recovery_max_active_ssd")) {
+    if (!maybe_override_options_for_qos() &&
+        changed.count("osd_max_backfills")) {
+      // Scheduler is not "mclock". Fallback to earlier behavior
+      service.local_reserver.set_max(cct->_conf->osd_max_backfills);
+      service.remote_reserver.set_max(cct->_conf->osd_max_backfills);
+    }
   }
   if (changed.count("osd_min_recovery_priority")) {
     service.local_reserver.set_min_priority(cct->_conf->osd_min_recovery_priority);
@@ -9949,42 +10032,207 @@ void OSD::handle_conf_change(const ConfigProxy& conf,
   if (changed.count("osd_client_message_cap")) {
     uint64_t newval = cct->_conf->osd_client_message_cap;
     Messenger::Policy pol = client_messenger->get_policy(entity_name_t::TYPE_CLIENT);
-    if (pol.throttler_messages && newval > 0) {
+    if (pol.throttler_messages) {
       pol.throttler_messages->reset_max(newval);
     }
   }
   if (changed.count("osd_client_message_size_cap")) {
     uint64_t newval = cct->_conf->osd_client_message_size_cap;
     Messenger::Policy pol = client_messenger->get_policy(entity_name_t::TYPE_CLIENT);
-    if (pol.throttler_bytes && newval > 0) {
+    if (pol.throttler_bytes) {
       pol.throttler_bytes->reset_max(newval);
     }
   }
+  if (changed.count("osd_object_clean_region_max_num_intervals")) {
+    ObjectCleanRegions::set_max_num_intervals(cct->_conf->osd_object_clean_region_max_num_intervals);
+  }
 
+  if (changed.count("osd_scrub_min_interval") ||
+      changed.count("osd_scrub_max_interval")) {
+    resched_all_scrubs();
+    dout(0) << __func__ << ": scrub interval change" << dendl;
+  }
   check_config();
+  if (changed.count("osd_asio_thread_count")) {
+    service.poolctx.stop();
+    service.poolctx.start(conf.get_val<std::uint64_t>("osd_asio_thread_count"));
+  }
+}
+
+void OSD::maybe_override_max_osd_capacity_for_qos()
+{
+  // If the scheduler enabled is mclock, override the default
+  // osd capacity with the value obtained from running the
+  // osd bench test. This is later used to setup mclock.
+  if ((cct->_conf.get_val<std::string>("osd_op_queue") == "mclock_scheduler") &&
+      (cct->_conf.get_val<bool>("osd_mclock_skip_benchmark") == false) &&
+      (!unsupported_objstore_for_qos())) {
+    std::string max_capacity_iops_config;
+    bool force_run_benchmark =
+      cct->_conf.get_val<bool>("osd_mclock_force_run_benchmark_on_init");
+
+    if (store_is_rotational) {
+      max_capacity_iops_config = "osd_mclock_max_capacity_iops_hdd";
+    } else {
+      max_capacity_iops_config = "osd_mclock_max_capacity_iops_ssd";
+    }
+
+    if (!force_run_benchmark) {
+      double default_iops = 0.0;
+
+      // Get the current osd iops capacity
+      double cur_iops = cct->_conf.get_val<double>(max_capacity_iops_config);
+
+      // Get the default max iops capacity
+      auto val = cct->_conf.get_val_default(max_capacity_iops_config);
+      if (!val.has_value()) {
+        derr << __func__ << " Unable to determine default value of "
+            << max_capacity_iops_config << dendl;
+        // Cannot determine default iops. Force a run of the OSD benchmark.
+        force_run_benchmark = true;
+      } else {
+        // Default iops
+        default_iops = std::stod(val.value());
+      }
+
+      // Determine if we really need to run the osd benchmark
+      if (!force_run_benchmark && (default_iops != cur_iops)) {
+        dout(1) << __func__ << std::fixed << std::setprecision(2)
+                << " default_iops: " << default_iops
+                << " cur_iops: " << cur_iops
+                << ". Skip OSD benchmark test." << dendl;
+        return;
+      }
+    }
+
+    // Run osd bench: write 100 4MiB objects with blocksize 4KiB
+    int64_t count = 12288000; // Count of bytes to write
+    int64_t bsize = 4096;     // Block size
+    int64_t osize = 4194304;  // Object size
+    int64_t onum = 100;       // Count of objects to write
+    double elapsed = 0.0;     // Time taken to complete the test
+    double iops = 0.0;
+    stringstream ss;
+    int ret = run_osd_bench_test(count, bsize, osize, onum, &elapsed, ss);
+    if (ret != 0) {
+      derr << __func__
+           << " osd bench err: " << ret
+           << " osd bench errstr: " << ss.str()
+           << dendl;
+      return;
+    }
+
+    double rate = count / elapsed;
+    iops = rate / bsize;
+    dout(1) << __func__
+            << " osd bench result -"
+            << std::fixed << std::setprecision(3)
+            << " bandwidth (MiB/sec): " << rate / (1024 * 1024)
+            << " iops: " << iops
+            << " elapsed_sec: " << elapsed
+            << dendl;
+
+    // Persist iops to the MON store
+    ret = mon_cmd_set_config(max_capacity_iops_config, std::to_string(iops));
+    if (ret < 0) {
+      // Fallback to setting the config within the in-memory "values" map.
+      cct->_conf.set_val(max_capacity_iops_config, std::to_string(iops));
+    }
+
+    // Override the max osd capacity for all shards
+    for (auto& shard : shards) {
+      shard->update_scheduler_config();
+    }
+  }
+}
+
+bool OSD::maybe_override_options_for_qos()
+{
+  // If the scheduler enabled is mclock, override the recovery, backfill
+  // and sleep options so that mclock can meet the QoS goals.
+  if (cct->_conf.get_val<std::string>("osd_op_queue") == "mclock_scheduler" &&
+      !unsupported_objstore_for_qos()) {
+    dout(1) << __func__
+            << ": Changing recovery/backfill/sleep settings for QoS" << dendl;
+
+    // Set high value for recovery max active
+    uint32_t rec_max_active = 1000;
+    cct->_conf.set_val(
+      "osd_recovery_max_active", std::to_string(rec_max_active));
+    cct->_conf.set_val(
+      "osd_recovery_max_active_hdd", std::to_string(rec_max_active));
+    cct->_conf.set_val(
+      "osd_recovery_max_active_ssd", std::to_string(rec_max_active));
+
+    // Set high value for osd_max_backfill
+    uint32_t max_backfills = 1000;
+    cct->_conf.set_val("osd_max_backfills", std::to_string(max_backfills));
+    service.local_reserver.set_max(max_backfills);
+    service.remote_reserver.set_max(max_backfills);
+
+    // Disable recovery sleep
+    cct->_conf.set_val("osd_recovery_sleep", std::to_string(0));
+    cct->_conf.set_val("osd_recovery_sleep_hdd", std::to_string(0));
+    cct->_conf.set_val("osd_recovery_sleep_ssd", std::to_string(0));
+    cct->_conf.set_val("osd_recovery_sleep_hybrid", std::to_string(0));
+
+    // Disable delete sleep
+    cct->_conf.set_val("osd_delete_sleep", std::to_string(0));
+    cct->_conf.set_val("osd_delete_sleep_hdd", std::to_string(0));
+    cct->_conf.set_val("osd_delete_sleep_ssd", std::to_string(0));
+    cct->_conf.set_val("osd_delete_sleep_hybrid", std::to_string(0));
+
+    // Disable snap trim sleep
+    cct->_conf.set_val("osd_snap_trim_sleep", std::to_string(0));
+    cct->_conf.set_val("osd_snap_trim_sleep_hdd", std::to_string(0));
+    cct->_conf.set_val("osd_snap_trim_sleep_ssd", std::to_string(0));
+    cct->_conf.set_val("osd_snap_trim_sleep_hybrid", std::to_string(0));
+
+    // Disable scrub sleep
+    cct->_conf.set_val("osd_scrub_sleep", std::to_string(0));
+    return true;
+  }
+  return false;
+}
+
+int OSD::mon_cmd_set_config(const std::string &key, const std::string &val)
+{
+  std::string cmd =
+    "{"
+      "\"prefix\": \"config set\", "
+      "\"who\": \"osd." + std::to_string(whoami) + "\", "
+      "\"name\": \"" + key + "\", "
+      "\"value\": \"" + val + "\""
+    "}";
+
+  vector<std::string> vcmd{cmd};
+  bufferlist inbl;
+  std::string outs;
+  C_SaferCond cond;
+  monc->start_mon_command(vcmd, inbl, nullptr, &outs, &cond);
+  int r = cond.wait();
+  if (r < 0) {
+    derr << __func__ << " Failed to set config key " << key
+         << " err: " << cpp_strerror(r)
+         << " errstr: " << outs << dendl;
+    return r;
+  }
+
+  return 0;
+}
+
+bool OSD::unsupported_objstore_for_qos()
+{
+  static const std::vector<std::string> unsupported_objstores = { "filestore" };
+  return std::find(unsupported_objstores.begin(),
+                   unsupported_objstores.end(),
+                   store->get_type()) != unsupported_objstores.end();
 }
 
 void OSD::update_log_config()
 {
-  map<string,string> log_to_monitors;
-  map<string,string> log_to_syslog;
-  map<string,string> log_channel;
-  map<string,string> log_prio;
-  map<string,string> log_to_graylog;
-  map<string,string> log_to_graylog_host;
-  map<string,string> log_to_graylog_port;
-  uuid_d fsid;
-  string host;
-
-  if (parse_log_client_options(cct, log_to_monitors, log_to_syslog,
-                              log_channel, log_prio, log_to_graylog,
-                              log_to_graylog_host, log_to_graylog_port,
-                              fsid, host) == 0)
-    clog->update_config(log_to_monitors, log_to_syslog,
-                       log_channel, log_prio, log_to_graylog,
-                       log_to_graylog_host, log_to_graylog_port,
-                       fsid, host);
-  derr << "log_to_monitors " << log_to_monitors << dendl;
+  auto parsed_options = clog->parse_client_options(cct);
+  derr << "log_to_monitors " << parsed_options.log_to_monitors << dendl;
 }
 
 void OSD::check_config()
@@ -9995,6 +10243,11 @@ void OSD::check_config()
                 << " is not > osd_pg_epoch_persisted_max_stale ("
                 << cct->_conf->osd_pg_epoch_persisted_max_stale << ")";
   }
+  if (cct->_conf->osd_object_clean_region_max_num_intervals < 0) {
+    clog->warn() << "osd_object_clean_region_max_num_intervals ("
+                 << cct->_conf->osd_object_clean_region_max_num_intervals
+                << ") is < 0";
+  }
 }
 
 // --------------------------------
@@ -10003,203 +10256,17 @@ void OSD::get_latest_osdmap()
 {
   dout(10) << __func__ << " -- start" << dendl;
 
-  C_SaferCond cond;
-  service.objecter->wait_for_latest_osdmap(&cond);
-  cond.wait();
+  boost::system::error_code ec;
+  service.objecter->wait_for_latest_osdmap(ceph::async::use_blocked[ec]);
 
   dout(10) << __func__ << " -- finish" << dendl;
 }
 
 // --------------------------------
 
-int OSD::init_op_flags(OpRequestRef& op)
-{
-  const MOSDOp *m = static_cast<const MOSDOp*>(op->get_req());
-  vector<OSDOp>::const_iterator iter;
-
-  // client flags have no bearing on whether an op is a read, write, etc.
-  op->rmw_flags = 0;
-
-  if (m->has_flag(CEPH_OSD_FLAG_RWORDERED)) {
-    op->set_force_rwordered();
-  }
-
-  // set bits based on op codes, called methods.
-  for (iter = m->ops.begin(); iter != m->ops.end(); ++iter) {
-    if ((iter->op.op == CEPH_OSD_OP_WATCH &&
-        iter->op.watch.op == CEPH_OSD_WATCH_OP_PING)) {
-      /* This a bit odd.  PING isn't actually a write.  It can't
-       * result in an update to the object_info.  PINGs also aren't
-       * resent, so there's no reason to write out a log entry.
-       *
-       * However, we pipeline them behind writes, so let's force
-       * the write_ordered flag.
-       */
-      op->set_force_rwordered();
-    } else {
-      if (ceph_osd_op_mode_modify(iter->op.op))
-       op->set_write();
-    }
-    if (ceph_osd_op_mode_read(iter->op.op))
-      op->set_read();
-
-    // set READ flag if there are src_oids
-    if (iter->soid.oid.name.length())
-      op->set_read();
-
-    // set PGOP flag if there are PG ops
-    if (ceph_osd_op_type_pg(iter->op.op))
-      op->set_pg_op();
-
-    if (ceph_osd_op_mode_cache(iter->op.op))
-      op->set_cache();
-
-    // check for ec base pool
-    int64_t poolid = m->get_pg().pool();
-    const pg_pool_t *pool = osdmap->get_pg_pool(poolid);
-    if (pool && pool->is_tier()) {
-      const pg_pool_t *base_pool = osdmap->get_pg_pool(pool->tier_of);
-      if (base_pool && base_pool->require_rollback()) {
-        if ((iter->op.op != CEPH_OSD_OP_READ) &&
-            (iter->op.op != CEPH_OSD_OP_CHECKSUM) &&
-            (iter->op.op != CEPH_OSD_OP_CMPEXT) &&
-            (iter->op.op != CEPH_OSD_OP_STAT) &&
-            (iter->op.op != CEPH_OSD_OP_ISDIRTY) &&
-            (iter->op.op != CEPH_OSD_OP_UNDIRTY) &&
-            (iter->op.op != CEPH_OSD_OP_GETXATTR) &&
-            (iter->op.op != CEPH_OSD_OP_GETXATTRS) &&
-            (iter->op.op != CEPH_OSD_OP_CMPXATTR) &&
-            (iter->op.op != CEPH_OSD_OP_ASSERT_VER) &&
-            (iter->op.op != CEPH_OSD_OP_LIST_WATCHERS) &&
-            (iter->op.op != CEPH_OSD_OP_LIST_SNAPS) &&
-            (iter->op.op != CEPH_OSD_OP_SETALLOCHINT) &&
-            (iter->op.op != CEPH_OSD_OP_WRITEFULL) &&
-            (iter->op.op != CEPH_OSD_OP_ROLLBACK) &&
-            (iter->op.op != CEPH_OSD_OP_CREATE) &&
-            (iter->op.op != CEPH_OSD_OP_DELETE) &&
-            (iter->op.op != CEPH_OSD_OP_SETXATTR) &&
-            (iter->op.op != CEPH_OSD_OP_RMXATTR) &&
-            (iter->op.op != CEPH_OSD_OP_STARTSYNC) &&
-            (iter->op.op != CEPH_OSD_OP_COPY_GET) &&
-            (iter->op.op != CEPH_OSD_OP_COPY_FROM)) {
-          op->set_promote();
-        }
-      }
-    }
-
-    switch (iter->op.op) {
-    case CEPH_OSD_OP_CALL:
-      {
-       bufferlist::iterator bp = const_cast<bufferlist&>(iter->indata).begin();
-       int is_write, is_read;
-       string cname, mname;
-       bp.copy(iter->op.cls.class_len, cname);
-       bp.copy(iter->op.cls.method_len, mname);
-
-       ClassHandler::ClassData *cls;
-       int r = class_handler->open_class(cname, &cls);
-       if (r) {
-         derr << "class " << cname << " open got " << cpp_strerror(r) << dendl;
-         if (r == -ENOENT)
-           r = -EOPNOTSUPP;
-         else if (r != -EPERM) // propagate permission errors
-           r = -EIO;
-         return r;
-       }
-       int flags = cls->get_method_flags(mname.c_str());
-       if (flags < 0) {
-         if (flags == -ENOENT)
-           r = -EOPNOTSUPP;
-         else
-           r = flags;
-         return r;
-       }
-       is_read = flags & CLS_METHOD_RD;
-       is_write = flags & CLS_METHOD_WR;
-        bool is_promote = flags & CLS_METHOD_PROMOTE;
-
-       dout(10) << "class " << cname << " method " << mname << " "
-                << "flags=" << (is_read ? "r" : "")
-                             << (is_write ? "w" : "")
-                             << (is_promote ? "p" : "")
-                 << dendl;
-       if (is_read)
-         op->set_class_read();
-       if (is_write)
-         op->set_class_write();
-        if (is_promote)
-          op->set_promote();
-        op->add_class(std::move(cname), std::move(mname), is_read, is_write,
-                      cls->whitelisted);
-       break;
-      }
-
-    case CEPH_OSD_OP_WATCH:
-      // force the read bit for watch since it is depends on previous
-      // watch state (and may return early if the watch exists) or, in
-      // the case of ping, is simply a read op.
-      op->set_read();
-      // fall through
-    case CEPH_OSD_OP_NOTIFY:
-    case CEPH_OSD_OP_NOTIFY_ACK:
-      {
-        op->set_promote();
-        break;
-      }
-
-    case CEPH_OSD_OP_DELETE:
-      // if we get a delete with FAILOK we can skip handle cache. without
-      // FAILOK we still need to promote (or do something smarter) to
-      // determine whether to return ENOENT or 0.
-      if (iter == m->ops.begin() &&
-         iter->op.flags == CEPH_OSD_OP_FLAG_FAILOK) {
-       op->set_skip_handle_cache();
-      }
-      // skip promotion when proxying a delete op
-      if (m->ops.size() == 1) {
-       op->set_skip_promote();
-      }
-      break;
-
-    case CEPH_OSD_OP_CACHE_TRY_FLUSH:
-    case CEPH_OSD_OP_CACHE_FLUSH:
-    case CEPH_OSD_OP_CACHE_EVICT:
-      // If try_flush/flush/evict is the only op, can skip handle cache.
-      if (m->ops.size() == 1) {
-       op->set_skip_handle_cache();
-      }
-      break;
-
-    case CEPH_OSD_OP_READ:
-    case CEPH_OSD_OP_SYNC_READ:
-    case CEPH_OSD_OP_SPARSE_READ:
-    case CEPH_OSD_OP_CHECKSUM:
-    case CEPH_OSD_OP_WRITEFULL:
-      if (m->ops.size() == 1 &&
-          (iter->op.flags & CEPH_OSD_OP_FLAG_FADVISE_NOCACHE ||
-           iter->op.flags & CEPH_OSD_OP_FLAG_FADVISE_DONTNEED)) {
-        op->set_skip_promote();
-      }
-      break;
-
-    // force promotion when pin an object in cache tier
-    case CEPH_OSD_OP_CACHE_PIN:
-      op->set_promote();
-      break;
-
-    default:
-      break;
-    }
-  }
-
-  if (op->rmw_flags == 0)
-    return -EINVAL;
-
-  return 0;
-}
-
-void OSD::set_perf_queries(
-    const std::map<OSDPerfMetricQuery, OSDPerfMetricLimits> &queries) {
+void OSD::set_perf_queries(const ConfigPayload &config_payload) {
+  const OSDConfigPayload &osd_config_payload = boost::get<OSDConfigPayload>(config_payload);
+  const std::map<OSDPerfMetricQuery, OSDPerfMetricLimits> &queries = osd_config_payload.config;
   dout(10) << "setting " << queries.size() << " queries" << dendl;
 
   std::list<OSDPerfMetricQuery> supported_queries;
@@ -10213,44 +10280,41 @@ void OSD::set_perf_queries(
     dout(1) << queries.size() - supported_queries.size()
             << " unsupported queries" << dendl;
   }
-
   {
-    Mutex::Locker locker(m_perf_queries_lock);
+    std::lock_guard locker{m_perf_queries_lock};
     m_perf_queries = supported_queries;
     m_perf_limits = queries;
   }
-
   std::vector<PGRef> pgs;
   _get_pgs(&pgs);
   for (auto& pg : pgs) {
-    if (pg->is_primary()) {
-      pg->lock();
-      pg->set_dynamic_perf_stats_queries(supported_queries);
-      pg->unlock();
-    }
+    std::scoped_lock l{*pg};
+    pg->set_dynamic_perf_stats_queries(supported_queries);
   }
 }
 
-void OSD::get_perf_reports(
-    std::map<OSDPerfMetricQuery, OSDPerfMetricReport> *reports) {
+MetricPayload OSD::get_perf_reports() {
+  OSDMetricPayload payload;
+  std::map<OSDPerfMetricQuery, OSDPerfMetricReport> &reports = payload.report;
+
   std::vector<PGRef> pgs;
   _get_pgs(&pgs);
   DynamicPerfStats dps;
   for (auto& pg : pgs) {
-    if (pg->is_primary()) {
-      // m_perf_queries can be modified only in set_perf_queries by mgr client
-      // request, and it is protected by by mgr client's lock, which is held
-      // when set_perf_queries/get_perf_reports are called, so we may not hold
-      // m_perf_queries_lock here.
-      DynamicPerfStats pg_dps(m_perf_queries);
-      pg->lock();
-      pg->get_dynamic_perf_stats(&pg_dps);
-      pg->unlock();
-      dps.merge(pg_dps);
-    }
+    // m_perf_queries can be modified only in set_perf_queries by mgr client
+    // request, and it is protected by by mgr client's lock, which is held
+    // when set_perf_queries/get_perf_reports are called, so we may not hold
+    // m_perf_queries_lock here.
+    DynamicPerfStats pg_dps(m_perf_queries);
+    pg->lock();
+    pg->get_dynamic_perf_stats(&pg_dps);
+    pg->unlock();
+    dps.merge(pg_dps);
   }
-  dps.add_to_reports(m_perf_limits, reports);
-  dout(20) << "reports for " << reports->size() << " queries" << dendl;
+  dps.add_to_reports(m_perf_limits, &reports);
+  dout(20) << "reports for " << reports.size() << " queries" << dendl;
+
+  return payload;
 }
 
 // =============================================================
@@ -10344,7 +10408,7 @@ epoch_t OSDShard::get_max_waiting_epoch()
 }
 
 void OSDShard::consume_map(
-  OSDMapRef& new_osdmap,
+  const OSDMapRef& new_osdmap,
   unsigned *pushes_to_free)
 {
   std::lock_guard l(shard_lock);
@@ -10357,7 +10421,7 @@ void OSDShard::consume_map(
   dout(10) << new_osdmap->get_epoch()
            << " (was " << (old_osdmap ? old_osdmap->get_epoch() : 0) << ")"
           << dendl;
-  bool queued = false;
+  int queued = 0;
 
   // check slots
   auto p = pg_slots.begin();
@@ -10384,8 +10448,7 @@ void OSDShard::consume_map(
        dout(20) << __func__ << "  " << pgid
                 << " pending_peering first epoch " << first
                 << " <= " << new_osdmap->get_epoch() << ", requeueing" << dendl;
-       _wake_pg_slot(pgid, slot);
-       queued = true;
+       queued += _wake_pg_slot(pgid, slot);
       }
       ++p;
       continue;
@@ -10425,14 +10488,18 @@ void OSDShard::consume_map(
   }
   if (queued) {
     std::lock_guard l{sdata_wait_lock};
-    sdata_cond.notify_one();
+    if (queued == 1)
+      sdata_cond.notify_one();
+    else
+      sdata_cond.notify_all();
   }
 }
 
-void OSDShard::_wake_pg_slot(
+int OSDShard::_wake_pg_slot(
   spg_t pgid,
   OSDShardPGSlot *slot)
 {
+  int count = 0;
   dout(20) << __func__ << " " << pgid
           << " to_process " << slot->to_process
           << " waiting " << slot->waiting
@@ -10440,13 +10507,15 @@ void OSDShard::_wake_pg_slot(
   for (auto i = slot->to_process.rbegin();
        i != slot->to_process.rend();
        ++i) {
-    _enqueue_front(std::move(*i), osd->op_prio_cutoff);
+    scheduler->enqueue_front(std::move(*i));
+    count++;
   }
   slot->to_process.clear();
   for (auto i = slot->waiting.rbegin();
        i != slot->waiting.rend();
        ++i) {
-    _enqueue_front(std::move(*i), osd->op_prio_cutoff);
+    scheduler->enqueue_front(std::move(*i));
+    count++;
   }
   slot->waiting.clear();
   for (auto i = slot->waiting_peering.rbegin();
@@ -10456,11 +10525,13 @@ void OSDShard::_wake_pg_slot(
     // items are waiting for maps we don't have yet.  FIXME, maybe,
     // someday, if we decide this inefficiency matters
     for (auto j = i->second.rbegin(); j != i->second.rend(); ++j) {
-      _enqueue_front(std::move(*j), osd->op_prio_cutoff);
+      scheduler->enqueue_front(std::move(*j));
+      count++;
     }
   }
   slot->waiting_peering.clear();
   ++slot->requeue_seq;
+  return count;
 }
 
 void OSDShard::identify_splits_and_merges(
@@ -10483,8 +10554,7 @@ void OSDShard::identify_splits_and_merges(
          split_pgs, nullptr);
       } else {
        dout(20) << __func__ << " slot " << pgid
-                << " has no pg and waiting_for_split "
-                << slot->waiting_for_split << dendl;
+                << " has no pg and waiting_for_split " << dendl;
       }
     }
   }
@@ -10594,15 +10664,16 @@ void OSDShard::prime_merges(const OSDMapRef& as_of_osdmap,
 
 void OSDShard::register_and_wake_split_child(PG *pg)
 {
+  dout(15) <<  __func__ << ": " << pg << " #:" << pg_slots.size() << dendl;
   epoch_t epoch;
   {
     std::lock_guard l(shard_lock);
-    dout(10) << pg->pg_id << " " << pg << dendl;
+    dout(10) << __func__ << ": " << pg->pg_id << " " << pg << dendl;
     auto p = pg_slots.find(pg->pg_id);
     ceph_assert(p != pg_slots.end());
     auto *slot = p->second.get();
-    dout(20) << pg->pg_id << " waiting_for_split " << slot->waiting_for_split
-            << dendl;
+    dout(20) << __func__ << ": " << pg->pg_id << " waiting_for_split "
+            << slot->waiting_for_split << dendl;
     ceph_assert(!slot->pg);
     ceph_assert(!slot->waiting_for_split.empty());
     _attach_pg(slot, pg);
@@ -10649,6 +10720,40 @@ void OSDShard::unprime_split_children(spg_t parent, unsigned old_pg_num)
   }
 }
 
+void OSDShard::update_scheduler_config()
+{
+  std::lock_guard l(shard_lock);
+  scheduler->update_configuration();
+}
+
+std::string OSDShard::get_scheduler_type()
+{
+  std::ostringstream scheduler_type;
+  scheduler_type << *scheduler;
+  return scheduler_type.str();
+}
+
+OSDShard::OSDShard(
+  int id,
+  CephContext *cct,
+  OSD *osd)
+  : shard_id(id),
+    cct(cct),
+    osd(osd),
+    shard_name(string("OSDShard.") + stringify(id)),
+    sdata_wait_lock_name(shard_name + "::sdata_wait_lock"),
+    sdata_wait_lock{make_mutex(sdata_wait_lock_name)},
+    osdmap_lock{make_mutex(shard_name + "::osdmap_lock")},
+    shard_lock_name(shard_name + "::shard_lock"),
+    shard_lock{make_mutex(shard_lock_name)},
+    scheduler(ceph::osd::scheduler::make_scheduler(
+      cct, osd->num_shards, osd->store->is_rotational(),
+      osd->store->get_type())),
+    context_queue(sdata_wait_lock, sdata_cond)
+{
+  dout(0) << "using op scheduler " << *scheduler << dendl;
+}
+
 
 // =============================================================
 
@@ -10660,7 +10765,7 @@ void OSDShard::unprime_split_children(spg_t parent, unsigned old_pg_num)
 void OSD::ShardedOpWQ::_add_slot_waiter(
   spg_t pgid,
   OSDShardPGSlot *slot,
-  OpQueueItem&& qi)
+  OpSchedulerItem&& qi)
 {
   if (qi.is_peering()) {
     dout(20) << __func__ << " " << pgid
@@ -10694,7 +10799,7 @@ void OSD::ShardedOpWQ::_process(uint32_t thread_index, heartbeat_handle_d *hb)
 
   // peek at spg_t
   sdata->shard_lock.lock();
-  if (sdata->pqueue->empty() &&
+  if (sdata->scheduler->empty() &&
       (!is_smallest_thread_index || sdata->context_queue.empty())) {
     std::unique_lock wait_lock{sdata->sdata_wait_lock};
     if (is_smallest_thread_index && !sdata->context_queue.empty()) {
@@ -10707,13 +10812,14 @@ void OSD::ShardedOpWQ::_process(uint32_t thread_index, heartbeat_handle_d *hb)
       sdata->sdata_cond.wait(wait_lock);
       wait_lock.unlock();
       sdata->shard_lock.lock();
-      if (sdata->pqueue->empty() &&
+      if (sdata->scheduler->empty() &&
          !(is_smallest_thread_index && !sdata->context_queue.empty())) {
        sdata->shard_lock.unlock();
        return;
       }
+      // found a work item; reapply default wq timeouts
       osd->cct->get_heartbeat_map()->reset_timeout(hb,
-         osd->cct->_conf->threadpool_default_timeout, 0);
+        timeout_interval, suicide_interval);
     } else {
       dout(20) << __func__ << " need return immediately" << dendl;
       wait_lock.unlock();
@@ -10723,25 +10829,63 @@ void OSD::ShardedOpWQ::_process(uint32_t thread_index, heartbeat_handle_d *hb)
   }
 
   list<Context *> oncommits;
-  if (is_smallest_thread_index && !sdata->context_queue.empty()) {
-    sdata->context_queue.swap(oncommits);
-  }
+  if (is_smallest_thread_index) {
+    sdata->context_queue.move_to(oncommits);
+  }
+
+  WorkItem work_item;
+  while (!std::get_if<OpSchedulerItem>(&work_item)) {
+    if (sdata->scheduler->empty()) {
+      if (osd->is_stopping()) {
+        sdata->shard_lock.unlock();
+        for (auto c : oncommits) {
+          dout(10) << __func__ << " discarding in-flight oncommit " << c << dendl;
+          delete c;
+        }
+        return;    // OSD shutdown, discard.
+      }
+      sdata->shard_lock.unlock();
+      handle_oncommits(oncommits);
+      return;
+    }
 
-  if (sdata->pqueue->empty()) {
+    work_item = sdata->scheduler->dequeue();
     if (osd->is_stopping()) {
       sdata->shard_lock.unlock();
       for (auto c : oncommits) {
-       dout(10) << __func__ << " discarding in-flight oncommit " << c << dendl;
-       delete c;
+        dout(10) << __func__ << " discarding in-flight oncommit " << c << dendl;
+        delete c;
       }
       return;    // OSD shutdown, discard.
     }
-    sdata->shard_lock.unlock();
-    handle_oncommits(oncommits);
-    return;
-  }
 
-  OpQueueItem item = sdata->pqueue->dequeue();
+    // If the work item is scheduled in the future, wait until
+    // the time returned in the dequeue response before retrying.
+    if (auto when_ready = std::get_if<double>(&work_item)) {
+      if (is_smallest_thread_index) {
+        sdata->shard_lock.unlock();
+        handle_oncommits(oncommits);
+        return;
+      }
+      std::unique_lock wait_lock{sdata->sdata_wait_lock};
+      auto future_time = ceph::real_clock::from_double(*when_ready);
+      dout(10) << __func__ << " dequeue future request at " << future_time << dendl;
+      // Disable heartbeat timeout until we find a non-future work item to process.
+      osd->cct->get_heartbeat_map()->clear_timeout(hb);
+      sdata->shard_lock.unlock();
+      ++sdata->waiting_threads;
+      sdata->sdata_cond.wait_until(wait_lock, future_time);
+      --sdata->waiting_threads;
+      wait_lock.unlock();
+      sdata->shard_lock.lock();
+      // Reapply default wq timeouts
+      osd->cct->get_heartbeat_map()->reset_timeout(hb,
+        timeout_interval, suicide_interval);
+    }
+  } // while
+
+  // Access the stored item
+  auto item = std::move(std::get<OpSchedulerItem>(work_item));
   if (osd->is_stopping()) {
     sdata->shard_lock.unlock();
     for (auto c : oncommits) {
@@ -10900,11 +11044,10 @@ void OSD::ShardedOpWQ::_process(uint32_t thread_index, heartbeat_handle_d *hb)
               << " no pg, shouldn't exist e" << osdmap->get_epoch()
               << ", dropping " << qi << dendl;
       // share map with client?
-      if (boost::optional<OpRequestRef> _op = qi.maybe_get_op()) {
-       auto priv = (*_op)->get_req()->get_connection()->get_priv();
-       if (auto session = static_cast<Session *>(priv.get()); session) {
-         osd->maybe_share_map(session, *_op, sdata->shard_osdmap);
-       }
+      if (std::optional<OpRequestRef> _op = qi.maybe_get_op()) {
+       osd->service.maybe_share_map((*_op)->get_req()->get_connection().get(),
+                                    sdata->shard_osdmap,
+                                    (*_op)->sent_epoch);
       }
       unsigned pushes_to_free = qi.get_reserved_pushes();
       if (pushes_to_free > 0) {
@@ -10942,7 +11085,7 @@ void OSD::ShardedOpWQ::_process(uint32_t thread_index, heartbeat_handle_d *hb)
   {
 #ifdef WITH_LTTNG
     osd_reqid_t reqid;
-    if (boost::optional<OpRequestRef> _op = qi.maybe_get_op()) {
+    if (std::optional<OpRequestRef> _op = qi.maybe_get_op()) {
       reqid = (*_op)->get_reqid();
     }
 #endif
@@ -10964,7 +11107,7 @@ void OSD::ShardedOpWQ::_process(uint32_t thread_index, heartbeat_handle_d *hb)
   {
 #ifdef WITH_LTTNG
     osd_reqid_t reqid;
-    if (boost::optional<OpRequestRef> _op = qi.maybe_get_op()) {
+    if (std::optional<OpRequestRef> _op = qi.maybe_get_op()) {
       reqid = (*_op)->get_reqid();
     }
 #endif
@@ -10975,31 +11118,47 @@ void OSD::ShardedOpWQ::_process(uint32_t thread_index, heartbeat_handle_d *hb)
   handle_oncommits(oncommits);
 }
 
-void OSD::ShardedOpWQ::_enqueue(OpQueueItem&& item) {
+void OSD::ShardedOpWQ::_enqueue(OpSchedulerItem&& item) {
+  if (unlikely(m_fast_shutdown) ) {
+    // stop enqueing when we are in the middle of a fast shutdown
+    return;
+  }
+
   uint32_t shard_index =
     item.get_ordering_token().hash_to_shard(osd->shards.size());
 
   OSDShard* sdata = osd->shards[shard_index];
   assert (NULL != sdata);
-  unsigned priority = item.get_priority();
-  unsigned cost = item.get_cost();
-  sdata->shard_lock.lock();
+  if (sdata->get_scheduler_type() == "mClockScheduler") {
+    item.maybe_set_is_qos_item();
+  }
 
   dout(20) << __func__ << " " << item << dendl;
-  if (priority >= osd->op_prio_cutoff)
-    sdata->pqueue->enqueue_strict(
-      item.get_owner(), priority, std::move(item));
-  else
-    sdata->pqueue->enqueue(
-      item.get_owner(), priority, cost, std::move(item));
-  sdata->shard_lock.unlock();
 
-  std::lock_guard l{sdata->sdata_wait_lock};
-  sdata->sdata_cond.notify_one();
+  bool empty = true;
+  {
+    std::lock_guard l{sdata->shard_lock};
+    empty = sdata->scheduler->empty();
+    sdata->scheduler->enqueue(std::move(item));
+  }
+
+  {
+    std::lock_guard l{sdata->sdata_wait_lock};
+    if (empty) {
+      sdata->sdata_cond.notify_all();
+    } else if (sdata->waiting_threads) {
+      sdata->sdata_cond.notify_one();
+    }
+  }
 }
 
-void OSD::ShardedOpWQ::_enqueue_front(OpQueueItem&& item)
+void OSD::ShardedOpWQ::_enqueue_front(OpSchedulerItem&& item)
 {
+  if (unlikely(m_fast_shutdown) ) {
+    // stop enqueing when we are in the middle of a fast shutdown
+    return;
+  }
+
   auto shard_index = item.get_ordering_token().hash_to_shard(osd->shards.size());
   auto& sdata = osd->shards[shard_index];
   ceph_assert(sdata);
@@ -11008,7 +11167,7 @@ void OSD::ShardedOpWQ::_enqueue_front(OpQueueItem&& item)
   if (p != sdata->pg_slots.end() &&
       !p->second->to_process.empty()) {
     // we may be racing with _process, which has dequeued a new item
-    // from pqueue, put it on to_process, and is now busy taking the
+    // from scheduler, put it on to_process, and is now busy taking the
     // pg lock.  ensure this old requeued item is ordered before any
     // such newer item in to_process.
     p->second->to_process.push_front(std::move(item));
@@ -11020,14 +11179,31 @@ void OSD::ShardedOpWQ::_enqueue_front(OpQueueItem&& item)
   } else {
     dout(20) << __func__ << " " << item << dendl;
   }
-  sdata->_enqueue_front(std::move(item), osd->op_prio_cutoff);
+  sdata->scheduler->enqueue_front(std::move(item));
   sdata->shard_lock.unlock();
   std::lock_guard l{sdata->sdata_wait_lock};
   sdata->sdata_cond.notify_one();
 }
 
-namespace ceph { 
-namespace osd_cmds { 
+void OSD::ShardedOpWQ::stop_for_fast_shutdown()
+{
+  uint32_t shard_index = 0;
+  m_fast_shutdown = true;
+
+  for (; shard_index < osd->num_shards; shard_index++) {
+    auto& sdata = osd->shards[shard_index];
+    ceph_assert(sdata);
+    sdata->shard_lock.lock();
+    int work_count = 0;
+    while(! sdata->scheduler->empty() ) {
+      auto work_item = sdata->scheduler->dequeue();
+      work_count++;
+    }
+    sdata->shard_lock.unlock();
+  }
+}
+
+namespace ceph::osd_cmds {
 
 int heap(CephContext& cct, const cmdmap_t& cmdmap, Formatter& f,
         std::ostream& os)
@@ -11036,43 +11212,24 @@ int heap(CephContext& cct, const cmdmap_t& cmdmap, Formatter& f,
         os << "could not issue heap profiler command -- not using tcmalloc!";
         return -EOPNOTSUPP;
   }
-  
+
   string cmd;
-  if (!cmd_getval(&cct, cmdmap, "heapcmd", cmd)) {
+  if (!cmd_getval(cmdmap, "heapcmd", cmd)) {
         os << "unable to get value for command \"" << cmd << "\"";
        return -EINVAL;
   }
-  
+
   std::vector<std::string> cmd_vec;
   get_str_vec(cmd, cmd_vec);
 
   string val;
-  if (cmd_getval(&cct, cmdmap, "value", val)) {
+  if (cmd_getval(cmdmap, "value", val)) {
     cmd_vec.push_back(val);
   }
-  
+
   ceph_heap_profiler_handle_command(cmd_vec, os);
-  
+
   return 0;
 }
-}} // namespace ceph::osd_cmds
 
-
-std::ostream& operator<<(std::ostream& out, const io_queue& q) {
-  switch(q) {
-  case io_queue::prioritized:
-    out << "prioritized";
-    break;
-  case io_queue::weightedpriority:
-    out << "weightedpriority";
-    break;
-  case io_queue::mclock_opclass:
-    out << "mclock_opclass";
-    break;
-  case io_queue::mclock_client:
-    out << "mclock_client";
-    break;
-  }
-  return out;
-}
+} // namespace ceph::osd_cmds