Whamcloud - gitweb
LU-1144 ptlrpc: implement a NUMA aware ptlrpcd binding policy
[fs/lustre-release.git] / lustre / ptlrpc / ptlrpcd.c
index 90220ce..4f646a1 100644 (file)
@@ -28,6 +28,8 @@
 /*
  * Copyright (c) 2003, 2010, Oracle and/or its affiliates. All rights reserved.
  * Use is subject to license terms.
+ *
+ * Copyright (c) 2011, 2012, Whamcloud, Inc.
  */
 /*
  * This file is part of Lustre, http://www.lustre.org/
 #include <cl_object.h> /* cl_env_{get,put}() */
 #include <lprocfs_status.h>
 
-enum pscope_thread {
-        PT_NORMAL,
-        PT_RECOVERY,
-        PT_NR
-};
+#include "ptlrpc_internal.h"
 
-struct ptlrpcd_scope_ctl {
-        struct ptlrpcd_thread {
-                const char        *pt_name;
-                struct ptlrpcd_ctl pt_ctl;
-        } pscope_thread[PT_NR];
+struct ptlrpcd {
+        int                pd_size;
+        int                pd_index;
+        int                pd_nthreads;
+        struct ptlrpcd_ctl pd_thread_rcv;
+        struct ptlrpcd_ctl pd_threads[0];
 };
 
-static struct ptlrpcd_scope_ctl ptlrpcd_scopes[PSCOPE_NR] = {
-        [PSCOPE_BRW] = {
-                .pscope_thread = {
-                        [PT_NORMAL] = {
-                                .pt_name = "ptlrpcd-brw"
-                        },
-                        [PT_RECOVERY] = {
-                                .pt_name = "ptlrpcd-brw-rcv"
-                        }
-                }
-        },
-        [PSCOPE_OTHER] = {
-                .pscope_thread = {
-                        [PT_NORMAL] = {
-                                .pt_name = "ptlrpcd"
-                        },
-                        [PT_RECOVERY] = {
-                                .pt_name = "ptlrpcd-rcv"
-                        }
-                }
-        }
-};
+#ifdef __KERNEL__
+static int max_ptlrpcds;
+CFS_MODULE_PARM(max_ptlrpcds, "i", int, 0644,
+                "Max ptlrpcd thread count to be started.");
+
+static int ptlrpcd_bind_policy = PDB_POLICY_PAIR;
+CFS_MODULE_PARM(ptlrpcd_bind_policy, "i", int, 0644,
+                "Ptlrpcd threads binding mode.");
+#endif
+static struct ptlrpcd *ptlrpcds;
 
-cfs_semaphore_t ptlrpcd_sem;
+cfs_mutex_t ptlrpcd_mutex;
 static int ptlrpcd_users = 0;
 
 void ptlrpcd_wake(struct ptlrpc_request *req)
@@ -119,6 +106,50 @@ void ptlrpcd_wake(struct ptlrpc_request *req)
         cfs_waitq_signal(&rq_set->set_waitq);
 }
 
