Whamcloud - gitweb
LU-6245 lnet: remove kernel defines in userland headers
[fs/lustre-release.git] / lustre / include / lustre_net.h
index 1af3a6a..73b2a45 100644 (file)
@@ -27,7 +27,7 @@
  * Copyright (c) 2007, 2010, Oracle and/or its affiliates. All rights reserved.
  * Use is subject to license terms.
  *
- * Copyright (c) 2010, 2013, Intel Corporation.
+ * Copyright (c) 2010, 2014, Intel Corporation.
  */
 /*
  * This file is part of Lustre, http://www.lustre.org/
@@ -55,9 +55,9 @@
  * @{
  */
 
-#include <linux/lustre_net.h>
 #include <libcfs/libcfs.h>
-#include <lnet/lnet.h>
+#include <lnet/nidstr.h>
+#include <lnet/api.h>
 #include <lustre/lustre_idl.h>
 #include <lustre_ha.h>
 #include <lustre_sec.h>
 /* Macro to hide a typecast. */
 #define ptlrpc_req_async_args(req) ((void *)&req->rq_async_args)
 
+struct ptlrpc_replay_async_args {
+       int             praa_old_state;
+       int             praa_old_status;
+};
+
 /**
  * Structure to single define portal connection.
  */
@@ -716,6 +721,9 @@ enum rq_phase {
 typedef int (*ptlrpc_interpterer_t)(const struct lu_env *env,
                                     struct ptlrpc_request *req,
                                     void *arg, int rc);
+/** Type of request resend call-back */
+typedef void (*ptlrpc_resend_cb_t)(struct ptlrpc_request *req,
+                                  void *arg);
 
 /**
  * Definition of request pool structure.
@@ -1769,6 +1777,167 @@ struct ptlrpc_hpreq_ops {
         void (*hpreq_fini)(struct ptlrpc_request *);
 };
 
+struct ptlrpc_cli_req {
+       /** For bulk requests on client only: bulk descriptor */
+       struct ptlrpc_bulk_desc         *cr_bulk;
+       /** optional time limit for send attempts */
+       cfs_duration_t                   cr_delay_limit;
+       /** time request was first queued */
+       cfs_time_t                       cr_queued_time;
+       /** request sent timeval */
+       struct timeval                   cr_sent_tv;
+       /** time for request really sent out */
+       time_t                           cr_sent_out;
+       /** when req reply unlink must finish. */
+       time_t                           cr_reply_deadline;
+       /** when req bulk unlink must finish. */
+       time_t                           cr_bulk_deadline;
+       /** Portal to which this request would be sent */
+       short                            cr_req_ptl;
+       /** Portal where to wait for reply and where reply would be sent */
+       short                            cr_rep_ptl;
+       /** request resending number */
+       unsigned int                     cr_resend_nr;
+       /** What was import generation when this request was sent */
+       int                              cr_imp_gen;
+       enum lustre_imp_state            cr_send_state;
+       /** Per-request waitq introduced by bug 21938 for recovery waiting */
+       wait_queue_head_t                cr_set_waitq;
+       /** Link item for request set lists */
+       struct list_head                 cr_set_chain;
+       /** link to waited ctx */
+       struct list_head                 cr_ctx_chain;
+
+       /** client's half ctx */
+       struct ptlrpc_cli_ctx           *cr_cli_ctx;
+       /** Link back to the request set */
+       struct ptlrpc_request_set       *cr_set;
+       /** outgoing request MD handle */
+       lnet_handle_md_t                 cr_req_md_h;
+       /** request-out callback parameter */
+       struct ptlrpc_cb_id              cr_req_cbid;
+       /** incoming reply MD handle */
+       lnet_handle_md_t                 cr_reply_md_h;
+       wait_queue_head_t                cr_reply_waitq;
+       /** reply callback parameter */
+       struct ptlrpc_cb_id              cr_reply_cbid;
+       /** Async completion handler, called when reply is received */
+       ptlrpc_interpterer_t             cr_reply_interp;
+       /** Resend handler, called when request is resend to update RPC data */
+       ptlrpc_resend_cb_t               cr_resend_cb;
+       /** Async completion context */
+       union ptlrpc_async_args          cr_async_args;
+       /** Opaq data for replay and commit callbacks. */
+       void                            *cr_cb_data;
+       /**
+        * Commit callback, called when request is committed and about to be
+        * freed.
+        */
+       void (*cr_commit_cb)(struct ptlrpc_request *);
+       /** Replay callback, called after request is replayed at recovery */
+       void (*cr_replay_cb)(struct ptlrpc_request *);
+};
+
+/** client request member alias */
+/* NB: these alias should NOT be used by any new code, instead they should
+ * be removed step by step to avoid potential abuse */
+#define rq_bulk                        rq_cli.cr_bulk
+#define rq_delay_limit         rq_cli.cr_delay_limit
+#define rq_queued_time         rq_cli.cr_queued_time
+#define rq_sent_tv             rq_cli.cr_sent_tv
+#define rq_real_sent           rq_cli.cr_sent_out
+#define rq_reply_deadline      rq_cli.cr_reply_deadline
+#define rq_bulk_deadline       rq_cli.cr_bulk_deadline
+#define rq_nr_resend           rq_cli.cr_resend_nr
+#define rq_request_portal      rq_cli.cr_req_ptl
+#define rq_reply_portal                rq_cli.cr_rep_ptl
+#define rq_import_generation   rq_cli.cr_imp_gen
+#define rq_send_state          rq_cli.cr_send_state
+#define rq_set_chain           rq_cli.cr_set_chain
+#define rq_ctx_chain           rq_cli.cr_ctx_chain
+#define rq_set                 rq_cli.cr_set
+#define rq_set_waitq           rq_cli.cr_set_waitq
+#define rq_cli_ctx             rq_cli.cr_cli_ctx
+#define rq_req_md_h            rq_cli.cr_req_md_h
+#define rq_req_cbid            rq_cli.cr_req_cbid
+#define rq_reply_md_h          rq_cli.cr_reply_md_h
+#define rq_reply_waitq         rq_cli.cr_reply_waitq
+#define rq_reply_cbid          rq_cli.cr_reply_cbid
+#define rq_interpret_reply     rq_cli.cr_reply_interp
+#define rq_resend_cb           rq_cli.cr_resend_cb
+#define rq_async_args          rq_cli.cr_async_args
+#define rq_cb_data             rq_cli.cr_cb_data
+#define rq_commit_cb           rq_cli.cr_commit_cb
+#define rq_replay_cb           rq_cli.cr_replay_cb
+
+struct ptlrpc_srv_req {
+       /** initial thread servicing this request */
+       struct ptlrpc_thread            *sr_svc_thread;
+       /**
+        * 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
+        */
+       struct list_head                 sr_timed_list;
+       /** server-side per-export list */
+       struct list_head                 sr_exp_list;
+       /** server-side history, used for debuging purposes. */
+       struct list_head                 sr_hist_list;
+       /** history sequence # */
+       __u64                            sr_hist_seq;
+       /** the index of service's srv_at_array into which request is linked */
+       time_t                           sr_at_index;
+       /** authed uid */
+       uid_t                            sr_auth_uid;
+       /** authed uid mapped to */
+       uid_t                            sr_auth_mapped_uid;
+       /** RPC is generated from what part of Lustre */
+       enum lustre_sec_part             sr_sp_from;
+       /** request session context */
+       struct lu_context                sr_ses;
+       /** \addtogroup  nrs
+        * @{
+        */
+       /** stub for NRS request */
+       struct ptlrpc_nrs_request        sr_nrq;
+       /** @} nrs */
+       /** request arrival time */
+       struct timeval                   sr_arrival_time;
+       /** server's half ctx */
+       struct ptlrpc_svc_ctx           *sr_svc_ctx;
+       /** (server side), pointed directly into req buffer */
+       struct ptlrpc_user_desc         *sr_user_desc;
+       /** separated reply state */
+       struct ptlrpc_reply_state       *sr_reply_state;
+       /** server-side hp handlers */
+       struct ptlrpc_hpreq_ops         *sr_ops;
+       /** incoming request buffer */
+       struct ptlrpc_request_buffer_desc *sr_rqbd;
+};
+
+/** server request member alias */
+/* NB: these alias should NOT be used by any new code, instead they should
+ * be removed step by step to avoid potential abuse */
+#define rq_svc_thread          rq_srv.sr_svc_thread
+#define rq_timed_list          rq_srv.sr_timed_list
+#define rq_exp_list            rq_srv.sr_exp_list
+#define rq_history_list                rq_srv.sr_hist_list
+#define rq_history_seq         rq_srv.sr_hist_seq
+#define rq_at_index            rq_srv.sr_at_index
+#define rq_auth_uid            rq_srv.sr_auth_uid
+#define rq_auth_mapped_uid     rq_srv.sr_auth_mapped_uid
+#define rq_sp_from             rq_srv.sr_sp_from
+#define rq_session             rq_srv.sr_ses
+#define rq_nrq                 rq_srv.sr_nrq
+#define rq_arrival_time                rq_srv.sr_arrival_time
+#define rq_reply_state         rq_srv.sr_reply_state
+#define rq_svc_ctx             rq_srv.sr_svc_ctx
+#define rq_user_desc           rq_srv.sr_user_desc
+#define rq_ops                 rq_srv.sr_ops
+#define rq_rqbd                        rq_srv.sr_rqbd
+
 /**
  * Represents remote procedure call.
  *
@@ -1777,47 +1946,19 @@ struct ptlrpc_hpreq_ops {
  */
 struct ptlrpc_request {
        /* Request type: one of PTL_RPC_MSG_* */
-       int                     rq_type;
+       int                              rq_type;
        /** Result of request processing */
-       int                     rq_status;
+       int                              rq_status;
        /**
         * Linkage item through which this request is included into
         * sending/delayed lists on client and into rqbd list on server
         */
-       struct list_head        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
+       struct list_head                 rq_list;
+       /** Lock to protect request flags and some other important bits, like
+        * rq_list
         */
-       struct list_head        rq_timed_list;
-       /** server-side history, used for debuging purposes. */
-       struct list_head        rq_history_list;
-       /** server-side per-export list */
-       struct list_head        rq_exp_list;
-       /** server-side hp handlers */
-       struct ptlrpc_hpreq_ops *rq_ops;
-
-       /** initial thread servicing this request */
-       struct ptlrpc_thread    *rq_svc_thread;
-
-        /** history sequence # */
-       __u64                   rq_history_seq;
-       /** \addtogroup  nrs
-        * @{
-        */
-       /** stub for NRS request */
-       struct ptlrpc_nrs_request rq_nrq;
-       /** @} nrs */
-        /** the index of service's srv_at_array into which request is linked */
-        time_t rq_at_index;
-        /** Lock to protect request flags and some other important bits, like
-         * rq_list
-         */
-       spinlock_t rq_lock;
-       /** client-side flags are serialized by rq_lock */
+       spinlock_t                       rq_lock;
+       /** client-side flags are serialized by rq_lock @{ */
        unsigned int rq_intr:1, rq_replied:1, rq_err:1,
                 rq_timedout:1, rq_resend:1, rq_restart:1,
                 /**
@@ -1833,37 +1974,39 @@ struct ptlrpc_request {
                 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_req_unlink:1, rq_reply_unlink:1,
-                rq_memalloc:1,      /* req originated from "kswapd" */
-                /* server-side flags */
-                rq_packed_final:1,  /* packed final reply */
-                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,
+               rq_req_unlinked:1,      /* unlinked request buffer from lnet */
+               rq_reply_unlinked:1,    /* unlinked reply buffer from lnet */
+               rq_memalloc:1,      /* req originated from "kswapd" */
+               rq_committed:1,
+               rq_reply_truncated:1,
+               /** whether the "rq_set" is a valid one */
+               rq_invalid_rqset:1,
                rq_generation_set:1,
-               /* do not resend request on -EINPROGRESS */
+               /** do not resend request on -EINPROGRESS */
                rq_no_retry_einprogress:1,
                /* allow the req to be sent if the import is in recovery
                 * status */
                rq_allow_replay:1,
                /* bulk request, sent to server, but uncommitted */
                rq_unstable:1;
+       /** @} */
 
-       unsigned int rq_nr_resend;
-
-       enum rq_phase rq_phase; /* one of RQ_PHASE_* */
-       enum rq_phase rq_next_phase; /* one of RQ_PHASE_* to be used next */
-       atomic_t rq_refcount;/* client-side refcount for SENT race,
-                                   server-side refcounf for multiple replies */
-
-       /** Portal to which this request would be sent */
-       short rq_request_portal;  /* XXX FIXME bug 249 */
-       /** Portal where to wait for reply and where reply would be sent */
-       short rq_reply_portal;    /* XXX FIXME bug 249 */
+       /** server-side flags @{ */
+       unsigned int
+               rq_hp:1,                /**< high priority RPC */
+               rq_at_linked:1,         /**< link into service's srv_at_array */
+               rq_packed_final:1;      /**< packed final reply */
+       /** @} */
 
+       /** one of RQ_PHASE_* */
+       enum rq_phase                    rq_phase;
+       /** one of RQ_PHASE_* to be used next */
+       enum rq_phase                    rq_next_phase;
+       /**
+        * client-side refcount for SENT race, server-side refcounf
+        * for multiple replies
+        */
+       atomic_t                         rq_refcount;
         /**
          * client-side:
          * !rq_truncate : # reply bytes actually received,
@@ -1874,6 +2017,8 @@ struct ptlrpc_request {
         int rq_reqlen;
         /** Reply length */
         int rq_replen;
+       /** Pool if request is from preallocated list */
+       struct ptlrpc_request_pool      *rq_pool;
        /** Request message - what client sent */
        struct lustre_msg *rq_reqmsg;
         /** Reply message - server response */
@@ -1882,22 +2027,23 @@ struct ptlrpc_request {
         __u64 rq_transno;
         /** xid */
         __u64 rq_xid;
-        /**
-         * List item to for replay list. Not yet commited requests get linked
-         * there.
-         * Also see \a rq_replay comment above.
-         */
-       struct list_head        rq_replay_list;
-
+       /**
+        * List item to for replay list. Not yet commited requests get linked
+        * there.
+        * Also see \a rq_replay comment above.
+        * It's also link chain on obd_export::exp_req_replay_queue
+        */
+       struct list_head                 rq_replay_list;
+       /** non-shared members for client & server request*/
+       union {
+               struct ptlrpc_cli_req    rq_cli;
+               struct ptlrpc_srv_req    rq_srv;
+       };
        /**
         * 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 */
-       struct list_head        rq_ctx_chain;   /**< link to waited ctx */
-
-       struct sptlrpc_flavor   rq_flvr;        /**< for client & server */
-       enum lustre_sec_part    rq_sp_from;
+       /** description of flavors for client & server */
+       struct sptlrpc_flavor            rq_flvr;
 
        /* client/server security flags */
        unsigned int
@@ -1916,19 +2062,16 @@ struct ptlrpc_request {
                                  rq_pack_bulk:1,
                                  /* doesn't expect reply FIXME */
                                  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 */
+                                rq_pill_init:1, /* pill initialized */
+                                rq_srv_req:1; /* server request */
 
-        /* (server side), pointed directly into req buffer */
-        struct ptlrpc_user_desc *rq_user_desc;
 
-        /* various buffer pointers */
-        struct lustre_msg       *rq_reqbuf;      /* req wrapper */
-       char                    *rq_repbuf;      /* rep buffer */
-       struct lustre_msg       *rq_repdata;     /* rep wrapper msg */
-       struct lustre_msg       *rq_clrbuf;      /* only in priv mode */
+       /** various buffer pointers */
+       struct lustre_msg               *rq_reqbuf;      /**< req wrapper */
+       char                            *rq_repbuf;      /**< rep buffer */
+       struct lustre_msg               *rq_repdata;     /**< rep wrapper msg */
+       /** only in priv mode */
+       struct lustre_msg               *rq_clrbuf;
         int                      rq_reqbuf_len;  /* req wrapper buf len */
         int                      rq_reqdata_len; /* req wrapper msg len */
         int                      rq_repbuf_len;  /* rep buffer len */
@@ -1937,105 +2080,37 @@ struct ptlrpc_request {
         int                      rq_clrdata_len; /* only in priv mode */
 
        /** early replies go to offset 0, regular replies go after that */
-       unsigned int             rq_reply_off;
-
-        /** @} */
-
-        /** 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;
-
-        /** how many early replies (for stats) */
-        int rq_early_count;
-
-        /** 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... */
-        /** 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;
-
-       /** client-only incoming reply */
-       lnet_handle_md_t     rq_reply_md_h;
-       wait_queue_head_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;
-
-        /** For bulk requests on client only: bulk descriptor */
-        struct ptlrpc_bulk_desc *rq_bulk;
-
-        /** 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;
+       unsigned int                     rq_reply_off;
 
-        /** Multi-rpc bits */
-        /** Per-request waitq introduced by bug 21938 for recovery waiting */
-       wait_queue_head_t rq_set_waitq;
-       /** Link item for request set lists */
-       struct list_head  rq_set_chain;
-        /** 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;
-
-        /** Pool if request is from preallocated list */
-        struct ptlrpc_request_pool *rq_pool;
-
-        struct lu_context           rq_session;
+       /** @} */
 
-        /** request format description */
-        struct req_capsule          rq_pill;
+       /** Fields that help to see if request and reply were swabbed or not */
+       __u32                            rq_req_swab_mask;
+       __u32                            rq_rep_swab_mask;
+
+       /** how many early replies (for stats) */
+       int                              rq_early_count;
+       /** Server-side, export on which request was received */
+       struct obd_export               *rq_export;
+       /** import where request is being sent */
+       struct obd_import               *rq_import;
+       /** our LNet NID */
+       lnet_nid_t                       rq_self;
+       /** Peer description (the other side) */
+       lnet_process_id_t                rq_peer;
+       /**
+        * service time estimate (secs)
+        * If the request is not served by this time, it is marked as timed out.
+        */
+       int                              rq_timeout;
+       /**
+        * when request/reply sent (secs), or time when request should be sent
+        */
+       time_t                           rq_sent;
+       /** when request must finish. */
+       time_t                           rq_deadline;
+       /** request format description */
+       struct req_capsule               rq_pill;
 };
 
 /**
@@ -2176,7 +2251,7 @@ ptlrpc_rqphase2str(struct ptlrpc_request *req)
 /**
  * Debugging functions and helpers to print request structure into debug log
  * @{
- */ 
+ */
 /* Spare the preprocessor, spoil the bugs. */
 #define FLAG(field, str) (field ? str : "")
 
@@ -2332,7 +2407,7 @@ struct ptlrpc_thread {
         /**
          * service thread pid
          */
-        pid_t t_pid; 
+        pid_t t_pid;
         /**
          * put watchdog in the structure per thread b=14840
          */
@@ -2873,7 +2948,6 @@ int ptlrpc_send_reply(struct ptlrpc_request *req, int flags);
 int ptlrpc_reply(struct ptlrpc_request *req);
 int ptlrpc_send_error(struct ptlrpc_request *req, int difficult);
 int ptlrpc_error(struct ptlrpc_request *req);
-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);
@@ -2894,7 +2968,6 @@ struct ptlrpc_connection *ptlrpc_uuid_to_connection(struct obd_uuid *uuid);
 
 int ptlrpc_queue_wait(struct ptlrpc_request *req);
 int ptlrpc_replay_req(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);
@@ -2905,16 +2978,11 @@ struct ptlrpc_request_set *ptlrpc_prep_fcset(int max, set_producer_func func,
                                             void *arg);
 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(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 *);
-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);
@@ -3100,14 +3168,14 @@ int ptlrpc_reconnect_import(struct obd_import *imp);
 /** @} */
 
 /**
- * ptlrpc msg buffer and swab interface 
+ * ptlrpc msg buffer and swab interface
  *
  * @{
  */
 int ptlrpc_buf_need_swab(struct ptlrpc_request *req, const int inout,
-                         int index);
+                        __u32 index);
 void ptlrpc_buf_set_swabbed(struct ptlrpc_request *req, const int inout,
-                                int index);
+                           __u32 index);
 int ptlrpc_unpack_rep_msg(struct ptlrpc_request *req, int len);
 int ptlrpc_unpack_req_msg(struct ptlrpc_request *req, int len);
 
