Whamcloud - gitweb
LU-80 lov: large stripe count support
[fs/lustre-release.git] / lustre / include / lustre_net.h
index 380418d..c065b86 100644 (file)
  * GPL HEADER END
  */
 /*
- * Copyright  2008 Sun Microsystems, Inc. All rights reserved
+ * Copyright (c) 2007, 2010, Oracle and/or its affiliates. All rights reserved.
  * Use is subject to license terms.
  */
 /*
+ * Copyright (c) 2011 Whamcloud, Inc.
+ */
+/*
  * This file is part of Lustre, http://www.lustre.org/
  * Lustre is a trademark of Sun Microsystems, Inc.
  */
+/** \defgroup PtlRPC Portal RPC and networking module.
+ *
+ * PortalRPC is the layer used by rest of lustre code to achieve network
+ * communications: establish connections with corresponding export and import
+ * states, listen for a service, send and receive RPCs.
+ * PortalRPC also includes base recovery framework: packet resending and
+ * replaying, reconnections, pinger.
+ *
+ * PortalRPC utilizes LNet as its transport layer.
+ *
+ * @{
+ */
+
 
 #ifndef _LUSTRE_NET_H
 #define _LUSTRE_NET_H
 
+/** \defgroup net net
+ *
+ * @{
+ */
+
 #if defined(__linux__)
 #include <linux/lustre_net.h>
 #elif defined(__APPLE__)
 #include <lu_object.h>
 #include <lustre_req_layout.h>
 
+#include <obd_support.h>
+#include <lustre_ver.h>
+
 /* MD flags we _always_ use */
 #define PTLRPC_MD_OPTIONS  0
 
-/* Define maxima for bulk I/O
+/**
+ * Define maxima for bulk I/O
  * CAVEAT EMPTOR, with multinet (i.e. routers forwarding between networks)
  * these limits are system wide and not interface-local. */
 #define PTLRPC_MAX_BRW_BITS     LNET_MTU_BITS
 # endif
 #endif /* __KERNEL__ */
 
-/* Size over which to OBD_VMALLOC() rather than OBD_ALLOC() service request
- * buffers */
-#define SVC_BUF_VMALLOC_THRESHOLD (2 * CFS_PAGE_SIZE)
-
-/* The following constants determine how memory is used to buffer incoming
+/**
+ * The following constants determine how memory is used to buffer incoming
  * service requests.
  *
  * ?_NBUFS              # buffers to allocate when growing the pool
  * Messages larger than ?_MAXREQSIZE are dropped.  Request buffers are
  * considered full when less than ?_MAXREQSIZE is left in them.
  */
-
-#define LDLM_THREADS_AUTO_MIN                                                 \
-        min((int)(num_online_cpus() * num_online_cpus() * 2), 8)
-#define LDLM_THREADS_AUTO_MAX (LDLM_THREADS_AUTO_MIN * 16)
+#define LDLM_THREADS_AUTO_MIN (2)
+#define LDLM_THREADS_AUTO_MAX min_t(unsigned, cfs_num_online_cpus() * \
+                                  cfs_num_online_cpus() * 32, 128)
 #define LDLM_BL_THREADS  LDLM_THREADS_AUTO_MIN
-#define LDLM_NBUFS      (64 * num_online_cpus())
+#define LDLM_NBUFS      (64 * cfs_num_online_cpus())
 #define LDLM_BUFSIZE    (8 * 1024)
 #define LDLM_MAXREQSIZE (5 * 1024)
 #define LDLM_MAXREPSIZE (1024)
 
+/** Absolute limits */
 #define MDT_MIN_THREADS 2UL
+#ifndef MDT_MAX_THREADS
 #define MDT_MAX_THREADS 512UL
-#define MDT_NUM_THREADS max(min_t(unsigned long, MDT_MAX_THREADS, \
-                                  num_physpages >> (25 - CFS_PAGE_SHIFT)), 2UL)
-#define FLD_NUM_THREADS max(min_t(unsigned long, MDT_MAX_THREADS, \
-                                  num_physpages >> (25 - CFS_PAGE_SHIFT)), 2UL)
-#define SEQ_NUM_THREADS max(min_t(unsigned long, MDT_MAX_THREADS, \
-                                  num_physpages >> (25 - CFS_PAGE_SHIFT)), 2UL)
-
-/* Absolute limits */
-#define MDS_THREADS_MIN 2
-#define MDS_THREADS_MAX 512
-#define MDS_THREADS_MIN_READPAGE 2
-#define MDS_NBUFS       (64 * num_online_cpus())
-#define MDS_BUFSIZE     (8 * 1024)
-/* Assume file name length = FNAME_MAX = 256 (true for ext3).
+#endif
+#define MDS_NBUFS       (64 * cfs_num_online_cpus())
+/**
+ * Assume file name length = FNAME_MAX = 256 (true for ext3).
  *        path name length = PATH_MAX = 4096
- *        LOV MD size max  = EA_MAX = 4000
+ *        LOV MD size max  = EA_MAX = 48000 (2000 stripes)
  * symlink:  FNAME_MAX + PATH_MAX  <- largest
  * link:     FNAME_MAX + PATH_MAX  (mds_rec_link < mds_rec_create)
  * rename:   FNAME_MAX + FNAME_MAX
  * open:     FNAME_MAX + EA_MAX
  *
  * MDS_MAXREQSIZE ~= 4736 bytes =
- * lustre_msg + ldlm_request + mds_body + mds_rec_create + FNAME_MAX + PATH_MAX
+ * lustre_msg + ldlm_request + mdt_body + mds_rec_create + FNAME_MAX + PATH_MAX
  * MDS_MAXREPSIZE ~= 8300 bytes = lustre_msg + llog_header
  * or, for mds_close() and mds_reint_unlink() on a many-OST filesystem:
- *      = 9210 bytes = lustre_msg + mds_body + 160 * (easize + cookiesize)
+ *      = 9210 bytes = lustre_msg + mdt_body + 160 * (easize + cookiesize)
  *
  * Realistic size is about 512 bytes (20 character name + 128 char symlink),
  * except in the open case where there are a large number of OSTs in a LOV.
  */
-#define MDS_MAXREQSIZE  (5 * 1024)
-#define MDS_MAXREPSIZE  max(9 * 1024, 280 + LOV_MAX_STRIPE_COUNT * 56)
+#define MDS_MAXREPSIZE  max(10 * 1024, 362 + LOV_MAX_STRIPE_COUNT * 56)
+#define MDS_MAXREQSIZE  MDS_MAXREPSIZE
+
+/** MDS_BUFSIZE = max_reqsize + max sptlrpc payload size */
+#define MDS_BUFSIZE     (MDS_MAXREQSIZE + 1024)
 
-/* FLD_MAXREQSIZE == lustre_msg + __u32 padding + ptlrpc_body + opc + md_fld */
+/** FLD_MAXREQSIZE == lustre_msg + __u32 padding + ptlrpc_body + opc */
 #define FLD_MAXREQSIZE  (160)
 
-/* FLD_MAXREPSIZE == lustre_msg + ptlrpc_body + md_fld */
+/** FLD_MAXREPSIZE == lustre_msg + ptlrpc_body */
 #define FLD_MAXREPSIZE  (152)
 
-/* SEQ_MAXREQSIZE == lustre_msg + __u32 padding + ptlrpc_body + opc + lu_range +
+/**
+ * SEQ_MAXREQSIZE == lustre_msg + __u32 padding + ptlrpc_body + opc + lu_range +
  * __u32 padding */
 #define SEQ_MAXREQSIZE  (160)
 
-/* SEQ_MAXREPSIZE == lustre_msg + ptlrpc_body + lu_range */
+/** SEQ_MAXREPSIZE == lustre_msg + ptlrpc_body + lu_range */
 #define SEQ_MAXREPSIZE  (152)
 
-#define MGS_THREADS_AUTO_MIN 2
+/** MGS threads must be >= 3, see bug 22458 comment #28 */
+#define MGS_THREADS_AUTO_MIN 3
 #define MGS_THREADS_AUTO_MAX 32
-#define MGS_NBUFS       (64 * num_online_cpus())
+#define MGS_NBUFS       (64 * cfs_num_online_cpus())
 #define MGS_BUFSIZE     (8 * 1024)
 #define MGS_MAXREQSIZE  (7 * 1024)
 #define MGS_MAXREPSIZE  (9 * 1024)
 
-/* Absolute limits */
-#define OSS_THREADS_MIN 2
+/** Absolute OSS limits */
+#define OSS_THREADS_MIN 3       /* difficult replies, HPQ, others */
 #define OSS_THREADS_MAX 512
-#define OST_NBUFS       (64 * num_online_cpus())
+#define OST_NBUFS       (64 * cfs_num_online_cpus())
 #define OST_BUFSIZE     (8 * 1024)
-/* OST_MAXREQSIZE ~= 4768 bytes =
+
+/**
+ * OST_MAXREQSIZE ~= 4768 bytes =
  * lustre_msg + obdo + 16 * obd_ioobj + 256 * niobuf_remote
  *
  * - single object with 16 pages is 512 bytes
 /* Macro to hide a typecast. */
 #define ptlrpc_req_async_args(req) ((void *)&req->rq_async_args)
 
+/**
+ * Structure to single define portal connection.
+ */
 struct ptlrpc_connection {
-        struct hlist_node       c_hash;
+        /** linkage for connections hash table */
+        cfs_hlist_node_t        c_hash;
+        /** Our own lnet nid for this connection */
         lnet_nid_t              c_self;
+        /** Remote side nid for this connection */
         lnet_process_id_t       c_peer;
+        /** UUID of the other side */
         struct obd_uuid         c_remote_uuid;
-        atomic_t                c_refcount;
+        /** reference counter for this connection */
+        cfs_atomic_t            c_refcount;
 };
 
+/** Client definition for PortalRPC */
 struct ptlrpc_client {
+        /** What lnet portal does this client send messages to by default */
         __u32                   cli_request_portal;
+        /** What portal do we expect replies on */
         __u32                   cli_reply_portal;
+        /** Name of the client */
         char                   *cli_name;
 };
 
-/* state flags of requests */
+/** state flags of requests */
 /* XXX only ones left are those used by the bulk descs as well! */
 #define PTL_RPC_FL_INTR      (1 << 0)  /* reply wait was interrupted by user */
 #define PTL_RPC_FL_TIMEOUT   (1 << 7)  /* request timed out waiting for reply */