+static struct ptlrpcd_ctl *
+ptlrpcd_select_pc(struct ptlrpc_request *req, pdl_policy_t policy, int index)
+{
+        int idx = 0;
+
+        if (req != NULL && req->rq_send_state != LUSTRE_IMP_FULL)
+                return &ptlrpcds->pd_thread_rcv;
+
+#ifdef __KERNEL__
+        switch (policy) {
+        case PDL_POLICY_SAME:
+                idx = cfs_smp_processor_id() % ptlrpcds->pd_nthreads;
+                break;
+        case PDL_POLICY_LOCAL:
+                /* Before CPU partition patches available, process it the same
+                 * as "PDL_POLICY_ROUND". */
+# ifdef CFS_CPU_MODE_NUMA
+# warning "fix this code to use new CPU partition APIs"
+# endif
+                /* Fall through to PDL_POLICY_ROUND until the CPU
+                 * CPU partition patches are available. */
+                index = -1;
+        case PDL_POLICY_PREFERRED:
+                if (index >= 0 && index < cfs_num_online_cpus()) {
+                        idx = index % ptlrpcds->pd_nthreads;
+                        break;
+                }
+                /* Fall through to PDL_POLICY_ROUND for bad index. */
+        default:
+                /* Fall through to PDL_POLICY_ROUND for unknown policy. */
+        case PDL_POLICY_ROUND:
+                /* We do not care whether it is strict load balance. */
+                idx = ptlrpcds->pd_index + 1;
+                if (idx == cfs_smp_processor_id())
+                        idx++;
+                idx %= ptlrpcds->pd_nthreads;
+                ptlrpcds->pd_index = idx;
+                break;
+        }
+#endif /* __KERNEL__ */
+
+        return &ptlrpcds->pd_threads[idx];
+}
+
 /**
  * Move all request from an existing request set to the ptlrpcd queue.
  * All requests from the set must be in phase RQ_PHASE_NEW.
@@ -126,6 +157,14 @@ void ptlrpcd_wake(struct ptlrpc_request *req)
 void ptlrpcd_add_rqset(struct ptlrpc_request_set *set)
 {
         cfs_list_t *tmp, *pos;
+#ifdef __KERNEL__
+        struct ptlrpcd_ctl *pc;
+        struct ptlrpc_request_set *new;
+        int count, i;
+
+        pc = ptlrpcd_select_pc(NULL, PDL_POLICY_LOCAL, -1);
+        new = pc->pc_set;
+#endif
 
         cfs_list_for_each_safe(pos, tmp, &set->set_requests) {
                 struct ptlrpc_request *req =
@@ -133,119 +172,175 @@ void ptlrpcd_add_rqset(struct ptlrpc_request_set *set)
                                        rq_set_chain);
 
                 LASSERT(req->rq_phase == RQ_PHASE_NEW);
+#ifdef __KERNEL__
+                req->rq_set = new;
+                req->rq_queued_time = cfs_time_current();
+#else
                 cfs_list_del_init(&req->rq_set_chain);
                 req->rq_set = NULL;
-                ptlrpcd_add_req(req, PSCOPE_OTHER);
+                ptlrpcd_add_req(req, PDL_POLICY_LOCAL, -1);
                 cfs_atomic_dec(&set->set_remaining);
+#endif
         }
-        LASSERT(cfs_atomic_read(&set->set_remaining) == 0);
+
+#ifdef __KERNEL__
+        cfs_spin_lock(&new->set_new_req_lock);
+        cfs_list_splice_init(&set->set_requests, &new->set_new_requests);
+        i = cfs_atomic_read(&set->set_remaining);
+        count = cfs_atomic_add_return(i, &new->set_new_count);
+        cfs_atomic_set(&set->set_remaining, 0);
+        cfs_spin_unlock(&new->set_new_req_lock);
+        if (count == i) {
+                cfs_waitq_signal(&new->set_waitq);
+
+                /* XXX: It maybe unnecessary to wakeup all the partners. But to
+                 *      guarantee the async RPC can be processed ASAP, we have
+                 *      no other better choice. It maybe fixed in future. */
+                for (i = 0; i < pc->pc_npartners; i++)
+                        cfs_waitq_signal(&pc->pc_partners[i]->pc_set->set_waitq);
+        }
+#endif
 }
 EXPORT_SYMBOL(ptlrpcd_add_rqset);
 
+#ifdef __KERNEL__
+/**
+ * Return transferred RPCs count.
+ */
+static int ptlrpcd_steal_rqset(struct ptlrpc_request_set *des,
+                               struct ptlrpc_request_set *src)
+{
+        cfs_list_t *tmp, *pos;
+        struct ptlrpc_request *req;
+        int rc = 0;
+
+        cfs_spin_lock(&src->set_new_req_lock);
+        if (likely(!cfs_list_empty(&src->set_new_requests))) {
+                cfs_list_for_each_safe(pos, tmp, &src->set_new_requests) {
+                        req = cfs_list_entry(pos, struct ptlrpc_request,
+                                             rq_set_chain);
+                        req->rq_set = des;
+                }
+                cfs_list_splice_init(&src->set_new_requests,
+                                     &des->set_requests);
+                rc = cfs_atomic_read(&src->set_new_count);
+                cfs_atomic_add(rc, &des->set_remaining);
+                cfs_atomic_set(&src->set_new_count, 0);
+        }
+        cfs_spin_unlock(&src->set_new_req_lock);
+        return rc;
+}
+#endif
+
 /**
  * Requests that are added to the ptlrpcd queue are sent via
  * ptlrpcd_check->ptlrpc_check_set().
  */
