Whamcloud - gitweb
LU-8130 lov: convert lo[v|d]_pool to use rhashtable
[fs/lustre-release.git] / lustre / include / obd.h
index 4094966..8461df7 100644 (file)
@@ -23,7 +23,7 @@
  * Copyright (c) 2007, 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/
 #ifndef __OBD_H
 #define __OBD_H
 
+#include <linux/kobject.h>
 #include <linux/spinlock.h>
+#include <linux/sysfs.h>
 
-#include <lustre/lustre_idl.h>
+#include <uapi/linux/lustre/lustre_idl.h>
 #include <lustre_lib.h>
 #include <libcfs/bitmap.h>
 #ifdef HAVE_SERVER_SUPPORT
@@ -76,10 +78,6 @@ static inline void loi_kms_set(struct lov_oinfo *oinfo, __u64 kms)
         oinfo->loi_kms_valid = 1;
 }
 
-static inline void loi_init(struct lov_oinfo *loi)
-{
-}
-
 struct lov_stripe_md;
 struct obd_info;
 
@@ -89,6 +87,8 @@ typedef int (*obd_enqueue_update_f)(void *cookie, int rc);
 struct obd_info {
        /* OBD_STATFS_* flags */
        __u64                   oi_flags;
+       struct obd_device      *oi_obd;
+       struct lu_tgt_desc     *oi_tgt;
         /* statfs data specific for every OSC, if needed at all. */
         struct obd_statfs      *oi_osfs;
         /* An update callback which is called to update some data on upper
@@ -99,17 +99,18 @@ struct obd_info {
 };
 
 struct obd_type {
-       struct list_head         typ_chain;
-       struct obd_ops          *typ_dt_ops;
-       struct md_ops           *typ_md_ops;
+       const struct obd_ops    *typ_dt_ops;
+       const struct md_ops     *typ_md_ops;
        struct proc_dir_entry   *typ_procroot;
-       struct proc_dir_entry   *typ_procsym;
-       __u32                    typ_sym_filter;
-       char                    *typ_name;
-       int                      typ_refcnt;
+       struct dentry           *typ_debugfs_entry;
+#ifdef HAVE_SERVER_SUPPORT
+       bool                     typ_sym_filter;
+#endif
+       atomic_t                 typ_refcnt;
        struct lu_device_type   *typ_lu;
-       spinlock_t               obd_type_lock;
+       struct kobject           typ_kobj;
 };
+#define typ_name typ_kobj.name
 
 struct brw_page {
        u64              off;
@@ -120,7 +121,7 @@ struct brw_page {
 
 struct timeout_item {
        enum timeout_event ti_event;
-       cfs_time_t         ti_timeout;
+       time64_t           ti_timeout;
        timeout_cb_t       ti_cb;
        void              *ti_cb_data;
        struct list_head   ti_obd_list;
@@ -130,16 +131,15 @@ struct timeout_item {
 #define OBD_MAX_RIF_DEFAULT    8
 #define OBD_MAX_RIF_MAX                512
 #define OSC_MAX_RIF_MAX                256
-#define OSC_MAX_DIRTY_DEFAULT  (OBD_MAX_RIF_DEFAULT * 4)
+#define OSC_MAX_DIRTY_DEFAULT  2000     /* Arbitrary large value */
 #define OSC_MAX_DIRTY_MB_MAX   2048     /* arbitrary, but < MAX_LONG bytes */
 #define OSC_DEFAULT_RESENDS    10
 
-/* possible values for fo_sync_lock_cancel */
-enum {
-        NEVER_SYNC_ON_CANCEL = 0,
-        BLOCKING_SYNC_ON_CANCEL = 1,
-        ALWAYS_SYNC_ON_CANCEL = 2,
-        NUM_SYNC_ON_CANCEL_STATES
+/* possible values for lut_sync_lock_cancel */
+enum tgt_sync_lock_cancel {
+       SYNC_LOCK_CANCEL_NEVER    = 0,
+       SYNC_LOCK_CANCEL_BLOCKING = 1,
+       SYNC_LOCK_CANCEL_ALWAYS   = 2,
 };
 
 /*
@@ -149,7 +149,15 @@ enum {
  * vmalloc(). Excessive use of vmalloc() may cause spinlock contention
  * on the MDS.
  */
-#define OBD_MAX_DEFAULT_EA_SIZE                4096
+#define OBD_MAX_DEFAULT_EA_SIZE        4096
+
+/*
+ * Lustre can handle larger xattrs internally, but we must respect the Linux
+ * VFS limitation or tools like tar cannot interact with Lustre volumes
+ * correctly.
+ */
+#define OBD_MAX_EA_SIZE                XATTR_SIZE_MAX
+
 
 enum obd_cl_sem_lock_class {
        OBD_CLI_SEM_NORMAL,
@@ -185,6 +193,17 @@ struct client_obd {
         * run-time if a larger observed size is advertised by the MDT. */
        __u32                    cl_max_mds_easize;
 
+       /* Data-on-MDT specific value to set larger reply buffer for possible
+        * data read along with open/stat requests. By default it tries to use
+        * unused space in reply buffer.
+        * This value is used to ensure that reply buffer has at least as
+        * much free space as value indicates. That free space is gained from
+        * LOV EA buffer which is small for DoM files and on big systems can
+        * provide up to 32KB of extra space in reply buffer.
+        * Default value is 8K now.
+        */
+       __u32                    cl_dom_min_inline_repsize;
+
        enum lustre_sec_part     cl_sp_me;
        enum lustre_sec_part     cl_sp_to;
        struct sptlrpc_flavor    cl_flvr_mgc; /* fixed flavor of mgc->mgs */
@@ -192,7 +211,6 @@ struct client_obd {
        /* the grant values are protected by loi_list_lock below */
        unsigned long            cl_dirty_pages;      /* all _dirty_ in pages */
        unsigned long            cl_dirty_max_pages;  /* allowed w/o rpc */
-       unsigned long            cl_dirty_transit;    /* dirty synchronous */
        unsigned long            cl_avail_grant;   /* bytes of credit for ost */
        unsigned long            cl_lost_grant;    /* lost credits (trunc) */
        /* grant consumed for dirty pages */
@@ -204,9 +222,9 @@ struct client_obd {
         * See osc_{reserve|unreserve}_grant for details. */
        long                    cl_reserved_grant;
        struct list_head        cl_cache_waiters; /* waiting for cache/grant */
-       cfs_time_t              cl_next_shrink_grant;   /* jiffies */
-       struct list_head        cl_grant_shrink_list;  /* Timeout event list */
-       int                     cl_grant_shrink_interval; /* seconds */
+       time64_t                cl_next_shrink_grant;   /* seconds */
+       struct list_head        cl_grant_chain;
+       time64_t                cl_grant_shrink_interval; /* seconds */
 
        /* A chunk is an optimal size used by osc_extent to determine
         * the extent size. A chunk is max(PAGE_SIZE, OST block size) */
@@ -244,8 +262,9 @@ struct client_obd {
        /* just a sum of the loi/lop pending numbers to be exported by /proc */
        atomic_t                cl_pending_w_pages;
        atomic_t                cl_pending_r_pages;
-       __u32                   cl_max_pages_per_rpc;
-       __u32                   cl_max_rpcs_in_flight;
+       u32                     cl_max_pages_per_rpc;
+       u32                     cl_max_rpcs_in_flight;
+       u32                     cl_max_short_io_bytes;
        struct obd_histogram    cl_read_rpc_hist;
        struct obd_histogram    cl_write_rpc_hist;
        struct obd_histogram    cl_read_page_hist;
@@ -292,8 +311,6 @@ struct client_obd {
        atomic_t                 cl_destroy_in_flight;
        wait_queue_head_t        cl_destroy_waitq;
 
-        struct mdc_rpc_lock     *cl_rpc_lock;
-
        /* modify rpcs in flight
         * currently used for metadata only */
        spinlock_t               cl_mod_rpcs_lock;
@@ -308,29 +325,41 @@ struct client_obd {
        struct mutex              cl_mgc_mutex;
        struct local_oid_storage *cl_mgc_los;
        struct dt_object         *cl_mgc_configs_dir;
-       atomic_t                  cl_mgc_refcount;
        struct obd_export        *cl_mgc_mgsexp;
+       atomic_t                  cl_mgc_refcount;
+       /* in-flight control list and total RPCs counter */
+       struct list_head         cl_flight_waiters;
+       __u32                    cl_rpcs_in_flight;
 
         /* checksumming for data sent over the network */
-        unsigned int             cl_checksum:1; /* 0 = disabled, 1 = enabled */
+       unsigned int             cl_checksum:1, /* 0 = disabled, 1 = enabled */
+                                cl_checksum_dump:1; /* same */
         /* supported checksum types that are worked out at connect time */
         __u32                    cl_supp_cksum_types;
         /* checksum algorithm to be used */
-        cksum_type_t             cl_cksum_type;
+       enum cksum_types         cl_cksum_type;
+       /* preferred checksum algorithm to be used */
+       enum cksum_types         cl_preferred_cksum_type;
 
         /* also protected by the poorly named _loi_list_lock lock above */
         struct osc_async_rc      cl_ar;
 
        /* sequence manager */
        struct lu_client_seq    *cl_seq;
+       struct rw_semaphore      cl_seq_rwsem;
 
-       atomic_t             cl_resends; /* resend count */
+       atomic_t                 cl_resends; /* resend count */
 
        /* ptlrpc work for writeback in ptlrpcd context */
        void                    *cl_writeback_work;
        void                    *cl_lru_work;
+       struct mutex              cl_quota_mutex;
        /* hash tables for osc_quota_info */
        struct cfs_hash         *cl_quota_hash[LL_MAXQUOTAS];
+       /* the xid of the request updating the hash tables */
+       __u64                    cl_quota_last_xid;
+       /* Links to the global list of registered changelog devices */
+       struct list_head         cl_chg_dev_linkage;
 };
 #define obd2cli_tgt(obd) ((char *)(obd)->u.cli.cl_target_uuid.uuid)
 
@@ -347,34 +376,22 @@ struct echo_client_obd {
        __u64                   ec_unique;
 };
 
-/* Generic subset of OSTs */
-struct ost_pool {
-        __u32              *op_array;      /* array of index of
-                                                   lov_obd->lov_tgts */
-        unsigned int        op_count;      /* number of OSTs in the array */
-        unsigned int        op_size;       /* allocated size of lp_array */
-       struct rw_semaphore op_rw_sem;     /* to protect ost_pool use */
-};
-
 /* allow statfs data caching for 1 second */
 #define OBD_STATFS_CACHE_SECONDS 1
+/* arbitrary maximum. larger would be useless, allows catching bogus input */
+#define OBD_STATFS_CACHE_MAX_AGE 3600 /* seconds */
 
-struct lov_tgt_desc {
-       struct list_head    ltd_kill;
-        struct obd_uuid     ltd_uuid;
-        struct obd_device  *ltd_obd;
-        struct obd_export  *ltd_exp;
-        __u32               ltd_gen;
-        __u32               ltd_index;   /* index in lov_obd->tgts */
-        unsigned long       ltd_active:1,/* is this target up for requests */
-                            ltd_activate:1,/* should  target be activated */
-                            ltd_reap:1;  /* should this target be deleted */
+#define lov_tgt_desc lu_tgt_desc
+
+struct lov_md_tgt_desc {
+       struct obd_device *lmtd_mdc;
+       __u32              lmtd_index;
 };
 
 struct lov_obd {
        struct lov_desc         desc;
        struct lov_tgt_desc   **lov_tgts;               /* sparse array */
-       struct ost_pool         lov_packed;             /* all OSTs in a packed
+       struct lu_tgt_pool      lov_packed;             /* all OSTs in a packed
                                                           array */
        struct mutex            lov_lock;
        struct obd_connect_data lov_ocd;
@@ -384,7 +401,7 @@ struct lov_obd {
        __u32                   lov_tgt_size;   /* size of tgts array */
        int                     lov_connects;
        int                     lov_pool_count;
-       struct cfs_hash        *lov_pools_hash_body; /* used for key access */
+       struct rhashtable       lov_pools_hash_body; /* used for key access */
        struct list_head        lov_pool_list;  /* used for sequential access */
        struct proc_dir_entry  *lov_pool_proc_entry;
        enum lustre_sec_part    lov_sp_me;
@@ -393,34 +410,38 @@ struct lov_obd {
        struct cl_client_cache *lov_cache;
 
        struct rw_semaphore     lov_notify_lock;
-};
+       /* Data-on-MDT: MDC array */
+       struct lov_md_tgt_desc  *lov_mdc_tgts;
 
-struct lmv_tgt_desc {
-       struct obd_uuid         ltd_uuid;
-       struct obd_export       *ltd_exp;
-       __u32                   ltd_idx;
-       struct mutex            ltd_fid_mutex;
-       unsigned long           ltd_active:1; /* target up for requests */
+       struct kobject          *lov_tgts_kobj;
 };
 
+#define lmv_tgt_desc lu_tgt_desc
+
 struct lmv_obd {
        struct lu_client_fld    lmv_fld;
        spinlock_t              lmv_lock;
-       struct lmv_desc         desc;
-       struct obd_uuid         cluuid;
-       struct proc_dir_entry   *targets_proc_entry;
 
-       struct mutex            lmv_init_mutex;
        int                     connected;
        int                     max_easize;
        int                     max_def_easize;
+       u32                     lmv_statfs_start;
 
-       __u32                   tgts_size; /* size of tgts array */
-       struct lmv_tgt_desc     **tgts;
+       struct lu_tgt_descs     lmv_mdt_descs;
 
        struct obd_connect_data conn_data;
+       struct kobject          *lmv_tgts_kobj;
+       void                    *lmv_cache;
+
+       __u32                   lmv_qos_rr_index;
 };
 
+#define lmv_mdt_count  lmv_mdt_descs.ltd_lmv_desc.ld_tgt_count
+#define lmv_qos                lmv_mdt_descs.ltd_qos
+
+/* Minimum sector size is 512 */
+#define MAX_GUARD_NUMBER (PAGE_SIZE / 512)
+
 struct niobuf_local {
        __u64           lnb_file_offset;
        __u32           lnb_page_offset;
@@ -429,6 +450,11 @@ struct niobuf_local {
        int             lnb_rc;
        struct page     *lnb_page;
        void            *lnb_data;
+       __u16           lnb_guards[MAX_GUARD_NUMBER];
+       __u16           lnb_guard_rpc:1;
+       __u16           lnb_guard_disk:1;
+       /* separate unlock for read path to allow shared access */
+       __u16           lnb_locked:1;
 };
 
 struct tgt_thread_big_cache {
@@ -534,30 +560,18 @@ enum obd_notify_event {
         OBD_NOTIFY_INACTIVE,
         /* Connect data for import were changed */
         OBD_NOTIFY_OCD,
-        /* Sync request */
-        OBD_NOTIFY_SYNC_NONBLOCK,
-        OBD_NOTIFY_SYNC,
-        /* Configuration event */
-        OBD_NOTIFY_CONFIG,
         /* Administratively deactivate/activate event */
         OBD_NOTIFY_DEACTIVATE,
         OBD_NOTIFY_ACTIVATE
 };
 
-/* bit-mask flags for config events */
-enum config_flags {
-        CONFIG_LOG      = 0x1,  /* finished processing config log */
-        CONFIG_SYNC     = 0x2,  /* mdt synced 1 ost */
-        CONFIG_TARGET   = 0x4   /* one target is added */
-};
-
 /*
  * Data structure used to pass obd_notify()-event to non-obd listeners (llite
- * and liblustre being main examples).
+ * being main example).
  */
 struct obd_notify_upcall {
-        int (*onu_upcall)(struct obd_device *host, struct obd_device *watched,
-                          enum obd_notify_event ev, void *owner, void *data);
+       int (*onu_upcall)(struct obd_device *host, struct obd_device *watched,
+                         enum obd_notify_event ev, void *owner);
         /* Opaque datum supplied by upper layer listener */
         void *onu_owner;
 };
@@ -573,7 +587,6 @@ struct obd_llog_group {
        struct llog_ctxt   *olg_ctxts[LLOG_MAX_CTXTS];
        wait_queue_head_t  olg_waitq;
        spinlock_t         olg_lock;
-       struct mutex       olg_cat_processing;
 };
 
 /* corresponds to one of the obd's */
@@ -608,13 +621,13 @@ struct obd_device {
                                         * (for /proc/status only!!) */
                obd_no_ir:1,            /* no imperative recovery. */
                obd_process_conf:1,     /* device is processing mgs config */
-               obd_uses_nid_stats:1;   /* maintain per-client OBD stats */
+               obd_checksum_dump:1;    /* dump pages upon cksum error */
 
         /* use separate field as it is set in interrupt to don't mess with
          * protection of other bits using _bh lock */
         unsigned long obd_recovery_expired:1;
         /* uuid-export hash body */
-       struct cfs_hash             *obd_uuid_hash;
+       struct rhashtable               obd_uuid_hash;
         /* nid-export hash body */
        struct cfs_hash             *obd_nid_hash;
        /* nid stats body */
@@ -635,7 +648,7 @@ struct obd_device {
        spinlock_t              obd_dev_lock; /* protect OBD bitfield above */
        spinlock_t              obd_osfs_lock;
        struct obd_statfs       obd_osfs;       /* locked by obd_osfs_lock */
-       __u64                   obd_osfs_age;
+       time64_t                obd_osfs_age;
        __u64                   obd_last_committed;
        struct mutex            obd_dev_mutex;
        struct lvfs_run_ctxt    obd_lvfs_ctxt;
@@ -647,9 +660,9 @@ struct obd_device {
        struct obd_export       *obd_lwp_export;
        /* list of exports in LRU order, for ping evictor, with obd_dev_lock */
        struct list_head        obd_exports_timed;
-       time_t                  obd_eviction_timer;     /* for ping evictor */
+       time64_t                obd_eviction_timer;     /* for ping evictor */
 
-       int                     obd_max_recoverable_clients;
+       atomic_t                obd_max_recoverable_clients;
        atomic_t                obd_connected_clients;
        int                     obd_stale_clients;
         /* this lock protects all recovery list_heads, timer and
@@ -660,13 +673,16 @@ struct obd_device {
        int                     obd_requests_queued_for_recovery;
        wait_queue_head_t       obd_next_transno_waitq;
        /* protected by obd_recovery_task_lock */
-       struct timer_list       obd_recovery_timer;
+       struct hrtimer          obd_recovery_timer;
        /* seconds */
-       time_t                  obd_recovery_start;
+       time64_t                obd_recovery_start;
        /* seconds, for lprocfs_status */
-       time_t                  obd_recovery_end;
-       int                     obd_recovery_time_hard;
-       int                     obd_recovery_timeout;
+       time64_t                obd_recovery_end;
+       /* To tell timeouts from time stamps Lustre uses time_t
+        * instead of time64_t.
+        */
+       time_t                  obd_recovery_time_hard;
+       time_t                  obd_recovery_timeout;
        int                     obd_recovery_ir_factor;
 
        /* new recovery stuff from CMD2 */
@@ -695,15 +711,15 @@ struct obd_device {
 
        /* Fields used by LProcFS */
        struct lprocfs_stats            *obd_stats;
-       unsigned int                    obd_cntr_base;
 
-       unsigned int                     obd_md_cntr_base;
        struct lprocfs_stats            *obd_md_stats;
 
+       struct dentry                   *obd_debugfs_entry;
        struct proc_dir_entry   *obd_proc_entry;
        struct proc_dir_entry   *obd_proc_exports_entry;
-       struct proc_dir_entry   *obd_svc_procroot;
+       struct dentry                   *obd_svc_debugfs_entry;
        struct lprocfs_stats    *obd_svc_stats;
+       const struct attribute         **obd_attrs;
        struct lprocfs_vars     *obd_vars;
        atomic_t                obd_evict_inprogress;
        wait_queue_head_t       obd_evict_inprogress_waitq;
@@ -721,8 +737,19 @@ struct obd_device {
        /**
         * List of outstanding class_incref()'s fo this OBD. For debugging. */
        struct lu_ref                   obd_reference;
+
+       struct kset                     obd_kset; /* sysfs object collection */
+       struct kobj_type                obd_ktype;
+       struct completion               obd_kobj_unregister;
 };
 
+int obd_uuid_add(struct obd_device *obd, struct obd_export *export);
+void obd_uuid_del(struct obd_device *obd, struct obd_export *export);
+#ifdef HAVE_SERVER_SUPPORT
+struct obd_export *obd_uuid_lookup(struct obd_device *obd,
+                                  struct obd_uuid *uuid);
+#endif
+
 /* get/set_info keys */
 #define KEY_ASYNC               "async"
 #define KEY_CHANGELOG_CLEAR     "changelog_clear"
@@ -749,10 +776,20 @@ struct obd_device {
 /*      KEY_SET_INFO in lustre_idl.h */
 #define KEY_SPTLRPC_CONF        "sptlrpc_conf"
 
-#define KEY_CACHE_SET          "cache_set"
 #define KEY_CACHE_LRU_SHRINK   "cache_lru_shrink"
 #define KEY_OSP_CONNECTED      "osp_connected"
 
+/* Flags for op_xvalid */
+enum op_xvalid {
+       OP_XVALID_CTIME_SET     = BIT(0),       /* 0x0001 */
+       OP_XVALID_BLOCKS        = BIT(1),       /* 0x0002 */
+       OP_XVALID_OWNEROVERRIDE = BIT(2),       /* 0x0004 */
+       OP_XVALID_FLAGS         = BIT(3),       /* 0x0008 */
+       OP_XVALID_PROJID        = BIT(4),       /* 0x0010 */
+       OP_XVALID_LAZYSIZE      = BIT(5),       /* 0x0020 */
+       OP_XVALID_LAZYBLOCKS    = BIT(6),       /* 0x0040 */
+};
+
 struct lu_context;
 
 static inline int it_to_lock_mode(struct lookup_intent *it)
@@ -760,15 +797,14 @@ static inline int it_to_lock_mode(struct lookup_intent *it)
        /* CREAT needs to be tested before open (both could be set) */
        if (it->it_op & IT_CREAT)
                return LCK_CW;
-       else if (it->it_op & (IT_GETATTR | IT_OPEN | IT_LOOKUP |
-                             IT_LAYOUT))
+       else if (it->it_op & (IT_GETATTR | IT_OPEN | IT_LOOKUP))
                return LCK_CR;
+       else if (it->it_op & IT_LAYOUT)
+               return (it->it_flags & FMODE_WRITE) ? LCK_EX : LCK_CR;
        else if (it->it_op &  IT_READDIR)
                return LCK_PR;
        else if (it->it_op &  IT_GETXATTR)
                return LCK_PR;
-       else if (it->it_op &  IT_SETXATTR)
-               return LCK_PW;
 
        LASSERTF(0, "Invalid it_op: %d\n", it->it_op);
        return -EINVAL;
@@ -788,6 +824,15 @@ enum md_cli_flags {
        CLI_HASH64      = 1 << 2,
        CLI_API32       = 1 << 3,
        CLI_MIGRATE     = 1 << 4,
+       CLI_DIRTY_DATA  = 1 << 5,
+};
+
+enum md_op_code {
+       LUSTRE_OPC_MKDIR        = 0,
+       LUSTRE_OPC_SYMLINK      = 1,
+       LUSTRE_OPC_MKNOD        = 2,
+       LUSTRE_OPC_CREATE       = 3,
+       LUSTRE_OPC_ANY          = 5,
 };
 
 /**
@@ -797,43 +842,45 @@ enum md_cli_flags {
  */
 static inline bool it_has_reply_body(const struct lookup_intent *it)
 {
-       return it->it_op & (IT_OPEN | IT_UNLINK | IT_LOOKUP | IT_GETATTR);
+       return it->it_op & (IT_OPEN | IT_LOOKUP | IT_GETATTR);
 }
 
 struct md_op_data {
-        struct lu_fid           op_fid1; /* operation fid1 (usualy parent) */
-        struct lu_fid           op_fid2; /* operation fid2 (usualy child) */
-        struct lu_fid           op_fid3; /* 2 extra fids to find conflicting */
-        struct lu_fid           op_fid4; /* to the operation locks. */
+       struct lu_fid           op_fid1; /* operation fid1 (usualy parent) */
+       struct lu_fid           op_fid2; /* operation fid2 (usualy child) */
+       struct lu_fid           op_fid3; /* 2 extra fids to find conflicting */
+       struct lu_fid           op_fid4; /* to the operation locks. */
        u32                     op_mds;  /* what mds server open will go to */
-       struct lustre_handle    op_handle;
+       __u32                   op_mode;
+       enum md_op_code         op_code;
+       struct lustre_handle    op_open_handle;
        s64                     op_mod_time;
-        const char             *op_name;
+       const char              *op_name;
        size_t                  op_namelen;
-        __u32                   op_mode;
-        struct lmv_stripe_md   *op_mea1;
-        struct lmv_stripe_md   *op_mea2;
-        __u32                   op_suppgids[2];
-        __u32                   op_fsuid;
-        __u32                   op_fsgid;
-        cfs_cap_t               op_cap;
-        void                   *op_data;
+       struct rw_semaphore     *op_mea1_sem;
+       struct rw_semaphore     *op_mea2_sem;
+       struct lmv_stripe_md    *op_mea1;
+       struct lmv_stripe_md    *op_mea2;
+       struct lmv_stripe_md    *op_default_mea1;       /* default LMV */
+       __u32                   op_suppgids[2];
+       __u32                   op_fsuid;
+       __u32                   op_fsgid;
+       cfs_cap_t               op_cap;
+       void                    *op_data;
        size_t                  op_data_size;
 
-        /* iattr fields and blocks. */
+       /* iattr fields and blocks. */
        struct iattr            op_attr;
+       enum op_xvalid          op_xvalid;      /* eXtra validity flags */
        loff_t                  op_attr_blocks;
-       unsigned int            op_attr_flags; /* LUSTRE_{SYNC,..}_FL */
-       __u64                   op_valid; /* OBD_MD_* */
+       u64                     op_valid;       /* OBD_MD_* */
+       unsigned int            op_attr_flags;  /* LUSTRE_{SYNC,..}_FL */
 
        enum md_op_flags        op_flags;
 
        /* Various operation flags. */
        enum mds_op_bias        op_bias;
 
-       /* Used by readdir */
-       unsigned int            op_max_pages;
-
        /* used to transfer info between the stacks of MD client
         * see enum op_cli_flags */
        enum md_cli_flags       op_cli_flags;
@@ -842,13 +889,32 @@ struct md_op_data {
        __u64                   op_data_version;
        struct lustre_handle    op_lease_handle;
 
-       /* File security context, for creates. */
+       /* File security context, for creates/metadata ops */
        const char             *op_file_secctx_name;
+       __u32                   op_file_secctx_name_size;
        void                   *op_file_secctx;
        __u32                   op_file_secctx_size;
 
-       /* default stripe offset */
-       __u32                   op_default_stripe_offset;
+       __u32                   op_projid;
+
+       /* Used by readdir */
+       unsigned int            op_max_pages;
+
+       __u16                   op_mirror_id;
+
+       /*
+        * used to access migrating dir: if it's set, assume migration is
+        * finished, use the new layout to access dir, otherwise use old layout.
+        * By default it's not set, because new files are created under new
+        * layout, if we can't find file with name under both old and new
+        * layout, we are sure file with name doesn't exist, but in reverse
+        * order there may be a race with creation by others.
+        */
+       bool                    op_post_migrate;
+       /* used to access dir with bash hash */
+       __u32                   op_stripe_index;
+       /* Archive ID for PCC attach */
+       __u32                   op_archive_id;
 };
 
 struct md_callback {
@@ -920,9 +986,9 @@ struct obd_ops {
         * about this.
         */
        int (*o_statfs)(const struct lu_env *, struct obd_export *exp,
-                       struct obd_statfs *osfs, __u64 max_age, __u32 flags);
+                       struct obd_statfs *osfs, time64_t max_age, __u32 flags);
        int (*o_statfs_async)(struct obd_export *exp, struct obd_info *oinfo,
-                             __u64 max_age, struct ptlrpc_request_set *set);
+                             time64_t max_age, struct ptlrpc_request_set *set);
        int (*o_create)(const struct lu_env *env, struct obd_export *exp,
                        struct obdo *oa);
        int (*o_destroy)(const struct lu_env *env, struct obd_export *exp,
@@ -947,7 +1013,7 @@ struct obd_ops {
                              enum obd_import_event);
 
        int (*o_notify)(struct obd_device *obd, struct obd_device *watched,
-                       enum obd_notify_event ev, void *data);
+                       enum obd_notify_event ev);
 
        int (*o_health_check)(const struct lu_env *env, struct obd_device *);
        struct obd_uuid *(*o_get_uuid) (struct obd_export *exp);
@@ -956,8 +1022,6 @@ struct obd_ops {
        int (*o_quotactl)(struct obd_device *, struct obd_export *,
                          struct obd_quotactl *);
 
-       int (*o_ping)(const struct lu_env *, struct obd_export *exp);
-
        /* pools methods */
        int (*o_pool_new)(struct obd_device *obd, char *poolname);
        int (*o_pool_del)(struct obd_device *obd, char *poolname);
@@ -965,20 +1029,18 @@ struct obd_ops {
                          char *ostname);
        int (*o_pool_rem)(struct obd_device *obd, char *poolname,
                          char *ostname);
-       void (*o_getref)(struct obd_device *obd);
-       void (*o_putref)(struct obd_device *obd);
-       /*
-        * NOTE: If adding ops, add another LPROCFS_OBD_OP_INIT() line
-        * to lprocfs_alloc_obd_stats() in obdclass/lprocfs_status.c.
-        * Also, add a wrapper function in include/linux/obd_class.h. */
 };
 
 /* lmv structures */
 struct lustre_md {
        struct mdt_body         *body;
        struct lu_buf            layout;
-       struct lmv_stripe_md    *lmv;
-#ifdef CONFIG_FS_POSIX_ACL
+       union {
+               struct lmv_stripe_md    *lmv;
+               struct lmv_foreign_md   *lfm;
+       };
+       struct lmv_stripe_md    *default_lmv;
+#ifdef CONFIG_LUSTRE_FS_POSIX_ACL
        struct posix_acl        *posix_acl;
 #endif
 };
@@ -992,7 +1054,7 @@ struct md_open_data {
 };
 
 struct obd_client_handle {
-       struct lustre_handle     och_fh;
+       struct lustre_handle     och_open_handle;
        struct lu_fid            och_fid;
        struct md_open_data     *och_mod;
        struct lustre_handle     och_lease_handle; /* open lock for lease */
@@ -1006,18 +1068,6 @@ struct lookup_intent;
 struct cl_attr;
 
 struct md_ops {
-       /* Every operation from MD_STATS_FIRST_OP up to and including
-        * MD_STATS_LAST_OP will be counted by EXP_MD_OP_INCREMENT()
-        * and will appear in /proc/fs/lustre/{lmv,mdc}/.../md_stats.
-        * Operations after MD_STATS_LAST_OP are excluded from stats.
-        * There are a few reasons for doing this: we prune the 17
-        * counters which will be of minimal use in understanding
-        * metadata utilization, we save memory by allocating 15
-        * instead of 32 counters, we save cycles by not counting.
-        *
-        * MD_STATS_FIRST_OP must be the first member of md_ops.
-        */
-#define MD_STATS_FIRST_OP m_close
        int (*m_close)(struct obd_export *, struct md_op_data *,
                       struct md_open_data *, struct ptlrpc_request **);
 
@@ -1058,12 +1108,11 @@ struct md_ops {
                        struct ptlrpc_request **);
 
        int (*m_setxattr)(struct obd_export *, const struct lu_fid *,
-                         u64, const char *, const char *, int, int, int, u32,
-                         struct ptlrpc_request **);
+                         u64, const char *, const void *, size_t, unsigned int,
+                         u32, struct ptlrpc_request **);
 
        int (*m_getxattr)(struct obd_export *, const struct lu_fid *,
-                         u64, const char *, const char *, int, int, int,
-                         struct ptlrpc_request **);
+                         u64, const char *, size_t, struct ptlrpc_request **);
 
        int (*m_intent_getattr_async)(struct obd_export *,
                                      struct md_enqueue_info *);
@@ -1071,7 +1120,7 @@ struct md_ops {
         int (*m_revalidate_lock)(struct obd_export *, struct lookup_intent *,
                                  struct lu_fid *, __u64 *bits);
 
-#define MD_STATS_LAST_OP m_revalidate_lock
+       int (*m_file_resync)(struct obd_export *, struct md_op_data *);
 
        int (*m_get_root)(struct obd_export *, const char *, struct lu_fid *);
        int (*m_null_inode)(struct obd_export *, const struct lu_fid *);
@@ -1116,6 +1165,8 @@ struct md_ops {
                                  struct lu_fid *fid);
        int (*m_unpackmd)(struct obd_export *exp, struct lmv_stripe_md **plsm,
                          const union lmv_mds_md *lmv, size_t lmv_size);
+       int (*m_rmfid)(struct obd_export *exp, struct fid_array *fa, int *rcs,
+                      struct ptlrpc_request_set *set);
 };
 
 static inline struct md_open_data *obd_mod_alloc(void)
@@ -1140,6 +1191,7 @@ static inline struct md_open_data *obd_mod_alloc(void)
 
 void obdo_from_inode(struct obdo *dst, struct inode *src, u64 valid);
 void obdo_set_parent_fid(struct obdo *dst, const struct lu_fid *parent);
+void obdo_set_o_projid(struct obdo *dst, u32 projid);
 
 /* return 1 if client should be resend request */
 static inline int client_should_resend(int resend, struct client_obd *cli)
@@ -1209,7 +1261,8 @@ static inline int cli_brw_size(struct obd_device *obd)
        return obd->u.cli.cl_max_pages_per_rpc << PAGE_SHIFT;
 }
 
-/* when RPC size or the max RPCs in flight is increased, the max dirty pages
+/*
+ * When RPC size or the max RPCs in flight is increased, the max dirty pages
  * of the client should be increased accordingly to avoid sending fragmented
  * RPCs over the network when the client runs out of the maximum dirty space
  * when so many RPCs are being generated.
@@ -1217,10 +1270,10 @@ static inline int cli_brw_size(struct obd_device *obd)
 static inline void client_adjust_max_dirty(struct client_obd *cli)
 {
         /* initializing */
-       if (cli->cl_dirty_max_pages <= 0)
-               cli->cl_dirty_max_pages = (OSC_MAX_DIRTY_DEFAULT * 1024 * 1024)
-                                                       >> PAGE_SHIFT;
-       else {
+       if (cli->cl_dirty_max_pages <= 0) {
+               cli->cl_dirty_max_pages =
+                       (OSC_MAX_DIRTY_DEFAULT * 1024 * 1024) >> PAGE_SHIFT;
+       else {
                unsigned long dirty_max = cli->cl_max_rpcs_in_flight *
                                          cli->cl_max_pages_per_rpc;
 
@@ -1228,8 +1281,14 @@ static inline void client_adjust_max_dirty(struct client_obd *cli)
                        cli->cl_dirty_max_pages = dirty_max;
        }
 
-       if (cli->cl_dirty_max_pages > totalram_pages / 8)
-               cli->cl_dirty_max_pages = totalram_pages / 8;
+       if (cli->cl_dirty_max_pages > cfs_totalram_pages() / 8)
+               cli->cl_dirty_max_pages = cfs_totalram_pages() / 8;
+
+       /* This value is exported to userspace through the max_dirty_mb
+        * parameter.  So we round up the number of pages to make it a round
+        * number of MBs. */
+       cli->cl_dirty_max_pages = round_up(cli->cl_dirty_max_pages,
+                                          1 << (20 - PAGE_SHIFT));
 }
 
 #endif /* __OBD_H */