@@ -3127,30 +3195,30 @@ 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);
-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);
+__u32 lustre_msg_hdr_size(__u32 magic, __u32 count);
+__u32 lustre_msg_size(__u32 magic, int count, __u32 *lengths);
+__u32 lustre_msg_size_v2(int count, __u32 *lengths);
+__u32 lustre_packed_msg_size(struct lustre_msg *msg);
+__u32 lustre_msg_early_size(void);
+void *lustre_msg_buf_v2(struct lustre_msg_v2 *m, __u32 n, __u32 min_size);
+void *lustre_msg_buf(struct lustre_msg *m, __u32 n, __u32 minlen);
+__u32 lustre_msg_buflen(struct lustre_msg *m, __u32 n);
+void lustre_msg_set_buflen(struct lustre_msg *m, __u32 n, __u32 len);
+__u32 lustre_msg_bufcount(struct lustre_msg *m);
+char *lustre_msg_string(struct lustre_msg *m, __u32 n, __u32 max_len);
 __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);
-void lustre_msg_add_flags(struct lustre_msg *msg, int flags);
-void lustre_msg_set_flags(struct lustre_msg *msg, int flags);
-void lustre_msg_clear_flags(struct lustre_msg *msg, int flags);
+void lustre_msg_add_flags(struct lustre_msg *msg, __u32 flags);
+void lustre_msg_set_flags(struct lustre_msg *msg, __u32 flags);
+void lustre_msg_clear_flags(struct lustre_msg *msg, __u32 flags);
 __u32 lustre_msg_get_op_flags(struct lustre_msg *msg);
