]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/librbd/ManagedLock.cc
import ceph 15.2.10
[ceph.git] / ceph / src / librbd / ManagedLock.cc
index f584d6af5fb5e9518388c087cd9a43be12be83e2..1d3aa2d0581ba0e06228fb517da188d1f6b409ba 100644 (file)
 #include "cls/lock/cls_lock_client.h"
 #include "common/dout.h"
 #include "common/errno.h"
+#include "common/Cond.h"
 #include "common/WorkQueue.h"
 #include "librbd/Utils.h"
 
 #define dout_subsys ceph_subsys_rbd
 #undef dout_prefix
 #define dout_prefix *_dout << "librbd::ManagedLock: " << this << " " \
-                           <<  __func__
+                           <<  __func__ << ": "
 
 namespace librbd {
 
@@ -66,7 +67,7 @@ ManagedLock<I>::ManagedLock(librados::IoCtx &ioctx, ContextWQ *work_queue,
                             const string& oid, Watcher *watcher, Mode mode,
                             bool blacklist_on_break_lock,
                             uint32_t blacklist_expire_seconds)
-  : m_lock(unique_lock_name("librbd::ManagedLock<I>::m_lock", this)),
+  : m_lock(ceph::make_mutex(unique_lock_name("librbd::ManagedLock<I>::m_lock", this))),
     m_ioctx(ioctx), m_cct(reinterpret_cast<CephContext *>(ioctx.cct())),
     m_work_queue(work_queue),
     m_oid(oid),
@@ -79,9 +80,9 @@ ManagedLock<I>::ManagedLock(librados::IoCtx &ioctx, ContextWQ *work_queue,
 
 template <typename I>
 ManagedLock<I>::~ManagedLock() {
-  Mutex::Locker locker(m_lock);
-  assert(m_state == STATE_SHUTDOWN || m_state == STATE_UNLOCKED ||
-         m_state == STATE_UNINITIALIZED);
+  std::lock_guard locker{m_lock};
+  ceph_assert(m_state == STATE_SHUTDOWN || m_state == STATE_UNLOCKED ||
+              m_state == STATE_UNINITIALIZED);
   if (m_state == STATE_UNINITIALIZED) {
     // never initialized -- ensure any in-flight ops are complete
     // since we wouldn't expect shut_down to be invoked
@@ -89,20 +90,20 @@ ManagedLock<I>::~ManagedLock() {
     m_async_op_tracker.wait_for_ops(&ctx);
     ctx.wait();
   }
-  assert(m_async_op_tracker.empty());
+  ceph_assert(m_async_op_tracker.empty());
 }
 
 template <typename I>
 bool ManagedLock<I>::is_lock_owner() const {
-  Mutex::Locker locker(m_lock);
+  std::lock_guard locker{m_lock};
 
   return is_lock_owner(m_lock);
 }
 
 template <typename I>
-bool ManagedLock<I>::is_lock_owner(Mutex &lock) const {
+bool ManagedLock<I>::is_lock_owner(ceph::mutex &lock) const {
 
-  assert(m_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(m_lock));
 
   bool lock_owner;
 
@@ -127,8 +128,18 @@ template <typename I>
 void ManagedLock<I>::shut_down(Context *on_shut_down) {
   ldout(m_cct, 10) << dendl;
 
-  Mutex::Locker locker(m_lock);
-  assert(!is_state_shutdown());
+  std::lock_guard locker{m_lock};
+  ceph_assert(!is_state_shutdown());
+
+  if (m_state == STATE_WAITING_FOR_REGISTER) {
+    // abort stalled acquire lock state
+    ldout(m_cct, 10) << "woke up waiting (re)acquire" << dendl;
+    Action active_action = get_active_action();
+    ceph_assert(active_action == ACTION_TRY_LOCK ||
+                active_action == ACTION_ACQUIRE_LOCK);
+    complete_active_action(STATE_UNLOCKED, -ESHUTDOWN);
+  }
+
   execute_action(ACTION_SHUT_DOWN, on_shut_down);
 }
 
@@ -136,7 +147,7 @@ template <typename I>
 void ManagedLock<I>::acquire_lock(Context *on_acquired) {
   int r = 0;
   {
-    Mutex::Locker locker(m_lock);
+    std::lock_guard locker{m_lock};
     if (is_state_shutdown()) {
       r = -ESHUTDOWN;
     } else if (m_state != STATE_LOCKED || !m_actions_contexts.empty()) {
@@ -155,7 +166,7 @@ template <typename I>
 void ManagedLock<I>::try_acquire_lock(Context *on_acquired) {
   int r = 0;
   {
-    Mutex::Locker locker(m_lock);
+    std::lock_guard locker{m_lock};
     if (is_state_shutdown()) {
       r = -ESHUTDOWN;
     } else if (m_state != STATE_LOCKED || !m_actions_contexts.empty()) {
@@ -174,7 +185,7 @@ template <typename I>
 void ManagedLock<I>::release_lock(Context *on_released) {
   int r = 0;
   {
-    Mutex::Locker locker(m_lock);
+    std::lock_guard locker{m_lock};
     if (is_state_shutdown()) {
       r = -ESHUTDOWN;
     } else if (m_state != STATE_UNLOCKED || !m_actions_contexts.empty()) {
@@ -192,14 +203,14 @@ void ManagedLock<I>::release_lock(Context *on_released) {
 template <typename I>
 void ManagedLock<I>::reacquire_lock(Context *on_reacquired) {
   {
-    Mutex::Locker locker(m_lock);
+    std::lock_guard locker{m_lock};
 
     if (m_state == STATE_WAITING_FOR_REGISTER) {
       // restart the acquire lock process now that watch is valid
-      ldout(m_cct, 10) << ": " << "woke up waiting acquire" << dendl;
+      ldout(m_cct, 10) << "woke up waiting (re)acquire" << dendl;
       Action active_action = get_active_action();
-      assert(active_action == ACTION_TRY_LOCK ||
-             active_action == ACTION_ACQUIRE_LOCK);
+      ceph_assert(active_action == ACTION_TRY_LOCK ||
+                  active_action == ACTION_ACQUIRE_LOCK);
       execute_next_action();
     } else if (!is_state_shutdown() &&
                (m_state == STATE_LOCKED ||
@@ -226,7 +237,7 @@ void ManagedLock<I>::get_locker(managed_lock::Locker *locker,
 
   int r;
   {
-    Mutex::Locker l(m_lock);
+    std::lock_guard l{m_lock};
     if (is_state_shutdown()) {
       r = -ESHUTDOWN;
     } else {
@@ -248,7 +259,7 @@ void ManagedLock<I>::break_lock(const managed_lock::Locker &locker,
 
   int r;
   {
-    Mutex::Locker l(m_lock);
+    std::lock_guard l{m_lock};
     if (is_state_shutdown()) {
       r = -ESHUTDOWN;
     } else if (is_lock_owner(m_lock)) {
@@ -273,7 +284,7 @@ int ManagedLock<I>::assert_header_locked() {
 
   librados::ObjectReadOperation op;
   {
-    Mutex::Locker locker(m_lock);
+    std::lock_guard locker{m_lock};
     rados::cls::lock::assert_locked(&op, RBD_LOCK_NAME,
                                     (m_mode == EXCLUSIVE ? LOCK_EXCLUSIVE :
                                                            LOCK_SHARED),
@@ -360,7 +371,7 @@ bool ManagedLock<I>::is_transition_state() const {
 
 template <typename I>
 void ManagedLock<I>::append_context(Action action, Context *ctx) {
-  assert(m_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(m_lock));
 
   for (auto &action_ctxs : m_actions_contexts) {
     if (action == action_ctxs.first) {
@@ -380,7 +391,7 @@ void ManagedLock<I>::append_context(Action action, Context *ctx) {
 
 template <typename I>
 void ManagedLock<I>::execute_action(Action action, Context *ctx) {
-  assert(m_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(m_lock));
 
   append_context(action, ctx);
   if (!is_transition_state()) {
@@ -390,8 +401,8 @@ void ManagedLock<I>::execute_action(Action action, Context *ctx) {
 
 template <typename I>
 void ManagedLock<I>::execute_next_action() {
-  assert(m_lock.is_locked());
-  assert(!m_actions_contexts.empty());
+  ceph_assert(ceph_mutex_is_locked(m_lock));
+  ceph_assert(!m_actions_contexts.empty());
   switch (get_active_action()) {
   case ACTION_ACQUIRE_LOCK:
   case ACTION_TRY_LOCK:
@@ -407,32 +418,32 @@ void ManagedLock<I>::execute_next_action() {
     send_shutdown();
     break;
   default:
-    assert(false);
+    ceph_abort();
     break;
   }
 }
 
 template <typename I>
 typename ManagedLock<I>::Action ManagedLock<I>::get_active_action() const {
-  assert(m_lock.is_locked());
-  assert(!m_actions_contexts.empty());
+  ceph_assert(ceph_mutex_is_locked(m_lock));
+  ceph_assert(!m_actions_contexts.empty());
   return m_actions_contexts.front().first;
 }
 
 template <typename I>
 void ManagedLock<I>::complete_active_action(State next_state, int r) {
-  assert(m_lock.is_locked());
-  assert(!m_actions_contexts.empty());
+  ceph_assert(ceph_mutex_is_locked(m_lock));
+  ceph_assert(!m_actions_contexts.empty());
 
   ActionContexts action_contexts(std::move(m_actions_contexts.front()));
   m_actions_contexts.pop_front();
   m_state = next_state;
 
-  m_lock.Unlock();
+  m_lock.unlock();
   for (auto ctx : action_contexts.second) {
     ctx->complete(r);
   }
-  m_lock.Lock();
+  m_lock.lock();
 
   if (!is_transition_state() && !m_actions_contexts.empty()) {
     execute_next_action();
@@ -441,33 +452,47 @@ void ManagedLock<I>::complete_active_action(State next_state, int r) {
 
 template <typename I>
 bool ManagedLock<I>::is_state_shutdown() const {
-  assert(m_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(m_lock));
+
+  switch (m_state) {
+  case STATE_PRE_SHUTTING_DOWN:
+  case STATE_SHUTTING_DOWN:
+  case STATE_SHUTDOWN:
+    return true;
+  default:
+    break;
+  }
 
-  return ((m_state == STATE_SHUTDOWN) ||
-          (!m_actions_contexts.empty() &&
-           m_actions_contexts.back().first == ACTION_SHUT_DOWN));
+  return (!m_actions_contexts.empty() &&
+          m_actions_contexts.back().first == ACTION_SHUT_DOWN);
 }
 
 template <typename I>
 void ManagedLock<I>::send_acquire_lock() {
-  assert(m_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(m_lock));
   if (m_state == STATE_LOCKED) {
     complete_active_action(STATE_LOCKED, 0);
     return;
   }
 
   ldout(m_cct, 10) << dendl;
-  m_state = STATE_ACQUIRING;
 
   uint64_t watch_handle = m_watcher->get_watch_handle();
   if (watch_handle == 0) {
     lderr(m_cct) << "watcher not registered - delaying request" << dendl;
     m_state = STATE_WAITING_FOR_REGISTER;
+
+    // shut down might race w/ release/re-acquire of the lock
+    if (is_state_shutdown()) {
+      complete_active_action(STATE_UNLOCKED, -ESHUTDOWN);
+    }
     return;
   }
+
+  m_state = STATE_ACQUIRING;
   m_cookie = encode_lock_cookie(watch_handle);
 
-  m_work_queue->queue(new FunctionContext([this](int r) {
+  m_work_queue->queue(new LambdaContext([this](int r) {
     pre_acquire_lock_handler(create_context_callback<
         ManagedLock<I>, &ManagedLock<I>::handle_pre_acquire_lock>(this));
   }));
@@ -475,7 +500,7 @@ void ManagedLock<I>::send_acquire_lock() {
 
 template <typename I>
 void ManagedLock<I>::handle_pre_acquire_lock(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
   if (r < 0) {
     handle_acquire_lock(r);
@@ -493,20 +518,20 @@ void ManagedLock<I>::handle_pre_acquire_lock(int r) {
 
 template <typename I>
 void ManagedLock<I>::handle_acquire_lock(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
   if (r == -EBUSY || r == -EAGAIN) {
-    ldout(m_cct, 5) << "unable to acquire exclusive lock" << dendl;
+    ldout(m_cct, 5) << "unable to acquire exclusive lock" << dendl;
   } else if (r < 0) {
-    lderr(m_cct) << "failed to acquire exclusive lock:" << cpp_strerror(r)
+    lderr(m_cct) << "failed to acquire exclusive lock:" << cpp_strerror(r)
                << dendl;
   } else {
-    ldout(m_cct, 5) << "successfully acquired exclusive lock" << dendl;
+    ldout(m_cct, 5) << "successfully acquired exclusive lock" << dendl;
   }
 
   m_post_next_state = (r < 0 ? STATE_UNLOCKED : STATE_LOCKED);
 
-  m_work_queue->queue(new FunctionContext([this, r](int ret) {
+  m_work_queue->queue(new LambdaContext([this, r](int ret) {
     post_acquire_lock_handler(r, create_context_callback<
         ManagedLock<I>, &ManagedLock<I>::handle_post_acquire_lock>(this));
   }));
@@ -514,9 +539,9 @@ void ManagedLock<I>::handle_acquire_lock(int r) {
 
 template <typename I>
 void ManagedLock<I>::handle_post_acquire_lock(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
-  Mutex::Locker locker(m_lock);
+  std::lock_guard locker{m_lock};
 
   if (r < 0 && m_post_next_state == STATE_LOCKED) {
     // release_lock without calling pre and post handlers
@@ -529,14 +554,14 @@ void ManagedLock<I>::handle_post_acquire_lock(int r) {
 
 template <typename I>
 void ManagedLock<I>::revert_to_unlock_state(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
   using managed_lock::ReleaseRequest;
   ReleaseRequest<I>* req = ReleaseRequest<I>::create(m_ioctx, m_watcher,
       m_work_queue, m_oid, m_cookie,
-      new FunctionContext([this, r](int ret) {
-        Mutex::Locker locker(m_lock);
-        assert(ret == 0);
+      new LambdaContext([this, r](int ret) {
+        std::lock_guard locker{m_lock};
+        ceph_assert(ret == 0);
         complete_active_action(STATE_UNLOCKED, r);
       }));
   m_work_queue->queue(new C_SendLockRequest<ReleaseRequest<I>>(req));
@@ -544,36 +569,42 @@ void ManagedLock<I>::revert_to_unlock_state(int r) {
 
 template <typename I>
 void ManagedLock<I>::send_reacquire_lock() {
-  assert(m_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(m_lock));
 
   if (m_state != STATE_LOCKED) {
     complete_active_action(m_state, 0);
     return;
   }
 
+  ldout(m_cct, 10) << dendl;
+  m_state = STATE_REACQUIRING;
+
   uint64_t watch_handle = m_watcher->get_watch_handle();
   if (watch_handle == 0) {
-     // watch (re)failed while recovering
-     lderr(m_cct) << ": aborting reacquire due to invalid watch handle"
-                  << dendl;
-     complete_active_action(STATE_LOCKED, 0);
-     return;
+    // watch (re)failed while recovering
+    lderr(m_cct) << "aborting reacquire due to invalid watch handle"
+                 << dendl;
+
+    // treat double-watch failure as a lost lock and invoke the
+    // release/acquire handlers
+    release_acquire_lock();
+    complete_active_action(STATE_LOCKED, 0);
+    return;
   }
 
   m_new_cookie = encode_lock_cookie(watch_handle);
-  if (m_cookie == m_new_cookie) {
-    ldout(m_cct, 10) << "skipping reacquire since cookie still valid"
+  if (m_cookie == m_new_cookie && m_blacklist_on_break_lock) {
+    ldout(m_cct, 10) << "skipping reacquire since cookie still valid"
                      << dendl;
-    complete_active_action(STATE_LOCKED, 0);
+    auto ctx = create_context_callback<
+      ManagedLock, &ManagedLock<I>::handle_no_op_reacquire_lock>(this);
+    post_reacquire_lock_handler(0, ctx);
     return;
   }
 
-  ldout(m_cct, 10) << dendl;
-  m_state = STATE_REACQUIRING;
-
   auto ctx = create_context_callback<
     ManagedLock, &ManagedLock<I>::handle_reacquire_lock>(this);
-  ctx = new FunctionContext([this, ctx](int r) {
+  ctx = new LambdaContext([this, ctx](int r) {
       post_reacquire_lock_handler(r, ctx);
     });
 
@@ -585,54 +616,63 @@ void ManagedLock<I>::send_reacquire_lock() {
 
 template <typename I>
 void ManagedLock<I>::handle_reacquire_lock(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
-  Mutex::Locker locker(m_lock);
-  assert(m_state == STATE_REACQUIRING);
+  std::lock_guard locker{m_lock};
+  ceph_assert(m_state == STATE_REACQUIRING);
 
   if (r < 0) {
     if (r == -EOPNOTSUPP) {
-      ldout(m_cct, 10) << "updating lock is not supported" << dendl;
+      ldout(m_cct, 10) << "updating lock is not supported" << dendl;
     } else {
-      lderr(m_cct) << "failed to update lock cookie: " << cpp_strerror(r)
+      lderr(m_cct) << "failed to update lock cookie: " << cpp_strerror(r)
                    << dendl;
     }
 
-    if (!is_state_shutdown()) {
-      // queue a release and re-acquire of the lock since cookie cannot
-      // be updated on older OSDs
-      execute_action(ACTION_RELEASE_LOCK, nullptr);
-
-      assert(!m_actions_contexts.empty());
-      ActionContexts &action_contexts(m_actions_contexts.front());
-
-      // reacquire completes when the request lock completes
-      Contexts contexts;
-      std::swap(contexts, action_contexts.second);
-      if (contexts.empty()) {
-        execute_action(ACTION_ACQUIRE_LOCK, nullptr);
-      } else {
-        for (auto ctx : contexts) {
-          ctx = new FunctionContext([ctx, r](int acquire_ret_val) {
-              if (acquire_ret_val >= 0) {
-                acquire_ret_val = r;
-              }
-              ctx->complete(acquire_ret_val);
-            });
-          execute_action(ACTION_ACQUIRE_LOCK, ctx);
-        }
-      }
-    }
+    release_acquire_lock();
   } else {
     m_cookie = m_new_cookie;
   }
 
-  complete_active_action(STATE_LOCKED, r);
+  complete_active_action(STATE_LOCKED, 0);
+}
+
+template <typename I>
+void ManagedLock<I>::handle_no_op_reacquire_lock(int r) {
+  ldout(m_cct, 10) << "r=" << r << dendl;
+  ceph_assert(m_state == STATE_REACQUIRING);
+  ceph_assert(r >= 0);
+  complete_active_action(STATE_LOCKED, 0);
+}
+
+template <typename I>
+void ManagedLock<I>::release_acquire_lock() {
+  assert(ceph_mutex_is_locked(m_lock));
+
+  if (!is_state_shutdown()) {
+    // queue a release and re-acquire of the lock since cookie cannot
+    // be updated on older OSDs
+    execute_action(ACTION_RELEASE_LOCK, nullptr);
+
+    ceph_assert(!m_actions_contexts.empty());
+    ActionContexts &action_contexts(m_actions_contexts.front());
+
+    // reacquire completes when the request lock completes
+    Contexts contexts;
+    std::swap(contexts, action_contexts.second);
+    if (contexts.empty()) {
+      execute_action(ACTION_ACQUIRE_LOCK, nullptr);
+    } else {
+      for (auto ctx : contexts) {
+        execute_action(ACTION_ACQUIRE_LOCK, ctx);
+      }
+    }
+  }
 }
 
 template <typename I>
 void ManagedLock<I>::send_release_lock() {
-  assert(m_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(m_lock));
   if (m_state == STATE_UNLOCKED) {
     complete_active_action(STATE_UNLOCKED, 0);
     return;
@@ -641,7 +681,7 @@ void ManagedLock<I>::send_release_lock() {
   ldout(m_cct, 10) << dendl;
   m_state = STATE_PRE_RELEASING;
 
-  m_work_queue->queue(new FunctionContext([this](int r) {
+  m_work_queue->queue(new LambdaContext([this](int r) {
     pre_release_lock_handler(false, create_context_callback<
         ManagedLock<I>, &ManagedLock<I>::handle_pre_release_lock>(this));
   }));
@@ -649,11 +689,11 @@ void ManagedLock<I>::send_release_lock() {
 
 template <typename I>
 void ManagedLock<I>::handle_pre_release_lock(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
   {
-    Mutex::Locker locker(m_lock);
-    assert(m_state == STATE_PRE_RELEASING);
+    std::lock_guard locker{m_lock};
+    ceph_assert(m_state == STATE_PRE_RELEASING);
     m_state = STATE_RELEASING;
   }
 
@@ -672,18 +712,19 @@ void ManagedLock<I>::handle_pre_release_lock(int r) {
 
 template <typename I>
 void ManagedLock<I>::handle_release_lock(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
-  Mutex::Locker locker(m_lock);
-  assert(m_state == STATE_RELEASING);
+  std::lock_guard locker{m_lock};
+  ceph_assert(m_state == STATE_RELEASING);
 
-  if (r >= 0) {
+  if (r >= 0 || r == -EBLACKLISTED || r == -ENOENT) {
     m_cookie = "";
+    m_post_next_state = STATE_UNLOCKED;
+  } else {
+    m_post_next_state = STATE_LOCKED;
   }
 
-  m_post_next_state = r < 0 ? STATE_LOCKED : STATE_UNLOCKED;
-
-  m_work_queue->queue(new FunctionContext([this, r](int ret) {
+  m_work_queue->queue(new LambdaContext([this, r](int ret) {
     post_release_lock_handler(false, r, create_context_callback<
         ManagedLock<I>, &ManagedLock<I>::handle_post_release_lock>(this));
   }));
@@ -691,36 +732,36 @@ void ManagedLock<I>::handle_release_lock(int r) {
 
 template <typename I>
 void ManagedLock<I>::handle_post_release_lock(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
-  Mutex::Locker locker(m_lock);
+  std::lock_guard locker{m_lock};
   complete_active_action(m_post_next_state, r);
 }
 
 template <typename I>
 void ManagedLock<I>::send_shutdown() {
   ldout(m_cct, 10) << dendl;
-  assert(m_lock.is_locked());
+  ceph_assert(ceph_mutex_is_locked(m_lock));
   if (m_state == STATE_UNLOCKED) {
     m_state = STATE_SHUTTING_DOWN;
-    m_work_queue->queue(new FunctionContext([this](int r) {
+    m_work_queue->queue(new LambdaContext([this](int r) {
       shutdown_handler(r, create_context_callback<
           ManagedLock<I>, &ManagedLock<I>::handle_shutdown>(this));
     }));
     return;
   }
 
-  assert(m_state == STATE_LOCKED);
+  ceph_assert(m_state == STATE_LOCKED);
   m_state = STATE_PRE_SHUTTING_DOWN;
 
-  m_lock.Unlock();
+  m_lock.unlock();
   m_work_queue->queue(new C_ShutDownRelease(this), 0);
-  m_lock.Lock();
+  m_lock.lock();
 }
 
 template <typename I>
 void ManagedLock<I>::handle_shutdown(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
   wait_for_tracked_ops(r);
 }
@@ -729,9 +770,9 @@ template <typename I>
 void ManagedLock<I>::send_shutdown_release() {
   ldout(m_cct, 10) << dendl;
 
-  Mutex::Locker locker(m_lock);
+  std::lock_guard locker{m_lock};
 
-  m_work_queue->queue(new FunctionContext([this](int r) {
+  m_work_queue->queue(new LambdaContext([this](int r) {
     pre_release_lock_handler(true, create_context_callback<
         ManagedLock<I>, &ManagedLock<I>::handle_shutdown_pre_release>(this));
   }));
@@ -739,22 +780,23 @@ void ManagedLock<I>::send_shutdown_release() {
 
 template <typename I>
 void ManagedLock<I>::handle_shutdown_pre_release(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
   std::string cookie;
   {
-    Mutex::Locker locker(m_lock);
+    std::lock_guard locker{m_lock};
     cookie = m_cookie;
 
-    assert(m_state == STATE_PRE_SHUTTING_DOWN);
+    ceph_assert(m_state == STATE_PRE_SHUTTING_DOWN);
     m_state = STATE_SHUTTING_DOWN;
   }
 
   using managed_lock::ReleaseRequest;
   ReleaseRequest<I>* req = ReleaseRequest<I>::create(m_ioctx, m_watcher,
       m_work_queue, m_oid, cookie,
-      new FunctionContext([this](int r) {
-        post_release_lock_handler(true, r, create_context_callback<
+      new LambdaContext([this, r](int l) {
+        int rst = r < 0 ? r : l;
+        post_release_lock_handler(true, rst, create_context_callback<
             ManagedLock<I>, &ManagedLock<I>::handle_shutdown_post_release>(this));
       }));
   req->send();
@@ -763,16 +805,16 @@ void ManagedLock<I>::handle_shutdown_pre_release(int r) {
 
 template <typename I>
 void ManagedLock<I>::handle_shutdown_post_release(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
   wait_for_tracked_ops(r);
 }
 
 template <typename I>
 void ManagedLock<I>::wait_for_tracked_ops(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
-  Context *ctx = new FunctionContext([this, r](int ret) {
+  Context *ctx = new LambdaContext([this, r](int ret) {
       complete_shutdown(r);
     });
 
@@ -781,7 +823,7 @@ void ManagedLock<I>::wait_for_tracked_ops(int r) {
 
 template <typename I>
 void ManagedLock<I>::complete_shutdown(int r) {
-  ldout(m_cct, 10) << "r=" << r << dendl;
+  ldout(m_cct, 10) << "r=" << r << dendl;
 
   if (r < 0) {
     lderr(m_cct) << "failed to shut down lock: " << cpp_strerror(r)
@@ -790,9 +832,9 @@ void ManagedLock<I>::complete_shutdown(int r) {
 
   ActionContexts action_contexts;
   {
-    Mutex::Locker locker(m_lock);
-    assert(m_lock.is_locked());
-    assert(m_actions_contexts.size() == 1);
+    std::lock_guard locker{m_lock};
+    ceph_assert(ceph_mutex_is_locked(m_lock));
+    ceph_assert(m_actions_contexts.size() == 1);
 
     action_contexts = std::move(m_actions_contexts.front());
     m_actions_contexts.pop_front();