]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/mds/Mutation.h
import ceph 15.2.14
[ceph.git] / ceph / src / mds / Mutation.h
index 6f617c574c187f2acce06ef2ed44e77ac23410a2..536f6e806f519d6677ded7eda9c0ebf214c43ad4 100644 (file)
 #include "include/filepath.h"
 
 #include "MDSCacheObject.h"
+#include "MDSContext.h"
 
 #include "SimpleLock.h"
 #include "Capability.h"
+#include "BatchOp.h"
 
 #include "common/TrackedOp.h"
+#include "messages/MClientRequest.h"
+#include "messages/MMDSSlaveRequest.h"
+#include "messages/MClientReply.h"
 
 class LogSegment;
-class Capability;
 class CInode;
 class CDir;
 class CDentry;
 class Session;
 class ScatterLock;
-class MClientRequest;
-class MMDSSlaveRequest;
+struct sr_t;
+struct MDLockCache;
 
 struct MutationImpl : public TrackedOp {
-  metareqid_t reqid;
-  __u32 attempt = 0;      // which attempt for this request
-  LogSegment *ls = nullptr;  // the log segment i'm committing to
-
-private:
-  utime_t mds_stamp; ///< mds-local timestamp (real time)
-  utime_t op_stamp;  ///< op timestamp (client provided)
-
 public:
-  // flag mutation as slave
-  mds_rank_t slave_to_mds = MDS_RANK_NONE;  // this is a slave request if >= 0.
-
-  // -- my pins and locks --
-  // cache pins (so things don't expire)
-  set< MDSCacheObject* > pins;
-  set<CInode*> stickydirs;
+  // -- my pins and auth_pins --
+  struct ObjectState {
+    bool pinned = false;
+    bool auth_pinned = false;
+    mds_rank_t remote_auth_pinned = MDS_RANK_NONE;
+  };
 
-  // auth pins
-  map<MDSCacheObject*, mds_rank_t> remote_auth_pins;
-  set< MDSCacheObject* > auth_pins;
-  
   // held locks
-  set< SimpleLock* > rdlocks;  // always local.
-  set< SimpleLock* > wrlocks;  // always local.
-  map< SimpleLock*, mds_rank_t > remote_wrlocks;
-  set< SimpleLock* > xlocks;   // local or remote.
-  set< SimpleLock*, SimpleLock::ptr_lt > locks;  // full ordering
+  struct LockOp {
+    enum {
+      RDLOCK           = 1,
+      WRLOCK           = 2,
+      XLOCK            = 4,
+      REMOTE_WRLOCK    = 8,
+      STATE_PIN                = 16, // no RW after locked, just pin lock state
+    };
+
+    LockOp(SimpleLock *l, unsigned f=0, mds_rank_t t=MDS_RANK_NONE) :
+      lock(l), flags(f), wrlock_target(t) {}
+
+    bool is_rdlock() const { return !!(flags & RDLOCK); }
+    bool is_xlock() const { return !!(flags & XLOCK); }
+    bool is_wrlock() const { return !!(flags & WRLOCK); }
+    void clear_wrlock() const { flags &= ~WRLOCK; }
+    bool is_remote_wrlock() const { return !!(flags & REMOTE_WRLOCK); }
+    void clear_remote_wrlock() const {
+      flags &= ~REMOTE_WRLOCK;
+      wrlock_target = MDS_RANK_NONE;
+    }
+    bool is_state_pin() const { return !!(flags & STATE_PIN); }
+    bool operator<(const LockOp& r) const {
+      return lock < r.lock;
+    }
 
-  // lock we are currently trying to acquire.  if we give up for some reason,
-  // be sure to eval() this.
-  SimpleLock *locking = nullptr;
-  mds_rank_t locking_target_mds = -1;
+    SimpleLock* lock;
+    mutable unsigned flags;
+    mutable mds_rank_t wrlock_target;
+  };
 
-  // if this flag is set, do not attempt to acquire further locks.
-  //  (useful for wrlock, which may be a moving auth target)
-  bool done_locking = false;
-  bool committing = false;
-  bool aborted = false;
-  bool killed = false;
+  struct LockOpVec : public vector<LockOp> {
+    LockOpVec() {
+      reserve(32);
+    }
 
-  // for applying projected inode changes
-  list<CInode*> projected_inodes;
-  list<CDir*> projected_fnodes;
-  list<ScatterLock*> updated_locks;
+    void add_rdlock(SimpleLock *lock) {
+      emplace_back(lock, LockOp::RDLOCK);
+    }
+    void erase_rdlock(SimpleLock *lock);
+    void add_xlock(SimpleLock *lock, int idx=-1) {
+      if (idx >= 0)
+       emplace(cbegin() + idx, lock, LockOp::XLOCK);
+      else
+       emplace_back(lock, LockOp::XLOCK);
+    }
+    void add_wrlock(SimpleLock *lock, int idx=-1) {
+      if (idx >= 0)
+       emplace(cbegin() + idx, lock, LockOp::WRLOCK);
+      else
+       emplace_back(lock, LockOp::WRLOCK);
+    }
+    void add_remote_wrlock(SimpleLock *lock, mds_rank_t rank) {
+      ceph_assert(rank != MDS_RANK_NONE);
+      emplace_back(lock, LockOp::REMOTE_WRLOCK, rank);
+    }
+    void lock_scatter_gather(SimpleLock *lock) {
+      emplace_back(lock, LockOp::WRLOCK | LockOp::STATE_PIN);
+    }
+    void sort_and_merge();
+  };
 
-  list<CInode*> dirty_cow_inodes;
-  list<pair<CDentry*,version_t> > dirty_cow_dentries;
+  using lock_set = set<LockOp>;
+  using lock_iterator = lock_set::iterator;
 
   // keep our default values synced with MDRequestParam's
   MutationImpl() : TrackedOp(nullptr, utime_t()) {}
   MutationImpl(OpTracker *tracker, utime_t initiated,
-              metareqid_t ri, __u32 att=0, mds_rank_t slave_to=MDS_RANK_NONE)
+              const metareqid_t &ri, __u32 att=0, mds_rank_t slave_to=MDS_RANK_NONE)
     : TrackedOp(tracker, initiated),
       reqid(ri), attempt(att),
