]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/osd/OSD.h
import quincy beta 17.1.0
[ceph.git] / ceph / src / osd / OSD.h
index 517a280b83046300933dfb81addbbc86da89d74f..30d0b0b4aef07a78d284fec2a782fc074bdca1c0 100644 (file)
@@ -1,4 +1,4 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
 // vim: ts=8 sw=2 smarttab
 /*
  * Ceph - scalable distributed file system
@@ -7,9 +7,9 @@
  *
  * This is free software; you can redistribute it and/or
  * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software 
+ * License version 2.1, as published by the Free Software
  * Foundation.  See file COPYING.
- * 
+ *
  */
 
 #ifndef CEPH_OSD_H
 
 #include "msg/Dispatcher.h"
 
-#include "common/Mutex.h"
-#include "common/RWLock.h"
+#include "common/async/context_pool.h"
 #include "common/Timer.h"
 #include "common/WorkQueue.h"
 #include "common/AsyncReserver.h"
 #include "common/ceph_context.h"
 #include "common/config_cacher.h"
 #include "common/zipkin_trace.h"
+#include "common/ceph_timer.h"
 
 #include "mgr/MgrClient.h"
 
 #include "os/ObjectStore.h"
-#include "OSDCap.h" 
-#include "auth/KeyRing.h"
-
-#include "osd/ClassHandler.h"
 
 #include "include/CompatSet.h"
+#include "include/common_fwd.h"
 
 #include "OpRequest.h"
 #include "Session.h"
 
-#include "osd/OpQueueItem.h"
+#include "osd/scheduler/OpScheduler.h"
 
 #include <atomic>
 #include <map>
 
 #include "common/shared_cache.hpp"
 #include "common/simple_cache.hpp"
-#include "common/sharedptr_registry.hpp"
-#include "common/WeightedPriorityQueue.h"
-#include "common/PrioritizedQueue.h"
-#include "osd/mClockOpClassQueue.h"
-#include "osd/mClockClientQueue.h"
 #include "messages/MOSDOp.h"
 #include "common/EventTrace.h"
+#include "osd/osd_perf_counters.h"
+#include "common/Finisher.h"
+#include "scrubber/osd_scrub_sched.h"
 
 #define CEPH_OSD_PROTOCOL    10 /* cluster internal */
 
 
   */
 
-enum {
-  l_osd_first = 10000,
-  l_osd_op_wip,
-  l_osd_op,
-  l_osd_op_inb,
-  l_osd_op_outb,
-  l_osd_op_lat,
-  l_osd_op_process_lat,
-  l_osd_op_prepare_lat,
-  l_osd_op_r,
-  l_osd_op_r_outb,
-  l_osd_op_r_lat,
-  l_osd_op_r_lat_outb_hist,
-  l_osd_op_r_process_lat,
-  l_osd_op_r_prepare_lat,
-  l_osd_op_w,
-  l_osd_op_w_inb,
-  l_osd_op_w_lat,
-  l_osd_op_w_lat_inb_hist,
-  l_osd_op_w_process_lat,
-  l_osd_op_w_prepare_lat,
-  l_osd_op_rw,
-  l_osd_op_rw_inb,
-  l_osd_op_rw_outb,
-  l_osd_op_rw_lat,
-  l_osd_op_rw_lat_inb_hist,
-  l_osd_op_rw_lat_outb_hist,
-  l_osd_op_rw_process_lat,
-  l_osd_op_rw_prepare_lat,
-
-  l_osd_op_before_queue_op_lat,
-  l_osd_op_before_dequeue_op_lat,
-
-  l_osd_sop,
-  l_osd_sop_inb,
-  l_osd_sop_lat,
-  l_osd_sop_w,
-  l_osd_sop_w_inb,
-  l_osd_sop_w_lat,
-  l_osd_sop_pull,
-  l_osd_sop_pull_lat,
-  l_osd_sop_push,
-  l_osd_sop_push_inb,
-  l_osd_sop_push_lat,
-
-  l_osd_pull,
-  l_osd_push,
-  l_osd_push_outb,
-
-  l_osd_rop,
-  l_osd_rbytes,
-
-  l_osd_loadavg,
-  l_osd_cached_crc,
-  l_osd_cached_crc_adjusted,
-  l_osd_missed_crc,
-
-  l_osd_pg,
-  l_osd_pg_primary,
-  l_osd_pg_replica,
-  l_osd_pg_stray,
-  l_osd_pg_removing,
-  l_osd_hb_to,
-  l_osd_map,
-  l_osd_mape,
-  l_osd_mape_dup,
-
-  l_osd_waiting_for_map,
-
-  l_osd_map_cache_hit,
-  l_osd_map_cache_miss,
-  l_osd_map_cache_miss_low,
-  l_osd_map_cache_miss_low_avg,
-  l_osd_map_bl_cache_hit,
-  l_osd_map_bl_cache_miss,
-
-  l_osd_stat_bytes,
-  l_osd_stat_bytes_used,
-  l_osd_stat_bytes_avail,
-
-  l_osd_copyfrom,
-
-  l_osd_tier_promote,
-  l_osd_tier_flush,
-  l_osd_tier_flush_fail,
-  l_osd_tier_try_flush,
-  l_osd_tier_try_flush_fail,
-  l_osd_tier_evict,
-  l_osd_tier_whiteout,
-  l_osd_tier_dirty,
-  l_osd_tier_clean,
-  l_osd_tier_delay,
-  l_osd_tier_proxy_read,
-  l_osd_tier_proxy_write,
-
-  l_osd_agent_wake,
-  l_osd_agent_skip,
-  l_osd_agent_flush,
-  l_osd_agent_evict,
-
-  l_osd_object_ctx_cache_hit,
-  l_osd_object_ctx_cache_total,
-
-  l_osd_op_cache_hit,
-  l_osd_tier_flush_lat,
-  l_osd_tier_promote_lat,
-  l_osd_tier_r_lat,
-
-  l_osd_pg_info,
-  l_osd_pg_fastinfo,
-  l_osd_pg_biginfo,
-
-  l_osd_last,
-};
-
-// RecoveryState perf counters
-enum {
-  rs_first = 20000,
-  rs_initial_latency,
-  rs_started_latency,
-  rs_reset_latency,
-  rs_start_latency,
-  rs_primary_latency,
-  rs_peering_latency,
-  rs_backfilling_latency,
-  rs_waitremotebackfillreserved_latency,
-  rs_waitlocalbackfillreserved_latency,
-  rs_notbackfilling_latency,
-  rs_repnotrecovering_latency,
-  rs_repwaitrecoveryreserved_latency,
-  rs_repwaitbackfillreserved_latency,
-  rs_reprecovering_latency,
-  rs_activating_latency,
-  rs_waitlocalrecoveryreserved_latency,
-  rs_waitremoterecoveryreserved_latency,
-  rs_recovering_latency,
-  rs_recovered_latency,
-  rs_clean_latency,
-  rs_active_latency,
-  rs_replicaactive_latency,
-  rs_stray_latency,
-  rs_getinfo_latency,
-  rs_getlog_latency,
-  rs_waitactingchange_latency,
-  rs_incomplete_latency,
-  rs_down_latency,
-  rs_getmissing_latency,
-  rs_waitupthru_latency,
-  rs_notrecovering_latency,
-  rs_last,
-};
-
 class Messenger;
 class Message;
 class MonClient;
-class PerfCounters;
 class ObjectStore;
 class FuseStore;
 class OSDMap;
@@ -243,25 +84,24 @@ class TestOpsSocketHook;
 struct C_FinishSplits;
 struct C_OpenPGs;
 class LogChannel;
-class CephContext;
-class MOSDOp;
 
 class MOSDPGCreate2;
-class MOSDPGQuery;
 class MOSDPGNotify;
 class MOSDPGInfo;
 class MOSDPGRemove;
 class MOSDForceRecovery;
