]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/mon/Paxos.cc
update sources to ceph Nautilus 14.2.1
[ceph.git] / ceph / src / mon / Paxos.cc
index 3ad444c76fb514e76d6e50bb3158e06bc023158f..5d07457fdbdecb5d8a331a914c8f3e6970b1c50c 100644 (file)
@@ -19,7 +19,7 @@
 
 #include "mon/mon_types.h"
 #include "common/config.h"
-#include "include/assert.h"
+#include "include/ceph_assert.h"
 #include "include/stringify.h"
 #include "common/Timer.h"
 #include "messages/PaxosServiceMessage.h"
@@ -60,8 +60,8 @@ void Paxos::read_and_prepare_transactions(MonitorDBStore::TransactionRef tx,
     dout(30) << __func__ << " apply version " << v << dendl;
     bufferlist bl;
     int err = get_store()->get(get_name(), v, bl);
-    assert(err == 0);
-    assert(bl.length());
+    ceph_assert(err == 0);
+    ceph_assert(bl.length());
     decode_append_transaction(tx, bl);
   }
   dout(15) << __func__ << " total versions " << (last-first) << dendl;
@@ -80,7 +80,7 @@ void Paxos::init()
           << " first_committed: " << first_committed << dendl;
 
   dout(10) << "init" << dendl;
-  assert(is_consistent());
+  ceph_assert(is_consistent());
 }
 
 void Paxos::init_logger()
@@ -98,17 +98,17 @@ void Paxos::init_logger()
   pcb.add_time_avg(l_paxos_refresh_latency, "refresh_latency", "Refresh latency");
   pcb.add_u64_counter(l_paxos_begin, "begin", "Started and handled begins");
   pcb.add_u64_avg(l_paxos_begin_keys, "begin_keys", "Keys in transaction on begin");
-  pcb.add_u64_avg(l_paxos_begin_bytes, "begin_bytes", "Data in transaction on begin", NULL, 0, unit_t(BYTES));
+  pcb.add_u64_avg(l_paxos_begin_bytes, "begin_bytes", "Data in transaction on begin", NULL, 0, unit_t(UNIT_BYTES));
   pcb.add_time_avg(l_paxos_begin_latency, "begin_latency", "Latency of begin operation");
   pcb.add_u64_counter(l_paxos_commit, "commit",
       "Commits", "cmt");
   pcb.add_u64_avg(l_paxos_commit_keys, "commit_keys", "Keys in transaction on commit");
-  pcb.add_u64_avg(l_paxos_commit_bytes, "commit_bytes", "Data in transaction on commit", NULL, 0, unit_t(BYTES));
+  pcb.add_u64_avg(l_paxos_commit_bytes, "commit_bytes", "Data in transaction on commit", NULL, 0, unit_t(UNIT_BYTES));
   pcb.add_time_avg(l_paxos_commit_latency, "commit_latency",
       "Commit latency", "clat");
   pcb.add_u64_counter(l_paxos_collect, "collect", "Peon collects");
   pcb.add_u64_avg(l_paxos_collect_keys, "collect_keys", "Keys in transaction on peon collect");
-  pcb.add_u64_avg(l_paxos_collect_bytes, "collect_bytes", "Data in transaction on peon collect", NULL, 0, unit_t(BYTES));
+  pcb.add_u64_avg(l_paxos_collect_bytes, "collect_bytes", "Data in transaction on peon collect", NULL, 0, unit_t(UNIT_BYTES));
   pcb.add_time_avg(l_paxos_collect_latency, "collect_latency", "Peon collect latency");
   pcb.add_u64_counter(l_paxos_collect_uncommitted, "collect_uncommitted", "Uncommitted values in started and handled collects");
   pcb.add_u64_counter(l_paxos_collect_timeout, "collect_timeout", "Collect timeouts");
@@ -117,11 +117,11 @@ void Paxos::init_logger()
   pcb.add_u64_counter(l_paxos_lease_timeout, "lease_timeout", "Lease timeouts");
   pcb.add_u64_counter(l_paxos_store_state, "store_state", "Store a shared state on disk");
   pcb.add_u64_avg(l_paxos_store_state_keys, "store_state_keys", "Keys in transaction in stored state");