-      slave_to_mds(slave_to) { }
+      slave_to_mds(slave_to) {}
   ~MutationImpl() override {
-    assert(locking == NULL);
-    assert(pins.empty());
-    assert(auth_pins.empty());
-    assert(xlocks.empty());
-    assert(rdlocks.empty());
-    assert(wrlocks.empty());
-    assert(remote_wrlocks.empty());
+    ceph_assert(!locking);
+    ceph_assert(!lock_cache);
+    ceph_assert(num_pins == 0);
+    ceph_assert(num_auth_pins == 0);
+  }
+
+  const ObjectState* find_object_state(MDSCacheObject *obj) const {
+    auto it = object_states.find(obj);
+    return it != object_states.end() ? &it->second : nullptr;
+  }
+
+  bool is_any_remote_auth_pin() const { return num_remote_auth_pins > 0; }
+
+  void disable_lock_cache() {
+    lock_cache_disabled = true;
+  }
+
+  lock_iterator emplace_lock(SimpleLock *l, unsigned f=0, mds_rank_t t=MDS_RANK_NONE) {
+    last_locked = l;
+    return locks.emplace(l, f, t).first;
+  }
+
+  bool is_rdlocked(SimpleLock *lock) const;
+  bool is_wrlocked(SimpleLock *lock) const;
+  bool is_xlocked(SimpleLock *lock) const {
+    auto it = locks.find(lock);
+    return it != locks.end() && it->is_xlock();
+  }
+  bool is_remote_wrlocked(SimpleLock *lock) const {
+    auto it = locks.find(lock);
+    return it != locks.end() && it->is_remote_wrlock();
+  }
+  bool is_last_locked(SimpleLock *lock) const {
+    return lock == last_locked;
   }
 
   bool is_master() const { return slave_to_mds == MDS_RANK_NONE; }