+class MMonGetPurgedSnapsReply;
 
 class OSD;
 
 class OSDService {
+  using OpSchedulerItem = ceph::osd::scheduler::OpSchedulerItem;
 public:
   OSD *osd;
   CephContext *cct;
   ObjectStore::CollectionHandle meta_ch;
   const int whoami;
-  ObjectStore *&store;
+  ObjectStore * const store;
   LogClient &log_client;
   LogChannelRef clog;
   PGRecoveryStats &pg_recovery_stats;
@@ -272,13 +112,12 @@ public:
   PerfCounters *&logger;
   PerfCounters *&recoverystate_perf;
   MonClient   *&monc;
-  ClassHandler  *&class_handler;
 
   md_config_cacher_t<Option::size_t> osd_max_object_size;
   md_config_cacher_t<bool> osd_skip_data_digest;
 
-  void enqueue_back(OpQueueItem&& qi);
-  void enqueue_front(OpQueueItem&& qi);
+  void enqueue_back(OpSchedulerItem&& qi);
+  void enqueue_front(OpSchedulerItem&& qi);
 
   void maybe_inject_dispatch_delay() {
     if (g_conf()->osd_debug_inject_dispatch_delay_probability > 0) {
@@ -291,6 +130,8 @@ public:
     }
   }
 
+  ceph::signedspan get_mnow();
+
 private:
   // -- superblock --
   ceph::mutex publish_lock, pre_publish_lock; // pre-publish orders before publish
@@ -327,11 +168,11 @@ public:
   }
 
   /*
-   * osdmap - current published map
-   * next_osdmap - pre_published map that is about to be published.
+   * osdmap - current published std::map
+   * next_osdmap - pre_published std::map that is about to be published.
    *
    * We use the next_osdmap to send messages and initiate connections,
-   * but only if the target is the same instance as the one in the map
+   * but only if the target is the same instance as the one in the std::map
    * epoch the current user is working from (i.e., the result is
    * equivalent to what is in next_osdmap).
    *
@@ -343,6 +184,7 @@ public:
 private:
   OSDMapRef next_osdmap;
   ceph::condition_variable pre_publish_cond;
+  int pre_publish_waiter = 0;
 
 public:
   void pre_publish_map(OSDMapRef map) {
@@ -352,184 +194,115 @@ public:
 
   void activate_map();
   /// map epochs reserved below
-  map<epoch_t, unsigned> map_reservations;
+  std::map<epoch_t, unsigned> map_reservations;
 
   /// gets ref to next_osdmap and registers the epoch as reserved
   OSDMapRef get_nextmap_reserved() {
     std::lock_guard l(pre_publish_lock);
-    if (!next_osdmap)
-      return OSDMapRef();
     epoch_t e = next_osdmap->get_epoch();
-    map<epoch_t, unsigned>::iterator i =
-      map_reservations.insert(make_pair(e, 0)).first;
+    std::map<epoch_t, unsigned>::iterator i =
+      map_reservations.insert(std::make_pair(e, 0)).first;
     i->second++;
     return next_osdmap;
   }
   /// releases reservation on map
   void release_map(OSDMapRef osdmap) {
     std::lock_guard l(pre_publish_lock);
-    map<epoch_t, unsigned>::iterator i =
+    std::map<epoch_t, unsigned>::iterator i =
       map_reservations.find(osdmap->get_epoch());
     ceph_assert(i != map_reservations.end());
     ceph_assert(i->second > 0);
     if (--(i->second) == 0) {
       map_reservations.erase(i);
     }
-    pre_publish_cond.notify_all();
+    if (pre_publish_waiter) {
+      pre_publish_cond.notify_all();
+    }
   }
   /// blocks until there are no reserved maps prior to next_osdmap
   void await_reserved_maps() {
     std::unique_lock l{pre_publish_lock};
     ceph_assert(next_osdmap);
+    pre_publish_waiter++;
     pre_publish_cond.wait(l, [this] {
       auto i = map_reservations.cbegin();
       return (i == map_reservations.cend() ||
              i->first >= next_osdmap->get_epoch());
     });
+    pre_publish_waiter--;
   }
   OSDMapRef get_next_osdmap() {
     std::lock_guard l(pre_publish_lock);
-    if (!next_osdmap)
-      return OSDMapRef();
     return next_osdmap;
   }
 
-private:
-  Mutex peer_map_epoch_lock;
-  map<int, epoch_t> peer_map_epoch;
-public:
-  epoch_t get_peer_epoch(int p);
-  epoch_t note_peer_epoch(int p, epoch_t e);
-  void forget_peer_epoch(int p, epoch_t e);
+  void maybe_share_map(Connection *con,
+                      const OSDMapRef& osdmap,
+                      epoch_t peer_epoch_lb=0);
 
   void send_map(class MOSDMap *m, Connection *con);
-  void send_incremental_map(epoch_t since, Connection *con, OSDMapRef& osdmap);
+  void send_incremental_map(epoch_t since, Connection *con,
+                           const OSDMapRef& osdmap);
   MOSDMap *build_incremental_map_msg(epoch_t from, epoch_t to,
                                        OSDSuperblock& superblock);
-  bool should_share_map(entity_name_t name, Connection *con, epoch_t epoch,
-                        const OSDMapRef& osdmap, const epoch_t *sent_epoch_p);
-  void share_map(entity_name_t name, Connection *con, epoch_t epoch,
-                 OSDMapRef& osdmap, epoch_t *sent_epoch_p);
-  void share_map_peer(int peer, Connection *con,
-                      OSDMapRef map = OSDMapRef());
 
   ConnectionRef get_con_osd_cluster(int peer, epoch_t from_epoch);
-  pair<ConnectionRef,ConnectionRef> get_con_osd_hb(int peer, epoch_t from_epoch);  // (back, front)
+  std::pair<ConnectionRef,ConnectionRef> get_con_osd_hb(int peer, epoch_t from_epoch);  // (back, front)
   void send_message_osd_cluster(int peer, Message *m, epoch_t from_epoch);
-  void send_message_osd_cluster(Message *m, Connection *con) {
-    con->send_message(m);
+  void send_message_osd_cluster(std::vector<std::pair<int, Message*>>& messages, epoch_t from_epoch);
+  void send_message_osd_cluster(MessageRef m, Connection *con) {
+    con->send_message2(std::move(m));
   }
   void send_message_osd_cluster(Message *m, const ConnectionRef& con) {
     con->send_message(m);
   }
-  void send_message_osd_client(Message *m, Connection *con) {
-    con->send_message(m);
-  }
   void send_message_osd_client(Message *m, const ConnectionRef& con) {
     con->send_message(m);
   }
   entity_name_t get_cluster_msgr_name() const;
 
-private:
-  // -- scrub scheduling --
-  Mutex sched_scrub_lock;
-  int scrubs_pending;
-  int scrubs_active;
 
 public:
-  struct ScrubJob {
-    CephContext* cct;
-    /// pg to be scrubbed
-    spg_t pgid;
-    /// a time scheduled for scrub. but the scrub could be delayed if system
-    /// load is too high or it fails to fall in the scrub hours
-    utime_t sched_time;
-    /// the hard upper bound of scrub time
-    utime_t deadline;
-    ScrubJob() : cct(nullptr) {}
-    explicit ScrubJob(CephContext* cct, const spg_t& pg,
-                     const utime_t& timestamp,
-                     double pool_scrub_min_interval = 0,
-                     double pool_scrub_max_interval = 0, bool must = true);
-    /// order the jobs by sched_time
-    bool operator<(const ScrubJob& rhs) const;
-  };
-  set<ScrubJob> sched_scrub_pg;
-
-  /// @returns the scrub_reg_stamp used for unregister the scrub job
-  utime_t reg_pg_scrub(spg_t pgid, utime_t t, double pool_scrub_min_interval,
-                      double pool_scrub_max_interval, bool must) {
-    ScrubJob scrub(cct, pgid, t, pool_scrub_min_interval, pool_scrub_max_interval,
-                  must);
-    std::lock_guard l(sched_scrub_lock);
-    sched_scrub_pg.insert(scrub);
-    return scrub.sched_time;
-  }
-  void unreg_pg_scrub(spg_t pgid, utime_t t) {
-    std::lock_guard l(sched_scrub_lock);
-    size_t removed = sched_scrub_pg.erase(ScrubJob(cct, pgid, t));
-    ceph_assert(removed);
-  }
-  bool first_scrub_stamp(ScrubJob *out) {
-    std::lock_guard l(sched_scrub_lock);
-    if (sched_scrub_pg.empty())
-      return false;
-    set<ScrubJob>::iterator iter = sched_scrub_pg.begin();
-    *out = *iter;
-    return true;
-  }
-  bool next_scrub_stamp(const ScrubJob& next,
-                       ScrubJob *out) {
-    std::lock_guard l(sched_scrub_lock);
-    if (sched_scrub_pg.empty())
-      return false;
-    set<ScrubJob>::const_iterator iter = sched_scrub_pg.lower_bound(next);
-    if (iter == sched_scrub_pg.cend())
-      return false;
-    ++iter;
-    if (iter == sched_scrub_pg.cend())
-      return false;
-    *out = *iter;
-    return true;
-  }
-
-  void dumps_scrub(Formatter *f) {
-    ceph_assert(f != nullptr);
-    std::lock_guard l(sched_scrub_lock);
-
-    f->open_array_section("scrubs");
-    for (const auto &i: sched_scrub_pg) {
-      f->open_object_section("scrub");
-      f->dump_stream("pgid") << i.pgid;
-      f->dump_stream("sched_time") << i.sched_time;
-      f->dump_stream("deadline") << i.deadline;
-      f->dump_bool("forced", i.sched_time == i.deadline);
-      f->close_section();
-    }
-    f->close_section();
-  }
-
-  bool can_inc_scrubs_pending();
-  bool inc_scrubs_pending();
-  void inc_scrubs_active(bool reserved);
-  void dec_scrubs_pending();
-  void dec_scrubs_active();
 
   void reply_op_error(OpRequestRef op, int err);
-  void reply_op_error(OpRequestRef op, int err, eversion_t v, version_t uv);
+  void reply_op_error(OpRequestRef op, int err, eversion_t v, version_t uv,
+                     std::vector<pg_log_op_return_item_t> op_returns);
   void handle_misdirected_op(PG *pg, OpRequestRef op);
 
+ private:
+  /**
+   * The entity that maintains the set of PGs we may scrub (i.e. - those that we
+   * are their primary), and schedules their scrubbing.
+   */
+  ScrubQueue m_scrub_queue;
 
-private:
+ public:
+  ScrubQueue& get_scrub_services() { return m_scrub_queue; }
+
+  /**
+   * A callback used by the ScrubQueue object to initiate a scrub on a specific PG.
+   *
+   * The request might fail for multiple reasons, as ScrubQueue cannot by its own
+   * check some of the PG-specific preconditions and those are checked here. See
+   * attempt_t definition.
+   *
+   * @param pgid to scrub
+   * @param allow_requested_repair_only
+   * @return a Scrub::attempt_t detailing either a success, or the failure reason.
+   */
+  Scrub::schedule_result_t initiate_a_scrub(spg_t pgid, bool allow_requested_repair_only);
+
+
+ private:
   // -- agent shared state --
-  Mutex agent_lock;
-  Cond agent_cond;
-  map<uint64_t, set<PGRef> > agent_queue;
-  set<PGRef>::iterator agent_queue_pos;
+  ceph::mutex agent_lock = ceph::make_mutex("OSDService::agent_lock");
+  ceph::condition_variable agent_cond;
+  std::map<uint64_t, std::set<PGRef> > agent_queue;
+  std::set<PGRef>::iterator agent_queue_pos;
   bool agent_valid_iterator;
   int agent_ops;
   int flush_mode_high_count; //once have one pg with FLUSH_MODE_HIGH then flush objects with high speed
-  set<hobject_t> agent_oids;
+  std::set<hobject_t> agent_oids;
   bool agent_active;
   struct AgentThread : public Thread {
     OSDService *osd;
@@ -540,7 +313,7 @@ private:
     }
   } agent_thread;
   bool agent_stop_flag;
-  Mutex agent_timer_lock;
+  ceph::mutex agent_timer_lock = ceph::make_mutex("OSDService::agent_timer_lock");
   SafeTimer agent_timer;
 
 public:
@@ -551,15 +324,15 @@ public:
     if (!agent_queue.empty() &&
        agent_queue.rbegin()->first < priority)
       agent_valid_iterator = false;  // inserting higher-priority queue
-    set<PGRef>& nq = agent_queue[priority];
+    std::set<PGRef>& nq = agent_queue[priority];
     if (nq.empty())
-      agent_cond.Signal();
+      agent_cond.notify_all();
     nq.insert(pg);
   }
 
   void _dequeue(PG *pg, uint64_t old_priority) {
-    set<PGRef>& oq = agent_queue[old_priority];
-    set<PGRef>::iterator p = oq.find(pg);
+    std::set<PGRef>& oq = agent_queue[old_priority];
+    std::set<PGRef>::iterator p = oq.find(pg);
     ceph_assert(p != oq.end());
     if (p == agent_queue_pos)
       ++agent_queue_pos;
@@ -602,7 +375,7 @@ public:
     std::lock_guard l(agent_lock);
     ceph_assert(agent_ops > 0);
     --agent_ops;
-    agent_cond.Signal();
+    agent_cond.notify_all();
   }
 
   /// note start of an async (flush) op
@@ -620,7 +393,7 @@ public:
     --agent_ops;
     ceph_assert(agent_oids.count(oid) == 1);
     agent_oids.erase(oid);
-    agent_cond.Signal();
+    agent_cond.notify_all();
   }
 
   /// check if we are operating on an object
@@ -670,14 +443,21 @@ public:
     promote_counter.finish(bytes);
   }
   void promote_throttle_recalibrate();