-void lustre_msg_add_op_flags(struct lustre_msg *msg, int flags);
-void lustre_msg_set_op_flags(struct lustre_msg *msg, int flags);
+void lustre_msg_add_op_flags(struct lustre_msg *msg, __u32 flags);
+void lustre_msg_set_op_flags(struct lustre_msg *msg, __u32 flags);
 struct lustre_handle *lustre_msg_get_handle(struct lustre_msg *msg);
 __u32 lustre_msg_get_type(struct lustre_msg *msg);
 __u32 lustre_msg_get_version(struct lustre_msg *msg);
-void lustre_msg_add_version(struct lustre_msg *msg, int version);
+void lustre_msg_add_version(struct lustre_msg *msg, __u32 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);
@@ -3257,7 +3325,7 @@ ptlrpc_rqphase_move(struct ptlrpc_request *req, enum rq_phase new_phase)
 }
 
 /**
- * Returns true if request \a req got early reply and hard deadline is not met 
+ * 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)
@@ -3301,8 +3369,8 @@ ptlrpc_client_recv_or_unlink(struct ptlrpc_request *req)
                spin_unlock(&req->rq_lock);
                return 1;
        }
-       rc = req->rq_receiving_reply ;
-       rc = rc || req->rq_req_unlink || req->rq_reply_unlink;
+       rc = !req->rq_req_unlinked || !req->rq_reply_unlinked ||
+            req->rq_receiving_reply;
        spin_unlock(&req->rq_lock);
        return rc;
 }
@@ -3493,7 +3561,7 @@ void ptlrpcd_decref(void);
  * @{
  */
 const char* ll_opcode2str(__u32 opcode);
-#ifdef LPROCFS
+#ifdef CONFIG_PROC_FS
 void ptlrpc_lprocfs_register_obd(struct obd_device *obd);
 void ptlrpc_lprocfs_unregister_obd(struct obd_device *obd);
 void ptlrpc_lprocfs_brw(struct ptlrpc_request *req, int bytes);