-int ptlrpcd_add_req(struct ptlrpc_request *req, enum ptlrpcd_scope scope)
+void ptlrpcd_add_req(struct ptlrpc_request *req, pdl_policy_t policy, int idx)
 {
         struct ptlrpcd_ctl *pc;
-        enum pscope_thread  pt;
-        int rc;
 
-        LASSERT(scope < PSCOPE_NR);
-        
         cfs_spin_lock(&req->rq_lock);
         if (req->rq_invalid_rqset) {
-                cfs_duration_t timeout;
-                struct l_wait_info lwi;
+                struct l_wait_info lwi = LWI_TIMEOUT(cfs_time_seconds(5),
+                                                     back_to_sleep, NULL);
 
                 req->rq_invalid_rqset = 0;
                 cfs_spin_unlock(&req->rq_lock);
-
-                timeout = cfs_time_seconds(5);
-                lwi = LWI_TIMEOUT(timeout, back_to_sleep, NULL);
                 l_wait_event(req->rq_set_waitq, (req->rq_set == NULL), &lwi);
         } else if (req->rq_set) {
+                /* If we have a vaid "rq_set", just reuse it to avoid double
+                 * linked. */
                 LASSERT(req->rq_phase == RQ_PHASE_NEW);
                 LASSERT(req->rq_send_state == LUSTRE_IMP_REPLAY);
 
                 /* ptlrpc_check_set will decrease the count */
                 cfs_atomic_inc(&req->rq_set->set_remaining);
                 cfs_spin_unlock(&req->rq_lock);
-
                 cfs_waitq_signal(&req->rq_set->set_waitq);
+                return;
         } else {
                 cfs_spin_unlock(&req->rq_lock);
         }
 
-        pt = req->rq_send_state == LUSTRE_IMP_FULL ? PT_NORMAL : PT_RECOVERY;
-        pc = &ptlrpcd_scopes[scope].pscope_thread[pt].pt_ctl;
-        rc = ptlrpc_set_add_new_req(pc, req);
-        /*
-         * XXX disable this for CLIO: environment is needed for interpreter.
-         *     add debug temporary to check rc.
-         */
-        LASSERTF(rc == 0, "ptlrpcd_add_req failed (rc = %d)\n", rc);
-        if (rc && 0) {
-                /*
-                 * Thread is probably in stop now so we need to
-                 * kill this rpc as it was not added. Let's call
-                 * interpret for it to let know we're killing it
-                 * so that higher levels might free associated
-                 * resources.
-                 */
-                ptlrpc_req_interpret(NULL, req, -EBADR);
-                req->rq_set = NULL;
-                ptlrpc_req_finished(req);
-        } else if (req->rq_send_state == LUSTRE_IMP_CONNECTING) {
-                /*
-                 * The request is for recovery, should be sent ASAP.
-                 */
-                cfs_waitq_signal(&pc->pc_set->set_waitq);
-        }
+        pc = ptlrpcd_select_pc(req, policy, idx);
 
-        return rc;
+        DEBUG_REQ(D_INFO, req, "add req [%p] to pc [%s:%d]",
+                  req, pc->pc_name, pc->pc_index);
+
+        ptlrpc_set_add_new_req(pc, req);
+}
+
+static inline void ptlrpc_reqset_get(struct ptlrpc_request_set *set)
+{
+        cfs_atomic_inc(&set->set_refcount);
 }
 
 /**
  * Check if there is more work to do on ptlrpcd set.
  * Returns 1 if yes.
  */
