SUNRPC: Move the task->tk_bytes_sent and tk_rtt to struct rpc_rqst
[safe/jmp/linux-2.6] / net / sunrpc / xprt.c
index d1afec6..65fe2e4 100644 (file)
@@ -12,8 +12,9 @@
  *  -  Next, the caller puts together the RPC message, stuffs it into
  *     the request struct, and calls xprt_transmit().
  *  -  xprt_transmit sends the message and installs the caller on the
- *     transport's wait list. At the same time, it installs a timer that
- *     is run after the packet's timeout has expired.
+ *     transport's wait list. At the same time, if a reply is expected,
+ *     it installs a timer that is run after the packet's timeout has
+ *     expired.
  *  -  When a packet arrives, the data_ready handler walks the list of
  *     pending requests for that transport. If a matching XID is found, the
  *     caller is woken up, and the timer removed.
 #include <linux/interrupt.h>
 #include <linux/workqueue.h>
 #include <linux/net.h>
+#include <linux/ktime.h>
 
 #include <linux/sunrpc/clnt.h>
 #include <linux/sunrpc/metrics.h>
+#include <linux/sunrpc/bc_xprt.h>
+
+#include "sunrpc.h"
 
 /*
  * Local variables
@@ -58,7 +63,6 @@
  * Local functions
  */
 static void    xprt_request_init(struct rpc_task *, struct rpc_xprt *);
-static inline void     do_xprt_reserve(struct rpc_task *);
 static void    xprt_connect_status(struct rpc_task *task);
 static int      __xprt_get_cong(struct rpc_xprt *, struct rpc_task *);
 
@@ -192,8 +196,8 @@ EXPORT_SYMBOL_GPL(xprt_load_transport);
  */
 int xprt_reserve_xprt(struct rpc_task *task)
 {
-       struct rpc_xprt *xprt = task->tk_xprt;
        struct rpc_rqst *req = task->tk_rqstp;
+       struct rpc_xprt *xprt = req->rq_xprt;
 
        if (test_and_set_bit(XPRT_LOCKED, &xprt->state)) {
                if (task == xprt->snd_task)
@@ -611,7 +615,7 @@ void xprt_disconnect_done(struct rpc_xprt *xprt)
        dprintk("RPC:       disconnected transport %p\n", xprt);
        spin_lock_bh(&xprt->transport_lock);
        xprt_clear_connected(xprt);
-       xprt_wake_pending_tasks(xprt, -ENOTCONN);
+       xprt_wake_pending_tasks(xprt, -EAGAIN);
        spin_unlock_bh(&xprt->transport_lock);
 }
 EXPORT_SYMBOL_GPL(xprt_disconnect_done);
@@ -629,7 +633,7 @@ void xprt_force_disconnect(struct rpc_xprt *xprt)
        /* Try to schedule an autoclose RPC call */
        if (test_and_set_bit(XPRT_LOCKED, &xprt->state) == 0)
                queue_work(rpciod_workqueue, &xprt->task_cleanup);
-       xprt_wake_pending_tasks(xprt, -ENOTCONN);
+       xprt_wake_pending_tasks(xprt, -EAGAIN);
        spin_unlock_bh(&xprt->transport_lock);
 }
 
@@ -656,7 +660,7 @@ void xprt_conditional_disconnect(struct rpc_xprt *xprt, unsigned int cookie)
        /* Try to schedule an autoclose RPC call */
        if (test_and_set_bit(XPRT_LOCKED, &xprt->state) == 0)
                queue_work(rpciod_workqueue, &xprt->task_cleanup);
-       xprt_wake_pending_tasks(xprt, -ENOTCONN);
+       xprt_wake_pending_tasks(xprt, -EAGAIN);
 out:
        spin_unlock_bh(&xprt->transport_lock);
 }
@@ -672,10 +676,8 @@ xprt_init_autodisconnect(unsigned long data)
        if (test_and_set_bit(XPRT_LOCKED, &xprt->state))
                goto out_abort;
        spin_unlock(&xprt->transport_lock);
-       if (xprt_connecting(xprt))
-               xprt_release_write(xprt, NULL);
-       else
-               queue_work(rpciod_workqueue, &xprt->task_cleanup);
+       set_bit(XPRT_CONNECTION_CLOSE, &xprt->state);
+       queue_work(rpciod_workqueue, &xprt->task_cleanup);
        return;
 out_abort:
        spin_unlock(&xprt->transport_lock);
