]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
mds: manage Message lifetime with intrusive_ptr
authorPatrick Donnelly <pdonnell@redhat.com>
Sun, 29 Jul 2018 01:16:02 +0000 (18:16 -0700)
committerPatrick Donnelly <pdonnell@redhat.com>
Wed, 15 Aug 2018 20:35:30 +0000 (13:35 -0700)
This change turned out to be far more extensive than I hoped but the end result
should prevent all Message-related memory leaks. I believe I fixed several
incidentally.

Fixes: http://tracker.ceph.com/issues/24306
Signed-off-by: Patrick Donnelly <pdonnell@redhat.com>
86 files changed:
src/client/MetaSession.h
src/mds/Beacon.cc
src/mds/Beacon.h
src/mds/CDentry.cc
src/mds/CDentry.h
src/mds/CInode.cc
src/mds/CInode.h
src/mds/FSMap.h
src/mds/Locker.cc
src/mds/Locker.h
src/mds/MDBalancer.cc
src/mds/MDBalancer.h
src/mds/MDCache.cc
src/mds/MDCache.h
src/mds/MDSCacheObject.h
src/mds/MDSContext.h
src/mds/MDSDaemon.cc
src/mds/MDSDaemon.h
src/mds/MDSMap.h
src/mds/MDSRank.cc
src/mds/MDSRank.h
src/mds/MDSTableClient.cc
src/mds/MDSTableClient.h
src/mds/MDSTableServer.cc
src/mds/MDSTableServer.h
src/mds/Migrator.cc
src/mds/Migrator.h
src/mds/Mutation.cc
src/mds/Mutation.h
src/mds/Server.cc
src/mds/Server.h
src/mds/SessionMap.h
src/mds/SimpleLock.h
src/mds/SnapClient.cc
src/mds/SnapClient.h
src/mds/SnapServer.cc
src/mds/SnapServer.h
src/mds/events/EImportStart.h
src/mds/journal.cc
src/mds/mdstypes.h
src/messages/MCacheExpire.h
src/messages/MClientCapRelease.h
src/messages/MClientCaps.h
src/messages/MClientLease.h
src/messages/MClientReconnect.h
src/messages/MClientReply.h
src/messages/MClientRequest.h
src/messages/MClientSession.h
src/messages/MClientSnap.h
src/messages/MCommand.h
src/messages/MDentryLink.h
src/messages/MDentryUnlink.h
src/messages/MDirUpdate.h
src/messages/MDiscover.h
src/messages/MDiscoverReply.h
src/messages/MExportCaps.h
src/messages/MExportCapsAck.h
src/messages/MExportDir.h
src/messages/MExportDirAck.h
src/messages/MExportDirCancel.h
src/messages/MExportDirDiscover.h
src/messages/MExportDirDiscoverAck.h
src/messages/MExportDirFinish.h
src/messages/MExportDirNotify.h
src/messages/MExportDirNotifyAck.h
src/messages/MExportDirPrep.h
src/messages/MExportDirPrepAck.h
src/messages/MGatherCaps.h
src/messages/MHeartbeat.h
src/messages/MInodeFileCaps.h
src/messages/MInterMDS.h [new file with mode: 0644]
src/messages/MLock.h
src/messages/MMDSBeacon.h
src/messages/MMDSCacheRejoin.h
src/messages/MMDSFindIno.h
src/messages/MMDSFindInoReply.h
src/messages/MMDSFragmentNotify.h
src/messages/MMDSMap.h
src/messages/MMDSOpenIno.h
src/messages/MMDSOpenInoReply.h
src/messages/MMDSResolve.h
src/messages/MMDSResolveAck.h
src/messages/MMDSSlaveRequest.h
src/messages/MMDSSnapUpdate.h
src/messages/MMDSTableRequest.h
src/mon/MDSMonitor.cc

index 0879abdbe9f1ad61c3cb5355ae6bd6789dfaf673..b00abad27e6115e15432927820174cdb6861f4e2 100644 (file)
@@ -47,7 +47,7 @@ struct MetaSession {
   std::set<ceph_tid_t> flushing_caps_tids;
   std::set<Inode*> early_flushing_caps;
 
-  boost::intrusive_ptr<MClientCapRelease> release;
+  MClientCapRelease::ref release;
 
   MetaSession(mds_rank_t mds_num, ConnectionRef con,
              const entity_addrvec_t& addrs)
index 38db6988b866b8a1282e54d9a956962adfb70e96..c9dc3d8af13c62e53a04150c3b6fcb14d3b83ab8 100644 (file)
@@ -19,7 +19,6 @@
 #include "include/stringify.h"
 #include "include/util.h"
 
-#include "messages/MMDSBeacon.h"
 #include "mon/MonClient.h"
 #include "mds/MDLog.h"
 #include "mds/MDSRank.h"
@@ -52,10 +51,9 @@ Beacon::~Beacon()
 }
 
 
-void Beacon::init(MDSMap const *mdsmap)
+void Beacon::init(const MDSMap &mdsmap)
 {
   Mutex::Locker l(lock);
-  assert(mdsmap != NULL);
 
   _notify_mdsmap(mdsmap);
   standby_for_rank = mds_rank_t(g_conf()->mds_standby_for_rank);
@@ -79,24 +77,22 @@ void Beacon::shutdown()
   timer.shutdown();
 }
 
-bool Beacon::ms_can_fast_dispatch(const Message *m) const
+bool Beacon::ms_can_fast_dispatch2(const Message::const_ref& m) const
 {
   return m->get_type() == MSG_MDS_BEACON;
 }
 
-void Beacon::ms_fast_dispatch(Message *m)
+void Beacon::ms_fast_dispatch2(const Message::ref& m)
 {
-  bool handled = ms_dispatch(m);
+  bool handled = ms_dispatch2(m);
   assert(handled);
 }
 
-bool Beacon::ms_dispatch(Message *m)
+bool Beacon::ms_dispatch2(const Message::ref& m)
 {
   if (m->get_type() == MSG_MDS_BEACON) {
     if (m->get_connection()->get_peer_type() == CEPH_ENTITY_TYPE_MON) {
-      handle_mds_beacon(static_cast<MMDSBeacon*>(m));
-    } else {
-      m->put();
+      handle_mds_beacon(boost::static_pointer_cast<MMDSBeacon::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     }
     return true;
   }
@@ -110,10 +106,9 @@ bool Beacon::ms_dispatch(Message *m)
  *
  * This function puts the passed message before returning
  */
-void Beacon::handle_mds_beacon(MMDSBeacon *m)
+void Beacon::handle_mds_beacon(const MMDSBeacon::const_ref &m)
 {
   Mutex::Locker l(lock);
-  assert(m != NULL);
 
   version_t seq = m->get_seq();
 
@@ -154,7 +149,6 @@ void Beacon::handle_mds_beacon(MMDSBeacon *m)
     dout(1) << "handle_mds_beacon " << ceph_mds_state_name(m->get_state())
            << " seq " << m->get_seq() << " dne" << dendl;
   }
-  m->put();
 }
 
 
@@ -214,13 +208,13 @@ void Beacon::_send()
 
   assert(want_state != MDSMap::STATE_NULL);
   
-  MMDSBeacon *beacon = new MMDSBeacon(
+  MMDSBeacon::ref beacon(new MMDSBeacon(
       monc->get_fsid(), mds_gid_t(monc->get_global_id()),
       name,
       epoch,
       want_state,
       last_seq,
-      CEPH_FEATURES_SUPPORTED_DEFAULT);
+      CEPH_FEATURES_SUPPORTED_DEFAULT), false);
 
   beacon->set_standby_for_rank(standby_for_rank);
   beacon->set_standby_for_name(standby_for_name);
@@ -235,29 +229,27 @@ void Beacon::_send()
     sys_info["addr"] = stringify(monc->get_myaddrs());
     beacon->set_sys_info(sys_info);
   }
-  monc->send_mon_message(beacon);
+  monc->send_mon_message(beacon.detach());
 }
 
 /**
  * Call this when there is a new MDSMap available
  */
-void Beacon::notify_mdsmap(MDSMap const *mdsmap)
+void Beacon::notify_mdsmap(const MDSMap &mdsmap)
 {
   Mutex::Locker l(lock);
-  assert(mdsmap != NULL);
 
   _notify_mdsmap(mdsmap);
 }
 
-void Beacon::_notify_mdsmap(MDSMap const *mdsmap)
+void Beacon::_notify_mdsmap(const MDSMap &mdsmap)
 {
-  assert(mdsmap != NULL);
-  assert(mdsmap->get_epoch() >= epoch);
+  assert(mdsmap.get_epoch() >= epoch);
 
-  if (mdsmap->get_epoch() != epoch) {
-    epoch = mdsmap->get_epoch();
+  if (mdsmap.get_epoch() != epoch) {
+    epoch = mdsmap.get_epoch();
     compat = MDSMap::get_compat_set_default();
-    compat.merge(mdsmap->compat);
+    compat.merge(mdsmap.compat);
   }
 }
 
@@ -295,7 +287,7 @@ utime_t Beacon::get_laggy_until() const
   return laggy_until;
 }
 
-void Beacon::set_want_state(MDSMap const *mdsmap, MDSMap::DaemonState const newstate)
+void Beacon::set_want_state(const MDSMap &mdsmap, MDSMap::DaemonState const newstate)
 {
   Mutex::Locker l(lock);
 
index 1bcbf85d45d0398cdc69db3183bbfdd50915e35e..556df59d560b0c0c5e8c196966eff450494ae3bd 100644 (file)
 #include "include/Context.h"
 #include "common/Mutex.h"
 #include "msg/Dispatcher.h"
+
 #include "messages/MMDSBeacon.h"
 
 class MonClient;
-class MMDSBeacon;
-class Message;
 class MDSRank;
 
 
@@ -45,25 +44,25 @@ public:
   Beacon(CephContext *cct_, MonClient *monc_, std::string_view name);
   ~Beacon() override;
 
-  void init(MDSMap const *mdsmap);
+  void init(const MDSMap &mdsmap);
   void shutdown();
 
   bool ms_can_fast_dispatch_any() const override { return true; }
-  bool ms_can_fast_dispatch(const Message *m) const override;
-  void ms_fast_dispatch(Message *m) override;
-  bool ms_dispatch(Message *m) override;
+  bool ms_can_fast_dispatch2(const Message::const_ref& m) const override;
+  void ms_fast_dispatch2(const Message::ref& m) override;
+  bool ms_dispatch2(const Message::ref &m) override;
   void ms_handle_connect(Connection *c) override {}
   bool ms_handle_reset(Connection *c) override {return false;}
   void ms_handle_remote_reset(Connection *c) override {}
   bool ms_handle_refused(Connection *c) override {return false;}
 
-  void notify_mdsmap(MDSMap const *mdsmap);
-  void notify_health(MDSRank const *mds);
+  void notify_mdsmap(const MDSMap &mdsmap);
+  void notify_health(const MDSRank *mds);
 
-  void handle_mds_beacon(MMDSBeacon *m);
+  void handle_mds_beacon(const MMDSBeacon::const_ref &m);
   void send();
 
-  void set_want_state(MDSMap const *mdsmap, MDSMap::DaemonState const newstate);
+  void set_want_state(const MDSMap &mdsmap, MDSMap::DaemonState const newstate);
   MDSMap::DaemonState get_want_state() const;
 
   /**
@@ -77,7 +76,7 @@ public:
   utime_t get_laggy_until() const;
 
 private:
-  void _notify_mdsmap(MDSMap const *mdsmap);
+  void _notify_mdsmap(const MDSMap &mdsmap);
   void _send();
 
   //CephContext *cct;
index f21e5b2f83c2ab4c1e8665da75723135793a11c7..6a29050987b545176a1d8902f4ff82c20e078594 100644 (file)
@@ -428,7 +428,7 @@ void CDentry::decode_replica(bufferlist::const_iterator& p, bool is_new)
 // ----------------------------
 // locking
 
-void CDentry::set_object_info(MDSCacheObjectInfo &info) 
+void CDentry::set_object_info(MDSCacheObjectInfo &info)
 {
   info.dirfrag = dir->dirfrag();
   info.dname = name;
@@ -457,7 +457,7 @@ void CDentry::encode_lock_state(int type, bufferlist& bl)
   else ceph_abort();
 }
 
-void CDentry::decode_lock_state(int type, bufferlist& bl)
+void CDentry::decode_lock_state(int type, const bufferlist& bl)
 {  
   auto p = bl.cbegin();
 
index 556a03a437a05d939f2be331b858c715d0face47..df1f74739d8b91bf3516fb5ffac8fa84a5b2c7de 100644 (file)
@@ -37,7 +37,6 @@
 class CInode;
 class CDir;
 class Locker;
-class Message;
 class CDentry;
 class LogSegment;
 
@@ -309,7 +308,7 @@ public:
   }
   void set_object_info(MDSCacheObjectInfo &info) override;
   void encode_lock_state(int type, bufferlist& bl) override;
-  void decode_lock_state(int type, bufferlist& bl) override;
+  void decode_lock_state(int type, const bufferlist& bl) override;
 
   // ---------------------------------------------
   // replicas (on clients)
index 7504a4616abd023db75289dff0d30c2be567f401..3f92e0bb0cdc0e7f8d03d57a86e12139b0beb473 100644 (file)
@@ -38,9 +38,6 @@
 
 #include "common/Clock.h"
 
-#include "messages/MLock.h"
-#include "messages/MClientCaps.h"
-
 #include "common/config.h"
 #include "global/global_context.h"
 #include "include/assert.h"
@@ -1702,7 +1699,7 @@ void CInode::encode_lock_state(int type, bufferlist& bl)
 
 /* for more info on scatterlocks, see comments by Locker::scatter_writebehind */
 
-void CInode::decode_lock_state(int type, bufferlist& bl)
+void CInode::decode_lock_state(int type, const bufferlist& bl)
 {
   auto p = bl.cbegin();
   utime_t tm;
@@ -2825,7 +2822,7 @@ void CInode::encode_snap_blob(bufferlist &snapbl)
     dout(20) << __func__ << " " << *snaprealm << dendl;
   }
 }
-void CInode::decode_snap_blob(bufferlist& snapbl)
+void CInode::decode_snap_blob(const bufferlist& snapbl)
 {
   using ceph::decode;
   if (snapbl.length()) {
@@ -3681,7 +3678,7 @@ int CInode::encode_inodestat(bufferlist& bl, Session *session,
   return valid;
 }
 
-void CInode::encode_cap_message(MClientCaps *m, Capability *cap)
+void CInode::encode_cap_message(const MClientCaps::ref &m, Capability *cap)
 {
   assert(cap);
 
index ef72340aa25c34b5936c7846d655ea4ba197676e..516e7741ef829db4b6f67110d2f6094d010a2ad6 100644 (file)
 #include "SnapRealm.h"
 #include "Mutation.h"
 
+#include "messages/MClientCaps.h"
+
 #define dout_context g_ceph_context
 
 class Context;
 class CDentry;
 class CDir;
-class Message;
 class CInode;
 class MDCache;
 class LogSegment;
 struct SnapRealm;
 class Session;
-class MClientCaps;
 struct ObjectOperation;
 class EMetaBlob;
 
@@ -799,7 +799,7 @@ public:
   bool is_dirty_pool() { return state_test(STATE_DIRTYPOOL); }
 
   void encode_snap_blob(bufferlist &bl);
-  void decode_snap_blob(bufferlist &bl);
+  void decode_snap_blob(const bufferlist &bl);
   void encode_store(bufferlist& bl, uint64_t features);
   void decode_store(bufferlist::const_iterator& bl);
 
@@ -866,7 +866,7 @@ public:
   int encode_inodestat(bufferlist& bl, Session *session, SnapRealm *realm,
                       snapid_t snapid=CEPH_NOSNAP, unsigned max_bytes=0,
                       int getattr_wants=0);
-  void encode_cap_message(MClientCaps *m, Capability *cap);
+  void encode_cap_message(const MClientCaps::ref &m, Capability *cap);
 
 
   // -- locks --
@@ -911,7 +911,7 @@ public:
 
   void set_object_info(MDSCacheObjectInfo &info) override;
   void encode_lock_state(int type, bufferlist& bl) override;
-  void decode_lock_state(int type, bufferlist& bl) override;
+  void decode_lock_state(int type, const bufferlist& bl) override;
 
   void _finish_frag_update(CDir *dir, MutationRef& mut);
 
index 047573cc7c49c3cb26aecce54b68de4e3044bcac..b6d98b978b66417896623bfc2c161c85e926c559 100644 (file)
@@ -25,7 +25,6 @@
 
 #include "include/types.h"
 #include "common/Clock.h"
-#include "msg/Message.h"
 #include "mds/MDSMap.h"
 
 #include "common/config.h"
index 2eb5471b98e3b936cc26e035a04658445246f7e3..1d521472f6846d0e8b0acb9dececf8d3aecea255 100644 (file)
@@ -89,33 +89,28 @@ Locker::Locker(MDSRank *m, MDCache *c) :
   mds(m), mdcache(c), need_snapflush_inodes(member_offset(CInode, item_caps)) {}
 
 
-/* This function DOES put the passed message before returning */
-void Locker::dispatch(Message *m)
+void Locker::dispatch(const Message::const_ref &m)
 {
 
   switch (m->get_type()) {
-
     // inter-mds locking
   case MSG_MDS_LOCK:
-    handle_lock(static_cast<MLock*>(m));
+    handle_lock(boost::static_pointer_cast<MLock::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
     // inter-mds caps
   case MSG_MDS_INODEFILECAPS:
-    handle_inode_file_caps(static_cast<MInodeFileCaps*>(m));
+    handle_inode_file_caps(boost::static_pointer_cast<MInodeFileCaps::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
-
     // client sync
   case CEPH_MSG_CLIENT_CAPS:
-    handle_client_caps(static_cast<MClientCaps*>(m));
-
+    handle_client_caps(boost::static_pointer_cast<MClientCaps::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case CEPH_MSG_CLIENT_CAPRELEASE:
-    handle_client_cap_release(static_cast<MClientCapRelease*>(m));
+    handle_client_cap_release(boost::static_pointer_cast<MClientCapRelease::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case CEPH_MSG_CLIENT_LEASE:
-    handle_client_lease(static_cast<MClientLease*>(m));
+    handle_client_lease(boost::static_pointer_cast<MClientLease::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
-    
   default:
     derr << "locker unknown message " << m->get_type() << dendl;
     assert(0 == "locker unknown message");
@@ -141,7 +136,7 @@ void Locker::send_lock_message(SimpleLock *lock, int msg)
     if (mds->is_cluster_degraded() &&
        mds->mdsmap->get_state(it.first) < MDSMap::STATE_REJOIN)
       continue;
-    MLock *m = new MLock(lock, msg, mds->get_nodeid());
+    MLock::ref m(new MLock(lock, msg, mds->get_nodeid()), false);
     mds->send_message_mds(m, it.first);
   }
 }
@@ -152,7 +147,7 @@ void Locker::send_lock_message(SimpleLock *lock, int msg, const bufferlist &data
     if (mds->is_cluster_degraded() &&
        mds->mdsmap->get_state(it.first) < MDSMap::STATE_REJOIN)
       continue;
-    MLock *m = new MLock(lock, msg, mds->get_nodeid());
+    MLock::ref m(new MLock(lock, msg, mds->get_nodeid()), false);
     m->set_data(data);
     mds->send_message_mds(m, it.first);
   }
@@ -454,8 +449,7 @@ bool Locker::acquire_locks(MDRequestRef& mdr,
        return false;
       }
       
-      MMDSSlaveRequest *req = new MMDSSlaveRequest(mdr->reqid, mdr->attempt,
-                                                  MMDSSlaveRequest::OP_AUTHPIN);
+      MMDSSlaveRequest::ref req(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_AUTHPIN), false);
       for (set<MDSCacheObject*>::iterator q = p->second.begin();
           q != p->second.end();
           ++q) {
@@ -737,8 +731,7 @@ void Locker::_drop_non_rdlocks(MutationImpl *mut, set<CInode*> *pneed_issue)
     if (!mds->is_cluster_degraded() ||
        mds->mdsmap->get_state(*p) >= MDSMap::STATE_REJOIN) {
       dout(10) << "_drop_non_rdlocks dropping remote locks on mds." << *p << dendl;
-      MMDSSlaveRequest *slavereq = new MMDSSlaveRequest(mut->reqid, mut->attempt,
-                                                       MMDSSlaveRequest::OP_DROPLOCKS);
+      MMDSSlaveRequest::ref slavereq(new MMDSSlaveRequest(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_DROPLOCKS), false);
       mds->send_message_mds(slavereq, *p);
     }
   }
@@ -895,13 +888,12 @@ void Locker::eval_gather(SimpleLock *lock, bool first, bool *pneed_issue, MDSInt
          mds->mdsmap->get_state(auth) >= MDSMap::STATE_REJOIN) {
        switch (lock->get_state()) {
        case LOCK_SYNC_LOCK:
-         mds->send_message_mds(new MLock(lock, LOCK_AC_LOCKACK, mds->get_nodeid()),
-                               auth);
+         mds->send_message_mds(MLock::ref(new MLock(lock, LOCK_AC_LOCKACK, mds->get_nodeid()), false), auth);
          break;
 
        case LOCK_MIX_SYNC:
          {
-           MLock *reply = new MLock(lock, LOCK_AC_SYNCACK, mds->get_nodeid());
+           MLock::ref reply(new MLock(lock, LOCK_AC_SYNCACK, mds->get_nodeid()), false);
            lock->encode_locked_state(reply->get_data());
            mds->send_message_mds(reply, auth);
            next = LOCK_MIX_SYNC2;
@@ -919,7 +911,7 @@ void Locker::eval_gather(SimpleLock *lock, bool first, bool *pneed_issue, MDSInt
          
        case LOCK_SYNC_MIX:
          { 
-           MLock *reply = new MLock(lock, LOCK_AC_MIXACK, mds->get_nodeid());
+           MLock::ref reply(new MLock(lock, LOCK_AC_MIXACK, mds->get_nodeid()), false);
            mds->send_message_mds(reply, auth);
            next = LOCK_SYNC_MIX2;
          }
@@ -929,7 +921,7 @@ void Locker::eval_gather(SimpleLock *lock, bool first, bool *pneed_issue, MDSInt
          {
            bufferlist data;
            lock->encode_locked_state(data);
-           mds->send_message_mds(new MLock(lock, LOCK_AC_LOCKACK, mds->get_nodeid(), data), auth);
+           mds->send_message_mds(MLock::ref(new MLock(lock, LOCK_AC_LOCKACK, mds->get_nodeid(), data), false), auth);
            (static_cast<ScatterLock *>(lock))->start_flush();
            // we'll get an AC_LOCKFLUSHED to complete
          }
@@ -1292,7 +1284,7 @@ bool Locker::_rdlock_kick(SimpleLock *lock, bool as_anon)
          mds->mdsmap->is_clientreplay_or_active_or_stopping(auth)) {
        dout(10) << "requesting rdlock from auth on "
                 << *lock << " on " << *lock->get_parent() << dendl;
-       mds->send_message_mds(new MLock(lock, LOCK_AC_REQRDLOCK, mds->get_nodeid()), auth);
+       mds->send_message_mds(MLock::ref(new MLock(lock, LOCK_AC_REQRDLOCK, mds->get_nodeid()), false), auth);
       }
       return false;
     }
@@ -1524,7 +1516,7 @@ bool Locker::wrlock_start(SimpleLock *lock, MDRequestRef& mut, bool nowait)
          mds->mdsmap->is_clientreplay_or_active_or_stopping(auth)) {
        dout(10) << "requesting scatter from auth on "
                 << *lock << " on " << *lock->get_parent() << dendl;
-       mds->send_message_mds(new MLock(lock, LOCK_AC_REQSCATTER, mds->get_nodeid()), auth);
+       mds->send_message_mds(MLock::ref(new MLock(lock, LOCK_AC_REQSCATTER, mds->get_nodeid()), false), auth));
       }
       break;
     }
@@ -1580,8 +1572,7 @@ void Locker::remote_wrlock_start(SimpleLock *lock, mds_rank_t target, MDRequestR
   // send lock request
   mut->start_locking(lock, target);
   mut->more()->slaves.insert(target);
-  MMDSSlaveRequest *r = new MMDSSlaveRequest(mut->reqid, mut->attempt,
-                                            MMDSSlaveRequest::OP_WRLOCK);
+  MMDSSlaveRequest::ref r(new MMDSSlaveRequest(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_WRLOCK), false);
   r->set_lock_type(lock->get_type());
   lock->get_parent()->set_object_info(r->get_object_info());
   mds->send_message_mds(r, target);
@@ -1602,8 +1593,7 @@ void Locker::remote_wrlock_finish(SimpleLock *lock, mds_rank_t target,
          << " " << *lock->get_parent()  << dendl;
   if (!mds->is_cluster_degraded() ||
       mds->mdsmap->get_state(target) >= MDSMap::STATE_REJOIN) {
-    MMDSSlaveRequest *slavereq = new MMDSSlaveRequest(mut->reqid, mut->attempt,
-                                                     MMDSSlaveRequest::OP_UNWRLOCK);
+    MMDSSlaveRequest::ref slavereq(new MMDSSlaveRequest(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_UNWRLOCK), false);
     slavereq->set_lock_type(lock->get_type());
     lock->get_parent()->set_object_info(slavereq->get_object_info());
     mds->send_message_mds(slavereq, target);
@@ -1684,8 +1674,7 @@ bool Locker::xlock_start(SimpleLock *lock, MDRequestRef& mut)
     // send lock request
     mut->more()->slaves.insert(auth);
     mut->start_locking(lock, auth);
-    MMDSSlaveRequest *r = new MMDSSlaveRequest(mut->reqid, mut->attempt,
-                                              MMDSSlaveRequest::OP_XLOCK);
+    MMDSSlaveRequest::ref r(new MMDSSlaveRequest(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_XLOCK), false);
     r->set_lock_type(lock->get_type());
     lock->get_parent()->set_object_info(r->get_object_info());
     mds->send_message_mds(r, auth);
@@ -1751,8 +1740,7 @@ void Locker::xlock_finish(SimpleLock *lock, MutationImpl *mut, bool *pneed_issue
     mds_rank_t auth = lock->get_parent()->authority().first;
     if (!mds->is_cluster_degraded() ||
        mds->mdsmap->get_state(auth) >= MDSMap::STATE_REJOIN) {
-      MMDSSlaveRequest *slavereq = new MMDSSlaveRequest(mut->reqid, mut->attempt,
-                                                       MMDSSlaveRequest::OP_UNXLOCK);
+      MMDSSlaveRequest::ref slavereq(new MMDSSlaveRequest(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_UNXLOCK), false);
       slavereq->set_lock_type(lock->get_type());
       lock->get_parent()->set_object_info(slavereq->get_object_info());
       mds->send_message_mds(slavereq, auth);
@@ -1818,11 +1806,11 @@ class C_Locker_FileUpdate_finish : public LockerLogContext {
   MutationRef mut;
   unsigned flags;
   client_t client;
-  MClientCaps *ack;
+  MClientCaps::ref ack;
 public:
   C_Locker_FileUpdate_finish(Locker *l, CInode *i, MutationRef& m, unsigned f,
-                            client_t c=-1, MClientCaps *a=nullptr)
-    : LockerLogContext(l), in(i), mut(m), flags(f), client(c), ack(a) {
+                             const MClientCaps::ref &ack, client_t c=-1)
+    : LockerLogContext(l), in(i), mut(m), flags(f), client(c), ack(ack) {
     in->get(CInode::PIN_PTRWAITER);
   }
   void finish(int r) override {
@@ -1838,23 +1826,22 @@ enum {
 };
 
 void Locker::file_update_finish(CInode *in, MutationRef& mut, unsigned flags,
-                               client_t client, MClientCaps *ack)
+                               client_t client, const MClientCaps::ref &ack)
 {
   dout(10) << "file_update_finish on " << *in << dendl;
   in->pop_and_dirty_projected_inode(mut->ls);
 
   mut->apply();
-  
+
   if (ack) {
     Session *session = mds->get_session(client);
     if (session) {
       // "oldest flush tid" > 0 means client uses unique TID for each flush
       if (ack->get_oldest_flush_tid() > 0)
-       session->add_completed_flush(ack->get_client_tid());
+        session->add_completed_flush(ack->get_client_tid());
       mds->send_message_client_counted(ack, session);
     } else {
       dout(10) << " no session for client." << client << " " << *ack << dendl;
-      ack->put();
     }
   }
 
@@ -2088,12 +2075,14 @@ bool Locker::issue_caps(CInode *in, Capability *only_cap)
                cap->reset_num_revoke_warnings();
        }
 
-       MClientCaps *m = new MClientCaps(op, in->ino(),
-                                        in->find_snaprealm()->inode->ino(),
-                                        cap->get_cap_id(), cap->get_last_seq(),
-                                        after, wanted, 0,
-                                        cap->get_mseq(),
-                                         mds->get_osd_epoch_barrier());
+       MClientCaps::ref m(new MClientCaps(op, in->ino(),
+                                           in->find_snaprealm()->inode->ino(),
+                                           cap->get_cap_id(),
+                                           cap->get_last_seq(),
+                                           after, wanted, 0,
+                                           cap->get_mseq(),
+                                           mds->get_osd_epoch_barrier())
+                           , false);
        in->encode_cap_message(m, cap);
 
        mds->send_message_client_counted(m, it->first);
@@ -2113,13 +2102,14 @@ void Locker::issue_truncate(CInode *in)
   
   for (auto &p : in->client_caps) {
     Capability *cap = &p.second;
-    MClientCaps *m = new MClientCaps(CEPH_CAP_OP_TRUNC,
-                                    in->ino(),
-                                    in->find_snaprealm()->inode->ino(),
-                                    cap->get_cap_id(), cap->get_last_seq(),
-                                    cap->pending(), cap->wanted(), 0,
-                                    cap->get_mseq(),
-                                     mds->get_osd_epoch_barrier());
+    MClientCaps::ref m(new MClientCaps(CEPH_CAP_OP_TRUNC,
+                                       in->ino(),
+                                       in->find_snaprealm()->inode->ino(),
+                                       cap->get_cap_id(), cap->get_last_seq(),
+                                       cap->pending(), cap->wanted(), 0,
+                                       cap->get_mseq(),
+                                       mds->get_osd_epoch_barrier())
+                       , false);
     in->encode_cap_message(m, cap);                         
     mds->send_message_client_counted(m, p.first);
   }
@@ -2253,13 +2243,11 @@ void Locker::request_inode_file_caps(CInode *in)
 
     if (!mds->is_cluster_degraded() ||
        mds->mdsmap->is_clientreplay_or_active_or_stopping(auth))
-      mds->send_message_mds(new MInodeFileCaps(in->ino(), in->replica_caps_wanted),
-                           auth);
+      mds->send_message_mds(MInodeFileCaps::ref(new MInodeFileCaps(in->ino(), in->replica_caps_wanted), false), auth);
   }
 }
 
-/* This function DOES put the passed message before returning */
-void Locker::handle_inode_file_caps(MInodeFileCaps *m)
+void Locker::handle_inode_file_caps(const MInodeFileCaps::const_ref &m)
 {
   // nobody should be talking to us during recovery.
   assert(mds->is_clientreplay() || mds->is_active() || mds->is_stopping());
@@ -2276,7 +2264,6 @@ void Locker::handle_inode_file_caps(MInodeFileCaps *m)
   in->set_mds_caps_wanted(from, m->get_caps());
 
   try_eval(in, CEPH_CAP_LOCKS);
-  m->put();
 }
 
 
@@ -2458,7 +2445,7 @@ bool Locker::check_inode_max_size(CInode *in, bool force_wrlock,
     mdcache->journal_dirty_inode(mut.get(), metablob, in);
   }
   mds->mdlog->submit_entry(le, new C_Locker_FileUpdate_finish(this, in, mut,
-                                                             UPDATE_SHAREMAX));
+      UPDATE_SHAREMAX, MClientCaps::ref()));
   wrlock_force(&in->filelock, mut);  // wrlock for duration of journal
   mut->auth_pin(in);
 
@@ -2491,13 +2478,16 @@ void Locker::share_inode_max_size(CInode *in, Capability *only_cap)
     if (cap->pending() & (CEPH_CAP_FILE_WR|CEPH_CAP_FILE_BUFFER)) {
       dout(10) << "share_inode_max_size with client." << client << dendl;
       cap->inc_last_seq();
-      MClientCaps *m = new MClientCaps(CEPH_CAP_OP_GRANT,
-                                      in->ino(),
-                                      in->find_snaprealm()->inode->ino(),
-                                      cap->get_cap_id(), cap->get_last_seq(),
-                                      cap->pending(), cap->wanted(), 0,
-                                       cap->get_mseq(),
-                                       mds->get_osd_epoch_barrier());
+      MClientCaps::ref m(new MClientCaps(CEPH_CAP_OP_GRANT,
+                                         in->ino(),
+                                         in->find_snaprealm()->inode->ino(),
+                                         cap->get_cap_id(),
+                                         cap->get_last_seq(),
+                                         cap->pending(),
+                                         cap->wanted(), 0,
+                                         cap->get_mseq(),
+                                         mds->get_osd_epoch_barrier())
+                         , false);
       in->encode_cap_message(m, cap);
       mds->send_message_client_counted(m, client);
     }
@@ -2620,7 +2610,7 @@ void Locker::_do_null_snapflush(CInode *head_in, client_t client, snapid_t last)
       CInode *sin = mdcache->pick_inode_snap(head_in, snapid - 1);
       assert(sin);
       assert(sin->first <= snapid);
-      _do_snap_update(sin, snapid, 0, sin->first - 1, client, NULL, NULL);
+      _do_snap_update(sin, snapid, 0, sin->first - 1, client, MClientCaps::ref(), MClientCaps::ref());
       head_in->remove_need_snapflush(sin, snapid, client);
     }
   }
@@ -2645,38 +2635,35 @@ bool Locker::should_defer_client_cap_frozen(CInode *in)
   return (in->is_freezing() && in->get_num_auth_pins() == 0) || in->is_frozen();
 }
 
-/*
- * This function DOES put the passed message before returning
- */
-void Locker::handle_client_caps(MClientCaps *m)
+void Locker::handle_client_caps(const MClientCaps::const_ref &m)
 {
   client_t client = m->get_source().num();
   snapid_t follows = m->get_snap_follows();
+  auto op = m->get_op();
+  auto dirty = m->get_dirty();
   dout(7) << "handle_client_caps "
          << " on " << m->get_ino()
          << " tid " << m->get_client_tid() << " follows " << follows
-         << " op " << ceph_cap_op_name(m->get_op())
+         << " op " << ceph_cap_op_name(op)
          << " flags 0x" << std::hex << m->flags << std::dec << dendl;
 
-  Session *session = mds->get_session(m);
+  Session *session = mds->get_session(boost::static_pointer_cast<Message::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
   if (!mds->is_clientreplay() && !mds->is_active() && !mds->is_stopping()) {
     if (!session) {
       dout(5) << " no session, dropping " << *m << dendl;
-      m->put();
       return;
     }
     if (session->is_closed() ||
        session->is_closing() ||
        session->is_killing()) {
       dout(7) << " session closed|closing|killing, dropping " << *m << dendl;
-      m->put();
       return;
     }
     if ((mds->is_reconnect() || mds->get_want_state() == MDSMap::STATE_RECONNECT) &&
-       m->get_dirty() && m->get_client_tid() > 0 &&
+       dirty && m->get_client_tid() > 0 &&
        !session->have_completed_flush(m->get_client_tid())) {
-      mdcache->set_reconnected_dirty_caps(client, m->get_ino(), m->get_dirty(),
-                                         m->get_op() == CEPH_CAP_OP_FLUSHSNAP);
+      mdcache->set_reconnected_dirty_caps(client, m->get_ino(), dirty,
+                                         op == CEPH_CAP_OP_FLUSHSNAP);
     }
     mds->wait_for_replay(new C_MDS_RetryMessage(mds, m));
     return;
@@ -2686,25 +2673,21 @@ void Locker::handle_client_caps(MClientCaps *m)
       session->have_completed_flush(m->get_client_tid())) {
     dout(7) << "handle_client_caps already flushed tid " << m->get_client_tid()
            << " for client." << client << dendl;
-    MClientCaps *ack;
-    if (m->get_op() == CEPH_CAP_OP_FLUSHSNAP) {
-      ack = new MClientCaps(CEPH_CAP_OP_FLUSHSNAP_ACK, m->get_ino(), 0, 0, 0, 0, 0,
-                           m->get_dirty(), 0, mds->get_osd_epoch_barrier());
+    MClientCaps::ref ack;
+    if (op == CEPH_CAP_OP_FLUSHSNAP) {
+      ack.reset(new MClientCaps(CEPH_CAP_OP_FLUSHSNAP_ACK, m->get_ino(), 0, 0, 0, 0, 0, dirty, 0, mds->get_osd_epoch_barrier()), false);
     } else {
-      ack = new MClientCaps(CEPH_CAP_OP_FLUSH_ACK, m->get_ino(), 0, m->get_cap_id(),
-                           m->get_seq(), m->get_caps(), 0, m->get_dirty(), 0,
-                           mds->get_osd_epoch_barrier());
+      ack.reset(new MClientCaps(CEPH_CAP_OP_FLUSH_ACK, m->get_ino(), 0, m->get_cap_id(), m->get_seq(), m->get_caps(), 0, dirty, 0, mds->get_osd_epoch_barrier()), false);
     }
     ack->set_snap_follows(follows);
     ack->set_client_tid(m->get_client_tid());
     mds->send_message_client_counted(ack, m->get_connection());
-    if (m->get_op() == CEPH_CAP_OP_FLUSHSNAP) {
-      m->put();
+    if (op == CEPH_CAP_OP_FLUSHSNAP) {
       return;
     } else {
       // fall-thru because the message may release some caps
-      m->clear_dirty();
-      m->set_op(CEPH_CAP_OP_UPDATE);
+      dirty = false;
+      op = CEPH_CAP_OP_UPDATE;
     }
   }
 
@@ -2749,7 +2732,6 @@ void Locker::handle_client_caps(MClientCaps *m)
      *   - mds receives cap messages from client
      */
     dout(7) << "handle_client_caps on unknown ino " << m->get_ino() << ", dropping" << dendl;
-    m->put();
     return;
   }
 
@@ -2769,7 +2751,6 @@ void Locker::handle_client_caps(MClientCaps *m)
   cap = head_in->get_client_cap(client);
   if (!cap) {
     dout(7) << "handle_client_caps no cap for client." << client << " on " << *head_in << dendl;
-    m->put();
     return;
   }  
   assert(cap);
@@ -2783,12 +2764,10 @@ void Locker::handle_client_caps(MClientCaps *m)
   if (ceph_seq_cmp(m->get_mseq(), cap->get_mseq()) < 0) {
     dout(7) << "handle_client_caps mseq " << m->get_mseq() << " < " << cap->get_mseq()
            << ", dropping" << dendl;
-    m->put();
     return;
   }
 
   bool need_unpin = false;
-  int op = m->get_op();
 
   // flushsnap?
   if (op == CEPH_CAP_OP_FLUSHSNAP) {
@@ -2818,9 +2797,9 @@ void Locker::handle_client_caps(MClientCaps *m)
     // we can prepare the ack now, since this FLUSHEDSNAP is independent of any
     // other cap ops.  (except possibly duplicate FLUSHSNAP requests, but worst
     // case we get a dup response, so whatever.)
-    MClientCaps *ack = 0;
-    if (m->get_dirty()) {
-      ack = new MClientCaps(CEPH_CAP_OP_FLUSHSNAP_ACK, in->ino(), 0, 0, 0, 0, 0, m->get_dirty(), 0, mds->get_osd_epoch_barrier());
+    MClientCaps::ref ack;
+    if (dirty) {
+      ack.reset(new MClientCaps(CEPH_CAP_OP_FLUSHSNAP_ACK, in->ino(), 0, 0, 0, 0, 0, dirty, 0, mds->get_osd_epoch_barrier()), false);
       ack->set_snap_follows(follows);
       ack->set_client_tid(m->get_client_tid());
       ack->set_oldest_flush_tid(m->get_oldest_flush_tid());
@@ -2836,7 +2815,7 @@ void Locker::handle_client_caps(MClientCaps *m)
       if (in == head_in)
        cap->client_follows = snap < CEPH_NOSNAP ? snap : realm->get_newest_seq();
    
-      _do_snap_update(in, snap, m->get_dirty(), follows, client, m, ack);
+      _do_snap_update(in, snap, dirty, follows, client, m, ack);
 
       if (in != head_in)
        head_in->remove_need_snapflush(in, snap, client);
@@ -2857,16 +2836,16 @@ void Locker::handle_client_caps(MClientCaps *m)
       // intermediate snap inodes
       while (in != head_in) {
        assert(in->last != CEPH_NOSNAP);
-       if (in->is_auth() && m->get_dirty()) {
+       if (in->is_auth() && dirty) {
          dout(10) << " updating intermediate snapped inode " << *in << dendl;
-         _do_cap_update(in, NULL, m->get_dirty(), follows, m);
+         _do_cap_update(in, NULL, dirty, follows, m, MClientCaps::ref());
        }
        in = mdcache->pick_inode_snap(head_in, in->last);
       }
     }
  
     // head inode, and cap
-    MClientCaps *ack = 0;
+    MClientCaps::ref ack;
 
     int caps = m->get_caps();
     if (caps & ~cap->issued()) {
@@ -2877,7 +2856,7 @@ void Locker::handle_client_caps(MClientCaps *m)
     cap->confirm_receipt(m->get_seq(), caps);
     dout(10) << " follows " << follows
             << " retains " << ccap_string(m->get_caps())
-            << " dirty " << ccap_string(m->get_dirty())
+            << " dirty " << ccap_string(dirty)
             << " on " << *in << dendl;
 
 
@@ -2900,17 +2879,17 @@ void Locker::handle_client_caps(MClientCaps *m)
     }
 
     bool need_snapflush = cap->need_snapflush();
-    if (m->get_dirty() && in->is_auth()) {
-      dout(7) << " flush client." << client << " dirty " << ccap_string(m->get_dirty())
+    if (dirty && in->is_auth()) {
+      dout(7) << " flush client." << client << " dirty " << ccap_string(dirty)
              << " seq " << m->get_seq() << " on " << *in << dendl;
-      ack = new MClientCaps(CEPH_CAP_OP_FLUSH_ACK, in->ino(), 0, cap->get_cap_id(), m->get_seq(),
-                           m->get_caps(), 0, m->get_dirty(), 0, mds->get_osd_epoch_barrier());
+      ack.reset(new MClientCaps(CEPH_CAP_OP_FLUSH_ACK, in->ino(), 0, cap->get_cap_id(), m->get_seq(),
+          m->get_caps(), 0, dirty, 0, mds->get_osd_epoch_barrier()), false);
       ack->set_client_tid(m->get_client_tid());
       ack->set_oldest_flush_tid(m->get_oldest_flush_tid());
 
       // client flushes and releases caps at the same time. make sure MDCache::cow_inode()
       // properly setup CInode::client_need_snapflush
-      if ((m->get_dirty() & ~cap->issued()) && !need_snapflush)
+      if ((dirty & ~cap->issued()) && !need_snapflush)
        cap->mark_needsnapflush();
     }
 
@@ -2927,7 +2906,7 @@ void Locker::handle_client_caps(MClientCaps *m)
     }
 
     bool updated = in->is_auth() &&
-                  _do_cap_update(in, cap, m->get_dirty(), follows, m, ack, &need_flush);
+                  _do_cap_update(in, cap, dirty, follows, m, ack, &need_flush);
 
     if (cap->need_snapflush() &&
        (!need_snapflush || !(m->flags & MClientCaps::FLAG_PENDING_CAPSNAP)))
@@ -2961,7 +2940,6 @@ void Locker::handle_client_caps(MClientCaps *m)
  out:
   if (need_unpin)
     head_in->auth_unpin(this);
-  m->put();
 }
 
 
@@ -3109,7 +3087,7 @@ void Locker::kick_cap_releases(MDRequestRef& mdr)
 /**
  * m and ack might be NULL, so don't dereference them unless dirty != 0
  */
-void Locker::_do_snap_update(CInode *in, snapid_t snap, int dirty, snapid_t follows, client_t client, MClientCaps *m, MClientCaps *ack)
+void Locker::_do_snap_update(CInode *in, snapid_t snap, int dirty, snapid_t follows, client_t client, const MClientCaps::const_ref &m, const MClientCaps::ref &ack)
 {
   dout(10) << "_do_snap_update dirty " << ccap_string(dirty)
           << " follows " << follows << " snap " << snap
@@ -3194,10 +3172,10 @@ void Locker::_do_snap_update(CInode *in, snapid_t snap, int dirty, snapid_t foll
                                  ack->get_oldest_flush_tid());
 
   mds->mdlog->submit_entry(le, new C_Locker_FileUpdate_finish(this, in, mut, UPDATE_SNAPFLUSH,
-                                                             client, ack));
+                                                             ack, client));
 }
 
-void Locker::_update_cap_fields(CInode *in, int dirty, MClientCaps *m, CInode::mempool_inode *pi)
+void Locker::_update_cap_fields(CInode *in, int dirty, const MClientCaps::const_ref &m, CInode::mempool_inode *pi)
 {
   if (dirty == 0)
     return;
@@ -3296,7 +3274,7 @@ void Locker::_update_cap_fields(CInode *in, int dirty, MClientCaps *m, CInode::m
  */
 bool Locker::_do_cap_update(CInode *in, Capability *cap,
                            int dirty, snapid_t follows,
-                           MClientCaps *m, MClientCaps *ack,
+                           const MClientCaps::const_ref &m, const MClientCaps::ref &ack,
                            bool *need_flush)
 {
   dout(10) << "_do_cap_update dirty " << ccap_string(dirty)
@@ -3396,7 +3374,7 @@ bool Locker::_do_cap_update(CInode *in, Capability *cap,
   if (!dirty && !change_max)
     return false;
 
-  Session *session = mds->get_session(m);
+  Session *session = mds->get_session(boost::static_pointer_cast<Message::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
   if (session->check_access(in, MAY_WRITE,
                            m->caller_uid, m->caller_gid, NULL, 0, 0) < 0) {
     dout(10) << "check_access failed, dropping cap update on " << *in << dendl;
@@ -3462,7 +3440,7 @@ bool Locker::_do_cap_update(CInode *in, Capability *cap,
   if (cap)
     update_flags |= UPDATE_NEEDSISSUE;
   mds->mdlog->submit_entry(le, new C_Locker_FileUpdate_finish(this, in, mut, update_flags,
-                                                             client, ack));
+                                                             ack, client));
   if (need_flush && !*need_flush &&
       ((change_max && new_max) || // max INCREASE
        _need_flush_mdlog(in, dirty)))
@@ -3471,8 +3449,7 @@ bool Locker::_do_cap_update(CInode *in, Capability *cap,
   return true;
 }
 
-/* This function DOES put the passed message before returning */
-void Locker::handle_client_cap_release(MClientCapRelease *m)
+void Locker::handle_client_cap_release(const MClientCapRelease::const_ref &m)
 {
   client_t client = m->get_source().num();
   dout(10) << "handle_client_cap_release " << *m << dendl;
@@ -3492,17 +3469,15 @@ void Locker::handle_client_cap_release(MClientCapRelease *m)
     mds->set_osd_epoch_barrier(m->osd_epoch_barrier);
   }
 
-  Session *session = mds->get_session(m);
+  Session *session = mds->get_session(boost::static_pointer_cast<Message::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
 
-  for (vector<ceph_mds_cap_item>::iterator p = m->caps.begin(); p != m->caps.end(); ++p) {
-    _do_cap_release(client, inodeno_t((uint64_t)p->ino) , p->cap_id, p->migrate_seq, p->seq);
+  for (const auto &cap : m->caps) {
+    _do_cap_release(client, inodeno_t((uint64_t)cap.ino) , cap.cap_id, cap.migrate_seq, cap.seq);
   }
 
   if (session) {
     session->notify_cap_release(m->caps.size());
   }
-
-  m->put();
 }
 
 class C_Locker_RetryCapRelease : public LockerContext {
@@ -3559,8 +3534,6 @@ void Locker::_do_cap_release(client_t client, inodeno_t ino, uint64_t cap_id,
   remove_client_cap(in, client);
 }
 
-/* This function DOES put the passed message before returning */
-
 void Locker::remove_client_cap(CInode *in, client_t client)
 {
   // clean out any pending snapflush state
@@ -3686,7 +3659,7 @@ void Locker::caps_tick()
 }
 
 
-void Locker::handle_client_lease(MClientLease *m)
+void Locker::handle_client_lease(const MClientLease::const_ref &m)
 {
   dout(10) << "handle_client_lease " << *m << dendl;
 
@@ -3696,7 +3669,6 @@ void Locker::handle_client_lease(MClientLease *m)
   CInode *in = mdcache->get_inode(m->get_ino(), m->get_last());
   if (!in) {
     dout(7) << "handle_client_lease don't have ino " << m->get_ino() << "." << m->get_last() << dendl;
-    m->put();
     return;
   }
   CDentry *dn = 0;
@@ -3707,7 +3679,6 @@ void Locker::handle_client_lease(MClientLease *m)
     dn = dir->lookup(m->dname);
   if (!dn) {
     dout(7) << "handle_client_lease don't have dn " << m->get_ino() << " " << m->dname << dendl;
-    m->put();
     return;
   }
   dout(10) << " on " << *dn << dendl;
@@ -3716,7 +3687,6 @@ void Locker::handle_client_lease(MClientLease *m)
   ClientLease *l = dn->get_client_lease(client);
   if (!l) {
     dout(7) << "handle_client_lease didn't have lease for client." << client << " of " << *dn << dendl;
-    m->put();
     return;
   } 
 
@@ -3730,7 +3700,6 @@ void Locker::handle_client_lease(MClientLease *m)
              << " on " << *dn << dendl;
       dn->remove_client_lease(l, this);
     }
-    m->put();
     break;
 
   case CEPH_MDS_LEASE_RENEW:
@@ -3738,16 +3707,17 @@ void Locker::handle_client_lease(MClientLease *m)
       dout(7) << "handle_client_lease client." << client << " renew on " << *dn
              << (!dn->lock.can_lease(client)?", revoking lease":"") << dendl;
       if (dn->lock.can_lease(client)) {
+        MClientLease::ref reply(new MClientLease(*m));
        int pool = 1;   // fixme.. do something smart!
-       m->h.duration_ms = (int)(1000 * mdcache->client_lease_durations[pool]);
-       m->h.seq = ++l->seq;
-       m->clear_payload();
+       reply->h.duration_ms = (int)(1000 * mdcache->client_lease_durations[pool]);
+       reply->h.seq = ++l->seq;
+       reply->clear_payload();
 
        utime_t now = ceph_clock_now();
        now += mdcache->client_lease_durations[pool];
        mdcache->touch_client_lease(l, pool, now);
 
-       mds->send_message_client_counted(m, m->get_connection());
+       mds->send_message_client_counted(reply, m->get_connection());
       }
     }
     break;
@@ -3808,12 +3778,8 @@ void Locker::revoke_client_leases(SimpleLock *lock)
     
     // i should also revoke the dir ICONTENT lease, if they have it!
     CInode *diri = dn->get_dir()->get_inode();
-    mds->send_message_client_counted(new MClientLease(CEPH_MDS_LEASE_REVOKE, l->seq,
-                                             mask,
-                                             diri->ino(),
-                                             diri->first, CEPH_NOSNAP,
-                                             dn->get_name()),
-                            l->client);
+    MClientLease::ref lease(new MClientLease(CEPH_MDS_LEASE_REVOKE, l->seq, mask, diri->ino(), diri->first, CEPH_NOSNAP, dn->get_name()), false);
+    mds->send_message_client_counted(lease, l->client);
   }
 }
 
@@ -3836,7 +3802,7 @@ void Locker::encode_lease(bufferlist& bl, const session_info_t& info,
 
 // locks ----------------------------------------------------------------
 
-SimpleLock *Locker::get_lock(int lock_type, MDSCacheObjectInfo &info) 
+SimpleLock *Locker::get_lock(int lock_type, const MDSCacheObjectInfo &info) 
 {
   switch (lock_type) {
   case CEPH_LOCK_DN:
@@ -3896,8 +3862,7 @@ SimpleLock *Locker::get_lock(int lock_type, MDSCacheObjectInfo &info)
   return 0;  
 }
 
-/* This function DOES put the passed message before returning */
-void Locker::handle_lock(MLock *m)
+void Locker::handle_lock(const MLock::const_ref &m)
 {
   // nobody should be talking to us during recovery.
   assert(mds->is_rejoin() || mds->is_clientreplay() || mds->is_active() || mds->is_stopping());
@@ -3905,7 +3870,6 @@ void Locker::handle_lock(MLock *m)
   SimpleLock *lock = get_lock(m->get_lock_type(), m->get_object_info());
   if (!lock) {
     dout(10) << "don't have object " << m->get_object_info() << ", must have trimmed, dropping" << dendl;
-    m->put();
     return;
   }
 
@@ -3945,7 +3909,7 @@ void Locker::handle_lock(MLock *m)
 
 /** This function may take a reference to m if it needs one, but does
  * not put references. */
-void Locker::handle_reqrdlock(SimpleLock *lock, MLock *m)
+void Locker::handle_reqrdlock(SimpleLock *lock, const MLock::const_ref &m)
 {
   MDSCacheObject *parent = lock->get_parent();
   if (parent->is_auth() &&
@@ -3959,7 +3923,7 @@ void Locker::handle_reqrdlock(SimpleLock *lock, MLock *m)
     } else {
       dout(7) << "handle_reqrdlock delaying request until lock is stable" << dendl;
       lock->add_waiter(SimpleLock::WAIT_STABLE | MDSCacheObject::WAIT_UNFREEZE,
-                       new C_MDS_RetryMessage(mds, m->get()));
+                       new C_MDS_RetryMessage(mds, m));
     }
   } else {
     dout(7) << "handle_reqrdlock dropping rdlock request on " << *lock
@@ -3968,8 +3932,7 @@ void Locker::handle_reqrdlock(SimpleLock *lock, MLock *m)
   }
 }
 
-/* This function DOES put the passed message before returning */
-void Locker::handle_simple_lock(SimpleLock *lock, MLock *m)
+void Locker::handle_simple_lock(SimpleLock *lock, const MLock::const_ref &m)
 {
   int from = m->get_asker();
   
@@ -3980,7 +3943,6 @@ void Locker::handle_simple_lock(SimpleLock *lock, MLock *m)
     if (lock->get_parent()->is_rejoining()) {
       dout(7) << "handle_simple_lock still rejoining " << *lock->get_parent()
              << ", dropping " << *m << dendl;
-      m->put();
       return;
     }
   }
@@ -4027,8 +3989,6 @@ void Locker::handle_simple_lock(SimpleLock *lock, MLock *m)
     break;
 
   }
-
-  m->put();
 }
 
 /* unused, currently.
@@ -4688,9 +4648,10 @@ void Locker::scatter_nudge(ScatterLock *lock, MDSInternalContextBase *c, bool fo
             << *lock << " on " << *p << dendl;
     // request unscatter?
     mds_rank_t auth = lock->get_parent()->authority().first;
-    if (!mds->is_cluster_degraded() ||
-       mds->mdsmap->is_clientreplay_or_active_or_stopping(auth))
-      mds->send_message_mds(new MLock(lock, LOCK_AC_NUDGE, mds->get_nodeid()), auth);
+    if (!mds->is_cluster_degraded() || mds->mdsmap->is_clientreplay_or_active_or_stopping(auth)) {
+      MLock::ref msg(new MLock(lock, LOCK_AC_NUDGE, mds->get_nodeid()), false);
+      mds->send_message_mds(msg, auth);
+    }
 
     // wait...
     if (c)
@@ -5199,8 +5160,7 @@ void Locker::file_recover(ScatterLock *lock)
 
 
 // messenger
-/* This function DOES put the passed message before returning */
-void Locker::handle_file_lock(ScatterLock *lock, MLock *m)
+void Locker::handle_file_lock(ScatterLock *lock, const MLock::const_ref &m)
 {
   CInode *in = static_cast<CInode*>(lock->get_parent());
   int from = m->get_asker();
@@ -5209,7 +5169,6 @@ void Locker::handle_file_lock(ScatterLock *lock, MLock *m)
     if (in->is_rejoining()) {
       dout(7) << "handle_file_lock still rejoining " << *in
              << ", dropping " << *m << dendl;
-      m->put();
       return;
     }
   }
@@ -5420,12 +5379,4 @@ void Locker::handle_file_lock(ScatterLock *lock, MLock *m)
   default:
     ceph_abort();
   }  
-  
-  m->put();
 }
-
-
-
-
-
-
index 74cdb45ac267bea4c2e59a53e1bf1f595513227a..c5fb9d9f3c4b970cf81aa2babde6f1381c9a69ab 100644 (file)
 #ifndef CEPH_MDS_LOCKER_H
 #define CEPH_MDS_LOCKER_H
 
-#include <string_view>
-
 #include "include/types.h"
 
+#include "messages/MClientCaps.h"
+#include "messages/MClientCapRelease.h"
+#include "messages/MClientLease.h"
+#include "messages/MLock.h"
+
 #include <map>
 #include <list>
 #include <set>
+#include <string_view>
 using std::map;
 using std::list;
 using std::set;
@@ -31,10 +35,6 @@ class Session;
 class CDentry;
 struct SnapRealm;
 
-class Message;
-
-class MLock;
-
 class Capability;
 
 class SimpleLock;
@@ -55,10 +55,10 @@ private:
  public:
   Locker(MDSRank *m, MDCache *c);
 
-  SimpleLock *get_lock(int lock_type, MDSCacheObjectInfo &info);
+  SimpleLock *get_lock(int lock_type, const MDSCacheObjectInfo &info);
   
-  void dispatch(Message *m);
-  void handle_lock(MLock *m);
+  void dispatch(const Message::const_ref &m);
+  void handle_lock(const MLock::const_ref &m);
 
   void tick();
 
@@ -137,7 +137,7 @@ public:
   bool simple_rdlock_try(SimpleLock *lock, MDSInternalContextBase *con);
 protected:
   void simple_eval(SimpleLock *lock, bool *need_issue);
-  void handle_simple_lock(SimpleLock *lock, MLock *m);
+  void handle_simple_lock(SimpleLock *lock, const MLock::const_ref &m);
 
 public:
   bool simple_sync(SimpleLock *lock, bool *need_issue=0);
@@ -155,7 +155,7 @@ public:
   void scatter_nudge(ScatterLock *lock, MDSInternalContextBase *c, bool forcelockchange=false);
 
 protected:
-  void handle_scatter_lock(ScatterLock *lock, MLock *m);
+  void handle_scatter_lock(ScatterLock *lock, const MLock::const_ref &m);
   bool scatter_scatter_fastpath(ScatterLock *lock);
   void scatter_scatter(ScatterLock *lock, bool nowait=false);
   void scatter_tempsync(ScatterLock *lock, bool *need_issue=0);
@@ -169,7 +169,7 @@ public:
   void mark_updated_scatterlock(ScatterLock *lock);
 
 
-  void handle_reqrdlock(SimpleLock *lock, MLock *m);
+  void handle_reqrdlock(SimpleLock *lock, const MLock::const_ref &m);
 
 
 
@@ -194,13 +194,13 @@ public:
 protected:
   bool _need_flush_mdlog(CInode *in, int wanted_caps);
   void adjust_cap_wanted(Capability *cap, int wanted, int issue_seq);
-  void handle_client_caps(class MClientCaps *m);
-  void _update_cap_fields(CInode *in, int dirty, MClientCaps *m, CInode::mempool_inode *pi);
-  void _do_snap_update(CInode *in, snapid_t snap, int dirty, snapid_t follows, client_t client, MClientCaps *m, MClientCaps *ack);
+  void handle_client_caps(const MClientCaps::const_ref &m);
+  void _update_cap_fields(CInode *in, int dirty, const MClientCaps::const_ref &m, CInode::mempool_inode *pi);
+  void _do_snap_update(CInode *in, snapid_t snap, int dirty, snapid_t follows, client_t client, const MClientCaps::const_ref &m, const MClientCaps::ref &ack);
   void _do_null_snapflush(CInode *head_in, client_t client, snapid_t last=CEPH_NOSNAP);
-  bool _do_cap_update(CInode *in, Capability *cap, int dirty, snapid_t follows, MClientCaps *m,
-                     MClientCaps *ack=0, bool *need_flush=NULL);
-  void handle_client_cap_release(class MClientCapRelease *m);
+  bool _do_cap_update(CInode *in, Capability *cap, int dirty, snapid_t follows, const MClientCaps::const_ref &m,
+                     const MClientCaps::ref &ack, bool *need_flush=NULL);
+  void handle_client_cap_release(const MClientCapRelease::const_ref &m);
   void _do_cap_release(client_t client, inodeno_t ino, uint64_t cap_id, ceph_seq_t mseq, ceph_seq_t seq);
   void caps_tick();
 
@@ -228,7 +228,7 @@ protected:
 public:
   void file_eval(ScatterLock *lock, bool *need_issue);
 protected:
-  void handle_file_lock(ScatterLock *lock, MLock *m);
+  void handle_file_lock(ScatterLock *lock, const MLock::const_ref &m);
   void scatter_mix(ScatterLock *lock, bool *need_issue=0);
   void file_excl(ScatterLock *lock, bool *need_issue=0);
   void file_xsyn(SimpleLock *lock, bool *need_issue=0);
@@ -256,10 +256,10 @@ public:
 public:
   void request_inode_file_caps(CInode *in);
 protected:
-  void handle_inode_file_caps(class MInodeFileCaps *m);
+  void handle_inode_file_caps(const MInodeFileCaps::const_ref &m);
 
   void file_update_finish(CInode *in, MutationRef& mut, unsigned flags,
-                         client_t client, MClientCaps *ack);
+                         client_t client, const MClientCaps::ref &ack);
 private:
   uint64_t calc_new_max_size(CInode::mempool_inode *pi, uint64_t size);
 public:
@@ -284,7 +284,7 @@ private:
   
   // -- client leases --
 public:
-  void handle_client_lease(struct MClientLease *m);
+  void handle_client_lease(const MClientLease::const_ref &m);
 
   void issue_client_lease(CDentry *dn, client_t client, bufferlist &bl, utime_t now, Session *session);
   void revoke_client_leases(SimpleLock *lock);
index 5a47645288586b98ba3ca64e755f0efafcb35a0c..744756c5332e83c4132a78f36f26ca296b4b26c5 100644 (file)
@@ -27,7 +27,6 @@
 
 #include "include/Context.h"
 #include "msg/Messenger.h"
-#include "messages/MHeartbeat.h"
 
 #include <fstream>
 #include <iostream>
@@ -60,13 +59,12 @@ using std::chrono::duration_cast;
 #define MIN_OFFLOAD 10   // point at which i stop trying, close enough
 
 
-/* This function DOES put the passed message before returning */
-int MDBalancer::proc_message(Message *m)
+int MDBalancer::proc_message(const Message::const_ref &m)
 {
   switch (m->get_type()) {
 
   case MSG_MDS_HEARTBEAT:
-    handle_heartbeat(static_cast<MHeartbeat*>(m));
+    handle_heartbeat(boost::static_pointer_cast<MHeartbeat::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
   default:
@@ -392,23 +390,22 @@ void MDBalancer::send_heartbeat()
 
   set<mds_rank_t> up;
   mds->get_mds_map()->get_up_mds_set(up);
-  for (set<mds_rank_t>::iterator p = up.begin(); p != up.end(); ++p) {
-    if (*p == mds->get_nodeid())
+  for (const auto& r : up) {
+    if (r == mds->get_nodeid())
       continue;
-    MHeartbeat *hb = new MHeartbeat(load, beat_epoch);
+    MHeartbeat::ref hb(new MHeartbeat(load, beat_epoch), false);
     hb->get_import_map() = import_map;
-    messenger->send_to_mds(hb, mds->mdsmap->get_addrs(*p));
+    mds->send_message_mds(hb, r);
   }
 }
 
-/* This function DOES put the passed message before returning */
-void MDBalancer::handle_heartbeat(MHeartbeat *m)
+void MDBalancer::handle_heartbeat(const MHeartbeat::const_ref &m)
 {
   mds_rank_t who = mds_rank_t(m->get_source().num());
   dout(25) << "=== got heartbeat " << m->get_beat() << " from " << m->get_source().num() << " " << m->get_load() << dendl;
 
   if (!mds->is_active())
-    goto out;
+    return;
 
   if (!mds->mdcache->is_open()) {
     dout(10) << "opening root on handle_heartbeat" << dendl;
@@ -418,7 +415,7 @@ void MDBalancer::handle_heartbeat(MHeartbeat *m)
 
   if (mds->is_cluster_degraded()) {
     dout(10) << " degraded, ignoring" << dendl;
-    goto out;
+    return;
   }
 
   if (mds->get_nodeid() != 0 && m->get_beat() > beat_epoch) {
@@ -442,7 +439,7 @@ void MDBalancer::handle_heartbeat(MHeartbeat *m)
   } else if (mds->get_nodeid() == 0) {
     if (beat_epoch != m->get_beat()) {
       dout(10) << " old heartbeat epoch, ignoring" << dendl;
-      goto out;
+      return;
     }
   }
 
@@ -463,7 +460,7 @@ void MDBalancer::handle_heartbeat(MHeartbeat *m)
       /* avoid spamming ceph -w if user does not turn mantle on */
       if (mds->mdsmap->get_balancer() != "") {
         int r = mantle_prep_rebalance();
-        if (!r) goto out;
+        if (!r) return;
        mds->clog->warn() << "using old balancer; mantle failed for "
                           << "balancer=" << mds->mdsmap->get_balancer()
                           << " : " << cpp_strerror(r);
@@ -471,10 +468,6 @@ void MDBalancer::handle_heartbeat(MHeartbeat *m)
       prep_rebalance(m->get_beat());
     }
   }
-
-  // done
- out:
-  m->put();
 }
 
 double MDBalancer::try_match(balance_state_t& state, mds_rank_t ex, double& maxex,
index 1ebda748a81838896685d7771d19980e5754d2a8..e9cb6838a047a89f69e7e5246a1c7dd254b16762 100644 (file)
@@ -26,9 +26,12 @@ using std::map;
 #include "common/Clock.h"
 #include "common/Cond.h"
 
-class MDSMap;
+#include "msg/Message.h"
+#include "messages/MHeartbeat.h"
+
+#include "MDSMap.h"
+
 class MDSRank;
-class Message;
 class MHeartbeat;
 class CInode;
 class CDir;
@@ -47,7 +50,7 @@ public:
                           const std::set <std::string> &changed,
                           const MDSMap &mds_map);
 
-  int proc_message(Message *m);
+  int proc_message(const Message::const_ref &m);
 
   /**
    * Regularly called upkeep function.
@@ -98,7 +101,7 @@ private:
   mds_load_t get_load();
   int localize_balancer();
   void send_heartbeat();
-  void handle_heartbeat(MHeartbeat *m);
+  void handle_heartbeat(const MHeartbeat::const_ref &m);
   void find_exports(CDir *dir,
                     double amount,
                     list<CDir*>& exports,
index ad3ffcd2bf9bc9fafaa7edab9cd2af2fa77b12b9..213301a4e88775519ca50641b83b42e81b58cc38 100644 (file)
@@ -42,6 +42,8 @@
 #include "include/filepath.h"
 #include "include/util.h"
 
+#include "messages/MClientCaps.h"
+
 #include "msg/Message.h"
 #include "msg/Messenger.h"
 
 #include "events/ECommitted.h"
 #include "events/ESessions.h"
 
-#include "messages/MGenericMessage.h"
-
-#include "messages/MMDSResolve.h"
-#include "messages/MMDSResolveAck.h"
-#include "messages/MMDSCacheRejoin.h"
-
-#include "messages/MDiscover.h"
-#include "messages/MDiscoverReply.h"
-
-//#include "messages/MInodeUpdate.h"
-#include "messages/MDirUpdate.h"
-#include "messages/MCacheExpire.h"
-
-#include "messages/MInodeFileCaps.h"
-
-#include "messages/MLock.h"
-#include "messages/MDentryLink.h"
-#include "messages/MDentryUnlink.h"
-
-#include "messages/MMDSFindIno.h"
-#include "messages/MMDSFindInoReply.h"
-
-#include "messages/MMDSOpenIno.h"
-#include "messages/MMDSOpenInoReply.h"
-
-#include "messages/MClientRequest.h"
-#include "messages/MClientCaps.h"
-#include "messages/MClientSnap.h"
-#include "messages/MClientQuota.h"
-
-#include "messages/MMDSSlaveRequest.h"
-
-#include "messages/MMDSFragmentNotify.h"
-#include "messages/MMDSSnapUpdate.h"
-
-#include "messages/MGatherCaps.h"
-
 #include "InoTable.h"
 
 #include "common/Timer.h"
@@ -819,7 +784,7 @@ CDentry *MDCache::get_or_create_stray_dentry(CInode *in)
 
 
 
-MDSCacheObject *MDCache::get_object(MDSCacheObjectInfo &info) 
+MDSCacheObject *MDCache::get_object(const MDSCacheObjectInfo &info)
 {
   // inode?
   if (info.ino) 
@@ -1025,7 +990,7 @@ void MDCache::eval_subtree_root(CInode *diri)
 }
 
 
-void MDCache::adjust_bounded_subtree_auth(CDir *dir, set<CDir*>& bounds, mds_authority_t auth)
+void MDCache::adjust_bounded_subtree_auth(CDir *dir, const set<CDir*>& bounds, mds_authority_t auth)
 {
   dout(7) << "adjust_bounded_subtree_auth " << dir->get_dir_auth() << " -> " << auth
          << " on " << *dir
@@ -1090,11 +1055,7 @@ void MDCache::adjust_bounded_subtree_auth(CDir *dir, set<CDir*>& bounds, mds_aut
   // - these may be new, or
   // - beneath existing ambiguous bounds (which will be collapsed),
   // - but NOT beneath unambiguous bounds.
-  for (set<CDir*>::iterator p = bounds.begin();
-       p != bounds.end();
-       ++p) {
-    CDir *bound = *p;
-    
+  for (const auto& bound : bounds) {
     // new bound?
     if (subtrees[dir].count(bound) == 0) {
       if (get_subtree_root(bound) == dir) {
@@ -1158,14 +1119,15 @@ void MDCache::adjust_bounded_subtree_auth(CDir *dir, set<CDir*>& bounds, mds_aut
  * fragmentation as necessary to get an equivalent bounding set.  That is, only
  * split if one of our frags spans the provided bounding set.  Never merge.
  */
-void MDCache::get_force_dirfrag_bound_set(vector<dirfrag_t>& dfs, set<CDir*>& bounds)
+void MDCache::get_force_dirfrag_bound_set(const vector<dirfrag_t>& dfs, set<CDir*>& bounds)
 {
   dout(10) << "get_force_dirfrag_bound_set " << dfs << dendl;
 
   // sort by ino
   map<inodeno_t, fragset_t> byino;
-  for (vector<dirfrag_t>::iterator p = dfs.begin(); p != dfs.end(); ++p)
-    byino[p->ino].insert(p->frag);
+  for (auto& frag : dfs) {
+    byino[frag.ino].insert(frag.frag);
+  }
   dout(10) << " by ino: " << byino << dendl;
 
   for (map<inodeno_t,fragset_t>::iterator p = byino.begin(); p != byino.end(); ++p) {
@@ -1209,7 +1171,7 @@ void MDCache::get_force_dirfrag_bound_set(vector<dirfrag_t>& dfs, set<CDir*>& bo
   }
 }
 
-void MDCache::adjust_bounded_subtree_auth(CDir *dir, vector<dirfrag_t>& bound_dfs, const mds_authority_t &auth)
+void MDCache::adjust_bounded_subtree_auth(CDir *dir, const vector<dirfrag_t>& bound_dfs, const mds_authority_t &auth)
 {
   dout(7) << "adjust_bounded_subtree_auth " << dir->get_dir_auth() << " -> " << auth
          << " on " << *dir << " bound_dfs " << bound_dfs << dendl;
@@ -1219,14 +1181,15 @@ void MDCache::adjust_bounded_subtree_auth(CDir *dir, vector<dirfrag_t>& bound_df
   adjust_bounded_subtree_auth(dir, bounds, auth);
 }
 
-void MDCache::map_dirfrag_set(list<dirfrag_t>& dfs, set<CDir*>& result)
+void MDCache::map_dirfrag_set(const list<dirfrag_t>& dfs, set<CDir*>& result)
 {
   dout(10) << "map_dirfrag_set " << dfs << dendl;
 
   // group by inode
   map<inodeno_t, fragset_t> ino_fragset;
-  for (list<dirfrag_t>::iterator p = dfs.begin(); p != dfs.end(); ++p)
-    ino_fragset[p->ino].insert(p->frag);
+  for (const auto &df : dfs) {
+    ino_fragset[df.ino].insert(df.frag);
+  }
 
   // get frags
   for (map<inodeno_t, fragset_t>::iterator p = ino_fragset.begin();
@@ -2768,13 +2731,13 @@ void MDCache::send_slave_resolves()
 {
   dout(10) << "send_slave_resolves" << dendl;
 
-  map<mds_rank_t, MMDSResolve*> resolves;
+  map<mds_rank_t, MMDSResolve::ref> resolves;
 
   if (mds->is_resolve()) {
     for (map<mds_rank_t, map<metareqid_t, MDSlaveUpdate*> >::iterator p = uncommitted_slave_updates.begin();
         p != uncommitted_slave_updates.end();
         ++p) {
-      resolves[p->first] = new MMDSResolve;
+      resolves[p->first].reset(new MMDSResolve, false);
       for (map<metareqid_t, MDSlaveUpdate*>::iterator q = p->second.begin();
           q != p->second.end();
           ++q) {
@@ -2798,7 +2761,7 @@ void MDCache::send_slave_resolves()
       if (resolve_set.count(master) || is_ambiguous_slave_update(p->first, master)) {
        dout(10) << " including uncommitted " << *mdr << dendl;
        if (!resolves.count(master))
-         resolves[master] = new MMDSResolve;
+         resolves[master].reset(new MMDSResolve, false);
        if (!mdr->committing &&
            mdr->has_more() && mdr->more()->is_inode_exporter) {
          // re-send cap exports
@@ -2816,12 +2779,10 @@ void MDCache::send_slave_resolves()
     }
   }
 
-  for (map<mds_rank_t, MMDSResolve*>::iterator p = resolves.begin();
-       p != resolves.end();
-       ++p) {
-    dout(10) << "sending slave resolve to mds." << p->first << dendl;
-    mds->send_message_mds(p->second, p->first);
-    resolve_ack_gather.insert(p->first);
+  for (auto &p : resolves) {
+    dout(10) << "sending slave resolve to mds." << p.first << dendl;
+    mds->send_message_mds(p.second, p.first);
+    resolve_ack_gather.insert(p.first);
   }
 }
 
@@ -2837,14 +2798,14 @@ void MDCache::send_subtree_resolves()
     return;  // not now
   }
 
-  map<mds_rank_t, MMDSResolve*> resolves;
+  map<mds_rank_t, MMDSResolve::ref> resolves;
   for (set<mds_rank_t>::iterator p = recovery_set.begin();
        p != recovery_set.end();
        ++p) {
     if (*p == mds->get_nodeid())
       continue;
     if (mds->is_resolve() || mds->mdsmap->is_resolve(*p))
-      resolves[*p] = new MMDSResolve;
+      resolves[*p].reset(new MMDSResolve, false);
   }
 
   map<dirfrag_t, vector<dirfrag_t> > my_subtrees;
@@ -2875,10 +2836,9 @@ void MDCache::send_subtree_resolves()
       dout(10) << " ambig " << dir->dirfrag() << " " << dfls << dendl;
     } else {
       // not ambiguous.
-      for (map<mds_rank_t, MMDSResolve*>::iterator q = resolves.begin();
-          q != resolves.end();
-          ++q)
-       resolves[q->first]->add_subtree(dir->dirfrag());
+      for (auto &q : resolves) {
+       resolves[q.first]->add_subtree(dir->dirfrag());
+      }
       // bounds too
       vector<dirfrag_t> dfls;
       for (set<CDir*>::iterator q = subtrees[dir].begin();
@@ -2920,10 +2880,8 @@ void MDCache::send_subtree_resolves()
   }
 
   // send
-  for (map<mds_rank_t, MMDSResolve*>::iterator p = resolves.begin();
-       p != resolves.end();
-       ++p) {
-    MMDSResolve* m = p->second;
+  for (auto &p : resolves) {
+    const MMDSResolve::ref &m = p.second;
     if (mds->is_resolve()) {
       m->add_table_commits(TABLE_SNAP, resolve_snapclient_commits);
     } else {
@@ -2931,8 +2889,8 @@ void MDCache::send_subtree_resolves()
     }
     m->subtrees = my_subtrees;
     m->ambiguous_imports = my_ambig_imports;
-    dout(10) << "sending subtee resolve to mds." << p->first << dendl;
-    mds->send_message_mds(m, p->first);
+    dout(10) << "sending subtee resolve to mds." << p.first << dendl;
+    mds->send_message_mds(m, p.first);
   }
   resolves_pending = false;
 }
@@ -2991,7 +2949,6 @@ void MDCache::handle_mds_failure(mds_rank_t who)
          assert(mdr->more()->srcdn_auth_mds == mds->get_nodeid());
          // will rollback, no need to wait
          if (mdr->slave_request) {
-           mdr->slave_request->put();
            mdr->slave_request = 0;
          }
          mdr->more()->waiting_on_slave.clear();
@@ -3022,7 +2979,7 @@ void MDCache::handle_mds_failure(mds_rank_t who)
        add_ambiguous_slave_update(p->first, mdr->slave_to_mds);
       }
     } else if (mdr->slave_request) {
-      MMDSSlaveRequest *slave_req = mdr->slave_request;
+      const MMDSSlaveRequest::const_ref &slave_req = mdr->slave_request;
       // FIXME: Slave rename request can arrive after we notice mds failure.
       //       This can cause mds to crash (does not affect integrity of FS).
       if (slave_req->get_op() == MMDSSlaveRequest::OP_RENAMEPREP &&
@@ -3195,7 +3152,7 @@ void MDCache::set_recovery_set(set<mds_rank_t>& s)
  *
  * This functions puts the passed message before returning
  */
-void MDCache::handle_resolve(MMDSResolve *m)
+void MDCache::handle_resolve(const MMDSResolve::const_ref &m)
 {
   dout(7) << "handle_resolve from " << m->get_source() << dendl;
   mds_rank_t from = mds_rank_t(m->get_source().num());
@@ -3206,7 +3163,6 @@ void MDCache::handle_resolve(MMDSResolve *m)
       return;
     }
     // wait until we reach the resolve stage!
-    m->put();
     return;
   }
 
@@ -3229,26 +3185,26 @@ void MDCache::handle_resolve(MMDSResolve *m)
       }
     }
 
-    MMDSResolveAck *ack = new MMDSResolveAck;
-    for (auto p = m->slave_requests.begin(); p != m->slave_requests.end(); ++p) {
-      if (uncommitted_masters.count(p->first)) {  //mds->sessionmap.have_completed_request(p->first)) {
+    MMDSResolveAck::ref ack(new MMDSResolveAck, false);
+    for (const auto &p : m->slave_requests) {
+      if (uncommitted_masters.count(p.first)) {  //mds->sessionmap.have_completed_request(p.first)) {
        // COMMIT
-       if (p->second.committing) {
+       if (p.second.committing) {
          // already committing, waiting for the OP_COMMITTED slave reply
-         dout(10) << " already committing slave request " << *p << " noop "<< dendl;
+         dout(10) << " already committing slave request " << p << " noop "<< dendl;
        } else {
-         dout(10) << " ambiguous slave request " << *p << " will COMMIT" << dendl;
-         ack->add_commit(p->first);
+         dout(10) << " ambiguous slave request " << p << " will COMMIT" << dendl;
+         ack->add_commit(p.first);
        }
-       uncommitted_masters[p->first].slaves.insert(from);   // wait for slave OP_COMMITTED before we log ECommitted
+       uncommitted_masters[p.first].slaves.insert(from);   // wait for slave OP_COMMITTED before we log ECommitted
 
-       if (p->second.inode_caps.length() > 0) {
+       if (p.second.inode_caps.length() > 0) {
          // slave wants to export caps (rename)
          assert(mds->is_resolve());
 
          inodeno_t ino;
          map<client_t,Capability::Export> cap_exports;
-         auto q = p->second.inode_caps.cbegin();
+         auto q = p.second.inode_caps.cbegin();
          decode(ino, q);
          decode(cap_exports, q);
 
@@ -3272,17 +3228,16 @@ void MDCache::handle_resolve(MMDSResolve *m)
          rejoin_slave_exports[ino].second.swap(cap_exports);
 
          // send information of imported caps back to slave
-         encode(rejoin_imported_caps[from][ino], ack->commit[p->first]);
+         encode(rejoin_imported_caps[from][ino], ack->commit[p.first]);
        }
       } else {
        // ABORT
-       dout(10) << " ambiguous slave request " << *p << " will ABORT" << dendl;
-       assert(!p->second.committing);
-       ack->add_abort(p->first);
+       dout(10) << " ambiguous slave request " << p << " will ABORT" << dendl;
+       assert(!p.second.committing);
+       ack->add_abort(p.first);
       }
     }
     mds->send_message(ack, m->get_connection());
-    m->put();
     return;
   }
 
@@ -3310,17 +3265,15 @@ void MDCache::handle_resolve(MMDSResolve *m)
        
        // check if sender claims the subtree
        bool claimed_by_sender = false;
-       for (map<dirfrag_t, vector<dirfrag_t> >::iterator q = m->subtrees.begin();
-            q != m->subtrees.end();
-            ++q) {
+       for (const auto &q : m->subtrees) {
          // an ambiguous import won't race with a refragmentation; it's appropriate to force here.
-         CDir *base = get_force_dirfrag(q->first, false);
+         CDir *base = get_force_dirfrag(q.first, false);
          if (!base || !base->contains(dir)) 
            continue;  // base not dir or an ancestor of dir, clearly doesn't claim dir.
 
          bool inside = true;
          set<CDir*> bounds;
-         get_force_dirfrag_bound_set(q->second, bounds);
+         get_force_dirfrag_bound_set(q.second, bounds);
          for (set<CDir*>::iterator p = bounds.begin(); p != bounds.end(); ++p) {
            CDir *bound = *p;
            if (bound->contains(dir)) {
@@ -3348,34 +3301,30 @@ void MDCache::handle_resolve(MMDSResolve *m)
   // update my dir_auth values
   //   need to do this on recoverying nodes _and_ bystanders (to resolve ambiguous
   //   migrations between other nodes)
-  for (map<dirfrag_t, vector<dirfrag_t> >::iterator pi = m->subtrees.begin();
-       pi != m->subtrees.end();
-       ++pi) {
-    dout(10) << "peer claims " << pi->first << " bounds " << pi->second << dendl;
-    CDir *dir = get_force_dirfrag(pi->first, !survivor);
+  for (const auto& p : m->subtrees) {
+    dout(10) << "peer claims " << p.first << " bounds " << p.second << dendl;
+    CDir *dir = get_force_dirfrag(p.first, !survivor);
     if (!dir)
       continue;
-    adjust_bounded_subtree_auth(dir, pi->second, from);
+    adjust_bounded_subtree_auth(dir, p.second, from);
     try_subtree_merge(dir);
   }
 
   show_subtrees();
 
   // note ambiguous imports too
-  for (map<dirfrag_t, vector<dirfrag_t> >::iterator pi = m->ambiguous_imports.begin();
-       pi != m->ambiguous_imports.end();
-       ++pi) {
-    dout(10) << "noting ambiguous import on " << pi->first << " bounds " << pi->second << dendl;
-    other_ambiguous_imports[from][pi->first].swap( pi->second );
+  for (const auto& p : m->ambiguous_imports) {
+    dout(10) << "noting ambiguous import on " << p.first << " bounds " << p.second << dendl;
+    other_ambiguous_imports[from][p.first] = p.second;
   }
 
   // learn other mds' pendina snaptable commits. later when resolve finishes, we will reload
   // snaptable cache from snapserver. By this way, snaptable cache get synced among all mds
-  for (auto p : m->table_clients) {
+  for (const auto& p : m->table_clients) {
     dout(10) << " noting " << get_mdstable_name(p.type)
             << " pending_commits " << p.pending_commits << dendl;
     MDSTableClient *client = mds->get_table_client(p.type);
-    for (auto q : p.pending_commits)
+    for (const auto& q : p.pending_commits)
       client->notify_commit(q);
   }
   
@@ -3383,25 +3332,21 @@ void MDCache::handle_resolve(MMDSResolve *m)
   resolve_gather.erase(from);
   
   maybe_resolve_finish();
-
-  m->put();
 }
 
 void MDCache::process_delayed_resolve()
 {
   dout(10) << "process_delayed_resolve" << dendl;
-  map<mds_rank_t, MMDSResolve*> tmp;
+  map<mds_rank_t, MMDSResolve::const_ref> tmp;
   tmp.swap(delayed_resolve);
-  for (map<mds_rank_t, MMDSResolve*>::iterator p = tmp.begin(); p != tmp.end(); ++p)
-    handle_resolve(p->second);
+  for (auto &p : tmp) {
+    handle_resolve(p.second);
+  }
 }
 
 void MDCache::discard_delayed_resolve(mds_rank_t who)
 {
-  if (delayed_resolve.count(who)) {
-      delayed_resolve[who]->put();
-      delayed_resolve.erase(who);
-  }
+  delayed_resolve.erase(who);
 }
 
 void MDCache::maybe_resolve_finish()
@@ -3430,15 +3375,13 @@ void MDCache::maybe_resolve_finish()
   }
 }
 
-/* This functions puts the passed message before returning */
-void MDCache::handle_resolve_ack(MMDSResolveAck *ack)
+void MDCache::handle_resolve_ack(const MMDSResolveAck::const_ref &ack)
 {
   dout(10) << "handle_resolve_ack " << *ack << " from " << ack->get_source() << dendl;
   mds_rank_t from = mds_rank_t(ack->get_source().num());
 
   if (!resolve_ack_gather.count(from) ||
       mds->mdsmap->get_state(from) < MDSMap::STATE_RESOLVE) {
-    ack->put();
     return;
   }
 
@@ -3447,46 +3390,42 @@ void MDCache::handle_resolve_ack(MMDSResolveAck *ack)
     assert(mds->is_clientreplay() || mds->is_active() || mds->is_stopping());
   }
 
-  for (map<metareqid_t, bufferlist>::iterator p = ack->commit.begin();
-       p != ack->commit.end();
-       ++p) {
-    dout(10) << " commit on slave " << p->first << dendl;
+  for (const auto &p : ack->commit) {
+    dout(10) << " commit on slave " << p.first << dendl;
     
     if (ambiguous_slave_updates.count(from)) {
-      remove_ambiguous_slave_update(p->first, from);
+      remove_ambiguous_slave_update(p.first, from);
       continue;
     }
 
     if (mds->is_resolve()) {
       // replay
-      MDSlaveUpdate *su = get_uncommitted_slave_update(p->first, from);
+      MDSlaveUpdate *su = get_uncommitted_slave_update(p.first, from);
       assert(su);
 
       // log commit
-      mds->mdlog->start_submit_entry(new ESlaveUpdate(mds->mdlog, "unknown", p->first, from,
+      mds->mdlog->start_submit_entry(new ESlaveUpdate(mds->mdlog, "unknown", p.first, from,
                                                      ESlaveUpdate::OP_COMMIT, su->origop),
-                                    new C_MDC_SlaveCommit(this, from, p->first));
+                                    new C_MDC_SlaveCommit(this, from, p.first));
       mds->mdlog->flush();
 
-      finish_uncommitted_slave_update(p->first, from);
+      finish_uncommitted_slave_update(p.first, from);
     } else {
-      MDRequestRef mdr = request_get(p->first);
+      MDRequestRef mdr = request_get(p.first);
       // information about master imported caps
-      if (p->second.length() > 0)
-       mdr->more()->inode_import.claim(p->second);
+      if (p.second.length() > 0)
+       mdr->more()->inode_import.share(p.second);
 
       assert(mdr->slave_request == 0);  // shouldn't be doing anything!
       request_finish(mdr);
     }
   }
 
-  for (vector<metareqid_t>::iterator p = ack->abort.begin();
-       p != ack->abort.end();
-       ++p) {
-    dout(10) << " abort on slave " << *p << dendl;
+  for (const auto &metareq : ack->abort) {
+    dout(10) << " abort on slave " << metareq << dendl;
 
     if (mds->is_resolve()) {
-      MDSlaveUpdate *su = get_uncommitted_slave_update(*p, from);
+      MDSlaveUpdate *su = get_uncommitted_slave_update(metareq, from);
       assert(su);
 
       // perform rollback (and journal a rollback entry)
@@ -3506,11 +3445,11 @@ void MDCache::handle_resolve_ack(MMDSResolveAck *ack)
        ceph_abort();
       }
     } else {
-      MDRequestRef mdr = request_get(*p);
+      MDRequestRef mdr = request_get(metareq);
       mdr->aborted = true;
       if (mdr->slave_request) {
        if (mdr->slave_did_prepare()) // journaling slave prepare ?
-         add_rollback(*p, from);
+         add_rollback(metareq, from);
       } else {
        request_finish(mdr);
       }
@@ -3521,8 +3460,6 @@ void MDCache::handle_resolve_ack(MMDSResolveAck *ack)
     resolve_ack_gather.erase(from);
     maybe_finish_slave_resolve();
   }
-
-  ack->put();
 }
 
 void MDCache::add_uncommitted_slave_update(metareqid_t reqid, mds_rank_t master, MDSlaveUpdate *su)
@@ -3772,9 +3709,7 @@ void MDCache::remove_inode_recursive(CInode *in)
   remove_inode(in);
 }
 
-bool MDCache::expire_recursive(
-  CInode *in,
-  map<mds_rank_t, MCacheExpire*>& expiremap)
+bool MDCache::expire_recursive(CInode *in, expiremap &expiremap)
 {
   assert(!in->is_auth());
 
@@ -4044,7 +3979,7 @@ void MDCache::rejoin_send_rejoins()
     disambiguate_other_imports();
   }
 
-  map<mds_rank_t, MMDSCacheRejoin*> rejoins;
+  map<mds_rank_t, MMDSCacheRejoin::ref> rejoins;
 
 
   // if i am rejoining, send a rejoin to everyone.
@@ -4055,9 +3990,9 @@ void MDCache::rejoin_send_rejoins()
     if (*p == mds->get_nodeid())  continue;  // nothing to myself!
     if (rejoin_sent.count(*p)) continue;     // already sent a rejoin to this node!
     if (mds->is_rejoin())
-      rejoins[*p] = new MMDSCacheRejoin(MMDSCacheRejoin::OP_WEAK);
+      rejoins[*p].reset(new MMDSCacheRejoin(MMDSCacheRejoin::OP_WEAK), false);
     else if (mds->mdsmap->is_rejoin(*p))
-      rejoins[*p] = new MMDSCacheRejoin(MMDSCacheRejoin::OP_STRONG);
+      rejoins[*p].reset(new MMDSCacheRejoin(MMDSCacheRejoin::OP_STRONG), false);
   }
 
   if (mds->is_rejoin()) {
@@ -4107,26 +4042,24 @@ void MDCache::rejoin_send_rejoins()
   }
   
   // rejoin root inodes, too
-  for (map<mds_rank_t, MMDSCacheRejoin*>::iterator p = rejoins.begin();
-       p != rejoins.end();
-       ++p) {
+  for (auto &p : rejoins) {
     if (mds->is_rejoin()) {
       // weak
-      if (p->first == 0 && root) {
-       p->second->add_weak_inode(root->vino());
+      if (p.first == 0 && root) {
+       p.second->add_weak_inode(root->vino());
        if (root->is_dirty_scattered()) {
          dout(10) << " sending scatterlock state on root " << *root << dendl;
-         p->second->add_scatterlock_state(root);
+         p.second->add_scatterlock_state(root);
        }
       }
-      if (CInode *in = get_inode(MDS_INO_MDSDIR(p->first))) { 
+      if (CInode *in = get_inode(MDS_INO_MDSDIR(p.first))) { 
        if (in)
-         p->second->add_weak_inode(in->vino());
+         p.second->add_weak_inode(in->vino());
       }
     } else {
       // strong
-      if (p->first == 0 && root) {
-       p->second->add_strong_inode(root->vino(),
+      if (p.first == 0 && root) {
+       p.second->add_strong_inode(root->vino(),
                                    root->get_replica_nonce(),
                                    root->get_caps_wanted(),
                                    root->filelock.get_state(),
@@ -4135,12 +4068,12 @@ void MDCache::rejoin_send_rejoins()
        root->state_set(CInode::STATE_REJOINING);
        if (root->is_dirty_scattered()) {
          dout(10) << " sending scatterlock state on root " << *root << dendl;
-         p->second->add_scatterlock_state(root);
+         p.second->add_scatterlock_state(root);
        }
       }
 
-      if (CInode *in = get_inode(MDS_INO_MDSDIR(p->first))) {
-       p->second->add_strong_inode(in->vino(),
+      if (CInode *in = get_inode(MDS_INO_MDSDIR(p.first))) {
+       p.second->add_strong_inode(in->vino(),
                                    in->get_replica_nonce(),
                                    in->get_caps_wanted(),
                                    in->filelock.get_state(),
@@ -4167,7 +4100,7 @@ void MDCache::rejoin_send_rejoins()
        if (!q->first->is_auth()) {
          assert(q->second == q->first->authority().first);
          if (rejoins.count(q->second) == 0) continue;
-         MMDSCacheRejoin *rejoin = rejoins[q->second];
+         const MMDSCacheRejoin::ref &rejoin = rejoins[q->second];
          
          dout(15) << " " << *mdr << " authpin on " << *q->first << dendl;
          MDSCacheObjectInfo i;
@@ -4190,7 +4123,7 @@ void MDCache::rejoin_send_rejoins()
        if (!(*q)->get_parent()->is_auth()) {
          mds_rank_t who = (*q)->get_parent()->authority().first;
          if (rejoins.count(who) == 0) continue;
-         MMDSCacheRejoin *rejoin = rejoins[who];
+         const MMDSCacheRejoin::ref &rejoin = rejoins[who];
          
          dout(15) << " " << *mdr << " xlock on " << **q << " " << *(*q)->get_parent() << dendl;
          MDSCacheObjectInfo i;
@@ -4209,7 +4142,7 @@ void MDCache::rejoin_send_rejoins()
           ++q) {
        mds_rank_t who = q->second;
        if (rejoins.count(who) == 0) continue;
-       MMDSCacheRejoin *rejoin = rejoins[who];
+       const MMDSCacheRejoin::ref &rejoin = rejoins[who];
 
        dout(15) << " " << *mdr << " wrlock on " << q->second
                 << " " << q->first->get_parent() << dendl;
@@ -4223,14 +4156,12 @@ void MDCache::rejoin_send_rejoins()
   }
 
   // send the messages
-  for (map<mds_rank_t,MMDSCacheRejoin*>::iterator p = rejoins.begin();
-       p != rejoins.end();
-       ++p) {
-    assert(rejoin_sent.count(p->first) == 0);
-    assert(rejoin_ack_gather.count(p->first) == 0);
-    rejoin_sent.insert(p->first);
-    rejoin_ack_gather.insert(p->first);
-    mds->send_message_mds(p->second, p->first);
+  for (auto &p : rejoins) {
+    assert(rejoin_sent.count(p.first) == 0);
+    assert(rejoin_ack_gather.count(p.first) == 0);
+    rejoin_sent.insert(p.first);
+    rejoin_ack_gather.insert(p.first);
+    mds->send_message_mds(p.second, p.first);
   }
   rejoin_ack_gather.insert(mds->get_nodeid());   // we need to complete rejoin_gather_finish, too
   rejoins_pending = false;
@@ -4258,7 +4189,7 @@ void MDCache::rejoin_send_rejoins()
  *  strong dentries (no connectivity!)
  *  strong inodes
  */
-void MDCache::rejoin_walk(CDir *dir, MMDSCacheRejoin *rejoin)
+void MDCache::rejoin_walk(CDir *dir, const MMDSCacheRejoin::ref &rejoin)
 {
   dout(10) << "rejoin_walk " << *dir << dendl;
 
@@ -4361,9 +4292,8 @@ void MDCache::rejoin_walk(CDir *dir, MMDSCacheRejoin *rejoin)
  *
  * if i am active|stopping, 
  *  - remove source from replica list for everything not referenced here.
- * This function puts the passed message before returning.
  */
-void MDCache::handle_cache_rejoin(MMDSCacheRejoin *m)
+void MDCache::handle_cache_rejoin(const MMDSCacheRejoin::const_ref &m)
 {
   dout(7) << "handle_cache_rejoin " << *m << " from " << m->get_source() 
          << " (" << m->get_payload().length() << " bytes)"
@@ -4383,7 +4313,6 @@ void MDCache::handle_cache_rejoin(MMDSCacheRejoin *m)
   default: 
     ceph_abort();
   }
-  m->put();
 }
 
 
@@ -4399,14 +4328,13 @@ void MDCache::handle_cache_rejoin(MMDSCacheRejoin *m)
  *  - may have incorrect (out of date) dentry/inode linkage
  *  - may have deleted/purged inodes
  * and i may have to go to disk to get accurate inode contents.  yuck.
- * This functions DOES NOT put the passed message before returning
  */
-void MDCache::handle_cache_rejoin_weak(MMDSCacheRejoin *weak)
+void MDCache::handle_cache_rejoin_weak(const MMDSCacheRejoin::const_ref &weak)
 {
   mds_rank_t from = mds_rank_t(weak->get_source().num());
 
   // possible response(s)
-  MMDSCacheRejoin *ack = 0;      // if survivor
+  MMDSCacheRejoin::ref ack;      // if survivor
   set<vinodeno_t> acked_inodes;  // if survivor
   set<SimpleLock *> gather_locks;  // if survivor
   bool survivor = false;  // am i a survivor?
@@ -4414,7 +4342,7 @@ void MDCache::handle_cache_rejoin_weak(MMDSCacheRejoin *weak)
   if (mds->is_clientreplay() || mds->is_active() || mds->is_stopping()) {
     survivor = true;
     dout(10) << "i am a surivivor, and will ack immediately" << dendl;
-    ack = new MMDSCacheRejoin(MMDSCacheRejoin::OP_ACK);
+    ack.reset(new MMDSCacheRejoin(MMDSCacheRejoin::OP_ACK), false);
 
     map<inodeno_t,map<client_t,Capability::Import> > imported_caps;
 
@@ -4462,14 +4390,12 @@ void MDCache::handle_cache_rejoin_weak(MMDSCacheRejoin *weak)
   }
 
   // assimilate any potentially dirty scatterlock state
-  for (map<inodeno_t,MMDSCacheRejoin::lock_bls>::iterator p = weak->inode_scatterlocks.begin();
-       p != weak->inode_scatterlocks.end();
-       ++p) {
-    CInode *in = get_inode(p->first);
+  for (const auto &p : weak->inode_scatterlocks) {
+    CInode *in = get_inode(p.first);
     assert(in);
-    in->decode_lock_state(CEPH_LOCK_IFILE, p->second.file);
-    in->decode_lock_state(CEPH_LOCK_INEST, p->second.nest);
-    in->decode_lock_state(CEPH_LOCK_IDFT, p->second.dft);
+    in->decode_lock_state(CEPH_LOCK_IFILE, p.second.file);
+    in->decode_lock_state(CEPH_LOCK_INEST, p.second.nest);
+    in->decode_lock_state(CEPH_LOCK_IDFT, p.second.dft);
     if (!survivor)
       rejoin_potential_updated_scatterlocks.insert(in);
   }
@@ -4480,36 +4406,34 @@ void MDCache::handle_cache_rejoin_weak(MMDSCacheRejoin *weak)
   
   // walk weak map
   set<CDir*> dirs_to_share;
-  for (set<dirfrag_t>::iterator p = weak->weak_dirfrags.begin();
-       p != weak->weak_dirfrags.end();
-       ++p) {
-    CInode *diri = get_inode(p->ino);
+  for (const auto &p : weak->weak_dirfrags) {
+    CInode *diri = get_inode(p.ino);
     if (!diri)
-      dout(0) << " missing dir ino " << p->ino << dendl;
+      dout(0) << " missing dir ino " << p.ino << dendl;
     assert(diri);
 
     list<frag_t> ls;
-    if (diri->dirfragtree.is_leaf(p->frag)) {
-      ls.push_back(p->frag);
+    if (diri->dirfragtree.is_leaf(p.frag)) {
+      ls.push_back(p.frag);
     } else {
-      diri->dirfragtree.get_leaves_under(p->frag, ls);
+      diri->dirfragtree.get_leaves_under(p.frag, ls);
       if (ls.empty())
-       ls.push_back(diri->dirfragtree[p->frag.value()]);
+       ls.push_back(diri->dirfragtree[p.frag.value()]);
     }
     for (list<frag_t>::iterator q = ls.begin(); q != ls.end(); ++q) {
       frag_t fg = *q;
       CDir *dir = diri->get_dirfrag(fg);
       if (!dir) {
-       dout(0) << " missing dir for " << p->frag << " (which maps to " << fg << ") on " << *diri << dendl;
+       dout(0) << " missing dir for " << p.frag << " (which maps to " << fg << ") on " << *diri << dendl;
        continue;
       }
       assert(dir);
       if (dirs_to_share.count(dir)) {
-       dout(10) << " already have " << p->frag << " -> " << fg << " " << *dir << dendl;
+       dout(10) << " already have " << p.frag << " -> " << fg << " " << *dir << dendl;
       } else {
        dirs_to_share.insert(dir);
        unsigned nonce = dir->add_replica(from);
-       dout(10) << " have " << p->frag << " -> " << fg << " " << *dir << dendl;
+       dout(10) << " have " << p.frag << " -> " << fg << " " << *dir << dendl;
        if (ack) {
          ack->add_strong_dirfrag(dir->dirfrag(), nonce, dir->dir_rep);
          ack->add_dirfrag_base(dir);
@@ -4518,22 +4442,18 @@ void MDCache::handle_cache_rejoin_weak(MMDSCacheRejoin *weak)
     }
   }
 
-  for (map<inodeno_t,map<string_snap_t,MMDSCacheRejoin::dn_weak> >::iterator p = weak->weak.begin();
-       p != weak->weak.end();
-       ++p) {
-    CInode *diri = get_inode(p->first);
+  for (const auto &p : weak->weak) {
+    CInode *diri = get_inode(p.first);
     if (!diri)
-      dout(0) << " missing dir ino " << p->first << dendl;
+      dout(0) << " missing dir ino " << p.first << dendl;
     assert(diri);
 
     // weak dentries
     CDir *dir = 0;
-    for (map<string_snap_t,MMDSCacheRejoin::dn_weak>::iterator q = p->second.begin();
-        q != p->second.end();
-        ++q) {
+    for (const auto &q : p.second) {
       // locate proper dirfrag.
       //  optimize for common case (one dirfrag) to avoid dirs_to_share set check
-      frag_t fg = diri->pick_dirfrag(q->first.name);
+      frag_t fg = diri->pick_dirfrag(q.first.name);
       if (!dir || dir->get_frag() != fg) {
        dir = diri->get_dirfrag(fg);
        if (!dir)
@@ -4543,7 +4463,7 @@ void MDCache::handle_cache_rejoin_weak(MMDSCacheRejoin *weak)
       }
 
       // and dentry
-      CDentry *dn = dir->lookup(q->first.name, q->first.snapid);
+      CDentry *dn = dir->lookup(q.first.name, q.first.snapid);
       assert(dn);
       CDentry::linkage_t *dnl = dn->get_linkage();
       assert(dnl->is_primary());
@@ -4604,10 +4524,8 @@ void MDCache::handle_cache_rejoin_weak(MMDSCacheRejoin *weak)
   rejoin_gather.erase(from);
   if (survivor) {
     // survivor.  do everything now.
-    for (map<inodeno_t,MMDSCacheRejoin::lock_bls>::iterator p = weak->inode_scatterlocks.begin();
-        p != weak->inode_scatterlocks.end();
-        ++p) {
-      CInode *in = get_inode(p->first);
+    for (const auto &p : weak->inode_scatterlocks) {
+      CInode *in = get_inode(p.first);
       assert(in);
       dout(10) << " including base inode (due to potential scatterlock update) " << *in << dendl;
       acked_inodes.insert(in->vino());
@@ -4637,7 +4555,7 @@ void MDCache::handle_cache_rejoin_weak(MMDSCacheRejoin *weak)
  * all validated replicas are acked with a strong nonce, etc.  if that isn't in the
  * ack, the replica dne, and we can remove it from our replica maps.
  */
-void MDCache::rejoin_scour_survivor_replicas(mds_rank_t from, MMDSCacheRejoin *ack,
+void MDCache::rejoin_scour_survivor_replicas(mds_rank_t from, const MMDSCacheRejoin::const_ref &ack,
                                             set<vinodeno_t>& acked_inodes,
                                             set<SimpleLock *>& gather_locks)
 {
@@ -4674,10 +4592,13 @@ void MDCache::rejoin_scour_survivor_replicas(mds_rank_t from, MMDSCacheRejoin *a
       for (auto &p : dir->items) {
        CDentry *dn = p.second;
        
-       if (dn->is_replica(from) &&
-           (ack == NULL ||
-            ack->strong_dentries.count(dir->dirfrag()) == 0 ||
-            ack->strong_dentries[dir->dirfrag()].count(string_snap_t(dn->get_name(), dn->last)) == 0)) {
+       if (dn->is_replica(from)) {
+          if (ack) {
+            const auto it = ack->strong_dentries.find(dir->dirfrag());
+            if (it != ack->strong_dentries.end() && it->second.count(string_snap_t(dn->get_name(), dn->last)) > 0) {
+              continue;
+            }
+          }
          dentry_remove_replica(dn, from, gather_locks);
          dout(10) << " rem " << *dn << dendl;
        }
@@ -4720,8 +4641,7 @@ CDir *MDCache::rejoin_invent_dirfrag(dirfrag_t df)
   return dir;
 }
 
-/* This functions DOES NOT put the passed message before returning */
-void MDCache::handle_cache_rejoin_strong(MMDSCacheRejoin *strong)
+void MDCache::handle_cache_rejoin_strong(const MMDSCacheRejoin::const_ref &strong)
 {
   mds_rank_t from = mds_rank_t(strong->get_source().num());
 
@@ -4729,14 +4649,12 @@ void MDCache::handle_cache_rejoin_strong(MMDSCacheRejoin *strong)
   assert(mds->is_rejoin());
 
   // assimilate any potentially dirty scatterlock state
-  for (map<inodeno_t,MMDSCacheRejoin::lock_bls>::iterator p = strong->inode_scatterlocks.begin();
-       p != strong->inode_scatterlocks.end();
-       ++p) {
-    CInode *in = get_inode(p->first);
+  for (const auto &p : strong->inode_scatterlocks) {
+    CInode *in = get_inode(p.first);
     assert(in);
-    in->decode_lock_state(CEPH_LOCK_IFILE, p->second.file);
-    in->decode_lock_state(CEPH_LOCK_INEST, p->second.nest);
-    in->decode_lock_state(CEPH_LOCK_IDFT, p->second.dft);
+    in->decode_lock_state(CEPH_LOCK_IFILE, p.second.file);
+    in->decode_lock_state(CEPH_LOCK_INEST, p.second.nest);
+    in->decode_lock_state(CEPH_LOCK_IDFT, p.second.dft);
     rejoin_potential_updated_scatterlocks.insert(in);
   }
 
@@ -4750,31 +4668,30 @@ void MDCache::handle_cache_rejoin_strong(MMDSCacheRejoin *strong)
 
   // strong dirfrags/dentries.
   //  also process auth_pins, xlocks.
-  for (map<dirfrag_t, MMDSCacheRejoin::dirfrag_strong>::iterator p = strong->strong_dirfrags.begin();
-       p != strong->strong_dirfrags.end();
-       ++p) {
-    CInode *diri = get_inode(p->first.ino);
+  for (const auto &p : strong->strong_dirfrags) {
+    auto& dirfrag = p.first;
+    CInode *diri = get_inode(dirfrag.ino);
     if (!diri)
-      diri = rejoin_invent_inode(p->first.ino, CEPH_NOSNAP);
-    CDir *dir = diri->get_dirfrag(p->first.frag);
+      diri = rejoin_invent_inode(dirfrag.ino, CEPH_NOSNAP);
+    CDir *dir = diri->get_dirfrag(dirfrag.frag);
     bool refragged = false;
     if (dir) {
       dout(10) << " have " << *dir << dendl;
     } else {
       if (diri->state_test(CInode::STATE_REJOINUNDEF))
        dir = rejoin_invent_dirfrag(dirfrag_t(diri->ino(), frag_t()));
-      else if (diri->dirfragtree.is_leaf(p->first.frag))
-       dir = rejoin_invent_dirfrag(p->first);
+      else if (diri->dirfragtree.is_leaf(dirfrag.frag))
+       dir = rejoin_invent_dirfrag(dirfrag);
     }
     if (dir) {
-      dir->add_replica(from, p->second.nonce);
-      dir->dir_rep = p->second.dir_rep;
+      dir->add_replica(from, p.second.nonce);
+      dir->dir_rep = p.second.dir_rep;
     } else {
-      dout(10) << " frag " << p->first << " doesn't match dirfragtree " << *diri << dendl;
+      dout(10) << " frag " << dirfrag << " doesn't match dirfragtree " << *diri << dendl;
       list<frag_t> ls;
-      diri->dirfragtree.get_leaves_under(p->first.frag, ls);
+      diri->dirfragtree.get_leaves_under(dirfrag.frag, ls);
       if (ls.empty())
-       ls.push_back(diri->dirfragtree[p->first.frag.value()]);
+       ls.push_back(diri->dirfragtree[dirfrag.frag.value()]);
       dout(10) << " maps to frag(s) " << ls << dendl;
       for (list<frag_t>::iterator q = ls.begin(); q != ls.end(); ++q) {
        CDir *dir = diri->get_dirfrag(*q);
@@ -4782,120 +4699,124 @@ void MDCache::handle_cache_rejoin_strong(MMDSCacheRejoin *strong)
          dir = rejoin_invent_dirfrag(dirfrag_t(diri->ino(), *q));
        else
          dout(10) << " have(approx) " << *dir << dendl;
-       dir->add_replica(from, p->second.nonce);
-       dir->dir_rep = p->second.dir_rep;
+       dir->add_replica(from, p.second.nonce);
+       dir->dir_rep = p.second.dir_rep;
       }
       refragged = true;
     }
     
-    map<string_snap_t,MMDSCacheRejoin::dn_strong>& dmap = strong->strong_dentries[p->first];
-    for (map<string_snap_t,MMDSCacheRejoin::dn_strong>::iterator q = dmap.begin();
-        q != dmap.end();
-        ++q) {
-      CDentry *dn;
-      if (!refragged)
-       dn = dir->lookup(q->first.name, q->first.snapid);
-      else {
-       frag_t fg = diri->pick_dirfrag(q->first.name);
-       dir = diri->get_dirfrag(fg);
-       assert(dir);
-       dn = dir->lookup(q->first.name, q->first.snapid);
-      }
-      if (!dn) {
-       if (q->second.is_remote()) {
-         dn = dir->add_remote_dentry(q->first.name, q->second.remote_ino, q->second.remote_d_type,
-                                     q->second.first, q->first.snapid);
-       } else if (q->second.is_null()) {
-         dn = dir->add_null_dentry(q->first.name, q->second.first, q->first.snapid);
-       } else {
-         CInode *in = get_inode(q->second.ino, q->first.snapid);
-         if (!in) in = rejoin_invent_inode(q->second.ino, q->first.snapid);
-         dn = dir->add_primary_dentry(q->first.name, in, q->second.first, q->first.snapid);
-       }
-       dout(10) << " invented " << *dn << dendl;
-      }
-      CDentry::linkage_t *dnl = dn->get_linkage();
-
-      // dn auth_pin?
-      if (strong->authpinned_dentries.count(p->first) &&
-         strong->authpinned_dentries[p->first].count(q->first)) {
-       for (list<MMDSCacheRejoin::slave_reqid>::iterator r = strong->authpinned_dentries[p->first][q->first].begin();
-            r != strong->authpinned_dentries[p->first][q->first].end();
-            ++r) {
-         dout(10) << " dn authpin by " << *r << " on " << *dn << dendl;
-
-         // get/create slave mdrequest
-         MDRequestRef mdr;
-         if (have_request(r->reqid))
-           mdr = request_get(r->reqid);
-         else
-           mdr = request_start_slave(r->reqid, r->attempt, strong);
-         mdr->auth_pin(dn);
-       }
-      }
-
-      // dn xlock?
-      if (strong->xlocked_dentries.count(p->first) &&
-         strong->xlocked_dentries[p->first].count(q->first)) {
-       MMDSCacheRejoin::slave_reqid r = strong->xlocked_dentries[p->first][q->first];
-       dout(10) << " dn xlock by " << r << " on " << *dn << dendl;
-       MDRequestRef mdr = request_get(r.reqid);  // should have this from auth_pin above.
-       assert(mdr->is_auth_pinned(dn));
-       if (!mdr->xlocks.count(&dn->versionlock)) {
-         assert(dn->versionlock.can_xlock_local());
-         dn->versionlock.get_xlock(mdr, mdr->get_client());
-         mdr->xlocks.insert(&dn->versionlock);
-         mdr->locks.insert(&dn->versionlock);
+    const auto it = strong->strong_dentries.find(dirfrag);
+    if (it != strong->strong_dentries.end()) {
+      const map<string_snap_t,MMDSCacheRejoin::dn_strong>& dmap = it->second;
+      for (const auto &q : dmap) {
+        const string_snap_t& ss = q.first;
+        const MMDSCacheRejoin::dn_strong& d = q.second;
+        CDentry *dn;
+        if (!refragged)
+         dn = dir->lookup(ss.name, ss.snapid);
+        else {
+         frag_t fg = diri->pick_dirfrag(ss.name);
+         dir = diri->get_dirfrag(fg);
+         assert(dir);
+         dn = dir->lookup(ss.name, ss.snapid);
+        }
+        if (!dn) {
+         if (d.is_remote()) {
+           dn = dir->add_remote_dentry(ss.name, d.remote_ino, d.remote_d_type, d.first, ss.snapid);
+         } else if (d.is_null()) {
+           dn = dir->add_null_dentry(ss.name, d.first, ss.snapid);
+         } else {
+           CInode *in = get_inode(d.ino, ss.snapid);
+           if (!in) in = rejoin_invent_inode(d.ino, ss.snapid);
+           dn = dir->add_primary_dentry(ss.name, in, d.first, ss.snapid);
+         }
+         dout(10) << " invented " << *dn << dendl;
+        }
+        CDentry::linkage_t *dnl = dn->get_linkage();
+
+        // dn auth_pin?
+        const auto pinned_it = strong->authpinned_dentries.find(dirfrag);
+        if (pinned_it != strong->authpinned_dentries.end()) {
+          const auto slave_reqid_it = pinned_it->second.find(ss);
+          if (slave_reqid_it != pinned_it->second.end()) {
+            for (const auto &r : slave_reqid_it->second) {
+             dout(10) << " dn authpin by " << r << " on " << *dn << dendl;
+
+             // get/create slave mdrequest
+             MDRequestRef mdr;
+             if (have_request(r.reqid))
+               mdr = request_get(r.reqid);
+             else
+               mdr = request_start_slave(r.reqid, r.attempt, strong);
+             mdr->auth_pin(dn);
+            }
+          }
        }
-       if (dn->lock.is_stable())
-         dn->auth_pin(&dn->lock);
-       dn->lock.set_state(LOCK_XLOCK);
-       dn->lock.get_xlock(mdr, mdr->get_client());
-       mdr->xlocks.insert(&dn->lock);
-       mdr->locks.insert(&dn->lock);
-      }
 
-      dn->add_replica(from, q->second.nonce);
-      dout(10) << " have " << *dn << dendl;
+        // dn xlock?
+        const auto xlocked_dentries_it = strong->xlocked_dentries.find(dirfrag);
+        if (xlocked_dentries_it != strong->xlocked_dentries.end()) {
+          const auto ss_req_it = xlocked_dentries_it->second.find(ss);
+          if (ss_req_it != xlocked_dentries_it->second.end()) {
+           const MMDSCacheRejoin::slave_reqid& r = ss_req_it->second;
+           dout(10) << " dn xlock by " << r << " on " << *dn << dendl;
+           MDRequestRef mdr = request_get(r.reqid);  // should have this from auth_pin above.
+           assert(mdr->is_auth_pinned(dn));
+           if (!mdr->xlocks.count(&dn->versionlock)) {
+             assert(dn->versionlock.can_xlock_local());
+             dn->versionlock.get_xlock(mdr, mdr->get_client());
+             mdr->xlocks.insert(&dn->versionlock);
+             mdr->locks.insert(&dn->versionlock);
+           }
+           if (dn->lock.is_stable())
+             dn->auth_pin(&dn->lock);
+           dn->lock.set_state(LOCK_XLOCK);
+           dn->lock.get_xlock(mdr, mdr->get_client());
+           mdr->xlocks.insert(&dn->lock);
+           mdr->locks.insert(&dn->lock);
+          }
+        }
 
-      if (dnl->is_primary()) {
-       if (q->second.is_primary()) {
-         if (vinodeno_t(q->second.ino, q->first.snapid) != dnl->get_inode()->vino()) {
-           // the survivor missed MDentryUnlink+MDentryLink messages ?
+        dn->add_replica(from, d.nonce);
+        dout(10) << " have " << *dn << dendl;
+
+        if (dnl->is_primary()) {
+         if (d.is_primary()) {
+           if (vinodeno_t(d.ino, ss.snapid) != dnl->get_inode()->vino()) {
+             // the survivor missed MDentryUnlink+MDentryLink messages ?
+             assert(strong->strong_inodes.count(dnl->get_inode()->vino()) == 0);
+             CInode *in = get_inode(d.ino, ss.snapid);
+             assert(in);
+             assert(in->get_parent_dn());
+             rejoin_unlinked_inodes[from].insert(in);
+             dout(7) << " sender has primary dentry but wrong inode" << dendl;
+           }
+         } else {
+           // the survivor missed MDentryLink message ?
            assert(strong->strong_inodes.count(dnl->get_inode()->vino()) == 0);
-           CInode *in = get_inode(q->second.ino, q->first.snapid);
+           dout(7) << " sender doesn't have primay dentry" << dendl;
+         }
+        } else {
+         if (d.is_primary()) {
+           // the survivor missed MDentryUnlink message ?
+           CInode *in = get_inode(d.ino, ss.snapid);
            assert(in);
            assert(in->get_parent_dn());
            rejoin_unlinked_inodes[from].insert(in);
-           dout(7) << " sender has primary dentry but wrong inode" << dendl;
+           dout(7) << " sender has primary dentry but we don't" << dendl;
          }
-       } else {
-         // the survivor missed MDentryLink message ?
-         assert(strong->strong_inodes.count(dnl->get_inode()->vino()) == 0);
-         dout(7) << " sender doesn't have primay dentry" << dendl;
-       }
-      } else {
-       if (q->second.is_primary()) {
-         // the survivor missed MDentryUnlink message ?
-         CInode *in = get_inode(q->second.ino, q->first.snapid);
-         assert(in);
-         assert(in->get_parent_dn());
-         rejoin_unlinked_inodes[from].insert(in);
-         dout(7) << " sender has primary dentry but we don't" << dendl;
-       }
+        }
       }
     }
   }
 
-  for (map<vinodeno_t, MMDSCacheRejoin::inode_strong>::iterator p = strong->strong_inodes.begin();
-       p != strong->strong_inodes.end();
-       ++p) {
-    CInode *in = get_inode(p->first);
+  for (const auto &p : strong->strong_inodes) {
+    CInode *in = get_inode(p.first);
     assert(in);
-    in->add_replica(from, p->second.nonce);
+    in->add_replica(from, p.second.nonce);
     dout(10) << " have " << *in << dendl;
 
-    MMDSCacheRejoin::inode_strong &is = p->second;
+    const MMDSCacheRejoin::inode_strong& is = p.second;
 
     // caps_wanted
     if (is.caps_wanted) {
@@ -4913,18 +4834,17 @@ void MDCache::handle_cache_rejoin_strong(MMDSCacheRejoin *strong)
     in->dirfragtreelock.infer_state_from_strong_rejoin(is.dftlock, false);
 
     // auth pin?
-    if (strong->authpinned_inodes.count(in->vino())) {
-      for (list<MMDSCacheRejoin::slave_reqid>::iterator r = strong->authpinned_inodes[in->vino()].begin();
-          r != strong->authpinned_inodes[in->vino()].end();
-          ++r) {
-       dout(10) << " inode authpin by " << *r << " on " << *in << dendl;
+    const auto authpinned_inodes_it = strong->authpinned_inodes.find(in->vino());
+    if (authpinned_inodes_it != strong->authpinned_inodes.end()) {
+      for (const auto& r : authpinned_inodes_it->second) {
+       dout(10) << " inode authpin by " << r << " on " << *in << dendl;
 
        // get/create slave mdrequest
        MDRequestRef mdr;
-       if (have_request(r->reqid))
-         mdr = request_get(r->reqid);
+       if (have_request(r.reqid))
+         mdr = request_get(r.reqid);
        else
-         mdr = request_start_slave(r->reqid, r->attempt, strong);
+         mdr = request_start_slave(r.reqid, r.attempt, strong);
        if (strong->frozen_authpin_inodes.count(in->vino())) {
          assert(!in->get_num_auth_pins());
          mdr->freeze_auth_pin(in);
@@ -4935,13 +4855,12 @@ void MDCache::handle_cache_rejoin_strong(MMDSCacheRejoin *strong)
       }
     }
     // xlock(s)?
-    if (strong->xlocked_inodes.count(in->vino())) {
-      for (map<int,MMDSCacheRejoin::slave_reqid>::iterator q = strong->xlocked_inodes[in->vino()].begin();
-          q != strong->xlocked_inodes[in->vino()].end();
-          ++q) {
-       SimpleLock *lock = in->get_lock(q->first);
-       dout(10) << " inode xlock by " << q->second << " on " << *lock << " on " << *in << dendl;
-       MDRequestRef mdr = request_get(q->second.reqid);  // should have this from auth_pin above.
+    const auto xlocked_inodes_it = strong->xlocked_inodes.find(in->vino());
+    if (xlocked_inodes_it != strong->xlocked_inodes.end()) {
+      for (const auto &q : xlocked_inodes_it->second) {
+       SimpleLock *lock = in->get_lock(q.first);
+       dout(10) << " inode xlock by " << q.second << " on " << *lock << " on " << *in << dendl;
+       MDRequestRef mdr = request_get(q.second.reqid);  // should have this from auth_pin above.
        assert(mdr->is_auth_pinned(in));
        if (!mdr->xlocks.count(&in->versionlock)) {
          assert(in->versionlock.can_xlock_local());
@@ -4961,19 +4880,13 @@ void MDCache::handle_cache_rejoin_strong(MMDSCacheRejoin *strong)
     }
   }
   // wrlock(s)?
-  for (map<vinodeno_t, map<int, list<MMDSCacheRejoin::slave_reqid> > >::iterator p = strong->wrlocked_inodes.begin();
-       p != strong->wrlocked_inodes.end();
-       ++p) {
-    CInode *in = get_inode(p->first);
-    for (map<int, list<MMDSCacheRejoin::slave_reqid> >::iterator q = p->second.begin();
-        q != p->second.end();
-       ++q) {
-      SimpleLock *lock = in->get_lock(q->first);
-      for (list<MMDSCacheRejoin::slave_reqid>::iterator r = q->second.begin();
-         r != q->second.end();
-         ++r) {
-       dout(10) << " inode wrlock by " << *r << " on " << *lock << " on " << *in << dendl;
-       MDRequestRef mdr = request_get(r->reqid);  // should have this from auth_pin above.
+  for (const auto &p : strong->wrlocked_inodes) {
+    CInode *in = get_inode(p.first);
+    for (const auto &q : p.second) {
+      SimpleLock *lock = in->get_lock(q.first);
+      for (const auto &r : q.second) {
+       dout(10) << " inode wrlock by " << r << " on " << *lock << " on " << *in << dendl;
+       MDRequestRef mdr = request_get(r.reqid);  // should have this from auth_pin above.
        if (in->is_auth())
          assert(mdr->is_auth_pinned(in));
        lock->set_state(LOCK_MIX);
@@ -4996,8 +4909,7 @@ void MDCache::handle_cache_rejoin_strong(MMDSCacheRejoin *strong)
   }
 }
 
-/* This functions DOES NOT put the passed message before returning */
-void MDCache::handle_cache_rejoin_ack(MMDSCacheRejoin *ack)
+void MDCache::handle_cache_rejoin_ack(const MMDSCacheRejoin::const_ref &ack)
 {
   dout(7) << "handle_cache_rejoin_ack from " << ack->get_source() << dendl;
   mds_rank_t from = mds_rank_t(ack->get_source().num());
@@ -5011,113 +4923,111 @@ void MDCache::handle_cache_rejoin_ack(MMDSCacheRejoin *ack)
   list<pair<CInode*,int> > updated_realms;
 
   // dirs
-  for (map<dirfrag_t, MMDSCacheRejoin::dirfrag_strong>::iterator p = ack->strong_dirfrags.begin();
-       p != ack->strong_dirfrags.end();
-       ++p) {
+  for (const auto &p : ack->strong_dirfrags) {
     // we may have had incorrect dir fragmentation; refragment based
     // on what they auth tells us.
-    CDir *dir = get_dirfrag(p->first);
+    CDir *dir = get_dirfrag(p.first);
     if (!dir) {
-      dir = get_force_dirfrag(p->first, false);
+      dir = get_force_dirfrag(p.first, false);
       if (dir)
        refragged_inodes.insert(dir->get_inode());
     }
     if (!dir) {
-      CInode *diri = get_inode(p->first.ino);
+      CInode *diri = get_inode(p.first.ino);
       if (!diri) {
        // barebones inode; the full inode loop below will clean up.
        diri = new CInode(this, false);
-       diri->inode.ino = p->first.ino;
+       diri->inode.ino = p.first.ino;
        diri->inode.mode = S_IFDIR;
        diri->inode.dir_layout.dl_dir_hash = g_conf()->mds_default_dir_hash;
        add_inode(diri);
-       if (MDS_INO_MDSDIR(from) == p->first.ino) {
+       if (MDS_INO_MDSDIR(from) == p.first.ino) {
          diri->inode_auth = mds_authority_t(from, CDIR_AUTH_UNKNOWN);
          dout(10) << " add inode " << *diri << dendl;
        } else {
          diri->inode_auth = CDIR_AUTH_DEFAULT;
          isolated_inodes.insert(diri);
-         dout(10) << " unconnected dirfrag " << p->first << dendl;
+         dout(10) << " unconnected dirfrag " << p.first << dendl;
        }
       }
       // barebones dirfrag; the full dirfrag loop below will clean up.
-      dir = diri->add_dirfrag(new CDir(diri, p->first.frag, this, false));
-      if (MDS_INO_MDSDIR(from) == p->first.ino ||
+      dir = diri->add_dirfrag(new CDir(diri, p.first.frag, this, false));
+      if (MDS_INO_MDSDIR(from) == p.first.ino ||
          (dir->authority() != CDIR_AUTH_UNDEF &&
           dir->authority().first != from))
        adjust_subtree_auth(dir, from);
       dout(10) << " add dirfrag " << *dir << dendl;
     }
 
-    dir->set_replica_nonce(p->second.nonce);
+    dir->set_replica_nonce(p.second.nonce);
     dir->state_clear(CDir::STATE_REJOINING);
     dout(10) << " got " << *dir << dendl;
 
     // dentries
-    map<string_snap_t,MMDSCacheRejoin::dn_strong>& dmap = ack->strong_dentries[p->first];
-    for (map<string_snap_t,MMDSCacheRejoin::dn_strong>::iterator q = dmap.begin();
-        q != dmap.end();
-        ++q) {
-      CDentry *dn = dir->lookup(q->first.name, q->first.snapid);
-      if(!dn)
-       dn = dir->add_null_dentry(q->first.name, q->second.first, q->first.snapid);
-
-      CDentry::linkage_t *dnl = dn->get_linkage();
-
-      assert(dn->last == q->first.snapid);
-      if (dn->first != q->second.first) {
-       dout(10) << " adjust dn.first " << dn->first << " -> " << q->second.first << " on " << *dn << dendl;
-       dn->first = q->second.first;
-      }
+    auto it = ack->strong_dentries.find(p.first);
+    if (it != ack->strong_dentries.end()) {
+      for (const auto &q : it->second) {
+        CDentry *dn = dir->lookup(q.first.name, q.first.snapid);
+        if(!dn)
+         dn = dir->add_null_dentry(q.first.name, q.second.first, q.first.snapid);
+
+        CDentry::linkage_t *dnl = dn->get_linkage();
+
+        assert(dn->last == q.first.snapid);
+        if (dn->first != q.second.first) {
+         dout(10) << " adjust dn.first " << dn->first << " -> " << q.second.first << " on " << *dn << dendl;
+         dn->first = q.second.first;
+        }
 
-      // may have bad linkage if we missed dentry link/unlink messages
-      if (dnl->is_primary()) {
-       CInode *in = dnl->get_inode();
-       if (!q->second.is_primary() ||
-           vinodeno_t(q->second.ino, q->first.snapid) != in->vino()) {
-         dout(10) << " had bad linkage for " << *dn << ", unlinking " << *in << dendl;
-         dir->unlink_inode(dn);
-       }
-      } else if (dnl->is_remote()) {
-       if (!q->second.is_remote() ||
-           q->second.remote_ino != dnl->get_remote_ino() ||
-           q->second.remote_d_type != dnl->get_remote_d_type()) {
-         dout(10) << " had bad linkage for " << *dn <<  dendl;
-         dir->unlink_inode(dn);
-       }
-      } else {
-       if (!q->second.is_null())
-         dout(10) << " had bad linkage for " << *dn <<  dendl;
-      }
+        // may have bad linkage if we missed dentry link/unlink messages
+        if (dnl->is_primary()) {
+         CInode *in = dnl->get_inode();
+         if (!q.second.is_primary() ||
+             vinodeno_t(q.second.ino, q.first.snapid) != in->vino()) {
+           dout(10) << " had bad linkage for " << *dn << ", unlinking " << *in << dendl;
+           dir->unlink_inode(dn);
+         }
+        } else if (dnl->is_remote()) {
+         if (!q.second.is_remote() ||
+             q.second.remote_ino != dnl->get_remote_ino() ||
+             q.second.remote_d_type != dnl->get_remote_d_type()) {
+           dout(10) << " had bad linkage for " << *dn <<  dendl;
+           dir->unlink_inode(dn);
+         }
+        } else {
+         if (!q.second.is_null())
+           dout(10) << " had bad linkage for " << *dn <<  dendl;
+        }
 
-      // hmm, did we have the proper linkage here?
-      if (dnl->is_null() && !q->second.is_null()) {
-       if (q->second.is_remote()) {
-         dn->dir->link_remote_inode(dn, q->second.remote_ino, q->second.remote_d_type);
-       } else {
-         CInode *in = get_inode(q->second.ino, q->first.snapid);
-         if (!in) {
-           // barebones inode; assume it's dir, the full inode loop below will clean up.
-           in = new CInode(this, false, q->second.first, q->first.snapid);
-           in->inode.ino = q->second.ino;
-           in->inode.mode = S_IFDIR;
-           in->inode.dir_layout.dl_dir_hash = g_conf()->mds_default_dir_hash;
-           add_inode(in);
-           dout(10) << " add inode " << *in << dendl;
-         } else if (in->get_parent_dn()) {
-           dout(10) << " had bad linkage for " << *(in->get_parent_dn())
-                    << ", unlinking " << *in << dendl;
-           in->get_parent_dir()->unlink_inode(in->get_parent_dn());
+        // hmm, did we have the proper linkage here?
+        if (dnl->is_null() && !q.second.is_null()) {
+         if (q.second.is_remote()) {
+           dn->dir->link_remote_inode(dn, q.second.remote_ino, q.second.remote_d_type);
+         } else {
+           CInode *in = get_inode(q.second.ino, q.first.snapid);
+           if (!in) {
+             // barebones inode; assume it's dir, the full inode loop below will clean up.
+             in = new CInode(this, false, q.second.first, q.first.snapid);
+             in->inode.ino = q.second.ino;
+             in->inode.mode = S_IFDIR;
+             in->inode.dir_layout.dl_dir_hash = g_conf()->mds_default_dir_hash;
+             add_inode(in);
+             dout(10) << " add inode " << *in << dendl;
+           } else if (in->get_parent_dn()) {
+             dout(10) << " had bad linkage for " << *(in->get_parent_dn())
+                      << ", unlinking " << *in << dendl;
+             in->get_parent_dir()->unlink_inode(in->get_parent_dn());
+           }
+           dn->dir->link_primary_inode(dn, in);
+           isolated_inodes.erase(in);
          }
-         dn->dir->link_primary_inode(dn, in);
-         isolated_inodes.erase(in);
-       }
-      }
+        }
 
-      dn->set_replica_nonce(q->second.nonce);
-      dn->lock.set_state_rejoin(q->second.lock, rejoin_waiters, survivor);
-      dn->state_clear(CDentry::STATE_REJOINING);
-      dout(10) << " got " << *dn << dendl;
+        dn->set_replica_nonce(q.second.nonce);
+        dn->lock.set_state_rejoin(q.second.lock, rejoin_waiters, survivor);
+        dn->state_clear(CDentry::STATE_REJOINING);
+        dout(10) << " got " << *dn << dendl;
+      }
     }
   }
 
@@ -5135,12 +5045,10 @@ void MDCache::handle_cache_rejoin_ack(MMDSCacheRejoin *ack)
   }
 
   // full dirfrags
-  for (map<dirfrag_t, bufferlist>::iterator p = ack->dirfrag_bases.begin();
-       p != ack->dirfrag_bases.end();
-       ++p) {
-    CDir *dir = get_dirfrag(p->first);
+  for (const auto &p : ack->dirfrag_bases) {
+    CDir *dir = get_dirfrag(p.first);
     assert(dir);
-    auto q = p->second.cbegin();
+    auto q = p.second.cbegin();
     dir->_decode_base(q);
     dout(10) << " got dir replica " << *dir << dendl;
   }
@@ -5648,15 +5556,16 @@ void MDCache::choose_lock_states_and_reconnect_caps()
 }
 
 void MDCache::prepare_realm_split(SnapRealm *realm, client_t client, inodeno_t ino,
-                                 map<client_t,MClientSnap*>& splits)
+                                 map<client_t,MClientSnap::ref>& splits)
 {
-  MClientSnap *snap;
+  MClientSnap::ref snap;
   auto it = splits.find(client);
   if (it != splits.end()) {
     snap = it->second;
     snap->head.op = CEPH_SNAP_OP_SPLIT;
   } else {
-    splits[client] = snap = new MClientSnap(CEPH_SNAP_OP_SPLIT);
+    snap.reset(new MClientSnap(CEPH_SNAP_OP_SPLIT), false);
+    splits.emplace(std::piecewise_construct, std::forward_as_tuple(client), std::forward_as_tuple(snap));
     snap->head.split = realm->inode->ino();
     snap->bl = realm->get_snap_trace();
 
@@ -5667,7 +5576,7 @@ void MDCache::prepare_realm_split(SnapRealm *realm, client_t client, inodeno_t i
 }
 
 void MDCache::prepare_realm_merge(SnapRealm *realm, SnapRealm *parent_realm,
-                                 map<client_t,MClientSnap*>& splits)
+                                 map<client_t,MClientSnap::ref>& splits)
 {
   assert(parent_realm);
 
@@ -5683,36 +5592,34 @@ void MDCache::prepare_realm_merge(SnapRealm *realm, SnapRealm *parent_realm,
        ++p)
     split_realms.push_back((*p)->inode->ino());
 
-  for (auto p : realm->client_caps) {
+  for (const auto& p : realm->client_caps) {
     assert(!p.second->empty());
-    if (splits.count(p.first) == 0) {
-      MClientSnap *update = new MClientSnap(CEPH_SNAP_OP_SPLIT);
-      splits[p.first] = update;
+    auto em = splits.emplace(std::piecewise_construct, std::forward_as_tuple(p.first), std::forward_as_tuple());
+    if (em.second) {
+      MClientSnap::ref update(new MClientSnap(CEPH_SNAP_OP_SPLIT), false);
       update->head.split = parent_realm->inode->ino();
       update->split_inos = split_inos;
       update->split_realms = split_realms;
       update->bl = parent_realm->get_snap_trace();
+      em.first->second = std::move(update);
     }
   }
 }
 
-void MDCache::send_snaps(map<client_t,MClientSnap*>& splits)
+void MDCache::send_snaps(map<client_t,MClientSnap::ref>& splits)
 {
   dout(10) << "send_snaps" << dendl;
   
-  for (map<client_t,MClientSnap*>::iterator p = splits.begin();
-       p != splits.end();
-       ++p) {
-    Session *session = mds->sessionmap.get_session(entity_name_t::CLIENT(p->first.v));
+  for (auto &p : splits) {
+    Session *session = mds->sessionmap.get_session(entity_name_t::CLIENT(p.first.v));
     if (session) {
-      dout(10) << " client." << p->first
-              << " split " << p->second->head.split
-              << " inos " << p->second->split_inos
+      dout(10) << " client." << p.first
+              << " split " << p.second->head.split
+              << " inos " << p.second->split_inos
               << dendl;
-      mds->send_message_client_counted(p->second, session);
+      mds->send_message_client_counted(p.second, session);
     } else {
-      dout(10) << " no session for client." << p->first << dendl;
-      p->second->put();
+      dout(10) << " no session for client." << p.first << dendl;
     }
   }
   splits.clear();
@@ -5924,7 +5831,7 @@ void MDCache::open_snaprealms()
        realm->open_parents(gather.new_sub())) {
       dout(10) << " past parents now open on " << *in << dendl;
 
-      map<client_t,MClientSnap*> splits;
+      map<client_t,MClientSnap::ref> splits;
       // finish off client snaprealm reconnects?
       map<inodeno_t,map<client_t,snapid_t> >::iterator q = reconnected_snaprealms.find(in->ino());
       if (q != reconnected_snaprealms.end()) {
@@ -6076,7 +5983,7 @@ void MDCache::opened_undef_inode(CInode *in) {
 }
 
 void MDCache::finish_snaprealm_reconnect(client_t client, SnapRealm *realm, snapid_t seq,
-                                        map<client_t,MClientSnap*>& updates)
+                                        map<client_t,MClientSnap::ref>& updates)
 {
   if (seq < realm->get_newest_seq()) {
     dout(10) << "finish_snaprealm_reconnect client." << client << " has old seq " << seq << " < " 
@@ -6624,7 +6531,7 @@ void MDCache::start_recovered_truncates()
 // ================================================================================
 // cache trimming
 
-void MDCache::trim_lru(uint64_t count, map<mds_rank_t, MCacheExpire*> &expiremap)
+void MDCache::trim_lru(uint64_t count, expiremap& expiremap)
 {
   bool is_standby_replay = mds->is_standby_replay();
   std::vector<CDentry *> unexpirables;
@@ -6689,7 +6596,7 @@ bool MDCache::trim(uint64_t count)
 {
   uint64_t used = cache_size();
   uint64_t limit = cache_memory_limit;
-  map<mds_rank_t, MCacheExpire*> expiremap;
+  expiremap expiremap;
 
   dout(7) << "trim bytes_used=" << bytes2str(used)
           << " limit=" << bytes2str(limit)
@@ -6756,8 +6663,9 @@ bool MDCache::trim(uint64_t count)
     if (!mdsdir_in)
       continue;
 
-    if (expiremap.count(rank) == 0)  {
-      expiremap[rank] = new MCacheExpire(mds->get_nodeid());
+    auto em = expiremap.emplace(std::piecewise_construct, std::forward_as_tuple(rank), std::forward_as_tuple());
+    if (em.second) {
+      em.first->second.reset(new MCacheExpire(mds->get_nodeid()), false);
     }
 
     dout(20) << __func__ << ": try expiring " << *mdsdir_in << " for stopping mds." << mds <<  dendl;
@@ -6800,26 +6708,24 @@ bool MDCache::trim(uint64_t count)
   return true;
 }
 
-void MDCache::send_expire_messages(map<mds_rank_t, MCacheExpire*>& expiremap)
+void MDCache::send_expire_messages(expiremap& expiremap)
 {
   // send expires
-  for (map<mds_rank_t, MCacheExpire*>::iterator it = expiremap.begin();
-       it != expiremap.end();
-       ++it) {
+  for (const auto &p : expiremap) {
     if (mds->is_cluster_degraded() &&
-       (mds->mdsmap->get_state(it->first) < MDSMap::STATE_REJOIN ||
-        (mds->mdsmap->get_state(it->first) == MDSMap::STATE_REJOIN &&
-         rejoin_sent.count(it->first) == 0))) {
-      it->second->put();
+       (mds->mdsmap->get_state(p.first) < MDSMap::STATE_REJOIN ||
+        (mds->mdsmap->get_state(p.first) == MDSMap::STATE_REJOIN &&
+         rejoin_sent.count(p.first) == 0))) {
       continue;
     }
-    dout(7) << "sending cache_expire to " << it->first << dendl;
-    mds->send_message_mds(it->second, it->first);
+    dout(7) << "sending cache_expire to " << p.first << dendl;
+    mds->send_message_mds(p.second, p.first);
   }
+  expiremap.clear();
 }
 
 
-bool MDCache::trim_dentry(CDentry *dn, map<mds_rank_t, MCacheExpire*>& expiremap)
+bool MDCache::trim_dentry(CDentry *dn, expiremap& expiremap)
 {
   dout(12) << "trim_dentry " << *dn << dendl;
   
@@ -6882,9 +6788,10 @@ bool MDCache::trim_dentry(CDentry *dn, map<mds_rank_t, MCacheExpire*>& expiremap
       
       dout(12) << "  sending expire to mds." << a << " on " << *dn << dendl;
       assert(a != mds->get_nodeid());
-      if (expiremap.count(a) == 0) 
-       expiremap[a] = new MCacheExpire(mds->get_nodeid());
-      expiremap[a]->add_dentry(con->dirfrag(), dir->dirfrag(), dn->get_name(), dn->last, dn->get_replica_nonce());
+      auto em = expiremap.emplace(std::piecewise_construct, std::forward_as_tuple(a), std::forward_as_tuple());
+      if (em.second)
+       em.first->second.reset(new MCacheExpire(mds->get_nodeid()), false); /* new */
+      em.first->second->add_dentry(con->dirfrag(), dir->dirfrag(), dn->get_name(), dn->last, dn->get_replica_nonce());
     }
   }
 
@@ -6901,7 +6808,7 @@ bool MDCache::trim_dentry(CDentry *dn, map<mds_rank_t, MCacheExpire*>& expiremap
 }
 
 
-void MDCache::trim_dirfrag(CDir *dir, CDir *con, map<mds_rank_t, MCacheExpire*>& expiremap)
+void MDCache::trim_dirfrag(CDir *dir, CDir *con, expiremap& expiremap)
 {
   dout(15) << "trim_dirfrag " << *dir << dendl;
 
@@ -6937,9 +6844,10 @@ void MDCache::trim_dirfrag(CDir *dir, CDir *con, map<mds_rank_t, MCacheExpire*>&
 
       dout(12) << "  sending expire to mds." << a << " on   " << *dir << dendl;
       assert(a != mds->get_nodeid());
-      if (expiremap.count(a) == 0) 
-       expiremap[a] = new MCacheExpire(mds->get_nodeid());
-      expiremap[a]->add_dir(condf, dir->dirfrag(), dir->replica_nonce);
+      auto em = expiremap.emplace(std::piecewise_construct, std::forward_as_tuple(a), std::forward_as_tuple());
+      if (em.second)
+       em.first->second.reset(new MCacheExpire(mds->get_nodeid()), false); /* new */
+      em.first->second->add_dir(condf, dir->dirfrag(), dir->replica_nonce);
     }
   }
   
@@ -6951,7 +6859,7 @@ void MDCache::trim_dirfrag(CDir *dir, CDir *con, map<mds_rank_t, MCacheExpire*>&
  *
  * @return true if the inode is still in cache, else false if it was trimmed
  */
-bool MDCache::trim_inode(CDentry *dn, CInode *in, CDir *con, map<mds_rank_t, MCacheExpire*>& expiremap)
+bool MDCache::trim_inode(CDentry *dn, CInode *in, CDir *con, expiremap& expiremap)
 {
   dout(15) << "trim_inode " << *in << dendl;
   assert(in->get_num_ref() == 0);
@@ -7005,9 +6913,10 @@ bool MDCache::trim_inode(CDentry *dn, CInode *in, CDir *con, map<mds_rank_t, MCa
 
       dout(12) << "  sending expire to mds." << a << " on " << *in << dendl;
       assert(a != mds->get_nodeid());
-      if (expiremap.count(a) == 0) 
-       expiremap[a] = new MCacheExpire(mds->get_nodeid());
-      expiremap[a]->add_inode(df, in->vino(), in->get_replica_nonce());
+      auto em = expiremap.emplace(std::piecewise_construct, std::forward_as_tuple(a), std::forward_as_tuple());
+      if (em.second)
+       em.first->second.reset(new MCacheExpire(mds->get_nodeid()), false); /* new */
+      em.first->second->add_inode(df, in->vino(), in->get_replica_nonce());
     }
   }
 
@@ -7332,28 +7241,24 @@ void MDCache::standby_trim_segment(LogSegment *ls)
   }
 }
 
-/* This function DOES put the passed message before returning */
-void MDCache::handle_cache_expire(MCacheExpire *m)
+void MDCache::handle_cache_expire(const MCacheExpire::const_ref &m)
 {
   mds_rank_t from = mds_rank_t(m->get_from());
   
   dout(7) << "cache_expire from mds." << from << dendl;
 
   if (mds->get_state() < MDSMap::STATE_REJOIN) {
-    m->put();
     return;
   }
 
   set<SimpleLock *> gather_locks;
   // loop over realms
-  for (map<dirfrag_t,MCacheExpire::realm>::iterator p = m->realms.begin();
-       p != m->realms.end();
-       ++p) {
+  for (const auto &p : m->realms) {
     // check container?
-    if (p->first.ino > 0) {
-      CInode *expired_inode = get_inode(p->first.ino);
+    if (p.first.ino > 0) {
+      CInode *expired_inode = get_inode(p.first.ino);
       assert(expired_inode);  // we had better have this.
-      CDir *parent_dir = expired_inode->get_approx_dirfrag(p->first.frag);
+      CDir *parent_dir = expired_inode->get_approx_dirfrag(p.first.frag);
       assert(parent_dir);
 
       int export_state = -1;
@@ -7376,11 +7281,13 @@ void MDCache::handle_cache_expire(MCacheExpire *m)
        assert(parent_dir->is_frozen_tree_root());
        
        // make a message container
-       if (delayed_expire[parent_dir].count(from) == 0)
-         delayed_expire[parent_dir][from] = new MCacheExpire(from);
-       
+
+        auto em = delayed_expire[parent_dir].emplace(std::piecewise_construct, std::forward_as_tuple(from), std::forward_as_tuple());
+        if (em.second)
+         em.first->second.reset(new MCacheExpire(from), false); /* new */
+
        // merge these expires into it
-       delayed_expire[parent_dir][from]->add_realm(p->first, p->second);
+       em.first->second->add_realm(p.first, p.second);
        continue;
       }
       assert(export_state <= Migrator::EXPORT_PREPPING ||
@@ -7393,14 +7300,12 @@ void MDCache::handle_cache_expire(MCacheExpire *m)
     }
 
     // INODES
-    for (map<vinodeno_t,uint32_t>::iterator it = p->second.inodes.begin();
-        it != p->second.inodes.end();
-        ++it) {
-      CInode *in = get_inode(it->first);
-      unsigned nonce = it->second;
+    for (const auto &q : p.second.inodes) {
+      CInode *in = get_inode(q.first);
+      unsigned nonce = q.second;
       
       if (!in) {
-       dout(0) << " inode expire on " << it->first << " from " << from 
+       dout(0) << " inode expire on " << q.first << " from " << from 
                << ", don't have it" << dendl;
        assert(in);
       }        
@@ -7424,21 +7329,19 @@ void MDCache::handle_cache_expire(MCacheExpire *m)
     }
     
     // DIRS
-    for (map<dirfrag_t,uint32_t>::iterator it = p->second.dirs.begin();
-        it != p->second.dirs.end();
-        ++it) {
-      CDir *dir = get_dirfrag(it->first);
-      unsigned nonce = it->second;
+    for (const auto &q : p.second.dirs) {
+      CDir *dir = get_dirfrag(q.first);
+      unsigned nonce = q.second;
       
       if (!dir) {
-       CInode *diri = get_inode(it->first.ino);
+       CInode *diri = get_inode(q.first.ino);
        if (diri) {
          if (mds->is_rejoin() &&
              rejoin_ack_gather.count(mds->get_nodeid()) && // haven't sent rejoin ack yet
              !diri->is_replica(from)) {
            list<CDir*> ls;
            diri->get_nested_dirfrags(ls);
-           dout(7) << " dir expire on dirfrag " << it->first << " from mds." << from
+           dout(7) << " dir expire on dirfrag " << q.first << " from mds." << from
                    << " while rejoining, inode isn't replicated" << dendl;
            for (list<CDir*>::iterator q = ls.begin(); q != ls.end(); ++q) {
              dir = *q;
@@ -7449,14 +7352,14 @@ void MDCache::handle_cache_expire(MCacheExpire *m)
            }
            continue;
          }
-         CDir *other = diri->get_approx_dirfrag(it->first.frag);
+         CDir *other = diri->get_approx_dirfrag(q.first.frag);
          if (other) {
-           dout(7) << " dir expire on dirfrag " << it->first << " from mds." << from
+           dout(7) << " dir expire on dirfrag " << q.first << " from mds." << from
                    << " have " << *other << ", mismatched frags, dropping" << dendl;
            continue;
          }
        }
-       dout(0) << " dir expire on " << it->first << " from " << from
+       dout(0) << " dir expire on " << q.first << " from " << from
                << ", don't have it" << dendl;
        assert(dir);
       }
@@ -7480,42 +7383,38 @@ void MDCache::handle_cache_expire(MCacheExpire *m)
     }
     
     // DENTRIES
-    for (map<dirfrag_t, map<pair<string,snapid_t>,uint32_t> >::iterator pd = p->second.dentries.begin();
-        pd != p->second.dentries.end();
-        ++pd) {
-      dout(10) << " dn expires in dir " << pd->first << dendl;
-      CInode *diri = get_inode(pd->first.ino);
+    for (const auto &pd : p.second.dentries) {
+      dout(10) << " dn expires in dir " << pd.first << dendl;
+      CInode *diri = get_inode(pd.first.ino);
       assert(diri);
-      CDir *dir = diri->get_dirfrag(pd->first.frag);
+      CDir *dir = diri->get_dirfrag(pd.first.frag);
       
       if (!dir) {
-       dout(0) << " dn expires on " << pd->first << " from " << from
+       dout(0) << " dn expires on " << pd.first << " from " << from
                << ", must have refragmented" << dendl;
       } else {
        assert(dir->is_auth());
       }
       
-      for (map<pair<string,snapid_t>,uint32_t>::iterator p = pd->second.begin();
-          p != pd->second.end();
-          ++p) {
-       unsigned nonce = p->second;
+      for (const auto &p : pd.second) {
+       unsigned nonce = p.second;
        CDentry *dn;
        
        if (dir) {
-         dn = dir->lookup(p->first.first, p->first.second);
+         dn = dir->lookup(p.first.first, p.first.second);
        } else {
          // which dirfrag for this dentry?
-         CDir *dir = diri->get_dirfrag(diri->pick_dirfrag(p->first.first));
+         CDir *dir = diri->get_dirfrag(diri->pick_dirfrag(p.first.first));
          assert(dir); 
          assert(dir->is_auth());
-         dn = dir->lookup(p->first.first, p->first.second);
+         dn = dir->lookup(p.first.first, p.first.second);
        }
 
        if (!dn) { 
          if (dir)
-           dout(0) << "  missing dentry for " << p->first.first << " snap " << p->first.second << " in " << *dir << dendl;
+           dout(0) << "  missing dentry for " << p.first.first << " snap " << p.first.second << " in " << *dir << dendl;
          else
-           dout(0) << "  missing dentry for " << p->first.first << " snap " << p->first.second << dendl;
+           dout(0) << "  missing dentry for " << p.first.first << " snap " << p.first.second << dendl;
        }
        assert(dn);
        
@@ -7532,9 +7431,6 @@ void MDCache::handle_cache_expire(MCacheExpire *m)
     }
   }
 
-  // done
-  m->put();
-
   for (set<SimpleLock*>::iterator p = gather_locks.begin(); p != gather_locks.end(); ++p) {
     if (!(*p)->is_stable())
       mds->locker->eval_gather(*p);
@@ -7544,20 +7440,15 @@ void MDCache::handle_cache_expire(MCacheExpire *m)
 void MDCache::process_delayed_expire(CDir *dir)
 {
   dout(7) << "process_delayed_expire on " << *dir << dendl;
-  for (map<mds_rank_t,MCacheExpire*>::iterator p = delayed_expire[dir].begin();
-       p != delayed_expire[dir].end();
-       ++p) 
-    handle_cache_expire(p->second);
+  for (const auto &p : delayed_expire[dir]) {
+    handle_cache_expire(p.second);
+  }
   delayed_expire.erase(dir);  
 }
 
 void MDCache::discard_delayed_expire(CDir *dir)
 {
   dout(7) << "discard_delayed_expire on " << *dir << dendl;
-  for (map<mds_rank_t,MCacheExpire*>::iterator p = delayed_expire[dir].begin();
-       p != delayed_expire[dir].end();
-       ++p) 
-    p->second->put();
   delayed_expire.erase(dir);  
 }
 
@@ -7962,66 +7853,65 @@ bool MDCache::shutdown_export_strays()
 
 // ========= messaging ==============
 
-/* This function DOES put the passed message before returning */
-void MDCache::dispatch(Message *m)
+void MDCache::dispatch(const Message::const_ref &m)
 {
   switch (m->get_type()) {
 
     // RESOLVE
   case MSG_MDS_RESOLVE:
-    handle_resolve(static_cast<MMDSResolve*>(m));
+    handle_resolve(boost::static_pointer_cast<MMDSResolve::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_RESOLVEACK:
-    handle_resolve_ack(static_cast<MMDSResolveAck*>(m));
+    handle_resolve_ack(boost::static_pointer_cast<MMDSResolveAck::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
     // REJOIN
   case MSG_MDS_CACHEREJOIN:
-    handle_cache_rejoin(static_cast<MMDSCacheRejoin*>(m));
+    handle_cache_rejoin(boost::static_pointer_cast<MMDSCacheRejoin::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
   case MSG_MDS_DISCOVER:
-    handle_discover(static_cast<MDiscover*>(m));
+    handle_discover(boost::static_pointer_cast<MDiscover::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_DISCOVERREPLY:
-    handle_discover_reply(static_cast<MDiscoverReply*>(m));
+    handle_discover_reply(boost::static_pointer_cast<MDiscoverReply::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
   case MSG_MDS_DIRUPDATE:
-    handle_dir_update(static_cast<MDirUpdate*>(m));
+    handle_dir_update(boost::static_pointer_cast<MDirUpdate::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
   case MSG_MDS_CACHEEXPIRE:
-    handle_cache_expire(static_cast<MCacheExpire*>(m));
+    handle_cache_expire(boost::static_pointer_cast<MCacheExpire::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
   case MSG_MDS_DENTRYLINK:
-    handle_dentry_link(static_cast<MDentryLink*>(m));
+    handle_dentry_link(boost::static_pointer_cast<MDentryLink::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_DENTRYUNLINK:
-    handle_dentry_unlink(static_cast<MDentryUnlink*>(m));
+    handle_dentry_unlink(boost::static_pointer_cast<MDentryUnlink::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
   case MSG_MDS_FRAGMENTNOTIFY:
-    handle_fragment_notify(static_cast<MMDSFragmentNotify*>(m));
+    handle_fragment_notify(boost::static_pointer_cast<MMDSFragmentNotify::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
   case MSG_MDS_FINDINO:
-    handle_find_ino(static_cast<MMDSFindIno *>(m));
+    handle_find_ino(boost::static_pointer_cast<MMDSFindIno::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_FINDINOREPLY:
-    handle_find_ino_reply(static_cast<MMDSFindInoReply *>(m));
+    handle_find_ino_reply(boost::static_pointer_cast<MMDSFindInoReply::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
   case MSG_MDS_OPENINO:
-    handle_open_ino(static_cast<MMDSOpenIno *>(m));
+    handle_open_ino(boost::static_pointer_cast<MMDSOpenIno::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_OPENINOREPLY:
-    handle_open_ino_reply(static_cast<MMDSOpenInoReply *>(m));
+    handle_open_ino_reply(boost::static_pointer_cast<MMDSOpenInoReply::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
   case MSG_MDS_SNAPUPDATE:
-    handle_snap_update(static_cast<MMDSSnapUpdate*>(m));
+    handle_snap_update(boost::static_pointer_cast<MMDSSnapUpdate::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
     
   default:
@@ -8030,31 +7920,17 @@ void MDCache::dispatch(Message *m)
   }
 }
 
-MDSInternalContextBase *MDCache::_get_waiter(MDRequestRef& mdr, Message *req, MDSInternalContextBase *fin)
-{
-  if (mdr) {
-    dout(20) << "_get_waiter retryrequest" << dendl;
-    return new C_MDS_RetryRequest(this, mdr);
-  } else if (req) {
-    dout(20) << "_get_waiter retrymessage" << dendl;
-    return new C_MDS_RetryMessage(mds, req);
-  } else {
-    return fin;
-  }
-}
-
-int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBase *fin,     // who
-                          const filepath& path,                   // what
+int MDCache::path_traverse(MDRequestRef& mdr, MDSContextFactory& cf,     // who
+                           const filepath& path,                   // what
                            vector<CDentry*> *pdnvec,         // result
-                          CInode **pin,
+                           CInode **pin,
                            int onfail)
 {
   bool discover = (onfail == MDS_TRAVERSE_DISCOVER);
   bool null_okay = (onfail == MDS_TRAVERSE_DISCOVERXLOCK);
   bool forward = (onfail == MDS_TRAVERSE_FORWARD);
 
-  assert(mdr || req || fin);
-  assert(!forward || mdr || req);  // forward requires a request
+  assert(!forward || mdr);  // forward requires a request
 
   snapid_t snapid = CEPH_NOSNAP;
   if (mdr)
@@ -8068,7 +7944,7 @@ int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBa
   CInode *cur = get_inode(path.get_ino());
   if (cur == NULL) {
     if (MDS_INO_IS_MDSDIR(path.get_ino())) 
-      open_foreign_mdsdir(path.get_ino(), _get_waiter(mdr, req, fin));
+      open_foreign_mdsdir(path.get_ino(), cf.build());
     else {
       //ceph_abort();  // hrm.. broken
       return -ESTALE;
@@ -8080,7 +7956,7 @@ int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBa
 
   // make sure snaprealm are open...
   if (mdr && cur->snaprealm && !cur->snaprealm->have_past_parents_open() &&
-      !cur->snaprealm->open_parents(_get_waiter(mdr, req, fin))) {
+      !cur->snaprealm->open_parents(cf.build())) {
     return 1;
   }
 
@@ -8124,7 +8000,7 @@ int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBa
          // snapshot, but snap update message hasn't been received.
          if (!t->snaplock.can_read(client)) {
            dout(10) << " non-readable snaplock on " << *t << dendl;
-           t->snaplock.add_waiter(SimpleLock::WAIT_RD, _get_waiter(mdr, req, fin));
+           t->snaplock.add_waiter(SimpleLock::WAIT_RD, cf.build());
            return 1;
          }
          CDentry *pdn = t->get_projected_parent_dn();
@@ -8145,14 +8021,14 @@ int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBa
         // parent dir frozen_dir?
         if (cur->is_frozen()) {
           dout(7) << "traverse: " << *cur << " is frozen, waiting" << dendl;
-          cur->add_waiter(CDir::WAIT_UNFREEZE, _get_waiter(mdr, req, fin));
+          cur->add_waiter(CDir::WAIT_UNFREEZE, cf.build());
           return 1;
         }
         curdir = cur->get_or_open_dirfrag(this, fg);
       } else {
         // discover?
        dout(10) << "traverse: need dirfrag " << fg << ", doing discover from " << *cur << dendl;
-       discover_path(cur, snapid, path.postfixpath(depth), _get_waiter(mdr, req, fin),
+       discover_path(cur, snapid, path.postfixpath(depth), cf.build(),
                      null_okay);
        if (mds->logger) mds->logger->inc(l_mds_traverse_discover);
         return 1;
@@ -8205,7 +8081,7 @@ int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBa
        !dn->lock.can_read(client) &&
        (dnl->is_null() || forward)) {
       dout(10) << "traverse: xlocked dentry at " << *dn << dendl;
-      dn->lock.add_waiter(SimpleLock::WAIT_RD, _get_waiter(mdr, req, fin));
+      dn->lock.add_waiter(SimpleLock::WAIT_RD, cf.build());
       if (mds->logger) mds->logger->inc(l_mds_traverse_lock);
       mds->mdlog->flush();
       return 1;
@@ -8225,7 +8101,7 @@ int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBa
         return -ENOENT;
       } else {
         dout(10) << "miss on dentry " << *dn << ", can't read due to lock" << dendl;
-        dn->lock.add_waiter(SimpleLock::WAIT_RD, _get_waiter(mdr, req, fin));
+        dn->lock.add_waiter(SimpleLock::WAIT_RD, cf.build());
         return 1;
       }
     }
@@ -8249,7 +8125,7 @@ int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBa
                     << *dn << dendl;
             return -EIO;
           }
-          open_remote_dentry(dn, true, _get_waiter(mdr, req, fin),
+          open_remote_dentry(dn, true, cf.build(),
                             (null_okay && depth == path.depth() - 1));
          if (mds->logger) mds->logger->inc(l_mds_traverse_remote_ino);
           return 1;
@@ -8259,7 +8135,7 @@ int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBa
       cur = in;
       // make sure snaprealm are open...
       if (mdr && cur->snaprealm && !cur->snaprealm->have_past_parents_open() &&
-         !cur->snaprealm->open_parents(_get_waiter(mdr, req, fin))) {
+         !cur->snaprealm->open_parents(cf.build())) {
        return 1;
       }
 
@@ -8319,7 +8195,7 @@ int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBa
        // directory isn't complete; reload
         dout(7) << "traverse: incomplete dir contents for " << *cur << ", fetching" << dendl;
         touch_inode(cur);
-        curdir->fetch(_get_waiter(mdr, req, fin), path[depth]);
+        curdir->fetch(cf.build(), path[depth]);
        if (mds->logger) mds->logger->inc(l_mds_traverse_dir_fetch);
         return 1;
       }
@@ -8338,7 +8214,7 @@ int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBa
 
       if ((discover || null_okay)) {
        dout(7) << "traverse: discover from " << path[depth] << " from " << *curdir << dendl;
-       discover_path(curdir, snapid, path.postfixpath(depth), _get_waiter(mdr, req, fin),
+       discover_path(curdir, snapid, path.postfixpath(depth), cf.build(),
                      null_okay);
        if (mds->logger) mds->logger->inc(l_mds_traverse_discover);
         return 1;
@@ -8350,23 +8226,19 @@ int MDCache::path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBa
        if (curdir->is_ambiguous_auth()) {
          // wait
          dout(7) << "traverse: waiting for single auth in " << *curdir << dendl;
-         curdir->add_waiter(CDir::WAIT_SINGLEAUTH, _get_waiter(mdr, req, fin));
+         curdir->add_waiter(CDir::WAIT_SINGLEAUTH, cf.build());
          return 1;
        } 
 
        dout(7) << "traverse: forwarding, not auth for " << *curdir << dendl;
-       
-       if (mdr) 
-         request_forward(mdr, dauth.first);
-       else
-         mds->forward_message_mds(req, dauth.first);
-       
+
+        request_forward(mdr, dauth.first);
+
        if (mds->logger) mds->logger->inc(l_mds_traverse_forward);
-       assert(fin == NULL);
        return 2;
-      }    
+      }
     }
-    
+
     ceph_abort();  // i shouldn't get here
   }
   
@@ -8552,10 +8424,10 @@ class C_IO_MDC_OpenInoBacktraceFetched : public MDCacheIOContext {
 
 struct C_MDC_OpenInoTraverseDir : public MDCacheContext {
   inodeno_t ino;
-  MMDSOpenIno *msg;
+  MMDSOpenIno::const_ref msg;
   bool parent;
   public:
-  C_MDC_OpenInoTraverseDir(MDCache *c, inodeno_t i, MMDSOpenIno *m,  bool p) :
+  C_MDC_OpenInoTraverseDir(MDCache *c, inodeno_t i, const MMDSOpenIno::const_ref &m,  bool p) :
     MDCacheContext(c), ino(i), msg(m), parent(p) {}
   void finish(int r) override {
     if (r < 0 && !parent)
@@ -8705,7 +8577,7 @@ void MDCache::_open_ino_traverse_dir(inodeno_t ino, open_ino_info_t& info, int r
   do_open_ino(ino, info, ret);
 }
 
-void MDCache::_open_ino_fetch_dir(inodeno_t ino, MMDSOpenIno *m, CDir *dir, bool parent)
+void MDCache::_open_ino_fetch_dir(inodeno_t ino, const MMDSOpenIno::const_ref &m, CDir *dir, bool parent)
 {
   if (dir->state_test(CDir::STATE_REJOINUNDEF))
     assert(dir->get_inode()->dirfragtree.is_leaf(dir->get_frag()));
@@ -8714,18 +8586,19 @@ void MDCache::_open_ino_fetch_dir(inodeno_t ino, MMDSOpenIno *m, CDir *dir, bool
     mds->logger->inc(l_mds_openino_dir_fetch);
 }
 
-int MDCache::open_ino_traverse_dir(inodeno_t ino, MMDSOpenIno *m,
-                                  vector<inode_backpointer_t>& ancestors,
+int MDCache::open_ino_traverse_dir(inodeno_t ino, const MMDSOpenIno::const_ref &m,
+                                  const vector<inode_backpointer_t>& ancestors,
                                   bool discover, bool want_xlocked, mds_rank_t *hint)
 {
   dout(10) << "open_ino_traverse_dir ino " << ino << " " << ancestors << dendl;
   int err = 0;
   for (unsigned i = 0; i < ancestors.size(); i++) {
-    CInode *diri = get_inode(ancestors[i].dirino);
+    const auto& ancestor = ancestors.at(i);
+    CInode *diri = get_inode(ancestor.dirino);
 
     if (!diri) {
-      if (discover && MDS_INO_IS_MDSDIR(ancestors[i].dirino)) {
-       open_foreign_mdsdir(ancestors[i].dirino, new C_MDC_OpenInoTraverseDir(this, ino, m, i == 0));
+      if (discover && MDS_INO_IS_MDSDIR(ancestor.dirino)) {
+       open_foreign_mdsdir(ancestor.dirino, new C_MDC_OpenInoTraverseDir(this, ino, m, i == 0));
        return 1;
       }
       continue;
@@ -8747,7 +8620,7 @@ int MDCache::open_ino_traverse_dir(inodeno_t ino, MMDSOpenIno *m,
       break;
     }
 
-    string &name = ancestors[i].dname;
+    const string& name = ancestor.dname;
     frag_t fg = diri->pick_dirfrag(name);
     CDir *dir = diri->get_dirfrag(fg);
     if (!dir) {
@@ -8764,7 +8637,7 @@ int MDCache::open_ino_traverse_dir(inodeno_t ino, MMDSOpenIno *m,
       }
     }
     if (dir) {
-      inodeno_t next_ino = i > 0 ? ancestors[i - 1].dirino : ino;
+      inodeno_t next_ino = i > 0 ? ancestors.at(i-1).dirino : ino;
       CDentry *dn = dir->lookup(name);
       CDentry::linkage_t *dnl = dn ? dn->get_linkage() : NULL;
       if (dir->is_auth()) {
@@ -8907,22 +8780,21 @@ void MDCache::do_open_ino_peer(inodeno_t ino, open_ino_info_t& info)
   }
 }
 
-void MDCache::handle_open_ino(MMDSOpenIno *m, int err)
+void MDCache::handle_open_ino(const MMDSOpenIno::const_ref &m, int err)
 {
   if (mds->get_state() < MDSMap::STATE_REJOIN &&
       mds->get_want_state() != CEPH_MDS_STATE_REJOIN) {
-    m->put();
     return;
   }
 
   dout(10) << "handle_open_ino " << *m << " err " << err << dendl;
 
   inodeno_t ino = m->ino;
-  MMDSOpenInoReply *reply;
+  MMDSOpenInoReply::ref reply;
   CInode *in = get_inode(ino);
   if (in) {
     dout(10) << " have " << *in << dendl;
-    reply = new MMDSOpenInoReply(m->get_tid(), ino, mds_rank_t(0));
+    reply.reset(new MMDSOpenInoReply(m->get_tid(), ino, mds_rank_t(0)), false);
     if (in->is_auth()) {
       touch_inode(in);
       while (1) {
@@ -8938,19 +8810,18 @@ void MDCache::handle_open_ino(MMDSOpenIno *m, int err)
       reply->hint = in->authority().first;
     }
   } else if (err < 0) {
-    reply = new MMDSOpenInoReply(m->get_tid(), ino, MDS_RANK_NONE, err);
+    reply.reset(new MMDSOpenInoReply(m->get_tid(), ino, MDS_RANK_NONE, err), false);
   } else {
     mds_rank_t hint = MDS_RANK_NONE;
     int ret = open_ino_traverse_dir(ino, m, m->ancestors, false, false, &hint);
     if (ret > 0)
       return;
-    reply = new MMDSOpenInoReply(m->get_tid(), ino, hint, ret);
+    reply.reset(new MMDSOpenInoReply(m->get_tid(), ino, hint, ret), false);
   }
-  m->get_connection()->send_message(reply);
-  m->put();
+  m->get_connection()->send_message2(reply); /* FIXME, why not send_client? */
 }
 
-void MDCache::handle_open_ino_reply(MMDSOpenInoReply *m)
+void MDCache::handle_open_ino_reply(const MMDSOpenInoReply::const_ref &m)
 {
   dout(10) << "handle_open_ino_reply " << *m << dendl;
 
@@ -8970,7 +8841,6 @@ void MDCache::handle_open_ino_reply(MMDSOpenInoReply *m)
       dout(10) << " found ino " << ino << " on mds." << from << dendl;
       if (!info.want_replica) {
        open_ino_finish(ino, info, from);
-        m->put();
        return;
       }
 
@@ -8990,7 +8860,6 @@ void MDCache::handle_open_ino_reply(MMDSOpenInoReply *m)
       do_open_ino_peer(ino, info);
     }
   }
-  m->put();
 }
 
 void MDCache::kick_open_ino_peers(mds_rank_t who)
@@ -9125,10 +8994,9 @@ void MDCache::_do_find_ino_peer(find_ino_peer_info_t& fip)
   }
 }
 
-void MDCache::handle_find_ino(MMDSFindIno *m)
+void MDCache::handle_find_ino(const MMDSFindIno::const_ref &m)
 {
   if (mds->get_state() < MDSMap::STATE_REJOIN) {
-    m->put();
     return;
   }
 
@@ -9140,11 +9008,10 @@ void MDCache::handle_find_ino(MMDSFindIno *m)
     dout(10) << " have " << r->path << " " << *in << dendl;
   }
   m->get_connection()->send_message(r);
-  m->put();
 }
 
 
-void MDCache::handle_find_ino_reply(MMDSFindInoReply *m)
+void MDCache::handle_find_ino_reply(const MMDSFindInoReply::const_ref &m)
 {
   map<ceph_tid_t, find_ino_peer_info_t>::iterator p = find_ino_peer.find(m->tid);
   if (p != find_ino_peer.end()) {
@@ -9156,7 +9023,6 @@ void MDCache::handle_find_ino_reply(MMDSFindInoReply *m)
       dout(10) << "handle_find_ino_reply successfully found " << fip.ino << dendl;
       mds->queue_waiter(fip.fin);
       find_ino_peer.erase(p);
-      m->put();
       return;
     }
 
@@ -9168,8 +9034,9 @@ void MDCache::handle_find_ino_reply(MMDSFindInoReply *m)
     if (!m->path.empty()) {
       // we got a path!
       vector<CDentry*> trace;
+      CF_MDS_RetryMessageFactory cf(mds, m);
       MDRequestRef null_ref;
-      int r = path_traverse(null_ref, m, NULL, m->path, &trace, NULL, MDS_TRAVERSE_DISCOVER);
+      int r = path_traverse(null_ref, cf, m->path, &trace, NULL, MDS_TRAVERSE_DISCOVER);
       if (r > 0)
        return; 
       dout(0) << "handle_find_ino_reply failed with " << r << " on " << m->path 
@@ -9183,7 +9050,6 @@ void MDCache::handle_find_ino_reply(MMDSFindInoReply *m)
   } else {
     dout(10) << "handle_find_ino_reply tid " << m->tid << " dne" << dendl;
   }  
-  m->put();
 }
 
 void MDCache::kick_find_ino_peers(mds_rank_t who)
@@ -9219,8 +9085,7 @@ int MDCache::get_num_client_requests()
   return count;
 }
 
-/* This function takes over the reference to the passed Message */
-MDRequestRef MDCache::request_start(MClientRequest *req)
+MDRequestRef MDCache::request_start(const MClientRequest::const_ref& req)
 {
   // did we win a forward race against a slave?
   if (active_requests.count(req->get_reqid())) {
@@ -9231,7 +9096,6 @@ MDRequestRef MDCache::request_start(MClientRequest *req)
       mdr->more()->waiting_for_finish.push_back(new C_MDS_RetryMessage(mds, req));
     } else {
       dout(10) << "request_start already processing " << *mdr << ", dropping new msg" << dendl;
-      req->put();
     }
     return MDRequestRef();
   }
@@ -9254,7 +9118,7 @@ MDRequestRef MDCache::request_start(MClientRequest *req)
   return mdr;
 }
 
-MDRequestRef MDCache::request_start_slave(metareqid_t ri, __u32 attempt, Message *m)
+MDRequestRef MDCache::request_start_slave(metareqid_t ri, __u32 attempt, const Message::const_ref &m)
 {
   int by = m->get_source().num();
   MDRequestImpl::Params params;
@@ -9573,7 +9437,7 @@ void MDCache::do_realm_invalidate_and_update_notify(CInode *in, int snapop, bool
   }
 
   set<SnapRealm*> past_children;
-  map<client_t, MClientSnap*> updates;
+  map<client_t, MClientSnap::ref> updates;
   list<SnapRealm*> q;
   q.push_back(in->snaprealm);
   while (!q.empty()) {
@@ -9584,17 +9448,19 @@ void MDCache::do_realm_invalidate_and_update_notify(CInode *in, int snapop, bool
     realm->invalidate_cached_snaps();
 
     if (notify_clients) {
-      for (map<client_t, xlist<Capability*>* >::iterator p = realm->client_caps.begin();
-          p != realm->client_caps.end();
-          ++p) {
-       assert(!p->second->empty());
-       if (updates.count(p->first) == 0) {
-         MClientSnap *update = new MClientSnap(snapop);
+      for (const auto& p : realm->client_caps) {
+        const auto& client = p.first;
+        const auto& caps = p.second;
+       assert(!caps->empty());
+
+        auto em = updates.emplace(std::piecewise_construct, std::forward_as_tuple(client), std::forward_as_tuple());
+        if (em.second) {
+          MClientSnap::ref update(new MClientSnap(CEPH_SNAP_OP_SPLIT), false);
          update->head.split = in->ino();
          update->split_inos = split_inos;
          update->split_realms = split_realms;
          update->bl = in->snaprealm->get_snap_trace();
-         updates[p->first] = update;
+         em.first->second = std::move(update);
        }
       }
     }
@@ -9673,7 +9539,7 @@ void MDCache::send_snap_update(CInode *in, version_t stid, int snap_op)
     in->encode_snap(snap_blob);
 
     for (auto p : mds_set) {
-      MMDSSnapUpdate *m = new MMDSSnapUpdate(in->ino(), stid, snap_op);
+      MMDSSnapUpdate::ref m(new MMDSSnapUpdate(in->ino(), stid, snap_op), false);
       m->snap_blob = snap_blob;
       mds->send_message_mds(m, p);
     }
@@ -9683,14 +9549,13 @@ void MDCache::send_snap_update(CInode *in, version_t stid, int snap_op)
     notify_global_snaprealm_update(snap_op);
 }
 
-void MDCache::handle_snap_update(MMDSSnapUpdate *m)
+void MDCache::handle_snap_update(const MMDSSnapUpdate::const_ref &m)
 {
   mds_rank_t from = mds_rank_t(m->get_source().num());
   dout(10) << __func__ << " " << *m << " from mds." << from << dendl;
 
   if (mds->get_state() < MDSMap::STATE_RESOLVE &&
       mds->get_want_state() != CEPH_MDS_STATE_RESOLVE) {
-    m->put();
     return;
   }
 
@@ -9721,8 +9586,6 @@ void MDCache::handle_snap_update(MMDSSnapUpdate *m)
       do_realm_invalidate_and_update_notify(in, m->get_snap_op(), notify_clients);
     }
   }
-
-  m->put();
 }
 
 void MDCache::notify_global_snaprealm_update(int snap_op)
@@ -9810,8 +9673,7 @@ void MDCache::fetch_backtrace(inodeno_t ino, int64_t pool, bufferlist& bl, Conte
 
 void MDCache::_send_discover(discover_info_t& d)
 {
-  MDiscover *dis = new MDiscover(d.ino, d.frag, d.snap, d.want_path,
-                                d.want_base_dir, d.want_xlocked);
+  MDiscover::ref dis(new MDiscover(d.ino, d.frag, d.snap, d.want_path, d.want_base_dir, d.want_xlocked), false);
   dis->set_tid(d.tid);
   mds->send_message_mds(dis, d.mds);
 }
@@ -9979,8 +9841,7 @@ void MDCache::kick_discovers(mds_rank_t who)
 }
 
 
-/* This function DOES put the passed message before returning */
-void MDCache::handle_discover(MDiscover *dis) 
+void MDCache::handle_discover(const MDiscover::const_ref &dis) 
 {
   mds_rank_t whoami = mds->get_nodeid();
   mds_rank_t from = mds_rank_t(dis->get_source().num());
@@ -9990,7 +9851,6 @@ void MDCache::handle_discover(MDiscover *dis)
   if (mds->get_state() <= MDSMap::STATE_REJOIN) {
     if (mds->get_state() < MDSMap::STATE_REJOIN &&
        mds->get_want_state() < CEPH_MDS_STATE_REJOIN) {
-      dis->put();
       return;
     }
 
@@ -10005,7 +9865,7 @@ void MDCache::handle_discover(MDiscover *dis)
 
 
   CInode *cur = 0;
-  MDiscoverReply *reply = new MDiscoverReply(dis);
+  MDiscoverReply::ref reply(new MDiscoverReply(*dis), false);
 
   snapid_t snapid = dis->get_snapid();
 
@@ -10128,7 +9988,6 @@ void MDCache::handle_discover(MDiscover *dis)
        }
        dout(7) << *cur << " is frozen, empty reply, waiting" << dendl;
        cur->add_waiter(CInode::WAIT_UNFREEZE, new C_MDS_RetryMessage(mds, dis));
-       reply->put();
        return;
       }
       curdir = cur->get_or_open_dirfrag(this, fg);
@@ -10145,7 +10004,6 @@ void MDCache::handle_discover(MDiscover *dis)
       }
       dout(7) << *curdir << " is frozen, empty reply, waiting" << dendl;
       curdir->add_waiter(CDir::WAIT_UNFREEZE, new C_MDS_RetryMessage(mds, dis));
-      reply->put();
       return;
     }
     
@@ -10188,7 +10046,6 @@ void MDCache::handle_discover(MDiscover *dis)
          // fetch and wait
          curdir->fetch(new C_MDS_RetryMessage(mds, dis),
                        dis->wants_base_dir() && curdir->get_version() == 0);
-         reply->put();
          return;
        } else {
          // initiate fetch, but send what we have so far
@@ -10233,7 +10090,6 @@ void MDCache::handle_discover(MDiscover *dis)
       } else if (reply->is_empty()) {
        dout(7) << "handle_discover blocking on xlocked " << *dn << dendl;
        dn->lock.add_waiter(SimpleLock::WAIT_RD, new C_MDS_RetryMessage(mds, dis));
-       reply->put();
        return;
       } else {
        dout(7) << "handle_discover non-empty reply, xlocked tail " << *dn << dendl;
@@ -10248,7 +10104,6 @@ void MDCache::handle_discover(MDiscover *dis)
       } else if (reply->is_empty()) {
        dout(7) << *dnl->get_inode() << " is frozen, empty reply, waiting" << dendl;
        dnl->get_inode()->add_waiter(CDir::WAIT_UNFREEZE, new C_MDS_RetryMessage(mds, dis));
-       reply->put();
        return;
       } else {
        dout(7) << *dnl->get_inode() << " is frozen, non-empty reply, stopping" << dendl;
@@ -10280,17 +10135,13 @@ void MDCache::handle_discover(MDiscover *dis)
   assert(!reply->is_empty());
   dout(7) << "handle_discover sending result back to asker mds." << from << dendl;
   mds->send_message(reply, dis->get_connection());
-
-  dis->put();
 }
 
-/* This function DOES put the passed message before returning */
-void MDCache::handle_discover_reply(MDiscoverReply *m) 
+void MDCache::handle_discover_reply(const MDiscoverReply::const_ref &m) 
 {
   /*
   if (mds->get_state() < MDSMap::STATE_ACTIVE) {
     dout(0) << "discover_reply NOT ACTIVE YET" << dendl;
-    m->put();
     return;
   }
   */
@@ -10434,9 +10285,6 @@ void MDCache::handle_discover_reply(MDiscoverReply *m)
   // waiters
   finish_contexts(g_ceph_context, error, -ENOENT);  // finish errors directly
   mds->queue_waiters(finished);
-
-  // done
-  m->put();
 }
 
 
@@ -10578,7 +10426,7 @@ void MDCache::replicate_stray(CDentry *straydn, mds_rank_t who, bufferlist& bl)
   replicate_dentry(straydn, who, bl);
 }
    
-CDentry *MDCache::add_replica_stray(bufferlist &bl, mds_rank_t from)
+CDentry *MDCache::add_replica_stray(const bufferlist &bl, mds_rank_t from)
 {
   MDSInternalContextBase::vec finished;
   auto p = bl.cbegin();
@@ -10638,8 +10486,7 @@ int MDCache::send_dir_updates(CDir *dir, bool bcast)
   return 0;
 }
 
-/* This function DOES put the passed message before returning */
-void MDCache::handle_dir_update(MDirUpdate *m)
+void MDCache::handle_dir_update(const MDirUpdate::const_ref &m)
 {
   dirfrag_t df = m->get_dirfrag();
   CDir *dir = get_dirfrag(df);
@@ -10655,8 +10502,9 @@ void MDCache::handle_dir_update(MDirUpdate *m)
       CInode *in;
       filepath path = m->get_path();
       dout(5) << "trying discover on dir_update for " << path << dendl;
+      CF_MDS_RetryMessageFactory cf(mds, m);
       MDRequestRef null_ref;
-      int r = path_traverse(null_ref, m, NULL, path, &trace, &in, MDS_TRAVERSE_DISCOVER);
+      int r = path_traverse(null_ref, cf, path, &trace, &in, MDS_TRAVERSE_DISCOVER);
       if (r > 0)
         return;
       if (r == 0 &&
@@ -10667,7 +10515,6 @@ void MDCache::handle_dir_update(MDirUpdate *m)
       }
     }
 
-    m->put();
     return;
   }
 
@@ -10680,9 +10527,6 @@ void MDCache::handle_dir_update(MDirUpdate *m)
       dir->dir_rep_by.insert(e);
     }
   }
-
-  // done
-  m->put();
 }
 
 
@@ -10723,10 +10567,8 @@ void MDCache::send_dentry_link(CDentry *dn, MDRequestRef& mdr)
   }
 }
 
-/* This function DOES put the passed message before returning */
-void MDCache::handle_dentry_link(MDentryLink *m)
+void MDCache::handle_dentry_link(const MDentryLink::const_ref &m)
 {
-
   CDentry *dn = NULL;
   CDir *dir = get_dirfrag(m->get_dirfrag());
   if (!dir) {
@@ -10765,7 +10607,6 @@ void MDCache::handle_dentry_link(MDentryLink *m)
   if (!finished.empty())
     mds->queue_waiters(finished);
 
-  m->put();
   return;
 }
 
@@ -10805,8 +10646,7 @@ void MDCache::send_dentry_unlink(CDentry *dn, CDentry *straydn, MDRequestRef& md
   }
 }
 
-/* This function DOES put the passed message before returning */
-void MDCache::handle_dentry_unlink(MDentryUnlink *m)
+void MDCache::handle_dentry_unlink(const MDentryUnlink::const_ref &m)
 {
   // straydn
   CDentry *straydn = NULL;
@@ -10868,13 +10708,10 @@ void MDCache::handle_dentry_unlink(MDentryUnlink *m)
   if (straydn) {
     assert(straydn->get_num_ref() == 0);
     assert(straydn->get_linkage()->is_null());
-    map<mds_rank_t, MCacheExpire*> expiremap;
-    trim_dentry(straydn, expiremap);
-    send_expire_messages(expiremap);
+    expiremap ex;
+    trim_dentry(straydn, ex);
+    send_expire_messages(ex);
   }
-
-  m->put();
-  return;
 }
 
 
@@ -11765,13 +11602,11 @@ void MDCache::_fragment_finish(dirfrag_t basedirfrag, list<CDir*>& resultfrags)
   finish_uncommitted_fragment(basedirfrag, EFragment::OP_FINISH);
 }
 
-/* This function DOES put the passed message before returning */
-void MDCache::handle_fragment_notify(MMDSFragmentNotify *notify)
+void MDCache::handle_fragment_notify(const MMDSFragmentNotify::const_ref &notify)
 {
   dout(10) << "handle_fragment_notify " << *notify << " from " << notify->get_source() << dendl;
 
   if (mds->get_state() < MDSMap::STATE_REJOIN) {
-    notify->put();
     return;
   }
 
@@ -11785,7 +11620,6 @@ void MDCache::handle_fragment_notify(MMDSFragmentNotify *notify)
        (bits > 0 && !diri->dirfragtree.is_leaf(base))) {
       dout(10) << " dft " << diri->dirfragtree << " state doesn't match " << base << " by " << bits
               << ", must have found out during resolve/rejoin?  ignoring. " << *diri << dendl;
-      notify->put();
       return;
     }
 */
@@ -11809,8 +11643,6 @@ void MDCache::handle_fragment_notify(MMDSFragmentNotify *notify)
   } else {
     ceph_abort();
   }
-
-  notify->put();
 }
 
 void MDCache::add_uncommitted_fragment(dirfrag_t basedirfrag, int bits, list<frag_t>& old_frags,
index 8e323dbade42136f74e102ce25d2f3a095e4d51f..bc73515095de8243d1780caa13c03439109d167f 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
 #include "include/filepath.h"
 #include "include/elist.h"
 
+#include "messages/MCacheExpire.h"
+#include "messages/MClientQuota.h"
+#include "messages/MClientRequest.h"
+#include "messages/MClientSnap.h"
+#include "messages/MDentryLink.h"
+#include "messages/MDentryUnlink.h"
+#include "messages/MDirUpdate.h"
+#include "messages/MDiscover.h"
+#include "messages/MDiscoverReply.h"
+#include "messages/MGatherCaps.h"
+#include "messages/MGenericMessage.h"
+#include "messages/MInodeFileCaps.h"
+#include "messages/MLock.h"
+#include "messages/MMDSCacheRejoin.h"
+#include "messages/MMDSFindIno.h"
+#include "messages/MMDSFindInoReply.h"
+#include "messages/MMDSFragmentNotify.h"
+#include "messages/MMDSOpenIno.h"
+#include "messages/MMDSOpenInoReply.h"
+#include "messages/MMDSResolve.h"
+#include "messages/MMDSResolveAck.h"
+#include "messages/MMDSSlaveRequest.h"
+#include "messages/MMDSSnapUpdate.h"
+
+
 #include "osdc/Filer.h"
 #include "CInode.h"
 #include "CDentry.h"
@@ -36,8 +61,6 @@
 #include "MDSMap.h"
 #include "Mutation.h"
 
-#include "messages/MClientRequest.h"
-#include "messages/MMDSSlaveRequest.h"
 
 class PerfCounters;
 
@@ -45,32 +68,8 @@ class MDSRank;
 class Session;
 class Migrator;
 
-class Message;
 class Session;
 
-class MMDSResolve;
-class MMDSResolveAck;
-class MMDSCacheRejoin;
-class MDiscover;
-class MDiscoverReply;
-class MCacheExpire;
-class MDirUpdate;
-class MDentryLink;
-class MDentryUnlink;
-class MLock;
-struct MMDSFindIno;
-struct MMDSFindInoReply;
-struct MMDSOpenIno;
-struct MMDSOpenInoReply;
-class MMDSSnapUpdate;
-
-class Message;
-class MClientRequest;
-class MMDSSlaveRequest;
-struct MClientSnap;
-
-class MMDSFragmentNotify;
-
 class ESubtreeMap;
 
 enum {
@@ -120,6 +119,8 @@ static const int PREDIRTY_SHALLOW = 4; // only go to immediate parent (for easie
 
 class MDCache {
  public:
+  typedef std::map<mds_rank_t, MCacheExpire::ref> expiremap;
+
   // my master
   MDSRank *mds;
 
@@ -314,15 +315,15 @@ public:
   void adjust_subtree_auth(CDir *root, mds_rank_t a, mds_rank_t b=CDIR_AUTH_UNKNOWN) {
     adjust_subtree_auth(root, mds_authority_t(a,b));
   }
-  void adjust_bounded_subtree_auth(CDir *dir, set<CDir*>& bounds, mds_authority_t auth);
-  void adjust_bounded_subtree_auth(CDir *dir, set<CDir*>& bounds, mds_rank_t a) {
+  void adjust_bounded_subtree_auth(CDir *dir, const set<CDir*>& bounds, mds_authority_t auth);
+  void adjust_bounded_subtree_auth(CDir *dir, const set<CDir*>& bounds, mds_rank_t a) {
     adjust_bounded_subtree_auth(dir, bounds, mds_authority_t(a, CDIR_AUTH_UNKNOWN));
   }
-  void adjust_bounded_subtree_auth(CDir *dir, vector<dirfrag_t>& bounds, const mds_authority_t &auth);
-  void adjust_bounded_subtree_auth(CDir *dir, vector<dirfrag_t>& bounds, mds_rank_t a) {
+  void adjust_bounded_subtree_auth(CDir *dir, const vector<dirfrag_t>& bounds, const mds_authority_t &auth);
+  void adjust_bounded_subtree_auth(CDir *dir, const vector<dirfrag_t>& bounds, mds_rank_t a) {
     adjust_bounded_subtree_auth(dir, bounds, mds_authority_t(a, CDIR_AUTH_UNKNOWN));
   }
-  void map_dirfrag_set(list<dirfrag_t>& dfs, set<CDir*>& result);
+  void map_dirfrag_set(const list<dirfrag_t>& dfs, set<CDir*>& result);
   void try_subtree_merge(CDir *root);
   void try_subtree_merge_at(CDir *root, set<CInode*> *to_eval, bool adjust_pop=true);
   void subtree_merge_writebehind_finish(CInode *in, MutationRef& mut);
@@ -354,18 +355,14 @@ public:
 
   
 protected:
-  // delayed cache expire
-  map<CDir*, map<mds_rank_t, MCacheExpire*> > delayed_expire; // subtree root -> expire msg
-
-
   // -- requests --
   ceph::unordered_map<metareqid_t, MDRequestRef> active_requests;
 
 public:
   int get_num_client_requests();
 
-  MDRequestRef request_start(MClientRequest *req);
-  MDRequestRef request_start_slave(metareqid_t rid, __u32 attempt, Message *m);
+  MDRequestRef request_start(const MClientRequest::const_ref& req);
+  MDRequestRef request_start_slave(metareqid_t rid, __u32 attempt, const Message::const_ref &m);
   MDRequestRef request_start_internal(int op);
   bool have_request(metareqid_t rid) {
     return active_requests.count(rid);
@@ -469,10 +466,10 @@ protected:
   set<mds_rank_t> resolve_ack_gather;  // nodes i need a resolve_ack from
   set<version_t> resolve_snapclient_commits;
   map<metareqid_t, mds_rank_t> resolve_need_rollback;  // rollbacks i'm writing to the journal
-  map<mds_rank_t, MMDSResolve*> delayed_resolve;
+  map<mds_rank_t, MMDSResolve::const_ref> delayed_resolve;
   
-  void handle_resolve(MMDSResolve *m);
-  void handle_resolve_ack(MMDSResolveAck *m);
+  void handle_resolve(const MMDSResolve::const_ref &m);
+  void handle_resolve_ack(const MMDSResolveAck::const_ref &m);
   void process_delayed_resolve();
   void discard_delayed_resolve(mds_rank_t who);
   void maybe_resolve_finish();
@@ -569,16 +566,16 @@ protected:
   list<SimpleLock*> rejoin_eval_locks;
   MDSInternalContextBase::vec rejoin_waiters;
 
-  void rejoin_walk(CDir *dir, MMDSCacheRejoin *rejoin);
-  void handle_cache_rejoin(MMDSCacheRejoin *m);
-  void handle_cache_rejoin_weak(MMDSCacheRejoin *m);
+  void rejoin_walk(CDir *dir, const MMDSCacheRejoin::ref &rejoin);
+  void handle_cache_rejoin(const MMDSCacheRejoin::const_ref &m);
+  void handle_cache_rejoin_weak(const MMDSCacheRejoin::const_ref &m);
   CInode* rejoin_invent_inode(inodeno_t ino, snapid_t last);
   CDir* rejoin_invent_dirfrag(dirfrag_t df);
-  void handle_cache_rejoin_strong(MMDSCacheRejoin *m);
-  void rejoin_scour_survivor_replicas(mds_rank_t from, MMDSCacheRejoin *ack,
+  void handle_cache_rejoin_strong(const MMDSCacheRejoin::const_ref &m);
+  void rejoin_scour_survivor_replicas(mds_rank_t from, const MMDSCacheRejoin::const_ref &ack,
                                      set<vinodeno_t>& acked_inodes,
                                      set<SimpleLock *>& gather_locks);
-  void handle_cache_rejoin_ack(MMDSCacheRejoin *m);
+  void handle_cache_rejoin_ack(const MMDSCacheRejoin::const_ref &m);
   void rejoin_send_acks();
   void rejoin_trim_undef_inodes();
   void maybe_send_pending_rejoins() {
@@ -592,14 +589,18 @@ public:
   void rejoin_gather_finish();
   void rejoin_send_rejoins();
   void rejoin_export_caps(inodeno_t ino, client_t client, const cap_reconnect_t& icr,
-                         int target=-1) {
+                         int target=-1, bool drop_path=false) {
     auto& ex = cap_exports[ino];
     ex.first = target;
-    ex.second[client] = icr;
+    auto &_icr = ex.second[client] = icr;
+    if (drop_path)
+      _icr.path.clear();
   }
   void rejoin_recovered_caps(inodeno_t ino, client_t client, const cap_reconnect_t& icr, 
-                            mds_rank_t frommds=MDS_RANK_NONE) {
-    cap_imports[ino][client][frommds] = icr;
+                            mds_rank_t frommds=MDS_RANK_NONE, bool drop_path=false) {
+    auto &_icr = cap_imports[ino][client][frommds] = icr;
+    if (drop_path)
+      _icr.path.clear();
   }
   void rejoin_recovered_client(client_t client, const entity_inst_t& inst) {
     rejoin_client_map.emplace(client, inst);
@@ -662,12 +663,12 @@ public:
   bool process_imported_caps();
   void choose_lock_states_and_reconnect_caps();
   void prepare_realm_split(SnapRealm *realm, client_t client, inodeno_t ino,
-                          map<client_t,MClientSnap*>& splits);
-  void prepare_realm_merge(SnapRealm *realm, SnapRealm *parent_realm, map<client_t,MClientSnap*>& splits);
-  void send_snaps(map<client_t,MClientSnap*>& splits);
+                          map<client_t,MClientSnap::ref>& splits);
+  void prepare_realm_merge(SnapRealm *realm, SnapRealm *parent_realm, map<client_t,MClientSnap::ref>& splits);
+  void send_snaps(map<client_t,MClientSnap::ref>& splits);
   Capability* rejoin_import_cap(CInode *in, client_t client, const cap_reconnect_t& icr, mds_rank_t frommds);
   void finish_snaprealm_reconnect(client_t client, SnapRealm *realm, snapid_t seq,
-                                 map<client_t,MClientSnap*>& updates);
+                                 map<client_t,MClientSnap::ref>& updates);
   void try_reconnect_cap(CInode *in, Session *session);
   void export_remaining_imported_caps();
 
@@ -733,13 +734,11 @@ public:
   // trimming
   bool trim(uint64_t count=0);
 private:
-  void trim_lru(uint64_t count, map<mds_rank_t, MCacheExpire*>& expiremap);
-  bool trim_dentry(CDentry *dn, map<mds_rank_t, MCacheExpire*>& expiremap);
-  void trim_dirfrag(CDir *dir, CDir *con,
-                   map<mds_rank_t, MCacheExpire*>& expiremap);
-  bool trim_inode(CDentry *dn, CInode *in, CDir *con,
-                 map<mds_rank_t,class MCacheExpire*>& expiremap);
-  void send_expire_messages(map<mds_rank_t, MCacheExpire*>& expiremap);
+  void trim_lru(uint64_t count, expiremap& expiremap);
+  bool trim_dentry(CDentry *dn, expiremap& expiremap);
+  void trim_dirfrag(CDir *dir, CDir *con, expiremap& expiremap);
+  bool trim_inode(CDentry *dn, CInode *in, CDir *con, expiremap&);
+  void send_expire_messages(expiremap& expiremap);
   void trim_non_auth();      // trim out trimmable non-auth items
 public:
   bool trim_non_auth_subtree(CDir *directory);
@@ -764,9 +763,7 @@ public:
    * @return false if we completed cleanly, true if caller should stop
    *         expiring because we hit something with refs.
    */
-  bool expire_recursive(
-    CInode *in,
-    std::map<mds_rank_t, MCacheExpire*>& expiremap);
+  bool expire_recursive(CInode *in, expiremap& expiremap);
 
   void trim_client_leases();
   void check_memory_usage();
@@ -841,7 +838,7 @@ public:
     return dir;
   }
 
-  MDSCacheObject *get_object(MDSCacheObjectInfo &info);
+  MDSCacheObject *get_object(const MDSCacheObjectInfo &info);
 
   
 
@@ -930,8 +927,6 @@ public:
   CDir *get_stray_dir(CInode *in);
   CDentry *get_or_create_stray_dentry(CInode *in);
 
-  MDSInternalContextBase *_get_waiter(MDRequestRef& mdr, Message *req, MDSInternalContextBase *fin);
-
   /**
    * Find the given dentry (and whether it exists or not), its ancestors,
    * and get them all into memory and usable on this MDS. This function
@@ -943,8 +938,7 @@ public:
    * At least one of the params mdr, req, and fin must be non-null.
    *
    * @param mdr The MDRequest associated with the path. Can be null.
-   * @param req The Message associated with the path. Can be null.
-   * @param fin The Context associated with the path. Can be null.
+   * @param cf A MDSContextFactory for waiter building.
    * @param path The path to traverse to.
    * @param pdnvec Data return parameter -- on success, contains a
    * vector of dentries. On failure, is either empty or contains the
@@ -966,7 +960,7 @@ public:
    * If it returns 2 the request has been forwarded, and again the requester
    * should unwind itself and back out.
    */
-  int path_traverse(MDRequestRef& mdr, Message *req, MDSInternalContextBase *fin, const filepath& path,
+  int path_traverse(MDRequestRef& mdr, MDSContextFactory& cf, const filepath& path,
                    vector<CDentry*> *pdnvec, CInode **pin, int onfail);
 
   CInode *cache_traverse(const filepath& path);
@@ -1012,15 +1006,15 @@ protected:
   void _open_ino_backtrace_fetched(inodeno_t ino, bufferlist& bl, int err);
   void _open_ino_parent_opened(inodeno_t ino, int ret);
   void _open_ino_traverse_dir(inodeno_t ino, open_ino_info_t& info, int err);
-  void _open_ino_fetch_dir(inodeno_t ino, MMDSOpenIno *m, CDir *dir, bool parent);
-  int open_ino_traverse_dir(inodeno_t ino, MMDSOpenIno *m,
-                           vector<inode_backpointer_t>& ancestors,
+  void _open_ino_fetch_dir(inodeno_t ino, const MMDSOpenIno::const_ref &m, CDir *dir, bool parent);
+  int open_ino_traverse_dir(inodeno_t ino, const MMDSOpenIno::const_ref &m,
+                           const vector<inode_backpointer_t>& ancestors,
                            bool discover, bool want_xlocked, mds_rank_t *hint);
   void open_ino_finish(inodeno_t ino, open_ino_info_t& info, int err);
   void do_open_ino(inodeno_t ino, open_ino_info_t& info, int err);
   void do_open_ino_peer(inodeno_t ino, open_ino_info_t& info);
-  void handle_open_ino(MMDSOpenIno *m, int err=0);
-  void handle_open_ino_reply(MMDSOpenInoReply *m);
+  void handle_open_ino(const MMDSOpenIno::const_ref &m, int err=0);
+  void handle_open_ino_reply(const MMDSOpenInoReply::const_ref &m);
   friend class C_IO_MDC_OpenInoBacktraceFetched;
   friend struct C_MDC_OpenInoTraverseDir;
   friend struct C_MDC_OpenInoParentOpened;
@@ -1047,8 +1041,8 @@ public:
 
   void find_ino_peers(inodeno_t ino, MDSInternalContextBase *c, mds_rank_t hint=MDS_RANK_NONE);
   void _do_find_ino_peer(find_ino_peer_info_t& fip);
-  void handle_find_ino(MMDSFindIno *m);
-  void handle_find_ino_reply(MMDSFindInoReply *m);
+  void handle_find_ino(const MMDSFindIno::const_ref &m);
+  void handle_find_ino_reply(const MMDSFindInoReply::const_ref &m);
   void kick_find_ino_peers(mds_rank_t who);
 
   // -- snaprealms --
@@ -1059,7 +1053,7 @@ public:
   void create_global_snaprealm();
   void do_realm_invalidate_and_update_notify(CInode *in, int snapop, bool notify_clients=true);
   void send_snap_update(CInode *in, version_t stid, int snap_op);
-  void handle_snap_update(MMDSSnapUpdate *m);
+  void handle_snap_update(const MMDSSnapUpdate::const_ref &m);
   void notify_global_snaprealm_update(int snap_op);
 
   // -- stray --
@@ -1075,12 +1069,12 @@ protected:
 
   // == messages ==
  public:
-  void dispatch(Message *m);
+  void dispatch(const Message::const_ref &m);
 
  protected:
   // -- replicas --
-  void handle_discover(MDiscover *dis);
-  void handle_discover_reply(MDiscoverReply *m);
+  void handle_discover(const MDiscover::const_ref &dis);
+  void handle_discover_reply(const MDiscoverReply::const_ref &m);
   friend class C_MDC_Join;
 
 public:
@@ -1094,15 +1088,15 @@ public:
   CInode *add_replica_inode(bufferlist::const_iterator& p, CDentry *dn, MDSInternalContextBase::vec& finished);
 
   void replicate_stray(CDentry *straydn, mds_rank_t who, bufferlist& bl);
-  CDentry *add_replica_stray(bufferlist &bl, mds_rank_t from);
+  CDentry *add_replica_stray(const bufferlist &bl, mds_rank_t from);
 
   // -- namespace --
 public:
   void send_dentry_link(CDentry *dn, MDRequestRef& mdr);
   void send_dentry_unlink(CDentry *dn, CDentry *straydn, MDRequestRef& mdr);
 protected:
-  void handle_dentry_link(MDentryLink *m);
-  void handle_dentry_unlink(MDentryUnlink *m);
+  void handle_dentry_link(const MDentryLink::const_ref &m);
+  void handle_dentry_unlink(const MDentryUnlink::const_ref &m);
 
 
   // -- fragmenting --
@@ -1142,7 +1136,7 @@ private:
                            MDSInternalContextBase::vec& waiters,
                            bool replay);
   CDir *force_dir_fragment(CInode *diri, frag_t fg, bool replay=true);
-  void get_force_dirfrag_bound_set(vector<dirfrag_t>& dfs, set<CDir*>& bounds);
+  void get_force_dirfrag_bound_set(const vector<dirfrag_t>& dfs, set<CDir*>& bounds);
 
   bool can_fragment(CInode *diri, list<CDir*>& dirs);
   void fragment_freeze_dirs(list<CDir*>& dirs);
@@ -1163,7 +1157,7 @@ private:
   friend class C_MDC_FragmentCommit;
   friend class C_IO_MDC_FragmentFinish;
 
-  void handle_fragment_notify(MMDSFragmentNotify *m);
+  void handle_fragment_notify(const MMDSFragmentNotify::const_ref &m);
 
   void add_uncommitted_fragment(dirfrag_t basedirfrag, int bits, list<frag_t>& old_frag,
                                LogSegment *ls, bufferlist *rollback=NULL);
@@ -1188,10 +1182,12 @@ public:
   //void handle_inode_update(MInodeUpdate *m);
 
   int send_dir_updates(CDir *in, bool bcast=false);
-  void handle_dir_update(MDirUpdate *m);
+  void handle_dir_update(const MDirUpdate::const_ref &m);
 
   // -- cache expiration --
-  void handle_cache_expire(MCacheExpire *m);
+  void handle_cache_expire(const MCacheExpire::const_ref &m);
+  // delayed cache expire
+  map<CDir*, expiremap> delayed_expire; // subtree root -> expire msg
   void process_delayed_expire(CDir *dir);
   void discard_delayed_expire(CDir *dir);
 
index c2bab86b5b06902e3678409a6257c451ee0695b1..017a5ae11983cc5180b033817e0f6c3368816a72 100644 (file)
@@ -376,7 +376,7 @@ protected:
   virtual SimpleLock* get_lock(int type) { ceph_abort(); return 0; }
   virtual void set_object_info(MDSCacheObjectInfo &info) { ceph_abort(); }
   virtual void encode_lock_state(int type, bufferlist& bl) { ceph_abort(); }
-  virtual void decode_lock_state(int type, bufferlist& bl) { ceph_abort(); }
+  virtual void decode_lock_state(int type, const bufferlist& bl) { ceph_abort(); }
   virtual void finish_lock_waiters(int type, uint64_t mask, int r=0) { ceph_abort(); }
   virtual void add_lock_waiter(int type, uint64_t mask, MDSInternalContextBase *c) { ceph_abort(); }
   virtual bool is_lock_waiting(int type, uint64_t mask) { ceph_abort(); return false; }
index e9c0d52ab0109fb63de27313e77c318899f5edb8..16457f61726a83b2f014de5064746839a4831969 100644 (file)
@@ -229,4 +229,6 @@ protected:
 
 typedef C_GatherBuilderBase<MDSInternalContextBase, MDSGather> MDSGatherBuilder;
 
+using MDSContextFactory = ContextFactory<MDSInternalContextBase>;
+
 #endif  // MDS_CONTEXT_H
index 98c524556d763535fdc0f895189fffb5cb20e8c6..4204f53c42072f4e519d2b1e8e736db79b9338f0 100644 (file)
 #include "events/ESession.h"
 #include "events/ESubtreeMap.h"
 
-#include "messages/MMDSMap.h"
-
-#include "messages/MGenericMessage.h"
-
-#include "messages/MMonCommand.h"
-#include "messages/MCommand.h"
-#include "messages/MCommandReply.h"
-
 #include "auth/AuthAuthorizeHandler.h"
 #include "auth/RotatingKeyRing.h"
 #include "auth/KeyRing.h"
@@ -99,7 +91,7 @@ MDSDaemon::MDSDaemon(std::string_view n, Messenger *m, MonClient *mc) :
 
   monc->set_messenger(messenger);
 
-  mdsmap = new MDSMap;
+  mdsmap.reset(new MDSMap);
 }
 
 MDSDaemon::~MDSDaemon() {
@@ -107,8 +99,6 @@ MDSDaemon::~MDSDaemon() {
 
   delete mds_rank;
   mds_rank = NULL;
-  delete mdsmap;
-  mdsmap = NULL;
 
   delete authorize_handler_service_registry;
   delete authorize_handler_cluster_registry;
@@ -530,7 +520,7 @@ int MDSDaemon::init()
 
   timer.init();
 
-  beacon.init(mdsmap);
+  beacon.init(*mdsmap);
   messenger->set_myname(entity_name_t::MDS(MDS_RANK_NONE));
 
   // schedule tick
@@ -565,7 +555,7 @@ void MDSDaemon::tick()
   }
 }
 
-void MDSDaemon::send_command_reply(MCommand *m, MDSRank *mds_rank,
+void MDSDaemon::send_command_reply(const MCommand::const_ref &m, MDSRank *mds_rank,
                                   int r, bufferlist outbl,
                                   std::string_view outs)
 {
@@ -588,14 +578,13 @@ void MDSDaemon::send_command_reply(MCommand *m, MDSRank *mds_rank,
   }
   priv.reset();
 
-  MCommandReply *reply = new MCommandReply(r, outs);
+  MCommandReply::ref reply(new MCommandReply(r, outs), false);
   reply->set_tid(m->get_tid());
   reply->set_data(outbl);
-  m->get_connection()->send_message(reply);
+  m->get_connection()->send_message2(reply);
 }
 
-/* This function DOES put the passed message before returning*/
-void MDSDaemon::handle_command(MCommand *m)
+void MDSDaemon::handle_command(const MCommand::const_ref &m)
 {
   auto priv = m->get_connection()->get_priv();
   auto session = static_cast<Session *>(priv.get());
@@ -635,8 +624,6 @@ void MDSDaemon::handle_command(MCommand *m)
   if (run_after) {
     run_after->complete(0);
   }
-
-  m->put();
 }
 
 
@@ -702,7 +689,7 @@ COMMAND("heap " \
 
 int MDSDaemon::_handle_command(
     const cmdmap_t &cmdmap,
-    MCommand *m,
+    const MCommand::const_ref &m,
     bufferlist *outbl,
     std::string *outs,
     Context **run_later,
@@ -878,7 +865,7 @@ out:
 
 /* This function deletes the passed message before returning. */
 
-void MDSDaemon::handle_mds_map(MMDSMap *m)
+void MDSDaemon::handle_mds_map(const MMDSMap::const_ref &m)
 {
   version_t epoch = m->get_epoch();
 
@@ -886,7 +873,6 @@ void MDSDaemon::handle_mds_map(MMDSMap *m)
   if (epoch <= mdsmap->get_epoch()) {
     dout(5) << "handle_mds_map old map epoch " << epoch << " <= "
             << mdsmap->get_epoch() << ", discarding" << dendl;
-    m->put();
     return;
   }
 
@@ -895,10 +881,11 @@ void MDSDaemon::handle_mds_map(MMDSMap *m)
   entity_addrvec_t addrs;
 
   // keep old map, for a moment
-  MDSMap *oldmap = mdsmap;
+  std::unique_ptr<MDSMap> oldmap;
+  oldmap.swap(mdsmap);
 
   // decode and process
-  mdsmap = new MDSMap;
+  mdsmap.reset(new MDSMap);
   mdsmap->decode(m->get_encoded());
   const MDSMap::DaemonState new_state = mdsmap->get_state_gid(mds_gid_t(monc->get_global_id()));
   const int incarnation = mdsmap->get_inc_gid(mds_gid_t(monc->get_global_id()));
@@ -922,12 +909,10 @@ void MDSDaemon::handle_mds_map(MMDSMap *m)
   }
 
   // mark down any failed peers
-  for (map<mds_gid_t,MDSMap::mds_info_t>::const_iterator p = oldmap->get_mds_info().begin();
-       p != oldmap->get_mds_info().end();
-       ++p) {
-    if (mdsmap->get_mds_info().count(p->first) == 0) {
-      dout(10) << " peer mds gid " << p->first << " removed from map" << dendl;
-      messenger->mark_down_addrs(p->second.addrs);
+  for (const auto &p : oldmap->get_mds_info()) {
+    if (mdsmap->get_mds_info().count(p.first) == 0) {
+      dout(10) << " peer mds gid " << p.first << " removed from map" << dendl;
+      messenger->mark_down_addrs(p.second.addrs);
     }
   }
 
@@ -958,7 +943,6 @@ void MDSDaemon::handle_mds_map(MMDSMap *m)
             // has taken our ID, we don't want to keep restarting and
             // fighting them for the ID.
             suicide();
-            m->put();
             return;
           }
         }
@@ -971,7 +955,7 @@ void MDSDaemon::handle_mds_map(MMDSMap *m)
     // MDSRank not active: process the map here to see if we have
     // been assigned a rank.
     dout(10) <<  __func__ << ": handling map in rankless mode" << dendl;
-    _handle_mds_map(oldmap);
+    _handle_mds_map(*mdsmap);
   } else {
 
     // Did we already hold a different rank?  MDSMonitor shouldn't try
@@ -996,18 +980,16 @@ void MDSDaemon::handle_mds_map(MMDSMap *m)
     // MDSRank is active: let him process the map, we have no say.
     dout(10) <<  __func__ << ": handling map as rank "
              << mds_rank->get_nodeid() << dendl;
-    mds_rank->handle_mds_map(m, oldmap);
+    mds_rank->handle_mds_map(m, *oldmap);
   }
 
 out:
-  beacon.notify_mdsmap(mdsmap);
-  m->put();
-  delete oldmap;
+  beacon.notify_mdsmap(*mdsmap);
 }
 
-void MDSDaemon::_handle_mds_map(MDSMap *oldmap)
+void MDSDaemon::_handle_mds_map(const MDSMap &mdsmap)
 {
-  MDSMap::DaemonState new_state = mdsmap->get_state_gid(mds_gid_t(monc->get_global_id()));
+  MDSMap::DaemonState new_state = mdsmap.get_state_gid(mds_gid_t(monc->get_global_id()));
 
   // Normal rankless case, we're marked as standby
   if (new_state == MDSMap::STATE_STANDBY) {
@@ -1068,7 +1050,7 @@ void MDSDaemon::suicide()
   clean_up_admin_socket();
 
   // Inform MDS we are going away, then shut down beacon
-  beacon.set_want_state(mdsmap, MDSMap::STATE_DNE);
+  beacon.set_want_state(*mdsmap, MDSMap::STATE_DNE);
   if (!mdsmap->is_dne_gid(mds_gid_t(monc->get_global_id()))) {
     // Notify the MDSMonitor that we're dying, so that it doesn't have to
     // wait for us to go laggy.  Only do this if we're actually in the
@@ -1143,7 +1125,7 @@ void MDSDaemon::respawn()
 
 
 
-bool MDSDaemon::ms_dispatch(Message *m)
+bool MDSDaemon::ms_dispatch2(const Message::ref &m)
 {
   Mutex::Locker l(mds_lock);
   if (stopping) {
@@ -1153,7 +1135,6 @@ bool MDSDaemon::ms_dispatch(Message *m)
   // Drop out early if shutting down
   if (beacon.get_want_state() == CEPH_MDS_STATE_DNE) {
     dout(10) << " stopping, discarding " << *m << dendl;
-    m->put();
     return true;
   }
 
@@ -1193,23 +1174,22 @@ bool MDSDaemon::ms_get_authorizer(int dest_type, AuthAuthorizer **authorizer, bo
 /*
  * high priority messages we always process
  */
-bool MDSDaemon::handle_core_message(Message *m)
+bool MDSDaemon::handle_core_message(const Message::const_ref &m)
 {
   switch (m->get_type()) {
   case CEPH_MSG_MON_MAP:
     ALLOW_MESSAGES_FROM(CEPH_ENTITY_TYPE_MON);
-    m->put();
     break;
 
     // MDS
   case CEPH_MSG_MDS_MAP:
     ALLOW_MESSAGES_FROM(CEPH_ENTITY_TYPE_MON | CEPH_ENTITY_TYPE_MDS);
-    handle_mds_map(static_cast<MMDSMap*>(m));
+    handle_mds_map(boost::static_pointer_cast<MMDSMap::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
     // OSD
   case MSG_COMMAND:
-    handle_command(static_cast<MCommand*>(m));
+    handle_command(boost::static_pointer_cast<MCommand::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case CEPH_MSG_OSD_MAP:
     ALLOW_MESSAGES_FROM(CEPH_ENTITY_TYPE_MON | CEPH_ENTITY_TYPE_OSD);
@@ -1217,13 +1197,11 @@ bool MDSDaemon::handle_core_message(Message *m)
     if (mds_rank) {
       mds_rank->handle_osd_map();
     }
-    m->put();
     break;
 
   case MSG_MON_COMMAND:
     ALLOW_MESSAGES_FROM(CEPH_ENTITY_TYPE_MON);
     clog->warn() << "dropping `mds tell` command from legacy monitor";
-    m->put();
     break;
 
   default:
@@ -1432,7 +1410,7 @@ void MDSDaemon::ms_handle_accept(Connection *con)
 
       // send out any queued messages
       while (!s->preopen_out_queue.empty()) {
-       con->send_message(s->preopen_out_queue.front());
+       con->send_message2(s->preopen_out_queue.front());
        s->preopen_out_queue.pop_front();
       }
     }
index 93d59008a18c8a32c54d92ce1851ea98675b02a7..745b4042816e1d5538a4e418cd175b4da50b66ea 100644 (file)
 
 #include <string_view>
 
+#include "messages/MCommand.h"
+#include "messages/MCommandReply.h"
+#include "messages/MGenericMessage.h"
+#include "messages/MMDSMap.h"
+#include "messages/MMonCommand.h"
+
 #include "common/LogClient.h"
 #include "common/Mutex.h"
 #include "common/Timer.h"
@@ -32,7 +38,6 @@
 #define CEPH_MDS_PROTOCOL    33 /* cluster internal */
 
 class AuthAuthorizeHandlerRegistry;
-class Message;
 class Messenger;
 class MonClient;
 
@@ -67,7 +72,7 @@ class MDSDaemon : public Dispatcher, public md_config_obs_t {
   Messenger    *messenger;
   MonClient    *monc;
   MgrClient     mgrc;
-  MDSMap       *mdsmap;
+  std::unique_ptr<MDSMap> mdsmap;
   LogClient    log_client;
   LogChannelRef clog;
 
@@ -104,7 +109,7 @@ class MDSDaemon : public Dispatcher, public md_config_obs_t {
   void wait_for_omap_osds();
 
  private:
-  bool ms_dispatch(Message *m) override;
+  bool ms_dispatch2(const Message::ref &m) override;
   bool ms_get_authorizer(int dest_type, AuthAuthorizer **authorizer, bool force_new) override;
   bool ms_verify_authorizer(Connection *con, int peer_type,
                               int protocol, bufferlist& authorizer_data, bufferlist& authorizer_reply,
@@ -144,26 +149,23 @@ class MDSDaemon : public Dispatcher, public md_config_obs_t {
 
   void tick();
   
-  // messages
-  bool _dispatch(Message *m, bool new_msg);
-
 protected:
-  bool handle_core_message(Message *m);
+  bool handle_core_message(const Message::const_ref &m);
   
   // special message types
   friend class C_MDS_Send_Command_Reply;
-  static void send_command_reply(MCommand *m, MDSRank* mds_rank, int r,
+  static void send_command_reply(const MCommand::const_ref &m, MDSRank* mds_rank, int r,
                                 bufferlist outbl, std::string_view outs);
   int _handle_command(
       const cmdmap_t &cmdmap,
-      MCommand *m,
+      const MCommand::const_ref &m,
       bufferlist *outbl,
       std::string *outs,
       Context **run_later,
       bool *need_reply);
-  void handle_command(class MCommand *m);
-  void handle_mds_map(class MMDSMap *m);
-  void _handle_mds_map(MDSMap *oldmap);
+  void handle_command(const MCommand::const_ref &m);
+  void handle_mds_map(const MMDSMap::const_ref &m);
+  void _handle_mds_map(const MDSMap &oldmap);
 
 private:
     mono_time starttime = mono_clock::zero();
index 766593ac4aa3920222ca54f197e4b38b96797de7..6ff298ac14ef5feb6c620ebb4bc8527c2b9e4ea4 100644 (file)
@@ -26,7 +26,6 @@
 
 #include "include/types.h"
 #include "common/Clock.h"
-#include "msg/Message.h"
 #include "include/health.h"
 
 #include "common/config.h"
@@ -605,9 +604,8 @@ public:
    * Get the MDS daemon entity_inst_t for a rank
    * known to be up.
    */
-  entity_addrvec_t get_addrs(mds_rank_t m) {
-    assert(up.count(m));
-    return mds_info[up[m]].get_addrs();
+  entity_addrvec_t get_addrs(mds_rank_t m) const {
+    return mds_info.at(up.at(m)).get_addrs();
   }
 
   mds_rank_t get_rank_gid(mds_gid_t gid) const {
@@ -626,7 +624,7 @@ public:
   }
   void encode(bufferlist& bl, uint64_t features) const;
   void decode(bufferlist::const_iterator& p);
-  void decode(bufferlist& bl) {
+  void decode(const bufferlist& bl) {
     auto p = bl.cbegin();
     decode(p);
   }
index be4bbb15f99e18d6225925f5d559a3b70ae90b48..e24fa335bd8afaded4206851ec6a9d66b60ef87c 100644 (file)
@@ -19,9 +19,7 @@
 
 #include "messages/MClientRequestForward.h"
 #include "messages/MMDSLoadTargets.h"
-#include "messages/MMDSMap.h"
 #include "messages/MMDSTableRequest.h"
-#include "messages/MCommand.h"
 #include "messages/MCommandReply.h"
 
 #include "MDSDaemon.h"
@@ -51,7 +49,7 @@ MDSRank::MDSRank(
     LogChannelRef &clog_,
     SafeTimer &timer_,
     Beacon &beacon_,
-    MDSMap *& mdsmap_,
+    std::unique_ptr<MDSMap>& mdsmap_,
     Messenger *msgr,
     MonClient *monc_,
     Context *respawn_hook_,
@@ -135,7 +133,6 @@ MDSRank::~MDSRank()
   if (inotable) { delete inotable; inotable = NULL; }
   if (snapserver) { delete snapserver; snapserver = NULL; }
   if (snapclient) { delete snapclient; snapclient = NULL; }
-  if (mdsmap) { delete mdsmap; mdsmap = 0; }
 
   if (server) { delete server; server = 0; }
   if (locker) { delete locker; locker = 0; }
@@ -501,7 +498,7 @@ void MDSRank::damaged()
   assert(whoami != MDS_RANK_NONE);
   assert(mds_lock.is_locked_by_me());
 
-  beacon.set_want_state(mdsmap, MDSMap::STATE_DAMAGED);
+  beacon.set_want_state(*mdsmap, MDSMap::STATE_DAMAGED);
   monc->flush_log();  // Flush any clog error from before we were called
   beacon.notify_health(this);  // Include latest status in our swan song
   beacon.send_and_wait(g_conf()->mds_mon_shutdown_timeout);
@@ -577,7 +574,7 @@ void MDSRank::ProgressThread::shutdown()
   }
 }
 
-bool MDSRankDispatcher::ms_dispatch(Message *m)
+bool MDSRankDispatcher::ms_dispatch(const Message::const_ref &m)
 {
   bool ret;
   inc_dispatch_depth();
@@ -586,12 +583,9 @@ bool MDSRankDispatcher::ms_dispatch(Message *m)
   return ret;
 }
 
-/* If this function returns true, it recognizes the message and has taken the
- * reference. If it returns false, it has done neither. */
-bool MDSRank::_dispatch(Message *m, bool new_msg)
+bool MDSRank::_dispatch(const Message::const_ref &m, bool new_msg)
 {
   if (is_stale_message(m)) {
-    m->put();
     return true;
   }
 
@@ -738,7 +732,7 @@ void MDSRank::update_mlogger()
 /*
  * lower priority messages we defer if we seem laggy
  */
-bool MDSRank::handle_deferrable_message(Message *m)
+bool MDSRank::handle_deferrable_message(const Message::const_ref &m)
 {
   int port = m->get_type() & 0xff00;
 
@@ -776,14 +770,14 @@ bool MDSRank::handle_deferrable_message(Message *m)
     case MSG_MDS_TABLE_REQUEST:
       ALLOW_MESSAGES_FROM(CEPH_ENTITY_TYPE_MDS);
       {
-       MMDSTableRequest *req = static_cast<MMDSTableRequest*>(m);
-       if (req->op < 0) {
-         MDSTableClient *client = get_table_client(req->table);
-             client->handle_request(req);
-       } else {
-         MDSTableServer *server = get_table_server(req->table);
-         server->handle_request(req);
-       }
+        const MMDSTableRequest::const_ref &req = boost::static_pointer_cast<MMDSTableRequest::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m);
+        if (req->op < 0) {
+          MDSTableClient *client = get_table_client(req->table);
+          client->handle_request(req);
+        } else {
+           MDSTableServer *server = get_table_server(req->table);
+           server->handle_request(req);
+        }
       }
       break;
 
@@ -836,16 +830,13 @@ void MDSRank::_advance_queues()
     if (beacon.is_laggy())
       break;
 
-    Message *old = waiting_for_nolaggy.front();
+    Message::const_ref old = waiting_for_nolaggy.front();
     waiting_for_nolaggy.pop_front();
 
-    if (is_stale_message(old)) {
-      old->put();
-    } else {
+    if (!is_stale_message(old)) {
       dout(7) << " processing laggy deferred " << *old << dendl;
       if (!handle_deferrable_message(old)) {
         dout(0) << "unrecognized message " << *old << dendl;
-        old->put();
       }
     }
 
@@ -873,7 +864,7 @@ void MDSRank::heartbeat_reset()
   g_ceph_context->get_heartbeat_map()->reset_timeout(hb, g_conf()->mds_beacon_grace, 0);
 }
 
-bool MDSRank::is_stale_message(Message *m) const
+bool MDSRank::is_stale_message(const Message::const_ref &m) const
 {
   // from bad mds?
   if (m->get_source().is_mds()) {
@@ -899,7 +890,7 @@ bool MDSRank::is_stale_message(Message *m) const
   return false;
 }
 
-Session *MDSRank::get_session(Message *m)
+Session *MDSRank::get_session(const Message::const_ref &m)
 {
   // do not carry ref
   auto session = static_cast<Session *>(m->get_connection()->get_priv().get());
@@ -919,8 +910,7 @@ Session *MDSRank::get_session(Message *m)
         imported_session->set_connection(session->get_connection().get());
         // send out any queued messages
         while (!session->preopen_out_queue.empty()) {
-          imported_session->get_connection()->send_message(
-           session->preopen_out_queue.front());
+          imported_session->get_connection()->send_message2(std::move(session->preopen_out_queue.front()));
           session->preopen_out_queue.pop_front();
         }
         imported_session->auth_caps = session->auth_caps;
@@ -935,83 +925,61 @@ Session *MDSRank::get_session(Message *m)
   return session;
 }
 
-void MDSRank::send_message(Message *m, Connection *c)
+void MDSRank::send_message(const Message::ref& m, const ConnectionRef& c)
 {
   assert(c);
-  c->send_message(m);
+  c->send_message2(m);
 }
 
 
-void MDSRank::send_message_mds(Message *m, mds_rank_t mds)
+void MDSRank::send_message_mds(const Message::ref& m, mds_rank_t mds)
 {
   if (!mdsmap->is_up(mds)) {
     dout(10) << "send_message_mds mds." << mds << " not up, dropping " << *m << dendl;
-    m->put();
     return;
   }
 
   // send mdsmap first?
   if (mds != whoami && peer_mdsmap_epoch[mds] < mdsmap->get_epoch()) {
-    messenger->send_to_mds(new MMDSMap(monc->get_fsid(), mdsmap),
-                          mdsmap->get_addrs(mds));
+    Message::ref _m = MMDSMap::ref(new MMDSMap(monc->get_fsid(), *mdsmap), false);
+    messenger->send_to_mds(_m.detach(), mdsmap->get_addrs(mds));
     peer_mdsmap_epoch[mds] = mdsmap->get_epoch();
   }
 
   // send message
-  messenger->send_to_mds(m, mdsmap->get_addrs(mds));
+  messenger->send_to_mds(Message::ref(m).detach(), mdsmap->get_addrs(mds));
 }
 
-void MDSRank::forward_message_mds(Message *m, mds_rank_t mds)
+void MDSRank::forward_message_mds(const MClientRequest::const_ref& m, mds_rank_t mds)
 {
   assert(mds != whoami);
 
-  // client request?
-  if (m->get_type() == CEPH_MSG_CLIENT_REQUEST &&
-      (static_cast<MClientRequest*>(m))->get_source().is_client()) {
-    MClientRequest *creq = static_cast<MClientRequest*>(m);
-    creq->inc_num_fwd();    // inc forward counter
-
-    /*
-     * don't actually forward if non-idempotent!
-     * client has to do it.  although the MDS will ignore duplicate requests,
-     * the affected metadata may migrate, in which case the new authority
-     * won't have the metareq_id in the completed request map.
-     */
-    // NEW: always make the client resend!
-    bool client_must_resend = true;  //!creq->can_forward();
-
-    // tell the client where it should go
-    messenger->send_message(new MClientRequestForward(creq->get_tid(), mds, creq->get_num_fwd(),
-                                                     client_must_resend),
-                           creq->get_source_inst());
-
-    if (client_must_resend) {
-      m->put();
-      return;
-    }
-  }
-
-  // these are the only types of messages we should be 'forwarding'; they
-  // explicitly encode their source mds, which gets clobbered when we resend
-  // them here.
-  assert(m->get_type() == MSG_MDS_DIRUPDATE ||
-        m->get_type() == MSG_MDS_EXPORTDIRDISCOVER);
-
-  // send mdsmap first?
-  if (peer_mdsmap_epoch[mds] < mdsmap->get_epoch()) {
-    messenger->send_to_mds(new MMDSMap(monc->get_fsid(), mdsmap),
-                          mdsmap->get_addrs(mds));
-    peer_mdsmap_epoch[mds] = mdsmap->get_epoch();
-  }
+  /*
+   * don't actually forward if non-idempotent!
+   * client has to do it.  although the MDS will ignore duplicate requests,
+   * the affected metadata may migrate, in which case the new authority
+   * won't have the metareq_id in the completed request map.
+   */
+  // NEW: always make the client resend!
+  bool client_must_resend = true;  //!creq->can_forward();
 
-  messenger->send_to_mds(m, mdsmap->get_addrs(mds));
+  // tell the client where it should go
+  auto f = MClientRequestForward::ref(new MClientRequestForward(m->get_tid(), mds, m->get_num_fwd()+1, client_must_resend), false);
+  messenger->send_message(f.detach(), m->get_source_inst());
 }
 
+void MDSRank::forward_message_mds(const MInterMDS::const_ref& m, mds_rank_t mds)
+{
+  assert(mds != whoami);
 
+  if (m->is_forwardable()) {
+    send_message_mds(m->forwardable(), mds);
+  }
+}
 
-void MDSRank::send_message_client_counted(Message *m, client_t client)
+void MDSRank::send_message_client_counted(const Message::ref& m, client_t client)
 {
-  Session *session =  sessionmap.get_session(entity_name_t::CLIENT(client.v));
+  Session *session = sessionmap.get_session(entity_name_t::CLIENT(client.v));
   if (session) {
     send_message_client_counted(m, session);
   } else {
@@ -1019,7 +987,7 @@ void MDSRank::send_message_client_counted(Message *m, client_t client)
   }
 }
 
-void MDSRank::send_message_client_counted(Message *m, Connection *connection)
+void MDSRank::send_message_client_counted(const Message::ref& m, const ConnectionRef& connection)
 {
   // do not carry ref
   auto session = static_cast<Session *>(connection->get_priv().get());
@@ -1031,23 +999,23 @@ void MDSRank::send_message_client_counted(Message *m, Connection *connection)
   }
 }
 
-void MDSRank::send_message_client_counted(Message *m, Session *session)
+void MDSRank::send_message_client_counted(const Message::ref& m, Session* session)
 {
   version_t seq = session->inc_push_seq();
   dout(10) << "send_message_client_counted " << session->info.inst.name << " seq "
           << seq << " " << *m << dendl;
   if (session->get_connection()) {
-    session->get_connection()->send_message(m);
+    session->get_connection()->send_message2(m);
   } else {
     session->preopen_out_queue.push_back(m);
   }
 }
 
-void MDSRank::send_message_client(Message *m, Session *session)
+void MDSRank::send_message_client(const Message::ref& m, Session* session)
 {
   dout(10) << "send_message_client " << session->info.inst << " " << *m << dendl;
   if (session->get_connection()) {
-    session->get_connection()->send_message(m);
+    session->get_connection()->send_message2(m);
   } else {
     session->preopen_out_queue.push_back(m);
   }
@@ -1066,7 +1034,7 @@ void MDSRank::set_osd_epoch_barrier(epoch_t e)
   osd_epoch_barrier = e;
 }
 
-void MDSRank::retry_dispatch(Message *m)
+void MDSRank::retry_dispatch(const Message::const_ref &m)
 {
   inc_dispatch_depth();
   _dispatch(m, false);
@@ -1091,7 +1059,7 @@ bool MDSRank::is_daemon_stopping() const
 void MDSRank::request_state(MDSMap::DaemonState s)
 {
   dout(3) << "request_state " << ceph_mds_state_name(s) << dendl;
-  beacon.set_want_state(mdsmap, s);
+  beacon.set_want_state(*mdsmap, s);
   beacon.send();
 }
 
@@ -1699,8 +1667,8 @@ void MDSRank::stopping_done()
 }
 
 void MDSRankDispatcher::handle_mds_map(
-    MMDSMap *m,
-    MDSMap *oldmap)
+    const MMDSMap::const_ref &m,
+    const MDSMap &oldmap)
 {
   // I am only to be passed MDSMaps in which I hold a rank
   assert(whoami != MDS_RANK_NONE);
@@ -1747,7 +1715,7 @@ void MDSRankDispatcher::handle_mds_map(
   if (objecter->get_client_incarnation() != incarnation)
     objecter->set_client_incarnation(incarnation);
 
-  if (oldmap->get_min_compat_client() != mdsmap->get_min_compat_client())
+  if (oldmap.get_min_compat_client() != mdsmap->get_min_compat_client())
     server->update_required_client_features();
 
   // for debug
@@ -1764,11 +1732,11 @@ void MDSRankDispatcher::handle_mds_map(
     // did someone fail?
     //   new down?
     set<mds_rank_t> olddown, down;
-    oldmap->get_down_mds_set(&olddown);
+    oldmap.get_down_mds_set(&olddown);
     mdsmap->get_down_mds_set(&down);
     for (const auto& r : down) {
-      if (oldmap->have_inst(r) && olddown.count(r) == 0) {
-       messenger->mark_down_addrs(oldmap->get_addrs(r));
+      if (oldmap.have_inst(r) && olddown.count(r) == 0) {
+       messenger->mark_down_addrs(oldmap.get_addrs(r));
        handle_mds_failure(r);
       }
     }
@@ -1779,8 +1747,8 @@ void MDSRankDispatcher::handle_mds_map(
     mdsmap->get_up_mds_set(up);
     for (const auto& r : up) {
       auto& info = mdsmap->get_info(r);
-      if (oldmap->have_inst(r)) {
-       auto& oldinfo = oldmap->get_info(r);
+      if (oldmap.have_inst(r)) {
+       auto& oldinfo = oldmap.get_info(r);
        if (info.inc != oldinfo.inc) {
          messenger->mark_down_addrs(oldinfo.get_addrs());
          if (info.state == MDSMap::STATE_REPLAY ||
@@ -1817,7 +1785,7 @@ void MDSRankDispatcher::handle_mds_map(
     dout(1) << "handle_mds_map state change "
            << ceph_mds_state_name(oldstate) << " --> "
            << ceph_mds_state_name(state) << dendl;
-    beacon.set_want_state(mdsmap, state);
+    beacon.set_want_state(*mdsmap, state);
 
     if (oldstate == MDSMap::STATE_STANDBY_REPLAY) {
         dout(10) << "Monitor activated us! Deactivating replay loop" << dendl;
@@ -1864,7 +1832,7 @@ void MDSRankDispatcher::handle_mds_map(
        snapserver->finish_recovery(s);
       } else {
        set<mds_rank_t> old_set, new_set;
-       oldmap->get_mds_set_lower_bound(old_set, MDSMap::STATE_RESOLVE);
+       oldmap.get_mds_set_lower_bound(old_set, MDSMap::STATE_RESOLVE);
        mdsmap->get_mds_set_lower_bound(new_set, MDSMap::STATE_RESOLVE);
        for (const auto& r : new_set) {
          if (r == whoami)
@@ -1876,7 +1844,7 @@ void MDSRankDispatcher::handle_mds_map(
       }
     }
 
-    if ((!oldmap->is_resolving() || !restart.empty()) && mdsmap->is_resolving()) {
+    if ((!oldmap.is_resolving() || !restart.empty()) && mdsmap->is_resolving()) {
       set<mds_rank_t> resolve;
       mdsmap->get_mds_set(resolve, MDSMap::STATE_RESOLVE);
       dout(10) << " resolve set is " << resolve << dendl;
@@ -1889,19 +1857,19 @@ void MDSRankDispatcher::handle_mds_map(
   // is everybody finally rejoining?
   if (state >= MDSMap::STATE_REJOIN) {
     // did we start?
-    if (!oldmap->is_rejoining() && mdsmap->is_rejoining())
+    if (!oldmap.is_rejoining() && mdsmap->is_rejoining())
       rejoin_joint_start();
 
     // did we finish?
     if (g_conf()->mds_dump_cache_after_rejoin &&
-       oldmap->is_rejoining() && !mdsmap->is_rejoining())
+       oldmap.is_rejoining() && !mdsmap->is_rejoining())
       mdcache->dump_cache();      // for DEBUG only
 
     if (oldstate >= MDSMap::STATE_REJOIN ||
        oldstate == MDSMap::STATE_STARTING) {
       // ACTIVE|CLIENTREPLAY|REJOIN => we can discover from them.
       set<mds_rank_t> olddis, dis;
-      oldmap->get_mds_set_lower_bound(olddis, MDSMap::STATE_REJOIN);
+      oldmap.get_mds_set_lower_bound(olddis, MDSMap::STATE_REJOIN);
       mdsmap->get_mds_set_lower_bound(dis, MDSMap::STATE_REJOIN);
       for (const auto& r : dis) {
        if (r == whoami)
@@ -1914,7 +1882,7 @@ void MDSRankDispatcher::handle_mds_map(
     }
   }
 
-  if (oldmap->is_degraded() && !cluster_degraded && state >= MDSMap::STATE_ACTIVE) {
+  if (oldmap.is_degraded() && !cluster_degraded && state >= MDSMap::STATE_ACTIVE) {
     dout(1) << "cluster recovered." << dendl;
     auto it = waiting_for_active_peer.find(MDS_RANK_NONE);
     if (it != waiting_for_active_peer.end()) {
@@ -1927,7 +1895,7 @@ void MDSRankDispatcher::handle_mds_map(
   if (state >= MDSMap::STATE_CLIENTREPLAY &&
       oldstate >= MDSMap::STATE_CLIENTREPLAY) {
     set<mds_rank_t> oldactive, active;
-    oldmap->get_mds_set_lower_bound(oldactive, MDSMap::STATE_CLIENTREPLAY);
+    oldmap.get_mds_set_lower_bound(oldactive, MDSMap::STATE_CLIENTREPLAY);
     mdsmap->get_mds_set_lower_bound(active, MDSMap::STATE_CLIENTREPLAY);
     for (const auto& r : active) {
       if (r == whoami)
@@ -1940,7 +1908,7 @@ void MDSRankDispatcher::handle_mds_map(
   if (is_clientreplay() || is_active() || is_stopping()) {
     // did anyone stop?
     set<mds_rank_t> oldstopped, stopped;
-    oldmap->get_stopped_mds_set(oldstopped);
+    oldmap.get_stopped_mds_set(oldstopped);
     mdsmap->get_stopped_mds_set(stopped);
     for (const auto& r : stopped)
       if (oldstopped.count(r) == 0) {     // newly so?
@@ -1987,7 +1955,7 @@ void MDSRankDispatcher::handle_mds_map(
       mdlog->set_write_iohint(CEPH_OSD_OP_FLAG_FADVISE_DONTNEED);
   }
 
-  if (oldmap->get_max_mds() != mdsmap->get_max_mds()) {
+  if (oldmap.get_max_mds() != mdsmap->get_max_mds()) {
     purge_queue.update_op_limit(*mdsmap);
   }
 }
@@ -2187,14 +2155,13 @@ bool MDSRankDispatcher::handle_asok_command(std::string_view command,
 class C_MDS_Send_Command_Reply : public MDSInternalContext
 {
 protected:
-  MCommand *m;
+  MCommand::const_ref m;
 public:
-  C_MDS_Send_Command_Reply(MDSRank *_mds, MCommand *_m) :
-    MDSInternalContext(_mds), m(_m) { m->get(); }
+  C_MDS_Send_Command_Reply(MDSRank *_mds, const MCommand::const_ref &_m) :
+    MDSInternalContext(_mds), m(_m) {}
   void send (int r, std::string_view out_str) {
     bufferlist bl;
     MDSDaemon::send_command_reply(m, mds, r, bl, out_str);
-    m->put();
   }
   void finish (int r) override {
     send(r, "");
@@ -2206,7 +2173,7 @@ public:
  * MDSRank after calling it (we could have gone into shutdown): just
  * send your result back to the calling client and finish.
  */
-void MDSRankDispatcher::evict_clients(const SessionFilter &filter, MCommand *m)
+void MDSRankDispatcher::evict_clients(const SessionFilter &filter, const MCommand::const_ref &m)
 {
   C_MDS_Send_Command_Reply *reply = new C_MDS_Send_Command_Reply(this, m);
 
@@ -3025,10 +2992,10 @@ void MDSRank::bcast_mds_map()
   // share the map with mounted clients
   set<Session*> clients;
   sessionmap.get_client_session_set(clients);
-  for (set<Session*>::const_iterator p = clients.begin();
-       p != clients.end();
-       ++p)
-    (*p)->get_connection()->send_message(new MMDSMap(monc->get_fsid(), mdsmap));
+  for (const auto &session : clients) {
+    MMDSMap::ref m(new MMDSMap(monc->get_fsid(), *mdsmap), false);
+    session->get_connection()->send_message2(std::move(m));
+  }
   last_client_mdsmap_bcast = mdsmap->get_epoch();
 }
 
@@ -3038,7 +3005,7 @@ MDSRankDispatcher::MDSRankDispatcher(
     LogChannelRef &clog_,
     SafeTimer &timer_,
     Beacon &beacon_,
-    MDSMap *& mdsmap_,
+    std::unique_ptr<MDSMap> &mdsmap_,
     Messenger *msgr,
     MonClient *monc_,
     Context *respawn_hook_,
@@ -3049,7 +3016,7 @@ MDSRankDispatcher::MDSRankDispatcher(
 
 bool MDSRankDispatcher::handle_command(
   const cmdmap_t &cmdmap,
-  MCommand *m,
+  const MCommand::const_ref &m,
   int *r,
   std::stringstream *ds,
   std::stringstream *ss,
index f2ab258a4fe21c9b6a89bab3fb6e2abf610abe5b..5366f2eafafa811ecb42e40b1caabe9515a65d12 100644 (file)
 #include "common/Timer.h"
 #include "common/TrackedOp.h"
 
+#include "messages/MClientRequest.h"
+#include "messages/MInterMDS.h"
 #include "messages/MCommand.h"
+#include "messages/MMDSMap.h"
 
 #include "Beacon.h"
 #include "DamageTable.h"
@@ -116,7 +119,6 @@ class Messenger;
 class Objecter;
 class MonClient;
 class Finisher;
-class MMDSMap;
 class ScrubStack;
 
 /**
@@ -162,7 +164,7 @@ class MDSRank {
     // to share a timer.
     SafeTimer &timer;
 
-    MDSMap *&mdsmap;
+    std::unique_ptr<MDSMap> &mdsmap; /* MDSDaemon::mdsmap */
 
     Objecter     *objecter;
 
@@ -188,7 +190,7 @@ class MDSRank {
     Session *get_session(client_t client) {
       return sessionmap.get_session(entity_name_t::CLIENT(client.v));
     }
-    Session *get_session(Message *m);
+    Session *get_session(const Message::const_ref &m);
 
     PerfCounters       *logger, *mlogger;
     OpTracker    op_tracker;
@@ -248,20 +250,20 @@ class MDSRank {
       void signal() {cond.Signal();}
     } progress_thread;
 
-    list<Message*> waiting_for_nolaggy;
+    list<Message::const_ref> waiting_for_nolaggy;
     MDSInternalContextBase::que finished_queue;
     // Dispatch, retry, queues
     int dispatch_depth;
     void inc_dispatch_depth() { ++dispatch_depth; }
     void dec_dispatch_depth() { --dispatch_depth; }
-    void retry_dispatch(Message *m);
-    bool handle_deferrable_message(Message *m);
+    void retry_dispatch(const Message::const_ref &m);
+    bool handle_deferrable_message(const Message::const_ref &m);
     void _advance_queues();
-    bool _dispatch(Message *m, bool new_msg);
+    bool _dispatch(const Message::const_ref &m, bool new_msg);
 
     ceph::heartbeat_handle_d *hb;  // Heartbeat for threads using mds_lock
 
-    bool is_stale_message(Message *m) const;
+    bool is_stale_message(const Message::const_ref &m) const;
 
     map<mds_rank_t, version_t> peer_mdsmap_epoch;
 
@@ -320,7 +322,7 @@ class MDSRank {
         LogChannelRef &clog_,
         SafeTimer &timer_,
         Beacon &beacon_,
-        MDSMap *& mdsmap_,
+        std::unique_ptr<MDSMap> & mdsmap_,
         Messenger *msgr,
         MonClient *monc_,
         Context *respawn_hook_,
@@ -369,19 +371,41 @@ class MDSRank {
     utime_t get_laggy_until() const;
     double get_dispatch_queue_max_age(utime_t now) const;
 
-    void send_message_mds(Message *m, mds_rank_t mds);
-    void forward_message_mds(Message *req, mds_rank_t mds);
+    void send_message_mds(const Message::ref& m, mds_rank_t mds);
+    void send_message_mds(Message* m, mds_rank_t mds) {
+      Message::ref mr(m, false);
+      send_message_mds(mr, mds);
+    }
+    void forward_message_mds(const MClientRequest::const_ref& req, mds_rank_t mds);
+    void forward_message_mds(const MInterMDS::const_ref& req, mds_rank_t mds);
+
+    void send_message_client_counted(const Message::ref& m, client_t client);
+    void send_message_client_counted(Message* m, client_t client) {
+      Message::ref mr(m, false);
+      send_message_client_counted(mr, client);
+    }
+
+    void send_message_client_counted(const Message::ref& m, Session* session);
+    void send_message_client_counted(Message* m, Session* session) {
+      Message::ref mr(m, false);
+      send_message_client_counted(mr, session);
+    }
+
+    void send_message_client_counted(const Message::ref& m, const ConnectionRef& connection);
+    void send_message_client_counted(Message* m, const ConnectionRef& connection) {
+      Message::ref mr(m, false);
+      send_message_client_counted(mr, connection);
+    }
 
-    void send_message_client_counted(Message *m, client_t client);
-    void send_message_client_counted(Message *m, Session *session);
-    void send_message_client_counted(Message *m, Connection *connection);
-    void send_message_client_counted(Message *m, const ConnectionRef& con) {
-      send_message_client_counted(m, con.get());
+    void send_message_client(const Message::ref& m, Session* session);
+    void send_message_client(Message* m, Session* session) {
+      Message::ref mr(m, false);
+      send_message_client(mr, session);
     }
-    void send_message_client(Message *m, Session *session);
-    void send_message(Message *m, Connection *c);
-    void send_message(Message *m, const ConnectionRef& c) {
-      send_message(m, c.get());
+
+    void send_message(const Message::ref& m, const ConnectionRef& c);
+    void send_message(Messagem, const ConnectionRef& c) {
+      send_message(m, c);
     }
 
     void wait_for_active_peer(mds_rank_t who, MDSInternalContextBase *c) { 
@@ -427,7 +451,7 @@ class MDSRank {
 
     Finisher     *finisher;
 
-    MDSMap *get_mds_map() { return mdsmap; }
+    MDSMap *get_mds_map() { return mdsmap.get(); }
 
     uint64_t get_num_requests() const { return logger->get(l_mds_request); }
   
@@ -556,19 +580,29 @@ private:
  * will put the Message exactly once.*/
 class C_MDS_RetryMessage : public MDSInternalContext {
 protected:
-  Message *m;
+  Message::const_ref m;
 public:
-  C_MDS_RetryMessage(MDSRank *mds, Message *m)
-    : MDSInternalContext(mds)
-  {
-    assert(m);
-    this->m = m;
-  }
+  C_MDS_RetryMessage(MDSRank *mds, const Message::const_ref &m)
+    : MDSInternalContext(mds), m(m) {}
   void finish(int r) override {
     mds->retry_dispatch(m);
   }
 };
 
+class CF_MDS_RetryMessageFactory : public MDSContextFactory {
+public:
+  CF_MDS_RetryMessageFactory(MDSRank *mds, const Message::const_ref &m)
+    : mds(mds), m(m) {}
+
+  MDSInternalContextBase *build() {
+    return new C_MDS_RetryMessage(mds, m);
+  }
+
+private:
+  MDSRank *mds;
+  Message::const_ref m;
+};
+
 /**
  * The aspect of MDSRank exposed to MDSDaemon but not subsystems: i.e.
  * the service/dispatcher stuff like init/shutdown that subsystems should
@@ -582,23 +616,23 @@ public:
   void shutdown();
   bool handle_asok_command(std::string_view command, const cmdmap_t& cmdmap,
                            Formatter *f, std::ostream& ss);
-  void handle_mds_map(MMDSMap *m, MDSMap *oldmap);
+  void handle_mds_map(const MMDSMap::const_ref &m, const MDSMap &oldmap);
   void handle_osd_map();
   void update_log_config();
 
   bool handle_command(
     const cmdmap_t &cmdmap,
-    MCommand *m,
+    const MCommand::const_ref &m,
     int *r,
     std::stringstream *ds,
     std::stringstream *ss,
     bool *need_reply);
 
   void dump_sessions(const SessionFilter &filter, Formatter *f) const;
-  void evict_clients(const SessionFilter &filter, MCommand *m);
+  void evict_clients(const SessionFilter &filter, const MCommand::const_ref &m);
 
   // Call into me from MDS::ms_dispatch
-  bool ms_dispatch(Message *m);
+  bool ms_dispatch(const Message::const_ref &m);
 
   MDSRankDispatcher(
       mds_rank_t whoami_,
@@ -606,7 +640,7 @@ public:
       LogChannelRef &clog_,
       SafeTimer &timer_,
       Beacon &beacon_,
-      MDSMap *& mdsmap_,
+      std::unique_ptr<MDSMap> &mdsmap_,
       Messenger *msgr,
       MonClient *monc_,
       Context *respawn_hook_,
@@ -619,7 +653,6 @@ do { \
   if (m->get_connection() && (m->get_connection()->get_peer_type() & (peers)) == 0) { \
     dout(0) << __FILE__ << "." << __LINE__ << ": filtered out request, peer=" << m->get_connection()->get_peer_type() \
            << " allowing=" << #peers << " message=" << *m << dendl; \
-    m->put();                                                      \
     return true; \
   } \
 } while (0)
index 11dcd8e147f4530684b5707c07ec993573457096..4a3a8b07776746f811bcf0abddc8a5476ccf1b2f 100644 (file)
@@ -26,8 +26,6 @@
 #include "MDSTableClient.h"
 #include "events/ETableClient.h"
 
-#include "messages/MMDSTableRequest.h"
-
 #include "common/config.h"
 
 #define dout_context g_ceph_context
@@ -48,7 +46,7 @@ public:
 };
 
 
-void MDSTableClient::handle_request(class MMDSTableRequest *m)
+void MDSTableClient::handle_request(const MMDSTableRequest::const_ref &m)
 {
   dout(10) << "handle_request " << *m << dendl;
   assert(m->table == table);
@@ -56,8 +54,6 @@ void MDSTableClient::handle_request(class MMDSTableRequest *m)
   if (mds->get_state() < MDSMap::STATE_RESOLVE) {
     if (mds->get_want_state() == CEPH_MDS_STATE_RESOLVE) {
       mds->wait_for_resolve(new C_MDS_RetryMessage(mds, m));
-    } else {
-      m->put();
     }
     return;
   }
@@ -106,7 +102,7 @@ void MDSTableClient::handle_request(class MMDSTableRequest *m)
       dout(10) << "stray agree on " << reqid << " tid " << tid
               << ", sending ROLLBACK" << dendl;
       assert(!server_ready);
-      MMDSTableRequest *req = new MMDSTableRequest(table, TABLESERVER_OP_ROLLBACK, 0, tid);
+      MMDSTableRequest::ref req(new MMDSTableRequest(table, TABLESERVER_OP_ROLLBACK, 0, tid), false);
       mds->send_message_mds(req, mds->get_mds_map()->get_tableserver());
     }
     break;
@@ -145,8 +141,6 @@ void MDSTableClient::handle_request(class MMDSTableRequest *m)
   default:
     assert(0 == "unrecognized mds_table_client request op");
   }
-
-  m->put();
 }
 
 
@@ -180,7 +174,7 @@ void MDSTableClient::_prepare(bufferlist& mutation, version_t *ptid, bufferlist
 
   if (server_ready) {
     // send message
-    MMDSTableRequest *req = new MMDSTableRequest(table, TABLESERVER_OP_PREPARE, reqid);
+    MMDSTableRequest::ref req(new MMDSTableRequest(table, TABLESERVER_OP_PREPARE, reqid), false);
     req->bl = mutation;
     mds->send_message_mds(req, mds->get_mds_map()->get_tableserver());
   } else
@@ -204,7 +198,7 @@ void MDSTableClient::commit(version_t tid, LogSegment *ls)
 
   if (server_ready) {
     // send message
-    MMDSTableRequest *req = new MMDSTableRequest(table, TABLESERVER_OP_COMMIT, 0, tid);
+    MMDSTableRequest::ref req(new MMDSTableRequest(table, TABLESERVER_OP_COMMIT, 0, tid), false);
     mds->send_message_mds(req, mds->get_mds_map()->get_tableserver());
   } else
     dout(10) << "tableserver is not ready yet, deferring request" << dendl;
@@ -238,7 +232,7 @@ void MDSTableClient::resend_commits()
        p != pending_commit.end();
        ++p) {
     dout(10) << "resending commit on " << p->first << dendl;
-    MMDSTableRequest *req = new MMDSTableRequest(table, TABLESERVER_OP_COMMIT, 0, p->first);
+    MMDSTableRequest::ref req(new MMDSTableRequest(table, TABLESERVER_OP_COMMIT, 0, p->first), false);
     mds->send_message_mds(req, mds->get_mds_map()->get_tableserver());
   }
 }
@@ -254,7 +248,7 @@ void MDSTableClient::resend_prepares()
        p != pending_prepare.end();
        ++p) {
     dout(10) << "resending prepare on " << p->first << dendl;
-    MMDSTableRequest *req = new MMDSTableRequest(table, TABLESERVER_OP_PREPARE, p->first);
+    MMDSTableRequest::ref req(new MMDSTableRequest(table, TABLESERVER_OP_PREPARE, p->first), false);
     req->bl = p->second.mutation;
     mds->send_message_mds(req, mds->get_mds_map()->get_tableserver());
   }
index 043ffc12664f22c2161091c34a0c7bf88357baf7..c09b54b2fd187442269996097e233f7a5d4fe379 100644 (file)
 #include "MDSContext.h"
 #include "mds_table_types.h"
 
+#include "messages/MMDSTableRequest.h"
+
 class MDSRank;
 class LogSegment;
-class MMDSTableRequest;
 
 class MDSTableClient {
 protected:
@@ -61,7 +62,7 @@ public:
     mds(m), table(tab), last_reqid(~0ULL), server_ready(false) {}
   virtual ~MDSTableClient() {}
 
-  void handle_request(MMDSTableRequest *m);
+  void handle_request(const MMDSTableRequest::const_ref &m);
 
   void _prepare(bufferlist& mutation, version_t *ptid, bufferlist *pbl, MDSInternalContextBase *onfinish);
   void commit(version_t tid, LogSegment *ls);
@@ -91,8 +92,8 @@ public:
 
   // child must implement
   virtual void resend_queries() = 0;
-  virtual void handle_query_result(MMDSTableRequest *m) = 0;
-  virtual void handle_notify_prep(MMDSTableRequest *m) = 0;
+  virtual void handle_query_result(const MMDSTableRequest::const_ref &m) = 0;
+  virtual void handle_notify_prep(const MMDSTableRequest::const_ref &m) = 0;
   virtual void notify_commit(version_t tid) = 0;
 
   // and friendly front-end for _prepare.
index c207425e9761e595f50ac69799c76b98a57ea304..250e3405602e471fc3697d850974967434618748 100644 (file)
@@ -17,7 +17,6 @@
 #include "MDLog.h"
 #include "msg/Messenger.h"
 
-#include "messages/MMDSTableRequest.h"
 #include "events/ETableServer.h"
 
 #define dout_context g_ceph_context
@@ -25,8 +24,7 @@
 #undef dout_prefix
 #define dout_prefix *_dout << "mds." << rank << ".tableserver(" << get_mdstable_name(table) << ") "
 
-/* This function DOES put the passed message before returning */
-void MDSTableServer::handle_request(MMDSTableRequest *req)
+void MDSTableServer::handle_request(const MMDSTableRequest::const_ref &req)
 {
   assert(req->op >= 0);
   switch (req->op) {
@@ -41,20 +39,19 @@ void MDSTableServer::handle_request(MMDSTableRequest *req)
 
 class C_Prepare : public MDSLogContextBase {
   MDSTableServer *server;
-  MMDSTableRequest *req;
+  MMDSTableRequest::const_ref req;
   version_t tid;
   MDSRank *get_mds() override { return server->mds; }
 public:
 
-  C_Prepare(MDSTableServer *s, MMDSTableRequest *r, version_t v) : server(s), req(r), tid(v) {}
+  C_Prepare(MDSTableServer *s, const MMDSTableRequest::const_ref r, version_t v) : server(s), req(r), tid(v) {}
   void finish(int r) override {
     server->_prepare_logged(req, tid);
   }
 };
 
 // prepare
-/* This function DOES put the passed message before returning */
-void MDSTableServer::handle_prepare(MMDSTableRequest *req)
+void MDSTableServer::handle_prepare(const MMDSTableRequest::const_ref &req)
 {
   dout(7) << "handle_prepare " << *req << dendl;
   mds_rank_t from = mds_rank_t(req->get_source().num());
@@ -71,7 +68,7 @@ void MDSTableServer::handle_prepare(MMDSTableRequest *req)
   mds->mdlog->flush();
 }
 
-void MDSTableServer::_prepare_logged(MMDSTableRequest *req, version_t tid)
+void MDSTableServer::_prepare_logged(const MMDSTableRequest::const_ref &req, version_t tid)
 {
   dout(7) << "_create_logged " << *req << " tid " << tid << dendl;
   mds_rank_t from = mds_rank_t(req->get_source().num());
@@ -79,11 +76,12 @@ void MDSTableServer::_prepare_logged(MMDSTableRequest *req, version_t tid)
   assert(g_conf()->mds_kill_mdstable_at != 2);
 
   _note_prepare(from, req->reqid);
-  _prepare(req->bl, req->reqid, from);
+  bufferlist out;
+  _prepare(req->bl, req->reqid, from, out);
   assert(version == tid);
 
-  MMDSTableRequest *reply = new MMDSTableRequest(table, TABLESERVER_OP_AGREE, req->reqid, tid);
-  reply->bl = req->bl;
+  MMDSTableRequest::ref reply(new MMDSTableRequest(table, TABLESERVER_OP_AGREE, req->reqid, tid), false);
+  reply->bl = std::move(out);
 
   if (_notify_prep(tid)) {
     auto& p = pending_notifies[tid];
@@ -93,10 +91,9 @@ void MDSTableServer::_prepare_logged(MMDSTableRequest *req, version_t tid)
   } else {
     mds->send_message_mds(reply, from);
   }
-  req->put();
 }
 
-void MDSTableServer::handle_notify_ack(MMDSTableRequest *m)
+void MDSTableServer::handle_notify_ack(const MMDSTableRequest::const_ref &m)
 {
   dout(7) << __func__ << " " << *m << dendl;
   mds_rank_t from = mds_rank_t(m->get_source().num());
@@ -117,23 +114,21 @@ void MDSTableServer::handle_notify_ack(MMDSTableRequest *m)
     }
   } else {
   }
-  m->put();
 }
 
 class C_Commit : public MDSLogContextBase {
   MDSTableServer *server;
-  MMDSTableRequest *req;
+  MMDSTableRequest::const_ref req;
   MDSRank *get_mds() override { return server->mds; }
 public:
-  C_Commit(MDSTableServer *s, MMDSTableRequest *r) : server(s), req(r) {}
+  C_Commit(MDSTableServer *s, const MMDSTableRequest::const_ref &r) : server(s), req(r) {}
   void finish(int r) override {
     server->_commit_logged(req);
   }
 };
 
 // commit
-/* This function DOES put the passed message before returning */
-void MDSTableServer::handle_commit(MMDSTableRequest *req)
+void MDSTableServer::handle_commit(const MMDSTableRequest::const_ref &req)
 {
   dout(7) << "handle_commit " << *req << dendl;
 
@@ -143,7 +138,6 @@ void MDSTableServer::handle_commit(MMDSTableRequest *req)
 
     if (committing_tids.count(tid)) {
       dout(0) << "got commit for tid " << tid << ", already committing, waiting." << dendl;
-      req->put();
       return;
     }
 
@@ -159,9 +153,8 @@ void MDSTableServer::handle_commit(MMDSTableRequest *req)
   else if (tid <= version) {
     dout(0) << "got commit for tid " << tid << " <= " << version
            << ", already committed, sending ack." << dendl;
-    MMDSTableRequest *reply = new MMDSTableRequest(table, TABLESERVER_OP_ACK, req->reqid, tid);
+    MMDSTableRequest::ref reply(new MMDSTableRequest(table, TABLESERVER_OP_ACK, req->reqid, tid), false);
     mds->send_message(reply, req->get_connection());
-    req->put();
   } 
   else {
     // wtf.
@@ -170,8 +163,7 @@ void MDSTableServer::handle_commit(MMDSTableRequest *req)
   }
 }
 
-/* This function DOES put the passed message before returning */
-void MDSTableServer::_commit_logged(MMDSTableRequest *req)
+void MDSTableServer::_commit_logged(const MMDSTableRequest::const_ref &req)
 {
   dout(7) << "_commit_logged, sending ACK" << dendl;
 
@@ -184,24 +176,23 @@ void MDSTableServer::_commit_logged(MMDSTableRequest *req)
   _commit(tid, req);
   _note_commit(tid);
 
-  MMDSTableRequest *reply = new MMDSTableRequest(table, TABLESERVER_OP_ACK, req->reqid, req->get_tid());
+  MMDSTableRequest::ref reply(new MMDSTableRequest(table, TABLESERVER_OP_ACK, req->reqid, req->get_tid()), false);
   mds->send_message_mds(reply, mds_rank_t(req->get_source().num()));
-  req->put();
 }
 
 class C_Rollback : public MDSLogContextBase {
   MDSTableServer *server;
-  MMDSTableRequest *req;
+  MMDSTableRequest::const_ref req;
   MDSRank *get_mds() override { return server->mds; }
 public:
-  C_Rollback(MDSTableServer *s, MMDSTableRequest *r) : server(s), req(r) {}
+  C_Rollback(MDSTableServer *s, const MMDSTableRequest::const_ref &r) : server(s), req(r) {}
   void finish(int r) override {
     server->_rollback_logged(req);
   }
 };
 
 // ROLLBACK
-void MDSTableServer::handle_rollback(MMDSTableRequest *req)
+void MDSTableServer::handle_rollback(const MMDSTableRequest::const_ref &req)
 {
   dout(7) << "handle_rollback " << *req << dendl;
 
@@ -218,8 +209,7 @@ void MDSTableServer::handle_rollback(MMDSTableRequest *req)
                                 new C_Rollback(this, req));
 }
 
-/* This function DOES put the passed message before returning */
-void MDSTableServer::_rollback_logged(MMDSTableRequest *req)
+void MDSTableServer::_rollback_logged(const MMDSTableRequest::const_ref &req)
 {
   dout(7) << "_rollback_logged " << *req << dendl;
 
@@ -230,8 +220,6 @@ void MDSTableServer::_rollback_logged(MMDSTableRequest *req)
 
   _rollback(tid);
   _note_rollback(tid);
-
-  req->put();
 }
 
 
@@ -293,13 +281,13 @@ void MDSTableServer::_do_server_recovery()
       next_reqids[who] = p.second.reqid + 1;
 
     version_t tid = p.second.tid;
-    MMDSTableRequest *reply = new MMDSTableRequest(table, TABLESERVER_OP_AGREE, p.second.reqid, tid);
+    MMDSTableRequest::ref reply(new MMDSTableRequest(table, TABLESERVER_OP_AGREE, p.second.reqid, tid), false);
     _get_reply_buffer(tid, &reply->bl);
     mds->send_message_mds(reply, who);
   }
 
   for (auto p : active_clients) {
-    MMDSTableRequest *reply = new MMDSTableRequest(table, TABLESERVER_OP_SERVER_READY, next_reqids[p]);
+    MMDSTableRequest::ref reply(new MMDSTableRequest(table, TABLESERVER_OP_SERVER_READY, next_reqids[p]), false);
     mds->send_message_mds(reply, p);
   }
   recovered = true;
@@ -343,12 +331,12 @@ void MDSTableServer::handle_mds_recovery(mds_rank_t who)
     if (p->second.reqid >= next_reqid)
       next_reqid = p->second.reqid + 1;
 
-    MMDSTableRequest *reply = new MMDSTableRequest(table, TABLESERVER_OP_AGREE, p->second.reqid, p->second.tid);
+    MMDSTableRequest::ref reply(new MMDSTableRequest(table, TABLESERVER_OP_AGREE, p->second.reqid, p->second.tid), false);
     _get_reply_buffer(p->second.tid, &reply->bl);
     mds->send_message_mds(reply, who);
   }
 
-  MMDSTableRequest *reply = new MMDSTableRequest(table, TABLESERVER_OP_SERVER_READY, next_reqid);
+  MMDSTableRequest::ref reply(new MMDSTableRequest(table, TABLESERVER_OP_SERVER_READY, next_reqid), false);
   mds->send_message_mds(reply, who);
 }
 
@@ -358,7 +346,7 @@ void MDSTableServer::handle_mds_failure_or_stop(mds_rank_t who)
 
   active_clients.erase(who);
 
-  list<MMDSTableRequest*> rollback;
+  list<MMDSTableRequest::ref> rollback;
   for (auto p = pending_notifies.begin(); p != pending_notifies.end(); ) {
     auto q = p++;
     if (q->second.mds == who) {
@@ -378,8 +366,8 @@ void MDSTableServer::handle_mds_failure_or_stop(mds_rank_t who)
     }
   }
 
-  for (auto p : rollback) {
-    p->op = TABLESERVER_OP_ROLLBACK;
-    handle_rollback(p);
+  for (auto &req : rollback) {
+    req->op = TABLESERVER_OP_ROLLBACK;
+    handle_rollback(req);
   }
 }
index 5d3f47ce4afa95ea2ab8fcdf2ff6a20da8eb84ed..065aa474eb5b04ac2013017a188663f580190889 100644 (file)
@@ -18,7 +18,7 @@
 #include "MDSTable.h"
 #include "MDSContext.h"
 
-class MMDSTableRequest;
+#include "messages/MMDSTableRequest.h"
 
 class MDSTableServer : public MDSTable {
 protected:
@@ -32,34 +32,34 @@ private:
   struct notify_info_t {
     set<mds_rank_t> notify_ack_gather;
     mds_rank_t mds;
-    MMDSTableRequest *reply;
+    MMDSTableRequest::ref reply;
     MDSInternalContextBase *onfinish;
     notify_info_t() : reply(NULL), onfinish(NULL) {}
   };
   map<version_t, notify_info_t> pending_notifies;
 
-  void handle_prepare(MMDSTableRequest *m);
-  void _prepare_logged(MMDSTableRequest *m, version_t tid);
+  void handle_prepare(const MMDSTableRequest::const_ref &m);
+  void _prepare_logged(const MMDSTableRequest::const_ref &m, version_t tid);
   friend class C_Prepare;
 
-  void handle_commit(MMDSTableRequest *m);
-  void _commit_logged(MMDSTableRequest *m);
+  void handle_commit(const MMDSTableRequest::const_ref &m);
+  void _commit_logged(const MMDSTableRequest::const_ref &m);
   friend class C_Commit;
 
-  void handle_rollback(MMDSTableRequest *m);
-  void _rollback_logged(MMDSTableRequest *m);
+  void handle_rollback(const MMDSTableRequest::const_ref &m);
+  void _rollback_logged(const MMDSTableRequest::const_ref &m);
   friend class C_Rollback;
 
   void _server_update_logged(bufferlist& bl);
   friend class C_ServerUpdate;
 
-  void handle_notify_ack(MMDSTableRequest *m);
+  void handle_notify_ack(const MMDSTableRequest::const_ref &m);
 
 public:
-  virtual void handle_query(MMDSTableRequest *m) = 0;
-  virtual void _prepare(bufferlist &bl, uint64_t reqid, mds_rank_t bymds) = 0;
+  virtual void handle_query(const MMDSTableRequest::const_ref &m) = 0;
+  virtual void _prepare(const bufferlist &bl, uint64_t reqid, mds_rank_t bymds, bufferlist& out) = 0;
   virtual void _get_reply_buffer(version_t tid, bufferlist *pbl) const = 0;
-  virtual void _commit(version_t tid, MMDSTableRequest *req=NULL) = 0;
+  virtual void _commit(version_t tid, MMDSTableRequest::const_ref req) = 0;
   virtual void _rollback(version_t tid) = 0;
   virtual void _server_update(bufferlist& bl) { ceph_abort(); }
   virtual bool _notify_prep(version_t tid) { return false; };
@@ -94,7 +94,7 @@ public:
     MDSTable(m, get_mdstable_name(tab), false), table(tab), recovered(false) {}
   ~MDSTableServer() override {}
 
-  void handle_request(MMDSTableRequest *m);
+  void handle_request(const MMDSTableRequest::const_ref &m);
   void do_server_update(bufferlist& bl);
 
   virtual void encode_server_state(bufferlist& bl) const = 0;
index 4fc8db4f6cb92ab2483b1139c44efd3240233b4b..c3ed9b63e013f9222dea6a697d025ba4de8a1c88 100644 (file)
 
 #include "messages/MClientCaps.h"
 
-#include "messages/MExportDirDiscover.h"
-#include "messages/MExportDirDiscoverAck.h"
-#include "messages/MExportDirCancel.h"
-#include "messages/MExportDirPrep.h"
-#include "messages/MExportDirPrepAck.h"
-#include "messages/MExportDir.h"
-#include "messages/MExportDirAck.h"
-#include "messages/MExportDirNotify.h"
-#include "messages/MExportDirNotifyAck.h"
-#include "messages/MExportDirFinish.h"
-
-#include "messages/MExportCaps.h"
-#include "messages/MExportCapsAck.h"
-#include "messages/MGatherCaps.h"
-
-
 /*
  * this is what the dir->dir_auth values look like
  *
@@ -107,60 +91,59 @@ public:
   }
 };
 
-/* This function DOES put the passed message before returning*/
-void Migrator::dispatch(Message *m)
+void Migrator::dispatch(const Message::const_ref &m)
 {
   switch (m->get_type()) {
     // import
   case MSG_MDS_EXPORTDIRDISCOVER:
-    handle_export_discover(static_cast<MExportDirDiscover*>(m));
+    handle_export_discover(boost::static_pointer_cast<MExportDirDiscover::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_EXPORTDIRPREP:
-    handle_export_prep(static_cast<MExportDirPrep*>(m));
+    handle_export_prep(boost::static_pointer_cast<MExportDirPrep::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_EXPORTDIR:
     if (unlikely(inject_session_race)) {
       dout(0) << "waiting for inject_session_race" << dendl;
       mds->wait_for_any_client_connection(new C_MDS_RetryMessage(mds, m));
     } else {
-      handle_export_dir(static_cast<MExportDir*>(m));
+      handle_export_dir(boost::static_pointer_cast<MExportDir::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     }
     break;
   case MSG_MDS_EXPORTDIRFINISH:
-    handle_export_finish(static_cast<MExportDirFinish*>(m));
+    handle_export_finish(boost::static_pointer_cast<MExportDirFinish::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_EXPORTDIRCANCEL:
-    handle_export_cancel(static_cast<MExportDirCancel*>(m));
+    handle_export_cancel(boost::static_pointer_cast<MExportDirCancel::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
     // export 
   case MSG_MDS_EXPORTDIRDISCOVERACK:
-    handle_export_discover_ack(static_cast<MExportDirDiscoverAck*>(m));
+    handle_export_discover_ack(boost::static_pointer_cast<MExportDirDiscoverAck::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_EXPORTDIRPREPACK:
-    handle_export_prep_ack(static_cast<MExportDirPrepAck*>(m));
+    handle_export_prep_ack(boost::static_pointer_cast<MExportDirPrepAck::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_EXPORTDIRACK:
-    handle_export_ack(static_cast<MExportDirAck*>(m));
+    handle_export_ack(boost::static_pointer_cast<MExportDirAck::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_EXPORTDIRNOTIFYACK:
-    handle_export_notify_ack(static_cast<MExportDirNotifyAck*>(m));
-    break;    
+    handle_export_notify_ack(boost::static_pointer_cast<MExportDirNotifyAck::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
+    break;
 
     // export 3rd party (dir_auth adjustments)
   case MSG_MDS_EXPORTDIRNOTIFY:
-    handle_export_notify(static_cast<MExportDirNotify*>(m));
+    handle_export_notify(boost::static_pointer_cast<MExportDirNotify::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
     // caps
   case MSG_MDS_EXPORTCAPS:
-    handle_export_caps(static_cast<MExportCaps*>(m));
+    handle_export_caps(boost::static_pointer_cast<MExportCaps::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_EXPORTCAPSACK:
-    handle_export_caps_ack(static_cast<MExportCapsAck*>(m));
+    handle_export_caps_ack(boost::static_pointer_cast<MExportCapsAck::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
   case MSG_MDS_GATHERCAPS:
-    handle_gather_caps(static_cast<MGatherCaps*>(m));
+    handle_gather_caps(boost::static_pointer_cast<MGatherCaps::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     break;
 
   default:
@@ -969,10 +952,8 @@ void Migrator::dispatch_export_dir(MDRequestRef& mdr, int count)
 /*
  * called on receipt of MExportDirDiscoverAck
  * the importer now has the directory's _inode_ in memory, and pinned.
- *
- * This function DOES put the passed message before returning
  */
-void Migrator::handle_export_discover_ack(MExportDirDiscoverAck *m)
+void Migrator::handle_export_discover_ack(const MExportDirDiscoverAck::const_ref &m)
 {
   CDir *dir = cache->get_dirfrag(m->get_dirfrag());
   mds_rank_t dest(m->get_source().num());
@@ -1007,8 +988,6 @@ void Migrator::handle_export_discover_ack(MExportDirDiscoverAck *m)
       export_try_cancel(dir, false);
     }
   }
-  
-  m->put();  // done
 }
 
 class C_M_ExportSessionsFlushed : public MigratorContext {
@@ -1290,8 +1269,7 @@ void Migrator::get_export_client_set(CInode *in, set<client_t>& client_set)
   }
 }
 
-/* This function DOES put the passed message before returning*/
-void Migrator::handle_export_prep_ack(MExportDirPrepAck *m)
+void Migrator::handle_export_prep_ack(const MExportDirPrepAck::const_ref &m)
 {
   CDir *dir = cache->get_dirfrag(m->get_dirfrag());
   mds_rank_t dest(m->get_source().num());
@@ -1307,7 +1285,6 @@ void Migrator::handle_export_prep_ack(MExportDirPrepAck *m)
       it->second.peer != mds_rank_t(m->get_source().num())) {
     // export must have aborted.  
     dout(7) << "export must have aborted" << dendl;
-    m->put();
     return;
   }
   assert(it->second.state == EXPORT_PREPPING);
@@ -1315,7 +1292,6 @@ void Migrator::handle_export_prep_ack(MExportDirPrepAck *m)
   if (!m->is_success()) {
     dout(7) << "peer couldn't acquire all needed locks or wasn't active, canceling" << dendl;
     export_try_cancel(dir, false);
-    m->put();
     return;
   }
 
@@ -1340,8 +1316,9 @@ void Migrator::handle_export_prep_ack(MExportDirPrepAck *m)
     MExportDirNotify *notify = new MExportDirNotify(dir->dirfrag(), it->second.tid, true,
                                                    mds_authority_t(mds->get_nodeid(),CDIR_AUTH_UNKNOWN),
                                                    mds_authority_t(mds->get_nodeid(),it->second.peer));
-    for (set<CDir*>::iterator q = bounds.begin(); q != bounds.end(); ++q)
-      notify->get_bounds().push_back((*q)->dirfrag());
+    for (auto &cdir : bounds) {
+      notify->get_bounds().push_back(cdir->dirfrag());
+    }
     mds->send_message_mds(notify, p.first);
     
   }
@@ -1352,9 +1329,6 @@ void Migrator::handle_export_prep_ack(MExportDirPrepAck *m)
   // nobody to warn?
   if (it->second.warning_ack_waiting.empty())
     export_go(dir);  // start export.
-    
-  // done.
-  m->put();
 }
 
 
@@ -1725,10 +1699,8 @@ public:
 
 /*
  * i should get an export_ack from the export target.
- *
- * This function DOES put the passed message before returning
  */
-void Migrator::handle_export_ack(MExportDirAck *m)
+void Migrator::handle_export_ack(const MExportDirAck::const_ref &m)
 {
   CDir *dir = cache->get_dirfrag(m->get_dirfrag());
   mds_rank_t dest(m->get_source().num());
@@ -1777,8 +1749,6 @@ void Migrator::handle_export_ack(MExportDirAck *m)
   mds->mdlog->submit_entry(le, new C_MDS_ExportFinishLogged(this, dir));
   mds->mdlog->flush();
   assert (g_conf()->mds_kill_export_at != 10);
-  
-  m->put();
 }
 
 void Migrator::export_notify_abort(CDir *dir, export_state_t& stat, set<CDir*>& bounds)
@@ -1938,10 +1908,8 @@ void Migrator::export_logged_finish(CDir *dir)
  * notify:
  *  i'll get an ack from each bystander.
  *  when i get them all, unfreeze and send the finish.
- *
- * This function DOES put the passed message before returning
  */
-void Migrator::handle_export_notify_ack(MExportDirNotifyAck *m)
+void Migrator::handle_export_notify_ack(const MExportDirNotifyAck::const_ref &m)
 {
   CDir *dir = cache->get_dirfrag(m->get_dirfrag());
   mds_rank_t dest(m->get_source().num());
@@ -1993,8 +1961,6 @@ void Migrator::handle_export_notify_ack(MExportDirNotifyAck *m)
       }
     }
   }
-
-  m->put();
 }
 
 void Migrator::export_finish(CDir *dir)
@@ -2091,15 +2057,31 @@ void Migrator::export_finish(CDir *dir)
 
 
 
+class C_MDS_ExportDiscover : public MigratorContext {
+public:
+  C_MDS_ExportDiscover(Migrator *mig, const MExportDirDiscover::const_ref& m) : MigratorContext(mig), m(m) {}
+  void finish(int r) override {
+    mig->handle_export_discover(m, true);
+  }
+private:
+  MExportDirDiscover::const_ref m;
+};
 
-
-
-
+class C_MDS_ExportDiscoverFactory : public MDSContextFactory {
+public:
+  C_MDS_ExportDiscoverFactory(Migrator *mig, MExportDirDiscover::const_ref m) : mig(mig), m(m) {}
+  MDSInternalContextBase *build() {
+    return new C_MDS_ExportDiscover(mig, m);
+  }
+private:
+  Migrator *mig;
+  MExportDirDiscover::const_ref m;
+};
 
 // ==========================================================
 // IMPORT
 
-void Migrator::handle_export_discover(MExportDirDiscover *m)
+void Migrator::handle_export_discover(const MExportDirDiscover::const_ref &m, bool started)
 {
   mds_rank_t from = m->get_source_mds();
   assert(from != mds->get_nodeid());
@@ -2112,16 +2094,14 @@ void Migrator::handle_export_discover(MExportDirDiscover *m)
   if (!mds->is_active()) {
     dout(7) << " not active, send NACK " << dendl;
     mds->send_message_mds(new MExportDirDiscoverAck(df, m->get_tid(), false), from);
-    m->put();
     return;
   }
 
   // only start discovering on this message once.
   import_state_t *p_state;
   map<dirfrag_t,import_state_t>::iterator it = import_state.find(df);
-  if (!m->started) {
+  if (!started) {
     assert(it == import_state.end());
-    m->started = true;
     p_state = &import_state[df];
     p_state->state = IMPORT_DISCOVERING;
     p_state->peer = from;
@@ -2132,16 +2112,16 @@ void Migrator::handle_export_discover(MExportDirDiscover *m)
        it->second.peer != from ||
        it->second.tid != m->get_tid()) {
       dout(7) << " dropping obsolete message" << dendl;
-      m->put();
       return;
     }
     assert(it->second.state == IMPORT_DISCOVERING);
     p_state = &it->second;
   }
 
+  C_MDS_ExportDiscoverFactory cf(this, m);
   if (!mds->mdcache->is_open()) {
     dout(5) << " waiting for root" << dendl;
-    mds->mdcache->wait_for_open(new C_MDS_RetryMessage(mds, m));
+    mds->mdcache->wait_for_open(cf.build());
     return;
   }
 
@@ -2154,7 +2134,7 @@ void Migrator::handle_export_discover(MExportDirDiscover *m)
     filepath fpath(m->get_path());
     vector<CDentry*> trace;
     MDRequestRef null_ref;
-    int r = cache->path_traverse(null_ref, m, NULL, fpath, &trace, NULL, MDS_TRAVERSE_DISCOVER);
+    int r = cache->path_traverse(null_ref, cf, fpath, &trace, NULL, MDS_TRAVERSE_DISCOVER);
     if (r > 0) return;
     if (r < 0) {
       dout(7) << "handle_export_discover_2 failed to discover or not dir " << m->get_path() << ", NAK" << dendl;
@@ -2176,7 +2156,6 @@ void Migrator::handle_export_discover(MExportDirDiscover *m)
   // reply
   dout(7) << " sending export_discover_ack on " << *in << dendl;
   mds->send_message_mds(new MExportDirDiscoverAck(df, m->get_tid()), p_state->peer);
-  m->put();
   assert (g_conf()->mds_kill_import_at != 2);
 }
 
@@ -2200,8 +2179,7 @@ void Migrator::import_reverse_prepping(CDir *dir, import_state_t& stat)
   import_reverse_final(dir);
 }
 
-/* This function DOES put the passed message before returning*/
-void Migrator::handle_export_cancel(MExportDirCancel *m)
+void Migrator::handle_export_cancel(const MExportDirCancel::const_ref &m)
 {
   dout(7) << "handle_export_cancel on " << m->get_dirfrag() << dendl;
   dirfrag_t df = m->get_dirfrag();
@@ -2230,11 +2208,30 @@ void Migrator::handle_export_cancel(MExportDirCancel *m)
   } else {
     assert(0 == "got export_cancel in weird state");
   }
-  m->put();
 }
 
-/* This function DOES put the passed message before returning*/
-void Migrator::handle_export_prep(MExportDirPrep *m)
+class C_MDS_ExportPrep : public MigratorContext {
+public:
+  C_MDS_ExportPrep(Migrator *mig, const MExportDirPrep::const_ref& m) : MigratorContext(mig), m(m) {}
+  void finish(int r) override {
+    mig->handle_export_prep(m, true);
+  }
+private:
+  MExportDirPrep::const_ref m;
+};
+
+class C_MDS_ExportPrepFactory : public MDSContextFactory {
+public:
+  C_MDS_ExportPrepFactory(Migrator *mig, MExportDirPrep::const_ref m) : mig(mig), m(m) {}
+  MDSInternalContextBase *build() {
+    return new C_MDS_ExportPrep(mig, m);
+  }
+private:
+  Migrator *mig;
+  MExportDirPrep::const_ref m;
+};
+
+void Migrator::handle_export_prep(const MExportDirPrep::const_ref &m, bool did_assim)
 {
   mds_rank_t oldauth = mds_rank_t(m->get_source().num());
   assert(oldauth != mds->get_nodeid());
@@ -2245,7 +2242,7 @@ void Migrator::handle_export_prep(MExportDirPrep *m)
 
   // assimilate root dir.
   map<dirfrag_t,import_state_t>::iterator it = import_state.find(m->get_dirfrag());
-  if (!m->did_assim()) {
+  if (!did_assim) {
     assert(it != import_state.end());
     assert(it->second.state == IMPORT_DISCOVERED);
     assert(it->second.peer == oldauth);
@@ -2259,7 +2256,6 @@ void Migrator::handle_export_prep(MExportDirPrep *m)
        it->second.peer != oldauth ||
        it->second.tid != m->get_tid()) {
       dout(7) << "handle_export_prep obsolete message, dropping" << dendl;
-      m->put();
       return;
     }
     assert(it->second.state == IMPORT_PREPPING);
@@ -2276,17 +2272,14 @@ void Migrator::handle_export_prep(MExportDirPrep *m)
 
   // build import bound map
   map<inodeno_t, fragset_t> import_bound_fragset;
-  for (list<dirfrag_t>::iterator p = m->get_bounds().begin();
-       p != m->get_bounds().end();
-       ++p) {
-    dout(10) << " bound " << *p << dendl;
-    import_bound_fragset[p->ino].insert(p->frag);
+  for (const auto &bound : m->get_bounds()) {
+    dout(10) << " bound " << bound << dendl;
+    import_bound_fragset[bound.ino].insert(bound.frag);
   }
 
   // assimilate contents?
-  if (!m->did_assim()) {
+  if (!did_assim) {
     dout(7) << "doing assim on " << *dir << dendl;
-    m->mark_assim();  // only do this the first time!
 
     // change import state
     it->second.state = IMPORT_PREPPING;
@@ -2304,15 +2297,13 @@ void Migrator::handle_export_prep(MExportDirPrep *m)
 
     // assimilate traces to exports
     // each trace is: df ('-' | ('f' dir | 'd') dentry inode (dir dentry inode)*)
-    for (list<bufferlist>::iterator p = m->traces.begin();
-        p != m->traces.end();
-        ++p) {
-      auto q = p->cbegin();
+    for (const auto &bl : m->traces) {
+      auto q = bl.cbegin();
       dirfrag_t df;
       decode(df, q);
       char start;
       decode(start, q);
-      dout(10) << " trace from " << df << " start " << start << " len " << p->length() << dendl;
+      dout(10) << " trace from " << df << " start " << start << " len " << bl.length() << dendl;
 
       CDir *cur = 0;
       if (start == 'd') {
@@ -2356,6 +2347,8 @@ void Migrator::handle_export_prep(MExportDirPrep *m)
     dout(7) << " not doing assim on " << *dir << dendl;
   }
 
+  C_MDS_ExportPrepFactory cf(this, m);
+
   if (!finished.empty())
     mds->queue_waiters(finished);
 
@@ -2382,8 +2375,7 @@ void Migrator::handle_export_prep(MExportDirPrep *m)
        CDir *bound = cache->get_dirfrag(dirfrag_t(p->first, *q));
        if (!bound) {
          dout(7) << "  opening bounding dirfrag " << *q << " on " << *in << dendl;
-         cache->open_remote_dirfrag(in, *q,
-             new C_MDS_RetryMessage(mds, m));
+         cache->open_remote_dirfrag(in, *q, cf.build());
          return;
        }
 
@@ -2434,8 +2426,6 @@ void Migrator::handle_export_prep(MExportDirPrep *m)
   mds->send_message(new MExportDirPrepAck(dir->dirfrag(), success, m->get_tid()), m->get_connection());
 
   assert(g_conf()->mds_kill_import_at != 4);
-  // done 
-  m->put();
 }
 
 
@@ -2456,8 +2446,7 @@ public:
   }
 };
 
-/* This function DOES put the passed message before returning*/
-void Migrator::handle_export_dir(MExportDir *m)
+void Migrator::handle_export_dir(const MExportDir::const_ref &m)
 {
   assert (g_conf()->mds_kill_import_at != 5);
   CDir *dir = cache->get_dirfrag(m->dirfrag);
@@ -2519,10 +2508,8 @@ void Migrator::handle_export_dir(MExportDir *m)
   
   // include bounds in EImportStart
   set<CDir*> import_bounds;
-  for (vector<dirfrag_t>::iterator p = m->bounds.begin();
-       p != m->bounds.end();
-       ++p) {
-    CDir *bd = cache->get_dirfrag(*p);
+  for (const auto &bound : m->bounds) {
+    CDir *bd = cache->get_dirfrag(bound);
     assert(bd);
     le->metablob.add_dir(bd, false);  // note that parent metadata is already in the event
     import_bounds.insert(bd);
@@ -2547,8 +2534,6 @@ void Migrator::handle_export_dir(MExportDir *m)
     mds->logger->inc(l_mds_imported);
     mds->logger->inc(l_mds_imported_inodes, num_imported_inodes);
   }
-
-  m->put();
 }
 
 
@@ -2862,8 +2847,7 @@ void Migrator::import_logged_start(dirfrag_t df, CDir *dir, mds_rank_t from,
   cache->show_subtrees();
 }
 
-/* This function DOES put the passed message before returning*/
-void Migrator::handle_export_finish(MExportDirFinish *m)
+void Migrator::handle_export_finish(const MExportDirFinish::const_ref &m)
 {
   CDir *dir = cache->get_dirfrag(m->get_dirfrag());
   assert(dir);
@@ -2874,8 +2858,6 @@ void Migrator::handle_export_finish(MExportDirFinish *m)
   assert(it->second.tid == m->get_tid());
 
   import_finish(dir, false, m->is_last());
-
-  m->put();
 }
 
 void Migrator::import_finish(CDir *dir, bool notify, bool last)
@@ -3251,11 +3233,9 @@ int Migrator::decode_import_dir(bufferlist::const_iterator& blp,
 
 // authority bystander
 
-/* This function DOES put the passed message before returning*/
-void Migrator::handle_export_notify(MExportDirNotify *m)
+void Migrator::handle_export_notify(const MExportDirNotify::const_ref &m)
 {
   if (!(mds->is_clientreplay() || mds->is_active() || mds->is_stopping())) {
-    m->put();
     return;
   }
 
@@ -3291,8 +3271,6 @@ void Migrator::handle_export_notify(MExportDirNotify *m)
     // aborted.  no ack.
     dout(7) << "handle_export_notify no ack requested" << dendl;
   }
-  
-  m->put();
 }
 
 /** cap exports **/
@@ -3314,8 +3292,7 @@ void Migrator::export_caps(CInode *in)
   mds->send_message_mds(ex, dest);
 }
 
-/* This function DOES put the passed message before returning*/
-void Migrator::handle_export_caps_ack(MExportCapsAck *ack)
+void Migrator::handle_export_caps_ack(const MExportCapsAck::const_ref &ack)
 {
   mds_rank_t from = ack->get_source().num();
   CInode *in = cache->get_inode(ack->ino);
@@ -3350,15 +3327,13 @@ void Migrator::handle_export_caps_ack(MExportCapsAck *ack)
     mds->locker->request_inode_file_caps(in);
     mds->locker->try_eval(in, CEPH_CAP_LOCKS);
   }
-
-  ack->put();
 }
 
-void Migrator::handle_gather_caps(MGatherCaps *m)
+void Migrator::handle_gather_caps(const MGatherCaps::const_ref &m)
 {
   CInode *in = cache->get_inode(m->ino);
   if (!in)
-    goto out;
+    return;
 
   dout(10) << "handle_gather_caps " << *m << " from " << m->get_source()
            << " on " << *in << dendl;
@@ -3368,9 +3343,6 @@ void Migrator::handle_gather_caps(MGatherCaps *m)
       !in->is_ambiguous_auth() &&
       !in->state_test(CInode::STATE_EXPORTINGCAPS))
     export_caps(in);
-
-out:
-  m->put();
 }
 
 class C_M_LoggedImportCaps : public MigratorLogContext {
@@ -3386,8 +3358,7 @@ public:
   }  
 };
 
-/* This function DOES put the passed message before returning*/
-void Migrator::handle_export_caps(MExportCaps *ex)
+void Migrator::handle_export_caps(const MExportCaps::const_ref &ex)
 {
   dout(10) << "handle_export_caps " << *ex << " from " << ex->get_source() << dendl;
   CInode *in = cache->get_inode(ex->ino);
@@ -3397,16 +3368,13 @@ void Migrator::handle_export_caps(MExportCaps *ex)
 
   // FIXME
   if (!in->can_auth_pin()) {
-    ex->put();
     return;
   }
 
   in->auth_pin(this);
 
-  map<client_t,entity_inst_t> client_map;
-  map<client_t,client_metadata_t> client_metadata_map;
-  client_map.swap(ex->client_map);
-  client_metadata_map.swap(ex->client_metadata_map);
+  map<client_t,entity_inst_t> client_map{ex->client_map};
+  map<client_t,client_metadata_t> client_metadata_map{ex->client_metadata_map};
 
   C_M_LoggedImportCaps *finish = new C_M_LoggedImportCaps(
       this, in, mds_rank_t(ex->get_source().num()));
@@ -3423,8 +3391,6 @@ void Migrator::handle_export_caps(MExportCaps *ex)
                                std::move(client_metadata_map));
   mds->mdlog->start_submit_entry(le, finish);
   mds->mdlog->flush();
-
-  ex->put();
 }
 
 
index c4bf0ebb5c532f8d7196a351e36f5113b16576b0..822c684c48880c1a5a69f307cd6feaf570746e1c 100644 (file)
@@ -35,20 +35,19 @@ class CInode;
 class CDentry;
 class Session;
 
-class MExportDirDiscover;
-class MExportDirDiscoverAck;
-class MExportDirCancel;
-class MExportDirPrep;
-class MExportDirPrepAck;
-class MExportDir;
-class MExportDirAck;
-class MExportDirNotify;
-class MExportDirNotifyAck;
-class MExportDirFinish;
-
-class MExportCaps;
-class MExportCapsAck;
-class MGatherCaps;
+#include "messages/MExportCaps.h"
+#include "messages/MExportCapsAck.h"
+#include "messages/MExportDir.h"
+#include "messages/MExportDirAck.h"
+#include "messages/MExportDirCancel.h"
+#include "messages/MExportDirDiscover.h"
+#include "messages/MExportDirDiscoverAck.h"
+#include "messages/MExportDirFinish.h"
+#include "messages/MExportDirNotify.h"
+#include "messages/MExportDirNotifyAck.h"
+#include "messages/MExportDirPrep.h"
+#include "messages/MExportDirPrepAck.h"
+#include "messages/MGatherCaps.h"
 
 class EImportStart;
 
@@ -152,10 +151,10 @@ protected:
 
   map<dirfrag_t, import_state_t>  import_state;
 
-  void handle_export_discover_ack(MExportDirDiscoverAck *m);
+  void handle_export_discover_ack(const MExportDirDiscoverAck::const_ref &m);
   void export_frozen(CDir *dir, uint64_t tid);
   void check_export_size(CDir *dir, export_state_t& stat, set<client_t> &client_set);
-  void handle_export_prep_ack(MExportDirPrepAck *m);
+  void handle_export_prep_ack(const MExportDirPrepAck::const_ref &m);
   void export_sessions_flushed(CDir *dir, uint64_t tid);
   void export_go(CDir *dir);
   void export_go_synced(CDir *dir, uint64_t tid);
@@ -163,25 +162,27 @@ protected:
   void export_cancel_finish(CDir *dir);
   void export_reverse(CDir *dir, export_state_t& stat);
   void export_notify_abort(CDir *dir, export_state_t& stat, set<CDir*>& bounds);
-  void handle_export_ack(MExportDirAck *m);
+  void handle_export_ack(const MExportDirAck::const_ref &m);
   void export_logged_finish(CDir *dir);
-  void handle_export_notify_ack(MExportDirNotifyAck *m);
+  void handle_export_notify_ack(const MExportDirNotifyAck::const_ref &m);
   void export_finish(CDir *dir);
 
-  void handle_gather_caps(MGatherCaps *m);
+  void handle_gather_caps(const MGatherCaps::const_ref &m);
 
   friend class C_MDC_ExportFreeze;
   friend class C_MDS_ExportFinishLogged;
   friend class C_M_ExportGo;
   friend class C_M_ExportSessionsFlushed;
+  friend class C_MDS_ExportDiscover;
+  friend class C_MDS_ExportPrep;
   friend class MigratorContext;
   friend class MigratorLogContext;
 
   // importer
-  void handle_export_discover(MExportDirDiscover *m);
-  void handle_export_cancel(MExportDirCancel *m);
-  void handle_export_prep(MExportDirPrep *m);
-  void handle_export_dir(MExportDir *m);
+  void handle_export_discover(const MExportDirDiscover::const_ref &m, bool started=false);
+  void handle_export_cancel(const MExportDirCancel::const_ref &m);
+  void handle_export_prep(const MExportDirPrep::const_ref &m, bool did_assim=false);
+  void handle_export_dir(const MExportDir::const_ref &m);
 
   void import_reverse_discovering(dirfrag_t df);
   void import_reverse_discovered(dirfrag_t df, CInode *diri);
@@ -193,10 +194,10 @@ protected:
   void import_notify_finish(CDir *dir, set<CDir*>& bounds);
   void import_logged_start(dirfrag_t df, CDir *dir, mds_rank_t from,
                           map<client_t,pair<Session*,uint64_t> >& imported_session_map);
-  void handle_export_finish(MExportDirFinish *m);
+  void handle_export_finish(const MExportDirFinish::const_ref &m);
 
-  void handle_export_caps(MExportCaps *m);
-  void handle_export_caps_ack(MExportCapsAck *m);
+  void handle_export_caps(const MExportCaps::const_ref &m);
+  void handle_export_caps_ack(const MExportCapsAck::const_ref &m);
   void logged_import_caps(CInode *in,
                          mds_rank_t from,
                          map<client_t,pair<Session*,uint64_t> >& imported_session_map,
@@ -208,12 +209,12 @@ protected:
   friend class C_M_LoggedImportCaps;
 
   // bystander
-  void handle_export_notify(MExportDirNotify *m);
+  void handle_export_notify(const MExportDirNotify::const_ref &m);
 
 
 public:
 
-  void dispatch(Message*);
+  void dispatch(const Message::const_ref &);
 
   void show_importing();
   void show_exporting();
index ba0790ed42d657d6e43429ec43c3ebcfce0d22ec..0e84ebc3e95dcbfe03bef4a5c4850966c42928be 100644 (file)
 #include "ScatterLock.h"
 #include "CDir.h"
 
-#include "messages/MClientRequest.h"
-#include "messages/MMDSSlaveRequest.h"
-
-
 // MutationImpl
 
 void MutationImpl::pin(MDSCacheObject *o)
@@ -182,10 +178,6 @@ void MutationImpl::_dump_op_descriptor_unlocked(ostream& stream) const
 
 MDRequestImpl::~MDRequestImpl()
 {
-  if (client_request)
-    client_request->put();
-  if (slave_request)
-    slave_request->put();
   delete _more;
 }
 
index 17b916abf9c87d8535f733e737403f56432e31ab..c69af3d1891ec3cc04ad60d57fcc3c7bba9599bc 100644 (file)
@@ -26,6 +26,8 @@
 #include "Capability.h"
 
 #include "common/TrackedOp.h"
+#include "messages/MClientRequest.h"
+#include "messages/MMDSSlaveRequest.h"
 
 class LogSegment;
 class Capability;
@@ -34,8 +36,6 @@ class CDir;
 class CDentry;
 class Session;
 class ScatterLock;
-class MClientRequest;
-class MMDSSlaveRequest;
 struct sr_t;
 
 struct MutationImpl : public TrackedOp {
@@ -180,7 +180,7 @@ struct MDRequestImpl : public MutationImpl {
   elist<MDRequestImpl*>::item item_session_request;  // if not on list, op is aborted.
 
   // -- i am a client (master) request
-  MClientRequest *client_request; // client request (if any)
+  MClientRequest::const_ref client_request; // client request (if any)
 
   // store up to two sets of dn vectors, inode pointers, for request path1 and path2.
   vector<CDentry*> dn[2];
@@ -207,7 +207,7 @@ struct MDRequestImpl : public MutationImpl {
   map<vinodeno_t, ceph_seq_t> cap_releases;  
 
   // -- i am a slave request
-  MMDSSlaveRequest *slave_request; // slave request (if one is pending; implies slave == true)
+  MMDSSlaveRequest::const_ref slave_request; // slave request (if one is pending; implies slave == true)
 
   // -- i am an internal op
   int internal_op;
@@ -290,15 +290,14 @@ struct MDRequestImpl : public MutationImpl {
   struct Params {
     metareqid_t reqid;
     __u32 attempt;
-    MClientRequest *client_req;
-    class Message *triggering_slave_req;
+    MClientRequest::const_ref client_req;
+    Message::const_ref triggering_slave_req;
     mds_rank_t slave_to;
     utime_t initiated;
     utime_t throttled, all_read, dispatched;
     int internal_op;
     // keep these default values synced to MutationImpl's
-    Params() : attempt(0), client_req(NULL),
-        triggering_slave_req(NULL), slave_to(MDS_RANK_NONE), internal_op(-1) {}
+    Params() : attempt(0), slave_to(MDS_RANK_NONE), internal_op(-1) {}
     const utime_t& get_recv_stamp() const {
       return initiated;
     }
@@ -318,7 +317,7 @@ struct MDRequestImpl : public MutationImpl {
     session(NULL), item_session_request(this),
     client_request(params->client_req), straydn(NULL), snapid(CEPH_NOSNAP),
     tracei(NULL), tracedn(NULL), alloc_ino(0), used_prealloc_ino(0),
-    slave_request(NULL), internal_op(params->internal_op), internal_op_finish(NULL),
+    internal_op(params->internal_op), internal_op_finish(NULL),
     internal_op_private(NULL),
     retry(0),
     waited_for_osdmap(false), _more(NULL) {
index 38708165542173d21a316f816a2122dbbd417894..6e4b83727f81494a87dae52dff31b5d60d23b76a 100644 (file)
 
 #include "osdc/Objecter.h"
 
-#include "messages/MClientSession.h"
-#include "messages/MClientRequest.h"
-#include "messages/MClientReply.h"
-#include "messages/MClientReconnect.h"
-#include "messages/MClientCaps.h"
-#include "messages/MClientSnap.h"
-
-#include "messages/MMDSSlaveRequest.h"
-
-#include "messages/MLock.h"
-
 #include "events/EUpdate.h"
 #include "events/ESlaveUpdate.h"
 #include "events/ESession.h"
@@ -191,25 +180,22 @@ Server::Server(MDSRank *m) :
   supported_features = feature_bitset_t(CEPHFS_FEATURES_MDS_SUPPORTED);
 }
 
-
-/* This function DOES put the passed message before returning*/
-void Server::dispatch(Message *m) 
+void Server::dispatch(const Message::const_ref &m)
 {
   switch (m->get_type()) {
   case CEPH_MSG_CLIENT_RECONNECT:
-    handle_client_reconnect(static_cast<MClientReconnect*>(m));
+    handle_client_reconnect(boost::static_pointer_cast<MClientReconnect::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     return;
   }
 
   // active?
   // handle_slave_request()/handle_client_session() will wait if necessary
   if (m->get_type() == CEPH_MSG_CLIENT_REQUEST && !mds->is_active()) {
-    MClientRequest *req = static_cast<MClientRequest*>(m);
+    const auto &req = boost::static_pointer_cast<MClientRequest::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m);
     if (mds->is_reconnect() || mds->get_want_state() == CEPH_MDS_STATE_RECONNECT) {
       Session *session = mds->get_session(req);
       if (!session || session->is_closed()) {
        dout(5) << "session is closed, dropping " << req->get_reqid() << dendl;
-       req->put();
        return;
       }
       bool queue_replay = false;
@@ -256,13 +242,13 @@ void Server::dispatch(Message *m)
 
   switch (m->get_type()) {
   case CEPH_MSG_CLIENT_SESSION:
-    handle_client_session(static_cast<MClientSession*>(m));
+    handle_client_session(boost::static_pointer_cast<MClientSession::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     return;
   case CEPH_MSG_CLIENT_REQUEST:
-    handle_client_request(static_cast<MClientRequest*>(m));
+    handle_client_request(boost::static_pointer_cast<MClientRequest::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     return;
   case MSG_MDS_SLAVE_REQUEST:
-    handle_slave_request(static_cast<MMDSSlaveRequest*>(m));
+    handle_slave_request(boost::static_pointer_cast<MMDSSlaveRequest::const_ref::element_type, std::remove_reference<decltype(m)>::type::element_type>(m));
     return;
   default:
     derr << "server unknown message " << m->get_type() << dendl;
@@ -297,8 +283,7 @@ public:
   }
 };
 
-/* This function DOES put the passed message before returning*/
-void Server::handle_client_session(MClientSession *m)
+void Server::handle_client_session(const MClientSession::const_ref &m)
 {
   version_t pv;
   Session *session = mds->get_session(m);
@@ -308,7 +293,6 @@ void Server::handle_client_session(MClientSession *m)
 
   if (!session) {
     dout(0) << " ignoring sessionless msg " << *m << dendl;
-    m->put();
     return;
   }
 
@@ -339,7 +323,6 @@ void Server::handle_client_session(MClientSession *m)
        session->is_killing() ||
        terminating_sessions) {
       dout(10) << "currently open|opening|stale|killing, dropping this req" << dendl;
-      m->put();
       return;
     }
     assert(session->is_closed() ||
@@ -347,7 +330,6 @@ void Server::handle_client_session(MClientSession *m)
 
     if (mds->is_stopping()) {
       dout(10) << "mds is stopping, dropping open req" << dendl;
-      m->put();
       return;
     }
 
@@ -371,8 +353,7 @@ void Server::handle_client_session(MClientSession *m)
        break;
       }
 
-      client_metadata_t client_metadata(std::move(m->metadata),
-                                       std::move(m->supported_features));
+      client_metadata_t client_metadata(m->metadata, m->supported_features);
       if (client_metadata.features.empty())
        infer_supported_features(session, client_metadata);
 
@@ -458,12 +439,10 @@ void Server::handle_client_session(MClientSession *m)
          session->is_closing() ||
          session->is_killing()) {
        dout(10) << "already closed|closing|killing, dropping this req" << dendl;
-       m->put();
        return;
       }
       if (session->is_importing()) {
        dout(10) << "ignoring close req on importing session" << dendl;
-       m->put();
        return;
       }
       assert(session->is_open() || 
@@ -472,7 +451,6 @@ void Server::handle_client_session(MClientSession *m)
       if (m->get_seq() < session->get_push_seq()) {
        dout(10) << "old push seq " << m->get_seq() << " < " << session->get_push_seq() 
                 << ", dropping" << dendl;
-       m->put();
        return;
       }
       // We are getting a seq that is higher than expected.
@@ -483,7 +461,6 @@ void Server::handle_client_session(MClientSession *m)
                << ", BUGGY!" << dendl;
        mds->clog->warn() << "incorrect push seq " << m->get_seq() << " != "
                          << session->get_push_seq() << ", dropping" << " from client : " << session->get_human_name();
-       m->put();
        return;
       }
       journal_close_session(session, Session::STATE_CLOSING, NULL);
@@ -502,7 +479,6 @@ void Server::handle_client_session(MClientSession *m)
   default:
     ceph_abort();
   }
-  m->put();
 }
 
 void Server::flush_client_sessions(set<client_t>& client_set, MDSGatherBuilder& gather)
@@ -950,8 +926,7 @@ void Server::reconnect_clients(MDSInternalContext *reconnect_done_)
   mds->sessionmap.dump();
 }
 
-/* This function DOES put the passed message before returning*/
-void Server::handle_client_reconnect(MClientReconnect *m)
+void Server::handle_client_reconnect(const MClientReconnect::const_ref &m)
 {
   dout(7) << "handle_client_reconnect " << m->get_source() << dendl;
   client_t from = m->get_source().num();
@@ -1006,7 +981,6 @@ void Server::handle_client_reconnect(MClientReconnect *m)
 
   if (deny) {
     m->get_connection()->send_message(new MClientSession(CEPH_SESSION_CLOSE));
-    m->put();
     if (session->is_open())
       kill_session(session, nullptr);
     return;
@@ -1029,10 +1003,8 @@ void Server::handle_client_reconnect(MClientReconnect *m)
   mds->clog->debug() << "reconnect by " << session->info.inst << " after " << delay;
   
   // snaprealms
-  for (vector<ceph_mds_snaprealm_reconnect>::iterator p = m->realms.begin();
-       p != m->realms.end();
-       ++p) {
-    CInode *in = mdcache->get_inode(inodeno_t(p->ino));
+  for (const auto &r : m->realms) {
+    CInode *in = mdcache->get_inode(inodeno_t(r.ino));
     if (in && in->state_test(CInode::STATE_PURGING))
       continue;
     if (in) {
@@ -1042,32 +1014,30 @@ void Server::handle_client_reconnect(MClientReconnect *m)
        // this can happen if we are non-auth or we rollback snaprealm
        dout(15) << "open snaprealm (null snaprealm) on " << *in << dendl;
       }
-      mdcache->add_reconnected_snaprealm(from, inodeno_t(p->ino), snapid_t(p->seq));
+      mdcache->add_reconnected_snaprealm(from, inodeno_t(r.ino), snapid_t(r.seq));
     } else {
-      dout(15) << "open snaprealm (w/o inode) on " << inodeno_t(p->ino)
-              << " seq " << p->seq << dendl;
-      mdcache->add_reconnected_snaprealm(from, inodeno_t(p->ino), snapid_t(p->seq));
+      dout(15) << "open snaprealm (w/o inode) on " << inodeno_t(r.ino)
+              << " seq " << r.seq << dendl;
+      mdcache->add_reconnected_snaprealm(from, inodeno_t(r.ino), snapid_t(r.seq));
     }
   }
 
   // caps
-  for (map<inodeno_t, cap_reconnect_t>::iterator p = m->caps.begin();
-       p != m->caps.end();
-       ++p) {
+  for (const auto &p : m->caps) {
     // make sure our last_cap_id is MAX over all issued caps
-    if (p->second.capinfo.cap_id > mdcache->last_cap_id)
-      mdcache->last_cap_id = p->second.capinfo.cap_id;
+    if (p.second.capinfo.cap_id > mdcache->last_cap_id)
+      mdcache->last_cap_id = p.second.capinfo.cap_id;
     
-    CInode *in = mdcache->get_inode(p->first);
+    CInode *in = mdcache->get_inode(p.first);
     if (in && in->state_test(CInode::STATE_PURGING))
       continue;
     if (in && in->is_auth()) {
       // we recovered it, and it's ours.  take note.
-      dout(15) << "open cap realm " << inodeno_t(p->second.capinfo.snaprealm)
+      dout(15) << "open cap realm " << inodeno_t(p.second.capinfo.snaprealm)
               << " on " << *in << dendl;
-      in->reconnect_cap(from, p->second, session);
-      mdcache->add_reconnected_cap(from, p->first, p->second);
-      recover_filelocks(in, p->second.flockbl, m->get_orig_source().num());
+      in->reconnect_cap(from, p.second, session);
+      mdcache->add_reconnected_cap(from, p.first, p.second);
+      recover_filelocks(in, p.second.flockbl, m->get_orig_source().num());
       continue;
     }
       
@@ -1075,14 +1045,12 @@ void Server::handle_client_reconnect(MClientReconnect *m)
       // not mine.
       dout(10) << "non-auth " << *in << ", will pass off to authority" << dendl;
       // add to cap export list.
-      p->second.path.clear(); // we don't need path
-      mdcache->rejoin_export_caps(p->first, from, p->second,
-                                 in->authority().first);
+      mdcache->rejoin_export_caps(p.first, from, p.second,
+                                 in->authority().first, true);
     } else {
       // don't know if the inode is mine
-      dout(10) << "missing ino " << p->first << ", will load later" << dendl;
-      p->second.path.clear(); // we don't need path
-      mdcache->rejoin_recovered_caps(p->first, from, p->second, MDS_RANK_NONE);
+      dout(10) << "missing ino " << p.first << ", will load later" << dendl;
+      mdcache->rejoin_recovered_caps(p.first, from, p.second, MDS_RANK_NONE);
     }
   }
   mdcache->rejoin_recovered_client(session->get_client(), session->info.inst);
@@ -1091,8 +1059,6 @@ void Server::handle_client_reconnect(MClientReconnect *m)
   client_reconnect_gather.erase(from);
   if (client_reconnect_gather.empty())
     reconnect_gather_finish();
-
-  m->put();
 }
 
 void Server::infer_supported_features(Session *session, client_metadata_t& client_metadata)
@@ -1373,7 +1339,7 @@ void Server::submit_mdlog_entry(LogEvent *le, MDSLogContextBase *fin, MDRequestR
 void Server::respond_to_request(MDRequestRef& mdr, int r)
 {
   if (mdr->client_request) {
-    reply_client_request(mdr, new MClientReply(mdr->client_request, r));
+    reply_client_request(mdr, MClientReply::ref(new MClientReply(*mdr->client_request, r), false));
   } else if (mdr->internal_op > -1) {
     dout(10) << "respond_to_request on internal request " << mdr << dendl;
     if (!mdr->internal_op_finish)
@@ -1384,7 +1350,7 @@ void Server::respond_to_request(MDRequestRef& mdr, int r)
 }
 
 // statistics mds req op number and latency 
-void Server::perf_gather_op_latency(const MClientRequestreq, utime_t lat)
+void Server::perf_gather_op_latency(const MClientRequest::const_ref &req, utime_t lat)
 {
   int code = l_mdss_first;
   switch(req->get_op()) {
@@ -1497,7 +1463,7 @@ void Server::early_reply(MDRequestRef& mdr, CInode *tracei, CDentry *tracedn)
     return;
   }
 
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   entity_inst_t client_inst = req->get_source_inst();
   if (client_inst.name.is_mds())
     return;
@@ -1508,7 +1474,7 @@ void Server::early_reply(MDRequestRef& mdr, CInode *tracei, CDentry *tracedn)
   }
 
 
-  MClientReply *reply = new MClientReply(req, 0);
+  MClientReply::ref reply(new MClientReply(*req, 0), false);
   reply->set_unsafe();
 
   // mark xlocks "done", indicating that we are exposing uncommitted changes.
@@ -1533,7 +1499,7 @@ void Server::early_reply(MDRequestRef& mdr, CInode *tracei, CDentry *tracedn)
   }
 
   reply->set_extra_bl(mdr->reply_extra_bl);
-  req->get_connection()->send_message(reply);
+  req->get_connection()->send_message2(reply);
 
   mdr->did_early_reply = true;
 
@@ -1551,10 +1517,10 @@ void Server::early_reply(MDRequestRef& mdr, CInode *tracei, CDentry *tracedn)
  * include a trace to tracei
  * Clean up mdr
  */
-void Server::reply_client_request(MDRequestRef& mdr, MClientReply *reply)
+void Server::reply_client_request(MDRequestRef& mdr, const MClientReply::ref &reply)
 {
   assert(mdr.get());
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   
   dout(7) << "reply_client_request " << reply->get_result()
           << " (" << cpp_strerror(reply->get_result())
@@ -1610,10 +1576,7 @@ void Server::reply_client_request(MDRequestRef& mdr, MClientReply *reply)
   mdcache->request_drop_non_rdlocks(mdr);
 
   // reply at all?
-  if (client_inst.name.is_mds() || !session) {
-    reply->put();   // mds doesn't need a reply
-    reply = 0;
-  } else {
+  if (!(client_inst.name.is_mds() || !session)) {
     // send reply.
     if (!did_early_reply &&   // don't issue leases if we sent an earlier reply already
        (tracei || tracedn)) {
@@ -1633,7 +1596,7 @@ void Server::reply_client_request(MDRequestRef& mdr, MClientReply *reply)
     reply->set_extra_bl(mdr->reply_extra_bl);
 
     reply->set_mdsmap_epoch(mds->mdsmap->get_epoch());
-    req->get_connection()->send_message(reply);
+    req->get_connection()->send_message2(reply);
   }
 
   if (req->is_queued_for_replay() &&
@@ -1663,7 +1626,7 @@ void Server::reply_client_request(MDRequestRef& mdr, MClientReply *reply)
  *
  * trace is in reverse order (i.e. root inode comes last)
  */
-void Server::set_trace_dist(Session *session, MClientReply *reply,
+void Server::set_trace_dist(Session *session, const MClientReply::ref &reply,
                            CInode *in, CDentry *dn,
                            snapid_t snapid,
                            int dentry_wanted,
@@ -1743,14 +1706,7 @@ void Server::set_trace_dist(Session *session, MClientReply *reply,
   reply->set_trace(bl);
 }
 
-
-
-
-/***
- * process a client request
- * This function DOES put the passed message before returning
- */
-void Server::handle_client_request(MClientRequest *req)
+void Server::handle_client_request(const MClientRequest::const_ref &req)
 {
   dout(4) << "handle_client_request " << *req << dendl;
 
@@ -1780,7 +1736,6 @@ void Server::handle_client_request(MClientRequest *req)
     if (!session) {
       if (req->is_queued_for_replay())
        mds->queue_one_replay();
-      req->put();
       return;
     }
   }
@@ -1805,18 +1760,17 @@ void Server::handle_client_request(MClientRequest *req)
           req->get_op() != CEPH_MDS_OP_OPEN &&
           req->get_op() != CEPH_MDS_OP_CREATE)) {
        dout(5) << "already completed " << req->get_reqid() << dendl;
-       MClientReply *reply = new MClientReply(req, 0);
+       MClientReply::ref reply(new MClientReply(*req, 0), false);
        if (created != inodeno_t()) {
          bufferlist extra;
          encode(created, extra);
          reply->set_extra_bl(extra);
        }
-       req->get_connection()->send_message(reply);
+       req->get_connection()->send_message2(reply);
 
        if (req->is_queued_for_replay())
          mds->queue_one_replay();
 
-       req->put();
        return;
       }
       if (req->get_op() != CEPH_MDS_OP_OPEN &&
@@ -1873,10 +1827,9 @@ void Server::handle_client_request(MClientRequest *req)
   //  (only if NOT replay!)
   if (!req->releases.empty() && req->get_source().is_client() && !req->is_replay()) {
     client_t client = req->get_source().num();
-    for (vector<MClientRequest::Release>::iterator p = req->releases.begin();
-        p != req->releases.end();
-        ++p)
-      mds->locker->process_request_cap_release(mdr, client, p->item, p->dname);
+    for (const auto &r : req->releases) {
+      mds->locker->process_request_cap_release(mdr, client, r.item, r.dname);
+    }
     req->releases.clear();
   }
 
@@ -1911,7 +1864,7 @@ void Server::dispatch_client_request(MDRequestRef& mdr)
     return;
   }
 
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
 
   if (logger) logger->inc(l_mdss_dispatch_client_request);
 
@@ -2062,8 +2015,7 @@ void Server::dispatch_client_request(MDRequestRef& mdr)
 // ---------------------------------------
 // SLAVE REQUESTS
 
-/* This function DOES put the passed message before returning*/
-void Server::handle_slave_request(MMDSSlaveRequest *m)
+void Server::handle_slave_request(const MMDSSlaveRequest::const_ref &m)
 {
   dout(4) << "handle_slave_request " << m->get_reqid() << " from " << m->get_source() << dendl;
   mds_rank_t from = mds_rank_t(m->get_source().num());
@@ -2077,10 +2029,8 @@ void Server::handle_slave_request(MMDSSlaveRequest *m)
   // the purpose of rename notify is enforcing causal message ordering. making sure
   // bystanders have received all messages from rename srcdn's auth MDS.
   if (m->get_op() == MMDSSlaveRequest::OP_RENAMENOTIFY) {
-    MMDSSlaveRequest *reply = new MMDSSlaveRequest(m->get_reqid(), m->get_attempt(),
-                                                  MMDSSlaveRequest::OP_RENAMENOTIFYACK);
+    MMDSSlaveRequest::ref reply(new MMDSSlaveRequest(m->get_reqid(), m->get_attempt(), MMDSSlaveRequest::OP_RENAMENOTIFYACK), false);
     mds->send_message(reply, m->get_connection());
-    m->put();
     return;
   }
 
@@ -2101,7 +2051,6 @@ void Server::handle_slave_request(MMDSSlaveRequest *m)
     if (mdr->attempt > m->get_attempt()) {
       dout(10) << "local request " << *mdr << " attempt " << mdr->attempt << " > " << m->get_attempt()
               << ", dropping " << *m << dendl;
-      m->put();
       return;
     }
 
@@ -2114,7 +2063,6 @@ void Server::handle_slave_request(MMDSSlaveRequest *m)
       mdr.reset();
     } else if (mdr->slave_to_mds != from) {
       dout(10) << "local request " << *mdr << " not slave to mds." << from << dendl;
-      m->put();
       return;
     }
 
@@ -2126,7 +2074,6 @@ void Server::handle_slave_request(MMDSSlaveRequest *m)
       } else {
        mdcache->request_finish(mdr);
       }
-      m->put();
       return;
     }
   }
@@ -2135,7 +2082,6 @@ void Server::handle_slave_request(MMDSSlaveRequest *m)
     if (m->get_op() == MMDSSlaveRequest::OP_FINISH) {
       dout(10) << "missing slave request for " << m->get_reqid() 
               << " OP_FINISH, must have lost race with a forward" << dendl;
-      m->put();
       return;
     }
     mdr = mdcache->request_start_slave(m->get_reqid(), m->get_attempt(), m);
@@ -2164,8 +2110,7 @@ void Server::handle_slave_request(MMDSSlaveRequest *m)
   dispatch_slave_request(mdr);
 }
 
-/* This function DOES put the passed message before returning*/
-void Server::handle_slave_request_reply(MMDSSlaveRequest *m)
+void Server::handle_slave_request_reply(const MMDSSlaveRequest::const_ref &m)
 {
   mds_rank_t from = mds_rank_t(m->get_source().num());
   
@@ -2174,7 +2119,6 @@ void Server::handle_slave_request_reply(MMDSSlaveRequest *m)
     if (!mdcache->have_uncommitted_master(r, from)) {
       dout(10) << "handle_slave_request_reply ignoring slave reply from mds."
               << from << " reqid " << r << dendl;
-      m->put();
       return;
     }
     dout(3) << "not clientreplay|active yet, waiting" << dendl;
@@ -2185,7 +2129,6 @@ void Server::handle_slave_request_reply(MMDSSlaveRequest *m)
   if (m->get_op() == MMDSSlaveRequest::OP_COMMITTED) {
     metareqid_t r = m->get_reqid();
     mdcache->committed_master_slave(r, from);
-    m->put();
     return;
   }
 
@@ -2193,7 +2136,6 @@ void Server::handle_slave_request_reply(MMDSSlaveRequest *m)
   if (m->get_attempt() != mdr->attempt) {
     dout(10) << "handle_slave_request_reply " << *mdr << " ignoring reply from other attempt "
             << m->get_attempt() << dendl;
-    m->put();
     return;
   }
 
@@ -2259,9 +2201,6 @@ void Server::handle_slave_request_reply(MMDSSlaveRequest *m)
   default:
     ceph_abort();
   }
-  
-  // done with reply.
-  m->put();
 }
 
 /* This function DOES put the mdr->slave_request before returning*/
@@ -2315,7 +2254,7 @@ void Server::dispatch_slave_request(MDRequestRef& mdr)
          return;
        
        // ack
-       MMDSSlaveRequest *r = new MMDSSlaveRequest(mdr->reqid, mdr->attempt, replycode);
+       MMDSSlaveRequest::ref r(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, replycode), false);
        r->set_lock_type(lock->get_type());
        lock->get_parent()->set_object_info(r->get_object_info());
        if (replycode == MMDSSlaveRequest::OP_XLOCKACK)
@@ -2324,7 +2263,6 @@ void Server::dispatch_slave_request(MDRequestRef& mdr)
       }
 
       // done.
-      mdr->slave_request->put();
       mdr->slave_request = 0;
     }
     break;
@@ -2348,14 +2286,12 @@ void Server::dispatch_slave_request(MDRequestRef& mdr)
        mds->locker->issue_caps(static_cast<CInode*>(lock->get_parent()));
 
       // done.  no ack necessary.
-      mdr->slave_request->put();
       mdr->slave_request = 0;
     }
     break;
 
   case MMDSSlaveRequest::OP_DROPLOCKS:
     mds->locker->drop_locks(mdr.get());
-    mdr->slave_request->put();
     mdr->slave_request = 0;
     break;
 
@@ -2379,7 +2315,7 @@ void Server::dispatch_slave_request(MDRequestRef& mdr)
   case MMDSSlaveRequest::OP_FINISH:
     // information about rename imported caps
     if (mdr->slave_request->inode_export.length() > 0)
-      mdr->more()->inode_import.claim(mdr->slave_request->inode_export);
+      mdr->more()->inode_import = mdr->slave_request->inode_export;
     // finish off request.
     mdcache->request_finish(mdr);
     break;
@@ -2406,18 +2342,16 @@ void Server::handle_slave_auth_pin(MDRequestRef& mdr)
   }
 
   if (!fail) {
-    for (vector<MDSCacheObjectInfo>::iterator p = mdr->slave_request->get_authpins().begin();
-       p != mdr->slave_request->get_authpins().end();
-       ++p) {
-      MDSCacheObject *object = mdcache->get_object(*p);
+    for (const auto &oi : mdr->slave_request->get_authpins()) {
+      MDSCacheObject *object = mdcache->get_object(oi);
       if (!object) {
-       dout(10) << " don't have " << *p << dendl;
+       dout(10) << " don't have " << oi << dendl;
        fail = true;
        break;
       }
 
       objects.push_back(object);
-      if (*p == mdr->slave_request->get_authpin_freeze())
+      if (oi == mdr->slave_request->get_authpin_freeze())
        auth_pin_freeze = static_cast<CInode*>(object);
     }
   }
@@ -2486,7 +2420,7 @@ void Server::handle_slave_auth_pin(MDRequestRef& mdr)
   }
 
   // ack!
-  MMDSSlaveRequest *reply = new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_AUTHPINACK);
+  MMDSSlaveRequest::ref reply(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_AUTHPINACK), false);
   
   // return list of my auth_pins (if any)
   for (set<MDSCacheObject*>::iterator p = mdr->auth_pins.begin();
@@ -2507,28 +2441,24 @@ void Server::handle_slave_auth_pin(MDRequestRef& mdr)
   mds->send_message_mds(reply, mdr->slave_to_mds);
   
   // clean up this request
-  mdr->slave_request->put();
   mdr->slave_request = 0;
   return;
 }
 
-/* This function DOES NOT put the passed ack before returning*/
-void Server::handle_slave_auth_pin_ack(MDRequestRef& mdr, MMDSSlaveRequest *ack)
+void Server::handle_slave_auth_pin_ack(MDRequestRef& mdr, const MMDSSlaveRequest::const_ref &ack)
 {
   dout(10) << "handle_slave_auth_pin_ack on " << *mdr << " " << *ack << dendl;
   mds_rank_t from = mds_rank_t(ack->get_source().num());
 
   // added auth pins?
   set<MDSCacheObject*> pinned;
-  for (vector<MDSCacheObjectInfo>::iterator p = ack->get_authpins().begin();
-       p != ack->get_authpins().end();
-       ++p) {
-    MDSCacheObject *object = mdcache->get_object(*p);
+  for (const auto &oi : ack->get_authpins()) {
+    MDSCacheObject *object = mdcache->get_object(oi);
     assert(object);  // we pinned it
     dout(10) << " remote has pinned " << *object << dendl;
     if (!mdr->is_auth_pinned(object))
       mdr->remote_auth_pins[object] = from;
-    if (*p == ack->get_authpin_freeze())
+    if (oi == ack->get_authpin_freeze())
       mdr->set_remote_frozen_auth_pin(static_cast<CInode *>(object));
     pinned.insert(object);
   }
@@ -2818,7 +2748,7 @@ CInode* Server::prepare_new_inode(MDRequestRef& mdr, CDir *dir, inodeno_t useino
 
   in->inode.change_attr = 0;
 
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   if (req->get_data().length()) {
     auto p = req->get_data().cbegin();
 
@@ -2890,6 +2820,17 @@ public:
   }
 };
 
+class CF_MDS_MDRContextFactory : public MDSContextFactory {
+public:
+  CF_MDS_MDRContextFactory(MDCache *cache, MDRequestRef &mdr) : cache(cache), mdr(mdr) {}
+  MDSInternalContextBase *build() {
+    return new C_MDS_RetryRequest(cache, mdr);
+  }
+private:
+  MDCache *cache;
+  MDRequestRef mdr;
+};
+
 CDir *Server::traverse_to_auth_dir(MDRequestRef& mdr, vector<CDentry*> &trace, filepath refpath)
 {
   // figure parent dir vs dname
@@ -2905,7 +2846,8 @@ CDir *Server::traverse_to_auth_dir(MDRequestRef& mdr, vector<CDentry*> &trace, f
 
   // traverse to parent dir
   CInode *diri;
-  int r = mdcache->path_traverse(mdr, NULL, NULL, refpath, &trace, &diri, MDS_TRAVERSE_FORWARD);
+  CF_MDS_MDRContextFactory cf(mdcache, mdr);
+  int r = mdcache->path_traverse(mdr, cf, refpath, &trace, &diri, MDS_TRAVERSE_FORWARD);
   if (r > 0) return 0; // delayed
   if (r < 0) {
     if (r == -ESTALE) {
@@ -2943,7 +2885,8 @@ CInode* Server::rdlock_path_pin_ref(MDRequestRef& mdr, int n,
     return mdr->in[n];
 
   // traverse
-  int r = mdcache->path_traverse(mdr, NULL, NULL, refpath, &mdr->dn[n], &mdr->in[n], MDS_TRAVERSE_FORWARD);
+  CF_MDS_MDRContextFactory cf(mdcache, mdr);
+  int r = mdcache->path_traverse(mdr, cf, refpath, &mdr->dn[n], &mdr->in[n], MDS_TRAVERSE_FORWARD);
   if (r > 0)
     return NULL; // delayed
   if (r < 0) {  // error
@@ -3165,7 +3108,7 @@ CDir* Server::try_open_auth_dirfrag(CInode *diri, frag_t fg, MDRequestRef& mdr)
 
 void Server::handle_client_getattr(MDRequestRef& mdr, bool is_lookup)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   set<SimpleLock*> rdlocks, wrlocks, xlocks;
 
   if (req->get_filepath().depth() == 0 && is_lookup) {
@@ -3259,7 +3202,7 @@ struct C_MDS_LookupIno2 : public ServerContext {
 void Server::handle_client_lookup_ino(MDRequestRef& mdr,
                                      bool want_parent, bool want_dentry)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
 
   if ((uint64_t)req->head.args.lookupino.snapid > 0)
     return _lookup_snap_ino(mdr);
@@ -3353,7 +3296,7 @@ void Server::handle_client_lookup_ino(MDRequestRef& mdr,
 
 void Server::_lookup_snap_ino(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
 
   vinodeno_t vino;
   vino.ino = req->get_filepath().get_ino();
@@ -3451,7 +3394,7 @@ void Server::_lookup_ino_2(MDRequestRef& mdr, int r)
 /* This function takes responsibility for the passed mdr*/
 void Server::handle_client_open(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   dout(7) << "open on " << req->get_filepath() << dendl;
 
   int flags = req->head.args.open.flags;
@@ -3673,7 +3616,7 @@ public:
 /* This function takes responsibility for the passed mdr*/
 void Server::handle_client_openc(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   client_t client = mdr->get_client();
 
   dout(7) << "open w/ O_CREAT on " << req->get_filepath() << dendl;
@@ -3687,7 +3630,8 @@ void Server::handle_client_openc(MDRequestRef& mdr)
   bool excl = req->head.args.open.flags & CEPH_O_EXCL;
 
   if (!excl) {
-    int r = mdcache->path_traverse(mdr, NULL, NULL, req->get_filepath(),
+    CF_MDS_MDRContextFactory cf(mdcache, mdr);
+    int r = mdcache->path_traverse(mdr, cf, req->get_filepath(),
                                   &mdr->dn[0], NULL, MDS_TRAVERSE_FORWARD);
     if (r > 0) return;
     if (r == 0) {
@@ -3854,7 +3798,7 @@ void Server::handle_client_openc(MDRequestRef& mdr)
 
 void Server::handle_client_readdir(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   client_t client = req->get_source().num();
   set<SimpleLock*> rdlocks, wrlocks, xlocks;
   CInode *diri = rdlock_path_pin_ref(mdr, 0, rdlocks, false, true);
@@ -4146,7 +4090,7 @@ public:
 
 void Server::handle_client_file_setlock(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   set<SimpleLock*> rdlocks, wrlocks, xlocks;
 
   // get the inode to operate on, and set up any locks needed for that
@@ -4249,7 +4193,7 @@ void Server::handle_client_file_setlock(MDRequestRef& mdr)
 
 void Server::handle_client_file_readlock(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   set<SimpleLock*> rdlocks, wrlocks, xlocks;
 
   // get the inode to operate on, and set up any locks needed for that
@@ -4302,7 +4246,7 @@ void Server::handle_client_file_readlock(MDRequestRef& mdr)
 
 void Server::handle_client_setattr(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   set<SimpleLock*> rdlocks, wrlocks, xlocks;
   CInode *cur = rdlock_path_pin_ref(mdr, 0, rdlocks, true);
   if (!cur) return;
@@ -4497,7 +4441,7 @@ void Server::do_open_truncate(MDRequestRef& mdr, int cmode)
 /* This function cleans up the passed mdr */
 void Server::handle_client_setlayout(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   set<SimpleLock*> rdlocks, wrlocks, xlocks;
   CInode *cur = rdlock_path_pin_ref(mdr, 0, rdlocks, true);
   if (!cur) return;
@@ -4584,7 +4528,7 @@ void Server::handle_client_setlayout(MDRequestRef& mdr)
 
 void Server::handle_client_setdirlayout(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   set<SimpleLock*> rdlocks, wrlocks, xlocks;
   file_layout_t *dir_layout = NULL;
   CInode *cur = rdlock_path_pin_ref(mdr, 0, rdlocks, true, false, &dir_layout);
@@ -4789,7 +4733,7 @@ void Server::create_quota_realm(CInode *in)
 {
   dout(10) << __func__ << " " << *in << dendl;
 
-  MClientRequest *req = new MClientRequest(CEPH_MDS_OP_SETXATTR);
+  MClientRequest::ref req(new MClientRequest(CEPH_MDS_OP_SETXATTR), false);
   req->set_filepath(filepath(in->ino()));
   req->set_string2("ceph.quota");
   // empty vxattr value
@@ -4809,7 +4753,7 @@ int Server::check_layout_vxattr(MDRequestRef& mdr,
                                 string value,
                                 file_layout_t *layout)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   epoch_t epoch;
   int r;
 
@@ -4873,7 +4817,7 @@ void Server::handle_set_vxattr(MDRequestRef& mdr, CInode *cur,
                               set<SimpleLock*> wrlocks,
                               set<SimpleLock*> xlocks)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   string name(req->get_path2());
   bufferlist bl = req->get_data();
   string value (bl.c_str(), bl.length());
@@ -5033,7 +4977,7 @@ void Server::handle_remove_vxattr(MDRequestRef& mdr, CInode *cur,
                                  set<SimpleLock*> wrlocks,
                                  set<SimpleLock*> xlocks)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   string name(req->get_path2());
 
   dout(10) << __func__ << " " << name << " on " << *cur << dendl;
@@ -5108,7 +5052,7 @@ public:
 
 void Server::handle_client_setxattr(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   string name(req->get_path2());
   set<SimpleLock*> rdlocks, wrlocks, xlocks;
   CInode *cur;
@@ -5205,7 +5149,7 @@ void Server::handle_client_setxattr(MDRequestRef& mdr)
 
 void Server::handle_client_removexattr(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   std::string name(req->get_path2());
   std::set<SimpleLock*> rdlocks, wrlocks, xlocks;
   file_layout_t *dir_layout = NULL;
@@ -5316,7 +5260,7 @@ public:
 
 void Server::handle_client_mknod(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   client_t client = mdr->get_client();
   set<SimpleLock*> rdlocks, wrlocks, xlocks;
   file_layout_t *dir_layout = NULL;
@@ -5411,7 +5355,7 @@ void Server::handle_client_mknod(MDRequestRef& mdr)
 /* This function takes responsibility for the passed mdr*/
 void Server::handle_client_mkdir(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   set<SimpleLock*> rdlocks, wrlocks, xlocks;
   CDentry *dn = rdlock_path_xlock_dentry(mdr, 0, rdlocks, wrlocks, xlocks, false, false, false);
   if (!dn) return;
@@ -5493,7 +5437,7 @@ void Server::handle_client_mkdir(MDRequestRef& mdr)
 
 void Server::handle_client_symlink(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   set<SimpleLock*> rdlocks, wrlocks, xlocks;
   CDentry *dn = rdlock_path_xlock_dentry(mdr, 0, rdlocks, wrlocks, xlocks, false, false, false);
   if (!dn) return;
@@ -5549,7 +5493,7 @@ void Server::handle_client_symlink(MDRequestRef& mdr)
 
 void Server::handle_client_link(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
 
   dout(7) << "handle_client_link " << req->get_filepath()
          << " to " << req->get_filepath2()
@@ -5743,7 +5687,7 @@ void Server::_link_remote(MDRequestRef& mdr, bool inc, CDentry *dn, CInode *targ
       op = MMDSSlaveRequest::OP_LINKPREP;
     else 
       op = MMDSSlaveRequest::OP_UNLINKPREP;
-    MMDSSlaveRequest *req = new MMDSSlaveRequest(mdr->reqid, mdr->attempt, op);
+    MMDSSlaveRequest::ref req(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, op), false);
     targeti->set_object_info(req->get_object_info());
     req->op_stamp = mdr->get_op_stamp();
     if (auto& desti_srnode = mdr->more()->desti_srnode)
@@ -5981,7 +5925,6 @@ void Server::_logged_slave_link(MDRequestRef& mdr, CInode *targeti, bool adjust_
   mds->balancer->hit_inode(targeti, META_POP_IWR);
 
   // done.
-  mdr->slave_request->put();
   mdr->slave_request = 0;
 
   if (adjust_realm) {
@@ -5992,8 +5935,7 @@ void Server::_logged_slave_link(MDRequestRef& mdr, CInode *targeti, bool adjust_
 
   // ack
   if (!mdr->aborted) {
-    MMDSSlaveRequest *reply = new MMDSSlaveRequest(mdr->reqid, mdr->attempt,
-                                                  MMDSSlaveRequest::OP_LINKPREPACK);
+    MMDSSlaveRequest::ref reply(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_LINKPREPACK));
     mds->send_message_mds(reply, mdr->slave_to_mds);
   } else {
     dout(10) << " abort flag set, finishing" << dendl;
@@ -6038,19 +5980,17 @@ void Server::_committed_slave(MDRequestRef& mdr)
 
   assert(g_conf()->mds_kill_link_at != 8);
 
-  MMDSSlaveRequest *req = new MMDSSlaveRequest(mdr->reqid, mdr->attempt, 
-                                              MMDSSlaveRequest::OP_COMMITTED);
+  MMDSSlaveRequest::ref req(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_COMMITTED), false);
   mds->send_message_mds(req, mdr->slave_to_mds);
   mdcache->request_finish(mdr);
 }
 
 struct C_MDS_LoggedLinkRollback : public ServerLogContext {
   MutationRef mut;
-  map<client_t,MClientSnap*> splits;
+  map<client_t,MClientSnap::ref> splits;
   C_MDS_LoggedLinkRollback(Server *s, MutationRef& m, MDRequestRef& r,
-                          map<client_t,MClientSnap*>& _splits) :
-    ServerLogContext(s, r), mut(m) {
-    splits.swap(_splits);
+                          map<client_t,MClientSnap::ref>&& _splits) :
+    ServerLogContext(s, r), mut(m), splits(std::move(_splits)) {
   }
   void finish(int r) override {
     server->_link_rollback_finish(mut, mdr, splits);
@@ -6105,7 +6045,7 @@ void Server::do_link_rollback(bufferlist &rbl, mds_rank_t master, MDRequestRef&
   else
     pi.inode.nlink++;
 
-  map<client_t,MClientSnap*> splits;
+  map<client_t,MClientSnap::ref> splits;
   if (rollback.snapbl.length() && in->snaprealm) {
     bool hadrealm;
     auto p = rollback.snapbl.cbegin();
@@ -6134,13 +6074,13 @@ void Server::do_link_rollback(bufferlist &rbl, mds_rank_t master, MDRequestRef&
   le->commit.add_dir(parent, true);
   le->commit.add_primary_dentry(in->get_projected_parent_dn(), 0, true);
   
-  submit_mdlog_entry(le, new C_MDS_LoggedLinkRollback(this, mut, mdr, splits),
+  submit_mdlog_entry(le, new C_MDS_LoggedLinkRollback(this, mut, mdr, std::move(splits)),
                      mdr, __func__);
   mdlog->flush();
 }
 
 void Server::_link_rollback_finish(MutationRef& mut, MDRequestRef& mdr,
-                                  map<client_t,MClientSnap*>& splits)
+                                  map<client_t,MClientSnap::ref>& splits)
 {
   dout(10) << "_link_rollback_finish" << dendl;
 
@@ -6160,8 +6100,7 @@ void Server::_link_rollback_finish(MutationRef& mut, MDRequestRef& mdr,
 }
 
 
-/* This function DOES NOT put the passed message before returning*/
-void Server::handle_slave_link_prep_ack(MDRequestRef& mdr, MMDSSlaveRequest *m)
+void Server::handle_slave_link_prep_ack(MDRequestRef& mdr, const MMDSSlaveRequest::const_ref &m)
 {
   dout(10) << "handle_slave_link_prep_ack " << *mdr 
           << " " << *m << dendl;
@@ -6195,7 +6134,7 @@ void Server::handle_slave_link_prep_ack(MDRequestRef& mdr, MMDSSlaveRequest *m)
 
 void Server::handle_client_unlink(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   client_t client = mdr->get_client();
 
   // rmdir or unlink?
@@ -6210,7 +6149,8 @@ void Server::handle_client_unlink(MDRequestRef& mdr)
   // traverse to path
   vector<CDentry*> trace;
   CInode *in;
-  int r = mdcache->path_traverse(mdr, NULL, NULL, req->get_filepath(), &trace, &in, MDS_TRAVERSE_FORWARD);
+  CF_MDS_MDRContextFactory cf(mdcache, mdr);
+  int r = mdcache->path_traverse(mdr, cf, req->get_filepath(), &trace, &in, MDS_TRAVERSE_FORWARD);
   if (r > 0) return;
   if (r < 0) {
     if (r == -ESTALE) {
@@ -6551,8 +6491,7 @@ bool Server::_rmdir_prepare_witness(MDRequestRef& mdr, mds_rank_t who, vector<CD
   }
   
   dout(10) << "_rmdir_prepare_witness mds." << who << dendl;
-  MMDSSlaveRequest *req = new MMDSSlaveRequest(mdr->reqid, mdr->attempt,
-                                              MMDSSlaveRequest::OP_RMDIRPREP);
+  MMDSSlaveRequest::ref req(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RMDIRPREP), false);
   req->srcdnpath = filepath(trace.front()->get_dir()->ino());
   for (auto dn : trace)
     req->srcdnpath.push_dentry(dn->get_name());
@@ -6598,7 +6537,8 @@ void Server::handle_slave_rmdir_prep(MDRequestRef& mdr)
   filepath srcpath(mdr->slave_request->srcdnpath);
   dout(10) << " src " << srcpath << dendl;
   CInode *in;
-  int r = mdcache->path_traverse(mdr, NULL, NULL, srcpath, &trace, &in, MDS_TRAVERSE_DISCOVERXLOCK);
+  CF_MDS_MDRContextFactory cf(mdcache, mdr);
+  int r = mdcache->path_traverse(mdr, cf, srcpath, &trace, &in, MDS_TRAVERSE_DISCOVERXLOCK);
   if (r > 0) return;
   if (r == -ESTALE) {
     mdcache->find_ino_peers(srcpath.get_ino(), new C_MDS_RetryRequest(mdcache, mdr),
@@ -6696,13 +6636,11 @@ void Server::_logged_slave_rmdir(MDRequestRef& mdr, CDentry *dn, CDentry *strayd
       mdcache->do_realm_invalidate_and_update_notify(in, CEPH_SNAP_OP_SPLIT, false);
 
   // done.
-  mdr->slave_request->put();
   mdr->slave_request = 0;
   mdr->straydn = 0;
 
   if (!mdr->aborted) {
-    MMDSSlaveRequest *reply = new MMDSSlaveRequest(mdr->reqid, mdr->attempt,
-                                                  MMDSSlaveRequest::OP_RMDIRPREPACK);
+    MMDSSlaveRequest::ref reply(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RMDIRPREPACK), false);
     if (!mdr->more()->slave_update_journaled)
       reply->mark_not_journaled();
     mds->send_message_mds(reply, mdr->slave_to_mds);
@@ -6712,7 +6650,7 @@ void Server::_logged_slave_rmdir(MDRequestRef& mdr, CDentry *dn, CDentry *strayd
   }
 }
 
-void Server::handle_slave_rmdir_prep_ack(MDRequestRef& mdr, MMDSSlaveRequest *ack)
+void Server::handle_slave_rmdir_prep_ack(MDRequestRef& mdr, const MMDSSlaveRequest::const_ref &ack)
 {
   dout(10) << "handle_slave_rmdir_prep_ack " << *mdr 
           << " " << *ack << dendl;
@@ -6967,7 +6905,7 @@ public:
  */
 void Server::handle_client_rename(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   dout(7) << "handle_client_rename " << *req << dendl;
 
   filepath destpath = req->get_filepath();
@@ -6994,7 +6932,8 @@ void Server::handle_client_rename(MDRequestRef& mdr)
   CDir *destdir = destdn->get_dir();
   assert(destdir->is_auth());
 
-  int r = mdcache->path_traverse(mdr, NULL, NULL, srcpath, &srctrace, NULL, MDS_TRAVERSE_DISCOVER);
+  CF_MDS_MDRContextFactory cf(mdcache, mdr);
+  int r = mdcache->path_traverse(mdr, cf, srcpath, &srctrace, NULL, MDS_TRAVERSE_DISCOVER);
   if (r > 0)
     return; // delayed
   if (r < 0) {
@@ -7473,8 +7412,7 @@ bool Server::_rename_prepare_witness(MDRequestRef& mdr, mds_rank_t who, set<mds_
   }
 
   dout(10) << "_rename_prepare_witness mds." << who << dendl;
-  MMDSSlaveRequest *req = new MMDSSlaveRequest(mdr->reqid, mdr->attempt,
-                                              MMDSSlaveRequest::OP_RENAMEPREP);
+  MMDSSlaveRequest::ref req(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREP), false);
 
   req->srcdnpath = filepath(srctrace.front()->get_dir()->ino());
   for (auto dn : srctrace)
@@ -8143,10 +8081,9 @@ void Server::handle_slave_rename_prep(MDRequestRef& mdr)
 
   if (mdr->slave_request->is_interrupted()) {
     dout(10) << " slave request interrupted, sending noop reply" << dendl;
-    MMDSSlaveRequest *reply= new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREPACK);
+    MMDSSlaveRequest::ref reply(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREPACK), false);
     reply->mark_interrupted();
     mds->send_message_mds(reply, mdr->slave_to_mds);
-    mdr->slave_request->put();
     mdr->slave_request = 0;
     return;
   }
@@ -8155,7 +8092,8 @@ void Server::handle_slave_rename_prep(MDRequestRef& mdr)
   filepath destpath(mdr->slave_request->destdnpath);
   dout(10) << " dest " << destpath << dendl;
   vector<CDentry*> trace;
-  int r = mdcache->path_traverse(mdr, NULL, NULL, destpath, &trace, NULL, MDS_TRAVERSE_DISCOVERXLOCK);
+  CF_MDS_MDRContextFactory cf(mdcache, mdr);
+  int r = mdcache->path_traverse(mdr, cf, destpath, &trace, NULL, MDS_TRAVERSE_DISCOVERXLOCK);
   if (r > 0) return;
   if (r == -ESTALE) {
     mdcache->find_ino_peers(destpath.get_ino(), new C_MDS_RetryRequest(mdcache, mdr),
@@ -8173,7 +8111,7 @@ void Server::handle_slave_rename_prep(MDRequestRef& mdr)
   filepath srcpath(mdr->slave_request->srcdnpath);
   dout(10) << " src " << srcpath << dendl;
   CInode *srci = nullptr;
-  r = mdcache->path_traverse(mdr, NULL, NULL, srcpath, &trace, &srci, MDS_TRAVERSE_DISCOVERXLOCK);
+  r = mdcache->path_traverse(mdr, cf, srcpath, &trace, &srci, MDS_TRAVERSE_DISCOVERXLOCK);
   if (r > 0) return;
   assert(r == 0);
 
@@ -8247,8 +8185,7 @@ void Server::handle_slave_rename_prep(MDRequestRef& mdr)
            (mds->is_cluster_degraded() &&
             !mds->mdsmap->is_clientreplay_or_active_or_stopping(*p)))
          continue;
-       MMDSSlaveRequest *notify = new MMDSSlaveRequest(mdr->reqid, mdr->attempt,
-           MMDSSlaveRequest::OP_RENAMENOTIFY);
+       MMDSSlaveRequest::ref notify(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMENOTIFY), false);
        mds->send_message_mds(notify, *p);
        mdr->more()->waiting_on_slave.insert(*p);
       }
@@ -8277,11 +8214,9 @@ void Server::handle_slave_rename_prep(MDRequestRef& mdr)
 
     if (reply_witness) {
       assert(!srcdnrep.empty());
-      MMDSSlaveRequest *reply = new MMDSSlaveRequest(mdr->reqid, mdr->attempt,
-                                                    MMDSSlaveRequest::OP_RENAMEPREPACK);
+      MMDSSlaveRequest::ref reply(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREPACK), false);
       reply->witnesses.swap(srcdnrep);
       mds->send_message_mds(reply, mdr->slave_to_mds);
-      mdr->slave_request->put();
       mdr->slave_request = 0;
       return;  
     }
@@ -8380,9 +8315,9 @@ void Server::_logged_slave_rename(MDRequestRef& mdr,
   dout(10) << "_logged_slave_rename " << *mdr << dendl;
 
   // prepare ack
-  MMDSSlaveRequest *reply = NULL;
+  MMDSSlaveRequest::ref reply;
   if (!mdr->aborted) {
-    reply= new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREPACK);
+    reply.reset(new MMDSSlaveRequest(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREPACK), false);
     if (!mdr->more()->slave_update_journaled)
       reply->mark_not_journaled();
   }
@@ -8438,7 +8373,6 @@ void Server::_logged_slave_rename(MDRequestRef& mdr,
     mds->balancer->hit_inode(destdnl->get_inode(), META_POP_IWR);
 
   // done.
-  mdr->slave_request->put();
   mdr->slave_request = 0;
   mdr->straydn = 0;
 
@@ -8585,11 +8519,11 @@ struct C_MDS_LoggedRenameRollback : public ServerLogContext {
   version_t srcdnpv;
   CDentry *destdn;
   CDentry *straydn;
-  map<client_t,MClientSnap*> splits[2];
+  map<client_t,MClientSnap::ref> splits[2];
   bool finish_mdr;
   C_MDS_LoggedRenameRollback(Server *s, MutationRef& m, MDRequestRef& r,
                             CDentry *sd, version_t pv, CDentry *dd, CDentry *st,
-                            map<client_t,MClientSnap*> _splits[2], bool f) :
+                            map<client_t,MClientSnap::ref> _splits[2], bool f) :
     ServerLogContext(s, r), mut(m), srcdn(sd), srcdnpv(pv), destdn(dd),
     straydn(st), finish_mdr(f) {
       splits[0].swap(_splits[0]);
@@ -8702,7 +8636,7 @@ void Server::do_rename_rollback(bufferlist &rbl, mds_rank_t master, MDRequestRef
                                    rollback.orig_src.remote_d_type);
   }
 
-  map<client_t,MClientSnap*> splits[2];
+  map<client_t,MClientSnap::ref> splits[2];
 
   CInode::mempool_inode *pip = nullptr;
   if (in) {
@@ -8915,7 +8849,7 @@ void Server::do_rename_rollback(bufferlist &rbl, mds_rank_t master, MDRequestRef
 
 void Server::_rename_rollback_finish(MutationRef& mut, MDRequestRef& mdr, CDentry *srcdn,
                                     version_t srcdnpv, CDentry *destdn, CDentry *straydn,
-                                    map<client_t,MClientSnap*> splits[2], bool finish_mdr)
+                                    map<client_t,MClientSnap::ref> splits[2], bool finish_mdr)
 {
   dout(10) << "_rename_rollback_finish " << mut->reqid << dendl;
 
@@ -8989,8 +8923,7 @@ void Server::_rename_rollback_finish(MutationRef& mut, MDRequestRef& mdr, CDentr
   mut->cleanup();
 }
 
-/* This function DOES put the passed message before returning*/
-void Server::handle_slave_rename_prep_ack(MDRequestRef& mdr, MMDSSlaveRequest *ack)
+void Server::handle_slave_rename_prep_ack(MDRequestRef& mdr, const MMDSSlaveRequest::const_ref &ack)
 {
   dout(10) << "handle_slave_rename_prep_ack " << *mdr 
           << " witnessed by " << ack->get_source()
@@ -9015,14 +8948,14 @@ void Server::handle_slave_rename_prep_ack(MDRequestRef& mdr, MMDSSlaveRequest *a
       mdr->more()->has_journaled_slaves = true;
   } else {
     dout(10) << " extra witnesses (srcdn replicas) are " << ack->witnesses << dendl;
-    mdr->more()->extra_witnesses.swap(ack->witnesses);
+    mdr->more()->extra_witnesses = ack->witnesses;
     mdr->more()->extra_witnesses.erase(mds->get_nodeid());  // not me!
   }
 
   // srci import?
   if (ack->inode_export.length()) {
     dout(10) << " got srci import" << dendl;
-    mdr->more()->inode_import.claim(ack->inode_export);
+    mdr->more()->inode_import.share(ack->inode_export);
     mdr->more()->inode_import_v = ack->inode_export_v;
   }
 
@@ -9036,7 +8969,7 @@ void Server::handle_slave_rename_prep_ack(MDRequestRef& mdr, MMDSSlaveRequest *a
     dout(10) << "still waiting on slaves " << mdr->more()->waiting_on_slave << dendl;
 }
 
-void Server::handle_slave_rename_notify_ack(MDRequestRef& mdr, MMDSSlaveRequest *ack)
+void Server::handle_slave_rename_notify_ack(MDRequestRef& mdr, const MMDSSlaveRequest::const_ref &ack)
 {
   dout(10) << "handle_slave_rename_notify_ack " << *mdr << " from mds."
           << ack->get_source() << dendl;
@@ -9075,7 +9008,7 @@ void Server::_slave_rename_sessions_flushed(MDRequestRef& mdr)
 /* This function takes responsibility for the passed mdr*/
 void Server::handle_client_lssnap(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
 
   // traverse to path
   CInode *diri = mdcache->get_inode(req->get_filepath().get_ino());
@@ -9190,7 +9123,7 @@ struct C_MDS_mksnap_finish : public ServerLogContext {
 /* This function takes responsibility for the passed mdr*/
 void Server::handle_client_mksnap(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   // make sure we have as new a map as the client
   if (req->get_mdsmap_epoch() > mds->mdsmap->get_epoch()) {
     mds->wait_for_mdsmap(req->get_mdsmap_epoch(), new C_MDS_RetryRequest(mdcache, mdr));
@@ -9356,7 +9289,7 @@ struct C_MDS_rmsnap_finish : public ServerLogContext {
 /* This function takes responsibility for the passed mdr*/
 void Server::handle_client_rmsnap(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
 
   CInode *diri = mdcache->get_inode(req->get_filepath().get_ino());
   if (!diri || diri->state_test(CInode::STATE_PURGING)) {
@@ -9488,7 +9421,7 @@ struct C_MDS_renamesnap_finish : public ServerLogContext {
 /* This function takes responsibility for the passed mdr*/
 void Server::handle_client_renamesnap(MDRequestRef& mdr)
 {
-  MClientRequest *req = mdr->client_request;
+  const MClientRequest::const_ref &req = mdr->client_request;
   if (req->get_filepath().get_ino() != req->get_filepath2().get_ino()) {
     respond_to_request(mdr, -EINVAL);
     return;
index 4719d4f374efe919d493026d192f833691acb535..bdbba34081adef0448178aab0bb4685add6ae3c4 100644 (file)
 
 #include <string_view>
 
+#include "messages/MClientCaps.h"
+#include "messages/MClientReconnect.h"
+#include "messages/MClientReply.h"
+#include "messages/MClientRequest.h"
+#include "messages/MClientSession.h"
+#include "messages/MClientSnap.h"
+#include "messages/MLock.h"
+
 #include "MDSRank.h"
 #include "Mutation.h"
 #include "MDSContext.h"
@@ -26,10 +34,7 @@ class PerfCounters;
 class LogEvent;
 class EMetaBlob;
 class EUpdate;
-class MMDSSlaveRequest;
 struct SnapInfo;
-class MClientRequest;
-class MClientReply;
 class MDLog;
 
 enum {
@@ -108,7 +113,7 @@ public:
   void create_logger();
 
   // message handler
-  void dispatch(Message *m);
+  void dispatch(const Message::const_ref &m);
 
   void handle_osd_map();
 
@@ -116,7 +121,7 @@ public:
   bool waiting_for_reconnect(client_t c) const;
   void dump_reconnect_status(Formatter *f) const;
 
-  void handle_client_session(class MClientSession *m);
+  void handle_client_session(const MClientSession::const_ref &m);
   void _session_logged(Session *session, uint64_t state_seq, 
                       bool open, version_t pv, interval_set<inodeno_t>& inos,version_t piv);
   version_t prepare_force_open_sessions(map<client_t,entity_inst_t> &cm,
@@ -132,7 +137,7 @@ public:
   size_t apply_blacklist(const std::set<entity_addr_t> &blacklist);
   void journal_close_session(Session *session, int state, Context *on_safe);
   void reconnect_clients(MDSInternalContext *reconnect_done_);
-  void handle_client_reconnect(class MClientReconnect *m);
+  void handle_client_reconnect(const MClientReconnect::const_ref &m);
   void infer_supported_features(Session *session, client_metadata_t& client_metadata);
   void update_required_client_features();
 
@@ -145,27 +150,27 @@ public:
   void force_clients_readonly();
 
   // -- requests --
-  void handle_client_request(MClientRequest *m);
+  void handle_client_request(const MClientRequest::const_ref &m);
 
   void journal_and_reply(MDRequestRef& mdr, CInode *tracei, CDentry *tracedn,
                         LogEvent *le, MDSLogContextBase *fin);
   void submit_mdlog_entry(LogEvent *le, MDSLogContextBase *fin,
                           MDRequestRef& mdr, const char *evt);
   void dispatch_client_request(MDRequestRef& mdr);
-  void perf_gather_op_latency(const MClientRequestreq, utime_t lat);
+  void perf_gather_op_latency(const MClientRequest::const_ref &req, utime_t lat);
   void early_reply(MDRequestRef& mdr, CInode *tracei, CDentry *tracedn);
   void respond_to_request(MDRequestRef& mdr, int r = 0);
-  void set_trace_dist(Session *session, MClientReply *reply, CInode *in, CDentry *dn,
+  void set_trace_dist(Session *session, const MClientReply::ref &reply, CInode *in, CDentry *dn,
                      snapid_t snapid,
                      int num_dentries_wanted,
                      MDRequestRef& mdr);
 
 
-  void handle_slave_request(MMDSSlaveRequest *m);
-  void handle_slave_request_reply(MMDSSlaveRequest *m);
+  void handle_slave_request(const MMDSSlaveRequest::const_ref &m);
+  void handle_slave_request_reply(const MMDSSlaveRequest::const_ref &m);
   void dispatch_slave_request(MDRequestRef& mdr);
   void handle_slave_auth_pin(MDRequestRef& mdr);
-  void handle_slave_auth_pin_ack(MDRequestRef& mdr, MMDSSlaveRequest *ack);
+  void handle_slave_auth_pin_ack(MDRequestRef& mdr, const MMDSSlaveRequest::const_ref &ack);
 
   // some helpers
   bool check_fragment_space(MDRequestRef& mdr, CDir *in);
@@ -255,10 +260,10 @@ public:
   void _logged_slave_link(MDRequestRef& mdr, CInode *targeti, bool adjust_realm);
   void _commit_slave_link(MDRequestRef& mdr, int r, CInode *targeti);
   void _committed_slave(MDRequestRef& mdr);  // use for rename, too
-  void handle_slave_link_prep_ack(MDRequestRef& mdr, MMDSSlaveRequest *m);
+  void handle_slave_link_prep_ack(MDRequestRef& mdr, const MMDSSlaveRequest::const_ref &m);
   void do_link_rollback(bufferlist &rbl, mds_rank_t master, MDRequestRef& mdr);
   void _link_rollback_finish(MutationRef& mut, MDRequestRef& mdr,
-                            map<client_t,MClientSnap*>& split);
+                            map<client_t,MClientSnap::ref>& split);
 
   // unlink
   void handle_client_unlink(MDRequestRef& mdr);
@@ -272,7 +277,7 @@ public:
   void handle_slave_rmdir_prep(MDRequestRef& mdr);
   void _logged_slave_rmdir(MDRequestRef& mdr, CDentry *srcdn, CDentry *straydn);
   void _commit_slave_rmdir(MDRequestRef& mdr, int r, CDentry *straydn);
-  void handle_slave_rmdir_prep_ack(MDRequestRef& mdr, MMDSSlaveRequest *ack);
+  void handle_slave_rmdir_prep_ack(MDRequestRef& mdr, const MMDSSlaveRequest::const_ref &ack);
   void do_rmdir_rollback(bufferlist &rbl, mds_rank_t master, MDRequestRef& mdr);
   void _rmdir_rollback_finish(MDRequestRef& mdr, metareqid_t reqid, CDentry *dn, CDentry *straydn);
 
@@ -305,18 +310,18 @@ public:
 
   // slaving
   void handle_slave_rename_prep(MDRequestRef& mdr);
-  void handle_slave_rename_prep_ack(MDRequestRef& mdr, MMDSSlaveRequest *m);
-  void handle_slave_rename_notify_ack(MDRequestRef& mdr, MMDSSlaveRequest *m);
+  void handle_slave_rename_prep_ack(MDRequestRef& mdr, const MMDSSlaveRequest::const_ref &m);
+  void handle_slave_rename_notify_ack(MDRequestRef& mdr, const MMDSSlaveRequest::const_ref &m);
   void _slave_rename_sessions_flushed(MDRequestRef& mdr);
   void _logged_slave_rename(MDRequestRef& mdr, CDentry *srcdn, CDentry *destdn, CDentry *straydn);
   void _commit_slave_rename(MDRequestRef& mdr, int r, CDentry *srcdn, CDentry *destdn, CDentry *straydn);
   void do_rename_rollback(bufferlist &rbl, mds_rank_t master, MDRequestRef& mdr, bool finish_mdr=false);
   void _rename_rollback_finish(MutationRef& mut, MDRequestRef& mdr, CDentry *srcdn, version_t srcdnpv,
-                              CDentry *destdn, CDentry *staydn, map<client_t,MClientSnap*> splits[2],
+                              CDentry *destdn, CDentry *staydn, map<client_t,MClientSnap::ref> splits[2],
                               bool finish_mdr);
 
 private:
-  void reply_client_request(MDRequestRef& mdr, MClientReply *reply);
+  void reply_client_request(MDRequestRef& mdr, const MClientReply::ref &reply);
 };
 
 #endif
index 10e87a713e121125f104ed234ed082790e4ef145..2e1f0bb7c17e109287813570622848bd4a64bb56 100644 (file)
@@ -144,7 +144,7 @@ public:
   entity_addr_t socket_addr;
   xlist<Session*>::item item_session_list;
 
-  list<Message*> preopen_out_queue;  ///< messages for client, queued before they connect
+  list<Message::ref> preopen_out_queue;  ///< messages for client, queued before they connect
 
   elist<MDRequestImpl*> requests;
   size_t get_request_count();
@@ -345,10 +345,7 @@ public:
     } else {
       assert(!item_session_list.is_on_list());
     }
-    while (!preopen_out_queue.empty()) {
-      preopen_out_queue.front()->put();
-      preopen_out_queue.pop_front();
-    }
+    preopen_out_queue.clear();
   }
 
   void set_connection(Connection *con) {
index 0c839338d6922297e8c3d31f34a311cf320af5ce..c3d8f0a6df2c90792a26e1b406bfdb666bdb7028 100644 (file)
@@ -296,7 +296,7 @@ public:
     }
   };
 
-  void decode_locked_state(bufferlist& bl) {
+  void decode_locked_state(const bufferlist& bl) {
     parent->decode_lock_state(type->type, bl);
   }
   void encode_locked_state(bufferlist& bl) {
index d3379fee2c9689d5d36f6bfc911315ceb611170d..b4fea91a6bfcf2d4310c94d01926ff507d3bd9c6 100644 (file)
@@ -40,7 +40,7 @@ void SnapClient::resend_queries()
   }
 }
 
-void SnapClient::handle_query_result(MMDSTableRequest *m)
+void SnapClient::handle_query_result(const MMDSTableRequest::const_ref &m)
 {
   dout(10) << __func__ << " " << *m << dendl;
 
@@ -111,11 +111,11 @@ void SnapClient::handle_query_result(MMDSTableRequest *m)
   }
 }
 
-void SnapClient::handle_notify_prep(MMDSTableRequest *m)
+void SnapClient::handle_notify_prep(const MMDSTableRequest::const_ref &m)
 {
   dout(10) << __func__ << " " << *m << dendl;
   handle_query_result(m);
-  MMDSTableRequest *ack = new MMDSTableRequest(table, TABLESERVER_OP_NOTIFY_ACK, 0, m->get_tid());
+  MMDSTableRequest::ref ack(new MMDSTableRequest(table, TABLESERVER_OP_NOTIFY_ACK, 0, m->get_tid()), false);
   mds->send_message(ack, m->get_connection());
 }
 
@@ -153,7 +153,7 @@ void SnapClient::refresh(version_t want, MDSInternalContextBase *onfinish)
     return;
 
   mds_rank_t ts = mds->mdsmap->get_tableserver();
-  MMDSTableRequest *req = new MMDSTableRequest(table, TABLESERVER_OP_QUERY, ++last_reqid, 0);
+  MMDSTableRequest::ref req(new MMDSTableRequest(table, TABLESERVER_OP_QUERY, ++last_reqid, 0), false);
   using ceph::encode;
   char op = 'F';
   encode(op, req->bl);
index cf711231b37aa1c57590b86fbf679ff89b1a1b6f..f587983eec2c406ac4d4993d47bcb293061db3ed 100644 (file)
@@ -45,8 +45,8 @@ public:
     sync_reqid(0), synced(false) {}
 
   void resend_queries() override;
-  void handle_query_result(MMDSTableRequest *m) override;
-  void handle_notify_prep(MMDSTableRequest *m) override;
+  void handle_query_result(const MMDSTableRequest::const_ref &m) override;
+  void handle_notify_prep(const MMDSTableRequest::const_ref &m) override;
   void notify_commit(version_t tid) override;
 
   void prepare_create(inodeno_t dirino, std::string_view name, utime_t stamp,
index ddfd3d07319d40da0772641185cbf2987e3065e6..78c5bc67ac00e15d6dd1ada92439bcf89b158ace 100644 (file)
@@ -67,7 +67,7 @@ void SnapServer::reset_state()
 
 // SERVER
 
-void SnapServer::_prepare(bufferlist &bl, uint64_t reqid, mds_rank_t bymds)
+void SnapServer::_prepare(const bufferlist& bl, uint64_t reqid, mds_rank_t bymds, bufferlist& out)
 {
   using ceph::decode;
   using ceph::encode;
@@ -90,8 +90,8 @@ void SnapServer::_prepare(bufferlist &bl, uint64_t reqid, mds_rank_t bymds)
        pending_noop.insert(version);
        dout(10) << "prepare v" << version << " noop" << dendl;
       }
-      bl.clear();
-      encode(last_snap, bl);
+
+      encode(last_snap, out);
     }
     break;
 
@@ -108,8 +108,7 @@ void SnapServer::_prepare(bufferlist &bl, uint64_t reqid, mds_rank_t bymds)
       pending_destroy[version] = pair<snapid_t,snapid_t>(snapid, last_snap);
       dout(10) << "prepare v" << version << " destroy " << snapid << " seq " << last_snap << dendl;
 
-      bl.clear();
-      encode(last_snap, bl);
+      encode(last_snap, out);
     }
     break;
 
@@ -123,8 +122,6 @@ void SnapServer::_prepare(bufferlist &bl, uint64_t reqid, mds_rank_t bymds)
 
       pending_update[version] = info;
       dout(10) << "prepare v" << version << " update " << info << dendl;
-
-      bl.clear();
     }
     break;
 
@@ -158,7 +155,7 @@ void SnapServer::_get_reply_buffer(version_t tid, bufferlist *pbl) const
   assert (0 == "tid not found");
 }
 
-void SnapServer::_commit(version_t tid, MMDSTableRequest *req)
+void SnapServer::_commit(version_t tid, MMDSTableRequest::const_ref req)
 {
   if (pending_update.count(tid)) {
     SnapInfo &info = pending_update[tid];
@@ -264,15 +261,15 @@ bool SnapServer::_notify_prep(version_t tid)
   encode(last_destroyed, bl);
   assert(version == tid);
 
-  for (auto p : active_clients) {
-    MMDSTableRequest *m = new MMDSTableRequest(table, TABLESERVER_OP_NOTIFY_PREP, 0, version);
+  for (auto &p : active_clients) {
+    MMDSTableRequest::ref m(new MMDSTableRequest(table, TABLESERVER_OP_NOTIFY_PREP, 0, version), false);
     m->bl = bl;
     mds->send_message_mds(m, p);
   }
   return true;
 }
 
-void SnapServer::handle_query(MMDSTableRequest *req)
+void SnapServer::handle_query(const MMDSTableRequest::const_ref &req)
 {
   using ceph::encode;
   using ceph::decode;
@@ -280,7 +277,7 @@ void SnapServer::handle_query(MMDSTableRequest *req)
   auto p = req->bl.cbegin();
   decode(op, p);
 
-  MMDSTableRequest *reply = new MMDSTableRequest(table, TABLESERVER_OP_QUERY_REPLY, req->reqid, version);
+  MMDSTableRequest::ref reply(new MMDSTableRequest(table, TABLESERVER_OP_QUERY_REPLY, req->reqid, version), false);
 
   switch (op) {
     case 'F': // full
@@ -306,7 +303,6 @@ void SnapServer::handle_query(MMDSTableRequest *req)
   };
 
   mds->send_message(reply, req->get_connection());
-  req->put();
 }
 
 void SnapServer::check_osd_map(bool force)
index 9c88fbec83d7c1cb0b3424a3244ac9ba4dfad7b3..1860dd4cafd2522ac3f3ddc9baa69639cc05ed67 100644 (file)
@@ -83,13 +83,13 @@ protected:
   }
 
   // server bits
-  void _prepare(bufferlist &bl, uint64_t reqid, mds_rank_t bymds) override;
+  void _prepare(const bufferlist &bl, uint64_t reqid, mds_rank_t bymds, bufferlist &out) override;
   void _get_reply_buffer(version_t tid, bufferlist *pbl) const override;
-  void _commit(version_t tid, MMDSTableRequest *req=NULL) override;
+  void _commit(version_t tid, MMDSTableRequest::const_ref req) override;
   void _rollback(version_t tid) override;
   void _server_update(bufferlist& bl) override;
   bool _notify_prep(version_t tid) override;
-  void handle_query(MMDSTableRequest *m) override;
+  void handle_query(const MMDSTableRequest::const_ref &m) override;
 
 public:
   SnapServer(MDSRank *m, MonClient *monc)
index c53dfa269ec64e36d126ebb9e4854f683543773a..276469e84cc1d1884c1559caa599cc1a52c75c00 100644 (file)
@@ -35,7 +35,7 @@ public:
   bufferlist client_map;  // encoded map<__u32,entity_inst_t>
   version_t cmapv{0};
 
-  EImportStart(MDLog *log, dirfrag_t di, vector<dirfrag_t>& b, mds_rank_t f) :
+  EImportStart(MDLog *log, dirfrag_t di, const vector<dirfrag_t>& b, mds_rank_t f) :
     LogEvent(EVENT_IMPORTSTART),
     base(di), bounds(b), from(f) { }
   EImportStart() :
index 222bba3b1bd05ffd5225e441596455ac71b1e19e..5d0baf335a2edc0895064ea89fc07933cdd14e49 100644 (file)
@@ -1913,12 +1913,15 @@ void ETableServer::replay(MDSRank *mds)
   assert(version-1 == server->get_version());
 
   switch (op) {
-  case TABLESERVER_OP_PREPARE:
+  case TABLESERVER_OP_PREPARE: {
     server->_note_prepare(bymds, reqid, true);
-    server->_prepare(mutation, reqid, bymds);
+    bufferlist out;
+    server->_prepare(mutation, reqid, bymds, out);
+    mutation = std::move(out);
     break;
+  }
   case TABLESERVER_OP_COMMIT:
-    server->_commit(tid);
+    server->_commit(tid, MMDSTableRequest::ref());
     server->_note_commit(tid, true);
     break;
   case TABLESERVER_OP_ROLLBACK:
index 62a7589aea002a2085df1108d548d15ca68690c4..a544597284012cfd26f5086b163203f1369ae4b6 100644 (file)
@@ -1125,6 +1125,8 @@ struct client_metadata_t {
     kv_map(std::move(other.kv_map)), features(std::move(other.features)) {}
   client_metadata_t(kv_map_t&& kv, feature_bitset_t &&f) :
     kv_map(std::move(kv)), features(std::move(f)) {}
+  client_metadata_t(const kv_map_t& kv, const feature_bitset_t &f) :
+    kv_map(kv), features(f) {}
   client_metadata_t& operator=(const client_metadata_t& other) {
     kv_map = other.kv_map;
     features = other.features;
index e82e2e5adca09c9589e420a8f25fda5406361d3d..e8542424467fd6411355932b3dd4d7dcc83efbde 100644 (file)
 
 #include <string_view>
 
+#include "msg/Message.h"
+
 #include "mds/mdstypes.h"
 
 class MCacheExpire : public Message {
   __s32 from;
 
 public:
+  typedef boost::intrusive_ptr<MCacheExpire> ref;
+  typedef boost::intrusive_ptr<MCacheExpire const> const_ref;
   /*
     group things by realm (auth delgation root), since that's how auth is determined.
     that makes it less work to process when exports are in progress.
@@ -32,16 +36,14 @@ public:
     map<dirfrag_t, uint32_t> dirs;
     map<dirfrag_t, map<pair<string,snapid_t>,uint32_t> > dentries;
 
-    void merge(realm& o) {
+    void merge(const realm& o) {
       inodes.insert(o.inodes.begin(), o.inodes.end());
       dirs.insert(o.dirs.begin(), o.dirs.end());
-      for (map<dirfrag_t,map<pair<string,snapid_t>,uint32_t> >::iterator p = o.dentries.begin();
-          p != o.dentries.end();
-          ++p) {
-       if (dentries.count(p->first) == 0)
-         dentries[p->first] = p->second;
-       else
-         dentries[p->first].insert(p->second.begin(), p->second.end());
+      for (const auto &p : o.dentries) {
+        auto em = dentries.emplace(std::piecewise_construct, std::forward_as_tuple(p.first), std::forward_as_tuple(p.second));
+        if (!em.second) {
+         em.first->second.insert(p.second.begin(), p.second.end());
+        }
       }
     }
 
@@ -62,7 +64,7 @@ public:
 
   map<dirfrag_t, realm> realms;
 
-  int get_from() { return from; }
+  int get_from() const { return from; }
 
   MCacheExpire() : Message(MSG_MDS_CACHEEXPIRE), from(-1) {}
   MCacheExpire(int f) : 
@@ -84,11 +86,11 @@ public:
     realms[r].dentries[df][pair<string,snapid_t>(dn,last)] = nonce;
   }
 
-  void add_realm(dirfrag_t df, realm& r) {
-    if (realms.count(df) == 0)
-      realms[df] = r;
-    else
-      realms[df].merge(r);
+  void add_realm(dirfrag_t df, const realm& r) {
+    auto em = realms.emplace(std::piecewise_construct, std::forward_as_tuple(df), std::forward_as_tuple(r));
+    if (!em.second) {
+      em.first->second.merge(r);
+    }
   }
 
   void decode_payload() override {
index 275e2d08cc9ffa96dca9fb1c03ffcb70b99b9383..07692a7263fff98d8da3315440eb98a65e13d194 100644 (file)
@@ -22,6 +22,9 @@ class MClientCapRelease : public Message {
   static const int HEAD_VERSION = 2;
   static const int COMPAT_VERSION = 1;
  public:
+  typedef boost::intrusive_ptr<MClientCapRelease> ref;
+  typedef boost::intrusive_ptr<MClientCapRelease const> const_ref;
+
   struct ceph_mds_cap_release head;
   vector<ceph_mds_cap_item> caps;
 
index cc93dbbf00fb3bc97005037d344a9c0d9ad4c154..57a21bae03d86aa3ffed8c861fe3a6bba9556277 100644 (file)
@@ -23,6 +23,9 @@ class MClientCaps : public Message {
   static const int COMPAT_VERSION = 1;
 
  public:
+  typedef boost::intrusive_ptr<MClientCaps> ref;
+  typedef boost::intrusive_ptr<MClientCaps const> const_ref;
+
   static const unsigned FLAG_SYNC              = (1<<0);
   static const unsigned FLAG_NO_CAPSNAP                = (1<<1); // unused
   static const unsigned FLAG_PENDING_CAPSNAP   = (1<<2);
@@ -56,32 +59,32 @@ class MClientCaps : public Message {
   /* advisory CLIENT_CAPS_* flags to send to mds */
   unsigned flags = 0;
 
-  int      get_caps() { return head.caps; }
-  int      get_wanted() { return head.wanted; }
-  int      get_dirty() { return head.dirty; }
-  ceph_seq_t get_seq() { return head.seq; }
-  ceph_seq_t get_issue_seq() { return head.issue_seq; }
-  ceph_seq_t get_mseq() { return head.migrate_seq; }
-
-  inodeno_t get_ino() { return inodeno_t(head.ino); }
-  inodeno_t get_realm() { return inodeno_t(head.realm); }
-  uint64_t get_cap_id() { return head.cap_id; }
-
-  uint64_t get_size() { return size;  }
-  uint64_t get_max_size() { return max_size;  }
-  __u32 get_truncate_seq() { return truncate_seq; }
-  uint64_t get_truncate_size() { return truncate_size; }
-  utime_t get_ctime() { return ctime; }
-  utime_t get_btime() { return btime; }
-  utime_t get_mtime() { return mtime; }
-  utime_t get_atime() { return atime; }
-  __u64 get_change_attr() { return change_attr; }
-  __u32 get_time_warp_seq() { return time_warp_seq; }
-  uint64_t get_nfiles() { return nfiles; }
-  uint64_t get_nsubdirs() { return nsubdirs; }
+  int      get_caps() const { return head.caps; }
+  int      get_wanted() const { return head.wanted; }
+  int      get_dirty() const { return head.dirty; }
+  ceph_seq_t get_seq() const { return head.seq; }
+  ceph_seq_t get_issue_seq() const { return head.issue_seq; }
+  ceph_seq_t get_mseq() const { return head.migrate_seq; }
+
+  inodeno_t get_ino() const { return inodeno_t(head.ino); }
+  inodeno_t get_realm() const { return inodeno_t(head.realm); }
+  uint64_t get_cap_id() const { return head.cap_id; }
+
+  uint64_t get_size() const { return size;  }
+  uint64_t get_max_size() const { return max_size;  }
+  __u32 get_truncate_seq() const { return truncate_seq; }
+  uint64_t get_truncate_size() const { return truncate_size; }
+  utime_t get_ctime() const { return ctime; }
+  utime_t get_btime() const { return btime; }
+  utime_t get_mtime() const { return mtime; }
+  utime_t get_atime() const { return atime; }
+  __u64 get_change_attr() const { return change_attr; }
+  __u32 get_time_warp_seq() const { return time_warp_seq; }
+  uint64_t get_nfiles() const { return nfiles; }
+  uint64_t get_nsubdirs() const { return nsubdirs; }
   bool dirstat_is_valid() const { return nfiles != -1 || nsubdirs != -1; }
 
-  const file_layout_t& get_layout() {
+  const file_layout_t& get_layout() const {
     return layout;
   }
 
@@ -89,13 +92,13 @@ class MClientCaps : public Message {
     layout = l;
   }
 
-  int       get_migrate_seq() { return head.migrate_seq; }
-  int       get_op() { return head.op; }
+  int       get_migrate_seq() const { return head.migrate_seq; }
+  int       get_op() const { return head.op; }
 
-  uint64_t get_client_tid() { return get_tid(); }
+  uint64_t get_client_tid() const { return get_tid(); }
   void set_client_tid(uint64_t s) { set_tid(s); }
 
-  snapid_t get_snap_follows() { return snapid_t(head.snap_follows); }
+  snapid_t get_snap_follows() const { return snapid_t(head.snap_follows); }
   void set_snap_follows(snapid_t s) { head.snap_follows = s; }
 
   void set_caps(int c) { head.caps = c; }
@@ -120,7 +123,7 @@ class MClientCaps : public Message {
   }
 
   void set_oldest_flush_tid(ceph_tid_t tid) { oldest_flush_tid = tid; }
-  ceph_tid_t get_oldest_flush_tid() { return oldest_flush_tid; }
+  ceph_tid_t get_oldest_flush_tid() const { return oldest_flush_tid; }
 
   void clear_dirty() { head.dirty = 0; }
 
index 2b6487e7e2fc33a371f483497068b498927c7119..b127c90e1fb63b5617c206a71e820355a1adc4d6 100644 (file)
 
 #include "msg/Message.h"
 
-struct MClientLease : public Message {
+class MClientLease : public Message {
+public:
+  typedef boost::intrusive_ptr<MClientLease> ref;
+  typedef boost::intrusive_ptr<MClientLease const> const_ref;
+
   struct ceph_mds_lease h;
   std::string dname;
   
@@ -32,6 +36,10 @@ struct MClientLease : public Message {
   snapid_t get_last() const { return snapid_t(h.last); }
 
   MClientLease() : Message(CEPH_MSG_CLIENT_LEASE) {}
+  MClientLease(const MClientLease& m) :
+    Message(CEPH_MSG_CLIENT_LEASE),
+    h(m.h),
+    dname(m.dname) {}
   MClientLease(int ac, ceph_seq_t seq, int m, uint64_t i, uint64_t sf, uint64_t sl) :
     Message(CEPH_MSG_CLIENT_LEASE) {
     h.action = ac;
index 6c888b89e690ca9f828fbba710826a1f9a69f6c2..6740fbb1d40ec05a18fa076bce52713f235b0389 100644 (file)
@@ -25,6 +25,8 @@ class MClientReconnect : public Message {
   const static int HEAD_VERSION = 3;
 
 public:
+  typedef boost::intrusive_ptr<MClientReconnect> ref;
+  typedef boost::intrusive_ptr<MClientReconnect const> const_ref;
   map<inodeno_t, cap_reconnect_t>  caps;   // only head inodes
   vector<ceph_mds_snaprealm_reconnect> realms;
 
index edd3d19c2bb360ad33429097d7080c10106f74db..34a017ffb092c1488b1710003cd4329eb2320936 100644 (file)
@@ -256,6 +256,8 @@ struct InodeStat {
 class MClientReply : public Message {
   // reply data
 public:
+  typedef boost::intrusive_ptr<MClientReply> ref;
+  typedef boost::intrusive_ptr<MClientReply const> const_ref;
   struct ceph_mds_reply_head head {};
   bufferlist trace_bl;
   bufferlist extra_bl;
@@ -278,11 +280,11 @@ public:
   bool is_safe() const { return head.safe; }
 
   MClientReply() : Message(CEPH_MSG_CLIENT_REPLY) {}
-  MClientReply(MClientRequest *req, int result = 0) : 
+  MClientReply(const MClientRequest &req, int result = 0) :
     Message(CEPH_MSG_CLIENT_REPLY) {
     memset(&head, 0, sizeof(head));
-    header.tid = req->get_tid();
-    head.op = req->get_op();
+    header.tid = req.get_tid();
+    head.op = req.get_op();
     head.result = result;
     head.safe = 1;
   }
index edb33e1acdf893910711fadca7a36f80aad990d8..f7771d133ffb1cb4c8bb58887a1484f132d1cedd 100644 (file)
@@ -53,7 +53,9 @@ class MClientRequest : public Message {
   static const int COMPAT_VERSION = 1;
 
 public:
-  struct ceph_mds_request_head head;
+  typedef boost::intrusive_ptr<MClientRequest> ref;
+  typedef boost::intrusive_ptr<MClientRequest const> const_ref;
+  mutable struct ceph_mds_request_head head; /* XXX HACK! */
   utime_t stamp;
 
   struct Release {
@@ -76,13 +78,14 @@ public:
       decode_nohead(item.dname_len, dname, bl);
     }
   };
-  vector<Release> releases;
+  mutable vector<Release> releases; /* XXX HACK! */
 
   // path arguments
   filepath path, path2;
   vector<uint64_t> gid_list;
 
-  bool queued_for_replay = false;
+  /* XXX HACK */
+  mutable bool queued_for_replay = false;
 
  public:
   // cons
@@ -98,7 +101,7 @@ private:
 
 public:
   void set_mdsmap_epoch(epoch_t e) { head.mdsmap_epoch = e; }
-  epoch_t get_mdsmap_epoch() { return head.mdsmap_epoch; }
+  epoch_t get_mdsmap_epoch() const { return head.mdsmap_epoch; }
   epoch_t get_osdmap_epoch() const {
     assert(head.op == CEPH_MDS_OP_SETXATTR);
     if (header.version >= 3)
@@ -111,7 +114,7 @@ public:
     head.args.setxattr.osdmap_epoch = e;
   }
 
-  metareqid_t get_reqid() {
+  metareqid_t get_reqid() const {
     // FIXME: for now, assume clients always have 1 incarnation
     return metareqid_t(get_orig_source(), header.tid); 
   }
@@ -119,7 +122,7 @@ public:
   /*bool open_file_mode_is_readonly() {
     return file_mode_is_readonly(ceph_flags_to_mode(head.args.open.flags));
     }*/
-  bool may_write() {
+  bool may_write() const {
     return
       (head.op & CEPH_MDS_OP_WRITE) || 
       (head.op == CEPH_MDS_OP_OPEN && (head.args.open.flags & (O_CREAT|O_TRUNC)));
@@ -128,7 +131,7 @@ public:
   int get_flags() const {
     return head.flags;
   }
-  bool is_replay() {
+  bool is_replay() const {
     return get_flags() & CEPH_MDS_FLAG_REPLAY;
   }
 
@@ -169,10 +172,10 @@ public:
   const string& get_path2() const { return path2.get_path(); }
   const filepath& get_filepath2() const { return path2; }
 
-  int get_dentry_wanted() { return get_flags() & CEPH_MDS_FLAG_WANT_DENTRY; }
+  int get_dentry_wanted() const { return get_flags() & CEPH_MDS_FLAG_WANT_DENTRY; }
 
-  void mark_queued_for_replay() { queued_for_replay = true; }
-  bool is_queued_for_replay() { return queued_for_replay; }
+  void mark_queued_for_replay() const { queued_for_replay = true; }
+  bool is_queued_for_replay() const { return queued_for_replay; }
 
   void decode_payload() override {
     auto p = payload.cbegin();
index 31d943c89fb1e184e429bc8a17b7f0150aed72a2..3589d291e6960159fab06cb18ef027931bd79121 100644 (file)
@@ -23,6 +23,8 @@ class MClientSession : public Message {
   static const int COMPAT_VERSION = 1;
 
 public:
+  typedef boost::intrusive_ptr<MClientSession> ref;
+  typedef boost::intrusive_ptr<MClientSession const> const_ref;
   ceph_mds_session_head head;
 
   std::map<std::string, std::string> metadata;
index 8ef9970650d8ed469464c14d486fd6a0aa1c5756..b7b8864c5c310eab4d559d3a821ab052744a8cb2 100644 (file)
@@ -18,6 +18,9 @@
 #include "msg/Message.h"
 
 struct MClientSnap : public Message {
+  typedef boost::intrusive_ptr<MClientSnap> ref;
+  typedef boost::intrusive_ptr<MClientSnap const> const_ref;
+
   ceph_mds_snap_head head;
   bufferlist bl;
   
index 1b7cfd6da7378cfe4708845f3d93dfad4529182f..e0773181be93cfdeff14ae64534d8f41fd54d04a 100644 (file)
@@ -21,6 +21,9 @@
 
 class MCommand : public Message {
  public:
+  typedef boost::intrusive_ptr<MCommand> ref;
+  typedef boost::intrusive_ptr<MCommand const> const_ref;
+
   uuid_d fsid;
   std::vector<string> cmd;
 
index 4819421fd6b36c3813b987dab0641bf42d5ad133..3799f393b61444d2c3d2be1f5eeacc667fe2a4ef 100644 (file)
@@ -18,6 +18,8 @@
 
 #include <string_view>
 
+#include "msg/Message.h"
+
 class MDentryLink : public Message {
   dirfrag_t subtree;
   dirfrag_t dirfrag;
@@ -25,6 +27,8 @@ class MDentryLink : public Message {
   bool is_primary = false;
 
  public:
+  typedef boost::intrusive_ptr<MDentryLink> ref;
+  typedef boost::intrusive_ptr<MDentryLink const> const_ref;
   dirfrag_t get_subtree() const { return subtree; }
   dirfrag_t get_dirfrag() const { return dirfrag; }
   const string& get_dn() const { return dn; }
index af86c5e92ff44ab6b778d6ef1d7ed9b36845c286..613c1077bbb0dea71745486b354fe0ff89efcd44 100644 (file)
 
 #include <string_view>
 
+#include "msg/Message.h"
+
 class MDentryUnlink : public Message {
   dirfrag_t dirfrag;
   string dn;
 
  public:
-  dirfrag_t get_dirfrag() { return dirfrag; }
-  string& get_dn() { return dn; }
+  typedef boost::intrusive_ptr<MDentryUnlink> ref;
+  typedef boost::intrusive_ptr<MDentryUnlink const> const_ref;
+  dirfrag_t get_dirfrag() const { return dirfrag; }
+  const string& get_dn() const { return dn; }
 
   bufferlist straybl;
   bufferlist snapbl;
index e73351bff9e43414cfa67141ff047ffc8ce0b38b..a35de0b8db21ba5bf1a0593e210af10eb3d6f577 100644 (file)
 #ifndef CEPH_MDIRUPDATE_H
 #define CEPH_MDIRUPDATE_H
 
-#include "msg/Message.h"
+#include "MInterMDS.h"
 
-class MDirUpdate : public Message {
+class MDirUpdate : public MInterMDS {
 public:
-  MDirUpdate() : Message(MSG_MDS_DIRUPDATE) {}
+  typedef boost::intrusive_ptr<MDirUpdate> ref;
+  typedef boost::intrusive_ptr<MDirUpdate const> const_ref;
+  MDirUpdate() : MInterMDS(MSG_MDS_DIRUPDATE) {}
   MDirUpdate(mds_rank_t f,
             dirfrag_t dirfrag,
              int dir_rep,
              const std::set<int32_t>& dir_rep_by,
              filepath& path,
              bool discover = false) :
-    Message(MSG_MDS_DIRUPDATE), from_mds(f), dirfrag(dirfrag),
+    MInterMDS(MSG_MDS_DIRUPDATE), from_mds(f), dirfrag(dirfrag),
     dir_rep(dir_rep), dir_rep_by(dir_rep_by), path(path) {
     this->discover = discover ? 5 : 0;
   }
@@ -40,7 +42,7 @@ public:
   const filepath& get_path() const { return path; }
 
   bool has_tried_discover() const { return tried_discover > 0; }
-  void inc_tried_discover() { ++tried_discover; }
+  void inc_tried_discover() const { ++tried_discover; }
 
   const char *get_type_name() const override { return "dir_update"; }
   void print(ostream& out) const override {
@@ -67,8 +69,27 @@ public:
     encode(path, payload);
   }
 
-private:
-  ~MDirUpdate() override {}
+  bool is_forwardable() const override {
+    return true;
+  }
+
+  MInterMDS::ref forwardable() const override {
+    MDirUpdate::ref m(new MDirUpdate(*this), false);
+    return m;
+  }
+
+protected:
+  ~MDirUpdate() {}
+  MDirUpdate(const MDirUpdate& m)
+  : MInterMDS(MSG_MDS_DIRUPDATE),
+    from_mds(m.from_mds),
+    dirfrag(m.dirfrag),
+    dir_rep(m.dir_rep),
+    discover(m.discover),
+    dir_rep_by(m.dir_rep_by),
+    path(m.path),
+    tried_discover(m.tried_discover)
+  {}
 
   mds_rank_t from_mds = -1;
   dirfrag_t dirfrag;
@@ -76,7 +97,7 @@ private:
   int32_t discover = 5;
   std::set<int32_t> dir_rep_by;
   filepath path;
-  int tried_discover = 0;
+  mutable int tried_discover = 0; // XXX HACK
 };
 
 #endif
index b891fa27c116404814f54839271b15765c24eb71..a1004f2e3252b1cff585161d43e946a11d29383e 100644 (file)
@@ -33,15 +33,17 @@ class MDiscover : public Message {
   bool want_xlocked = false;
 
  public:
-  inodeno_t get_base_ino() { return base_ino; }
-  frag_t    get_base_dir_frag() { return base_dir_frag; }
-  snapid_t  get_snapid() { return snapid; }
+  typedef boost::intrusive_ptr<MDiscover> ref;
+  typedef boost::intrusive_ptr<MDiscover const> const_ref;
+  inodeno_t get_base_ino() const { return base_ino; }
+  frag_t    get_base_dir_frag() const { return base_dir_frag; }
+  snapid_t  get_snapid() const { return snapid; }
 
-  filepath& get_want() { return want; }
-  const std::string& get_dentry(int n) { return want[n]; }
+  const filepath& get_want() const { return want; }
+  const std::string& get_dentry(int n) const { return want[n]; }
 
-  bool wants_base_dir() { return want_base_dir; }
-  bool wants_xlocked() { return want_xlocked; }
+  bool wants_base_dir() const { return want_base_dir; }
+  bool wants_xlocked() const { return want_xlocked; }
   
   void set_base_dir_frag(frag_t f) { base_dir_frag = f; }
 
index d6e0d8967cd949b7bb6688c64dfdd2f57b667367..8d86f267e44b8f4f2c70abff5f12369ebe41d4be 100644 (file)
@@ -83,47 +83,49 @@ class MDiscoverReply : public Message {
   mds_rank_t dir_auth_hint = 0;
 
  public:
+  typedef boost::intrusive_ptr<MDiscoverReply> ref;
+  typedef boost::intrusive_ptr<MDiscoverReply const> const_ref;
   __u8 starts_with = 0;
   bufferlist trace;
 
   enum { DIR, DENTRY, INODE };
 
   // accessors
-  inodeno_t get_base_ino() { return base_ino; }
-  frag_t get_base_dir_frag() { return base_dir_frag; }
-  bool get_wanted_base_dir() { return wanted_base_dir; }
-  bool get_wanted_xlocked() { return wanted_xlocked; }
-  snapid_t get_wanted_snapid() { return wanted_snapid; }
+  inodeno_t get_base_ino() const { return base_ino; }
+  frag_t get_base_dir_frag() const { return base_dir_frag; }
+  bool get_wanted_base_dir() const { return wanted_base_dir; }
+  bool get_wanted_xlocked() const { return wanted_xlocked; }
+  snapid_t get_wanted_snapid() const { return wanted_snapid; }
 
-  bool is_flag_error_dn() { return flag_error_dn; }
-  bool is_flag_error_dir() { return flag_error_dir; }
-  const std::string& get_error_dentry() { return error_dentry; }
+  bool is_flag_error_dn() const { return flag_error_dn; }
+  bool is_flag_error_dir() const { return flag_error_dir; }
+  const std::string& get_error_dentry() const { return error_dentry; }
 
-  int get_starts_with() { return starts_with; }
+  int get_starts_with() const { return starts_with; }
 
   mds_rank_t get_dir_auth_hint() const { return dir_auth_hint; }
 
-  bool is_unsolicited() { return unsolicited; }
+  bool is_unsolicited() const { return unsolicited; }
   void mark_unsolicited() { unsolicited = true; }
 
   void set_base_dir_frag(frag_t df) { base_dir_frag = df; }
 
   // cons
   MDiscoverReply() : Message(MSG_MDS_DISCOVERREPLY, HEAD_VERSION) { }
-  MDiscoverReply(MDiscover *dis) :
+  MDiscoverReply(const MDiscover &dis) :
     Message(MSG_MDS_DISCOVERREPLY, HEAD_VERSION),
-    base_ino(dis->get_base_ino()),
-    base_dir_frag(dis->get_base_dir_frag()),
-    wanted_base_dir(dis->wants_base_dir()),
-    wanted_xlocked(dis->wants_xlocked()),
-    wanted_snapid(dis->get_snapid()),
+    base_ino(dis.get_base_ino()),
+    base_dir_frag(dis.get_base_dir_frag()),
+    wanted_base_dir(dis.wants_base_dir()),
+    wanted_xlocked(dis.wants_xlocked()),
+    wanted_snapid(dis.get_snapid()),
     flag_error_dn(false),
     flag_error_dir(false),
     unsolicited(false),
     dir_auth_hint(CDIR_AUTH_UNKNOWN),
     starts_with(DIR)
   {
-    header.tid = dis->get_tid();
+    header.tid = dis.get_tid();
   }
   MDiscoverReply(dirfrag_t df) :
     Message(MSG_MDS_DISCOVERREPLY, HEAD_VERSION),
@@ -150,7 +152,7 @@ public:
   }
   
   // builders
-  bool is_empty() {
+  bool is_empty() const {
     return trace.length() == 0 &&
       !flag_error_dn &&
       !flag_error_dir &&
index 6d80b2eff82f2ae6c177d06269e82dfb41d70653..82d64935fab9aa987d75759c4bb11ffe441ac92c 100644 (file)
@@ -23,6 +23,8 @@ class MExportCaps : public Message {
   static const int HEAD_VERSION = 2;
   static const int COMPAT_VERSION = 1;
  public:  
+  typedef boost::intrusive_ptr<MExportCaps> ref;
+  typedef boost::intrusive_ptr<MExportCaps const> const_ref;
   inodeno_t ino;
   bufferlist cap_bl;
   map<client_t,entity_inst_t> client_map;
index 03b89bcd39d49dfd53a3a7ba9a95be6f72580796..2ff1643661f32cd4c046ae965a5feb9719300fcc 100644 (file)
@@ -21,6 +21,8 @@
 
 class MExportCapsAck : public Message {
  public:  
+  typedef boost::intrusive_ptr<MExportCapsAck> ref;
+  typedef boost::intrusive_ptr<MExportCapsAck const> const_ref;
   inodeno_t ino;
   bufferlist cap_bl;
 
index fc43720d81b4e0be1a34ce6d29c7fb7a84533e8f..29e9fa7fc04338b9d64e2c1f82cf3401cedfa177 100644 (file)
@@ -21,6 +21,8 @@
 
 class MExportDir : public Message {
  public:  
+  typedef boost::intrusive_ptr<MExportDir>ref;
+  typedef boost::intrusive_ptr<MExportDir const> const_ref;
   dirfrag_t dirfrag;
   bufferlist export_data;
   vector<dirfrag_t> bounds;
index dcf1c7d3a5c2819413145b160037a25081185308..ff076eca5d2acee418ae89e37d449ffb2f12cbec 100644 (file)
 #define CEPH_MEXPORTDIRACK_H
 
 #include "MExportDir.h"
+#include "msg/Message.h"
 
 class MExportDirAck : public Message {
 public:
+  typedef boost::intrusive_ptr<MExportDirAck> ref;
+  typedef boost::intrusive_ptr<MExportDirAck const> const_ref;
   dirfrag_t dirfrag;
   bufferlist imported_caps;
 
-  dirfrag_t get_dirfrag() { return dirfrag; }
+  dirfrag_t get_dirfrag() const { return dirfrag; }
   
   MExportDirAck() : Message(MSG_MDS_EXPORTDIRACK) {}
   MExportDirAck(dirfrag_t df, uint64_t tid) :
index 195aa316d43a72a4bb060f886f9ad9f073e983e1..96892a13a2879f32de21a63462a9a21c839a0e9b 100644 (file)
@@ -22,7 +22,9 @@ class MExportDirCancel : public Message {
   dirfrag_t dirfrag;
 
  public:
-  dirfrag_t get_dirfrag() { return dirfrag; }
+  typedef boost::intrusive_ptr<MExportDirCancel> ref;
+  typedef boost::intrusive_ptr<MExportDirCancel const> const_ref;
+  dirfrag_t get_dirfrag() const { return dirfrag; }
 
   MExportDirCancel() : Message(MSG_MDS_EXPORTDIRCANCEL) {}
   MExportDirCancel(dirfrag_t df, uint64_t tid) :
index c616939d7600ca69fca03fc16a3417d3dc1f6f7e..cb581d207db9433df483e2a20f001c1beac1288c 100644 (file)
@@ -24,10 +24,12 @@ class MExportDirDiscover : public Message {
   filepath path;
 
  public:
-  mds_rank_t get_source_mds() { return from; }
-  inodeno_t get_ino() { return dirfrag.ino; }
-  dirfrag_t get_dirfrag() { return dirfrag; }
-  filepath& get_path() { return path; }
+  typedef boost::intrusive_ptr<MExportDirDiscover> ref;
+  typedef boost::intrusive_ptr<MExportDirDiscover const> const_ref;
+  mds_rank_t get_source_mds() const { return from; }
+  inodeno_t get_ino() const { return dirfrag.ino; }
+  dirfrag_t get_dirfrag() const { return dirfrag; }
+  const filepath& get_path() const { return path; }
 
   bool started;
 
index 118d5f8c8e1fb16dc545fc9cd16b6e735284782e..c6bc716f9c574f501a731a3c8cb91beefa2b5c40 100644 (file)
@@ -23,9 +23,11 @@ class MExportDirDiscoverAck : public Message {
   bool success;
 
  public:
-  inodeno_t get_ino() { return dirfrag.ino; }
-  dirfrag_t get_dirfrag() { return dirfrag; }
-  bool is_success() { return success; }
+  typedef boost::intrusive_ptr<MExportDirDiscoverAck> ref;
+  typedef boost::intrusive_ptr<MExportDirDiscoverAck const> const_ref;
+  inodeno_t get_ino() const { return dirfrag.ino; }
+  dirfrag_t get_dirfrag() const { return dirfrag; }
+  bool is_success() const { return success; }
 
   MExportDirDiscoverAck() : Message(MSG_MDS_EXPORTDIRDISCOVERACK) {}
   MExportDirDiscoverAck(dirfrag_t df, uint64_t tid, bool s=true) :
index 38f4f0b51fe5d645cfc6bf4965d85eedef251777..6f7bb1aa2f8a588bec75300c8fd85970bc9209be 100644 (file)
@@ -22,8 +22,10 @@ class MExportDirFinish : public Message {
   bool last;
 
  public:
-  dirfrag_t get_dirfrag() { return dirfrag; }
-  bool is_last() { return last; }
+  typedef boost::intrusive_ptr<MExportDirFinish> ref;
+  typedef boost::intrusive_ptr<MExportDirFinish const> const_ref;
+  dirfrag_t get_dirfrag() const { return dirfrag; }
+  bool is_last() const { return last; }
   
   MExportDirFinish() : last(false) {}
   MExportDirFinish(dirfrag_t df, bool l, uint64_t tid) :
index 48190173fd1dffb232db142185f372f76dff8c06..21142a0a83f59ba5d2e329f10978e5809ab8c61e 100644 (file)
@@ -24,10 +24,13 @@ class MExportDirNotify : public Message {
   list<dirfrag_t> bounds;  // bounds; these dirs are _not_ included (tho the dirfragdes are)
 
  public:
-  dirfrag_t get_dirfrag() { return base; }
-  pair<__s32,__s32> get_old_auth() { return old_auth; }
-  pair<__s32,__s32> get_new_auth() { return new_auth; }
-  bool wants_ack() { return ack; }
+  typedef boost::intrusive_ptr<MExportDirNotify> ref;
+  typedef boost::intrusive_ptr<MExportDirNotify const> const_ref;
+  dirfrag_t get_dirfrag() const { return base; }
+  pair<__s32,__s32> get_old_auth() const { return old_auth; }
+  pair<__s32,__s32> get_new_auth() const { return new_auth; }
+  bool wants_ack() const { return ack; }
+  const list<dirfrag_t>& get_bounds() const { return bounds; }
   list<dirfrag_t>& get_bounds() { return bounds; }
 
   MExportDirNotify() {}
index 9248a376ddcc9e70c3da5c759b4abb327e0a2cf5..64c4a5272fde1b96b10d2332dc23a790b2e9359b 100644 (file)
@@ -22,8 +22,10 @@ class MExportDirNotifyAck : public Message {
   pair<__s32,__s32> new_auth;
 
  public:
-  dirfrag_t get_dirfrag() { return dirfrag; }
-  pair<__s32,__s32> get_new_auth() { return new_auth; }
+  typedef boost::intrusive_ptr<MExportDirNotifyAck>ref;
+  typedef boost::intrusive_ptr<MExportDirNotifyAck const> const_ref;
+  dirfrag_t get_dirfrag() const { return dirfrag; }
+  pair<__s32,__s32> get_new_auth() const { return new_auth; }
   
   MExportDirNotifyAck() {}
   MExportDirNotifyAck(dirfrag_t df, uint64_t tid, pair<__s32,__s32> na) :
index bb6540b04fc8b11c727d77249026340f2e010148..8eb0eda78c8f236145deb8eab0d02b234d75bdb0 100644 (file)
@@ -22,6 +22,8 @@
 class MExportDirPrep : public Message {
   dirfrag_t dirfrag;
  public:
+  typedef boost::intrusive_ptr<MExportDirPrep> ref;
+  typedef boost::intrusive_ptr<MExportDirPrep const> const_ref;
   bufferlist basedir;
   list<dirfrag_t> bounds;
   list<bufferlist> traces;
@@ -30,11 +32,11 @@ private:
   bool b_did_assim;
 
 public:
-  dirfrag_t get_dirfrag() { return dirfrag; }
-  list<dirfrag_t>& get_bounds() { return bounds; }
-  set<mds_rank_t> &get_bystanders() { return bystanders; }
+  dirfrag_t get_dirfrag() const { return dirfrag; }
+  const list<dirfrag_t>& get_bounds() const { return bounds; }
+  const set<mds_rank_t> &get_bystanders() const { return bystanders; }
 
-  bool did_assim() { return b_did_assim; }
+  bool did_assim() const { return b_did_assim; }
   void mark_assim() { b_did_assim = true; }
 
   MExportDirPrep() {
index b9155e2dc1565157b16503421cba498ef63bf8da..94a36bdcaa49f5dd4cc4c86c7847dde3e92c78db 100644 (file)
@@ -23,8 +23,10 @@ class MExportDirPrepAck : public Message {
   bool success = false;
 
  public:
-  dirfrag_t get_dirfrag() { return dirfrag; }
-  
+  typedef boost::intrusive_ptr<MExportDirPrepAck> ref;
+  typedef boost::intrusive_ptr<MExportDirPrepAck const> const_ref;
+  dirfrag_t get_dirfrag() const { return dirfrag; }
+
   MExportDirPrepAck() {}
   MExportDirPrepAck(dirfrag_t df, bool s, uint64_t tid) :
     Message(MSG_MDS_EXPORTDIRPREPACK), dirfrag(df), success(s) {
@@ -34,7 +36,7 @@ private:
   ~MExportDirPrepAck() override {}
 
 public:  
-  bool is_success() { return success; }
+  bool is_success() const { return success; }
   const char *get_type_name() const override { return "ExPAck"; }
   void print(ostream& o) const override {
     o << "export_prep_ack(" << dirfrag << (success ? " success)" : " fail)");
index 28737df212c9b345f5d5754cc91f72f29078363c..73d8d08d4091a8b18f83746a38e74f4727b37675 100644 (file)
@@ -5,7 +5,10 @@
 
 
 class MGatherCaps : public Message {
- public:
+public:
+  typedef boost::intrusive_ptr<MGatherCaps> ref;
+  typedef boost::intrusive_ptr<MGatherCaps const> const_ref;
+
   inodeno_t ino;
 
   MGatherCaps() :
index 367a1b3733d8d7a2f2e6245677c0eaea4b35a982..3d7f98dd02c97394ed700fd22fa324546870bfc3 100644 (file)
@@ -26,12 +26,13 @@ class MHeartbeat : public Message {
   map<mds_rank_t, float> import_map;
 
  public:
-  mds_load_t& get_load() { return load; }
-  int get_beat() { return beat; }
+  typedef boost::intrusive_ptr<MHeartbeat> ref;
+  typedef boost::intrusive_ptr<MHeartbeat const> const_ref;
+  const mds_load_t& get_load() const { return load; }
+  int get_beat() const { return beat; }
 
-  map<mds_rank_t, float>& get_import_map() {
-    return import_map;
-  }
+  const map<mds_rank_t, float>& get_import_map() const { return import_map; }
+  map<mds_rank_t, float>& get_import_map() { return import_map; }
 
   MHeartbeat() : Message(MSG_MDS_HEARTBEAT), load(DecayRate()) {}
   MHeartbeat(mds_load_t& load, int beat)
index de7d1fb276fa74885d29cd51f5bdc9cb6aded71b..a5e5c541652fa567d8a051e421a86a291dcd0f60 100644 (file)
 #ifndef CEPH_MINODEFILECAPS_H
 #define CEPH_MINODEFILECAPS_H
 
+#include "msg/Message.h"
+
 class MInodeFileCaps : public Message {
   inodeno_t ino;
   __u32     caps = 0;
 
  public:
-  inodeno_t get_ino() { return ino; }
-  int       get_caps() { return caps; }
+  typedef boost::intrusive_ptr<MInodeFileCaps> ref;
+  typedef boost::intrusive_ptr<MInodeFileCaps const> const_ref;
+
+  inodeno_t get_ino() const { return ino; }
+  int       get_caps() const { return caps; }
 
   MInodeFileCaps() : Message(MSG_MDS_INODEFILECAPS) {}
   MInodeFileCaps(inodeno_t ino, int caps) :
diff --git a/src/messages/MInterMDS.h b/src/messages/MInterMDS.h
new file mode 100644 (file)
index 0000000..fecd737
--- /dev/null
@@ -0,0 +1,42 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (c) 2018 Red Hat, Inc.
+ *
+ * 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
+ * Foundation.  See file COPYING.
+ * 
+ */
+
+
+#ifndef CEPH_MINTERMDS_H
+#define CEPH_MINTERMDS_H
+
+#include "include/types.h"
+#include "include/fs_types.h"
+
+#include "msg/Message.h"
+
+class MInterMDS : public Message {
+public:
+  typedef boost::intrusive_ptr<MInterMDS> ref;
+  typedef boost::intrusive_ptr<MInterMDS const> const_ref;
+
+  template <typename... T>
+  MInterMDS(T&&... args) : Message(std::forward<T>(args)...) {}
+
+  virtual bool is_forwardable() const { return false; }
+
+  // N.B. only some types of messages we should be 'forwarding'; they
+  // explicitly encode their source mds, which gets clobbered when resent
+  virtual MInterMDS::ref forwardable() const {ceph_abort();}
+
+protected:
+  virtual ~MInterMDS() {}
+};
+
+#endif
index 45086214d4edbcbe71719cd947c2764aa059660f..4b992ff1bc8bd5d9862debc032e082be43ad6632 100644 (file)
 
 #include "msg/Message.h"
 #include "mds/locks.h"
+#include "mds/SimpleLock.h"
 
 class MLock : public Message {
   int32_t     action = 0;  // action type
-  int32_t     asker = 0;  // who is initiating this request
+  mds_rank_t  asker = 0;  // who is initiating this request
   metareqid_t reqid;  // for remote lock requests
   
   __u16      lock_type = 0;  // lock object type
@@ -30,26 +31,31 @@ class MLock : public Message {
   bufferlist lockdata;  // and possibly some data
   
 public:
+  typedef boost::intrusive_ptr<MLock> ref;
+  typedef boost::intrusive_ptr<MLock const> const_ref;
+
   bufferlist& get_data() { return lockdata; }
-  int get_asker() { return asker; }
-  int get_action() { return action; }
-  metareqid_t get_reqid() { return reqid; }
+  const bufferlist& get_data() const { return lockdata; }
+  int get_asker() const { return asker; }
+  int get_action() const { return action; }
+  metareqid_t get_reqid() const { return reqid; }
   
-  int get_lock_type() { return lock_type; }
+  int get_lock_type() const { return lock_type; }
+  const MDSCacheObjectInfo &get_object_info() const { return object_info; }
   MDSCacheObjectInfo &get_object_info() { return object_info; }
   
   MLock() : Message(MSG_MDS_LOCK) {}
-  MLock(int ac, int as) :
+  MLock(int ac, mds_rank_t as) :
     Message(MSG_MDS_LOCK),
     action(ac), asker(as),
     lock_type(0) { }
-  MLock(SimpleLock *lock, int ac, int as) :
+  MLock(SimpleLock *lock, int ac, mds_rank_t as) :
     Message(MSG_MDS_LOCK),
     action(ac), asker(as),
     lock_type(lock->get_type()) {
     lock->get_parent()->set_object_info(object_info);
   }
-  MLock(SimpleLock *lock, int ac, int as, bufferlist& bl) :
+  MLock(SimpleLock *lock, int ac, mds_rank_t as, bufferlist& bl) :
     Message(MSG_MDS_LOCK),
     action(ac), asker(as), lock_type(lock->get_type()) {
     lock->get_parent()->set_object_info(object_info);
index 3ba46ed7fd9761f0427c78c60b22182aff5a7cbc..036c6d5a991e4972e99038189ac2f1f683aa08ff 100644 (file)
@@ -17,6 +17,7 @@
 
 #include <string_view>
 
+#include "msg/Message.h"
 #include "messages/PaxosServiceMessage.h"
 
 #include "include/types.h"
@@ -204,6 +205,9 @@ class MMDSBeacon : public PaxosServiceMessage {
   uint64_t mds_features;
 
  public:
+  typedef boost::intrusive_ptr<MMDSBeacon> ref;
+  typedef boost::intrusive_ptr<MMDSBeacon const> const_ref;
+
   MMDSBeacon()
     : PaxosServiceMessage(MSG_MDS_BEACON, 0, HEAD_VERSION, COMPAT_VERSION),
     global_id(0), state(MDSMap::STATE_NULL), standby_for_rank(MDS_RANK_NONE),
@@ -211,7 +215,7 @@ class MMDSBeacon : public PaxosServiceMessage {
     mds_features(0) {
     set_priority(CEPH_MSG_PRIO_HIGH);
   }
-  MMDSBeacon(const uuid_d &f, mds_gid_t g, string& n, epoch_t les, MDSMap::DaemonState st, version_t se, uint64_t feat) :
+  MMDSBeacon(const uuid_d &f, mds_gid_t g, const string& n, epoch_t les, MDSMap::DaemonState st, version_t se, uint64_t feat) :
     PaxosServiceMessage(MSG_MDS_BEACON, les, HEAD_VERSION, COMPAT_VERSION),
     fsid(f), global_id(g), name(n), state(st), seq(se),
     standby_for_rank(MDS_RANK_NONE), standby_for_fscid(FS_CLUSTER_ID_NONE),
@@ -222,16 +226,16 @@ private:
   ~MMDSBeacon() override {}
 
 public:
-  uuid_d& get_fsid() { return fsid; }
-  mds_gid_t get_global_id() { return global_id; }
-  string& get_name() { return name; }
-  epoch_t get_last_epoch_seen() { return version; }
-  MDSMap::DaemonState get_state() { return state; }
-  version_t get_seq() { return seq; }
+  const uuid_d& get_fsid() const { return fsid; }
+  mds_gid_t get_global_id() const { return global_id; }
+  const string& get_name() const { return name; }
+  epoch_t get_last_epoch_seen() const { return version; }
+  MDSMap::DaemonState get_state() const { return state; }
+  version_t get_seq() const { return seq; }
   const char *get_type_name() const override { return "mdsbeacon"; }
-  mds_rank_t get_standby_for_rank() { return standby_for_rank; }
-  const string& get_standby_for_name() { return standby_for_name; }
-  const fs_cluster_id_t& get_standby_for_fscid() { return standby_for_fscid; }
+  mds_rank_t get_standby_for_rank() const { return standby_for_rank; }
+  const string& get_standby_for_name() const { return standby_for_name; }
+  const fs_cluster_id_t& get_standby_for_fscid() const { return standby_for_fscid; }
   bool get_standby_replay() const { return standby_replay; }
   uint64_t get_mds_features() const { return mds_features; }
 
index ead58765a559572defdd5d34705f076a48a1748e..c49c2ecebddca35e9dac46b9e0149a57b4274125 100644 (file)
@@ -33,6 +33,9 @@ class MMDSCacheRejoin : public Message {
   static const int COMPAT_VERSION = 1;
 
  public:
+  typedef boost::intrusive_ptr<MMDSCacheRejoin> ref;
+  typedef boost::intrusive_ptr<MMDSCacheRejoin const> const_ref;
+
   static const int OP_WEAK    = 1;  // replica -> auth, i exist, + maybe open files.
   static const int OP_STRONG  = 2;  // replica -> auth, i exist, + open files and lock state.
   static const int OP_ACK     = 3;  // auth -> replica, here is your lock state.
@@ -102,9 +105,9 @@ class MMDSCacheRejoin : public Message {
       ino(0), remote_ino(0), remote_d_type(0), nonce(0), lock(0) {}
     dn_strong(snapid_t f, inodeno_t pi, inodeno_t ri, unsigned char rdt, int n, int l) : 
       first(f), ino(pi), remote_ino(ri), remote_d_type(rdt), nonce(n), lock(l) {}
-    bool is_primary() { return ino > 0; }
-    bool is_remote() { return remote_ino > 0; }
-    bool is_null() { return ino == 0 && remote_ino == 0; }
+    bool is_primary() const { return ino > 0; }
+    bool is_remote() const { return remote_ino > 0; }
+    bool is_null() const { return ino == 0 && remote_ino == 0; }
     void encode(bufferlist &bl) const {
       using ceph::encode;
       encode(first, bl);
index 06db79141387025f7f8f3495f0efac2591485de4..2d0cd8c6c6994dc6c75426ddaf1b1a691b40f79d 100644 (file)
 #include "msg/Message.h"
 #include "include/filepath.h"
 
-struct MMDSFindIno : public Message {
+class MMDSFindIno : public Message {
+public:
+  typedef boost::intrusive_ptr<MMDSFindIno> ref;
+  typedef boost::intrusive_ptr<MMDSFindIno const> const_ref;
   ceph_tid_t tid {0};
   inodeno_t ino;
 
index 46eef3f7775467df434a974b0f8196151339e85c..4680dd90d8a6b6cb441d3e717a0e1c9064f0d386 100644 (file)
 #include "msg/Message.h"
 #include "include/filepath.h"
 
-struct MMDSFindInoReply : public Message {
+class MMDSFindInoReply : public Message {
+public:
+  typedef boost::intrusive_ptr<MMDSFindInoReply> ref;
+  typedef boost::intrusive_ptr<MMDSFindInoReply const> const_ref;
   ceph_tid_t tid = 0;
   filepath path;
 
index f60909649bddafa5d67df1f572784d2c65635754..650bba7d0eb51c61d2331cdb739ee16e51ccb71a 100644 (file)
@@ -23,9 +23,11 @@ class MMDSFragmentNotify : public Message {
   int8_t bits = 0;
 
  public:
-  inodeno_t get_ino() { return ino; }
-  frag_t get_basefrag() { return basefrag; }
-  int get_bits() { return bits; }
+  typedef boost::intrusive_ptr<MMDSFragmentNotify> ref;
+  typedef boost::intrusive_ptr<MMDSFragmentNotify const> const_ref;
+  inodeno_t get_ino() const { return ino; }
+  frag_t get_basefrag() const { return basefrag; }
+  int get_bits() const { return bits; }
 
   bufferlist basebl;
 
index fea57ee7a22db014bf4046b367733c602e1c78e5..7f9f764d6922f11ee2d76d011fc0174303467b9c 100644 (file)
@@ -24,21 +24,23 @@ class MMDSMap : public Message {
   static const int HEAD_VERSION = 1;
   static const int COMPAT_VERSION = 1;
 public:
+  typedef boost::intrusive_ptr<MMDSMap> ref;
+  typedef boost::intrusive_ptr<MMDSMap const> const_ref;
 
   uuid_d fsid;
   epoch_t epoch = 0;
   bufferlist encoded;
 
   version_t get_epoch() const { return epoch; }
-  bufferlist& get_encoded() { return encoded; }
+  const bufferlist& get_encoded() const { return encoded; }
 
   MMDSMap() : 
     Message(CEPH_MSG_MDS_MAP, HEAD_VERSION, COMPAT_VERSION) {}
-  MMDSMap(const uuid_d &f, const MDSMap *mm) :
+  MMDSMap(const uuid_d &f, const MDSMap &mm) :
     Message(CEPH_MSG_MDS_MAP, HEAD_VERSION, COMPAT_VERSION),
     fsid(f) {
-    epoch = mm->get_epoch();
-    mm->encode(encoded, -1);  // we will reencode with fewer features as necessary
+    epoch = mm.get_epoch();
+    mm.encode(encoded, -1);  // we will reencode with fewer features as necessary
   }
 private:
   ~MMDSMap() override {}
index ea9d67cd8f3c0c4190f43fb4ac97aebbec3ea1b7..a7f775302bbe7315d88bf38d7f6a512bed71af90 100644 (file)
@@ -18,6 +18,9 @@
 #include "msg/Message.h"
 
 struct MMDSOpenIno : public Message {
+public:
+  typedef boost::intrusive_ptr<MMDSOpenIno> ref;
+  typedef boost::intrusive_ptr<MMDSOpenIno const> const_ref;
   inodeno_t ino;
   vector<inode_backpointer_t> ancestors;
 
index b52e33e341df9ad3f82a4e1ada6a35d721d1e6c8..555b9fd33c633b58f017a58120b89eb1cea786b1 100644 (file)
 
 #include "msg/Message.h"
 
-struct MMDSOpenInoReply : public Message {
+class MMDSOpenInoReply : public Message {
+public:
+  typedef boost::intrusive_ptr<MMDSOpenInoReply> ref;
+  typedef boost::intrusive_ptr<MMDSOpenInoReply const> const_ref;
+
   inodeno_t ino;
   vector<inode_backpointer_t> ancestors;
   mds_rank_t hint;
index a158a0c83db38590c3af7a678af9c574a4b46913..65de1f313312eb0405c0391146da0daa510f5d3a 100644 (file)
@@ -21,6 +21,8 @@
 
 class MMDSResolve : public Message {
 public:
+  typedef boost::intrusive_ptr<MMDSResolve> ref;
+  typedef boost::intrusive_ptr<MMDSResolve const> const_ref;
   map<dirfrag_t, vector<dirfrag_t> > subtrees;
   map<dirfrag_t, vector<dirfrag_t> > ambiguous_imports;
 
index 38b288523bd6dbebb5c494d3653969c5425746ea..68cd164124da54c88c6ef106253585ee90198804 100644 (file)
@@ -22,6 +22,8 @@
 
 class MMDSResolveAck : public Message {
  public:
+  typedef boost::intrusive_ptr<MMDSResolveAck> ref;
+  typedef boost::intrusive_ptr<MMDSResolveAck const> const_ref;
   map<metareqid_t, bufferlist> commit;
   vector<metareqid_t> abort;
 
index 78ef3c871207123415c25486e6e7a2ca929c0030..2b688921e1cbee8dffb84bc6665d4ef5d9076c0c 100644 (file)
@@ -21,6 +21,8 @@
 
 class MMDSSlaveRequest : public Message {
  public:
+  typedef boost::intrusive_ptr<MMDSSlaveRequest> ref;
+  typedef boost::intrusive_ptr<MMDSSlaveRequest const> const_ref;
   static const int OP_XLOCK =       1;
   static const int OP_XLOCKACK =   -1;
   static const int OP_UNXLOCK =     2;
@@ -53,7 +55,7 @@ class MMDSSlaveRequest : public Message {
   //static const int OP_COMMIT = 21;  // used for recovery only
 
 
-  const static char *get_opname(int o) {
+  static const char *get_opname(int o) {
     switch (o) { 
     case OP_XLOCK: return "xlock";
     case OP_XLOCKACK: return "xlock_ack";
@@ -94,7 +96,7 @@ class MMDSSlaveRequest : public Message {
   metareqid_t reqid;
   __u32 attempt;
   __s16 op;
-  __u16 flags;
+  mutable __u16 flags; /* XXX HACK for mark_interrupted */
 
   static const unsigned FLAG_NONBLOCK  =       1<<0;
   static const unsigned FLAG_WOULDBLOCK        =       1<<1;
@@ -120,35 +122,39 @@ class MMDSSlaveRequest : public Message {
   mds_rank_t srcdn_auth;
   utime_t op_stamp;
 
-  bufferlist straybl;  // stray dir + dentry
+  mutable bufferlist straybl;  // stray dir + dentry
   bufferlist srci_snapbl;
   bufferlist desti_snapbl;
 
 public:
-  metareqid_t get_reqid() { return reqid; }
+  metareqid_t get_reqid() const { return reqid; }
   __u32 get_attempt() const { return attempt; }
-  int get_op() { return op; }
-  bool is_reply() { return op < 0; }
+  int get_op() const { return op; }
+  bool is_reply() const { return op < 0; }
 
-  int get_lock_type() { return lock_type; }
+  int get_lock_type() const { return lock_type; }
+  const MDSCacheObjectInfo &get_object_info() const { return object_info; }
   MDSCacheObjectInfo &get_object_info() { return object_info; }
+  const MDSCacheObjectInfo &get_authpin_freeze() const { return object_info; }
   MDSCacheObjectInfo &get_authpin_freeze() { return object_info; }
 
+  const vector<MDSCacheObjectInfo>& get_authpins() const { return authpins; }
   vector<MDSCacheObjectInfo>& get_authpins() { return authpins; }
   void mark_nonblock() { flags |= FLAG_NONBLOCK; }
-  bool is_nonblock() { return (flags & FLAG_NONBLOCK); }
+  bool is_nonblock() const { return (flags & FLAG_NONBLOCK); }
   void mark_error_wouldblock() { flags |= FLAG_WOULDBLOCK; }
-  bool is_error_wouldblock() { return (flags & FLAG_WOULDBLOCK); }
+  bool is_error_wouldblock() const { return (flags & FLAG_WOULDBLOCK); }
   void mark_not_journaled() { flags |= FLAG_NOTJOURNALED; }
-  bool is_not_journaled() { return (flags & FLAG_NOTJOURNALED); }
+  bool is_not_journaled() const { return (flags & FLAG_NOTJOURNALED); }
   void mark_error_rofs() { flags |= FLAG_EROFS; }
-  bool is_error_rofs() { return (flags & FLAG_EROFS); }
-  bool is_abort() { return (flags & FLAG_ABORT); }
+  bool is_error_rofs() const { return (flags & FLAG_EROFS); }
+  bool is_abort() const { return (flags & FLAG_ABORT); }
   void mark_abort() { flags |= FLAG_ABORT; }
-  bool is_interrupted() { return (flags & FLAG_INTERRUPTED); }
-  void mark_interrupted() { flags |= FLAG_INTERRUPTED; }
+  bool is_interrupted() const { return (flags & FLAG_INTERRUPTED); }
+  void mark_interrupted() const { flags |= FLAG_INTERRUPTED; }
 
   void set_lock_type(int t) { lock_type = t; }
+  const bufferlist& get_lock_data() const { return inode_export; }
   bufferlist& get_lock_data() { return inode_export; }
 
 
index 2e132c1e362d05850fe97cd9cc02a5269604dc71..0c848ca03828162214f3a5e3453b4905d90237e0 100644 (file)
@@ -22,8 +22,10 @@ class MMDSSnapUpdate : public Message {
   __s16 snap_op;
 
 public:
-  inodeno_t get_ino() { return ino; }
-  int get_snap_op() { return snap_op; }
+  typedef boost::intrusive_ptr<MMDSSnapUpdate> ref;
+  typedef boost::intrusive_ptr<MMDSSnapUpdate const> const_ref;
+  inodeno_t get_ino() const { return ino; }
+  int get_snap_op() const { return snap_op; }
 
   bufferlist snap_blob;
 
index 13355bdccf605f04cc0f6e8f033e6076efbc55aa..cac5d33f8223a60da6c2a615a3d3ab5e1421bc19 100644 (file)
@@ -21,6 +21,9 @@
 
 class MMDSTableRequest : public Message {
  public:
+  typedef boost::intrusive_ptr<MMDSTableRequest> ref;
+  typedef boost::intrusive_ptr<MMDSTableRequest const> const_ref;
+
   __u16 table = 0;
   __s16 op = 0;
   uint64_t reqid = 0;
index 183b4001dd27f9a2593348e1edd9e9ad29d9c3db..768dffbb92d8f3cfc9b9cbc2361a0efd315d9a7d 100644 (file)
@@ -376,7 +376,7 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op)
       MDSMap null_map;
       null_map.epoch = fsmap.epoch;
       null_map.compat = fsmap.compat;
-      mon->send_reply(op, new MMDSMap(mon->monmap->fsid, &null_map));
+      mon->send_reply(op, new MMDSMap(mon->monmap->fsid, null_map));
       return true;
     } else {
       return false;  // not booted yet.
@@ -826,7 +826,7 @@ void MDSMonitor::_updated(MonOpRequestRef op)
     MDSMap null_map;
     null_map.epoch = fsmap.epoch;
     null_map.compat = fsmap.compat;
-    mon->send_reply(op, new MMDSMap(mon->monmap->fsid, &null_map));
+    mon->send_reply(op, new MMDSMap(mon->monmap->fsid, null_map));
   } else {
     mon->send_reply(op, new MMDSBeacon(mon->monmap->fsid,
                                       m->get_global_id(),
@@ -1570,7 +1570,7 @@ void MDSMonitor::check_sub(Subscription *sub)
     if (sub->next > mds_map->epoch) {
       return;
     }
-    auto msg = new MMDSMap(mon->monmap->fsid, mds_map);
+    auto msg = new MMDSMap(mon->monmap->fsid, *mds_map);
 
     sub->session->con->send_message(msg);
     if (sub->onetime) {