-  pcb.add_u64_avg(l_paxos_store_state_bytes, "store_state_bytes", "Data in transaction in stored state", NULL, 0, unit_t(BYTES));
+  pcb.add_u64_avg(l_paxos_store_state_bytes, "store_state_bytes", "Data in transaction in stored state", NULL, 0, unit_t(UNIT_BYTES));
   pcb.add_time_avg(l_paxos_store_state_latency, "store_state_latency", "Storing state latency");
   pcb.add_u64_counter(l_paxos_share_state, "share_state", "Sharings of state");
   pcb.add_u64_avg(l_paxos_share_state_keys, "share_state_keys", "Keys in shared state");
-  pcb.add_u64_avg(l_paxos_share_state_bytes, "share_state_bytes", "Data in shared state", NULL, 0, unit_t(BYTES));
+  pcb.add_u64_avg(l_paxos_share_state_bytes, "share_state_bytes", "Data in shared state", NULL, 0, unit_t(UNIT_BYTES));
   pcb.add_u64_counter(l_paxos_new_pn, "new_pn", "New proposal number queries");
   pcb.add_time_avg(l_paxos_new_pn_latency, "new_pn_latency", "New proposal number getting latency");
   logger = pcb.create_perf_counters();
@@ -147,7 +147,7 @@ void Paxos::collect(version_t oldpn)
 {
   // we're recoverying, it seems!
   state = STATE_RECOVERING;
-  assert(mon->is_leader());
+  ceph_assert(mon->is_leader());
 
   // reset the number of lasts received
   uncommitted_v = 0;
@@ -170,7 +170,7 @@ void Paxos::collect(version_t oldpn)
     uncommitted_v = last_committed+1;
 
     get_store()->get(get_name(), last_committed+1, uncommitted_value);
-    assert(uncommitted_value.length());
+    ceph_assert(uncommitted_value.length());
     dout(10) << "learned uncommitted " << (last_committed+1)
             << " pn " << uncommitted_pn
             << " (" << uncommitted_value.length() << " bytes) from myself" 
@@ -180,7 +180,7 @@ void Paxos::collect(version_t oldpn)
   }
 
   // pick new pn
-  accepted_pn = get_new_proposal_number(MAX(accepted_pn, oldpn));
+  accepted_pn = get_new_proposal_number(std::max(accepted_pn, oldpn));
   accepted_pn_from = last_committed;
   num_last = 1;
   dout(10) << "collect with pn " << accepted_pn << dendl;
@@ -196,13 +196,13 @@ void Paxos::collect(version_t oldpn)
     collect->last_committed = last_committed;
     collect->first_committed = first_committed;
     collect->pn = accepted_pn;