@@ -699,18 +701,26 @@ void xprt_connect(struct rpc_task *task)
        }
        if (!xprt_lock_write(xprt, task))
                return;
+
+       if (test_and_clear_bit(XPRT_CLOSE_WAIT, &xprt->state))
+               xprt->ops->close(xprt);
+
        if (xprt_connected(xprt))
                xprt_release_write(xprt, task);
        else {
                if (task->tk_rqstp)
                        task->tk_rqstp->rq_bytes_sent = 0;
 
-               task->tk_timeout = xprt->connect_timeout;
+               task->tk_timeout = task->tk_rqstp->rq_timeout;
                rpc_sleep_on(&xprt->pending, task, xprt_connect_status);
+
+               if (test_bit(XPRT_CLOSING, &xprt->state))
+                       return;
+               if (xprt_test_and_set_connecting(xprt))
+                       return;
                xprt->stat.connect_start = jiffies;
                xprt->ops->connect(task);
        }
-       return;
 }
 
 static void xprt_connect_status(struct rpc_task *task)
@@ -726,9 +736,8 @@ static void xprt_connect_status(struct rpc_task *task)
        }
 
        switch (task->tk_status) {
-       case -ENOTCONN:
-               dprintk("RPC: %5u xprt_connect_status: connection broken\n",
-                               task->tk_pid);
+       case -EAGAIN:
+               dprintk("RPC: %5u xprt_connect_status: retrying\n", task->tk_pid);
                break;
        case -ETIMEDOUT:
                dprintk("RPC: %5u xprt_connect_status: connect attempt timed "
@@ -766,25 +775,19 @@ struct rpc_rqst *xprt_lookup_rqst(struct rpc_xprt *xprt, __be32 xid)
 }
 EXPORT_SYMBOL_GPL(xprt_lookup_rqst);
 
-/**
- * xprt_update_rtt - update an RPC client's RTT state after receiving a reply
- * @task: RPC request that recently completed
- *
- */
-void xprt_update_rtt(struct rpc_task *task)
+static void xprt_update_rtt(struct rpc_task *task)
 {
        struct rpc_rqst *req = task->tk_rqstp;
        struct rpc_rtt *rtt = task->tk_client->cl_rtt;
        unsigned timer = task->tk_msg.rpc_proc->p_timer;
+       long m = usecs_to_jiffies(ktime_to_us(req->rq_rtt));
 
        if (timer) {
                if (req->rq_ntrans == 1)
-                       rpc_update_rtt(rtt, timer,
-                                       (long)jiffies - req->rq_xtime);
+                       rpc_update_rtt(rtt, timer, m);
                rpc_set_timeo(rtt, timer, req->rq_ntrans - 1);
        }
 }
-EXPORT_SYMBOL_GPL(xprt_update_rtt);
 
 /**
  * xprt_complete_rqst - called when reply processing is complete
@@ -802,13 +805,16 @@ void xprt_complete_rqst(struct rpc_task *task, int copied)
                        task->tk_pid, ntohl(req->rq_xid), copied);
 
        xprt->stat.recvs++;
-       task->tk_rtt = (long)jiffies - req->rq_xtime;
+       req->rq_rtt = ktime_sub(ktime_get(), req->rq_xtime);
+       if (xprt->ops->timer != NULL)
+               xprt_update_rtt(task);
 
        list_del_init(&req->rq_list);
        req->rq_private_buf.len = copied;
-       /* Ensure all writes are done before we update req->rq_received */
+       /* Ensure all writes are done before we update */
+       /* req->rq_reply_bytes_recvd */
        smp_wmb();
-       req->rq_received = copied;
+       req->rq_reply_bytes_recvd = copied;
        rpc_wake_up_queued_task(&xprt->pending, task);
 }
 EXPORT_SYMBOL_GPL(xprt_complete_rqst);
@@ -823,7 +829,7 @@ static void xprt_timer(struct rpc_task *task)
        dprintk("RPC: %5u xprt_timer\n", task->tk_pid);
 
        spin_lock_bh(&xprt->transport_lock);
-       if (!req->rq_received) {
+       if (!req->rq_reply_bytes_recvd) {
                if (xprt->ops->timer)
                        xprt->ops->timer(task);
        } else
@@ -831,6 +837,11 @@ static void xprt_timer(struct rpc_task *task)
        spin_unlock_bh(&xprt->transport_lock);
 }
 
