]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/mds/CInode.h
update source to Ceph Pacific 16.2.2
[ceph.git] / ceph / src / mds / CInode.h
index 1930c700591afdd3326d335fca9e63427ac8e5d0..177e02d063209fdbb513d49d211d22eafd704249 100644 (file)
@@ -21,6 +21,8 @@
 #include <string_view>
 
 #include "common/config.h"
+#include "common/RefCountedObj.h"
+#include "include/compat.h"
 #include "include/counter.h"
 #include "include/elist.h"
 #include "include/types.h"
@@ -35,7 +37,7 @@
 #include "CDentry.h"
 #include "SimpleLock.h"
 #include "ScatterLock.h"
-#include "LocalLock.h"
+#include "LocalLockC.h"
 #include "Capability.h"
 #include "SnapRealm.h"
 #include "Mutation.h"
@@ -59,6 +61,34 @@ struct cinode_lock_info_t {
   int wr_caps;
 };
 
+struct CInodeCommitOperation {
+public:
+  CInodeCommitOperation(int prio, int64_t po)
+    : pool(po), priority(prio) {
+  }
+  CInodeCommitOperation(int prio, int64_t po, file_layout_t l, uint64_t f)
+    : pool(po), priority(prio), _layout(l), _features(f) {
+      update_layout = true;
+  }
+
+  void update(ObjectOperation &op, inode_backtrace_t &bt);
+  int64_t get_pool() { return pool; }
+
+private:
+  int64_t pool;     ///< pool id
+  int priority;
+  bool update_layout = false;
+  file_layout_t _layout;
+  uint64_t _features;
+};
+
+struct CInodeCommitOperations {
+  std::vector<CInodeCommitOperation> ops_vec;
+  inode_backtrace_t bt;
+  version_t version;
+  CInode *in;
+};
+
 /**
  * Base class for CInode, containing the backing store data and
  * serialization methods.  This exists so that we can read and
@@ -67,41 +97,90 @@ struct cinode_lock_info_t {
  */
 class InodeStoreBase {
 public:
-  typedef inode_t<mempool::mds_co::pool_allocator> mempool_inode;
-  typedef old_inode_t<mempool::mds_co::pool_allocator> mempool_old_inode;
-  typedef mempool::mds_co::compact_map<snapid_t, mempool_old_inode> mempool_old_inode_map;
-  typedef xattr_map<mempool::mds_co::pool_allocator> mempool_xattr_map; // FIXME bufferptr not in mempool
+  using mempool_inode = inode_t<mempool::mds_co::pool_allocator>;
+  using inode_ptr = std::shared_ptr<mempool_inode>;
+  using inode_const_ptr = std::shared_ptr<const mempool_inode>;
+
+  template <typename ...Args>
+  static inode_ptr allocate_inode(Args && ...args) {
+    static mempool::mds_co::pool_allocator<mempool_inode> allocator;
+    return std::allocate_shared<mempool_inode>(allocator, std::forward<Args>(args)...);
+  }
+  
+  using mempool_xattr_map = xattr_map<mempool::mds_co::pool_allocator>; // FIXME bufferptr not in mempool
+  using xattr_map_ptr = std::shared_ptr<mempool_xattr_map>;
+  using xattr_map_const_ptr = std::shared_ptr<const mempool_xattr_map>;
+
+  template <typename ...Args>
+  static xattr_map_ptr allocate_xattr_map(Args && ...args) {
+    static mempool::mds_co::pool_allocator<mempool_xattr_map> allocator;
+    return std::allocate_shared<mempool_xattr_map>(allocator, std::forward<Args>(args)...);
+  }
+
+  using mempool_old_inode = old_inode_t<mempool::mds_co::pool_allocator>;
+  using mempool_old_inode_map = mempool::mds_co::map<snapid_t, mempool_old_inode>;
+  using old_inode_map_ptr = std::shared_ptr<mempool_old_inode_map>;
+  using old_inode_map_const_ptr = std::shared_ptr<const mempool_old_inode_map>;
 
-  InodeStoreBase() {}
+  template <typename ...Args>
+  static old_inode_map_ptr allocate_old_inode_map(Args && ...args) {
+    static mempool::mds_co::pool_allocator<mempool_old_inode_map> allocator;
+    return std::allocate_shared<mempool_old_inode_map>(allocator, std::forward<Args>(args)...);
+  }
+
+  void reset_inode(inode_const_ptr&& ptr) {
+    inode = std::move(ptr);
+  }
+
+  void reset_xattrs(xattr_map_const_ptr&& ptr) {
+    xattrs = std::move(ptr);
+  }
+
+  void reset_old_inodes(old_inode_map_const_ptr&& ptr) {
+    old_inodes = std::move(ptr);
+  }
+
+  void encode_xattrs(bufferlist &bl) const;
+  void decode_xattrs(bufferlist::const_iterator &p);
+  void encode_old_inodes(bufferlist &bl, uint64_t features) const;
+  void decode_old_inodes(bufferlist::const_iterator &p);
 
   /* Helpers */
-  bool is_file() const    { return inode.is_file(); }
-  bool is_symlink() const { return inode.is_symlink(); }
-  bool is_dir() const     { return inode.is_dir(); }
   static object_t get_object_name(inodeno_t ino, frag_t fg, std::string_view suffix);
 
   /* Full serialization for use in ".inode" root inode objects */
-  void encode(bufferlist &bl, uint64_t features, const bufferlist *snap_blob=NULL) const;
-  void decode(bufferlist::const_iterator &bl, bufferlist& snap_blob);
+  void encode(ceph::buffer::list &bl, uint64_t features, const ceph::buffer::list *snap_blob=NULL) const;
+  void decode(ceph::buffer::list::const_iterator &bl, ceph::buffer::list& snap_blob);
 
   /* Serialization without ENCODE_START/FINISH blocks for use embedded in dentry */
-  void encode_bare(bufferlist &bl, uint64_t features, const bufferlist *snap_blob=NULL) const;
-  void decode_bare(bufferlist::const_iterator &bl, bufferlist &snap_blob, __u8 struct_v=5);
+  void encode_bare(ceph::buffer::list &bl, uint64_t features, const ceph::buffer::list *snap_blob=NULL) const;
+  void decode_bare(ceph::buffer::list::const_iterator &bl, ceph::buffer::list &snap_blob, __u8 struct_v=5);
 
   /* For test/debug output */
-  void dump(Formatter *f) const;
+  void dump(ceph::Formatter *f) const;
 
+  void decode_json(JSONObj *obj);
+  static void xattrs_cb(InodeStoreBase::mempool_xattr_map& c, JSONObj *obj);
+  static void old_indoes_cb(InodeStoreBase::mempool_old_inode_map& c, JSONObj *obj);
+  
   /* For use by offline tools */
   __u32 hash_dentry_name(std::string_view dn);
   frag_t pick_dirfrag(std::string_view dn);
 
-  mempool_inode inode;        // the inode itself
-  mempool::mds_co::string symlink;      // symlink dest, if symlink
-  mempool_xattr_map xattrs;
-  fragtree_t dirfragtree;  // dir frag tree, if any.  always consistent with our dirfrag map.
-  mempool_old_inode_map old_inodes;   // key = last, value.first = first
-  snapid_t oldest_snap = CEPH_NOSNAP;
-  damage_flags_t damage_flags = 0;
+  mempool::mds_co::string      symlink;      // symlink dest, if symlink
+  fragtree_t                   dirfragtree;  // dir frag tree, if any.  always consistent with our dirfrag map.
+  snapid_t                     oldest_snap = CEPH_NOSNAP;
+  damage_flags_t               damage_flags = 0;
+
+protected:
+  static inode_const_ptr       empty_inode;
+
+  // Following members are pointers to constant data, the constant data can
+  // be shared by CInode and log events. To update these members in CInode,
+  // read-copy-update should be used.
+  inode_const_ptr              inode = empty_inode;
+  xattr_map_const_ptr          xattrs;
+  old_inode_map_const_ptr      old_inodes;   // key = last, value.first = first
 };
 
 inline void decode_noshare(InodeStoreBase::mempool_xattr_map& xattrs,
@@ -112,23 +191,34 @@ inline void decode_noshare(InodeStoreBase::mempool_xattr_map& xattrs,
 
 class InodeStore : public InodeStoreBase {
 public:
-  void encode(bufferlist &bl, uint64_t features) const {
+  mempool_inode* get_inode() {
+    if (inode == empty_inode)
+      reset_inode(allocate_inode());
+    return const_cast<mempool_inode*>(inode.get());
+  }
+  mempool_xattr_map* get_xattrs() { return const_cast<mempool_xattr_map*>(xattrs.get()); }
+
+  void encode(ceph::buffer::list &bl, uint64_t features) const {
     InodeStoreBase::encode(bl, features, &snap_blob);
   }
-  void decode(bufferlist::const_iterator &bl) {
+  void decode(ceph::buffer::list::const_iterator &bl) {
     InodeStoreBase::decode(bl, snap_blob);
   }
-  void encode_bare(bufferlist &bl, uint64_t features) const {
+  void encode_bare(ceph::buffer::list &bl, uint64_t features) const {
     InodeStoreBase::encode_bare(bl, features, &snap_blob);
   }
-  void decode_bare(bufferlist::const_iterator &bl) {
+  void decode_bare(ceph::buffer::list::const_iterator &bl) {
     InodeStoreBase::decode_bare(bl, snap_blob);
   }
 
   static void generate_test_instances(std::list<InodeStore*>& ls);
 
+  using InodeStoreBase::inode;
+  using InodeStoreBase::xattrs;
+  using InodeStoreBase::old_inodes;
+
   // FIXME bufferlist not part of mempool
-  bufferlist snap_blob;  // Encoded copy of SnapRealm, because we can't
+  ceph::buffer::list snap_blob;  // Encoded copy of SnapRealm, because we can't
                         // rehydrate it without full MDCache
 };
 WRITE_CLASS_ENCODER_FEATURES(InodeStore)
@@ -136,10 +226,10 @@ WRITE_CLASS_ENCODER_FEATURES(InodeStore)
 // just for ceph-dencoder
 class InodeStoreBare : public InodeStore {
 public:
-  void encode(bufferlist &bl, uint64_t features) const {
+  void encode(ceph::buffer::list &bl, uint64_t features) const {
     InodeStore::encode_bare(bl, features);
   }
-  void decode(bufferlist::const_iterator &bl) {
+  void decode(ceph::buffer::list::const_iterator &bl) {
     InodeStore::decode_bare(bl);
   }
   static void generate_test_instances(std::list<InodeStoreBare*>& ls);
@@ -184,7 +274,7 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
 
     validated_data() {}
 
-    void dump(Formatter *f) const;
+    void dump(ceph::Formatter *f) const;
 
     bool all_damage_repaired() const;
 
@@ -203,68 +293,23 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   friend class MDCache;
   friend class StrayManager;
   friend class CDir;
-  friend ostream& operator<<(ostream&, const CInode&);
+  friend std::ostream& operator<<(std::ostream&, const CInode&);
 
-  class scrub_stamp_info_t {
-  public:
-    scrub_stamp_info_t() {}
-    void reset() {
-      scrub_start_version = last_scrub_version = 0;
-      scrub_start_stamp = last_scrub_stamp = utime_t();
-    }
-    /// version we started our latest scrub (whether in-progress or finished)
-    version_t scrub_start_version = 0;
-    /// time we started our latest scrub (whether in-progress or finished)
-    utime_t scrub_start_stamp;
-    /// version we started our most recent finished scrub
-    version_t last_scrub_version = 0;
-    /// time we started our most recent finished scrub
-    utime_t last_scrub_stamp;
-  };
-
-  class scrub_info_t : public scrub_stamp_info_t {
+  class scrub_info_t {
   public:
     scrub_info_t() {}
 
-    CDentry *scrub_parent = nullptr;
-    MDSContext *on_finish = nullptr;
+    version_t last_scrub_version = 0;
+    utime_t last_scrub_stamp;
 
     bool last_scrub_dirty = false; /// are our stamps dirty with respect to disk state?
     bool scrub_in_progress = false; /// are we currently scrubbing?
-    bool children_scrubbed = false;
 
-    /// my own (temporary) stamps and versions for each dirfrag we have
-    std::map<frag_t, scrub_stamp_info_t> dirfrag_stamps; // XXX not part of mempool
+    fragset_t queued_frags;
 
     ScrubHeaderRef header;
   };
 
-  /**
-   * Projection methods, used to store inode changes until they have been journaled,
-   * at which point they are popped.
-   * Usage:
-   * project_inode as needed. If you're changing xattrs or sr_t, then pass true
-   * as needed then change the xattrs/snapnode member as needed. (Dirty
-   * exception: project_past_snaprealm_parent allows you to project the
-   * snapnode after doing project_inode (i.e. you don't need to pass
-   * snap=true).
-   *
-   * Then, journal. Once journaling is done, pop_and_dirty_projected_inode.
-   * This function will take care of the inode itself, the xattrs, and the snaprealm.
-   */
-
-  class projected_inode {
-  public:
-    static sr_t* const UNDEF_SRNODE;
-
-    projected_inode() = delete;
-    explicit projected_inode(const mempool_inode &in) : inode(in) {}
-
-    mempool_inode inode;
-    std::unique_ptr<mempool_xattr_map> xattrs;
-    sr_t *snapnode = UNDEF_SRNODE;
-  };
-
   // -- pins --
   static const int PIN_DIRFRAG =         -1; 
   static const int PIN_CAPS =             2;  // client caps
@@ -287,7 +332,6 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   static const int PIN_EXPORTINGCAPS =    22;
   static const int PIN_DIRTYPARENT =      23;
   static const int PIN_DIRWAITER =        24;
-  static const int PIN_SCRUBQUEUE =       25;
 
   // -- dump flags --
   static const int DUMP_INODE_STORE_BASE = (1 << 0);
@@ -366,7 +410,6 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
     close_dirfrags();
     close_snaprealm();
     clear_file_locks();
-    ceph_assert(num_projected_xattrs == 0);
     ceph_assert(num_projected_srnodes == 0);
     ceph_assert(num_caps_notable == 0);
     ceph_assert(num_subtree_roots == 0);
@@ -378,20 +421,17 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
 
   std::string_view pin_name(int p) const override;
 
-  ostream& print_db_line_prefix(ostream& out) override;
+  std::ostream& print_db_line_prefix(std::ostream& out) override;
 
-  const scrub_info_t *scrub_info() const{
+  const scrub_info_t *scrub_info() const {
     if (!scrub_infop)
       scrub_info_create();
-    return scrub_infop;
+    return scrub_infop.get();
   }
 
-  ScrubHeaderRef get_scrub_header() {
-    if (scrub_infop == nullptr) {
-      return nullptr;
-    } else {
-      return scrub_infop->header;
-    }
+  const ScrubHeaderRef& get_scrub_header() {
+    static const ScrubHeaderRef nullref;
+    return scrub_infop ? scrub_infop->header : nullref;
   }
 
   bool scrub_is_in_progress() const {
@@ -405,32 +445,7 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
    * @param scrub_version What version are we scrubbing at (usually, parent
    * directory's get_projected_version())
    */
-  void scrub_initialize(CDentry *scrub_parent,
-                       ScrubHeaderRef& header,
-                       MDSContext *f);
-  /**
-   * Get the next dirfrag to scrub. Gives you a frag_t in output param which
-   * you must convert to a CDir (and possibly load off disk).
-   * @param dir A pointer to frag_t, will be filled in with the next dirfrag to
-   * scrub if there is one.
-   * @returns 0 on success, you should scrub the passed-out frag_t right now;
-   * ENOENT: There are no remaining dirfrags to scrub
-   * <0 There was some other error (It will return -ENOTDIR if not a directory)
-   */
-  int scrub_dirfrag_next(frag_t* out_dirfrag);
-  /**
-   * Get the currently scrubbing dirfrags. When returned, the
-   * passed-in list will be filled in with all frag_ts which have
-   * been returned from scrub_dirfrag_next but not sent back
-   * via scrub_dirfrag_finished.
-   */
-  void scrub_dirfrags_scrubbing(frag_vec_t *out_dirfrags);
-  /**
-   * Report to the CInode that a dirfrag it owns has been scrubbed. Call
-   * this for every frag_t returned from scrub_dirfrag_next().
-   * @param dirfrag The frag_t that was scrubbed
-   */
-  void scrub_dirfrag_finished(frag_t dirfrag);
+  void scrub_initialize(ScrubHeaderRef& header);
   /**
    * Call this once the scrub has been completed, whether it's a full
    * recursive scrub on a directory or simply the data on a file (or
@@ -438,26 +453,20 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
    * @param c An out param which is filled in with a Context* that must
    * be complete()ed.
    */
-  void scrub_finished(MDSContext **c);
+  void scrub_finished();
 
-  void scrub_aborted(MDSContext **c);
+  void scrub_aborted();
 
-  /**
-   * Report to the CInode that alldirfrags it owns have been scrubbed.
-   */
-  void scrub_children_finished() {
-    scrub_infop->children_scrubbed = true;
-  }
-  void scrub_set_finisher(MDSContext *c) {
-    ceph_assert(!scrub_infop->on_finish);
-    scrub_infop->on_finish = c;
+  fragset_t& scrub_queued_frags() {
+    ceph_assert(scrub_infop);
+    return scrub_infop->queued_frags;
   }
 
   bool is_multiversion() const {
     return snaprealm ||  // other snaprealms will link to me
-      inode.is_dir() ||  // links to me in other snaps
-      inode.nlink > 1 || // there are remote links, possibly snapped, that will need to find me
-      !old_inodes.empty(); // once multiversion, always multiversion.  until old_inodes gets cleaned out.
+      get_inode()->is_dir() ||  // links to me in other snaps
+      get_inode()->nlink > 1 || // there are remote links, possibly snapped, that will need to find me
+      is_any_old_inodes(); // once multiversion, always multiversion.  until old_inodes gets cleaned out.
   }
   snapid_t get_oldest_snap();
 
@@ -467,51 +476,86 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   void mark_dirty_rstat();
   void clear_dirty_rstat();
 
-  CInode::projected_inode &project_inode(bool xattr = false, bool snap = false);
-  void pop_and_dirty_projected_inode(LogSegment *ls);
+  //bool hack_accessed = false;
+  //utime_t hack_load_stamp;
 
-  projected_inode *get_projected_node() {
-    if (projected_nodes.empty())
-      return NULL;
-    else
-      return &projected_nodes.back();
-  }
+  /**
+   * Projection methods, used to store inode changes until they have been journaled,
+   * at which point they are popped.
+   * Usage:
+   * project_inode as needed. If you're changing xattrs or sr_t, then pass true
+   * as needed then change the xattrs/snapnode member as needed. (Dirty
+   * exception: project_past_snaprealm_parent allows you to project the
+   * snapnode after doing project_inode (i.e. you don't need to pass
+   * snap=true).
+   *
+   * Then, journal. Once journaling is done, pop_and_dirty_projected_inode.
+   * This function will take care of the inode itself, the xattrs, and the snaprealm.
+   */
+
+  struct projected_inode {
+    static sr_t* const UNDEF_SRNODE;
+
+    inode_ptr const inode;
+    xattr_map_ptr const xattrs;
+    sr_t* const snapnode;
+
+    projected_inode() = delete;
+    explicit projected_inode(inode_ptr&& i, xattr_map_ptr&& x, sr_t *s=nullptr) :
+      inode(std::move(i)), xattrs(std::move(x)), snapnode(s) {}
+  };
+  projected_inode project_inode(const MutationRef& mut,
+                               bool xattr = false, bool snap = false);
+
+  void pop_and_dirty_projected_inode(LogSegment *ls, const MutationRef& mut);
 
   version_t get_projected_version() const {
     if (projected_nodes.empty())
-      return inode.version;
+      return get_inode()->version;
     else
-      return projected_nodes.back().inode.version;
+      return projected_nodes.back().inode->version;
   }
   bool is_projected() const {
     return !projected_nodes.empty();
   }
 
-  const mempool_inode *get_projected_inode() const {
+  const inode_const_ptr& get_projected_inode() const {
     if (projected_nodes.empty())
-      return &inode;
+      return get_inode();
+    else
+      return projected_nodes.back().inode;
+  }
+  // inode should have already been projected in caller's context
+  mempool_inode* _get_projected_inode() {
+    ceph_assert(!projected_nodes.empty());
+    return const_cast<mempool_inode*>(projected_nodes.back().inode.get());
+  }
+  const inode_const_ptr& get_previous_projected_inode() const {
+    ceph_assert(!projected_nodes.empty());
+    auto it = projected_nodes.rbegin();
+    ++it;
+    if (it != projected_nodes.rend())
+      return it->inode;
     else
-      return &projected_nodes.back().inode;
+      return get_inode();
   }
-  mempool_inode *get_projected_inode() {
+
+  const xattr_map_const_ptr& get_projected_xattrs() {
     if (projected_nodes.empty())
-      return &inode;
+      return xattrs;
     else
-      return &projected_nodes.back().inode;
+      return projected_nodes.back().xattrs;
   }
-  mempool_inode *get_previous_projected_inode() {
+  const xattr_map_const_ptr& get_previous_projected_xattrs() {
     ceph_assert(!projected_nodes.empty());
     auto it = projected_nodes.rbegin();
     ++it;
     if (it != projected_nodes.rend())
-      return &it->inode;
+      return it->xattrs;
     else
-      return &inode;
+      return xattrs;
   }
 
-  mempool_xattr_map *get_projected_xattrs();
-  mempool_xattr_map *get_previous_projected_xattrs();
-
   sr_t *prepare_new_srnode(snapid_t snapid);
   void project_snaprealm(sr_t *new_srnode);
   sr_t *project_snaprealm(snapid_t snapid=0) {
@@ -531,9 +575,9 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   void project_snaprealm_past_parent(SnapRealm *newparent);
   void early_pop_projected_snaprealm();
 
-  mempool_old_inode& cow_old_inode(snapid_t follows, bool cow_head);
+  const mempool_old_inode& cow_old_inode(snapid_t follows, bool cow_head);
   void split_old_inode(snapid_t snap);
-  mempool_old_inode *pick_old_inode(snapid_t last);
+  snapid_t pick_old_inode(snapid_t last) const;
   void pre_cow_old_inode();
   bool has_snap_data(snapid_t s);
   void purge_stale_snap_data(const std::set<snapid_t>& snaps);
@@ -592,22 +636,29 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   void verify_dirfrags();
 
   void get_stickydirs();
-  void put_stickydirs();  
+  void put_stickydirs();
 
   void add_need_snapflush(CInode *snapin, snapid_t snapid, client_t client);
   void remove_need_snapflush(CInode *snapin, snapid_t snapid, client_t client);
-  pair<bool,bool> split_need_snapflush(CInode *cowin, CInode *in);
+  std::pair<bool,bool> split_need_snapflush(CInode *cowin, CInode *in);
 
   // -- accessors --
-  bool is_root() const { return inode.ino == MDS_INO_ROOT; }
-  bool is_stray() const { return MDS_INO_IS_STRAY(inode.ino); }
+
+  inodeno_t ino() const { return get_inode()->ino; }
+  vinodeno_t vino() const { return vinodeno_t(ino(), last); }
+  int d_type() const { return IFTODT(get_inode()->mode); }
+  bool is_root() const { return ino() == MDS_INO_ROOT; }
+  bool is_stray() const { return MDS_INO_IS_STRAY(ino()); }
   mds_rank_t get_stray_owner() const {
-    return (mds_rank_t)MDS_INO_STRAY_OWNER(inode.ino);
+    return (mds_rank_t)MDS_INO_STRAY_OWNER(ino());
   }
-  bool is_mdsdir() const { return MDS_INO_IS_MDSDIR(inode.ino); }
-  bool is_base() const { return MDS_INO_IS_BASE(inode.ino); }
-  bool is_system() const { return inode.ino < MDS_INO_SYSTEM_BASE; }
+  bool is_mdsdir() const { return MDS_INO_IS_MDSDIR(ino()); }
+  bool is_base() const { return MDS_INO_IS_BASE(ino()); }
+  bool is_system() const { return ino() < MDS_INO_SYSTEM_BASE; }
   bool is_normal() const { return !(is_base() || is_system() || is_stray()); }
+  bool is_file() const    { return get_inode()->is_file(); }
+  bool is_symlink() const { return get_inode()->is_symlink(); }
+  bool is_dir() const     { return get_inode()->is_dir(); }
 
   bool is_head() const { return last == CEPH_NOSNAP; }
 
@@ -622,12 +673,22 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   void clear_ambiguous_auth(MDSContext::vec& finished);
   void clear_ambiguous_auth();
 
-  inodeno_t ino() const { return inode.ino; }
-  vinodeno_t vino() const { return vinodeno_t(inode.ino, last); }
-  int d_type() const { return IFTODT(inode.mode); }
+  const inode_const_ptr& get_inode() const {
+    return inode;
+  }
+
+  // only used for updating newly allocated CInode
+  mempool_inode* _get_inode() {
+    if (inode == empty_inode)
+      reset_inode(allocate_inode());
+    return const_cast<mempool_inode*>(inode.get());
+  }
+
+  const xattr_map_const_ptr& get_xattrs() const { return xattrs; }
+
+  bool is_any_old_inodes() const { return old_inodes && !old_inodes->empty(); }
+  const old_inode_map_const_ptr& get_old_inodes() const { return old_inodes; }
 
-  mempool_inode& get_inode() { return inode; }
-  const mempool_inode& get_inode() const { return inode; }
   CDentry* get_parent_dn() { return parent; }
   const CDentry* get_parent_dn() const { return parent; }
   CDentry* get_projected_parent_dn() { return !projected_parent.empty() ? projected_parent.back() : parent; }
@@ -657,11 +718,11 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   void name_stray_dentry(std::string& dname);
   
   // -- dirtyness --
-  version_t get_version() const { return inode.version; }
+  version_t get_version() const { return get_inode()->version; }
 
   version_t pre_dirty();
   void _mark_dirty(LogSegment *ls);
-  void mark_dirty(version_t projected_dirv, LogSegment *ls);
+  void mark_dirty(LogSegment *ls);
   void mark_clean();
 
   void store(MDSContext *fin);
@@ -675,23 +736,29 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
    */
   void flush(MDSContext *fin);
   void fetch(MDSContext *fin);
-  void _fetched(bufferlist& bl, bufferlist& bl2, Context *fin);  
+  void _fetched(ceph::buffer::list& bl, ceph::buffer::list& bl2, Context *fin);  
 
+  void _commit_ops(int r, C_GatherBuilder &gather_bld,
+                   std::vector<CInodeCommitOperation> &ops_vec,
+                   inode_backtrace_t &bt);
   void build_backtrace(int64_t pool, inode_backtrace_t& bt);
+  void _store_backtrace(std::vector<CInodeCommitOperation> &ops_vec,
+                        inode_backtrace_t &bt, int op_prio);
+  void store_backtrace(CInodeCommitOperations &op, int op_prio);
   void store_backtrace(MDSContext *fin, int op_prio=-1);
   void _stored_backtrace(int r, version_t v, Context *fin);
-  void fetch_backtrace(Context *fin, bufferlist *backtrace);
+  void fetch_backtrace(Context *fin, ceph::buffer::list *backtrace);
 
   void mark_dirty_parent(LogSegment *ls, bool dirty_pool=false);
   void clear_dirty_parent();
-  void verify_diri_backtrace(bufferlist &bl, int err);
+  void verify_diri_backtrace(ceph::buffer::list &bl, int err);
   bool is_dirty_parent() { return state_test(STATE_DIRTYPARENT); }
   bool is_dirty_pool() { return state_test(STATE_DIRTYPOOL); }
 
-  void encode_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);
+  void encode_snap_blob(ceph::buffer::list &bl);
+  void decode_snap_blob(const ceph::buffer::list &bl);
+  void encode_store(ceph::buffer::list& bl, uint64_t features);
+  void decode_store(ceph::buffer::list::const_iterator& bl);
 
   void add_dir_waiter(frag_t fg, MDSContext *c);
   void take_dir_waiting(frag_t fg, MDSContext::vec& ls);
@@ -702,18 +769,18 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   void take_waiting(uint64_t tag, MDSContext::vec& ls) override;
 
   // -- encode/decode helpers --
-  void _encode_base(bufferlist& bl, uint64_t features);
-  void _decode_base(bufferlist::const_iterator& p);
-  void _encode_locks_full(bufferlist& bl);
-  void _decode_locks_full(bufferlist::const_iterator& p);
-  void _encode_locks_state_for_replica(bufferlist& bl, bool need_recover);
-  void _encode_locks_state_for_rejoin(bufferlist& bl, int rep);
-  void _decode_locks_state_for_replica(bufferlist::const_iterator& p, bool is_new);
-  void _decode_locks_rejoin(bufferlist::const_iterator& p, MDSContext::vec& waiters,
+  void _encode_base(ceph::buffer::list& bl, uint64_t features);
+  void _decode_base(ceph::buffer::list::const_iterator& p);
+  void _encode_locks_full(ceph::buffer::list& bl);
+  void _decode_locks_full(ceph::buffer::list::const_iterator& p);
+  void _encode_locks_state_for_replica(ceph::buffer::list& bl, bool need_recover);
+  void _encode_locks_state_for_rejoin(ceph::buffer::list& bl, int rep);
+  void _decode_locks_state_for_replica(ceph::buffer::list::const_iterator& p, bool is_new);
+  void _decode_locks_rejoin(ceph::buffer::list::const_iterator& p, MDSContext::vec& waiters,
                            std::list<SimpleLock*>& eval_locks, bool survivor);
 
   // -- import/export --
-  void encode_export(bufferlist& bl);
+  void encode_export(ceph::buffer::list& bl);
   void finish_export();
   void abort_export() {
     put(PIN_TEMPEXPORTING);
@@ -721,38 +788,38 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
     state_clear(STATE_EXPORTINGCAPS);
     put(PIN_EXPORTINGCAPS);
   }
-  void decode_import(bufferlist::const_iterator& p, LogSegment *ls);
+  void decode_import(ceph::buffer::list::const_iterator& p, LogSegment *ls);
   
   // for giving to clients
-  int encode_inodestat(bufferlist& bl, Session *session, SnapRealm *realm,
+  int encode_inodestat(ceph::buffer::list& bl, Session *session, SnapRealm *realm,
                       snapid_t snapid=CEPH_NOSNAP, unsigned max_bytes=0,
                       int getattr_wants=0);
-  void encode_cap_message(const ref_t<MClientCaps> &m, Capability *cap);
+  void encode_cap_message(const ceph::ref_t<MClientCaps> &m, Capability *cap);
 
   SimpleLock* get_lock(int type) override;
 
   void set_object_info(MDSCacheObjectInfo &info) override;
 
-  void encode_lock_state(int type, bufferlist& bl) override;
-  void decode_lock_state(int type, const bufferlist& bl) override;
-  void encode_lock_iauth(bufferlist& bl);
-  void decode_lock_iauth(bufferlist::const_iterator& p);
-  void encode_lock_ilink(bufferlist& bl);
-  void decode_lock_ilink(bufferlist::const_iterator& p);
-  void encode_lock_idft(bufferlist& bl);
-  void decode_lock_idft(bufferlist::const_iterator& p);
-  void encode_lock_ifile(bufferlist& bl);
-  void decode_lock_ifile(bufferlist::const_iterator& p);
-  void encode_lock_inest(bufferlist& bl);
-  void decode_lock_inest(bufferlist::const_iterator& p);
-  void encode_lock_ixattr(bufferlist& bl);
-  void decode_lock_ixattr(bufferlist::const_iterator& p);
-  void encode_lock_isnap(bufferlist& bl);
-  void decode_lock_isnap(bufferlist::const_iterator& p);
-  void encode_lock_iflock(bufferlist& bl);
-  void decode_lock_iflock(bufferlist::const_iterator& p);
-  void encode_lock_ipolicy(bufferlist& bl);
-  void decode_lock_ipolicy(bufferlist::const_iterator& p);
+  void encode_lock_state(int type, ceph::buffer::list& bl) override;
+  void decode_lock_state(int type, const ceph::buffer::list& bl) override;
+  void encode_lock_iauth(ceph::buffer::list& bl);
+  void decode_lock_iauth(ceph::buffer::list::const_iterator& p);
+  void encode_lock_ilink(ceph::buffer::list& bl);
+  void decode_lock_ilink(ceph::buffer::list::const_iterator& p);
+  void encode_lock_idft(ceph::buffer::list& bl);
+  void decode_lock_idft(ceph::buffer::list::const_iterator& p);
+  void encode_lock_ifile(ceph::buffer::list& bl);
+  void decode_lock_ifile(ceph::buffer::list::const_iterator& p);
+  void encode_lock_inest(ceph::buffer::list& bl);
+  void decode_lock_inest(ceph::buffer::list::const_iterator& p);
+  void encode_lock_ixattr(ceph::buffer::list& bl);
+  void decode_lock_ixattr(ceph::buffer::list::const_iterator& p);
+  void encode_lock_isnap(ceph::buffer::list& bl);
+  void decode_lock_isnap(ceph::buffer::list::const_iterator& p);
+  void encode_lock_iflock(ceph::buffer::list& bl);
+  void decode_lock_iflock(ceph::buffer::list::const_iterator& p);
+  void encode_lock_ipolicy(ceph::buffer::list& bl);
+  void decode_lock_ipolicy(ceph::buffer::list::const_iterator& p);
 
   void _finish_frag_update(CDir *dir, MutationRef& mut);
 
@@ -763,15 +830,15 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   void start_scatter(ScatterLock *lock);
   void finish_scatter_update(ScatterLock *lock, CDir *dir,
                             version_t inode_version, version_t dir_accounted_version);
-  void finish_scatter_gather_update(int type);
-  void finish_scatter_gather_update_accounted(int type, MutationRef& mut, EMetaBlob *metablob);
+  void finish_scatter_gather_update(int type, MutationRef& mut);
+  void finish_scatter_gather_update_accounted(int type, EMetaBlob *metablob);
 
   // -- snap --
   void open_snaprealm(bool no_split=false);
   void close_snaprealm(bool no_join=false);
   SnapRealm *find_snaprealm() const;
-  void encode_snap(bufferlist& bl);
-  void decode_snap(bufferlist::const_iterator& p);
+  void encode_snap(ceph::buffer::list& bl);
+  void decode_snap(ceph::buffer::list::const_iterator& p);
 
   client_t get_loner() const { return loner_cap; }
   client_t get_wanted_loner() const { return want_loner_cap; }
@@ -838,7 +905,8 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   int get_caps_allowed_by_type(int type) const;
   int get_caps_careful() const;
   int get_xlocker_mask(client_t client) const;
-  int get_caps_allowed_for_client(Session *s, Capability *cap, mempool_inode *file_i) const;
+  int get_caps_allowed_for_client(Session *s, Capability *cap,
+                                 const mempool_inode *file_i) const;
 
   // caps issued, wanted
   int get_caps_issued(int *ploner = 0, int *pother = 0, int *pxlocker = 0,
@@ -931,47 +999,38 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
     return !projected_parent.empty();
   }
 
-  mds_rank_t get_export_pin(bool inherit=true, bool ephemeral=true) const;
+  mds_rank_t get_export_pin(bool inherit=true) const;
+  void check_pin_policy(mds_rank_t target);
   void set_export_pin(mds_rank_t rank);
   void queue_export_pin(mds_rank_t target);
   void maybe_export_pin(bool update=false);
 
-  void check_pin_policy();
+  void set_ephemeral_pin(bool dist, bool rand);
+  void clear_ephemeral_pin(bool dist, bool rand);
 
-  void set_ephemeral_dist(bool yes);
-  void maybe_ephemeral_dist(bool update=false);
-  void maybe_ephemeral_dist_children(bool update=false);
   void setxattr_ephemeral_dist(bool val=false);
   bool is_ephemeral_dist() const {
     return state_test(STATE_DISTEPHEMERALPIN);
   }
 
-  double get_ephemeral_rand(bool inherit=true) const;
-  void set_ephemeral_rand(bool yes);
-  void maybe_ephemeral_rand(bool fresh=false, double threshold=-1.0);
+  double get_ephemeral_rand() const;
+  void maybe_ephemeral_rand(double threshold=-1.0);
   void setxattr_ephemeral_rand(double prob=0.0);
   bool is_ephemeral_rand() const {
     return state_test(STATE_RANDEPHEMERALPIN);
   }
 
   bool has_ephemeral_policy() const {
-    return get_inode().export_ephemeral_random_pin > 0.0 ||
-           get_inode().export_ephemeral_distributed_pin;
+    return get_inode()->export_ephemeral_random_pin > 0.0 ||
+           get_inode()->export_ephemeral_distributed_pin;
   }
   bool is_ephemerally_pinned() const {
     return state_test(STATE_DISTEPHEMERALPIN) ||
            state_test(STATE_RANDEPHEMERALPIN);
   }
-  bool is_exportable(mds_rank_t dest) const;
-
-  void maybe_pin() {
-    maybe_export_pin();
-    maybe_ephemeral_dist();
-    maybe_ephemeral_rand();
-  }
 
-  void print(ostream& out) override;
-  void dump(Formatter *f, int flags = DUMP_DEFAULT) const;
+  void print(std::ostream& out) override;
+  void dump(ceph::Formatter *f, int flags = DUMP_DEFAULT) const;
 
   /**
    * Validate that the on-disk state of an inode matches what
@@ -989,7 +1048,7 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   void validate_disk_state(validated_data *results,
                            MDSContext *fin);
   static void dump_validation_results(const validated_data& results,
-                                      Formatter *f);
+                                      ceph::Formatter *f);
 
   //bool hack_accessed = false;
   //utime_t hack_load_stamp;
@@ -1019,7 +1078,6 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   elist<CInode*>::item item_dirty_dirfrag_dir;
   elist<CInode*>::item item_dirty_dirfrag_nest;
   elist<CInode*>::item item_dirty_dirfrag_dirfragtree;
-  elist<CInode*>::item item_scrub;
 
   // also update RecoveryQueue::RecoveryQueue() if you change this
   elist<CInode*>::item& item_recover_queue = item_dirty_dirfrag_dir;
@@ -1041,7 +1099,7 @@ class CInode : public MDSCacheObject, public InodeStoreBase, public Counter<CIno
   static LockType policylock_type;
 
   // FIXME not part of mempool
-  LocalLock  versionlock;
+  LocalLockC  versionlock;
   SimpleLock authlock;
   SimpleLock linklock;
   ScatterLock dirfragtreelock;
@@ -1079,7 +1137,7 @@ protected:
     clear_fcntl_lock_state();
     clear_flock_lock_state();
   }
-  void _encode_file_locks(bufferlist& bl) const {
+  void _encode_file_locks(ceph::buffer::list& bl) const {
     using ceph::encode;
     bool has_fcntl_locks = fcntl_locks && !fcntl_locks->empty();
     encode(has_fcntl_locks, bl);
@@ -1090,7 +1148,7 @@ protected:
     if (has_flock_locks)
       encode(*flock_locks, bl);
   }
-  void _decode_file_locks(bufferlist::const_iterator& p) {
+  void _decode_file_locks(ceph::buffer::list::const_iterator& p) {
     using ceph::decode;
     bool has_fcntl_locks;
     decode(has_fcntl_locks, p);
@@ -1158,8 +1216,18 @@ private:
   bool _validate_disk_state(class ValidationContinuation *c,
                             int rval, int stage);
 
-  mempool::mds_co::list<projected_inode> projected_nodes;   // projected values (only defined while dirty)
-  size_t num_projected_xattrs = 0;
+  struct projected_const_node {
+    inode_const_ptr inode;
+    xattr_map_const_ptr xattrs;
+    sr_t *snapnode;
+
+    projected_const_node() = delete;
+    projected_const_node(projected_const_node&&) = default;
+    explicit projected_const_node(const inode_const_ptr& i, const xattr_map_const_ptr& x, sr_t *s) :
+      inode(i), xattrs(x), snapnode(s) {}
+  };
+
+  mempool::mds_co::list<projected_const_node> projected_nodes;   // projected values (only defined while dirty)
   size_t num_projected_srnodes = 0;
 
   // -- cache infrastructure --
@@ -1170,12 +1238,11 @@ private:
   int num_exporting_dirs = 0;
 
   int stickydir_ref = 0;
-  scrub_info_t *scrub_infop = nullptr;
+  std::unique_ptr<scrub_info_t> scrub_infop;
   /** @} Scrubbing and fsck */
 };
 
-ostream& operator<<(ostream& out, const CInode& in);
-ostream& operator<<(ostream& out, const CInode::scrub_stamp_info_t& si);
+std::ostream& operator<<(std::ostream& out, const CInode& in);
 
 extern cinode_lock_info_t cinode_lock_info[];
 extern int num_cinode_locks;