+  unsigned get_num_shards() const {
+    return m_objecter_finishers;
+  }
+  Finisher* get_objecter_finisher(int shard) {
+    return objecter_finishers[shard].get();
+  }
 
   // -- Objecter, for tiering reads/writes from/to other OSDs --
-  Objecter *objecter;
+  ceph::async::io_context_pool& poolctx;
+  std::unique_ptr<Objecter> objecter;
   int m_objecter_finishers;
-  vector<Finisher*> objecter_finishers;
+  std::vector<std::unique_ptr<Finisher>> objecter_finishers;
 
   // -- Watch --
-  Mutex watch_lock;
+  ceph::mutex watch_lock = ceph::make_mutex("OSDService::watch_lock");
   SafeTimer watch_timer;
   uint64_t next_notif_id;
   uint64_t get_next_id(epoch_t cur_epoch) {
@@ -686,15 +466,15 @@ public:
   }
 
   // -- Recovery/Backfill Request Scheduling --
-  Mutex recovery_request_lock;
+  ceph::mutex recovery_request_lock = ceph::make_mutex("OSDService::recovery_request_lock");
   SafeTimer recovery_request_timer;
 
   // For async recovery sleep
   bool recovery_needs_sleep = true;
-  utime_t recovery_schedule_time = utime_t();
+  ceph::real_clock::time_point recovery_schedule_time;
 
   // For recovery & scrub & snap
-  Mutex sleep_lock;
+  ceph::mutex sleep_lock = ceph::make_mutex("OSDService::sleep_lock");
   SafeTimer sleep_timer;
 
   // -- tids --
@@ -706,16 +486,16 @@ public:
 
   // -- backfill_reservation --
   Finisher reserver_finisher;
-  AsyncReserver<spg_t> local_reserver;
-  AsyncReserver<spg_t> remote_reserver;
+  AsyncReserver<spg_t, Finisher> local_reserver;
+  AsyncReserver<spg_t, Finisher> remote_reserver;
 
   // -- pg merge --
-  Mutex merge_lock = {"OSD::merge_lock"};
-  map<pg_t,eversion_t> ready_to_merge_source;   // pg -> version
-  map<pg_t,std::tuple<eversion_t,epoch_t,epoch_t>> ready_to_merge_target;  // pg -> (version,les,lec)
-  set<pg_t> not_ready_to_merge_source;
-  map<pg_t,pg_t> not_ready_to_merge_target;
-  set<pg_t> sent_ready_to_merge_source;
+  ceph::mutex merge_lock = ceph::make_mutex("OSD::merge_lock");
+  std::map<pg_t,eversion_t> ready_to_merge_source;   // pg -> version
+  std::map<pg_t,std::tuple<eversion_t,epoch_t,epoch_t>> ready_to_merge_target;  // pg -> (version,les,lec)
+  std::set<pg_t> not_ready_to_merge_source;
+  std::map<pg_t,pg_t> not_ready_to_merge_target;
+  std::set<pg_t> sent_ready_to_merge_source;
 
   void set_ready_to_merge_source(PG *pg,
                                 eversion_t version);
@@ -729,55 +509,133 @@ public:
   void send_ready_to_merge();
   void _send_ready_to_merge();
   void clear_sent_ready_to_merge();
-  void prune_sent_ready_to_merge(OSDMapRef& osdmap);
+  void prune_sent_ready_to_merge(const OSDMapRef& osdmap);
 
   // -- pg_temp --
 private:
-  Mutex pg_temp_lock;
+  ceph::mutex pg_temp_lock = ceph::make_mutex("OSDService::pg_temp_lock");
   struct pg_temp_t {
-    vector<int> acting;
+    std::vector<int> acting;
     bool forced = false;
   };
-  map<pg_t, pg_temp_t> pg_temp_wanted;
-  map<pg_t, pg_temp_t> pg_temp_pending;
+  std::map<pg_t, pg_temp_t> pg_temp_wanted;
+  std::map<pg_t, pg_temp_t> pg_temp_pending;
   void _sent_pg_temp();
   friend std::ostream& operator<<(std::ostream&, const pg_temp_t&);
 public:
-  void queue_want_pg_temp(pg_t pgid, const vector<int>& want,
+  void queue_want_pg_temp(pg_t pgid, const std::vector<int>& want,
                          bool forced = false);
   void remove_want_pg_temp(pg_t pgid);
   void requeue_pg_temp();
   void send_pg_temp();
 
   ceph::mutex pg_created_lock = ceph::make_mutex("OSDService::pg_created_lock");
-  set<pg_t> pg_created;
+  std::set<pg_t> pg_created;
   void send_pg_created(pg_t pgid);
   void prune_pg_created();
   void send_pg_created();
 
-  AsyncReserver<spg_t> snap_reserver;
+  AsyncReserver<spg_t, Finisher> snap_reserver;
   void queue_recovery_context(PG *pg, GenContext<ThreadPool::TPHandle&> *c);
   void queue_for_snap_trim(PG *pg);
-  void queue_for_scrub(PG *pg, bool with_high_priority);
+  void queue_for_scrub(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  void queue_scrub_after_repair(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// queue the message (-> event) that all replicas have reserved scrub resources for us
+  void queue_for_scrub_granted(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// queue the message (-> event) that some replicas denied our scrub resources request
+  void queue_for_scrub_denied(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// Signals either (a) the end of a sleep period, or (b) a recheck of the availability
+  /// of the primary map being created by the backend.
+  void queue_for_scrub_resched(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// Signals a change in the number of in-flight recovery writes
+  void queue_scrub_pushes_update(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// Signals that all pending updates were applied
+  void queue_scrub_applied_update(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// Signals that the selected chunk (objects range) is available for scrubbing
+  void queue_scrub_chunk_free(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// The chunk selected is blocked by user operations, and cannot be scrubbed now
+  void queue_scrub_chunk_busy(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// The block-range that was locked and prevented the scrubbing - is freed
+  void queue_scrub_unblocking(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// Signals that all write OPs are done
+  void queue_scrub_digest_update(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// Signals that the the local (Primary's) scrub map is ready
+  void queue_scrub_got_local_map(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// Signals that we (the Primary) got all waited-for scrub-maps from our replicas
+  void queue_scrub_got_repl_maps(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// Signals that all chunks were handled
+  /// Note: always with high priority, as must be acted upon before the
+  /// next scrub request arrives from the Primary (and the primary is free
+  /// to send the request once the replica's map is received).
+  void queue_scrub_is_finished(PG* pg);
+
+  /// Signals that there are more chunks to handle
+  void queue_scrub_next_chunk(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  /// Signals that we have finished comparing the maps for this chunk
+  /// Note: required, as in Crimson this operation is 'futurized'.
+  void queue_scrub_maps_compared(PG* pg, Scrub::scrub_prio_t with_priority);
+
+  void queue_for_rep_scrub(PG* pg,
+                          Scrub::scrub_prio_t with_high_priority,
+                          unsigned int qu_priority,
+                          Scrub::act_token_t act_token);
+
+  /// Signals a change in the number of in-flight recovery writes
+  void queue_scrub_replica_pushes(PG *pg, Scrub::scrub_prio_t with_priority);
+
+  /// (not in Crimson) Queue a SchedReplica event to be sent to the replica, to
+  /// trigger a re-check of the availability of the scrub map prepared by the
+  /// backend.
+  void queue_for_rep_scrub_resched(PG* pg,
+                                  Scrub::scrub_prio_t with_high_priority,
+                                  unsigned int qu_priority,
+                                  Scrub::act_token_t act_token);
+
   void queue_for_pg_delete(spg_t pgid, epoch_t e);
   bool try_finish_pg_delete(PG *pg, unsigned old_pg_num);
 
 private:
   // -- pg recovery and associated throttling --
-  Mutex recovery_lock;
-  list<pair<epoch_t, PGRef> > awaiting_throttle;
+  ceph::mutex recovery_lock = ceph::make_mutex("OSDService::recovery_lock");
+  std::list<std::pair<epoch_t, PGRef> > awaiting_throttle;
+
+  /// queue a scrub-related message for a PG
+  template <class MSG_TYPE>
+  void queue_scrub_event_msg(PG* pg,
+                            Scrub::scrub_prio_t with_priority,
+                            unsigned int qu_priority,
+                            Scrub::act_token_t act_token);
+
+  /// An alternative version of queue_scrub_event_msg(), in which the queuing priority is
+  /// provided by the executing scrub (i.e. taken from PgScrubber::m_flags)
+  template <class MSG_TYPE>
+  void queue_scrub_event_msg(PG* pg, Scrub::scrub_prio_t with_priority);
 
   utime_t defer_recovery_until;
   uint64_t recovery_ops_active;
   uint64_t recovery_ops_reserved;
   bool recovery_paused;
 #ifdef DEBUG_RECOVERY_OIDS
-  map<spg_t, set<hobject_t> > recovery_oids;
+  std::map<spg_t, std::set<hobject_t> > recovery_oids;
 #endif
   bool _recover_now(uint64_t *available_pushes);
   void _maybe_queue_recovery();
   void _queue_for_recovery(
-    pair<epoch_t, PGRef> p, uint64_t reserved_pushes);
+    std::pair<epoch_t, PGRef> p, uint64_t reserved_pushes);
 public:
   void start_recovery_op(PG *pg, const hobject_t& soid);
   void finish_recovery_op(PG *pg, const hobject_t& soid, bool dequeue);
@@ -811,30 +669,34 @@ public:
        return awaiting.second.get() == pg;
       });
   }
+
+  unsigned get_target_pg_log_entries() const;
+
   // delayed pg activation
   void queue_for_recovery(PG *pg) {
     std::lock_guard l(recovery_lock);
 
     if (pg->is_forced_recovery_or_backfill()) {
-      awaiting_throttle.push_front(make_pair(pg->get_osdmap()->get_epoch(), pg));
+      awaiting_throttle.push_front(std::make_pair(pg->get_osdmap()->get_epoch(), pg));
     } else {
-      awaiting_throttle.push_back(make_pair(pg->get_osdmap()->get_epoch(), pg));
+      awaiting_throttle.push_back(std::make_pair(pg->get_osdmap()->get_epoch(), pg));
     }
     _maybe_queue_recovery();
   }
   void queue_recovery_after_sleep(PG *pg, epoch_t queued, uint64_t reserved_pushes) {
     std::lock_guard l(recovery_lock);
-    _queue_for_recovery(make_pair(queued, pg), reserved_pushes);
+    _queue_for_recovery(std::make_pair(queued, pg), reserved_pushes);
   }
 
+  void queue_check_readable(spg_t spgid,
+                           epoch_t lpr,
+                           ceph::signedspan delay = ceph::signedspan::zero());
+
   // osd map cache (past osd maps)
-  Mutex map_cache_lock;
+  ceph::mutex map_cache_lock = ceph::make_mutex("OSDService::map_cache_lock");
   SharedLRU<epoch_t, const OSDMap> map_cache;
-  SimpleLRU<epoch_t, bufferlist> map_bl_cache;
-  SimpleLRU<epoch_t, bufferlist> map_bl_inc_cache;
-
-  /// final pg_num values for recently deleted pools
-  map<int64_t,int> deleted_pool_pg_nums;
+  SimpleLRU<epoch_t, ceph::buffer::list> map_bl_cache;
+  SimpleLRU<epoch_t, ceph::buffer::list> map_bl_inc_cache;
 
   OSDMapRef try_get_map(epoch_t e);
   OSDMapRef get_map(epoch_t e) {
@@ -848,65 +710,40 @@ public:
   }
   OSDMapRef _add_map(OSDMap *o);
 
-  void add_map_bl(epoch_t e, bufferlist& bl) {
-    std::lock_guard l(map_cache_lock);
-    return _add_map_bl(e, bl);
-  }
-  void _add_map_bl(epoch_t e, bufferlist& bl);
-  bool get_map_bl(epoch_t e, bufferlist& bl) {
+  void _add_map_bl(epoch_t e, ceph::buffer::list& bl);
+  bool get_map_bl(epoch_t e, ceph::buffer::list& bl) {
     std::lock_guard l(map_cache_lock);
     return _get_map_bl(e, bl);
   }
-  bool _get_map_bl(epoch_t e, bufferlist& bl);
-
-  void add_map_inc_bl(epoch_t e, bufferlist& bl) {
-    std::lock_guard l(map_cache_lock);
-    return _add_map_inc_bl(e, bl);
-  }
-  void _add_map_inc_bl(epoch_t e, bufferlist& bl);
-  bool get_inc_map_bl(epoch_t e, bufferlist& bl);
+  bool _get_map_bl(epoch_t e, ceph::buffer::list& bl);
 
-  /// get last pg_num before a pool was deleted (if any)
-  int get_deleted_pool_pg_num(int64_t pool);
-
-  void store_deleted_pool_pg_num(int64_t pool, int pg_num) {
-    std::lock_guard l(map_cache_lock);
-    deleted_pool_pg_nums[pool] = pg_num;
-  }
-
-  /// get pgnum from newmap or, if pool was deleted, last map pool existed in
-  int get_possibly_deleted_pool_pg_num(OSDMapRef newmap,
-                                      int64_t pool) {
-    if (newmap->have_pg_pool(pool)) {
-      return newmap->get_pg_num(pool);
-    }
-    return get_deleted_pool_pg_num(pool);
-  }
+  void _add_map_inc_bl(epoch_t e, ceph::buffer::list& bl);
+  bool get_inc_map_bl(epoch_t e, ceph::buffer::list& bl);
 
   /// identify split child pgids over a osdmap interval
   void identify_splits_and_merges(
     OSDMapRef old_map,
     OSDMapRef new_map,
     spg_t pgid,
-    set<pair<spg_t,epoch_t>> *new_children,
-    set<pair<spg_t,epoch_t>> *merge_pgs);
+    std::set<std::pair<spg_t,epoch_t>> *new_children,
+    std::set<std::pair<spg_t,epoch_t>> *merge_pgs);
 
   void need_heartbeat_peer_update();
 
   void init();
-  void final_init();  
+  void final_init();
   void start_shutdown();
   void shutdown_reserver();
   void shutdown();
 
   // -- stats --
-  Mutex stat_lock;
+  ceph::mutex stat_lock = ceph::make_mutex("OSDService::stat_lock");
   osd_stat_t osd_stat;
   uint32_t seq = 0;
 
   void set_statfs(const struct store_statfs_t &stbuf,
     osd_alert_list_t& alerts);
-  osd_stat_t set_osd_stat(vector<int>& hb_peers, int num_pgs);
+  osd_stat_t set_osd_stat(std::vector<int>& hb_peers, int num_pgs);
   void inc_osd_stat_repaired(void);
   float compute_adjusted_ratio(osd_stat_t new_stat, float *pratio, uint64_t adjust_used = 0);
   osd_stat_t get_osd_stat() {
@@ -920,11 +757,17 @@ public:
     std::lock_guard l(stat_lock);
     return osd_stat.seq;
   }
+  void get_hb_pingtime(std::map<int, osd_stat_t::Interfaces> *pp)
+  {
+    std::lock_guard l(stat_lock);
+    *pp = osd_stat.hb_pingtime;
+    return;
+  }
 
   // -- OSD Full Status --
 private:
   friend TestOpsSocketHook;
-  mutable Mutex full_status_lock;
+  mutable ceph::mutex full_status_lock = ceph::make_mutex("OSDService::full_status_lock");
   enum s_names { INVALID = -1, NONE, NEARFULL, BACKFILLFULL, FULL, FAILSAFE } cur_state;  // ascending
   const char *get_full_state_name(s_names s) const {
     switch (s) {
@@ -936,7 +779,7 @@ private:
     default: return "???";
     }
   }
-  s_names get_full_state(string type) const {
+  s_names get_full_state(std::string type) const {
     if (type == "none")
       return NONE;
     else if (type == "failsafe")
@@ -958,7 +801,7 @@ private:
   bool _check_full(DoutPrefixProvider *dpp, s_names type) const;
 public:
   void check_full_status(float ratio, float pratio);
-  s_names recalc_full_state(float ratio, float pratio, string &inject);
+  s_names recalc_full_state(float ratio, float pratio, std::string &inject);
   bool _tentative_full(DoutPrefixProvider *dpp, s_names type, uint64_t adjust_used, osd_stat_t);
   bool check_failsafe_full(DoutPrefixProvider *dpp) const;
   bool check_full(DoutPrefixProvider *dpp) const;
@@ -971,24 +814,24 @@ public:
   bool is_nearfull() const;
   bool need_fullness_update();  ///< osdmap state needs update
   void set_injectfull(s_names type, int64_t count);
-  bool check_osdmap_full(const set<pg_shard_t> &missing_on);
 
 
   // -- epochs --
 private:
-  mutable Mutex epoch_lock; // protects access to boot_epoch, up_epoch, bind_epoch
+  // protects access to boot_epoch, up_epoch, bind_epoch
+  mutable ceph::mutex epoch_lock = ceph::make_mutex("OSDService::epoch_lock");
   epoch_t boot_epoch;  // _first_ epoch we were marked up (after this process started)
   epoch_t up_epoch;    // _most_recent_ epoch we were marked up
   epoch_t bind_epoch;  // epoch we last did a bind to new ip:ports
 public:
   /**
-   * Retrieve the boot_, up_, and bind_ epochs the OSD has set. The params
+   * Retrieve the boot_, up_, and bind_ epochs the OSD has std::set. The params
    * can be NULL if you don't care about them.
    */
   void retrieve_epochs(epoch_t *_boot_epoch, epoch_t *_up_epoch,
                        epoch_t *_bind_epoch) const;
   /**
-   * Set the boot, up, and bind epochs. Any NULL params will not be set.
+   * Std::set the boot, up, and bind epochs. Any NULL params will not be std::set.
    */
   void set_epochs(const epoch_t *_boot_epoch, const epoch_t *_up_epoch,
                   const epoch_t *_bind_epoch);
@@ -1010,9 +853,24 @@ public:
 
   void request_osdmap_update(epoch_t e);
 
+  // -- heartbeats --
+  ceph::mutex hb_stamp_lock = ceph::make_mutex("OSDServce::hb_stamp_lock");
+
+  /// osd -> heartbeat stamps
+  std::vector<HeartbeatStampsRef> hb_stamps;
+
+  /// get or create a ref for a peer's HeartbeatStamps
+  HeartbeatStampsRef get_hb_stamps(unsigned osd);
+
+
+  // Timer for readable leases
+  ceph::timer<ceph::mono_clock> mono_timer = ceph::timer<ceph::mono_clock>{ceph::construct_suspended};
+
+  void queue_renew_lease(epoch_t epoch, spg_t spgid);
+
   // -- stopping --
-  Mutex is_stopping_lock;
-  Cond is_stopping_cond;
+  ceph::mutex is_stopping_lock = ceph::make_mutex("OSDService::is_stopping_lock");
+  ceph::condition_variable is_stopping_cond;
   enum {
     NOT_STOPPING,
     PREPARING_TO_STOP,
@@ -1035,27 +893,18 @@ public:
 
 
 #ifdef PG_DEBUG_REFS
-  Mutex pgid_lock;
-  map<spg_t, int> pgid_tracker;
-  map<spg_t, PG*> live_pgs;
+  ceph::mutex pgid_lock = ceph::make_mutex("OSDService::pgid_lock");
+  std::map<spg_t, int> pgid_tracker;
+  std::map<spg_t, PG*> live_pgs;
   void add_pgid(spg_t pgid, PG *pg);
   void remove_pgid(spg_t pgid, PG *pg);
   void dump_live_pgids();
 #endif
 
-  explicit OSDService(OSD *osd);
-  ~OSDService();
-};
-
-
-enum class io_queue {
-  prioritized,
-  weightedpriority,
-  mclock_opclass,
-  mclock_client,
+  explicit OSDService(OSD *osd, ceph::async::io_context_pool& poolctx);
+  ~OSDService() = default;
 };
 
-
 /*
 
   Each PG slot includes queues for events that are processing and/or waiting
@@ -1063,7 +912,7 @@ enum class io_queue {
 
   These are the constraints:
 
-  - client ops must remained ordered by client, regardless of map epoch
+  - client ops must remained ordered by client, regardless of std::map epoch
   - peering messages/events from peers must remain ordered by peer
   - peering messages and client ops need not be ordered relative to each other
 
@@ -1085,7 +934,7 @@ enum class io_queue {
     don't affect the given PG.)
 
   - we maintain two separate wait lists, *waiting* and *waiting_peering*. The
-    OpQueueItem has an is_peering() bool to determine which we use.  Waiting
+    OpSchedulerItem has an is_peering() bool to determine which we use.  Waiting
     peering events are queued up by epoch required.
 
   - when we wake a PG slot (e.g., we finished split, or got a newer osdmap, or
@@ -1102,26 +951,27 @@ enum class io_queue {
 
   - when we advance the osdmap on the OSDShard, we scan pg slots and
     discard any slots with no pg (and not waiting_for_split) that no
-    longer map to the current host.
+    longer std::map to the current host.
 
   */
 
 struct OSDShardPGSlot {
+  using OpSchedulerItem = ceph::osd::scheduler::OpSchedulerItem;
   PGRef pg;                      ///< pg reference
-  deque<OpQueueItem> to_process; ///< order items for this slot
+  std::deque<OpSchedulerItem> to_process; ///< order items for this slot
   int num_running = 0;          ///< _process threads doing pg lookup/lock
 
-  deque<OpQueueItem> waiting;   ///< waiting for pg (or map + pg)
+  std::deque<OpSchedulerItem> waiting;   ///< waiting for pg (or map + pg)
 
   /// waiting for map (peering evt)
-  map<epoch_t,deque<OpQueueItem>> waiting_peering;
+  std::map<epoch_t,std::deque<OpSchedulerItem>> waiting_peering;
 
   /// incremented by wake_pg_waiters; indicates racing _process threads
   /// should bail out (their op has been requeued)
   uint64_t requeue_seq = 0;
 
   /// waiting for split child to materialize in these epoch(s)
-  set<epoch_t> waiting_for_split;
+  std::set<epoch_t> waiting_for_split;
 
   epoch_t epoch = 0;
   boost::intrusive::set_member_hook<> pg_epoch_item;
@@ -1135,13 +985,13 @@ struct OSDShard {
   CephContext *cct;
   OSD *osd;
 
-  string shard_name;
+  std::string shard_name;
 
-  string sdata_wait_lock_name;
+  std::string sdata_wait_lock_name;
   ceph::mutex sdata_wait_lock;
   ceph::condition_variable sdata_cond;
+  int waiting_threads = 0;
 
-  string osdmap_lock_name;
   ceph::mutex osdmap_lock;  ///< protect shard_osdmap updates vs users w/o shard_lock
   OSDMapRef shard_osdmap;
 
@@ -1150,13 +1000,13 @@ struct OSDShard {
     return shard_osdmap;
   }
 
-  string shard_lock_name;
+  std::string shard_lock_name;
   ceph::mutex shard_lock;   ///< protects remaining members below
 
   /// map of slots for each spg_t.  maintains ordering of items dequeued
-  /// from pqueue while _process thread drops shard lock to acquire the
+  /// from scheduler while _process thread drops shard lock to acquire the
   /// pg lock.  stale slots are removed by consume_map.
-  unordered_map<spg_t,unique_ptr<OSDShardPGSlot>> pg_slots;
+  std::unordered_map<spg_t,std::unique_ptr<OSDShardPGSlot>> pg_slots;
 
   struct pg_slot_compare_by_epoch {
     bool operator()(const OSDShardPGSlot& l, const OSDShardPGSlot& r) const {
@@ -1176,25 +1026,12 @@ struct OSDShard {
   ceph::condition_variable min_pg_epoch_cond;
 
   /// priority queue
-  std::unique_ptr<OpQueue<OpQueueItem, uint64_t>> pqueue;
+  ceph::osd::scheduler::OpSchedulerRef scheduler;
 
   bool stop_waiting = false;
 
   ContextQueue context_queue;
 
-  void _enqueue_front(OpQueueItem&& item, unsigned cutoff) {
-    unsigned priority = item.get_priority();
-    unsigned cost = item.get_cost();
-    if (priority >= cutoff)
-      pqueue->enqueue_strict_front(
-       item.get_owner(),
-       priority, std::move(item));
-    else
-      pqueue->enqueue_front(
-       item.get_owner(),
-       priority, cost, std::move(item));
-  }
-
   void _attach_pg(OSDShardPGSlot *slot, PG *pg);
   void _detach_pg(OSDShardPGSlot *slot);
 
@@ -1207,64 +1044,42 @@ struct OSDShard {
 
   /// push osdmap into shard
   void consume_map(
-    OSDMapRef& osdmap,
+    const OSDMapRef& osdmap,
     unsigned *pushes_to_free);
 
-  void _wake_pg_slot(spg_t pgid, OSDShardPGSlot *slot);
+  int _wake_pg_slot(spg_t pgid, OSDShardPGSlot *slot);
 
   void identify_splits_and_merges(
     const OSDMapRef& as_of_osdmap,
-    set<pair<spg_t,epoch_t>> *split_children,
-    set<pair<spg_t,epoch_t>> *merge_pgs);
-  void _prime_splits(set<pair<spg_t,epoch_t>> *pgids);
+    std::set<std::pair<spg_t,epoch_t>> *split_children,
+    std::set<std::pair<spg_t,epoch_t>> *merge_pgs);
+  void _prime_splits(std::set<std::pair<spg_t,epoch_t>> *pgids);
   void prime_splits(const OSDMapRef& as_of_osdmap,
-                   set<pair<spg_t,epoch_t>> *pgids);
+                   std::set<std::pair<spg_t,epoch_t>> *pgids);
   void prime_merges(const OSDMapRef& as_of_osdmap,
-                   set<pair<spg_t,epoch_t>> *merge_pgs);
+                   std::set<std::pair<spg_t,epoch_t>> *merge_pgs);
   void register_and_wake_split_child(PG *pg);
   void unprime_split_children(spg_t parent, unsigned old_pg_num);
+  void update_scheduler_config();
+  std::string get_scheduler_type();
 
   OSDShard(
     int id,
     CephContext *cct,
-    OSD *osd,
-    uint64_t max_tok_per_prio, uint64_t min_cost,
-    io_queue opqueue)
-    : 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_name(shard_name + "::osdmap_lock"),
-      osdmap_lock{make_mutex(osdmap_lock_name)},
-      shard_lock_name(shard_name + "::shard_lock"),
-      shard_lock{make_mutex(shard_lock_name)},
-      context_queue(sdata_wait_lock, sdata_cond) {
-    if (opqueue == io_queue::weightedpriority) {
-      pqueue = std::make_unique<
-       WeightedPriorityQueue<OpQueueItem,uint64_t>>(
-         max_tok_per_prio, min_cost);
-    } else if (opqueue == io_queue::prioritized) {
-      pqueue = std::make_unique<
-       PrioritizedQueue<OpQueueItem,uint64_t>>(
-         max_tok_per_prio, min_cost);
-    } else if (opqueue == io_queue::mclock_opclass) {
-      pqueue = std::make_unique<ceph::mClockOpClassQueue>(cct);
-    } else if (opqueue == io_queue::mclock_client) {
-      pqueue = std::make_unique<ceph::mClockClientQueue>(cct);
-    }
-  }
+    OSD *osd);
 };
 
 class OSD : public Dispatcher,
            public md_config_obs_t {
+  using OpSchedulerItem = ceph::osd::scheduler::OpSchedulerItem;
+
   /** OSD **/
-  Mutex osd_lock;          // global lock
+  // global lock
+  ceph::mutex osd_lock = ceph::make_mutex("OSD::osd_lock");
   SafeTimer tick_timer;    // safe timer (osd_lock)
 
   // Tick timer for those stuff that do not need osd_lock
-  Mutex tick_timer_lock;
+  ceph::mutex tick_timer_lock = ceph::make_mutex("OSD::tick_timer_lock");
   SafeTimer tick_timer_without_osd_lock;
   std::string gss_ktfile_client{};
 
@@ -1288,7 +1103,7 @@ protected:
   MgrClient   mgrc;
   PerfCounters      *logger;
   PerfCounters      *recoverystate_perf;
-  ObjectStore *store;
+  std::unique_ptr<ObjectStore> store;
 #ifdef HAVE_LIBFUSE
   FuseStore *fuse_store = nullptr;
 #endif
@@ -1298,7 +1113,7 @@ protected:
   int whoami;
   std::string dev_path, journal_path;
 
-  int last_require_osd_release = 0;
+  ceph_release_t last_require_osd_release{ceph_release_t::unknown};
 
   int numa_node = -1;
   size_t numa_cpu_set_size = 0;
@@ -1308,8 +1123,8 @@ protected:
   bool journal_is_rotational = true;
 
   ZTracer::Endpoint trace_endpoint;
-  void create_logger();
-  void create_recoverystate_perf();
+  PerfCounters* create_logger();
+  PerfCounters* create_recoverystate_perf();
   void tick();
   void tick_without_osd_lock();
   void _dispatch(Message *m);
@@ -1320,13 +1135,27 @@ protected:
   // asok
   friend class OSDSocketHook;
   class OSDSocketHook *asok_hook;
-  bool asok_command(std::string_view admin_command, const cmdmap_t& cmdmap,
-                   std::string_view format, std::ostream& ss);
+  using PGRefOrError = std::tuple<std::optional<PGRef>, int>;
+    PGRefOrError locate_asok_target(const cmdmap_t& cmdmap,
+                                   std::stringstream& ss, bool only_primary);
+  int asok_route_to_pg(bool only_primary,
+    std::string_view prefix,
+    cmdmap_t cmdmap,
+    Formatter *f,
+    std::stringstream& ss,
+    const bufferlist& inbl,
+    bufferlist& outbl,
+    std::function<void(int, const std::string&, bufferlist&)> on_finish);
+  void asok_command(
+    std::string_view prefix,
+    const cmdmap_t& cmdmap,
+    ceph::Formatter *f,
+    const ceph::buffer::list& inbl,
+    std::function<void(int,const std::string&,ceph::buffer::list&)> on_finish);
 
 public:
-  ClassHandler  *class_handler = nullptr;
   int get_nodeid() { return whoami; }
-  
+
   static ghobject_t get_osdmap_pobject_name(epoch_t epoch) {
     char foo[20];
     snprintf(foo, sizeof(foo), "osdmap.%d", epoch);
@@ -1344,19 +1173,25 @@ public:
        object_t("snapmapper"),
        0)));
   }
+  static ghobject_t make_purged_snaps_oid() {
+    return ghobject_t(hobject_t(
+      sobject_t(
+       object_t("purged_snaps"),
+       0)));
+  }
 
   static ghobject_t make_pg_log_oid(spg_t pg) {
-    stringstream ss;
+    std::stringstream ss;
     ss << "pglog_" << pg;
-    string s;
+    std::string s;
     getline(ss, s);
     return ghobject_t(hobject_t(sobject_t(object_t(s.c_str()), 0)));
   }
-  
+
   static ghobject_t make_pg_biginfo_oid(spg_t pg) {
-    stringstream ss;
+    std::stringstream ss;
     ss << "pginfo_" << pg;
-    string s;
+    std::string s;
     getline(ss, s);
     return ghobject_t(hobject_t(sobject_t(object_t(s.c_str()), 0)));
   }
@@ -1369,7 +1204,7 @@ public:
     return ghobject_t(
       hobject_t(
        sobject_t(
-         object_t(string("final_pool_") + stringify(pool)),
+         object_t(std::string("final_pool_") + stringify(pool)),
          CEPH_NOSNAP)));
   }
 
@@ -1385,7 +1220,7 @@ public:
   /**
    * get_osd_initial_compat_set()
    *
-   * Get the initial feature set for this OSD.  Features
+   * Get the initial feature std::set for this OSD.  Features
    * here are automatically upgraded.
    *
    * Return value: Initial osd CompatSet
@@ -1400,7 +1235,7 @@ public:
    * Return value: CompatSet of all supported features
    */
   static CompatSet get_osd_compat_set();
-  
+
 
 private:
   class C_Tick;
@@ -1475,43 +1310,40 @@ public:
 private:
 
   ShardedThreadPool osd_op_tp;
-  ThreadPool command_tp;
 
   void get_latest_osdmap();
 
   // -- sessions --
 private:
-  void dispatch_session_waiting(SessionRef session, OSDMapRef osdmap);
-  void maybe_share_map(Session *session, OpRequestRef op, OSDMapRef osdmap);
+  void dispatch_session_waiting(const ceph::ref_t<Session>& session, OSDMapRef osdmap);
 
-  Mutex session_waiting_lock;
-  set<SessionRef> session_waiting_for_map;
+  ceph::mutex session_waiting_lock = ceph::make_mutex("OSD::session_waiting_lock");
+  std::set<ceph::ref_t<Session>> session_waiting_for_map;
 
   /// Caller assumes refs for included Sessions
-  void get_sessions_waiting_for_map(set<SessionRef> *out) {
+  void get_sessions_waiting_for_map(std::set<ceph::ref_t<Session>> *out) {
     std::lock_guard l(session_waiting_lock);
     out->swap(session_waiting_for_map);
   }
-  void register_session_waiting_on_map(SessionRef session) {
+  void register_session_waiting_on_map(const ceph::ref_t<Session>& session) {
     std::lock_guard l(session_waiting_lock);
     session_waiting_for_map.insert(session);
   }
-  void clear_session_waiting_on_map(SessionRef session) {
+  void clear_session_waiting_on_map(const ceph::ref_t<Session>& session) {
     std::lock_guard l(session_waiting_lock);
     session_waiting_for_map.erase(session);
   }
   void dispatch_sessions_waiting_on_map() {
-    set<SessionRef> sessions_to_check;
+    std::set<ceph::ref_t<Session>> sessions_to_check;
     get_sessions_waiting_for_map(&sessions_to_check);
     for (auto i = sessions_to_check.begin();
         i != sessions_to_check.end();
         sessions_to_check.erase(i++)) {
       std::lock_guard l{(*i)->session_dispatch_lock};
-      SessionRef session = *i;
-      dispatch_session_waiting(session, osdmap);
+      dispatch_session_waiting(*i, get_osdmap());
     }
   }
-  void session_handle_reset(SessionRef session) {
+  void session_handle_reset(const ceph::ref_t<Session>& session) {
     std::lock_guard l(session->session_dispatch_lock);
     clear_session_waiting_on_map(session);
 
@@ -1543,7 +1375,7 @@ private:
   void osdmap_subscribe(version_t epoch, bool force_request);
   /** @} monc helpers */
 
-  Mutex osdmap_subscribe_lock;
+  ceph::mutex osdmap_subscribe_lock = ceph::make_mutex("OSD::osdmap_subscribe_lock");
   epoch_t latest_subscribed_epoch{0};
 
   // -- heartbeat --
@@ -1561,9 +1393,35 @@ private:
     static constexpr int HEARTBEAT_MAX_CONN = 2;
     /// history of inflight pings, arranging by timestamp we sent
     /// send time -> deadline -> remaining replies
-    map<utime_t, pair<utime_t, int>> ping_history;
+    std::map<utime_t, std::pair<utime_t, int>> ping_history;
+
+    utime_t hb_interval_start;
+    uint32_t hb_average_count = 0;
+    uint32_t hb_index = 0;
+
+    uint32_t hb_total_back = 0;
+    uint32_t hb_min_back = UINT_MAX;
+    uint32_t hb_max_back = 0;
+    std::vector<uint32_t> hb_back_pingtime;
+    std::vector<uint32_t> hb_back_min;
+    std::vector<uint32_t> hb_back_max;
+
+    uint32_t hb_total_front = 0;
+    uint32_t hb_min_front = UINT_MAX;
+    uint32_t hb_max_front = 0;
+    std::vector<uint32_t> hb_front_pingtime;
+    std::vector<uint32_t> hb_front_min;
+    std::vector<uint32_t> hb_front_max;
+
+    bool is_stale(utime_t stale) const {
+      if (ping_history.empty()) {
+        return false;
+      }
+      utime_t oldest_deadline = ping_history.begin()->second.first;
+      return oldest_deadline <= stale;
+    }
 
-    bool is_unhealthy(utime_t now) {
+    bool is_unhealthy(utime_t now) const {
       if (ping_history.empty()) {
         /// we haven't sent a ping yet or we have got all replies,
         /// in either way we are safe and healthy for now
@@ -1574,7 +1432,7 @@ private:
       return now > oldest_deadline;
     }
 
-    bool is_healthy(utime_t now) {
+    bool is_healthy(utime_t now) const {
       if (last_rx_front == utime_t() || last_rx_back == utime_t()) {
         // only declare to be healthy until we have received the first
         // replies from both front/back connections
@@ -1582,18 +1440,27 @@ private:
       }
       return !is_unhealthy(now);
     }
+
+    void clear_mark_down(Connection *except = nullptr) {
+      if (con_back && con_back != except) {
+       con_back->mark_down();
+       con_back->clear_priv();
+       con_back.reset(nullptr);
+      }
+      if (con_front && con_front != except) {
+       con_front->mark_down();
+       con_front->clear_priv();
+       con_front.reset(nullptr);
+      }
+    }
   };
-  /// state attached to outgoing heartbeat connections
-  struct HeartbeatSession : public RefCountedObject {
-    int peer;
-    explicit HeartbeatSession(int p) : peer(p) {}
-  };
-  Mutex heartbeat_lock;
-  map<int, int> debug_heartbeat_drops_remaining;
-  Cond heartbeat_cond;
+
+  ceph::mutex heartbeat_lock = ceph::make_mutex("OSD::heartbeat_lock");
+  std::map<int, int> debug_heartbeat_drops_remaining;
+  ceph::condition_variable heartbeat_cond;
   bool heartbeat_stop;
-  std::atomic<bool> heartbeat_need_update;   
-  map<int,HeartbeatInfo> heartbeat_peers;  ///< map of osd id to HeartbeatInfo
+  std::atomic<bool> heartbeat_need_update;
+  std::map<int,HeartbeatInfo> heartbeat_peers;  ///< map of osd id to HeartbeatInfo
   utime_t last_mon_heartbeat;
   Messenger *hb_front_client_messenger;
   Messenger *hb_back_client_messenger;
@@ -1601,12 +1468,17 @@ private:
   Messenger *hb_back_server_messenger;
   utime_t last_heartbeat_resample;   ///< last time we chose random peers in waiting-for-healthy state
   double daily_loadavg;
-  
+  ceph::mono_time startup_time;
+
+  // Track ping repsonse times using vector as a circular buffer
+  // MUST BE A POWER OF 2
+  const uint32_t hb_vector_size = 16;
+
   void _add_heartbeat_peer(int p);
   void _remove_heartbeat_peer(int p);
   bool heartbeat_reset(Connection *con);
   void maybe_update_heartbeat_peers();
-  void reset_heartbeat_peers();
+  void reset_heartbeat_peers(bool all);
   bool heartbeat_peers_need_update() {
     return heartbeat_need_update.load();
   }
@@ -1623,7 +1495,7 @@ private:
 
   void heartbeat_kick() {
     std::lock_guard l(heartbeat_lock);
-    heartbeat_cond.Signal();
+    heartbeat_cond.notify_all();
   }
 
   struct T_Heartbeat : public Thread {
@@ -1668,56 +1540,38 @@ public:
     int ms_handle_authentication(Connection *con) override {
       return true;
     }
-    bool ms_get_authorizer(int dest_type, AuthAuthorizer **authorizer) override {
-      // some pre-nautilus OSDs get confused if you include an
-      // authorizer but they are not expecting it.  do not try to authorize
-      // heartbeat connections until all OSDs are nautilus.
-      if (osd->get_osdmap()->require_osd_release >= CEPH_RELEASE_NAUTILUS) {
-       return osd->ms_get_authorizer(dest_type, authorizer);
-      }
-      return false;
-    }
-    KeyStore *ms_get_auth1_authorizer_keystore() override {
-      return osd->ms_get_auth1_authorizer_keystore();
-    }
   } heartbeat_dispatcher;
 
 private:
   // -- waiters --
-  list<OpRequestRef> finished;
-  
-  void take_waiters(list<OpRequestRef>& ls) {
-    ceph_assert(osd_lock.is_locked());
+  std::list<OpRequestRef> finished;
+
+  void take_waiters(std::list<OpRequestRef>& ls) {
+    ceph_assert(ceph_mutex_is_locked(osd_lock));
     finished.splice(finished.end(), ls);
   }
   void do_waiters();
-  
+
   // -- op tracking --
   OpTracker op_tracker;
-  void test_ops(std::string command, std::string args, ostream& ss);
+  void test_ops(std::string command, std::string args, std::ostream& ss);
   friend class TestOpsSocketHook;
   TestOpsSocketHook *test_ops_hook;
   friend struct C_FinishSplits;
   friend struct C_OpenPGs;
 
-  // -- op queue --
-  friend std::ostream& operator<<(std::ostream& out, const io_queue& q);
-
-  const io_queue op_queue;
-public:
-  const unsigned int op_prio_cutoff;
 protected:
 
   /*
    * The ordered op delivery chain is:
    *
-   *   fast dispatch -> pqueue back
-   *                    pqueue front <-> to_process back
+   *   fast dispatch -> scheduler back
+   *                    scheduler front <-> to_process back
    *                                     to_process front  -> RunVis(item)
    *                                                      <- queue_front()
    *
-   * The pqueue is per-shard, and to_process is per pg_slot.  Items can be
-   * pushed back up into to_process and/or pqueue while order is preserved.
+   * The scheduler is per-shard, and to_process is per pg_slot.  Items can be
+   * pushed back up into to_process and/or scheduler while order is preserved.
    *
    * Multiple worker threads can operate on each shard.
    *
@@ -1728,39 +1582,40 @@ protected:
    * wake_pg_waiters, and (2) when wake_pg_waiters just ran, waiting_for_pg
    * and already requeued the items.
    */
-  friend class PGOpItem;
-  friend class PGPeeringItem;
-  friend class PGRecovery;
-  friend class PGDelete;
+  friend class ceph::osd::scheduler::PGOpItem;
+  friend class ceph::osd::scheduler::PGPeeringItem;
+  friend class ceph::osd::scheduler::PGRecovery;
+  friend class ceph::osd::scheduler::PGRecoveryMsg;
+  friend class ceph::osd::scheduler::PGDelete;
 
   class ShardedOpWQ
-    : public ShardedThreadPool::ShardedWQ<OpQueueItem>
+    : public ShardedThreadPool::ShardedWQ<OpSchedulerItem>
   {
     OSD *osd;
 
   public:
     ShardedOpWQ(OSD *o,
-               time_t ti,
-               time_t si,
+               ceph::timespan ti,
+               ceph::timespan si,
                ShardedThreadPool* tp)
-      : ShardedThreadPool::ShardedWQ<OpQueueItem>(ti, si, tp),
+      : ShardedThreadPool::ShardedWQ<OpSchedulerItem>(ti, si, tp),
         osd(o) {
     }
 
     void _add_slot_waiter(
       spg_t token,
       OSDShardPGSlot *slot,
-      OpQueueItem&& qi);
+      OpSchedulerItem&& qi);
 
     /// try to do some work
-    void _process(uint32_t thread_index, heartbeat_handle_d *hb) override;
+    void _process(uint32_t thread_index, ceph::heartbeat_handle_d *hb) override;
 
     /// enqueue a new item
-    void _enqueue(OpQueueItem&& item) override;
+    void _enqueue(OpSchedulerItem&& item) override;
 
     /// requeue an old item (at the front of the line)
-    void _enqueue_front(OpQueueItem&& item) override;
-      
+    void _enqueue_front(OpSchedulerItem&& item) override;
+
     void return_waiting_threads() override {
       for(uint32_t i = 0; i < osd->num_shards; i++) {
        OSDShard* sdata = osd->shards[i];
@@ -1780,7 +1635,7 @@ protected:
       }
     }
 
-    void dump(Formatter *f) {
+    void dump(ceph::Formatter *f) {
       for(uint32_t i = 0; i < osd->num_shards; i++) {
        auto &&sdata = osd->shards[i];
 
@@ -1790,7 +1645,7 @@ protected:
 
        std::scoped_lock l{sdata->shard_lock};
        f->open_object_section(queue_name);
-       sdata->pqueue->dump(f);
+       sdata->scheduler->dump(*f);
        f->close_section();
       }
     }
@@ -1801,13 +1656,13 @@ protected:
       ceph_assert(sdata);
       std::lock_guard l(sdata->shard_lock);
       if (thread_index < osd->num_shards) {
-       return sdata->pqueue->empty() && sdata->context_queue.empty();
+       return sdata->scheduler->empty() && sdata->context_queue.empty();
       } else {
-       return sdata->pqueue->empty();
+       return sdata->scheduler->empty();
       }
     }
 
-    void handle_oncommits(list<Context*>& oncommits) {
+    void handle_oncommits(std::list<Context*>& oncommits) {
       for (auto p : oncommits) {
        p->complete(0);
       }
@@ -1823,9 +1678,6 @@ protected:
   void enqueue_peering_evt(
     spg_t pgid,
     PGPeeringEventRef ref);
-  void enqueue_peering_evt_front(
-    spg_t pgid,
-    PGPeeringEventRef ref);
   void dequeue_peering_evt(
     OSDShard *sdata,
     PG *pg,
@@ -1839,27 +1691,33 @@ protected:
     ThreadPool::TPHandle& handle);
 
   friend class PG;
-  friend class OSDShard;
+  friend struct OSDShard;
   friend class PrimaryLogPG;
+  friend class PgScrubber;
 
 
  protected:
 
   // -- osd map --
-  OSDMapRef       osdmap;
-  OSDMapRef get_osdmap() {
-    return osdmap;
+  // TODO: switch to std::atomic<OSDMapRef> when C++20 will be available.
+  OSDMapRef       _osdmap;
+  void set_osdmap(OSDMapRef osdmap) {
+    std::atomic_store(&_osdmap, osdmap);
+  }
+  OSDMapRef get_osdmap() const {
+    return std::atomic_load(&_osdmap);
   }
   epoch_t get_osdmap_epoch() const {
+    // XXX: performance?
+    auto osdmap = get_osdmap();
     return osdmap ? osdmap->get_epoch() : 0;
   }
 
   pool_pg_num_history_t pg_num_history;
 
-  utime_t         had_map_since;
-  RWLock          map_lock;
-  list<OpRequestRef>  waiting_for_osdmap;
-  deque<utime_t> osd_markdown_log;
+  ceph::shared_mutex map_lock = ceph::make_shared_mutex("OSD::map_lock");
+  std::list<OpRequestRef>  waiting_for_osdmap;
+  std::deque<utime_t> osd_markdown_log;
 
   friend struct send_map_on_destruct;
 
@@ -1869,13 +1727,13 @@ protected:
   void trim_maps(epoch_t oldest, int nreceived, bool skip_maps);
   void note_down_osd(int osd);
   void note_up_osd(int osd);
-  friend class C_OnMapCommit;
+  friend struct C_OnMapCommit;
 
   bool advance_pg(
     epoch_t advance_to,
     PG *pg,
     ThreadPool::TPHandle &handle,
-    PG::RecoveryCtx *rctx);
+    PeeringCtx &rctx);
   void consume_map();
   void activate_map();
 
@@ -1886,19 +1744,13 @@ protected:
   OSDMapRef add_map(OSDMap *o) {
     return service.add_map(o);
   }
-  void add_map_bl(epoch_t e, bufferlist& bl) {
-    return service.add_map_bl(e, bl);
-  }
-  bool get_map_bl(epoch_t e, bufferlist& bl) {
+  bool get_map_bl(epoch_t e, ceph::buffer::list& bl) {
     return service.get_map_bl(e, bl);
   }
-  void add_map_inc_bl(epoch_t e, bufferlist& bl) {
-    return service.add_map_inc_bl(e, bl);
-  }
 
 public:
   // -- shards --
-  vector<OSDShard*> shards;
+  std::vector<OSDShard*> shards;
   uint32_t num_shards = 0;
 
   void inc_num_pgs() {
@@ -1912,9 +1764,9 @@ public:
   }
 
 protected:
-  Mutex merge_lock = {"OSD::merge_lock"};
+  ceph::mutex merge_lock = ceph::make_mutex("OSD::merge_lock");
   /// merge epoch -> target pgid -> source pgid -> pg
-  map<epoch_t,map<spg_t,map<spg_t,PGRef>>> merge_waiters;
+  std::map<epoch_t,std::map<spg_t,std::map<spg_t,PGRef>>> merge_waiters;
 
   bool add_merge_waiter(OSDMapRef nextmap, spg_t target, PGRef source,
                        unsigned need);
@@ -1923,7 +1775,7 @@ protected:
   std::atomic<size_t> num_pgs = {0};
 
   std::mutex pending_creates_lock;
-  using create_from_osd_t = std::pair<pg_t, bool /* is primary*/>;
+  using create_from_osd_t = std::pair<spg_t, bool /* is primary*/>;
   std::set<create_from_osd_t> pending_creates_from_osd;
   unsigned pending_creates_from_mon = 0;
 
@@ -1934,8 +1786,8 @@ protected:
   void register_pg(PGRef pg);
   bool try_finish_pg_delete(PG *pg, unsigned old_pg_num);
 
-  void _get_pgs(vector<PGRef> *v, bool clear_too=false);
-  void _get_pgids(vector<spg_t> *v);
+  void _get_pgs(std::vector<PGRef> *v, bool clear_too=false);
+  void _get_pgids(std::vector<spg_t> *v);
 
 public:
   PGRef lookup_lock_pg(spg_t pgid);
@@ -1965,14 +1817,14 @@ protected:
 
   void split_pgs(
     PG *parent,
-    const set<spg_t> &childpgids, set<PGRef> *out_pgs,
+    const std::set<spg_t> &childpgids, std::set<PGRef> *out_pgs,
     OSDMapRef curmap,
     OSDMapRef nextmap,
-    PG::RecoveryCtx *rctx);
-  void _finish_splits(set<PGRef>& pgs);
+    PeeringCtx &rctx);
+  void _finish_splits(std::set<PGRef>& pgs);
 
   // == monitor interaction ==
-  Mutex mon_report_lock;
+  ceph::mutex mon_report_lock = ceph::make_mutex("OSD::mon_report_lock");
   utime_t last_mon_report;
   Finisher boot_finisher;
 
@@ -1981,14 +1833,16 @@ protected:
   void _got_mon_epochs(epoch_t oldest, epoch_t newest);
   void _preboot(epoch_t oldest, epoch_t newest);
   void _send_boot();
-  void _collect_metadata(map<string,string> *pmeta);
+  void _collect_metadata(std::map<std::string,std::string> *pmeta);
+  void _get_purged_snaps();
+  void handle_get_purged_snaps_reply(MMonGetPurgedSnapsReply *r);
 
   void start_waiting_for_healthy();
   bool _is_healthy();
 
   void send_full_update();
-  
-  friend struct C_OSD_GetVersion;
+
+  friend struct CB_OSD_GetVersion;
 
   // -- alive --
   epoch_t up_thru_wanted;
@@ -2010,8 +1864,8 @@ protected:
   void got_full_map(epoch_t e);
 
   // -- failures --
-  map<int,utime_t> failure_queue;
-  map<int,pair<utime_t,entity_addrvec_t> > failure_pending;
+  std::map<int,utime_t> failure_queue;
+  std::map<int,std::pair<utime_t,entity_addrvec_t> > failure_pending;
 
   void requeue_failures();
   void send_failures();
@@ -2019,7 +1873,7 @@ protected:
   void cancel_pending_failures();
 
   ceph::coarse_mono_clock::time_point last_sent_beacon;
-  Mutex min_last_epoch_clean_lock{"OSD::min_last_epoch_clean_lock"};
+  ceph::mutex min_last_epoch_clean_lock = ceph::make_mutex("OSD::min_last_epoch_clean_lock");
   epoch_t min_last_epoch_clean = 0;
   // which pgs were scanned for min_lec
   std::vector<pg_t> min_last_epoch_clean_pgs;
@@ -2030,21 +1884,8 @@ protected:
   }
 
   // -- generic pg peering --
-  PG::RecoveryCtx create_context();
-  void dispatch_context(PG::RecoveryCtx &ctx, PG *pg, OSDMapRef curmap,
+  void dispatch_context(PeeringCtx &ctx, PG *pg, OSDMapRef curmap,
                         ThreadPool::TPHandle *handle = NULL);
-  void dispatch_context_transaction(PG::RecoveryCtx &ctx, PG *pg,
-                                    ThreadPool::TPHandle *handle = NULL);
-  void discard_context(PG::RecoveryCtx &ctx);
-  void do_notifies(map<int,
-                      vector<pair<pg_notify_t, PastIntervals> > >&
-                      notify_list,
-                  OSDMapRef map);
-  void do_queries(map<int, map<spg_t,pg_query_t> >& query_map,
-                 OSDMapRef map);
-  void do_infos(map<int,
-                   vector<pair<pg_notify_t, PastIntervals> > >& info_map,
-               OSDMapRef map);
 
   bool require_mon_peer(const Message *m);
   bool require_mon_or_mgr_peer(const Message *m);
@@ -2056,17 +1897,16 @@ protected:
   bool require_self_aliveness(const Message *m, epoch_t alive_since);
   /**
    * Verifies that the OSD who sent the given op has the same
-   * address as in the given map.
+   * address as in the given std::map.
    * @pre op was sent by an OSD using the cluster messenger
    */
-  bool require_same_peer_instance(const Message *m, OSDMapRef& map,
+  bool require_same_peer_instance(const Message *m, const OSDMapRef& map,
                                  bool is_fast_dispatch);
 
   bool require_same_or_newer_map(OpRequestRef& op, epoch_t e,
                                 bool is_fast_dispatch);
 
   void handle_fast_pg_create(MOSDPGCreate2 *m);
-  void handle_fast_pg_query(MOSDPGQuery *m);
   void handle_pg_query_nopg(const MQuery& q);
   void handle_fast_pg_notify(MOSDPGNotify *m);
   void handle_pg_notify_nopg(const MNotifyRec& q);
@@ -2081,64 +1921,7 @@ protected:
   void handle_fast_force_recovery(MOSDForceRecovery *m);
 
   // -- commands --
-  struct Command {
-    vector<string> cmd;
-    ceph_tid_t tid;
-    bufferlist indata;
-    ConnectionRef con;
-
-    Command(vector<string>& c, ceph_tid_t t, bufferlist& bl, Connection *co)
-      : cmd(c), tid(t), indata(bl), con(co) {}
-  };
-  list<Command*> command_queue;
-  struct CommandWQ : public ThreadPool::WorkQueue<Command> {
-    OSD *osd;
-    CommandWQ(OSD *o, time_t ti, time_t si, ThreadPool *tp)
-      : ThreadPool::WorkQueue<Command>("OSD::CommandWQ", ti, si, tp), osd(o) {}
-
-    bool _empty() override {
-      return osd->command_queue.empty();
-    }
-    bool _enqueue(Command *c) override {
-      osd->command_queue.push_back(c);
-      return true;
-    }
-    void _dequeue(Command *pg) override {
-      ceph_abort();
-    }
-    Command *_dequeue() override {
-      if (osd->command_queue.empty())
-       return NULL;
-      Command *c = osd->command_queue.front();
-      osd->command_queue.pop_front();
-      return c;
-    }
-    void _process(Command *c, ThreadPool::TPHandle &) override {
-      osd->osd_lock.lock();
-      if (osd->is_stopping()) {
-       osd->osd_lock.unlock();
-       delete c;
-       return;
-      }
-      osd->do_command(c->con.get(), c->tid, c->cmd, c->indata);
-      osd->osd_lock.unlock();
-      delete c;
-    }
-    void _clear() override {
-      while (!osd->command_queue.empty()) {
-       Command *c = osd->command_queue.front();
-       osd->command_queue.pop_front();
-       delete c;
-      }
-    }
-  } command_wq;
-
-  void handle_command(class MMonCommand *m);
   void handle_command(class MCommand *m);
-  void do_command(Connection *con, ceph_tid_t tid, vector<string>& cmd, bufferlist& data);
-  int _do_command(
-    Connection *con, cmdmap_t& cmdmap, ceph_tid_t tid, bufferlist& data,
-    bufferlist& odata, stringstream& ss, stringstream& ds);
 
 
   // -- pg recovery --
@@ -2148,9 +1931,8 @@ protected:
 
   // -- scrubbing --
   void sched_scrub();
+  void resched_all_scrubs();
   bool scrub_random_backoff();
-  bool scrub_load_below_threshold();
-  bool scrub_time_permit(utime_t now);
 
   // -- status reporting --
   MPGStats *collect_pg_stats();
@@ -2169,8 +1951,11 @@ private:
     case MSG_MON_COMMAND:
     case MSG_OSD_PG_CREATE2:
     case MSG_OSD_PG_QUERY:
+    case MSG_OSD_PG_QUERY2:
     case MSG_OSD_PG_INFO:
+    case MSG_OSD_PG_INFO2:
     case MSG_OSD_PG_NOTIFY:
+    case MSG_OSD_PG_NOTIFY2:
     case MSG_OSD_PG_LOG:
     case MSG_OSD_PG_TRIM:
     case MSG_OSD_PG_REMOVE:
@@ -2195,6 +1980,8 @@ private:
     case MSG_OSD_PG_UPDATE_LOG_MISSING_REPLY:
     case MSG_OSD_PG_RECOVERY_DELETE:
     case MSG_OSD_PG_RECOVERY_DELETE_REPLY:
+    case MSG_OSD_PG_LEASE:
+    case MSG_OSD_PG_LEASE_ACK:
       return true;
     default:
       return false;
@@ -2202,54 +1989,19 @@ private:
   }
   void ms_fast_dispatch(Message *m) override;
   bool ms_dispatch(Message *m) override;
-  bool ms_get_authorizer(int dest_type, AuthAuthorizer **authorizer) override;
   void ms_handle_connect(Connection *con) override;
   void ms_handle_fast_connect(Connection *con) override;
   void ms_handle_fast_accept(Connection *con) override;
   int ms_handle_authentication(Connection *con) override;
-  KeyStore *ms_get_auth1_authorizer_keystore() override;
   bool ms_handle_reset(Connection *con) override;
   void ms_handle_remote_reset(Connection *con) override {}
   bool ms_handle_refused(Connection *con) override;
 
-  io_queue get_io_queue() const {
-    if (cct->_conf->osd_op_queue == "debug_random") {
-      static io_queue index_lookup[] = { io_queue::prioritized,
-                                        io_queue::weightedpriority,
-                                        io_queue::mclock_opclass,
-                                        io_queue::mclock_client };
-      srand(time(NULL));
-      unsigned which = rand() % (sizeof(index_lookup) / sizeof(index_lookup[0]));
-      return index_lookup[which];
-    } else if (cct->_conf->osd_op_queue == "prioritized") {
-      return io_queue::prioritized;
-    } else if (cct->_conf->osd_op_queue == "mclock_opclass") {
-      return io_queue::mclock_opclass;
-    } else if (cct->_conf->osd_op_queue == "mclock_client") {
-      return io_queue::mclock_client;
-    } else {
-      // default / catch-all is 'wpq'
-      return io_queue::weightedpriority;
-    }
-  }
-
-  unsigned int get_io_prio_cut() const {
-    if (cct->_conf->osd_op_queue_cut_off == "debug_random") {
-      srand(time(NULL));
-      return (rand() % 2 < 1) ? CEPH_MSG_PRIO_HIGH : CEPH_MSG_PRIO_LOW;
-    } else if (cct->_conf->osd_op_queue_cut_off == "high") {
-      return CEPH_MSG_PRIO_HIGH;
-    } else {
-      // default / catch-all is 'low'
-      return CEPH_MSG_PRIO_LOW;
-    }
-  }
-
  public:
   /* internal and external can point to the same messenger, they will still
    * be cleaned up properly*/
   OSD(CephContext *cct_,
-      ObjectStore *store_,
+      std::unique_ptr<ObjectStore> store_,
       int id,
       Messenger *internal,
       Messenger *external,
@@ -2258,15 +2010,20 @@ private:
       Messenger *hb_front_server,
       Messenger *hb_back_server,
       Messenger *osdc_messenger,
-      MonClient *mc, const std::string &dev, const std::string &jdev);
+      MonClient *mc, const std::string &dev, const std::string &jdev,
+      ceph::async::io_context_pool& poolctx);
   ~OSD() override;
 
   // static bits
-  static int mkfs(CephContext *cct, ObjectStore *store, uuid_d fsid, int whoami);
-
-  /* remove any non-user xattrs from a map of them */
-  void filter_xattrs(map<string, bufferptr>& attrs) {
-    for (map<string, bufferptr>::iterator iter = attrs.begin();
+  static int mkfs(CephContext *cct,
+                 std::unique_ptr<ObjectStore> store,
+                 uuid_d fsid,
+                 int whoami,
+                 std::string osdspec_affinity);
+
+  /* remove any non-user xattrs from a std::map of them */
+  void filter_xattrs(std::map<std::string, ceph::buffer::ptr>& attrs) {
+    for (std::map<std::string, ceph::buffer::ptr>::iterator iter = attrs.begin();
         iter != attrs.end();
         ) {
       if (('_' != iter->first.at(0)) || (iter->first.size() == 1))
@@ -2276,36 +2033,49 @@ private:
   }
 
 private:
-  int mon_cmd_maybe_osd_create(string &cmd);
+  int mon_cmd_maybe_osd_create(std::string &cmd);
   int update_crush_device_class();
   int update_crush_location();
 
   static int write_meta(CephContext *cct,
                        ObjectStore *store,
-                       uuid_d& cluster_fsid, uuid_d& osd_fsid, int whoami);
+                       uuid_d& cluster_fsid, uuid_d& osd_fsid, int whoami, std::string& osdspec_affinity);
 
-  void handle_scrub(struct MOSDScrub *m);
-  void handle_fast_scrub(struct MOSDScrub2 *m);
+  void handle_scrub(class MOSDScrub *m);
+  void handle_fast_scrub(class MOSDScrub2 *m);
   void handle_osd_ping(class MOSDPing *m);
 
-  int init_op_flags(OpRequestRef& op);
-
+  size_t get_num_cache_shards();
   int get_num_op_shards();
   int get_num_op_threads();
 
   float get_osd_recovery_sleep();
   float get_osd_delete_sleep();
-
-  void probe_smart(const string& devid, ostream& ss);
+  float get_osd_snap_trim_sleep();
+
+  int get_recovery_max_active();
+  void maybe_override_max_osd_capacity_for_qos();
+  bool maybe_override_options_for_qos();
+  int run_osd_bench_test(int64_t count,
+                         int64_t bsize,
+                         int64_t osize,
+                         int64_t onum,
+                         double *elapsed,
+                         std::ostream& ss);
+  int mon_cmd_set_config(const std::string &key, const std::string &val);
+  bool unsupported_objstore_for_qos();
+
+  void scrub_purged_snaps();
+  void probe_smart(const std::string& devid, std::ostream& ss);
 
 public:
   static int peek_meta(ObjectStore *store,
-                      string *magic,
+                      std::string *magic,
                       uuid_d *cluster_fsid,
                       uuid_d *osd_fsid,
                       int *whoami,
-                      int *min_osd_release);
-  
+                      ceph_release_t *min_osd_release);
+
 
   // startup/shutdown
   int pre_init();
@@ -2328,20 +2098,15 @@ public:
   friend class OSDService;
 
 private:
-  void set_perf_queries(
-      const std::map<OSDPerfMetricQuery, OSDPerfMetricLimits> &queries);
-  void get_perf_reports(
-      std::map<OSDPerfMetricQuery, OSDPerfMetricReport> *reports);
+  void set_perf_queries(const ConfigPayload &config_payload);
+  MetricPayload get_perf_reports();
 
-  Mutex m_perf_queries_lock = {"OSD::m_perf_queries_lock"};
+  ceph::mutex m_perf_queries_lock = ceph::make_mutex("OSD::m_perf_queries_lock");
   std::list<OSDPerfMetricQuery> m_perf_queries;
   std::map<OSDPerfMetricQuery, OSDPerfMetricLimits> m_perf_limits;
 };
 
 
-std::ostream& operator<<(std::ostream& out, const io_queue& q);
-
-
 //compatibility of the executable
 extern const CompatSet::Feature ceph_osd_feature_compat[];
 extern const CompatSet::Feature ceph_osd_feature_ro_compat[];