+static inline int xprt_has_timer(struct rpc_xprt *xprt)
+{
+       return xprt->idle_timeout != 0;
+}
+
 /**
  * xprt_prepare_transmit - reserve the transport before sending a request
  * @task: RPC task about to send a request
@@ -845,19 +856,12 @@ int xprt_prepare_transmit(struct rpc_task *task)
        dprintk("RPC: %5u xprt_prepare_transmit\n", task->tk_pid);
 
        spin_lock_bh(&xprt->transport_lock);
-       if (req->rq_received && !req->rq_bytes_sent) {
-               err = req->rq_received;
+       if (req->rq_reply_bytes_recvd && !req->rq_bytes_sent) {
+               err = req->rq_reply_bytes_recvd;
                goto out_unlock;
        }
-       if (!xprt->ops->reserve_xprt(task)) {
+       if (!xprt->ops->reserve_xprt(task))
                err = -EAGAIN;
-               goto out_unlock;
-       }
-
-       if (!xprt_connected(xprt)) {
-               err = -ENOTCONN;
-               goto out_unlock;
-       }
 out_unlock:
        spin_unlock_bh(&xprt->transport_lock);
        return err;
@@ -865,7 +869,7 @@ out_unlock:
 
 void xprt_end_transmit(struct rpc_task *task)
 {
-       xprt_release_write(task->tk_xprt, task);
+       xprt_release_write(task->tk_rqstp->rq_xprt, task);
 }
 
 /**
@@ -882,8 +886,11 @@ void xprt_transmit(struct rpc_task *task)
 
        dprintk("RPC: %5u xprt_transmit(%u)\n", task->tk_pid, req->rq_slen);
 
-       if (!req->rq_received) {
-               if (list_empty(&req->rq_list)) {
+       if (!req->rq_reply_bytes_recvd) {
+               if (list_empty(&req->rq_list) && rpc_reply_expected(task)) {
+                       /*
+                        * Add to the list only if we're expecting a reply
+                        */
                        spin_lock_bh(&xprt->transport_lock);
                        /* Update the softirq receive buffer */
                        memcpy(&req->rq_private_buf, &req->rq_rcv_buf,
@@ -899,37 +906,36 @@ void xprt_transmit(struct rpc_task *task)
                return;
 
        req->rq_connect_cookie = xprt->connect_cookie;
-       req->rq_xtime = jiffies;
+       req->rq_xtime = ktime_get();
        status = xprt->ops->send_request(task);
-       if (status == 0) {
-               dprintk("RPC: %5u xmit complete\n", task->tk_pid);
-               spin_lock_bh(&xprt->transport_lock);
-
-               xprt->ops->set_retrans_timeout(task);
-
-               xprt->stat.sends++;
-               xprt->stat.req_u += xprt->stat.sends - xprt->stat.recvs;
-               xprt->stat.bklog_u += xprt->backlog.qlen;
-
-               /* Don't race with disconnect */
-               if (!xprt_connected(xprt))
-                       task->tk_status = -ENOTCONN;
-               else if (!req->rq_received)
-                       rpc_sleep_on(&xprt->pending, task, xprt_timer);
-               spin_unlock_bh(&xprt->transport_lock);
+       if (status != 0) {
+               task->tk_status = status;
                return;
        }
 
-       /* Note: at this point, task->tk_sleeping has not yet been set,
-        *       hence there is no danger of the waking up task being put on
-        *       schedq, and being picked up by a parallel run of rpciod().
-        */
-       task->tk_status = status;
-       if (status == -ECONNREFUSED)
-               rpc_sleep_on(&xprt->sending, task, NULL);
+       dprintk("RPC: %5u xmit complete\n", task->tk_pid);
+       spin_lock_bh(&xprt->transport_lock);
+
+       xprt->ops->set_retrans_timeout(task);
+
+       xprt->stat.sends++;
+       xprt->stat.req_u += xprt->stat.sends - xprt->stat.recvs;
+       xprt->stat.bklog_u += xprt->backlog.qlen;
+
+       /* Don't race with disconnect */
+       if (!xprt_connected(xprt))
+               task->tk_status = -ENOTCONN;
+       else if (!req->rq_reply_bytes_recvd && rpc_reply_expected(task)) {
+               /*
+                * Sleep on the pending queue since
+                * we're expecting a reply.
+                */
+               rpc_sleep_on(&xprt->pending, task, xprt_timer);
+       }
+       spin_unlock_bh(&xprt->transport_lock);
 }
 
-static inline void do_xprt_reserve(struct rpc_task *task)
+static void xprt_alloc_slot(struct rpc_task *task)
 {
        struct rpc_xprt *xprt = task->tk_xprt;
 
@@ -949,6 +955,16 @@ static inline void do_xprt_reserve(struct rpc_task *task)
        rpc_sleep_on(&xprt->backlog, task, NULL);
 }
 
+static void xprt_free_slot(struct rpc_xprt *xprt, struct rpc_rqst *req)
+{
+       memset(req, 0, sizeof(*req));   /* mark unused */
+
+       spin_lock(&xprt->reserve_lock);
+       list_add(&req->rq_list, &xprt->free);
+       rpc_wake_up_next(&xprt->backlog);
+       spin_unlock(&xprt->reserve_lock);
+}
+
 /**
  * xprt_reserve - allocate an RPC request slot
  * @task: RPC task requesting a slot allocation
@@ -962,7 +978,7 @@ void xprt_reserve(struct rpc_task *task)
 
        task->tk_status = -EIO;
        spin_lock(&xprt->reserve_lock);
-       do_xprt_reserve(task);
+       xprt_alloc_slot(task);
        spin_unlock(&xprt->reserve_lock);
 }
 
@@ -998,11 +1014,13 @@ static void xprt_request_init(struct rpc_task *task, struct rpc_xprt *xprt)
  */
 void xprt_release(struct rpc_task *task)
 {
-       struct rpc_xprt *xprt = task->tk_xprt;
+       struct rpc_xprt *xprt;
        struct rpc_rqst *req;
 
        if (!(req = task->tk_rqstp))
                return;
+
+       xprt = req->rq_xprt;
        rpc_count_iostats(task);
        spin_lock_bh(&xprt->transport_lock);
        xprt->ops->release_xprt(xprt, task);
@@ -1011,22 +1029,21 @@ void xprt_release(struct rpc_task *task)
        if (!list_empty(&req->rq_list))
                list_del(&req->rq_list);
        xprt->last_used = jiffies;
-       if (list_empty(&xprt->recv))
+       if (list_empty(&xprt->recv) && xprt_has_timer(xprt))
                mod_timer(&xprt->timer,
                                xprt->last_used + xprt->idle_timeout);
        spin_unlock_bh(&xprt->transport_lock);
-       xprt->ops->buf_free(req->rq_buffer);
+       if (req->rq_buffer)
+               xprt->ops->buf_free(req->rq_buffer);
        task->tk_rqstp = NULL;
        if (req->rq_release_snd_buf)
                req->rq_release_snd_buf(req);
-       memset(req, 0, sizeof(*req));   /* mark unused */
 
        dprintk("RPC: %5u release request %p\n", task->tk_pid, req);
-
-       spin_lock(&xprt->reserve_lock);
-       list_add(&req->rq_list, &xprt->free);
-       rpc_wake_up_next(&xprt->backlog);
-       spin_unlock(&xprt->reserve_lock);
+       if (likely(!bc_prealloc(req)))
+               xprt_free_slot(xprt, req);
+       else
+               xprt_free_bc_request(req);
 }
 
 /**
@@ -1065,9 +1082,17 @@ found:
 
        INIT_LIST_HEAD(&xprt->free);
        INIT_LIST_HEAD(&xprt->recv);
+#if defined(CONFIG_NFS_V4_1)
+       spin_lock_init(&xprt->bc_pa_lock);
+       INIT_LIST_HEAD(&xprt->bc_pa_list);
+#endif /* CONFIG_NFS_V4_1 */
+
        INIT_WORK(&xprt->task_cleanup, xprt_autoclose);
-       setup_timer(&xprt->timer, xprt_init_autodisconnect,
-                       (unsigned long)xprt);
+       if (xprt_has_timer(xprt))
+               setup_timer(&xprt->timer, xprt_init_autodisconnect,
+                           (unsigned long)xprt);
+       else
+               init_timer(&xprt->timer);
        xprt->last_used = jiffies;
        xprt->cwnd = RPC_INITCWND;
        xprt->bind_index = 0;
@@ -1086,7 +1111,6 @@ found:
 
        dprintk("RPC:       created transport %p with %u slots\n", xprt,
                        xprt->max_reqs);
-
        return xprt;
 }