-    mon->messenger->send_message(collect, mon->monmap->get_inst(*p));
+    mon->send_mon_message(collect, *p);
   }
 
   // set timeout event
   collect_timeout_event = mon->timer.add_event_after(
-    g_conf->mon_accept_timeout_factor *
-    g_conf->mon_lease,
+    g_conf()->mon_accept_timeout_factor *
+    g_conf()->mon_lease,
     new C_MonContext(mon, [this](int r) {
        if (r == -ECANCELED)
          return;
@@ -220,7 +220,7 @@ void Paxos::handle_collect(MonOpRequestRef op)
   MMonPaxos *collect = static_cast<MMonPaxos*>(op->get_req());
   dout(10) << "handle_collect " << *collect << dendl;
 
-  assert(mon->is_peon()); // mon epoch filter should catch strays
+  ceph_assert(mon->is_peon()); // mon epoch filter should catch strays
 
   // we're recoverying, it seems!
   state = STATE_RECOVERING;
@@ -266,12 +266,12 @@ void Paxos::handle_collect(MonOpRequestRef op)
     logger->inc(l_paxos_collect);
     logger->inc(l_paxos_collect_keys, t->get_keys());
     logger->inc(l_paxos_collect_bytes, t->get_bytes());
-    utime_t start = ceph_clock_now();
 
+    auto start = ceph::coarse_mono_clock::now();
     get_store()->apply_transaction(t);
+    auto end = ceph::coarse_mono_clock::now();
 
-    utime_t end = ceph_clock_now();
-    logger->tinc(l_paxos_collect_latency, end - start);
+    logger->tinc(l_paxos_collect_latency, to_timespan(end - start));
   } else {
     // don't accept!
     dout(10) << "NOT accepting pn " << collect->pn << " from " << collect->pn_from
@@ -291,7 +291,7 @@ void Paxos::handle_collect(MonOpRequestRef op)
   if (collect->last_committed <= last_committed &&
       get_store()->exists(get_name(), last_committed+1)) {
     get_store()->get(get_name(), last_committed+1, bl);
-    assert(bl.length() > 0);
+    ceph_assert(bl.length() > 0);
     dout(10) << " sharing our accepted but uncommitted value for " 
             << last_committed+1 << " (" << bl.length() << " bytes)" << dendl;
     last->values[last_committed+1] = bl;
@@ -330,7 +330,7 @@ void Paxos::handle_collect(MonOpRequestRef op)
 void Paxos::share_state(MMonPaxos *m, version_t peer_first_committed,
                        version_t peer_last_committed)
 {
-  assert(peer_last_committed < last_committed);
+  ceph_assert(peer_last_committed < last_committed);
 
   dout(10) << "share_state peer has fc " << peer_first_committed 
           << " lc " << peer_last_committed << dendl;
@@ -341,7 +341,7 @@ void Paxos::share_state(MMonPaxos *m, version_t peer_first_committed,
   for ( ; v <= last_committed; v++) {
     if (get_store()->exists(get_name(), v)) {
       get_store()->get(get_name(), v, m->values[v]);
-      assert(m->values[v].length());
+      ceph_assert(m->values[v].length());
       dout(10) << " sharing " << v << " ("
               << m->values[v].length() << " bytes)" << dendl;
       bytes += m->values[v].length() + 16;  // paxos_ + 10 digits = 16
@@ -433,12 +433,12 @@ bool Paxos::store_state(MMonPaxos *m)
     logger->inc(l_paxos_store_state);
     logger->inc(l_paxos_store_state_bytes, t->get_bytes());
     logger->inc(l_paxos_store_state_keys, t->get_keys());
-    utime_t start = ceph_clock_now();
 
+    auto start = ceph::coarse_mono_clock::now();
     get_store()->apply_transaction(t);
+    auto end = ceph::coarse_mono_clock::now();
 
-    utime_t end = ceph_clock_now();
-    logger->tinc(l_paxos_store_state_latency, end - start);
+    logger->tinc(l_paxos_store_state_latency, to_timespan(end-start));
 
     // refresh first_committed; this txn may have trimmed.
     first_committed = get_store()->get(get_name(), "first_committed");
@@ -453,7 +453,7 @@ bool Paxos::store_state(MMonPaxos *m)
 void Paxos::_sanity_check_store()
 {
   version_t lc = get_store()->get(get_name(), "last_committed");
-  assert(lc == last_committed);
+  ceph_assert(lc == last_committed);
 }
 
 
@@ -488,12 +488,12 @@ void Paxos::handle_last(MonOpRequestRef op)
     return;
   }
 
-  assert(g_conf->paxos_kill_at != 1);
+  ceph_assert(g_conf()->paxos_kill_at != 1);
 
   // store any committed values if any are specified in the message
   need_refresh = store_state(last);
 
-  assert(g_conf->paxos_kill_at != 2);
+  ceph_assert(g_conf()->paxos_kill_at != 2);
 
   // is everyone contiguous and up to date?
   for (map<int,version_t>::iterator p = peer_last_committed.begin();
@@ -516,7 +516,7 @@ void Paxos::handle_last(MonOpRequestRef op)
                                        MMonPaxos::OP_COMMIT,
                                        ceph_clock_now());
       share_state(commit, peer_first_committed[p->first], p->second);
-      mon->messenger->send_message(commit, mon->monmap->get_inst(p->first));
+      mon->send_mon_message(commit, p->first);
     }
   }
 
@@ -597,7 +597,7 @@ void Paxos::collect_timeout()
   dout(1) << "collect timeout, calling fresh election" << dendl;
   collect_timeout_event = 0;
   logger->inc(l_paxos_collect_timeout);
-  assert(mon->is_leader());
+  ceph_assert(mon->is_leader());
   mon->bootstrap();
 }
 
@@ -609,15 +609,15 @@ void Paxos::begin(bufferlist& v)
           << v.length() << " bytes"
           << dendl;
 
-  assert(mon->is_leader());
-  assert(is_updating() || is_updating_previous());
+  ceph_assert(mon->is_leader());
+  ceph_assert(is_updating() || is_updating_previous());
 
   // we must already have a majority for this to work.
-  assert(mon->get_quorum().size() == 1 ||
+  ceph_assert(mon->get_quorum().size() == 1 ||
         num_last > (unsigned)mon->monmap->size()/2);
   
   // and no value, yet.
-  assert(new_value.length() == 0);
+  ceph_assert(new_value.length() == 0);
 
   // accept it ourselves
   accepted.clear();
@@ -650,7 +650,7 @@ void Paxos::begin(bufferlist& v)
   t->dump(&f);
   f.flush(*_dout);
   auto debug_tx(std::make_shared<MonitorDBStore::Transaction>());
-  bufferlist::iterator new_value_it = new_value.begin();
+  auto new_value_it = new_value.cbegin();
   debug_tx->decode(new_value_it);
   debug_tx->dump(&f);
   *_dout << "\nbl dump:\n";
@@ -660,14 +660,14 @@ void Paxos::begin(bufferlist& v)
   logger->inc(l_paxos_begin);
   logger->inc(l_paxos_begin_keys, t->get_keys());
   logger->inc(l_paxos_begin_bytes, t->get_bytes());
-  utime_t start = ceph_clock_now();
 
+  auto start = ceph::coarse_mono_clock::now();
   get_store()->apply_transaction(t);
+  auto end = ceph::coarse_mono_clock::now();
 
-  utime_t end = ceph_clock_now();
-  logger->tinc(l_paxos_begin_latency, end - start);
+  logger->tinc(l_paxos_begin_latency, to_timespan(end - start));
 
-  assert(g_conf->paxos_kill_at != 3);
+  ceph_assert(g_conf()->paxos_kill_at != 3);
 
   if (mon->get_quorum().size() == 1) {
     // we're alone, take it easy
@@ -688,12 +688,12 @@ void Paxos::begin(bufferlist& v)
     begin->last_committed = last_committed;
     begin->pn = accepted_pn;
     
-    mon->messenger->send_message(begin, mon->monmap->get_inst(*p));
+    mon->send_mon_message(begin, *p);
   }
 
   // set timeout event
   accept_timeout_event = mon->timer.add_event_after(
-    g_conf->mon_accept_timeout_factor * g_conf->mon_lease,
+    g_conf()->mon_accept_timeout_factor * g_conf()->mon_lease,
     new C_MonContext(mon, [this](int r) {
        if (r == -ECANCELED)
          return;
@@ -714,10 +714,10 @@ void Paxos::handle_begin(MonOpRequestRef op)
     op->mark_paxos_event("have higher pn, ignore");
     return;
   }
-  assert(begin->pn == accepted_pn);
-  assert(begin->last_committed == last_committed);
+  ceph_assert(begin->pn == accepted_pn);
+  ceph_assert(begin->last_committed == last_committed);
   
-  assert(g_conf->paxos_kill_at != 4);
+  ceph_assert(g_conf()->paxos_kill_at != 4);
 
   logger->inc(l_paxos_begin);
 
@@ -744,14 +744,14 @@ void Paxos::handle_begin(MonOpRequestRef op)
   *_dout << dendl;
 
   logger->inc(l_paxos_begin_bytes, t->get_bytes());
-  utime_t start = ceph_clock_now();
 
+  auto start = ceph::coarse_mono_clock::now();
   get_store()->apply_transaction(t);
+  auto end = ceph::coarse_mono_clock::now();
 
-  utime_t end = ceph_clock_now();
-  logger->tinc(l_paxos_begin_latency, end - start);
+  logger->tinc(l_paxos_begin_latency, to_timespan(end - start));
 
-  assert(g_conf->paxos_kill_at != 5);
+  ceph_assert(g_conf()->paxos_kill_at != 5);
 
   // reply
   MMonPaxos *accept = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_ACCEPT,
@@ -781,15 +781,15 @@ void Paxos::handle_accept(MonOpRequestRef op)
     op->mark_paxos_event("old round, ignore");
     return;
   }
-  assert(accept->last_committed == last_committed ||   // not committed
+  ceph_assert(accept->last_committed == last_committed ||   // not committed
         accept->last_committed == last_committed-1);  // committed
 
-  assert(is_updating() || is_updating_previous());
-  assert(accepted.count(from) == 0);
+  ceph_assert(is_updating() || is_updating_previous());
+  ceph_assert(accepted.count(from) == 0);
   accepted.insert(from);
   dout(10) << " now " << accepted << " have accepted" << dendl;
 
-  assert(g_conf->paxos_kill_at != 6);
+  ceph_assert(g_conf()->paxos_kill_at != 6);
 
   // only commit (and expose committed state) when we get *all* quorum
   // members to accept.  otherwise, they may still be sharing the now
@@ -808,8 +808,8 @@ void Paxos::accept_timeout()
 {
   dout(1) << "accept timeout, calling fresh election" << dendl;
   accept_timeout_event = 0;
-  assert(mon->is_leader());
-  assert(is_updating() || is_updating_previous() || is_writing() ||
+  ceph_assert(mon->is_leader());
+  ceph_assert(is_updating() || is_updating_previous() || is_writing() ||
         is_writing_previous());
   logger->inc(l_paxos_accept_timeout);
   mon->bootstrap();
@@ -819,8 +819,8 @@ struct C_Committed : public Context {
   Paxos *paxos;
   explicit C_Committed(Paxos *p) : paxos(p) {}
   void finish(int r) override {
-    assert(r >= 0);
-    Mutex::Locker l(paxos->mon->lock);
+    ceph_assert(r >= 0);
+    std::lock_guard l(paxos->mon->lock);
     if (paxos->is_shutdown()) {
       paxos->abort_commit();
       return;
@@ -831,7 +831,7 @@ struct C_Committed : public Context {
 
 void Paxos::abort_commit()
 {
-  assert(commits_started > 0);
+  ceph_assert(commits_started > 0);
   --commits_started;
   if (commits_started == 0)
     shutdown_cond.Signal();
@@ -841,7 +841,7 @@ void Paxos::commit_start()
 {
   dout(10) << __func__ << " " << (last_committed+1) << dendl;
 
-  assert(g_conf->paxos_kill_at != 7);
+  ceph_assert(g_conf()->paxos_kill_at != 7);
 
   auto t(std::make_shared<MonitorDBStore::Transaction>());
 
@@ -886,7 +886,7 @@ void Paxos::commit_finish()
   utime_t end = ceph_clock_now();
   logger->tinc(l_paxos_commit_latency, end - commit_start_stamp);
 
-  assert(g_conf->paxos_kill_at != 8);
+  ceph_assert(g_conf()->paxos_kill_at != 8);
 
   // cancel lease - it was for the old value.
   //  (this would only happen if message layer lost the 'begin', but
@@ -914,20 +914,21 @@ void Paxos::commit_finish()
     commit->pn = accepted_pn;
     commit->last_committed = last_committed;
 
-    mon->messenger->send_message(commit, mon->monmap->get_inst(*p));
+    mon->send_mon_message(commit, *p);
   }
 
-  assert(g_conf->paxos_kill_at != 9);
+  ceph_assert(g_conf()->paxos_kill_at != 9);
 
   // get ready for a new round.
   new_value.clear();
 
   // WRITING -> REFRESH
-  // among other things, this lets do_refresh() -> mon->bootstrap() know
-  // it doesn't need to flush the store queue
-  assert(is_writing() || is_writing_previous());
+  // among other things, this lets do_refresh() -> mon->bootstrap() ->
+  // wait_for_paxos_write() know that it doesn't need to flush the store
+  // queue. and it should not, as we are in the async completion thread now!
+  ceph_assert(is_writing() || is_writing_previous());
   state = STATE_REFRESH;
-  assert(commits_started > 0);
+  ceph_assert(commits_started > 0);
   --commits_started;
 
   if (do_refresh()) {
@@ -936,9 +937,7 @@ void Paxos::commit_finish()
       extend_lease();
     }
 
-    finish_contexts(g_ceph_context, waiting_for_commit);
-
-    assert(g_conf->paxos_kill_at != 10);
+    ceph_assert(g_conf()->paxos_kill_at != 10);
 
     finish_round();
   }
@@ -962,22 +961,20 @@ void Paxos::handle_commit(MonOpRequestRef op)
   op->mark_paxos_event("store_state");
   store_state(commit);
 
-  if (do_refresh()) {
-    finish_contexts(g_ceph_context, waiting_for_commit);
-  }
+  (void)do_refresh();
 }
 
 void Paxos::extend_lease()
 {
-  assert(mon->is_leader());
+  ceph_assert(mon->is_leader());
   //assert(is_active());
 
   lease_expire = ceph_clock_now();
-  lease_expire += g_conf->mon_lease;
+  lease_expire += g_conf()->mon_lease;
   acked_lease.clear();
   acked_lease.insert(mon->rank);
 
-  dout(7) << "extend_lease now+" << g_conf->mon_lease 
+  dout(7) << "extend_lease now+" << g_conf()->mon_lease
          << " (" << lease_expire << ")" << dendl;
 
   // bcast
@@ -990,14 +987,14 @@ void Paxos::extend_lease()
     lease->last_committed = last_committed;
     lease->lease_timestamp = lease_expire;
     lease->first_committed = first_committed;
-    mon->messenger->send_message(lease, mon->monmap->get_inst(*p));
+    mon->send_mon_message(lease, *p);
   }
 
   // set timeout event.
   //  if old timeout is still in place, leave it.
   if (!lease_ack_timeout_event) {
     lease_ack_timeout_event = mon->timer.add_event_after(
-      g_conf->mon_lease_ack_timeout_factor * g_conf->mon_lease,
+      g_conf()->mon_lease_ack_timeout_factor * g_conf()->mon_lease,
       new C_MonContext(mon, [this](int r) {
          if (r == -ECANCELED)
            return;
@@ -1007,8 +1004,8 @@ void Paxos::extend_lease()
 
   // set renew event
   utime_t at = lease_expire;
-  at -= g_conf->mon_lease;
-  at += g_conf->mon_lease_renew_interval_factor * g_conf->mon_lease;
+  at -= g_conf()->mon_lease;
+  at += g_conf()->mon_lease_renew_interval_factor * g_conf()->mon_lease;
   lease_renew_event = mon->timer.add_event_at(
     at, new C_MonContext(mon, [this](int r) {
        if (r == -ECANCELED)
@@ -1022,10 +1019,10 @@ void Paxos::warn_on_future_time(utime_t t, entity_name_t from)
   utime_t now = ceph_clock_now();
   if (t > now) {
     utime_t diff = t - now;
-    if (diff > g_conf->mon_clock_drift_allowed) {
+    if (diff > g_conf()->mon_clock_drift_allowed) {
       utime_t warn_diff = now - last_clock_drift_warn;
       if (warn_diff >
-         pow(g_conf->mon_clock_drift_warn_backoff, clock_drift_warned)) {
+         pow(g_conf()->mon_clock_drift_warn_backoff, clock_drift_warned)) {
        mon->clog->warn() << "message from " << from << " was stamped " << diff
                         << "s in the future, clocks not synchronized";
        last_clock_drift_warn = ceph_clock_now();
@@ -1040,14 +1037,13 @@ bool Paxos::do_refresh()
 {
   bool need_bootstrap = false;
 
-  utime_t start = ceph_clock_now();
-
   // make sure we have the latest state loaded up
+  auto start = ceph::coarse_mono_clock::now();
   mon->refresh_from_paxos(&need_bootstrap);
+  auto end = ceph::coarse_mono_clock::now();
 
-  utime_t end = ceph_clock_now();
   logger->inc(l_paxos_refresh);
-  logger->tinc(l_paxos_refresh_latency, end - start);
+  logger->tinc(l_paxos_refresh_latency, to_timespan(end - start));
 
   if (need_bootstrap) {
     dout(10) << " doing requested bootstrap" << dendl;
@@ -1061,8 +1057,8 @@ bool Paxos::do_refresh()
 void Paxos::commit_proposal()
 {
   dout(10) << __func__ << dendl;
-  assert(mon->is_leader());
-  assert(is_refresh());
+  ceph_assert(mon->is_leader());
+  ceph_assert(is_refresh());
 
   finish_contexts(g_ceph_context, committing_finishers);
 }
@@ -1070,7 +1066,7 @@ void Paxos::commit_proposal()
 void Paxos::finish_round()
 {
   dout(10) << __func__ << dendl;
-  assert(mon->is_leader());
+  ceph_assert(mon->is_leader());
 
   // ok, now go active!
   state = STATE_ACTIVE;
@@ -1132,7 +1128,7 @@ void Paxos::handle_lease(MonOpRequestRef op)
   ack->last_committed = last_committed;
   ack->first_committed = first_committed;
   ack->lease_timestamp = ceph_clock_now();
-  ::encode(mon->session_map.feature_map, ack->feature_map);
+  encode(mon->session_map.feature_map, ack->feature_map);
   lease->get_connection()->send_message(ack);
 
   // (re)set timeout event.
@@ -1157,9 +1153,9 @@ void Paxos::handle_lease_ack(MonOpRequestRef op)
   } else if (acked_lease.count(from) == 0) {
     acked_lease.insert(from);
     if (ack->feature_map.length()) {
-      auto p = ack->feature_map.begin();
+      auto p = ack->feature_map.cbegin();
       FeatureMap& t = mon->quorum_feature_map[from];
-      ::decode(t, p);
+      decode(t, p);
     }
     if (acked_lease == mon->get_quorum()) {
       // yay!
@@ -1186,8 +1182,8 @@ void Paxos::handle_lease_ack(MonOpRequestRef op)
 void Paxos::lease_ack_timeout()
 {
   dout(1) << "lease_ack_timeout -- calling new election" << dendl;
-  assert(mon->is_leader());
-  assert(is_active());
+  ceph_assert(mon->is_leader());
+  ceph_assert(is_active());
   logger->inc(l_paxos_lease_ack_timeout);
   lease_ack_timeout_event = 0;
   mon->bootstrap();
@@ -1199,7 +1195,7 @@ void Paxos::reset_lease_timeout()
   if (lease_timeout_event)
     mon->timer.cancel_event(lease_timeout_event);
   lease_timeout_event = mon->timer.add_event_after(
-    g_conf->mon_lease_ack_timeout_factor * g_conf->mon_lease,
+    g_conf()->mon_lease_ack_timeout_factor * g_conf()->mon_lease,
     new C_MonContext(mon, [this](int r) {
        if (r == -ECANCELED)
          return;
@@ -1210,7 +1206,7 @@ void Paxos::reset_lease_timeout()
 void Paxos::lease_timeout()
 {
   dout(1) << "lease_timeout -- calling new election" << dendl;
-  assert(mon->is_peon());
+  ceph_assert(mon->is_peon());
   logger->inc(l_paxos_lease_timeout);
   lease_timeout_event = 0;
   mon->bootstrap();
@@ -1228,9 +1224,9 @@ void Paxos::lease_renew_timeout()
  */
 void Paxos::trim()
 {
-  assert(should_trim());
-  version_t end = MIN(get_version() - g_conf->paxos_min,
-                     get_first_committed() + g_conf->paxos_trim_max);
+  ceph_assert(should_trim());
+  version_t end = std::min(get_version() - g_conf()->paxos_min,
+                     get_first_committed() + g_conf()->paxos_trim_max);
 
   if (first_committed >= end)
     return;
@@ -1244,7 +1240,7 @@ void Paxos::trim()
     t->erase(get_name(), v);
   }
   t->put(get_name(), "first_committed", end);
-  if (g_conf->mon_compact_on_trim) {
+  if (g_conf()->mon_compact_on_trim) {
     dout(10) << " compacting trimmed range" << dendl;
     t->compact_range(get_name(), stringify(first_committed - 1), stringify(end));
   }
@@ -1278,12 +1274,12 @@ version_t Paxos::get_new_proposal_number(version_t gt)
   *_dout << dendl;
 
   logger->inc(l_paxos_new_pn);
-  utime_t start = ceph_clock_now();
 
+  auto start = ceph::coarse_mono_clock::now();
   get_store()->apply_transaction(t);
+  auto end = ceph::coarse_mono_clock::now();
 
-  utime_t end = ceph_clock_now();
-  logger->tinc(l_paxos_new_pn_latency, end - start);
+  logger->tinc(l_paxos_new_pn_latency, to_timespan(end - start));
 
   dout(10) << "get_new_proposal_number = " << last_pn << dendl;
   return last_pn;
@@ -1330,7 +1326,6 @@ void Paxos::shutdown()
     shutdown_cond.Wait(mon->lock);
 
   finish_contexts(g_ceph_context, waiting_for_writeable, -ECANCELED);
-  finish_contexts(g_ceph_context, waiting_for_commit, -ECANCELED);
   finish_contexts(g_ceph_context, waiting_for_readable, -ECANCELED);
   finish_contexts(g_ceph_context, waiting_for_active, -ECANCELED);
   finish_contexts(g_ceph_context, pending_finishers, -ECANCELED);
@@ -1381,7 +1376,6 @@ void Paxos::peon_init()
 
   // no chance to write now!
   finish_contexts(g_ceph_context, waiting_for_writeable, -EAGAIN);
-  finish_contexts(g_ceph_context, waiting_for_commit, -EAGAIN);
   finish_contexts(g_ceph_context, pending_finishers, -EAGAIN);
   finish_contexts(g_ceph_context, committing_finishers, -EAGAIN);
 
@@ -1408,7 +1402,6 @@ void Paxos::restart()
 
   finish_contexts(g_ceph_context, committing_finishers, -EAGAIN);
   finish_contexts(g_ceph_context, pending_finishers, -EAGAIN);
-  finish_contexts(g_ceph_context, waiting_for_commit, -EAGAIN);
   finish_contexts(g_ceph_context, waiting_for_active, -EAGAIN);
 
   logger->inc(l_paxos_restart);
@@ -1417,7 +1410,7 @@ void Paxos::restart()
 
 void Paxos::dispatch(MonOpRequestRef op)
 {
-  assert(op->is_type_paxos());
+  ceph_assert(op->is_type_paxos());
   op->mark_paxos_event("dispatch");
   PaxosServiceMessage *m = static_cast<PaxosServiceMessage*>(op->get_req());
   // election in progress?
@@ -1427,7 +1420,7 @@ void Paxos::dispatch(MonOpRequestRef op)
   }
 
   // check sanity
-  assert(mon->is_leader() || 
+  ceph_assert(mon->is_leader() || 
         (mon->is_peon() && m->get_source().num() == mon->get_leader()));
   
   switch (m->get_type()) {
@@ -1528,8 +1521,8 @@ bool Paxos::is_writeable()
 
 void Paxos::propose_pending()
 {
-  assert(is_active());
-  assert(pending_proposal);
+  ceph_assert(is_active());
+  ceph_assert(pending_proposal);
 
   cancel_events();
 
@@ -1554,16 +1547,16 @@ void Paxos::propose_pending()
 void Paxos::queue_pending_finisher(Context *onfinished)
 {
   dout(5) << __func__ << " " << onfinished << dendl;
-  assert(onfinished);
+  ceph_assert(onfinished);
   pending_finishers.push_back(onfinished);
 }
 
 MonitorDBStore::TransactionRef Paxos::get_pending_transaction()
 {
-  assert(mon->is_leader());
+  ceph_assert(mon->is_leader());
   if (!pending_proposal) {
     pending_proposal.reset(new MonitorDBStore::Transaction);
-    assert(pending_finishers.empty());
+    ceph_assert(pending_finishers.empty());
   }
   return pending_proposal;
 }