Whamcloud - gitweb
LU-4198 clio: AIO support for direct IO
[fs/lustre-release.git] / lustre / include / cl_object.h
index 78d0926..7979b14 100644 (file)
@@ -23,7 +23,7 @@
  * Copyright (c) 2008, 2010, Oracle and/or its affiliates. All rights reserved.
  * Use is subject to license terms.
  *
- * Copyright (c) 2011, 2016, Intel Corporation.
+ * Copyright (c) 2011, 2017, Intel Corporation.
  */
 /*
  * This file is part of Lustre, http://www.lustre.org/
 /*
  * super-class definitions.
  */
+#include <linux/aio.h>
+#include <linux/fs.h>
+
 #include <libcfs/libcfs.h>
-#include <libcfs/libcfs_ptask.h>
 #include <lu_object.h>
 #include <linux/atomic.h>
 #include <linux/mutex.h>
 #include <linux/radix-tree.h>
 #include <linux/spinlock.h>
 #include <linux/wait.h>
+#include <linux/pagevec.h>
 #include <lustre_dlm.h>
 
 struct obd_info;
@@ -118,8 +121,6 @@ struct cl_io_slice;
 
 struct cl_req_attr;
 
-extern struct cfs_ptask_engine *cl_io_engine;
-
 /**
  * Device in the client stack.
  *
@@ -293,6 +294,8 @@ struct cl_layout {
        u32             cl_layout_gen;
        /** whether layout is a composite one */
        bool            cl_is_composite;
+       /** Whether layout is a HSM released one */
+       bool            cl_is_released;
 };
 
 /**
@@ -415,6 +418,13 @@ struct cl_object_operations {
        void (*coo_req_attr_set)(const struct lu_env *env,
                                 struct cl_object *obj,
                                 struct cl_req_attr *attr);
+       /**
+        * Flush \a obj data corresponding to \a lock. Used for DoM
+        * locks in llite's cancelling blocking ast callback.
+        */
+       int (*coo_object_flush)(const struct lu_env *env,
+                               struct cl_object *obj,
+                               struct ldlm_lock *lock);
 };
 
 /**
@@ -703,7 +713,7 @@ enum cl_page_type {
 
         /** Transient page, the transient cl_page is used to bind a cl_page
          *  to vmpage which is not belonging to the same object of cl_page.
-         *  it is used in DirectIO, lockless IO and liblustre. */
+         *  it is used in DirectIO and lockless IO. */
         CPT_TRANSIENT,
 };
 
@@ -751,6 +761,8 @@ struct cl_page {
        struct lu_ref_link       cp_queue_ref;
        /** Assigned if doing a sync_io */
        struct cl_sync_io       *cp_sync_io;
+       /** layout_entry + stripe index, composed using lov_comp_index() */
+       unsigned int            cp_lov_index;
 };
 
 /**
@@ -864,6 +876,13 @@ struct cl_page_operations {
          */
         int (*cpo_is_vmlocked)(const struct lu_env *env,
                                const struct cl_page_slice *slice);
+
+       /**
+        * Update file attributes when all we have is this page.  Used for tiny
+        * writes to update attributes when we don't have a full cl_io.
+        */
+       void (*cpo_page_touch)(const struct lu_env *env,
+                              const struct cl_page_slice *slice, size_t to);
         /**
          * Page destruction.
          */