@@ -127,9 +184,10 @@ public:
   }
 
   // pin items in cache
-  void pin(MDSCacheObject *o);
-  void unpin(MDSCacheObject *o);
+  void pin(MDSCacheObject *object);
+  void unpin(MDSCacheObject *object);
   void set_stickydirs(CInode *in);
+  void put_stickydirs();
   void drop_pins();
 
   void start_locking(SimpleLock *lock, int target=-1);
@@ -140,6 +198,9 @@ public:
   void auth_pin(MDSCacheObject *object);
   void auth_unpin(MDSCacheObject *object);
   void drop_local_auth_pins();
+  void set_remote_auth_pinned(MDSCacheObject* object, mds_rank_t from);
+  void _clear_remote_auth_pinned(ObjectState& stat);
+
   void add_projected_inode(CInode *in);
   void pop_and_dirty_projected_inodes();
   void add_projected_fnode(CDir *dir);
@@ -156,71 +217,71 @@ public:
 
   virtual void dump(Formatter *f) const {}
   void _dump_op_descriptor_unlocked(ostream& stream) const override;
-};
-
-inline ostream& operator<<(ostream &out, const MutationImpl &mut)
-{
-  mut.print(out);
-  return out;
-}
 
-typedef boost::intrusive_ptr<MutationImpl> MutationRef;
-
-
-
-/** active_request_t
- * state we track for requests we are currently processing.
- * mostly information about locks held, so that we can drop them all
- * the request is finished or forwarded.  see request_*().
- */
-struct MDRequestImpl : public MutationImpl {
-  Session *session;
-  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)
-
-  // store up to two sets of dn vectors, inode pointers, for request path1 and path2.
-  vector<CDentry*> dn[2];
-  CDentry *straydn;
-  CInode *in[2];
-  snapid_t snapid;
+  metareqid_t reqid;
+  __u32 attempt = 0;      // which attempt for this request
+  LogSegment *ls = nullptr;  // the log segment i'm committing to
 
-  CInode *tracei;
-  CDentry *tracedn;
+  // flag mutation as slave
+  mds_rank_t slave_to_mds = MDS_RANK_NONE;  // this is a slave request if >= 0.
 
-  inodeno_t alloc_ino, used_prealloc_ino;  
-  interval_set<inodeno_t> prealloc_inos;
+  ceph::unordered_map<MDSCacheObject*, ObjectState> object_states;
+  int num_pins = 0;
+  int num_auth_pins = 0;
+  int num_remote_auth_pins = 0;
+  // cache pins (so things don't expire)
+  CInode* stickydiri = nullptr;
 
-  int snap_caps = 0;
-  int getattr_caps = 0;                ///< caps requested by getattr
-  bool no_early_reply = false;
-  bool did_early_reply = false;
-  bool o_trunc = false;                ///< request is an O_TRUNC mutation
-  bool has_completed = false;  ///< request has already completed
+  lock_set locks;  // full ordering
+  MDLockCache* lock_cache = nullptr;
+  bool lock_cache_disabled = false;
+  SimpleLock *last_locked = nullptr;
+  // Lock we are currently trying to acquire. If we give up for some reason,
+  // be sure to eval() this.
+  SimpleLock *locking = nullptr;
+  mds_rank_t locking_target_mds = -1;
 
-  bufferlist reply_extra_bl;
+  // if this flag is set, do not attempt to acquire further locks.
+  //  (useful for wrlock, which may be a moving auth target)
+  enum {
+    SNAP_LOCKED                = 1,
+    SNAP2_LOCKED       = 2,
+    PATH_LOCKED                = 4,
+    ALL_LOCKED         = 8,
+  };
+  int locking_state = 0;
 