-static int ptlrpcd_check(const struct lu_env *env, struct ptlrpcd_ctl *pc)
+static int ptlrpcd_check(struct lu_env *env, struct ptlrpcd_ctl *pc)
 {
         cfs_list_t *tmp, *pos;
         struct ptlrpc_request *req;
+        struct ptlrpc_request_set *set = pc->pc_set;
         int rc = 0;
+        int rc2;
         ENTRY;
 
-        cfs_spin_lock(&pc->pc_set->set_new_req_lock);
-        cfs_list_for_each_safe(pos, tmp, &pc->pc_set->set_new_requests) {
-                req = cfs_list_entry(pos, struct ptlrpc_request, rq_set_chain);
-                cfs_list_del_init(&req->rq_set_chain);
-                ptlrpc_set_add_req(pc->pc_set, req);
+        if (cfs_atomic_read(&set->set_new_count)) {
+                cfs_spin_lock(&set->set_new_req_lock);
+                if (likely(!cfs_list_empty(&set->set_new_requests))) {
+                        cfs_list_splice_init(&set->set_new_requests,
+                                             &set->set_requests);
+                        cfs_atomic_add(cfs_atomic_read(&set->set_new_count),
+                                       &set->set_remaining);
+                        cfs_atomic_set(&set->set_new_count, 0);
+                        /*
+                         * Need to calculate its timeout.
+                         */
+                        rc = 1;
+                }
+                cfs_spin_unlock(&set->set_new_req_lock);
+        }
+
+        /* We should call lu_env_refill() before handling new requests to make
+         * sure that env key the requests depending on really exists.
+         */
+        rc2 = lu_env_refill(env);
+        if (rc2 != 0) {
                 /*
-                 * Need to calculate its timeout.
+                 * XXX This is very awkward situation, because
+                 * execution can neither continue (request
+                 * interpreters assume that env is set up), nor repeat
+                 * the loop (as this potentially results in a tight
+                 * loop of -ENOMEM's).
+                 *
+                 * Fortunately, refill only ever does something when
+                 * new modules are loaded, i.e., early during boot up.
                  */
-                rc = 1;
+                CERROR("Failure to refill session: %d\n", rc2);
+                RETURN(rc);
         }
-        cfs_spin_unlock(&pc->pc_set->set_new_req_lock);
 
-        if (cfs_atomic_read(&pc->pc_set->set_remaining)) {
-                rc = rc | ptlrpc_check_set(env, pc->pc_set);
+        if (cfs_atomic_read(&set->set_remaining))
+                rc |= ptlrpc_check_set(env, set);
 
+        if (!cfs_list_empty(&set->set_requests)) {
                 /*
                  * XXX: our set never completes, so we prune the completed
                  * reqs after each iteration. boy could this be smarter.
                  */
-                cfs_list_for_each_safe(pos, tmp, &pc->pc_set->set_requests) {
+                cfs_list_for_each_safe(pos, tmp, &set->set_requests) {
                         req = cfs_list_entry(pos, struct ptlrpc_request,
-                                         rq_set_chain);
+                                             rq_set_chain);
                         if (req->rq_phase != RQ_PHASE_COMPLETE)
                                 continue;
 
                         cfs_list_del_init(&req->rq_set_chain);
                         req->rq_set = NULL;
-                        ptlrpc_req_finished (req);
+                        ptlrpc_req_finished(req);
                 }
         }
 
@@ -253,9 +348,45 @@ static int ptlrpcd_check(const struct lu_env *env, struct ptlrpcd_ctl *pc)
                 /*
                  * If new requests have been added, make sure to wake up.
                  */
-                cfs_spin_lock(&pc->pc_set->set_new_req_lock);
-                rc = !cfs_list_empty(&pc->pc_set->set_new_requests);
-                cfs_spin_unlock(&pc->pc_set->set_new_req_lock);
+                rc = cfs_atomic_read(&set->set_new_count);
+
+#ifdef __KERNEL__
+                /* If we have nothing to do, check whether we can take some
+                 * work from our partner threads. */
+                if (rc == 0 && pc->pc_npartners > 0) {
+                        struct ptlrpcd_ctl *partner;
+                        struct ptlrpc_request_set *ps;
+                        int first = pc->pc_cursor;
+
+                        do {
+                                partner = pc->pc_partners[pc->pc_cursor++];
+                                if (pc->pc_cursor >= pc->pc_npartners)
+                                        pc->pc_cursor = 0;
+                                if (partner == NULL)
+                                        continue;
+
+                                cfs_spin_lock(&partner->pc_lock);
+                                ps = partner->pc_set;
+                                if (ps == NULL) {
+                                        cfs_spin_unlock(&partner->pc_lock);
+                                        continue;
+                                }
+
+                                ptlrpc_reqset_get(ps);
+                                cfs_spin_unlock(&partner->pc_lock);
+
+                                if (cfs_atomic_read(&ps->set_new_count)) {
+                                        rc = ptlrpcd_steal_rqset(set, ps);
+                                        if (rc > 0)
+                                                CDEBUG(D_RPCTRACE, "transfer %d"
+                                                       " async RPCs [%d->%d]\n",
+                                                        rc, partner->pc_index,
+                                                        pc->pc_index);
+                                }
+                                ptlrpc_reqset_put(ps);
+                        } while (rc == 0 && pc->pc_cursor != first);
+                }
+#endif
         }
 
         RETURN(rc);
@@ -271,26 +402,37 @@ static int ptlrpcd_check(const struct lu_env *env, struct ptlrpcd_ctl *pc)
 static int ptlrpcd(void *arg)
 {
         struct ptlrpcd_ctl *pc = arg;
+        struct ptlrpc_request_set *set = pc->pc_set;
         struct lu_env env = { .le_ses = NULL };
         int rc, exit = 0;
         ENTRY;
 
-        rc = cfs_daemonize_ctxt(pc->pc_name);
-        if (rc == 0) {
-                /*
-                 * XXX So far only "client" ptlrpcd uses an environment. In
-                 * the future, ptlrpcd thread (or a thread-set) has to given
-                 * an argument, describing its "scope".
-                 */
-                rc = lu_context_init(&env.le_ctx,
-                                     LCT_CL_THREAD|LCT_REMEMBER|LCT_NOREF);
-        }
+        cfs_daemonize_ctxt(pc->pc_name);
+#if defined(CONFIG_SMP) && defined(HAVE_NODE_TO_CPUMASK)
+        if (cfs_test_bit(LIOD_BIND, &pc->pc_flags)) {
+                int index = pc->pc_index;
 
+                if (index >= 0 && index < cfs_num_possible_cpus()) {
+                        while (!cpu_online(index)) {
+                                if (++index >= cfs_num_possible_cpus())
+                                        index = 0;
+                        }
+                        cfs_set_cpus_allowed(cfs_current(),
+                                     node_to_cpumask(cpu_to_node(index)));
+                }
+        }
+#endif
+        /*
+         * XXX So far only "client" ptlrpcd uses an environment. In
+         * the future, ptlrpcd thread (or a thread-set) has to given
+         * an argument, describing its "scope".
+         */
+        rc = lu_context_init(&env.le_ctx,
+                             LCT_CL_THREAD|LCT_REMEMBER|LCT_NOREF);
         cfs_complete(&pc->pc_starting);
 
         if (rc != 0)
                 RETURN(rc);
-        env.le_ctx.lc_cookie = 0x7;
 
         /*
          * This mainloop strongly resembles ptlrpc_set_wait() except that our
@@ -302,28 +444,12 @@ static int ptlrpcd(void *arg)
                 struct l_wait_info lwi;
                 int timeout;
 
-                rc = lu_env_refill(&env);
-                if (rc != 0) {
-                        /*
-                         * XXX This is very awkward situation, because
-                         * execution can neither continue (request
-                         * interpreters assume that env is set up), nor repeat
-                         * the loop (as this potentially results in a tight
-                         * loop of -ENOMEM's).
-                         *
-                         * Fortunately, refill only ever does something when
-                         * new modules are loaded, i.e., early during boot up.
-                         */
-                        CERROR("Failure to refill session: %d\n", rc);
-                        continue;
-                }
-
-                timeout = ptlrpc_set_next_timeout(pc->pc_set);
+                timeout = ptlrpc_set_next_timeout(set);
                 lwi = LWI_TIMEOUT(cfs_time_seconds(timeout ? timeout : 1),
-                                  ptlrpc_expired_set, pc->pc_set);
+                                  ptlrpc_expired_set, set);
 
                 lu_context_enter(&env.le_ctx);
-                l_wait_event(pc->pc_set->set_waitq,
+                l_wait_event(set->set_waitq,
                              ptlrpcd_check(&env, pc), &lwi);
                 lu_context_exit(&env.le_ctx);
 
@@ -332,7 +458,7 @@ static int ptlrpcd(void *arg)
                  */
                 if (cfs_test_bit(LIOD_STOP, &pc->pc_flags)) {
                         if (cfs_test_bit(LIOD_FORCE, &pc->pc_flags))
-                                ptlrpc_abort_set(pc->pc_set);
+                                ptlrpc_abort_set(set);
                         exit++;
                 }
 
@@ -345,17 +471,155 @@ static int ptlrpcd(void *arg)
         /*
          * Wait for inflight requests to drain.
          */
-        if (!cfs_list_empty(&pc->pc_set->set_requests))
-                ptlrpc_set_wait(pc->pc_set);
+        if (!cfs_list_empty(&set->set_requests))
+                ptlrpc_set_wait(set);
         lu_context_fini(&env.le_ctx);
         cfs_complete(&pc->pc_finishing);
 
         cfs_clear_bit(LIOD_START, &pc->pc_flags);
         cfs_clear_bit(LIOD_STOP, &pc->pc_flags);
         cfs_clear_bit(LIOD_FORCE, &pc->pc_flags);
+        cfs_clear_bit(LIOD_BIND, &pc->pc_flags);
         return 0;
 }
 
+/* XXX: We want multiple CPU cores to share the async RPC load. So we start many
+ *      ptlrpcd threads. We also want to reduce the ptlrpcd overhead caused by
+ *      data transfer cross-CPU cores. So we bind ptlrpcd thread to specified
+ *      CPU core. But binding all ptlrpcd threads maybe cause response delay
+ *      because of some CPU core(s) busy with other loads.
+ *
+ *      For example: "ls -l", some async RPCs for statahead are assigned to
+ *      ptlrpcd_0, and ptlrpcd_0 is bound to CPU_0, but CPU_0 may be quite busy
+ *      with other non-ptlrpcd, like "ls -l" itself (we want to the "ls -l"
+ *      thread, statahead thread, and ptlrpcd thread can run in parallel), under
+ *      such case, the statahead async RPCs can not be processed in time, it is
+ *      unexpected. If ptlrpcd_0 can be re-scheduled on other CPU core, it may
+ *      be better. But it breaks former data transfer policy.
+ *
+ *      So we shouldn't be blind for avoiding the data transfer. We make some
+ *      compromise: divide the ptlrpcd threds pool into two parts. One part is
+ *      for bound mode, each ptlrpcd thread in this part is bound to some CPU
+ *      core. The other part is for free mode, all the ptlrpcd threads in the
+ *      part can be scheduled on any CPU core. We specify some partnership
+ *      between bound mode ptlrpcd thread(s) and free mode ptlrpcd thread(s),
+ *      and the async RPC load within the partners are shared.
+ *
+ *      It can partly avoid data transfer cross-CPU (if the bound mode ptlrpcd
+ *      thread can be scheduled in time), and try to guarantee the async RPC
+ *      processed ASAP (as long as the free mode ptlrpcd thread can be scheduled
+ *      on any CPU core).
+ *
+ *      As for how to specify the partnership between bound mode ptlrpcd
+ *      thread(s) and free mode ptlrpcd thread(s), the simplest way is to use
+ *      <free bound> pair. In future, we can specify some more complex
+ *      partnership based on the patches for CPU partition. But before such
+ *      patches are available, we prefer to use the simplest one.
+ */
+# ifdef CFS_CPU_MODE_NUMA
+# warning "fix ptlrpcd_bind() to use new CPU partition APIs"
+# endif
+static int ptlrpcd_bind(int index, int max)
+{
+        struct ptlrpcd_ctl *pc;
+        int rc = 0;
+#if defined(CONFIG_NUMA) && defined(HAVE_NODE_TO_CPUMASK)
+        struct ptlrpcd_ctl *ppc;
+        int node, i, pidx;
+        cpumask_t mask;
+#endif
+        ENTRY;
+
+        LASSERT(index <= max - 1);
+        pc = &ptlrpcds->pd_threads[index];
+        switch (ptlrpcd_bind_policy) {
+        case PDB_POLICY_NONE:
+                pc->pc_npartners = -1;
+                break;
+        case PDB_POLICY_FULL:
+                pc->pc_npartners = 0;
+                cfs_set_bit(LIOD_BIND, &pc->pc_flags);
+                break;
+        case PDB_POLICY_PAIR:
+                LASSERT(max % 2 == 0);
+                pc->pc_npartners = 1;
+                break;
+        case PDB_POLICY_NEIGHBOR:
+#if defined(CONFIG_NUMA) && defined(HAVE_NODE_TO_CPUMASK)
+                node = cpu_to_node(index);
+                mask = node_to_cpumask(node);
+                for (i = max; i < cfs_num_online_cpus(); i++)
+                        cpu_clear(i, mask);
+                pc->pc_npartners = cpus_weight(mask) - 1;
+                cfs_set_bit(LIOD_BIND, &pc->pc_flags);
+#else
+                LASSERT(max >= 3);
+                pc->pc_npartners = 2;
+#endif
+                break;
+        default:
+                CERROR("unknown ptlrpcd bind policy %d\n", ptlrpcd_bind_policy);
+                rc = -EINVAL;
+        }
+
+        if (rc == 0 && pc->pc_npartners > 0) {
+                OBD_ALLOC(pc->pc_partners,
+                          sizeof(struct ptlrpcd_ctl *) * pc->pc_npartners);
+                if (pc->pc_partners == NULL) {
+                        pc->pc_npartners = 0;
+                        rc = -ENOMEM;
+                } else {
+                        switch (ptlrpcd_bind_policy) {
+                        case PDB_POLICY_PAIR:
+                                if (index & 0x1) {
+                                        cfs_set_bit(LIOD_BIND, &pc->pc_flags);
+                                        pc->pc_partners[0] = &ptlrpcds->
+                                                pd_threads[index - 1];
+                                        ptlrpcds->pd_threads[index - 1].
+                                                pc_partners[0] = pc;
+                                }
+                                break;
+                        case PDB_POLICY_NEIGHBOR:
+#if defined(CONFIG_NUMA) && defined(HAVE_NODE_TO_CPUMASK)
+                                /* partners are cores in the same NUMA node.
+                                 * setup partnership only with ptlrpcd threads
+                                 * that are already initialized
+                                 */
+                                for (pidx = 0, i = 0; i < index; i++) {
+                                        if (cpu_isset(i, mask)) {
+                                                ppc = &ptlrpcds->pd_threads[i];
+                                                pc->pc_partners[pidx++] = ppc;
+                                                ppc->pc_partners[ppc->
+                                                          pc_npartners++] = pc;
+                                        }
+                                }
+                                /* adjust number of partners to the number
+                                 * of partnership really setup */
+                                pc->pc_npartners = pidx;
+#else
+                                if (index & 0x1)
+                                        cfs_set_bit(LIOD_BIND, &pc->pc_flags);
+                                if (index > 0) {
+                                        pc->pc_partners[0] = &ptlrpcds->
+                                                pd_threads[index - 1];
+                                        ptlrpcds->pd_threads[index - 1].
+                                                pc_partners[1] = pc;
+                                        if (index == max - 1) {
+                                                pc->pc_partners[1] =
+                                                &ptlrpcds->pd_threads[0];
+                                                ptlrpcds->pd_threads[0].
+                                                pc_partners[0] = pc;
+                                        }
+                                }
+#endif
+                                break;
+                        }
+                }
+        }
+
+        RETURN(rc);
+}
+
 #else /* !__KERNEL__ */
 
 /**
@@ -378,7 +642,6 @@ int ptlrpcd_check_async_rpcs(void *arg)
                 if (rc == 0) {
                         lu_context_enter(&pc->pc_env.le_ctx);
                         rc = ptlrpcd_check(&pc->pc_env, pc);
-                        lu_context_exit(&pc->pc_env.le_ctx);
                         if (!rc)
                                 ptlrpc_expired_set(pc->pc_set);
                         /*
@@ -386,6 +649,7 @@ int ptlrpcd_check_async_rpcs(void *arg)
                          */
                         if (cfs_test_bit(LIOD_RECOVERY, &pc->pc_flags))
                                 rc = ptlrpcd_check(&pc->pc_env, pc);
+                        lu_context_exit(&pc->pc_env.le_ctx);
                 }
         }
 