@@ -888,7 +907,8 @@ struct cl_page_operations {
                            const struct cl_page_slice *slice);
         /** Destructor. Frees resources and slice itself. */
         void (*cpo_fini)(const struct lu_env *env,
-                         struct cl_page_slice *slice);
+                        struct cl_page_slice *slice,
+                        struct pagevec *pvec);
         /**
          * Optional debugging helper. Prints given page slice.
          *
@@ -1071,15 +1091,13 @@ static inline bool __page_in_use(const struct cl_page *page, int refc)
  * (struct cl_lock) and a list of layers (struct cl_lock_slice), linked to
  * cl_lock::cll_layers list through cl_lock_slice::cls_linkage.
  *
- * Typical cl_lock consists of the two layers:
+ * Typical cl_lock consists of one layer:
  *
- *     - vvp_lock (vvp specific data), and
  *     - lov_lock (lov specific data).
  *
  * lov_lock contains an array of sub-locks. Each of these sub-locks is a
  * normal cl_lock: it has a header (struct cl_lock) and a list of layers:
  *
- *     - lovsub_lock, and
  *     - osc_lock
  *
  * Each sub-lock is associated with a cl_object (representing stripe
@@ -1199,7 +1217,7 @@ struct cl_lock {
 /**
  * Per-layer part of cl_lock
  *
- * \see vvp_lock, lov_lock, lovsub_lock, osc_lock
+ * \see lov_lock, osc_lock
  */
 struct cl_lock_slice {
         struct cl_lock                  *cls_lock;
@@ -1213,7 +1231,7 @@ struct cl_lock_slice {
 
 /**
  *
- * \see vvp_lock_ops, lov_lock_ops, lovsub_lock_ops, osc_lock_ops
+ * \see lov_lock_ops, osc_lock_ops
  */
 struct cl_lock_operations {
        /** @{ */
@@ -1225,8 +1243,7 @@ struct cl_lock_operations {
         *              @anchor for resources
         * \retval -ve  failure
         *
-        * \see vvp_lock_enqueue(), lov_lock_enqueue(), lovsub_lock_enqueue(),
-        * \see osc_lock_enqueue()
+        * \see lov_lock_enqueue(), osc_lock_enqueue()
         */
        int  (*clo_enqueue)(const struct lu_env *env,
                            const struct cl_lock_slice *slice,
@@ -1241,8 +1258,7 @@ struct cl_lock_operations {
        /**
         * Destructor. Frees resources and the slice.
         *
-        * \see vvp_lock_fini(), lov_lock_fini(), lovsub_lock_fini(),
-        * \see osc_lock_fini()
+        * \see lov_lock_fini(), osc_lock_fini()
         */
         void (*clo_fini)(const struct lu_env *env, struct cl_lock_slice *slice);
         /**
@@ -1297,7 +1313,7 @@ struct cl_page_list {
        struct task_struct      *pl_owner;
 };
 
-/** 
+/**
  * A 2-queue of pages. A convenience data-type for common use case, 2-queue
  * contains an incoming page list and an outgoing page list.
  */
@@ -1379,6 +1395,10 @@ enum cl_io_type {
         */
        CIT_FSYNC,
        /**
+        * glimpse. An io context to acquire glimpse lock.
+        */
+       CIT_GLIMPSE,
+       /**
          * Miscellaneous io. This is used for occasional io activity that
          * doesn't fit into other types. Currently this is used for:
          *
@@ -1389,8 +1409,6 @@ enum cl_io_type {
          *     - VM induced page write-out. An io context for writing page out
          *     for memory cleansing;
          *
-         *     - glimpse. An io context to acquire glimpse lock.
-         *
          *     - grouplock. An io context to acquire group lock.
          *
          * CIT_MISC io is used simply as a context in which locks and pages
@@ -1452,20 +1470,22 @@ struct cl_io_slice {
 };
 
 typedef void (*cl_commit_cbt)(const struct lu_env *, struct cl_io *,
-                             struct cl_page *);
+                             struct pagevec *);
 
 struct cl_read_ahead {
        /* Maximum page index the readahead window will end.
         * This is determined DLM lock coverage, RPC and stripe boundary.
         * cra_end is included. */
-       pgoff_t cra_end;
+       pgoff_t         cra_end_idx;
        /* optimal RPC size for this read, by pages */
-       unsigned long cra_rpc_size;
+       unsigned long   cra_rpc_pages;
        /* Release callback. If readahead holds resources underneath, this
         * function should be called to release it. */
-       void    (*cra_release)(const struct lu_env *env, void *cbdata);
+       void            (*cra_release)(const struct lu_env *env, void *cbdata);
        /* Callback data for cra_release routine */
-       void    *cra_cbdata;
+       void            *cra_cbdata;
+       /* whether lock is in contention */
+       bool            cra_contention;
 };
 
 static inline void cl_read_ahead_release(const struct lu_env *env,
@@ -1607,25 +1627,30 @@ enum cl_enq_flags {
          * -EWOULDBLOCK is returned immediately.
          */
         CEF_NONBLOCK     = 0x00000001,
-        /**
-         * take lock asynchronously (out of order), as it cannot
-         * deadlock. This is for LDLM_FL_HAS_INTENT locks used for glimpsing.
-         */
-        CEF_ASYNC        = 0x00000002,
+       /**
+        * Tell lower layers this is a glimpse request, translated to
+        * LDLM_FL_HAS_INTENT at LDLM layer.
+        *
+        * Also, because glimpse locks never block other locks, we count this
+        * as automatically compatible with other osc locks.
+        * (see osc_lock_compatible)
+        */
+       CEF_GLIMPSE        = 0x00000002,
         /**
          * tell the server to instruct (though a flag in the blocking ast) an
          * owner of the conflicting lock, that it can drop dirty pages
          * protected by this lock, without sending them to the server.
          */
         CEF_DISCARD_DATA = 0x00000004,
-        /**
-         * tell the sub layers that it must be a `real' lock. This is used for
-         * mmapped-buffer locks and glimpse locks that must be never converted
-         * into lockless mode.
-         *
-         * \see vvp_mmap_locks(), cl_glimpse_lock().
-         */
-        CEF_MUST         = 0x00000008,
+       /**
+        * tell the sub layers that it must be a `real' lock. This is used for
+        * mmapped-buffer locks, glimpse locks, manually requested locks
+        * (LU_LADVISE_LOCKAHEAD) that must never be converted into lockless
+        * mode.
+        *
+        * \see vvp_mmap_locks(), cl_glimpse_lock, cl_request_lock().
+        */
+       CEF_MUST         = 0x00000008,
         /**
          * tell the sub layers that never request a `real' lock. This flag is
          * not used currently.
@@ -1638,9 +1663,16 @@ enum cl_enq_flags {
          */
         CEF_NEVER        = 0x00000010,
         /**
-         * for async glimpse lock.
+        * tell the dlm layer this is a speculative lock request
+        * speculative lock requests are locks which are not requested as part
+        * of an I/O operation.  Instead, they are requested because we expect
+        * to use them in the future.  They are requested asynchronously at the
+        * ptlrpc layer.
+        *
+        * Currently used for asynchronous glimpse locks and manually requested
+        * locks (LU_LADVISE_LOCKAHEAD).
          */
-        CEF_AGL          = 0x00000020,
+       CEF_SPECULATIVE          = 0x00000020,
        /**
         * enqueue a lock to test DLM lock existence.
         */
@@ -1651,9 +1683,13 @@ enum cl_enq_flags {
         */
        CEF_LOCK_MATCH  = 0x00000080,
        /**
+        * tell the DLM layer to lock only the requested range
+        */
+       CEF_LOCK_NO_EXPAND    = 0x00000100,
+       /**
         * mask of enq_flags.
         */
-       CEF_MASK         = 0x000000ff,
+       CEF_MASK         = 0x000001ff,
 };
 
 /**
@@ -1731,21 +1767,10 @@ enum cl_fsync_mode {
        CL_FSYNC_ALL   = 3
 };
 
-struct cl_io_range {
-       loff_t cir_pos;
-       size_t cir_count;
-};
-
-struct cl_io_pt {
-       struct cl_io_pt         *cip_next;
-       struct cfs_ptask         cip_task;
-       struct kiocb             cip_iocb;
-       struct iov_iter          cip_iter;
-       struct file             *cip_file;
-       enum cl_io_type          cip_iot;
-       loff_t                   cip_pos;
-       size_t                   cip_count;
-       ssize_t                  cip_result;
+struct cl_io_rw_common {
+       loff_t  crw_pos;
+       size_t  crw_count;
+       int     crw_nonblock;
 };
 
 /**
@@ -1775,27 +1800,30 @@ struct cl_io {
         struct cl_lockset              ci_lockset;
         /** lock requirements, this is just a help info for sublayers. */
         enum cl_io_lock_dmd            ci_lockreq;
-        union {
-               struct cl_rw_io {
-                       struct iov_iter          rw_iter;
-                       struct kiocb             rw_iocb;
-                       struct cl_io_range       rw_range;
-                       struct file             *rw_file;
-                       unsigned int             rw_nonblock:1,
-                                                rw_append:1,
-                                                rw_sync:1;
-                       int (*rw_ptask)(struct cfs_ptask *ptask);
-               } ci_rw;
+       /** layout version when this IO occurs */
+       __u32                           ci_layout_version;
+       union {
+               struct cl_rd_io {
+                       struct cl_io_rw_common rd;
+               } ci_rd;
+               struct cl_wr_io {
+                       struct cl_io_rw_common wr;
+                       int                    wr_append;
+                       int                    wr_sync;
+               } ci_wr;
+               struct cl_io_rw_common ci_rw;
                struct cl_setattr_io {
                        struct ost_lvb           sa_attr;
                        unsigned int             sa_attr_flags;
-                       unsigned int             sa_valid;
+                       unsigned int             sa_avalid; /* ATTR_* */
+                       unsigned int             sa_xvalid; /* OP_XVALID */
                        int                      sa_stripe_index;
                        struct ost_layout        sa_layout;
                        const struct lu_fid     *sa_parent_fid;
                } ci_setattr;
                struct cl_data_version_io {
                        u64 dv_data_version;
+                       u32 dv_layout_version;
                        int dv_flags;
                } ci_data_version;
                 struct cl_fault_io {
@@ -1850,8 +1878,10 @@ struct cl_io {
         */
                             ci_ignore_layout:1,
        /**
-        * Need MDS intervention to complete a write. This usually means the
-        * corresponding component is not initialized for the writing extent.
+        * Need MDS intervention to complete a write.
+        * Write intent is required for the following cases:
+        * 1. component being written is not initialized, or
+        * 2. the mirrored files are NOT in WRITE_PENDING state.
         */
                             ci_need_write_intent:1,
        /**
@@ -1870,12 +1900,47 @@ struct cl_io {
         * O_NOATIME
         */
                             ci_noatime:1,
-       /** Set to 1 if parallel execution is allowed for current I/O? */
-                            ci_pio:1;
+       /* Tell sublayers not to expand LDLM locks requested for this IO */
+                            ci_lock_no_expand:1,
+       /**
+        * Set if non-delay RPC should be used for this IO.
+        *
+        * If this file has multiple mirrors, and if the OSTs of the current
+        * mirror is inaccessible, non-delay RPC would error out quickly so
+        * that the upper layer can try to access the next mirror.
+        */
+                            ci_ndelay:1,
+       /**
+        * Set if IO is triggered by async workqueue readahead.
+        */
+                            ci_async_readahead:1,
+       /**
+        * Ignore lockless and do normal locking for this io.
+        */
+                            ci_ignore_lockless:1,
+       /**
+        * Set if we've tried all mirrors for this read IO, if it's not set,
+        * the read IO will check to-be-read OSCs' status, and make fast-switch
+        * another mirror if some of the OSTs are not healthy.
+        */
+                            ci_tried_all_mirrors:1;
+       /**
+        * How many times the read has retried before this one.
+        * Set by the top level and consumed by the LOV.
+        */
+       unsigned             ci_ndelay_tried;
+       /**
+        * Designated mirror index for this I/O.
+        */
+       unsigned             ci_designated_mirror;
        /**
         * Number of pages owned by this IO. For invariant checking.
         */
        unsigned             ci_owned_nr;
+       /**
+        * Range of write intent. Valid if ci_need_write_intent is set.
+        */
+       struct lu_extent        ci_write_intent;
 };
 
 /** @} cl_io */
@@ -2058,6 +2123,9 @@ int cl_object_fiemap(const struct lu_env *env, struct cl_object *obj,
 int cl_object_layout_get(const struct lu_env *env, struct cl_object *obj,
                         struct cl_layout *cl);
 loff_t cl_object_maxbytes(struct cl_object *obj);
+int cl_object_flush(const struct lu_env *env, struct cl_object *obj,
+                   struct ldlm_lock *lock);
+
 
 /**
  * Returns true, iff \a o0 and \a o1 are slices of the same object.
@@ -2112,6 +2180,9 @@ struct cl_page *cl_page_alloc       (const struct lu_env *env,
 void            cl_page_get         (struct cl_page *page);
 void            cl_page_put         (const struct lu_env *env,
                                      struct cl_page *page);
+void           cl_pagevec_put      (const struct lu_env *env,
+                                    struct cl_page *page,
+                                    struct pagevec *pvec);
 void            cl_page_print       (const struct lu_env *env, void *cookie,
                                      lu_printer_t printer,
                                      const struct cl_page *pg);
@@ -2179,6 +2250,8 @@ void    cl_page_discard(const struct lu_env *env, struct cl_io *io,
 void    cl_page_delete(const struct lu_env *env, struct cl_page *pg);
 int     cl_page_is_vmlocked(const struct lu_env *env,
                            const struct cl_page *pg);
+void   cl_page_touch(const struct lu_env *env, const struct cl_page *pg,
+                     size_t to);
 void    cl_page_export(const struct lu_env *env,
                       struct cl_page *pg, int uptodate);
 loff_t  cl_offset(const struct cl_object *obj, pgoff_t idx);
@@ -2306,12 +2379,12 @@ int   cl_io_cancel       (const struct lu_env *env, struct cl_io *io,
  */
 static inline int cl_io_is_append(const struct cl_io *io)
 {
-       return io->ci_type == CIT_WRITE && io->u.ci_rw.rw_append;
+       return io->ci_type == CIT_WRITE && io->u.ci_wr.wr_append;
 }
 
 static inline int cl_io_is_sync_write(const struct cl_io *io)
 {
-       return io->ci_type == CIT_WRITE && io->u.ci_rw.rw_sync;
+       return io->ci_type == CIT_WRITE && io->u.ci_wr.wr_sync;
 }
 
 static inline int cl_io_is_mkwrite(const struct cl_io *io)
@@ -2324,8 +2397,8 @@ static inline int cl_io_is_mkwrite(const struct cl_io *io)
  */
 static inline int cl_io_is_trunc(const struct cl_io *io)
 {
-        return io->ci_type == CIT_SETATTR &&
-                (io->u.ci_setattr.sa_valid & ATTR_SIZE);
+       return io->ci_type == CIT_SETATTR &&
+               (io->u.ci_setattr.sa_avalid & ATTR_SIZE);
 }
 
 struct cl_io *cl_io_top(struct cl_io *io);
@@ -2333,13 +2406,12 @@ struct cl_io *cl_io_top(struct cl_io *io);
 void cl_io_print(const struct lu_env *env, void *cookie,
                  lu_printer_t printer, const struct cl_io *io);
 
-#define CL_IO_SLICE_CLEAN(foo_io, base)                                 \
-do {                                                                    \
-        typeof(foo_io) __foo_io = (foo_io);                             \
-                                                                        \
-        CLASSERT(offsetof(typeof(*__foo_io), base) == 0);               \
-        memset(&__foo_io->base + 1, 0,                                  \
-               (sizeof *__foo_io) - sizeof __foo_io->base);             \
+#define CL_IO_SLICE_CLEAN(foo_io, base)                                        \
+do {                                                                   \
+       typeof(foo_io) __foo_io = (foo_io);                             \
+                                                                       \
+       memset(&__foo_io->base, 0,                                      \
+              sizeof(*__foo_io) - offsetof(typeof(*__foo_io), base));  \
 } while (0)
 
 /** @} cl_io */
@@ -2411,6 +2483,23 @@ void cl_req_attr_set(const struct lu_env *env, struct cl_object *obj,
 /** \defgroup cl_sync_io cl_sync_io
  * @{ */
 
+struct cl_sync_io;
+struct cl_dio_aio;
+
+typedef void (cl_sync_io_end_t)(const struct lu_env *, struct cl_sync_io *);
+
+void cl_sync_io_init_notify(struct cl_sync_io *anchor, int nr,
+                           struct cl_dio_aio *aio, cl_sync_io_end_t *end);
+
+int  cl_sync_io_wait(const struct lu_env *env, struct cl_sync_io *anchor,
+                    long timeout);
+void cl_sync_io_note(const struct lu_env *env, struct cl_sync_io *anchor,
+                    int ioret);
+static inline void cl_sync_io_init(struct cl_sync_io *anchor, int nr)
+{
+       cl_sync_io_init_notify(anchor, nr, NULL, NULL);
+}
+
 /**
  * Anchor for synchronous transfer. This is allocated on a stack by thread
  * doing synchronous transfer, and a pointer to this structure is set up in
@@ -2422,22 +2511,21 @@ struct cl_sync_io {
        atomic_t                csi_sync_nr;
        /** error code. */
        int                     csi_sync_rc;
-       /** barrier of destroy this structure */
-       atomic_t                csi_barrier;
        /** completion to be signaled when transfer is complete. */
        wait_queue_head_t       csi_waitq;
        /** callback to invoke when this IO is finished */
-       void                    (*csi_end_io)(const struct lu_env *,
-                                             struct cl_sync_io *);
+       cl_sync_io_end_t       *csi_end_io;
+       /** aio private data */
+       struct cl_dio_aio      *csi_aio;
 };
 
-void cl_sync_io_init(struct cl_sync_io *anchor, int nr,
-                    void (*end)(const struct lu_env *, struct cl_sync_io *));
-int  cl_sync_io_wait(const struct lu_env *env, struct cl_sync_io *anchor,
-                    long timeout);
-void cl_sync_io_note(const struct lu_env *env, struct cl_sync_io *anchor,
-                    int ioret);
-void cl_sync_io_end(const struct lu_env *env, struct cl_sync_io *anchor);
+/** To support Direct AIO */
+struct cl_dio_aio {
+       struct cl_sync_io       cda_sync;
+       struct cl_page_list     cda_pages;
+       struct kiocb            *cda_iocb;
+       ssize_t                 cda_bytes;
+};
 
 /** @} cl_sync_io */