-  // inos we did a embedded cap release on, and may need to eval if we haven't since reissued
-  map<vinodeno_t, ceph_seq_t> cap_releases;  
+  bool committing = false;
+  bool aborted = false;
+  bool killed = false;
 
-  // -- i am a slave request
-  MMDSSlaveRequest *slave_request; // slave request (if one is pending; implies slave == true)
+  // for applying projected inode changes
+  list<CInode*> projected_inodes;
+  std::vector<CDir*> projected_fnodes;
+  list<ScatterLock*> updated_locks;
 
-  // -- i am an internal op
-  int internal_op;
-  Context *internal_op_finish;
-  void *internal_op_private;
+  list<CInode*> dirty_cow_inodes;
+  list<pair<CDentry*,version_t> > dirty_cow_dentries;
 
-  // indicates how may retries of request have been made
-  int retry;
+private:
+  utime_t mds_stamp; ///< mds-local timestamp (real time)
+  utime_t op_stamp;  ///< op timestamp (client provided)
+};
 
-  // indicator for vxattr osdmap update
-  bool waited_for_osdmap;
+/**
+ * MDRequestImpl: state we track for requests we are currently processing.
+ * mostly information about locks held, so that we can drop them all
+ * the request is finished or forwarded. see request_*().
+ */
+struct MDRequestImpl : public MutationImpl {
+  // TrackedOp stuff
+  typedef boost::intrusive_ptr<MDRequestImpl> Ref;
 
   // break rarely-used fields into a separately allocated structure 
   // to save memory for most ops
   struct More {
+    More() {}
+
     int slave_error = 0;
     set<mds_rank_t> slaves;           // mds nodes that have slave requests to me (implies client_request)
     set<mds_rank_t> waiting_on_slave; // peers i'm waiting for slavereq replies from. 
@@ -243,6 +304,7 @@ struct MDRequestImpl : public MutationImpl {
     bool is_ambiguous_auth = false;
     bool is_remote_frozen_authpin = false;
     bool is_inode_exporter = false;
+    bool rdonly_checks = false;
 
     map<client_t, pair<Session*, uint64_t> > imported_session_map;
     map<CInode*, map<client_t,Capability::Export> > cap_imports;
@@ -254,11 +316,14 @@ struct MDRequestImpl : public MutationImpl {
     version_t stid = 0;
     bufferlist snapidbl;
 
+    sr_t *srci_srnode = nullptr;
+    sr_t *desti_srnode = nullptr;
+
     // called when slave commits or aborts
     Context *slave_commit = nullptr;
     bufferlist rollback_bl;
 
-    list<MDSInternalContextBase*> waiting_for_finish;
+    MDSContext::vec waiting_for_finish;
 
     // export & fragment
     CDir* export_dir = nullptr;
@@ -267,43 +332,38 @@ struct MDRequestImpl : public MutationImpl {
     // for internal ops doing lookup
     filepath filepath1;
     filepath filepath2;
-
-    More() {}
-  } *_more;
-
+  } *_more = nullptr;
 
   // ---------------------------------------------------
   struct Params {
+    // keep these default values synced to MutationImpl's
+    Params() {}
+    const utime_t& get_recv_stamp() const {
+      return initiated;
+    }
+    const utime_t& get_throttle_stamp() const {
+      return throttled;
+    }
+    const utime_t& get_recv_complete_stamp() const {
+      return all_read;
+    }
+    const utime_t& get_dispatch_stamp() const {
+      return dispatched;
+    }
     metareqid_t reqid;
-    __u32 attempt;
-    MClientRequest *client_req;
-    class Message *triggering_slave_req;
-    mds_rank_t slave_to;
+    __u32 attempt = 0;
+    cref_t<MClientRequest> client_req;
+    cref_t<Message> triggering_slave_req;
+    mds_rank_t slave_to = MDS_RANK_NONE;
     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) {}
+    int internal_op = -1;
   };
-  MDRequestImpl(const Params& params, OpTracker *tracker) :
-    MutationImpl(tracker, params.initiated,
-                params.reqid, params.attempt, params.slave_to),
-    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_private(NULL),
-    retry(0),
-    waited_for_osdmap(false), _more(NULL) {
-    in[0] = in[1] = NULL;
-    if (!params.throttled.is_zero())
-      mark_event("throttled", params.throttled);
-    if (!params.all_read.is_zero())
-      mark_event("all_read", params.all_read);
-    if (!params.dispatched.is_zero())
-      mark_event("dispatched", params.dispatched);
-  }
+  MDRequestImpl(const Params* params, OpTracker *tracker) :
+    MutationImpl(tracker, params->initiated,
+                params->reqid, params->attempt, params->slave_to),
+    item_session_request(this), client_request(params->client_req),
+    internal_op(params->internal_op) {}
   ~MDRequestImpl() override;
   
   More* more();
@@ -311,7 +371,6 @@ struct MDRequestImpl : public MutationImpl {
   bool has_witnesses();
   bool slave_did_prepare();
   bool slave_rolling_back();
-  bool did_ino_allocation() const;
   bool freeze_auth_pin(CInode *inode);
   void unfreeze_auth_pin(bool clear_inode=false);
   void set_remote_frozen_auth_pin(CInode *inode);
@@ -324,55 +383,148 @@ struct MDRequestImpl : public MutationImpl {
   void set_filepath(const filepath& fp);
   void set_filepath2(const filepath& fp);
   bool is_queued_for_replay() const;
+  int compare_paths();
+
+  bool can_batch();
+  bool is_batch_head() {
+    return batch_op_map != nullptr;
+  }
+  std::unique_ptr<BatchOp> release_batch_op();
 
   void print(ostream &out) const override;
   void dump(Formatter *f) const override;
 
-  MClientRequest* release_client_request();
-  void reset_slave_request(MMDSSlaveRequest *req=nullptr);
+  cref_t<MClientRequest> release_client_request();
+  void reset_slave_request(const cref_t<MMDSSlaveRequest>& req=nullptr);
+
+  Session *session = nullptr;
+  elist<MDRequestImpl*>::item item_session_request;  // if not on list, op is aborted.
+
+  // -- i am a client (master) request
+  cref_t<MClientRequest> client_request; // client request (if any)
+
+  // tree and depth info of path1 and path2
+  inodeno_t dir_root[2] = {0, 0};
+  int dir_depth[2] = {-1, -1};
+  file_layout_t dir_layout;
+  // store up to two sets of dn vectors, inode pointers, for request path1 and path2.
+  vector<CDentry*> dn[2];
+  CInode *in[2] = {};
+  CDentry *straydn = nullptr;
+  snapid_t snapid = CEPH_NOSNAP;
+
+  CInode *tracei = nullptr;
+  CDentry *tracedn = nullptr;
+
+  inodeno_t alloc_ino = 0, used_prealloc_ino = 0;
+  interval_set<inodeno_t> prealloc_inos;
+
+  int snap_caps = 0;
+  int getattr_caps = 0;                ///< caps requested by getattr
+  bool no_early_reply = false;
+  bool did_early_reply = false;
+  bool o_trunc = false;                ///< request is an O_TRUNC mutation
+  bool has_completed = false;  ///< request has already completed
+
+  bufferlist reply_extra_bl;
+
+  // inos we did a embedded cap release on, and may need to eval if we haven't since reissued
+  map<vinodeno_t, ceph_seq_t> cap_releases;
+
+  // -- i am a slave request
+  cref_t<MMDSSlaveRequest> slave_request; // slave request (if one is pending; implies slave == true)
+
+  // -- i am an internal op
+  int internal_op;
+  Context *internal_op_finish = nullptr;
+  void *internal_op_private = nullptr;
+
+  // indicates how may retries of request have been made
+  int retry = 0;
+
+  std::map<int, std::unique_ptr<BatchOp> > *batch_op_map = nullptr;
+
+  // indicator for vxattr osdmap update
+  bool waited_for_osdmap = false;
 
-  // TrackedOp stuff
-  typedef boost::intrusive_ptr<MDRequestImpl> Ref;
 protected:
   void _dump(Formatter *f) const override;
   void _dump_op_descriptor_unlocked(ostream& stream) const override;
 private:
-  class {
-    std::atomic_flag _lock = ATOMIC_FLAG_INIT;
-  public:
-    void lock() {
-      while(_lock.test_and_set(std::memory_order_acquire))
-       ;
-    }
-    void unlock() {
-      _lock.clear(std::memory_order_release);
-    }
-  } mutable msg_lock;
+  mutable ceph::spinlock msg_lock;
 };
 
-typedef boost::intrusive_ptr<MDRequestImpl> MDRequestRef;
-
-
 struct MDSlaveUpdate {
-  int origop;
-  bufferlist rollback;
-  elist<MDSlaveUpdate*>::item item;
-  Context *waiter;
-  set<CInode*> olddirs;
-  set<CInode*> unlinked;
-  MDSlaveUpdate(int oo, bufferlist &rbl, elist<MDSlaveUpdate*> &list) :
-    origop(oo),
-    item(this),
-    waiter(0) {
+  MDSlaveUpdate(int oo, bufferlist &rbl) :
+    origop(oo) {
     rollback.claim(rbl);
-    list.push_back(&item);
   }
   ~MDSlaveUpdate() {
-    item.remove_myself();
     if (waiter)
       waiter->complete(0);
   }
+  int origop;
+  bufferlist rollback;
+  Context *waiter = nullptr;
+  set<CInode*> olddirs;
+  set<CInode*> unlinked;
 };
 
+struct MDLockCacheItem {
+  MDLockCache *parent = nullptr;
+  elist<MDLockCacheItem*>::item item_lock;
+};
+
+struct MDLockCache : public MutationImpl {
+  using LockItem = MDLockCacheItem;
+
+  struct DirItem {
+    MDLockCache *parent = nullptr;
+    elist<DirItem*>::item item_dir;
+  };
+
+  MDLockCache(Capability *cap, int op) :
+    MutationImpl(), diri(cap->get_inode()), client_cap(cap), opcode(op) {
+    client_cap->lock_caches.push_back(&item_cap_lock_cache);
+  }
+
+  CInode *get_dir_inode() { return diri; }
+  void set_dir_layout(file_layout_t& layout) {
+    dir_layout = layout;
+  }
+  const file_layout_t& get_dir_layout() const {
+    return dir_layout;
+  }
+
+  void attach_locks();
+  void attach_dirfrags(std::vector<CDir*>&& dfv);
+  void detach_locks();
+  void detach_dirfrags();
+
+  CInode *diri;
+  Capability *client_cap;
+  int opcode;
+  file_layout_t dir_layout;
+
+  elist<MDLockCache*>::item item_cap_lock_cache;
 
+  // link myself to locked locks
+  std::unique_ptr<LockItem[]> items_lock;
+
+  // link myself to auth-pinned dirfrags
+  std::unique_ptr<DirItem[]> items_dir;
+  std::vector<CDir*> auth_pinned_dirfrags;
+
+  int ref = 1;
+  bool invalidating = false;
+};
+
+typedef boost::intrusive_ptr<MutationImpl> MutationRef;
+typedef boost::intrusive_ptr<MDRequestImpl> MDRequestRef;
+
+inline ostream& operator<<(ostream &out, const MutationImpl &mut)
+{
+  mut.print(out);
+  return out;
+}
 #endif