@@ -397,26 +661,28 @@ int ptlrpcd_idle(void *arg)
 {
         struct ptlrpcd_ctl *pc = arg;
 
-        return (cfs_list_empty(&pc->pc_set->set_new_requests) &&
+        return (cfs_atomic_read(&pc->pc_set->set_new_count) == 0 &&
                 cfs_atomic_read(&pc->pc_set->set_remaining) == 0);
 }
 
 #endif
 
-int ptlrpcd_start(const char *name, struct ptlrpcd_ctl *pc)
+int ptlrpcd_start(int index, int max, const char *name, struct ptlrpcd_ctl *pc)
 {
         int rc;
+        int env = 0;
         ENTRY;
 
         /*
          * Do not allow start second thread for one pc.
          */
         if (cfs_test_and_set_bit(LIOD_START, &pc->pc_flags)) {
-                CERROR("Starting second thread (%s) for same pc %p\n",
+                CWARN("Starting second thread (%s) for same pc %p\n",
                        name, pc);
-                RETURN(-EALREADY);
+                RETURN(0);
         }
 
+        pc->pc_index = index;
         cfs_init_completion(&pc->pc_starting);
         cfs_init_completion(&pc->pc_finishing);
         cfs_spin_lock_init(&pc->pc_lock);
@@ -430,18 +696,21 @@ int ptlrpcd_start(const char *name, struct ptlrpcd_ctl *pc)
          * describing its "scope".
          */
         rc = lu_context_init(&pc->pc_env.le_ctx, LCT_CL_THREAD|LCT_REMEMBER);
-        if (rc != 0) {
-                ptlrpc_set_destroy(pc->pc_set);
+        if (rc != 0)
                 GOTO(out, rc);
-        }
 
+        env = 1;
 #ifdef __KERNEL__
+        if (index >= 0) {
+                rc = ptlrpcd_bind(index, max);
+                if (rc < 0)
+                        GOTO(out, rc);
+        }
+
         rc = cfs_create_thread(ptlrpcd, pc, 0);
-        if (rc < 0)  {
-                lu_context_fini(&pc->pc_env.le_ctx);
-                ptlrpc_set_destroy(pc->pc_set);
+        if (rc < 0)
                 GOTO(out, rc);
-        }
+
         rc = 0;
         cfs_wait_for_completion(&pc->pc_starting);
 #else
@@ -453,16 +722,35 @@ int ptlrpcd_start(const char *name, struct ptlrpcd_ctl *pc)
                                                  &ptlrpcd_idle, pc);
 #endif
 out:
-        if (rc)
+        if (rc) {
+#ifdef __KERNEL__
+                if (pc->pc_set != NULL) {
+                        struct ptlrpc_request_set *set = pc->pc_set;
+
+                        cfs_spin_lock(&pc->pc_lock);
+                        pc->pc_set = NULL;
+                        cfs_spin_unlock(&pc->pc_lock);
+                        ptlrpc_set_destroy(set);
+                }
+                if (env != 0)
+                        lu_context_fini(&pc->pc_env.le_ctx);
+                cfs_clear_bit(LIOD_BIND, &pc->pc_flags);
+#else
+                SET_BUT_UNUSED(env);
+#endif
                 cfs_clear_bit(LIOD_START, &pc->pc_flags);
+        }
         RETURN(rc);
 }
 
 void ptlrpcd_stop(struct ptlrpcd_ctl *pc, int force)
 {
+       struct ptlrpc_request_set *set = pc->pc_set;
+        ENTRY;
+
         if (!cfs_test_bit(LIOD_START, &pc->pc_flags)) {
-                CERROR("Thread for pc %p was not started\n", pc);
-                return;
+                CWARN("Thread for pc %p was not started\n", pc);
+                goto out;
         }
 
         cfs_set_bit(LIOD_STOP, &pc->pc_flags);
@@ -476,64 +764,125 @@ void ptlrpcd_stop(struct ptlrpcd_ctl *pc, int force)
         liblustre_deregister_idle_callback(pc->pc_idle_callback);
 #endif
         lu_context_fini(&pc->pc_env.le_ctx);
-        ptlrpc_set_destroy(pc->pc_set);
+
+        cfs_spin_lock(&pc->pc_lock);
+        pc->pc_set = NULL;
+        cfs_spin_unlock(&pc->pc_lock);
+        ptlrpc_set_destroy(set);
+
+out:
+#ifdef __KERNEL__
+        if (pc->pc_npartners > 0) {
+                LASSERT(pc->pc_partners != NULL);
+
+                OBD_FREE(pc->pc_partners,
+                         sizeof(struct ptlrpcd_ctl *) * pc->pc_npartners);
+                pc->pc_partners = NULL;
+        }
+        pc->pc_npartners = 0;
+#endif
+        EXIT;
 }
 
-void ptlrpcd_fini(void)
+static void ptlrpcd_fini(void)
 {
         int i;
-        int j;
+        ENTRY;
+
+        if (ptlrpcds != NULL) {
+                for (i = 0; i < ptlrpcds->pd_nthreads; i++)
+                        ptlrpcd_stop(&ptlrpcds->pd_threads[i], 0);
+                ptlrpcd_stop(&ptlrpcds->pd_thread_rcv, 0);
+                OBD_FREE(ptlrpcds, ptlrpcds->pd_size);
+                ptlrpcds = NULL;
+        }
 
+        EXIT;
+}
+
+static int ptlrpcd_init(void)
+{
+        int nthreads = cfs_num_online_cpus();
+        char name[16];
+        int size, i = -1, j, rc = 0;
         ENTRY;
 
-        for (i = 0; i < PSCOPE_NR; ++i) {
-                for (j = 0; j < PT_NR; ++j) {
-                        struct ptlrpcd_ctl *pc;
+#ifdef __KERNEL__
+        if (max_ptlrpcds > 0 && max_ptlrpcds < nthreads)
+                nthreads = max_ptlrpcds;
+        if (nthreads < 2)
+                nthreads = 2;
+        if (nthreads < 3 && ptlrpcd_bind_policy == PDB_POLICY_NEIGHBOR)
+                ptlrpcd_bind_policy = PDB_POLICY_PAIR;
+        else if (nthreads % 2 != 0 && ptlrpcd_bind_policy == PDB_POLICY_PAIR)
+                nthreads &= ~1; /* make sure it is even */
+#else
+        nthreads = 1;
+#endif
+
+        size = offsetof(struct ptlrpcd, pd_threads[nthreads]);
+        OBD_ALLOC(ptlrpcds, size);
+        if (ptlrpcds == NULL)
+                GOTO(out, rc = -ENOMEM);
 
-                        pc = &ptlrpcd_scopes[i].pscope_thread[j].pt_ctl;
+        snprintf(name, 15, "ptlrpcd_rcv");
+        cfs_set_bit(LIOD_RECOVERY, &ptlrpcds->pd_thread_rcv.pc_flags);
+        rc = ptlrpcd_start(-1, nthreads, name, &ptlrpcds->pd_thread_rcv);
+        if (rc < 0)
+                GOTO(out, rc);
 
-                        if (cfs_test_bit(LIOD_START, &pc->pc_flags))
-                                ptlrpcd_stop(pc, 0);
-                }
+        /* XXX: We start nthreads ptlrpc daemons. Each of them can process any
+         *      non-recovery async RPC to improve overall async RPC efficiency.
+         *
+         *      But there are some issues with async I/O RPCs and async non-I/O
+         *      RPCs processed in the same set under some cases. The ptlrpcd may
+         *      be blocked by some async I/O RPC(s), then will cause other async
+         *      non-I/O RPC(s) can not be processed in time.
+         *
+         *      Maybe we should distinguish blocked async RPCs from non-blocked
+         *      async RPCs, and process them in different ptlrpcd sets to avoid
+         *      unnecessary dependency. But how to distribute async RPCs load
+         *      among all the ptlrpc daemons becomes another trouble. */
+        for (i = 0; i < nthreads; i++) {
+                snprintf(name, 15, "ptlrpcd_%d", i);
+                rc = ptlrpcd_start(i, nthreads, name, &ptlrpcds->pd_threads[i]);
+                if (rc < 0)
+                        GOTO(out, rc);
         }
-        EXIT;
+
+        ptlrpcds->pd_size = size;
+        ptlrpcds->pd_index = 0;
+        ptlrpcds->pd_nthreads = nthreads;
+
+out:
+        if (rc != 0 && ptlrpcds != NULL) {
+                for (j = 0; j <= i; j++)
+                        ptlrpcd_stop(&ptlrpcds->pd_threads[j], 0);
+                ptlrpcd_stop(&ptlrpcds->pd_thread_rcv, 0);
+                OBD_FREE(ptlrpcds, size);
+                ptlrpcds = NULL;
+        }
+
+        RETURN(0);
 }
 
 int ptlrpcd_addref(void)
 {
         int rc = 0;
-        int i;
-        int j;
         ENTRY;
 
-        cfs_mutex_down(&ptlrpcd_sem);
-        if (++ptlrpcd_users == 1) {
-                for (i = 0; rc == 0 && i < PSCOPE_NR; ++i) {
-                        for (j = 0; rc == 0 && j < PT_NR; ++j) {
-                                struct ptlrpcd_thread *pt;
-                                struct ptlrpcd_ctl    *pc;
-
-                                pt = &ptlrpcd_scopes[i].pscope_thread[j];
-                                pc = &pt->pt_ctl;
-                                if (j == PT_RECOVERY)
-                                        cfs_set_bit(LIOD_RECOVERY, &pc->pc_flags);
-                                rc = ptlrpcd_start(pt->pt_name, pc);
-                        }
-                }
-                if (rc != 0) {
-                        --ptlrpcd_users;
-                        ptlrpcd_fini();
-                }
-        }
-        cfs_mutex_up(&ptlrpcd_sem);
+        cfs_mutex_lock(&ptlrpcd_mutex);
+        if (++ptlrpcd_users == 1)
+                rc = ptlrpcd_init();
+        cfs_mutex_unlock(&ptlrpcd_mutex);
         RETURN(rc);
 }
 
 void ptlrpcd_decref(void)
 {
-        cfs_mutex_down(&ptlrpcd_sem);
+        cfs_mutex_lock(&ptlrpcd_mutex);
         if (--ptlrpcd_users == 0)
                 ptlrpcd_fini();
-        cfs_mutex_up(&ptlrpcd_sem);
+        cfs_mutex_unlock(&ptlrpcd_mutex);
 }
 /** @} ptlrpcd */