@@ -205,41 +237,77 @@ struct ptlrpc_client {
 #define REQ_MAX_ACK_LOCKS 8
 
 union ptlrpc_async_args {
-        /* Scratchpad for passing args to completion interpreter. Users
-         * cast to the struct of their choosing, and LASSERT that this is
+        /**
+         * Scratchpad for passing args to completion interpreter. Users
+         * cast to the struct of their choosing, and CLASSERT that this is
          * big enough.  For _tons_ of context, OBD_ALLOC a struct and store
          * a pointer to it here.  The pointer_arg ensures this struct is at
-         * least big enough for that. */
-        void      *pointer_arg[9];
-        __u64      space[4];
+         * least big enough for that.
+         */
+        void      *pointer_arg[11];
+        __u64      space[6];
 };
 
 struct ptlrpc_request_set;
 typedef int (*set_interpreter_func)(struct ptlrpc_request_set *, void *, int);
 
+/**
+ * Definition of request set structure.
+ * Request set is a list of requests (not necessary to the same target) that
+ * once populated with RPCs could be sent in parallel.
+ * There are two kinds of request sets. General purpose and with dedicated
+ * serving thread. Example of the latter is ptlrpcd set.
+ * For general purpose sets once request set started sending it is impossible
+ * to add new requests to such set.
+ * Provides a way to call "completion callbacks" when all requests in the set
+ * returned.
+ */
 struct ptlrpc_request_set {
-        int               set_remaining; /* # uncompleted requests */
-        cfs_waitq_t       set_waitq;
-        cfs_waitq_t      *set_wakeup_ptr;
-        struct list_head  set_requests;
-        struct list_head  set_cblist; /* list of completion callbacks */
-        set_interpreter_func    set_interpret; /* completion callback */
-        void              *set_arg; /* completion context */
-        /* locked so that any old caller can communicate requests to
-         * the set holder who can then fold them into the lock-free set */
-        spinlock_t        set_new_req_lock;
-        struct list_head  set_new_requests;
+        cfs_atomic_t          set_refcount;
+        /** number of in queue requests */
+        cfs_atomic_t          set_new_count;
+        /** number of uncompleted requests */
+        cfs_atomic_t          set_remaining;
+        /** wait queue to wait on for request events */
+        cfs_waitq_t           set_waitq;
+        cfs_waitq_t          *set_wakeup_ptr;
+        /** List of requests in the set */
+        cfs_list_t            set_requests;
+        /**
+         * List of completion callbacks to be called when the set is completed
+         * This is only used if \a set_interpret is NULL.
+         * Links struct ptlrpc_set_cbdata.
+         */
+        cfs_list_t            set_cblist;
+        /** Completion callback, if only one. */
+        set_interpreter_func  set_interpret;
+        /** opaq argument passed to completion \a set_interpret callback. */
+        void                 *set_arg;
+        /**
+         * Lock for \a set_new_requests manipulations
+         * locked so that any old caller can communicate requests to
+         * the set holder who can then fold them into the lock-free set
+         */
+        cfs_spinlock_t        set_new_req_lock;
+        /** List of new yet unsent requests. Only used with ptlrpcd now. */
+        cfs_list_t            set_new_requests;
 };
 
+/**
+ * Description of a single ptrlrpc_set callback
+ */
 struct ptlrpc_set_cbdata {
-        struct list_head        psc_item;
+        /** List linkage item */
+        cfs_list_t              psc_item;
+        /** Pointer to interpreting function */
         set_interpreter_func    psc_interpret;
+        /** Opaq argument to pass to the callback */
         void                   *psc_data;
 };
 
 struct ptlrpc_bulk_desc;
 
-/*
+/**
  * ptlrpc callback & work item stuff
  */
 struct ptlrpc_cb_id {
@@ -247,121 +315,253 @@ struct ptlrpc_cb_id {
         void    *cbid_arg;                      /* additional arg */
 };
 
+/** Maximum number of locks to fit into reply state */
 #define RS_MAX_LOCKS 8
-#define RS_DEBUG     1
-
+#define RS_DEBUG     0
+
+/**
+ * Structure to define reply state on the server
+ * Reply state holds various reply message information. Also for "difficult"
+ * replies (rep-ack case) we store the state after sending reply and wait
+ * for the client to acknowledge the reception. In these cases locks could be
+ * added to the state for replay/failover consistency guarantees.
+ */
 struct ptlrpc_reply_state {
+        /** Callback description */
         struct ptlrpc_cb_id    rs_cb_id;
-        struct list_head       rs_list;
-        struct list_head       rs_exp_list;
-        struct list_head       rs_obd_list;
+        /** Linkage for list of all reply states in a system */
+        cfs_list_t             rs_list;
+        /** Linkage for list of all reply states on same export */
+        cfs_list_t             rs_exp_list;
+        /** Linkage for list of all reply states for same obd */
+        cfs_list_t             rs_obd_list;
 #if RS_DEBUG
-        struct list_head       rs_debug_list;
+        cfs_list_t             rs_debug_list;
 #endif
-        /* updates to following flag serialised by srv_request_lock */
+        /** A spinlock to protect the reply state flags */
+        cfs_spinlock_t         rs_lock;
+        /** Reply state flags */
         unsigned long          rs_difficult:1;     /* ACK/commit stuff */
+        unsigned long          rs_no_ack:1;    /* no ACK, even for
+                                                  difficult requests */
         unsigned long          rs_scheduled:1;     /* being handled? */
         unsigned long          rs_scheduled_ever:1;/* any schedule attempts? */
         unsigned long          rs_handled:1;  /* been handled yet? */
         unsigned long          rs_on_net:1;   /* reply_out_callback pending? */
         unsigned long          rs_prealloc:1; /* rs from prealloc list */
-
+        unsigned long          rs_committed:1;/* the transaction was committed
+                                                 and the rs was dispatched
+                                                 by ptlrpc_commit_replies */
+        /** Size of the state */
         int                    rs_size;
+        /** opcode */
+        __u32                  rs_opc;
+        /** Transaction number */
         __u64                  rs_transno;
+        /** xid */
         __u64                  rs_xid;
         struct obd_export     *rs_export;
         struct ptlrpc_service *rs_service;
+        /** Lnet metadata handle for the reply */
         lnet_handle_md_t       rs_md_h;
-        atomic_t               rs_refcount;
+        cfs_atomic_t           rs_refcount;
 
+        /** Context for the sevice thread */
         struct ptlrpc_svc_ctx *rs_svc_ctx;
+        /** Reply buffer (actually sent to the client), encoded if needed */
         struct lustre_msg     *rs_repbuf;       /* wrapper */
+        /** Size of the reply buffer */
         int                    rs_repbuf_len;   /* wrapper buf length */
+        /** Size of the reply message */
         int                    rs_repdata_len;  /* wrapper msg length */
+        /**
+         * Actual reply message. Its content is encrupted (if needed) to
+         * produce reply buffer for actual sending. In simple case
+         * of no network encryption we jus set \a rs_repbuf to \a rs_msg
+         */
         struct lustre_msg     *rs_msg;          /* reply message */
 
-        /* locks awaiting client reply ACK */
+        /** Number of locks awaiting client ACK */
         int                    rs_nlocks;
+        /** Handles of locks awaiting client reply ACK */
         struct lustre_handle   rs_locks[RS_MAX_LOCKS];
+        /** Lock modes of locks in \a rs_locks */
         ldlm_mode_t            rs_modes[RS_MAX_LOCKS];
 };
 
 struct ptlrpc_thread;
 
+/** RPC stages */
 enum rq_phase {
-        RQ_PHASE_NEW         = 0xebc0de00,
-        RQ_PHASE_RPC         = 0xebc0de01,
-        RQ_PHASE_BULK        = 0xebc0de02,
-        RQ_PHASE_INTERPRET   = 0xebc0de03,
-        RQ_PHASE_COMPLETE    = 0xebc0de04,
+        RQ_PHASE_NEW            = 0xebc0de00,
+        RQ_PHASE_RPC            = 0xebc0de01,
+        RQ_PHASE_BULK           = 0xebc0de02,
+        RQ_PHASE_INTERPRET      = 0xebc0de03,
+        RQ_PHASE_COMPLETE       = 0xebc0de04,
+        RQ_PHASE_UNREGISTERING  = 0xebc0de05,
+        RQ_PHASE_UNDEFINED      = 0xebc0de06
 };
 
+/** Type of request interpreter call-back */
+typedef int (*ptlrpc_interpterer_t)(const struct lu_env *env,
+                                    struct ptlrpc_request *req,
+                                    void *arg, int rc);
+
+/**
+ * Definition of request pool structure.
+ * The pool is used to store empty preallocated requests for the case
+ * when we would actually need to send something without performing
+ * any allocations (to avoid e.g. OOM).
+ */
 struct ptlrpc_request_pool {
-        spinlock_t prp_lock;
-        struct list_head prp_req_list;    /* list of ptlrpc_request structs */
+        /** Locks the list */
+        cfs_spinlock_t prp_lock;
+        /** list of ptlrpc_request structs */
+        cfs_list_t prp_req_list;
+        /** Maximum message size that would fit into a rquest from this pool */
         int prp_rq_size;
+        /** Function to allocate more requests for this pool */
         void (*prp_populate)(struct ptlrpc_request_pool *, int);
 };
 
 struct lu_context;
 struct lu_env;
 
+struct ldlm_lock;
+
+/**
+ * Basic request prioritization operations structure.
+ * The whole idea is centered around locks and RPCs that might affect locks.
+ * When a lock is contended we try to give priority to RPCs that might lead
+ * to fastest release of that lock.
+ * Currently only implemented for OSTs only in a way that makes all
+ * IO and truncate RPCs that are coming from a locked region where a lock is
+ * contended a priority over other requests.
+ */
+struct ptlrpc_hpreq_ops {
+        /**
+         * Check if the lock handle of the given lock is the same as
+         * taken from the request.
+         */
+        int  (*hpreq_lock_match)(struct ptlrpc_request *, struct ldlm_lock *);
+        /**
+         * Check if the request is a high priority one.
+         */
+        int  (*hpreq_check)(struct ptlrpc_request *);
+};
+
+/**
+ * Represents remote procedure call.
+ *
+ * This is a staple structure used by everybody wanting to send a request
+ * in Lustre.
+ */
 struct ptlrpc_request {
-        int rq_type; /* one of PTL_RPC_MSG_* */
-        struct list_head rq_list;
-        struct list_head rq_timed_list;         /* server-side early replies */
-        struct list_head rq_history_list;       /* server-side history */
-        __u64            rq_history_seq;        /* history sequence # */
+        /* Request type: one of PTL_RPC_MSG_* */
+        int rq_type;
+        /**
+         * Linkage item through which this request is included into
+         * sending/delayed lists on client and into rqbd list on server
+         */
+        cfs_list_t rq_list;
+        /**
+         * Server side list of incoming unserved requests sorted by arrival
+         * time.  Traversed from time to time to notice about to expire
+         * requests and sent back "early replies" to clients to let them
+         * know server is alive and well, just very busy to service their
+         * requests in time
+         */
+        cfs_list_t rq_timed_list;
+        /** server-side history, used for debuging purposes. */
+        cfs_list_t rq_history_list;
+        /** server-side per-export list */
+        cfs_list_t rq_exp_list;
+        /** server-side hp handlers */
+        struct ptlrpc_hpreq_ops *rq_ops;
+        /** history sequence # */
+        __u64 rq_history_seq;
+        /** the index of service's srv_at_array into which request is linked */
+        time_t rq_at_index;
+        /** Result of request processing */
         int rq_status;
-        spinlock_t rq_lock;
-        /* client-side flags are serialized by rq_lock */
+        /** Lock to protect request flags and some other important bits, like
+         * rq_list
+         */
+        cfs_spinlock_t rq_lock;
+        /** client-side flags are serialized by rq_lock */
         unsigned long rq_intr:1, rq_replied:1, rq_err:1,
                 rq_timedout:1, rq_resend:1, rq_restart:1,
-                /*
+                /**
                  * when ->rq_replay is set, request is kept by the client even
                  * after server commits corresponding transaction. This is
                  * used for operations that require sequence of multiple
                  * requests to be replayed. The only example currently is file
-                 * open/close/dw/setattr. When last request in such a sequence
-                 * is committed, ->rq_replay is cleared on all requests in the
+                 * open/close. When last request in such a sequence is
+                 * committed, ->rq_replay is cleared on all requests in the
                  * sequence.
                  */
                 rq_replay:1,
-                /* this is the last request in the sequence. */
-                rq_sequence:1,
                 rq_no_resend:1, rq_waiting:1, rq_receiving_reply:1,
                 rq_no_delay:1, rq_net_err:1, rq_wait_ctx:1,
                 rq_early:1, rq_must_unlink:1,
+                rq_fake:1,          /* this fake req */
+                rq_memalloc:1,      /* req originated from "kswapd" */
                 /* server-side flags */
                 rq_packed_final:1,  /* packed final reply */
-                rq_sent_final:1;    /* stop sending early replies */
+                rq_hp:1,            /* high priority RPC */
+                rq_at_linked:1,     /* link into service's srv_at_array */
+                rq_reply_truncate:1,
+                rq_committed:1,
+                /* whether the "rq_set" is a valid one */
+                rq_invalid_rqset:1;
 
         enum rq_phase rq_phase; /* one of RQ_PHASE_* */
-        atomic_t rq_refcount;   /* client-side refcount for SENT race,
-                                   server-side refcounf for multiple replies */
+        enum rq_phase rq_next_phase; /* one of RQ_PHASE_* to be used next */
+        cfs_atomic_t rq_refcount;/* client-side refcount for SENT race,
+                                    server-side refcounf for multiple replies */
 
-        struct ptlrpc_thread *rq_svc_thread; /* initial thread servicing req */
+        /** initial thread servicing this request */
+        struct ptlrpc_thread *rq_svc_thread;
 
+        /** Portal to which this request would be sent */
         int rq_request_portal;  /* XXX FIXME bug 249 */
+        /** Portal where to wait for reply and where reply would be sent */
         int rq_reply_portal;    /* XXX FIXME bug 249 */
 
-        int rq_nob_received; /* client-side # reply bytes actually received  */
-
+        /**
+         * client-side:
+         * !rq_truncate : # reply bytes actually received,
+         *  rq_truncate : required repbuf_len for resend
+         */
+        int rq_nob_received;
+        /** Request length */
         int rq_reqlen;
+         /** Request message - what client sent */
         struct lustre_msg *rq_reqmsg;
 
+        /** Reply length */
         int rq_replen;
+        /** Reply message - server response */
         struct lustre_msg *rq_repmsg;
+        /** Transaction number */
         __u64 rq_transno;
+        /** xid */
         __u64 rq_xid;
-        struct list_head rq_replay_list;
-        struct list_head rq_mod_list;
+        /**
+         * List item to for replay list. Not yet commited requests get linked
+         * there.
+         * Also see \a rq_replay comment above.
+         */
+        cfs_list_t rq_replay_list;
 
-        struct ptlrpc_cli_ctx   *rq_cli_ctx;     /* client's half ctx */
-        struct ptlrpc_svc_ctx   *rq_svc_ctx;     /* server's half ctx */
-        struct list_head         rq_ctx_chain;   /* link to waited ctx */
+        /**
+         * security and encryption data
+         * @{ */
+        struct ptlrpc_cli_ctx   *rq_cli_ctx;     /**< client's half ctx */
+        struct ptlrpc_svc_ctx   *rq_svc_ctx;     /**< server's half ctx */
+        cfs_list_t               rq_ctx_chain;   /**< link to waited ctx */
 
-        struct sptlrpc_flavor    rq_flvr;        /* client & server */
+        struct sptlrpc_flavor    rq_flvr;        /**< for client & server */
         enum lustre_sec_part     rq_sp_from;
 
         unsigned long            /* client/server security flags */
@@ -374,11 +574,13 @@ struct ptlrpc_request {
                                  rq_auth_remote:1,   /* authed as remote user */
                                  rq_auth_usr_root:1, /* authed as root */
                                  rq_auth_usr_mdt:1,  /* authed as mdt */
+                                 rq_auth_usr_ost:1,  /* authed as ost */
                                  /* security tfm flags */
                                  rq_pack_udesc:1,
                                  rq_pack_bulk:1,
                                  /* doesn't expect reply FIXME */
-                                 rq_no_reply:1;
+                                 rq_no_reply:1,
+                                 rq_pill_init:1;     /* pill initialized */
 
         uid_t                    rq_auth_uid;        /* authed uid */
         uid_t                    rq_auth_mapped_uid; /* authed uid mapped to */
@@ -386,7 +588,7 @@ struct ptlrpc_request {
         /* (server side), pointed directly into req buffer */
         struct ptlrpc_user_desc *rq_user_desc;
 
-        /* early replies go to offset 0, regular replies go after that */
+        /** early replies go to offset 0, regular replies go after that */
         unsigned int             rq_reply_off;
 
         /* various buffer pointers */
@@ -401,102 +603,186 @@ struct ptlrpc_request {
         int                      rq_clrbuf_len;  /* only in priv mode */
         int                      rq_clrdata_len; /* only in priv mode */
 
+        /** @} */
+
+        /** Fields that help to see if request and reply were swabbed or not */
         __u32 rq_req_swab_mask;
         __u32 rq_rep_swab_mask;
 
+        /** What was import generation when this request was sent */
         int rq_import_generation;
         enum lustre_imp_state rq_send_state;
 
-        int rq_early_count;           /* how many early replies (for stats) */
+        /** how many early replies (for stats) */
+        int rq_early_count;
 
-        /* client+server request */
+        /** client+server request */
         lnet_handle_md_t     rq_req_md_h;
         struct ptlrpc_cb_id  rq_req_cbid;
+        /** optional time limit for send attempts */
+        cfs_duration_t       rq_delay_limit;
+        /** time request was first queued */
+        cfs_time_t           rq_queued_time;
 
         /* server-side... */
-        struct timeval       rq_arrival_time;       /* request arrival time */
-        struct ptlrpc_reply_state *rq_reply_state;  /* separated reply state */
-        struct ptlrpc_request_buffer_desc *rq_rqbd; /* incoming request buffer*/
+        /** request arrival time */
+        struct timeval       rq_arrival_time;
+        /** separated reply state */
+        struct ptlrpc_reply_state *rq_reply_state;
+        /** incoming request buffer */
+        struct ptlrpc_request_buffer_desc *rq_rqbd;
 #ifdef CRAY_XT3
         __u32                rq_uid;            /* peer uid, used in MDS only */
 #endif
 
-        /* client-only incoming reply */
+        /** client-only incoming reply */
         lnet_handle_md_t     rq_reply_md_h;
         cfs_waitq_t          rq_reply_waitq;
         struct ptlrpc_cb_id  rq_reply_cbid;
 
+        /** our LNet NID */
         lnet_nid_t           rq_self;
+        /** Peer description (the other side) */
         lnet_process_id_t    rq_peer;
+        /** Server-side, export on which request was received */
         struct obd_export   *rq_export;
+        /** Client side, import where request is being sent */
         struct obd_import   *rq_import;
 
+        /** Replay callback, called after request is replayed at recovery */
         void (*rq_replay_cb)(struct ptlrpc_request *);
+        /**
+         * Commit callback, called when request is committed and about to be
+         * freed.
+         */
         void (*rq_commit_cb)(struct ptlrpc_request *);
+        /** Opaq data for replay and commit callbacks. */
         void  *rq_cb_data;
 
-        struct ptlrpc_bulk_desc *rq_bulk;/* client side bulk */
+        /** For bulk requests on client only: bulk descriptor */
+        struct ptlrpc_bulk_desc *rq_bulk;
 
-        /* client outgoing req */
-        time_t rq_sent;                  /* when request/reply sent (secs), or
-                                          * time when request should be sent */
+        /** client outgoing req */
+        /**
+         * when request/reply sent (secs), or time when request should be sent
+         */
+        time_t rq_sent;
+        /** time for request really sent out */
+        time_t rq_real_sent;
+
+        /** when request must finish. volatile
+         * so that servers' early reply updates to the deadline aren't
+         * kept in per-cpu cache */
+        volatile time_t rq_deadline;
+        /** when req reply unlink must finish. */
+        time_t rq_reply_deadline;
+        /** when req bulk unlink must finish. */
+        time_t rq_bulk_deadline;
+        /**
+         * service time estimate (secs) 
+         * If the requestsis not served by this time, it is marked as timed out.
+         */
+        int    rq_timeout;
+
+        /** Multi-rpc bits */
+        /** Link item for request set lists */
+        cfs_list_t  rq_set_chain;
+        /** Per-request waitq introduced by bug 21938 for recovery waiting */
+        cfs_waitq_t rq_set_waitq;
+        /** Link back to the request set */
+        struct ptlrpc_request_set *rq_set;
+        /** Async completion handler, called when reply is received */
+        ptlrpc_interpterer_t rq_interpret_reply;
+        /** Async completion context */
+        union ptlrpc_async_args rq_async_args;
 
-        volatile time_t rq_deadline;     /* when request must finish. volatile
-               so that servers' early reply updates to the deadline aren't
-               kept in per-cpu cache */
-        int    rq_timeout;               /* service time estimate (secs) */
+        /** Pool if request is from preallocated list */
+        struct ptlrpc_request_pool *rq_pool;
 
-        /* Multi-rpc bits */
-        struct list_head rq_set_chain;
-        struct ptlrpc_request_set *rq_set;
-        void *rq_interpret_reply;               /* Async completion handler */
-        union ptlrpc_async_args rq_async_args;  /* Async completion context */
-        struct ptlrpc_request_pool *rq_pool;    /* Pool if request from
-                                                   preallocated list */
         struct lu_context           rq_session;
+        struct lu_context           rq_recov_session;
 
-        /* request format */
+        /** request format description */
         struct req_capsule          rq_pill;
 };
 
-static inline void ptlrpc_close_replay_seq(struct ptlrpc_request *req)
+/**
+ * Call completion handler for rpc if any, return it's status or original
+ * rc if there was no handler defined for this request.
+ */
+static inline int ptlrpc_req_interpret(const struct lu_env *env,
+                                       struct ptlrpc_request *req, int rc)
 {
-        spin_lock(&req->rq_lock);
-        req->rq_replay = 0;
-        req->rq_sequence = 1;
-        spin_unlock(&req->rq_lock);
+        if (req->rq_interpret_reply != NULL) {
+                req->rq_status = req->rq_interpret_reply(env, req,
+                                                         &req->rq_async_args,
+                                                         rc);
+                return req->rq_status;
+        }
+        return rc;
 }
 
-static inline void lustre_set_req_swabbed(struct ptlrpc_request *req, int index)
+/**
+ * Returns 1 if request buffer at offset \a index was already swabbed
+ */
+static inline int lustre_req_swabbed(struct ptlrpc_request *req, int index)
 {
         LASSERT(index < sizeof(req->rq_req_swab_mask) * 8);
-        LASSERT((req->rq_req_swab_mask & (1 << index)) == 0);
-        req->rq_req_swab_mask |= 1 << index;
+        return req->rq_req_swab_mask & (1 << index);
 }
 
-static inline void lustre_set_rep_swabbed(struct ptlrpc_request *req, int index)
+/**
+ * Returns 1 if request reply buffer at offset \a index was already swabbed
+ */
+static inline int lustre_rep_swabbed(struct ptlrpc_request *req, int index)
 {
         LASSERT(index < sizeof(req->rq_rep_swab_mask) * 8);
-        LASSERT((req->rq_rep_swab_mask & (1 << index)) == 0);
-        req->rq_rep_swab_mask |= 1 << index;
+        return req->rq_rep_swab_mask & (1 << index);
 }
 
-static inline int lustre_req_swabbed(struct ptlrpc_request *req, int index)
+/**
+ * Returns 1 if request needs to be swabbed into local cpu byteorder
+ */
+static inline int ptlrpc_req_need_swab(struct ptlrpc_request *req)
+{
+        return lustre_req_swabbed(req, MSG_PTLRPC_HEADER_OFF);
+}
+
+/**
+ * Returns 1 if request reply needs to be swabbed into local cpu byteorder
+ */
+static inline int ptlrpc_rep_need_swab(struct ptlrpc_request *req)
+{
+        return lustre_rep_swabbed(req, MSG_PTLRPC_HEADER_OFF);
+}
+
+/**
+ * Mark request buffer at offset \a index that it was already swabbed
+ */
+static inline void lustre_set_req_swabbed(struct ptlrpc_request *req, int index)
 {
         LASSERT(index < sizeof(req->rq_req_swab_mask) * 8);
-        return req->rq_req_swab_mask & (1 << index);
+        LASSERT((req->rq_req_swab_mask & (1 << index)) == 0);
+        req->rq_req_swab_mask |= 1 << index;
 }
 
-static inline int lustre_rep_swabbed(struct ptlrpc_request *req, int index)
+/**
+ * Mark request reply buffer at offset \a index that it was already swabbed
+ */
+static inline void lustre_set_rep_swabbed(struct ptlrpc_request *req, int index)
 {
         LASSERT(index < sizeof(req->rq_rep_swab_mask) * 8);
-        return req->rq_rep_swab_mask & (1 << index);
+        LASSERT((req->rq_rep_swab_mask & (1 << index)) == 0);
+        req->rq_rep_swab_mask |= 1 << index;
 }
 
+/**
+ * Convert numerical request phase value \a phase into text string description
+ */
 static inline const char *
-ptlrpc_rqphase2str(const struct ptlrpc_request *req)
+ptlrpc_phase2str(enum rq_phase phase)
 {
-        switch (req->rq_phase) {
+        switch (phase) {
         case RQ_PHASE_NEW:
                 return "New";
         case RQ_PHASE_RPC:
@@ -507,14 +793,31 @@ ptlrpc_rqphase2str(const struct ptlrpc_request *req)
                 return "Interpret";
         case RQ_PHASE_COMPLETE:
                 return "Complete";
+        case RQ_PHASE_UNREGISTERING:
+                return "Unregistering";
         default:
                 return "?Phase?";
         }
 }
 
+/**
+ * Convert numerical request phase of the request \a req into text stringi
+ * description
+ */
+static inline const char *
+ptlrpc_rqphase2str(struct ptlrpc_request *req)
+{
+        return ptlrpc_phase2str(req->rq_phase);
+}
+
+/**
+ * Debugging functions and helpers to print request structure into debug log
+ * @{
+ */ 
 /* Spare the preprocessor, spoil the bugs. */
 #define FLAG(field, str) (field ? str : "")
 
+/** Convert bit flags into a string */
 #define DEBUG_REQ_FLAGS(req)                                                    \
         ptlrpc_rqphase2str(req),                                                \
         FLAG(req->rq_intr, "I"), FLAG(req->rq_replied, "R"),                    \
@@ -523,17 +826,22 @@ ptlrpc_rqphase2str(const struct ptlrpc_request *req)
         FLAG(req->rq_restart, "T"), FLAG(req->rq_replay, "P"),                  \
         FLAG(req->rq_no_resend, "N"),                                           \
         FLAG(req->rq_waiting, "W"),                                             \
-        FLAG(req->rq_wait_ctx, "C")
+        FLAG(req->rq_wait_ctx, "C"), FLAG(req->rq_hp, "H"),                     \
+        FLAG(req->rq_committed, "M")
 
-#define REQ_FLAGS_FMT "%s:%s%s%s%s%s%s%s%s%s%s"
+#define REQ_FLAGS_FMT "%s:%s%s%s%s%s%s%s%s%s%s%s%s"
 
 void _debug_req(struct ptlrpc_request *req, __u32 mask,
                 struct libcfs_debug_msg_data *data, const char *fmt, ...)
         __attribute__ ((format (printf, 4, 5)));
 
+/**
+ * Helper that decides if we need to print request accordig to current debug
+ * level settings
+ */
 #define debug_req(cdls, level, req, file, func, line, fmt, a...)              \
 do {                                                                          \
-        CHECK_STACK();                                                        \
+        CFS_CHECK_STACK();                                                    \
                                                                               \
         if (((level) & D_CANTMASK) != 0 ||                                    \
             ((libcfs_debug & (level)) != 0 &&                                 \
@@ -544,7 +852,10 @@ do {                                                                          \
         }                                                                     \
 } while(0)
 
-/* for most callers (level is a constant) this is resolved at compile time */
+/**
+ * This is the debug print function you need to use to print request sturucture
+ * content into lustre debug log.
+ * for most callers (level is a constant) this is resolved at compile time */
 #define DEBUG_REQ(level, req, fmt, args...)                                   \
 do {                                                                          \
         if ((level) & (D_ERROR | D_WARNING)) {                                \
@@ -555,11 +866,21 @@ do {                                                                          \
                 debug_req(NULL, level, req, __FILE__, __func__, __LINE__,     \
                           "@@@ "fmt" ", ## args);                             \
 } while (0)
+/** @} */
 
+/**
+ * Structure that defines a single page of a bulk transfer
+ */
 struct ptlrpc_bulk_page {
-        struct list_head bp_link;
+        /** Linkage to list of pages in a bulk */
+        cfs_list_t       bp_link;
+        /**
+         * Number of bytes in a page to transfer starting from \a bp_pageoffset
+         */
         int              bp_buflen;
-        int              bp_pageoffset; /* offset within a page */
+        /** offset within a page */
+        int              bp_pageoffset;
+        /** The page itself */
         struct page     *bp_page;
 };
 
@@ -568,17 +889,37 @@ struct ptlrpc_bulk_page {
 #define BULK_GET_SINK     2
 #define BULK_PUT_SOURCE   3
 
+/**
+ * Definition of buk descriptor.
+ * Bulks are special "Two phase" RPCs where initial request message
+ * is sent first and it is followed bt a transfer (o receiving) of a large
+ * amount of data to be settled into pages referenced from the bulk descriptors.
+ * Bulks transfers (the actual data following the small requests) are done
+ * on separate LNet portals.
+ * In lustre we use bulk transfers for READ and WRITE transfers from/to OSTs.
+ *  Another user is readpage for MDT.
+ */
 struct ptlrpc_bulk_desc {
-        unsigned long bd_success:1;              /* completed successfully */
-        unsigned long bd_network_rw:1;           /* accessible to the network */
-        unsigned long bd_type:2;                 /* {put,get}{source,sink} */
-        unsigned long bd_registered:1;           /* client side */
-        spinlock_t   bd_lock;                   /* serialise with callback */
+        /** completed successfully */
+        unsigned long bd_success:1;
+        /** accessible to the network (network io potentially in progress) */
+        unsigned long bd_network_rw:1;
+        /** {put,get}{source,sink} */
+        unsigned long bd_type:2;
+        /** client side */
+        unsigned long bd_registered:1;
+        /** For serialization with callback */
+        cfs_spinlock_t bd_lock;
+        /** Import generation when request for this bulk was sent */
         int bd_import_generation;
+        /** Server side - export this bulk created for */
         struct obd_export *bd_export;
+        /** Client side - import this bulk was sent on */
         struct obd_import *bd_import;
+        /** LNet portal for this bulk */
         __u32 bd_portal;
-        struct ptlrpc_request *bd_req;          /* associated request */
+        /** Back pointer to the request */
+        struct ptlrpc_request *bd_req;
         cfs_waitq_t            bd_waitq;        /* server side only WQ */
         int                    bd_iov_count;    /* # entries in bd_iov */
         int                    bd_max_iov;      /* allocated size of bd_iov */
@@ -591,145 +932,362 @@ struct ptlrpc_bulk_desc {
         lnet_handle_md_t       bd_md_h;         /* associated MD */
         lnet_nid_t             bd_sender;       /* stash event::sender */
 
-        cfs_page_t           **bd_enc_pages;
 #if defined(__KERNEL__)
+        /*
+         * encrypt iov, size is either 0 or bd_iov_count.
+         */
+        lnet_kiov_t           *bd_enc_iov;
+
         lnet_kiov_t            bd_iov[0];
 #else
         lnet_md_iovec_t        bd_iov[0];
 #endif
 };
 
-struct ptlrpc_thread {
-
-        struct list_head t_link; /* active threads in svc->srv_threads */
+enum {
+        SVC_STOPPED     = 1 << 0,
+        SVC_STOPPING    = 1 << 1,
+        SVC_STARTING    = 1 << 2,
+        SVC_RUNNING     = 1 << 3,
+        SVC_EVENT       = 1 << 4,
+        SVC_SIGNAL      = 1 << 5,
+};
 
-        void *t_data;            /* thread-private data (preallocated memory) */
+/**
+ * Definition of server service thread structure
+ */
+struct ptlrpc_thread {
+        /**
+         * List of active threads in svc->srv_threads
+         */
+        cfs_list_t t_link;
+        /**
+         * thread-private data (preallocated memory)
+         */
+        void *t_data;
         __u32 t_flags;
-
-        unsigned int t_id; /* service thread index, from ptlrpc_start_threads */
+        /**
+         * service thread index, from ptlrpc_start_threads
+         */
+        unsigned int t_id;
+        /**
+         * service thread pid
+         */
+        pid_t t_pid; 
+        /**
+         * put watchdog in the structure per thread b=14840
+         */
+        struct lc_watchdog *t_watchdog;
+        /**
+         * the svc this thread belonged to b=18582
+         */
+        struct ptlrpc_service *t_svc;
         cfs_waitq_t t_ctl_waitq;
         struct lu_env *t_env;
 };
 
+static inline int thread_is_stopped(struct ptlrpc_thread *thread)
+{
+        return !!(thread->t_flags & SVC_STOPPED);
+}
+
+static inline int thread_is_stopping(struct ptlrpc_thread *thread)
+{
+        return !!(thread->t_flags & SVC_STOPPING);
+}
+
+static inline int thread_is_starting(struct ptlrpc_thread *thread)
+{
+        return !!(thread->t_flags & SVC_STARTING);
+}
+
+static inline int thread_is_running(struct ptlrpc_thread *thread)
+{
+        return !!(thread->t_flags & SVC_RUNNING);
+}
+
+static inline int thread_is_event(struct ptlrpc_thread *thread)
+{
+        return !!(thread->t_flags & SVC_EVENT);
+}
+
+static inline int thread_is_signal(struct ptlrpc_thread *thread)
+{
+        return !!(thread->t_flags & SVC_SIGNAL);
+}
+
+static inline void thread_clear_flags(struct ptlrpc_thread *thread, __u32 flags)
+{
+        thread->t_flags &= ~flags;
+}
+
+static inline void thread_set_flags(struct ptlrpc_thread *thread, __u32 flags)
+{
+        thread->t_flags = flags;
+}
+
+static inline void thread_add_flags(struct ptlrpc_thread *thread, __u32 flags)
+{
+        thread->t_flags |= flags;
+}
+
+static inline int thread_test_and_clear_flags(struct ptlrpc_thread *thread,
+                                              __u32 flags)
+{
+        if (thread->t_flags & flags) {
+                thread->t_flags &= ~flags;
+                return 1;
+        }
+        return 0;
+}
+
+/**
+ * Request buffer descriptor structure.
+ * This is a structure that contains one posted request buffer for service.
+ * Once data land into a buffer, event callback creates actual request and
+ * notifies wakes one of the service threads to process new incoming request.
+ * More than one request can fit into the buffer.
+ */
 struct ptlrpc_request_buffer_desc {
-        struct list_head       rqbd_list;
-        struct list_head       rqbd_reqs;
+        /** Link item for rqbds on a service */
+        cfs_list_t             rqbd_list;
+        /** History of requests for this buffer */
+        cfs_list_t             rqbd_reqs;
+        /** Back pointer to service for which this buffer is registered */
         struct ptlrpc_service *rqbd_service;
+        /** LNet descriptor */
         lnet_handle_md_t       rqbd_md_h;
         int                    rqbd_refcount;
+        /** The buffer itself */
         char                  *rqbd_buffer;
         struct ptlrpc_cb_id    rqbd_cbid;
+        /**
+         * This "embedded" request structure is only used for the
+         * last request to fit into the buffer
+         */
         struct ptlrpc_request  rqbd_req;
 };
 
-typedef int (*svc_handler_t)(struct ptlrpc_request *req);
-typedef void (*svcreq_printfn_t)(void *, struct ptlrpc_request *);
+typedef int  (*svc_thr_init_t)(struct ptlrpc_thread *thread);
+typedef void (*svc_thr_done_t)(struct ptlrpc_thread *thread);
+typedef int  (*svc_handler_t)(struct ptlrpc_request *req);
+typedef int  (*svc_hpreq_handler_t)(struct ptlrpc_request *);
+typedef void (*svc_req_printfn_t)(void *, struct ptlrpc_request *);
 
-struct ptlrpc_service {
-        struct list_head srv_list;              /* chain thru all services */
-        int              srv_max_req_size;      /* biggest request to receive */
-        int              srv_max_reply_size;    /* biggest reply to send */
-        int              srv_buf_size;          /* size of individual buffers */
-        int              srv_nbuf_per_group;    /* # buffers to allocate in 1 group */
-        int              srv_nbufs;             /* total # req buffer descs allocated */
-        int              srv_threads_min;       /* threads to start at SOW */
-        int              srv_threads_max;       /* thread upper limit */
-        int              srv_threads_started;   /* index of last started thread */
-        int              srv_threads_running;   /* # running threads */
-        int              srv_n_difficult_replies; /* # 'difficult' replies */
-        int              srv_n_active_reqs;     /* # reqs being served */
-        cfs_duration_t   srv_rqbd_timeout;      /* timeout before re-posting reqs, in tick */
-        int              srv_watchdog_factor;   /* soft watchdog timeout mutiplier */
-        unsigned         srv_cpu_affinity:1;    /* bind threads to CPUs */
-        unsigned         srv_at_check:1;        /* check early replies */
-        cfs_time_t       srv_at_checktime;      /* debug */
-
-        __u32            srv_req_portal;
-        __u32            srv_rep_portal;
-
-        /* AT stuff */
-        struct adaptive_timeout srv_at_estimate;/* estimated rpc service time */
-        spinlock_t        srv_at_lock;
-        struct list_head  srv_at_list;          /* reqs waiting for replies */
-        cfs_timer_t       srv_at_timer;         /* early reply timer */
-
-        int               srv_n_queued_reqs;    /* # reqs in either of the queues below */
-        struct list_head  srv_req_in_queue;     /* incoming reqs */
-        struct list_head  srv_request_queue;    /* reqs waiting for service */
-
-        struct list_head  srv_request_history;  /* request history */
-        __u64             srv_request_seq;      /* next request sequence # */
-        __u64             srv_request_max_cull_seq; /* highest seq culled from history */
-        svcreq_printfn_t  srv_request_history_print_fn; /* service-specific print fn */
-
-        struct list_head  srv_idle_rqbds;       /* request buffers to be reposted */
-        struct list_head  srv_active_rqbds;     /* req buffers receiving */
-        struct list_head  srv_history_rqbds;    /* request buffer history */
-        int               srv_nrqbd_receiving;  /* # posted request buffers */
-        int               srv_n_history_rqbds;  /* # request buffers in history */
-        int               srv_max_history_rqbds;/* max # request buffers in history */
-
-        atomic_t          srv_outstanding_replies;
-        struct list_head  srv_active_replies;   /* all the active replies */
-        struct list_head  srv_reply_queue;      /* replies waiting for service */
-
-        cfs_waitq_t       srv_waitq; /* all threads sleep on this. This
-                                      * wait-queue is signalled when new
-                                      * incoming request arrives and when
-                                      * difficult reply has to be handled. */
-
-        struct list_head   srv_threads;         /* service thread list */
-        svc_handler_t      srv_handler;
-
-        char *srv_name; /* only statically allocated strings here; we don't clean them */
-        char *srv_thread_name; /* only statically allocated strings here; we don't clean them */
-
-        spinlock_t               srv_lock;
-
-        cfs_proc_dir_entry_t    *srv_procroot;
-        struct lprocfs_stats    *srv_stats;
-
-        /* List of free reply_states */
-        struct list_head         srv_free_rs_list;
-        /* waitq to run, when adding stuff to srv_free_rs_list */
-        cfs_waitq_t              srv_free_rs_waitq;
+#ifndef __cfs_cacheline_aligned
+/* NB: put it here for reducing patche dependence */
+# define __cfs_cacheline_aligned
+#endif
 
-        /*
-         * Tags for lu_context associated with this thread, see struct
-         * lu_context.
-         */
-        __u32                    srv_ctx_tags;
-        /*
+/**
+ * How many high priority requests to serve before serving one normal
+ * priority request
+ */
+#define PTLRPC_SVC_HP_RATIO 10
+
+/**
+ * Definition of PortalRPC service.
+ * The service is listening on a particular portal (like tcp port)
+ * and perform actions for a specific server like IO service for OST
+ * or general metadata service for MDS.
+ *
+ * ptlrpc service has four locks:
+ * \a srv_lock
+ *    serialize operations on rqbd and requests waiting for preprocess
+ * \a srv_rq_lock
+ *    serialize operations active requests sent to this portal
+ * \a srv_at_lock
+ *    serialize adaptive timeout stuff
+ * \a srv_rs_lock
+ *    serialize operations on RS list (reply states)
+ *
+ * We don't have any use-case to take two or more locks at the same time
+ * for now, so there is no lock order issue.
+ */
+struct ptlrpc_service {
+        /** most often accessed fields */
+        /** chain thru all services */
+        cfs_list_t                      srv_list;
+        /** only statically allocated strings here; we don't clean them */
+        char                           *srv_name;
+        /** only statically allocated strings here; we don't clean them */
+        char                           *srv_thread_name;
+        /** service thread list */
+        cfs_list_t                      srv_threads;
+        /** threads to start at beginning of service */
+        int                             srv_threads_min;
+        /** thread upper limit */
+        int                             srv_threads_max;
+        /** always increasing number */
+        unsigned                        srv_threads_next_id;
+        /** # of starting threads */
+        int                             srv_threads_starting;
+        /** # running threads */
+        int                             srv_threads_running;
+
+        /** service operations, move to ptlrpc_svc_ops_t in the future */
+        /** @{ */
+        /**
          * if non-NULL called during thread creation (ptlrpc_start_thread())
          * to initialize service specific per-thread state.
          */
-        int (*srv_init)(struct ptlrpc_thread *thread);
-        /*
+        svc_thr_init_t                  srv_init;
+        /**
          * if non-NULL called during thread shutdown (ptlrpc_main()) to
          * destruct state created by ->srv_init().
          */
-        void (*srv_done)(struct ptlrpc_thread *thread);
+        svc_thr_done_t                  srv_done;
+        /** Handler function for incoming requests for this service */
+        svc_handler_t                   srv_handler;
+        /** hp request handler */
+        svc_hpreq_handler_t             srv_hpreq_handler;
+        /** service-specific print fn */
+        svc_req_printfn_t               srv_req_printfn;
+        /** @} */
+
+        /** Root of /proc dir tree for this service */
+        cfs_proc_dir_entry_t           *srv_procroot;
+        /** Pointer to statistic data for this service */
+        struct lprocfs_stats           *srv_stats;
+        /** # hp per lp reqs to handle */
+        int                             srv_hpreq_ratio;
+        /** biggest request to receive */
+        int                             srv_max_req_size;
+        /** biggest reply to send */
+        int                             srv_max_reply_size;
+        /** size of individual buffers */
+        int                             srv_buf_size;
+        /** # buffers to allocate in 1 group */
+        int                             srv_nbuf_per_group;
+        /** Local portal on which to receive requests */
+        __u32                           srv_req_portal;
+        /** Portal on the client to send replies to */
+        __u32                           srv_rep_portal;
+        /**
+         * Tags for lu_context associated with this thread, see struct
+         * lu_context.
+         */
+        __u32                           srv_ctx_tags;
+        /** soft watchdog timeout multiplier */
+        int                             srv_watchdog_factor;
+        /** bind threads to CPUs */
+        unsigned                        srv_cpu_affinity:1;
+        /** under unregister_service */
+        unsigned                        srv_is_stopping:1;
+
+        /**
+         * serialize the following fields, used for protecting
+         * rqbd list and incoming requests waiting for preprocess
+         */
+        cfs_spinlock_t                  srv_lock  __cfs_cacheline_aligned;
+        /** incoming reqs */
+        cfs_list_t                      srv_req_in_queue;
+        /** total # req buffer descs allocated */
+        int                             srv_nbufs;
+        /** # posted request buffers */
+        int                             srv_nrqbd_receiving;
+        /** timeout before re-posting reqs, in tick */
+        cfs_duration_t                  srv_rqbd_timeout;
+        /** request buffers to be reposted */
+        cfs_list_t                      srv_idle_rqbds;
+        /** req buffers receiving */
+        cfs_list_t                      srv_active_rqbds;
+        /** request buffer history */
+        cfs_list_t                      srv_history_rqbds;
+        /** # request buffers in history */
+        int                             srv_n_history_rqbds;
+        /** max # request buffers in history */
+        int                             srv_max_history_rqbds;
+        /** request history */
+        cfs_list_t                      srv_request_history;
+        /** next request sequence # */
+        __u64                           srv_request_seq;
+        /** highest seq culled from history */
+        __u64                           srv_request_max_cull_seq;
+        /**
+         * all threads sleep on this. This wait-queue is signalled when new
+         * incoming request arrives and when difficult reply has to be handled.
+         */
+        cfs_waitq_t                     srv_waitq;
+
+        /**
+         * serialize the following fields, used for processing requests
+         * sent to this portal
+         */
+        cfs_spinlock_t                  srv_rq_lock __cfs_cacheline_aligned;
+        /** # reqs in either of the queues below */
+        /** reqs waiting for service */
+        cfs_list_t                      srv_request_queue;
+        /** high priority queue */
+        cfs_list_t                      srv_request_hpq;
+        /** # incoming reqs */
+        int                             srv_n_queued_reqs;
+        /** # reqs being served */
+        int                             srv_n_active_reqs;
+        /** # HPreqs being served */
+        int                             srv_n_active_hpreq;
+        /** # hp requests handled */
+        int                             srv_hpreq_count;
+
+        /** AT stuff */
+        /** @{ */
+        /**
+         * serialize the following fields, used for changes on
+         * adaptive timeout
+         */
+        cfs_spinlock_t                  srv_at_lock __cfs_cacheline_aligned;
+        /** estimated rpc service time */
+        struct adaptive_timeout         srv_at_estimate;
+        /** reqs waiting for replies */
+        struct ptlrpc_at_array          srv_at_array;
+        /** early reply timer */
+        cfs_timer_t                     srv_at_timer;
+        /** check early replies */
+        unsigned                        srv_at_check;
+        /** debug */
+        cfs_time_t                      srv_at_checktime;
+        /** @} */
 
+        /**
+         * serialize the following fields, used for processing
+         * replies for this portal
+         */
+        cfs_spinlock_t                  srv_rs_lock __cfs_cacheline_aligned;
+        /** all the active replies */
+        cfs_list_t                      srv_active_replies;
+#ifndef __KERNEL__
+        /** replies waiting for service */
+        cfs_list_t                      srv_reply_queue;
+#endif
+        /** List of free reply_states */
+        cfs_list_t                      srv_free_rs_list;
+        /** waitq to run, when adding stuff to srv_free_rs_list */
+        cfs_waitq_t                     srv_free_rs_waitq;
+        /** # 'difficult' replies */
+        cfs_atomic_t                    srv_n_difficult_replies;
         //struct ptlrpc_srv_ni srv_interfaces[0];
 };
 
+/**
+ * Declaration of ptlrpcd control structure
+ */
 struct ptlrpcd_ctl {
         /**
-         * Ptlrpc thread control flags (LIOD_START, LIOD_STOP, LIOD_STOP_FORCE)
+         * Ptlrpc thread control flags (LIOD_START, LIOD_STOP, LIOD_FORCE)
          */
         unsigned long               pc_flags;
         /**
          * Thread lock protecting structure fields.
          */
-        spinlock_t                  pc_lock;
+        cfs_spinlock_t              pc_lock;
         /**
          * Start completion.
          */
-        struct completion           pc_starting;
+        cfs_completion_t            pc_starting;
         /**
          * Stop completion.
          */
-        struct completion           pc_finishing;
+        cfs_completion_t            pc_finishing;
         /**
          * Thread requests set.
          */
@@ -738,6 +1296,26 @@ struct ptlrpcd_ctl {
          * Thread name used in cfs_daemonize()
          */
         char                        pc_name[16];
+        /**
+         * Environment for request interpreters to run in.
+         */
+        struct lu_env               pc_env;
+        /**
+         * Index of ptlrpcd thread in the array.
+         */
+        int                         pc_index;
+        /**
+         * Number of the ptlrpcd's partners.
+         */
+        int                         pc_npartners;
+        /**
+         * Pointer to the array of partners' ptlrpcd_ctl structure.
+         */
+        struct ptlrpcd_ctl        **pc_partners;
+        /**
+         * Record the partner index to be processed next.
+         */
+        int                         pc_cursor;
 #ifndef __KERNEL__
         /**
          * Async rpcs flag to make sure that ptlrpcd_check() is called only
@@ -770,22 +1348,37 @@ enum ptlrpcd_ctl_flags {
          */
         LIOD_STOP        = 1 << 1,
         /**
-         * Ptlrpc thread stop force flag. This will cause also
-         * aborting any inflight rpcs handled by thread.
+         * Ptlrpc thread force flag (only stop force so far).
+         * This will cause aborting any inflight rpcs handled
+         * by thread if LIOD_STOP is specified.
+         */
+        LIOD_FORCE       = 1 << 2,
+        /**
+         * This is a recovery ptlrpc thread.
          */
-        LIOD_STOP_FORCE  = 1 << 2
+        LIOD_RECOVERY    = 1 << 3,
+        /**
+         * The ptlrpcd is bound to some CPU core.
+         */
+        LIOD_BIND        = 1 << 4,
 };
 
 /* ptlrpc/events.c */
 extern lnet_handle_eq_t ptlrpc_eq_h;
 extern int ptlrpc_uuid_to_peer(struct obd_uuid *uuid,
                                lnet_process_id_t *peer, lnet_nid_t *self);
+/**
+ * These callbacks are invoked by LNet when something happened to
+ * underlying buffer
+ * @{
+ */
 extern void request_out_callback (lnet_event_t *ev);
 extern void reply_in_callback(lnet_event_t *ev);
 extern void client_bulk_callback (lnet_event_t *ev);
 extern void request_in_callback(lnet_event_t *ev);
 extern void reply_out_callback(lnet_event_t *ev);
 extern void server_bulk_callback (lnet_event_t *ev);
+/** @} */
 
 /* ptlrpc/connection.c */
 struct ptlrpc_connection *ptlrpc_connection_get(lnet_process_id_t peer,
@@ -798,19 +1391,45 @@ void ptlrpc_connection_fini(void);
 extern lnet_pid_t ptl_get_pid(void);
 
 /* ptlrpc/niobuf.c */
+/**
+ * Actual interfacing with LNet to put/get/register/unregister stuff
+ * @{
+ */
 int ptlrpc_start_bulk_transfer(struct ptlrpc_bulk_desc *desc);
 void ptlrpc_abort_bulk(struct ptlrpc_bulk_desc *desc);
 int ptlrpc_register_bulk(struct ptlrpc_request *req);
-void ptlrpc_unregister_bulk (struct ptlrpc_request *req);
+int ptlrpc_unregister_bulk(struct ptlrpc_request *req, int async);
 
-static inline int ptlrpc_bulk_active (struct ptlrpc_bulk_desc *desc)
+static inline int ptlrpc_server_bulk_active(struct ptlrpc_bulk_desc *desc)
 {
-        int           rc;
+        int rc;
+
+        LASSERT(desc != NULL);
 
-        spin_lock(&desc->bd_lock);
+        cfs_spin_lock(&desc->bd_lock);
         rc = desc->bd_network_rw;
-        spin_unlock(&desc->bd_lock);
-        return (rc);
+        cfs_spin_unlock(&desc->bd_lock);
+        return rc;
+}
+
+static inline int ptlrpc_client_bulk_active(struct ptlrpc_request *req)
+{
+        struct ptlrpc_bulk_desc *desc = req->rq_bulk;
+        int                      rc;
+
+        LASSERT(req != NULL);
+
+        if (OBD_FAIL_CHECK(OBD_FAIL_PTLRPC_LONG_BULK_UNLINK) &&
+            req->rq_bulk_deadline > cfs_time_current_sec())
+                return 1;
+
+        if (!desc)
+                return 0;
+
+        cfs_spin_lock(&desc->bd_lock);
+        rc = desc->bd_network_rw;
+        cfs_spin_unlock(&desc->bd_lock);
+        return rc;
 }
 
 #define PTLRPC_REPLY_MAYBE_DIFFICULT 0x01
@@ -823,53 +1442,40 @@ void ptlrpc_resend_req(struct ptlrpc_request *request);
 int ptlrpc_at_get_net_latency(struct ptlrpc_request *req);
 int ptl_send_rpc(struct ptlrpc_request *request, int noreply);
 int ptlrpc_register_rqbd (struct ptlrpc_request_buffer_desc *rqbd);
+/** @} */
 
 /* ptlrpc/client.c */
+/**
+ * Client-side portals API. Everything to send requests, receive replies,
+ * request queues, request management, etc.
+ * @{
+ */
 void ptlrpc_init_client(int req_portal, int rep_portal, char *name,
                         struct ptlrpc_client *);
 void ptlrpc_cleanup_client(struct obd_import *imp);
 struct ptlrpc_connection *ptlrpc_uuid_to_connection(struct obd_uuid *uuid);
 
-static inline int
-ptlrpc_client_recv_or_unlink (struct ptlrpc_request *req)
-{
-        int           rc;
-
-        spin_lock(&req->rq_lock);
-        rc = req->rq_receiving_reply || req->rq_must_unlink;
-        spin_unlock(&req->rq_lock);
-        return (rc);
-}
-
-static inline void
-ptlrpc_wake_client_req (struct ptlrpc_request *req)
-{
-        if (req->rq_set == NULL)
-                cfs_waitq_signal(&req->rq_reply_waitq);
-        else
-                cfs_waitq_signal(&req->rq_set->set_waitq);
-}
-
 int ptlrpc_queue_wait(struct ptlrpc_request *req);
 int ptlrpc_replay_req(struct ptlrpc_request *req);
-void ptlrpc_unregister_reply(struct ptlrpc_request *req);
+int ptlrpc_unregister_reply(struct ptlrpc_request *req, int async);
 void ptlrpc_restart_req(struct ptlrpc_request *req);
 void ptlrpc_abort_inflight(struct obd_import *imp);
+void ptlrpc_cleanup_imp(struct obd_import *imp);
 void ptlrpc_abort_set(struct ptlrpc_request_set *set);
 
 struct ptlrpc_request_set *ptlrpc_prep_set(void);
 int ptlrpc_set_add_cb(struct ptlrpc_request_set *set,
                       set_interpreter_func fn, void *data);
 int ptlrpc_set_next_timeout(struct ptlrpc_request_set *);
-int ptlrpc_check_set(struct ptlrpc_request_set *set);
+int ptlrpc_check_set(const struct lu_env *env, struct ptlrpc_request_set *set);
 int ptlrpc_set_wait(struct ptlrpc_request_set *);
 int ptlrpc_expired_set(void *data);
 void ptlrpc_interrupted_set(void *data);
 void ptlrpc_mark_interrupted(struct ptlrpc_request *req);
 void ptlrpc_set_destroy(struct ptlrpc_request_set *);
 void ptlrpc_set_add_req(struct ptlrpc_request_set *, struct ptlrpc_request *);
-int ptlrpc_set_add_new_req(struct ptlrpcd_ctl *pc,
-                           struct ptlrpc_request *req);
+void ptlrpc_set_add_new_req(struct ptlrpcd_ctl *pc,
+                            struct ptlrpc_request *req);
 
 void ptlrpc_free_rq_pool(struct ptlrpc_request_pool *pool);
 void ptlrpc_add_rqs_to_pool(struct ptlrpc_request_pool *pool, int num_rq);
@@ -893,6 +1499,11 @@ struct ptlrpc_request *ptlrpc_request_alloc_pack(struct obd_import *imp,
 int ptlrpc_request_bufs_pack(struct ptlrpc_request *request,
                              __u32 version, int opcode, char **bufs,
                              struct ptlrpc_cli_ctx *ctx);
+struct ptlrpc_request *ptlrpc_prep_fakereq(struct obd_import *imp,
+                                           unsigned int timeout,
+                                           ptlrpc_interpterer_t interpreter);
+void ptlrpc_fakereq_finished(struct ptlrpc_request *req);
+
 struct ptlrpc_request *ptlrpc_prep_req(struct obd_import *imp, __u32 version,
                                        int opcode, int count, __u32 *lengths,
                                        char **bufs);
@@ -900,7 +1511,6 @@ struct ptlrpc_request *ptlrpc_prep_req_pool(struct obd_import *imp,
                                              __u32 version, int opcode,
                                             int count, __u32 *lengths, char **bufs,
                                             struct ptlrpc_request_pool *pool);
-void ptlrpc_free_req(struct ptlrpc_request *request);
 void ptlrpc_req_finished(struct ptlrpc_request *request);
 void ptlrpc_req_finished_with_imp_lock(struct ptlrpc_request *request);
 struct ptlrpc_request *ptlrpc_request_addref(struct ptlrpc_request *req);
@@ -917,6 +1527,8 @@ __u64 ptlrpc_next_xid(void);
 __u64 ptlrpc_sample_next_xid(void);
 __u64 ptlrpc_req_xid(struct ptlrpc_request *request);
 
+/** @} */
+
 struct ptlrpc_service_conf {
         int psc_nbufs;
         int psc_bufsize;
@@ -931,14 +1543,21 @@ struct ptlrpc_service_conf {
 };
 
 /* ptlrpc/service.c */
+/**
+ * Server-side services API. Register/unregister service, request state
+ * management, service thread management
+ *
+ * @{
+ */
 void ptlrpc_save_lock (struct ptlrpc_request *req,
-                       struct lustre_handle *lock, int mode);
-void ptlrpc_commit_replies (struct obd_device *obd);
+                       struct lustre_handle *lock, int mode, int no_ack);
+void ptlrpc_commit_replies(struct obd_export *exp);
+void ptlrpc_dispatch_difficult_reply (struct ptlrpc_reply_state *rs);
 void ptlrpc_schedule_difficult_reply (struct ptlrpc_reply_state *rs);
 struct ptlrpc_service *ptlrpc_init_svc_conf(struct ptlrpc_service_conf *c,
                                             svc_handler_t h, char *name,
                                             struct proc_dir_entry *proc_entry,
-                                            svcreq_printfn_t prntfn,
+                                            svc_req_printfn_t prntfn,
                                             char *threadname);
 
 struct ptlrpc_service *ptlrpc_init_svc(int nbufs, int bufsize, int max_req_size,
@@ -947,35 +1566,62 @@ struct ptlrpc_service *ptlrpc_init_svc(int nbufs, int bufsize, int max_req_size,
                                        int watchdog_factor,
                                        svc_handler_t, char *name,
                                        cfs_proc_dir_entry_t *proc_entry,
-                                       svcreq_printfn_t,
+                                       svc_req_printfn_t,
                                        int min_threads, int max_threads,
-                                       char *threadname, __u32 ctx_tags);
+                                       char *threadname, __u32 ctx_tags,
+                                       svc_hpreq_handler_t);
 void ptlrpc_stop_all_threads(struct ptlrpc_service *svc);
 
-int ptlrpc_start_threads(struct obd_device *dev, struct ptlrpc_service *svc);
-int ptlrpc_start_thread(struct obd_device *dev, struct ptlrpc_service *svc);
+int ptlrpc_start_threads(struct ptlrpc_service *svc);
+int ptlrpc_start_thread(struct ptlrpc_service *svc);
 int ptlrpc_unregister_service(struct ptlrpc_service *service);
 int liblustre_check_services (void *arg);
 void ptlrpc_daemonize(char *name);
 int ptlrpc_service_health_check(struct ptlrpc_service *);
+void ptlrpc_hpreq_reorder(struct ptlrpc_request *req);
+void ptlrpc_server_drop_request(struct ptlrpc_request *req);
 
+#ifdef __KERNEL__
+int ptlrpc_hr_init(void);
+void ptlrpc_hr_fini(void);
+#else
+# define ptlrpc_hr_init() (0)
+# define ptlrpc_hr_fini() do {} while(0)
+#endif
 
 struct ptlrpc_svc_data {
         char *name;
         struct ptlrpc_service *svc;
         struct ptlrpc_thread *thread;
-        struct obd_device *dev;
 };
+/** @} */
 
 /* ptlrpc/import.c */
-int ptlrpc_connect_import(struct obd_import *imp, char * new_uuid);
+/**
+ * Import API
+ * @{
+ */
+int ptlrpc_connect_import(struct obd_import *imp);
 int ptlrpc_init_import(struct obd_import *imp);
 int ptlrpc_disconnect_import(struct obd_import *imp, int noclose);
 int ptlrpc_import_recovery_state_machine(struct obd_import *imp);
 
 /* ptlrpc/pack_generic.c */
 int ptlrpc_reconnect_import(struct obd_import *imp);
-int lustre_msg_swabbed(struct lustre_msg *msg);
+/** @} */
+
+/**
+ * ptlrpc msg buffer and swab interface 
+ *
+ * @{
+ */
+int ptlrpc_buf_need_swab(struct ptlrpc_request *req, const int inout,
+                         int index);
+void ptlrpc_buf_set_swabbed(struct ptlrpc_request *req, const int inout,
+                                int index);
+int ptlrpc_unpack_rep_msg(struct ptlrpc_request *req, int len);
+int ptlrpc_unpack_req_msg(struct ptlrpc_request *req, int len);
+
 int lustre_msg_check_version(struct lustre_msg *msg, __u32 version);
 void lustre_init_msg_v2(struct lustre_msg_v2 *msg, int count, __u32 *lens,
                         char **bufs);
@@ -991,23 +1637,18 @@ int lustre_pack_reply_flags(struct ptlrpc_request *, int count, __u32 *lens,
 int lustre_shrink_msg(struct lustre_msg *msg, int segment,
                       unsigned int newlen, int move_data);
 void lustre_free_reply_state(struct ptlrpc_reply_state *rs);
+int __lustre_unpack_msg(struct lustre_msg *m, int len);
 int lustre_msg_hdr_size(__u32 magic, int count);
 int lustre_msg_size(__u32 magic, int count, __u32 *lengths);
 int lustre_msg_size_v2(int count, __u32 *lengths);
 int lustre_packed_msg_size(struct lustre_msg *msg);
 int lustre_msg_early_size(void);
-int lustre_unpack_msg(struct lustre_msg *m, int len);
 void *lustre_msg_buf_v2(struct lustre_msg_v2 *m, int n, int min_size);
 void *lustre_msg_buf(struct lustre_msg *m, int n, int minlen);
 int lustre_msg_buflen(struct lustre_msg *m, int n);
 void lustre_msg_set_buflen(struct lustre_msg *m, int n, int len);
 int lustre_msg_bufcount(struct lustre_msg *m);
 char *lustre_msg_string (struct lustre_msg *m, int n, int max_len);
-void *lustre_swab_buf(struct lustre_msg *, int n, int minlen, void *swabber);
-void *lustre_swab_reqbuf(struct ptlrpc_request *req, int n, int minlen,
-                         void *swabber);
-void *lustre_swab_repbuf(struct ptlrpc_request *req, int n, int minlen,
-                         void *swabber);
 __u32 lustre_msghdr_get_flags(struct lustre_msg *msg);
 void lustre_msghdr_set_flags(struct lustre_msg *msg, __u32 flags);
 __u32 lustre_msg_get_flags(struct lustre_msg *msg);
@@ -1024,6 +1665,7 @@ void lustre_msg_add_version(struct lustre_msg *msg, int version);
 __u32 lustre_msg_get_opc(struct lustre_msg *msg);
 __u64 lustre_msg_get_last_xid(struct lustre_msg *msg);
 __u64 lustre_msg_get_last_committed(struct lustre_msg *msg);
+__u64 *lustre_msg_get_versions(struct lustre_msg *msg);
 __u64 lustre_msg_get_transno(struct lustre_msg *msg);
 __u64 lustre_msg_get_slv(struct lustre_msg *msg);
 __u32 lustre_msg_get_limit(struct lustre_msg *msg);
@@ -1036,12 +1678,18 @@ __u32 lustre_msg_get_magic(struct lustre_msg *msg);
 __u32 lustre_msg_get_timeout(struct lustre_msg *msg);
 __u32 lustre_msg_get_service_time(struct lustre_msg *msg);
 __u32 lustre_msg_get_cksum(struct lustre_msg *msg);
+#if LUSTRE_VERSION_CODE < OBD_OCD_VERSION(2, 9, 0, 0)
+__u32 lustre_msg_calc_cksum(struct lustre_msg *msg, int compat18);
+#else
+# warning "remove checksum compatibility support for b1_8"
 __u32 lustre_msg_calc_cksum(struct lustre_msg *msg);
+#endif
 void lustre_msg_set_handle(struct lustre_msg *msg,struct lustre_handle *handle);
 void lustre_msg_set_type(struct lustre_msg *msg, __u32 type);
 void lustre_msg_set_opc(struct lustre_msg *msg, __u32 opc);
 void lustre_msg_set_last_xid(struct lustre_msg *msg, __u64 last_xid);
 void lustre_msg_set_last_committed(struct lustre_msg *msg,__u64 last_committed);
+void lustre_msg_set_versions(struct lustre_msg *msg, __u64 *versions);
 void lustre_msg_set_transno(struct lustre_msg *msg, __u64 transno);
 void lustre_msg_set_status(struct lustre_msg *msg, __u32 status);
 void lustre_msg_set_conn_cnt(struct lustre_msg *msg, __u32 conn_cnt);
@@ -1060,19 +1708,103 @@ lustre_shrink_reply(struct ptlrpc_request *req, int segment,
         req->rq_replen = lustre_shrink_msg(req->rq_repmsg, segment,
                                            newlen, move_data);
 }
+/** @} */
+
+/** Change request phase of \a req to \a new_phase */
+static inline void
+ptlrpc_rqphase_move(struct ptlrpc_request *req, enum rq_phase new_phase)
+{
+        if (req->rq_phase == new_phase)
+                return;
+
+        if (new_phase == RQ_PHASE_UNREGISTERING) {
+                req->rq_next_phase = req->rq_phase;
+                if (req->rq_import)
+                        cfs_atomic_inc(&req->rq_import->imp_unregistering);
+        }
+
+        if (req->rq_phase == RQ_PHASE_UNREGISTERING) {
+                if (req->rq_import)
+                        cfs_atomic_dec(&req->rq_import->imp_unregistering);
+        }
+
+        DEBUG_REQ(D_INFO, req, "move req \"%s\" -> \"%s\"",
+                  ptlrpc_rqphase2str(req), ptlrpc_phase2str(new_phase));
+
+        req->rq_phase = new_phase;
+}
+
+/**
+ * Returns true if request \a req got early reply and hard deadline is not met 
+ */
+static inline int
+ptlrpc_client_early(struct ptlrpc_request *req)
+{
+        if (OBD_FAIL_CHECK(OBD_FAIL_PTLRPC_LONG_REPL_UNLINK) &&
+            req->rq_reply_deadline > cfs_time_current_sec())
+                return 0;
+        return req->rq_early;
+}
+
+/**
+ * Returns true if we got real reply from server for this request
+ */
+static inline int
+ptlrpc_client_replied(struct ptlrpc_request *req)
+{
+        if (OBD_FAIL_CHECK(OBD_FAIL_PTLRPC_LONG_REPL_UNLINK) &&
+            req->rq_reply_deadline > cfs_time_current_sec())
+                return 0;
+        return req->rq_replied;
+}
+
+/** Returns true if request \a req is in process of receiving server reply */
+static inline int
+ptlrpc_client_recv(struct ptlrpc_request *req)
+{
+        if (OBD_FAIL_CHECK(OBD_FAIL_PTLRPC_LONG_REPL_UNLINK) &&
+            req->rq_reply_deadline > cfs_time_current_sec())
+                return 1;
+        return req->rq_receiving_reply;
+}
+
+static inline int
+ptlrpc_client_recv_or_unlink(struct ptlrpc_request *req)
+{
+        int rc;
+
+        cfs_spin_lock(&req->rq_lock);
+        if (OBD_FAIL_CHECK(OBD_FAIL_PTLRPC_LONG_REPL_UNLINK) &&
+            req->rq_reply_deadline > cfs_time_current_sec()) {
+                cfs_spin_unlock(&req->rq_lock);
+                return 1;
+        }
+        rc = req->rq_receiving_reply || req->rq_must_unlink;
+        cfs_spin_unlock(&req->rq_lock);
+        return rc;
+}
+
+static inline void
+ptlrpc_client_wake_req(struct ptlrpc_request *req)
+{
+        if (req->rq_set == NULL)
+                cfs_waitq_signal(&req->rq_reply_waitq);
+        else
+                cfs_waitq_signal(&req->rq_set->set_waitq);
+}
 
 static inline void
 ptlrpc_rs_addref(struct ptlrpc_reply_state *rs)
 {
-        LASSERT(atomic_read(&rs->rs_refcount) > 0);
-        atomic_inc(&rs->rs_refcount);
+        LASSERT(cfs_atomic_read(&rs->rs_refcount) > 0);
+        cfs_atomic_inc(&rs->rs_refcount);
 }
 
 static inline void
 ptlrpc_rs_decref(struct ptlrpc_reply_state *rs)
 {
-        LASSERT(atomic_read(&rs->rs_refcount) > 0);
-        if (atomic_dec_and_test(&rs->rs_refcount))
+        LASSERT(cfs_atomic_read(&rs->rs_refcount) > 0);
+        if (cfs_atomic_dec_and_test(&rs->rs_refcount))
                 lustre_free_reply_state(rs);
 }
 
@@ -1103,22 +1835,69 @@ static inline int ptlrpc_req_get_repsize(struct ptlrpc_request *req)
         }
 }
 
+static inline int ptlrpc_send_limit_expired(struct ptlrpc_request *req)
+{
+        if (req->rq_delay_limit != 0 &&
+            cfs_time_before(cfs_time_add(req->rq_queued_time,
+                                         cfs_time_seconds(req->rq_delay_limit)),
+                            cfs_time_current())) {
+                return 1;
+        }
+        return 0;
+}
+
+static inline int ptlrpc_no_resend(struct ptlrpc_request *req)
+{
+        if (!req->rq_no_resend && ptlrpc_send_limit_expired(req)) {
+                cfs_spin_lock(&req->rq_lock);
+                req->rq_no_resend = 1;
+                cfs_spin_unlock(&req->rq_lock);
+        }
+        return req->rq_no_resend;
+}
+
 /* ldlm/ldlm_lib.c */
+/**
+ * Target client logic
+ * @{
+ */
 int client_obd_setup(struct obd_device *obddev, struct lustre_cfg *lcfg);
 int client_obd_cleanup(struct obd_device *obddev);
 int client_connect_import(const struct lu_env *env,
-                          struct lustre_handle *conn, struct obd_device *obd,
+                          struct obd_export **exp, struct obd_device *obd,
                           struct obd_uuid *cluuid, struct obd_connect_data *,
                           void *localdata);
 int client_disconnect_export(struct obd_export *exp);
 int client_import_add_conn(struct obd_import *imp, struct obd_uuid *uuid,
                            int priority);
 int client_import_del_conn(struct obd_import *imp, struct obd_uuid *uuid);
+int client_import_find_conn(struct obd_import *imp, lnet_nid_t peer,
+                            struct obd_uuid *uuid);
 int import_set_conn_priority(struct obd_import *imp, struct obd_uuid *uuid);
+void client_destroy_import(struct obd_import *imp);
+/** @} */
+
+int server_disconnect_export(struct obd_export *exp);
 
 /* ptlrpc/pinger.c */
+/**
+ * Pinger API (client side only)
+ * @{
+ */
+enum timeout_event {
+        TIMEOUT_GRANT = 1
+};
+struct timeout_item;
+typedef int (*timeout_cb_t)(struct timeout_item *, void *);
 int ptlrpc_pinger_add_import(struct obd_import *imp);
 int ptlrpc_pinger_del_import(struct obd_import *imp);
+int ptlrpc_add_timeout_client(int time, enum timeout_event event,
+                              timeout_cb_t cb, void *data,
+                              cfs_list_t *obd_list);
+int ptlrpc_del_timeout_client(cfs_list_t *obd_list,
+                              enum timeout_event event);
+struct ptlrpc_request * ptlrpc_prep_ping(struct obd_import *imp);
+int ptlrpc_obd_ping(struct obd_device *obd);
 cfs_time_t ptlrpc_suspend_wakeup_time(void);
 #ifdef __KERNEL__
 void ping_evictor_start(void);
@@ -1128,16 +1907,51 @@ void ping_evictor_stop(void);
 #define ping_evictor_stop()     do {} while (0)
 #endif
 int ptlrpc_check_and_wait_suspend(struct ptlrpc_request *req);
+/** @} */
+
+/* ptlrpc daemon bind policy */
+typedef enum {
+        /* all ptlrpcd threads are free mode */
+        PDB_POLICY_NONE          = 1,
+        /* all ptlrpcd threads are bound mode */
+        PDB_POLICY_FULL          = 2,
+        /* <free1 bound1> <free2 bound2> ... <freeN boundN> */
+        PDB_POLICY_PAIR          = 3,
+        /* <free1 bound1> <bound1 free2> ... <freeN boundN> <boundN free1>,
+         * means each ptlrpcd[X] has two partners: thread[X-1] and thread[X+1]*/
+        PDB_POLICY_NEIGHBOR      = 4,
+} pdb_policy_t;
+
+/* ptlrpc daemon load policy
+ * It is caller's duty to specify how to push the async RPC into some ptlrpcd
+ * queue, but it is not enforced, affected by "ptlrpcd_bind_policy". If it is
+ * "PDB_POLICY_FULL", then the RPC will be processed by the selected ptlrpcd,
+ * Otherwise, the RPC may be processed by the selected ptlrpcd or its partner,
+ * depends on which is scheduled firstly, to accelerate the RPC processing. */
+typedef enum {
+        /* on the same CPU core as the caller */
+        PDL_POLICY_SAME         = 1,
+        /* within the same CPU partition, but not the same core as the caller */
+        PDL_POLICY_LOCAL        = 2,
+        /* round-robin on all CPU cores, but not the same core as the caller */
+        PDL_POLICY_ROUND        = 3,
+        /* the specified CPU core is preferred, but not enforced */
+        PDL_POLICY_PREFERRED    = 4,
+} pdl_policy_t;
 
 /* ptlrpc/ptlrpcd.c */
-int ptlrpcd_start(char *name, struct ptlrpcd_ctl *pc);
 void ptlrpcd_stop(struct ptlrpcd_ctl *pc, int force);
 void ptlrpcd_wake(struct ptlrpc_request *req);
-void ptlrpcd_add_req(struct ptlrpc_request *req);
+void ptlrpcd_add_req(struct ptlrpc_request *req, pdl_policy_t policy, int idx);
+void ptlrpcd_add_rqset(struct ptlrpc_request_set *set);
 int ptlrpcd_addref(void);
 void ptlrpcd_decref(void);
 
 /* ptlrpc/lproc_ptlrpc.c */
+/**
+ * procfs output related functions
+ * @{
+ */
 const char* ll_opcode2str(__u32 opcode);
 #ifdef LPROCFS
 void ptlrpc_lprocfs_register_obd(struct obd_device *obd);
@@ -1148,6 +1962,7 @@ static inline void ptlrpc_lprocfs_register_obd(struct obd_device *obd) {}
 static inline void ptlrpc_lprocfs_unregister_obd(struct obd_device *obd) {}
 static inline void ptlrpc_lprocfs_brw(struct ptlrpc_request *req, int bytes) {}
 #endif
+/** @} */
 
 /* ptlrpc/llog_server.c */
 int llog_origin_handle_create(struct ptlrpc_request *req);
@@ -1162,4 +1977,7 @@ int llog_catinfo(struct ptlrpc_request *req);
 /* ptlrpc/llog_client.c */
 extern struct llog_operations llog_client_ops;
 
+/** @} net */
+
 #endif
+/** @} PtlRPC */