Whamcloud - gitweb
LU-9683 ptlrpc: fix argument misorder
[fs/lustre-release.git] / lustre / ptlrpc / service.c
index 4b57b9e..ecde6ab 100644 (file)
@@ -176,24 +176,21 @@ ptlrpc_grow_req_bufs(struct ptlrpc_service_part *svcpt, int post)
  * Puts a lock and its mode into reply state assotiated to request reply.
  */
 void
-ptlrpc_save_lock(struct ptlrpc_request *req,
-                 struct lustre_handle *lock, int mode, int no_ack)
+ptlrpc_save_lock(struct ptlrpc_request *req, struct lustre_handle *lock,
+                int mode, bool no_ack, bool convert_lock)
 {
-        struct ptlrpc_reply_state *rs = req->rq_reply_state;
-        int                        idx;
+       struct ptlrpc_reply_state *rs = req->rq_reply_state;
+       int idx;
 
-        LASSERT(rs != NULL);
-        LASSERT(rs->rs_nlocks < RS_MAX_LOCKS);
+       LASSERT(rs != NULL);
+       LASSERT(rs->rs_nlocks < RS_MAX_LOCKS);
 
-        if (req->rq_export->exp_disconnected) {
-                ldlm_lock_decref(lock, mode);
-        } else {
-                idx = rs->rs_nlocks++;
-                rs->rs_locks[idx] = *lock;
-                rs->rs_modes[idx] = mode;
-                rs->rs_difficult = 1;
-                rs->rs_no_ack = !!no_ack;
-        }
+       idx = rs->rs_nlocks++;
+       rs->rs_locks[idx] = *lock;
+       rs->rs_modes[idx] = mode;
+       rs->rs_difficult = 1;
+       rs->rs_no_ack = no_ack;
+       rs->rs_convert_lock = convert_lock;
 }
 EXPORT_SYMBOL(ptlrpc_save_lock);
 
@@ -204,7 +201,7 @@ struct ptlrpc_hr_thread {
        int                             hrt_id;         /* thread ID */
        spinlock_t                      hrt_lock;
        wait_queue_head_t               hrt_waitq;
-       struct list_head                        hrt_queue;      /* RS queue */
+       struct list_head                hrt_queue;
        struct ptlrpc_hr_partition      *hrt_partition;
 };
 
@@ -559,14 +556,13 @@ ptlrpc_server_nthreads_check(struct ptlrpc_service *svc,
                 * have too many threads no matter how many cores/HTs
                 * there are.
                 */
-               if (cfs_cpu_ht_nsiblings(smp_processor_id()) > 1) {
+               if (cpumask_weight(topology_sibling_cpumask(smp_processor_id())) > 1) {
                        /* weight is # of HTs */
                        /* depress thread factor for hyper-thread */
                        factor = factor - (factor >> 1) + (factor >> 3);
                }
 
                weight = cfs_cpt_weight(svc->srv_cptable, 0);
-               LASSERT(weight > 0);
 
                for (; factor > 0 && weight > 0; factor--, weight -= fade)
                        nthrs += min(weight, fade) * factor;
@@ -687,6 +683,7 @@ ptlrpc_service_part_init(struct ptlrpc_service *svc,
  */
 struct ptlrpc_service *
 ptlrpc_register_service(struct ptlrpc_service_conf *conf,
+                       struct kset *parent,
                        struct proc_dir_entry *proc_entry)
 {
        struct ptlrpc_service_cpt_conf  *cconf = &conf->psc_cpt;
@@ -804,6 +801,12 @@ ptlrpc_register_service(struct ptlrpc_service_conf *conf,
        list_add(&service->srv_list, &ptlrpc_all_services);
        mutex_unlock(&ptlrpc_all_services_mutex);
 
+       if (parent) {
+               rc = ptlrpc_sysfs_register_service(parent, service);
+               if (rc)
+                       GOTO(failed, rc);
+       }
+
        if (proc_entry != NULL)
                ptlrpc_lprocfs_register_service(proc_entry, service);
 
@@ -938,10 +941,22 @@ void ptlrpc_server_drop_request(struct ptlrpc_request *req)
                        spin_lock(&svcpt->scp_lock);
                        /*
                         * now all reqs including the embedded req has been
-                        * disposed, schedule request buffer for re-use.
+                        * disposed, schedule request buffer for re-use
+                        * or free it to drain some in excess.
                         */
                        LASSERT(atomic_read(&rqbd->rqbd_req.rq_refcount) == 0);
-                       list_add_tail(&rqbd->rqbd_list, &svcpt->scp_rqbd_idle);
+                       if (svcpt->scp_nrqbds_posted >=
+                           svc->srv_nbuf_per_group &&
+                           !test_req_buffer_pressure) {
+                               /* like in ptlrpc_free_rqbd() */
+                               svcpt->scp_nrqbds_total--;
+                               OBD_FREE_LARGE(rqbd->rqbd_buffer,
+                                              svc->srv_buf_size);
+                               OBD_FREE_PTR(rqbd);
+                       } else {
+                               list_add_tail(&rqbd->rqbd_list,
+                                             &svcpt->scp_rqbd_idle);
+                       }
                }
 
                spin_unlock(&svcpt->scp_lock);
@@ -1091,7 +1106,7 @@ void ptlrpc_update_export_timer(struct obd_export *exp, long extra_delay)
                          * we better wait for 3. */
                         exp->exp_obd->obd_eviction_timer =
                                 cfs_time_current_sec() + 3 * PING_INTERVAL;
-                        CDEBUG(D_HA, "%s: Think about evicting %s from "CFS_TIME_T"\n",
+                       CDEBUG(D_HA, "%s: Think about evicting %s from %ld\n",
                                exp->exp_obd->obd_name,
                                obd_export_nid2str(oldest_exp), oldest_time);
                 }
@@ -1166,7 +1181,7 @@ static void ptlrpc_at_set_timer(struct ptlrpc_service_part *svcpt)
        }
 
        /* Set timer for closest deadline */
-       next = (__s32)(array->paa_deadline - cfs_time_current_sec() -
+       next = (__s32)(array->paa_deadline - ktime_get_real_seconds() -
                       at_early_margin);
        if (next <= 0) {
                ptlrpc_at_timer((unsigned long)svcpt);
@@ -1197,7 +1212,7 @@ static int ptlrpc_at_add_timed(struct ptlrpc_request *req)
        spin_lock(&svcpt->scp_at_lock);
        LASSERT(list_empty(&req->rq_timed_list));
 
-        index = (unsigned long)req->rq_deadline % array->paa_size;
+       div_u64_rem(req->rq_deadline, array->paa_size, &index);
         if (array->paa_reqs_count[index] > 0) {
                 /* latest rpcs will have the latest deadlines in the list,
                  * so search backward. */
@@ -1260,8 +1275,8 @@ static int ptlrpc_at_send_early_reply(struct ptlrpc_request *req)
        struct ptlrpc_service_part *svcpt = req->rq_rqbd->rqbd_svcpt;
        struct ptlrpc_request *reqcopy;
        struct lustre_msg *reqmsg;
-       cfs_duration_t olddl = req->rq_deadline - cfs_time_current_sec();
-       time_t  newdl;
+       time64_t olddl = req->rq_deadline - ktime_get_real_seconds();
+       time64_t newdl;
        int rc;
 
        ENTRY;
@@ -1274,18 +1289,18 @@ static int ptlrpc_at_send_early_reply(struct ptlrpc_request *req)
         /* deadline is when the client expects us to reply, margin is the
            difference between clients' and servers' expectations */
         DEBUG_REQ(D_ADAPTTO, req,
-                  "%ssending early reply (deadline %+lds, margin %+lds) for "
+                 "%ssending early reply (deadline %+llds, margin %+llds) for "
                   "%d+%d", AT_OFF ? "AT off - not " : "",
-                 olddl, olddl - at_get(&svcpt->scp_at_estimate),
+                 (s64)olddl, (s64)(olddl - at_get(&svcpt->scp_at_estimate)),
                  at_get(&svcpt->scp_at_estimate), at_extra);
 
         if (AT_OFF)
                 RETURN(0);
 
         if (olddl < 0) {
-                DEBUG_REQ(D_WARNING, req, "Already past deadline (%+lds), "
+               DEBUG_REQ(D_WARNING, req, "Already past deadline (%+llds), "
                           "not sending early reply. Consider increasing "
-                          "at_early_margin (%d)?", olddl, at_early_margin);
+                         "at_early_margin (%d)?", (s64)olddl, at_early_margin);
 
                 /* Return an error so we're not re-added to the timed list. */
                 RETURN(-ETIMEDOUT);
@@ -1300,17 +1315,22 @@ static int ptlrpc_at_send_early_reply(struct ptlrpc_request *req)
        if (req->rq_export &&
            lustre_msg_get_flags(req->rq_reqmsg) &
            (MSG_REPLAY | MSG_REQ_REPLAY_DONE | MSG_LOCK_REPLAY_DONE)) {
+               struct obd_device *obd_exp = req->rq_export->exp_obd;
+
                /* During recovery, we don't want to send too many early
                 * replies, but on the other hand we want to make sure the
                 * client has enough time to resend if the rpc is lost. So
                 * during the recovery period send at least 4 early replies,
                 * spacing them every at_extra if we can. at_estimate should
-                * always equal this fixed value during recovery. */
+                * always equal this fixed value during recovery.
+                */
                /* Don't account request processing time into AT history
                 * during recovery, it is not service time we need but
-                * includes also waiting time for recovering clients */
-               newdl = cfs_time_current_sec() + min(at_extra,
-                       req->rq_export->exp_obd->obd_recovery_timeout / 4);
+                * includes also waiting time for recovering clients
+                */
+               newdl = min_t(time64_t, at_extra,
+                             obd_exp->obd_recovery_timeout / 4) +
+                       ktime_get_real_seconds();
        } else {
                /* We want to extend the request deadline by at_extra seconds,
                 * so we set our service estimate to reflect how much time has
@@ -1320,7 +1340,7 @@ static int ptlrpc_at_send_early_reply(struct ptlrpc_request *req)
                 * account for network latency). See ptlrpc_at_recv_early_reply
                 */
                at_measured(&svcpt->scp_at_estimate, at_extra +
-                           cfs_time_current_sec() -
+                           ktime_get_real_seconds() -
                            req->rq_arrival_time.tv_sec);
                newdl = req->rq_arrival_time.tv_sec +
                        at_get(&svcpt->scp_at_estimate);
@@ -1329,9 +1349,8 @@ static int ptlrpc_at_send_early_reply(struct ptlrpc_request *req)
        /* Check to see if we've actually increased the deadline -
         * we may be past adaptive_max */
        if (req->rq_deadline >= newdl) {
-               DEBUG_REQ(D_WARNING, req, "Couldn't add any time "
-                         "(%ld/%ld), not sending early reply\n",
-                         olddl, newdl - cfs_time_current_sec());
+               DEBUG_REQ(D_WARNING, req, "Couldn't add any time (%lld/%lld), not sending early reply\n",
+                         (s64)olddl, (s64)(newdl - ktime_get_real_seconds()));
                RETURN(-ETIMEDOUT);
        }
 
@@ -1418,8 +1437,8 @@ static int ptlrpc_at_check_timed(struct ptlrpc_service_part *svcpt)
         struct ptlrpc_request *rq, *n;
        struct list_head work_list;
         __u32  index, count;
-        time_t deadline;
-        time_t now = cfs_time_current_sec();
+       time64_t deadline;
+       time64_t now = ktime_get_real_seconds();
         cfs_duration_t delay;
         int first, counter = 0;
         ENTRY;
@@ -1450,7 +1469,7 @@ static int ptlrpc_at_check_timed(struct ptlrpc_service_part *svcpt)
           server will take. Send early replies to everyone expiring soon. */
        INIT_LIST_HEAD(&work_list);
        deadline = -1;
-       index = (unsigned long)array->paa_deadline % array->paa_size;
+       div_u64_rem(array->paa_deadline, array->paa_size, &index);
        count = array->paa_count;
        while (count > 0) {
                count -= array->paa_reqs_count[index];
@@ -1493,8 +1512,7 @@ static int ptlrpc_at_check_timed(struct ptlrpc_service_part *svcpt)
                 LCONSOLE_WARN("%s: This server is not able to keep up with "
                              "request traffic (cpu-bound).\n",
                              svcpt->scp_service->srv_name);
-               CWARN("earlyQ=%d reqQ=%d recA=%d, svcEst=%d, "
-                     "delay="CFS_DURATION_T"(jiff)\n",
+               CWARN("earlyQ=%d reqQ=%d recA=%d, svcEst=%d, delay=%ld(jiff)\n",
                      counter, svcpt->scp_nreqs_incoming,
                      svcpt->scp_nreqs_active,
                      at_get(&svcpt->scp_at_estimate), delay);
@@ -1944,10 +1962,10 @@ ptlrpc_server_handle_req_in(struct ptlrpc_service_part *svcpt,
         }
 
         /* req_in handling should/must be fast */
-        if (cfs_time_current_sec() - req->rq_arrival_time.tv_sec > 5)
-                DEBUG_REQ(D_WARNING, req, "Slow req_in handling "CFS_DURATION_T"s",
-                          cfs_time_sub(cfs_time_current_sec(),
-                                       req->rq_arrival_time.tv_sec));
+       if (ktime_get_real_seconds() - req->rq_arrival_time.tv_sec > 5)
+               DEBUG_REQ(D_WARNING, req, "Slow req_in handling %llds",
+                         (s64)(ktime_get_real_seconds() -
+                               req->rq_arrival_time.tv_sec));
 
         /* Set rpc server deadline and add it to the timed list */
         deadline = (lustre_msghdr_get_flags(req->rq_reqmsg) &
@@ -2005,12 +2023,14 @@ static int
 ptlrpc_server_handle_request(struct ptlrpc_service_part *svcpt,
                             struct ptlrpc_thread *thread)
 {
-       struct ptlrpc_service   *svc = svcpt->scp_service;
-       struct ptlrpc_request   *request;
-       struct timeval           work_start;
-       struct timeval           work_end;
-       long                     timediff;
-       int                      fail_opc = 0;
+       struct ptlrpc_service *svc = svcpt->scp_service;
+       struct ptlrpc_request *request;
+       ktime_t work_start;
+       ktime_t work_end;
+       ktime_t arrived;
+       s64 timediff_usecs;
+       s64 arrived_usecs;
+       int fail_opc = 0;
 
        ENTRY;
 
@@ -2033,11 +2053,12 @@ ptlrpc_server_handle_request(struct ptlrpc_service_part *svcpt,
        if(OBD_FAIL_CHECK(OBD_FAIL_PTLRPC_DUMP_LOG))
                libcfs_debug_dumplog();
 
-       do_gettimeofday(&work_start);
-       timediff = cfs_timeval_sub(&work_start, &request->rq_arrival_time,NULL);
+       work_start = ktime_get_real();
+       arrived = timespec64_to_ktime(request->rq_arrival_time);
+       timediff_usecs = ktime_us_delta(work_start, arrived);
        if (likely(svc->srv_stats != NULL)) {
                 lprocfs_counter_add(svc->srv_stats, PTLRPC_REQWAIT_CNTR,
-                                    timediff);
+                                   timediff_usecs);
                 lprocfs_counter_add(svc->srv_stats, PTLRPC_REQQDEPTH_CNTR,
                                    svcpt->scp_nreqs_incoming);
                lprocfs_counter_add(svc->srv_stats, PTLRPC_REQACTIVE_CNTR,
@@ -2049,19 +2070,18 @@ ptlrpc_server_handle_request(struct ptlrpc_service_part *svcpt,
        if (likely(request->rq_export)) {
                if (unlikely(ptlrpc_check_req(request)))
                        goto put_conn;
-                ptlrpc_update_export_timer(request->rq_export, timediff >> 19);
+               ptlrpc_update_export_timer(request->rq_export,
+                                          timediff_usecs >> 19);
         }
 
         /* Discard requests queued for longer than the deadline.
            The deadline is increased if we send an early reply. */
-        if (cfs_time_current_sec() > request->rq_deadline) {
-                DEBUG_REQ(D_ERROR, request, "Dropping timed-out request from %s"
-                          ": deadline "CFS_DURATION_T":"CFS_DURATION_T"s ago\n",
+       if (ktime_get_real_seconds() > request->rq_deadline) {
+               DEBUG_REQ(D_ERROR, request, "Dropping timed-out request from %s: deadline %lld:%llds ago\n",
                           libcfs_id2str(request->rq_peer),
-                          cfs_time_sub(request->rq_deadline,
-                          request->rq_arrival_time.tv_sec),
-                          cfs_time_sub(cfs_time_current_sec(),
-                          request->rq_deadline));
+                         request->rq_deadline -
+                         request->rq_arrival_time.tv_sec,
+                         ktime_get_real_seconds() - request->rq_deadline);
                 goto put_conn;
         }
 
@@ -2092,21 +2112,17 @@ ptlrpc_server_handle_request(struct ptlrpc_service_part *svcpt,
        ptlrpc_rqphase_move(request, RQ_PHASE_COMPLETE);
 
 put_conn:
-       if (unlikely(cfs_time_current_sec() > request->rq_deadline)) {
-                    DEBUG_REQ(D_WARNING, request, "Request took longer "
-                              "than estimated ("CFS_DURATION_T":"CFS_DURATION_T"s);"
-                              " client may timeout.",
-                              cfs_time_sub(request->rq_deadline,
-                                           request->rq_arrival_time.tv_sec),
-                              cfs_time_sub(cfs_time_current_sec(),
-                                           request->rq_deadline));
-       }
-
-       do_gettimeofday(&work_end);
-       timediff = cfs_timeval_sub(&work_end, &work_start, NULL);
-       CDEBUG(D_RPCTRACE, "Handled RPC pname:cluuid+ref:pid:xid:nid:opc "
-              "%s:%s+%d:%d:x%llu:%s:%d Request procesed in "
-              "%ldus (%ldus total) trans %llu rc %d/%d\n",
+       if (unlikely(ktime_get_real_seconds() > request->rq_deadline)) {
+               DEBUG_REQ(D_WARNING, request, "Request took longer than estimated (%lld:%llds); client may timeout.",
+                         request->rq_deadline -
+                         request->rq_arrival_time.tv_sec,
+                         ktime_get_real_seconds() - request->rq_deadline);
+       }
+
+       work_end = ktime_get_real();
+       timediff_usecs = ktime_us_delta(work_end, work_start);
+       arrived_usecs = ktime_us_delta(work_end, arrived);
+       CDEBUG(D_RPCTRACE, "Handled RPC pname:cluuid+ref:pid:xid:nid:opc %s:%s+%d:%d:x%llu:%s:%d Request procesed in %lldus (%lldus total) trans %llu rc %d/%d\n",
                current_comm(),
                (request->rq_export ?
                 (char *)request->rq_export->exp_client_uuid.uuid : "0"),
@@ -2116,8 +2132,8 @@ put_conn:
                 request->rq_xid,
                 libcfs_id2str(request->rq_peer),
                 lustre_msg_get_opc(request->rq_reqmsg),
-                timediff,
-                cfs_timeval_sub(&work_end, &request->rq_arrival_time, NULL),
+               timediff_usecs,
+               arrived_usecs,
                 (request->rq_repmsg ?
                  lustre_msg_get_transno(request->rq_repmsg) :
                  request->rq_transno),
@@ -2131,16 +2147,14 @@ put_conn:
                         LASSERT(opc < LUSTRE_MAX_OPCODES);
                         lprocfs_counter_add(svc->srv_stats,
                                             opc + EXTRA_MAX_OPCODES,
-                                            timediff);
+                                           timediff_usecs);
                 }
         }
         if (unlikely(request->rq_early_count)) {
                 DEBUG_REQ(D_ADAPTTO, request,
-                          "sent %d early replies before finishing in "
-                          CFS_DURATION_T"s",
+                         "sent %d early replies before finishing in %llds",
                           request->rq_early_count,
-                          cfs_time_sub(work_end.tv_sec,
-                          request->rq_arrival_time.tv_sec));
+                         arrived_usecs / USEC_PER_SEC);
         }
 
        ptlrpc_server_finish_active_request(svcpt, request);
@@ -2156,10 +2170,10 @@ ptlrpc_handle_rs(struct ptlrpc_reply_state *rs)
 {
        struct ptlrpc_service_part *svcpt = rs->rs_svcpt;
        struct ptlrpc_service     *svc = svcpt->scp_service;
-        struct obd_export         *exp;
-        int                        nlocks;
-        int                        been_handled;
-        ENTRY;
+       struct obd_export         *exp;
+       int                        nlocks;
+       int                        been_handled;
+       ENTRY;
 
        exp = rs->rs_export;
 
@@ -2167,58 +2181,104 @@ ptlrpc_handle_rs(struct ptlrpc_reply_state *rs)
        LASSERT(rs->rs_scheduled);
        LASSERT(list_empty(&rs->rs_list));
 
-       spin_lock(&exp->exp_lock);
-       /* Noop if removed already */
-       list_del_init(&rs->rs_exp_list);
-       spin_unlock(&exp->exp_lock);
-
-        /* The disk commit callback holds exp_uncommitted_replies_lock while it
-         * iterates over newly committed replies, removing them from
-         * exp_uncommitted_replies.  It then drops this lock and schedules the
-         * replies it found for handling here.
-         *
-         * We can avoid contention for exp_uncommitted_replies_lock between the
-         * HRT threads and further commit callbacks by checking rs_committed
-         * which is set in the commit callback while it holds both
-         * rs_lock and exp_uncommitted_reples.
-         *
-         * If we see rs_committed clear, the commit callback _may_ not have
-         * handled this reply yet and we race with it to grab
-         * exp_uncommitted_replies_lock before removing the reply from
-         * exp_uncommitted_replies.  Note that if we lose the race and the
-         * reply has already been removed, list_del_init() is a noop.
-         *
-         * If we see rs_committed set, we know the commit callback is handling,
-         * or has handled this reply since store reordering might allow us to
-         * see rs_committed set out of sequence.  But since this is done
-         * holding rs_lock, we can be sure it has all completed once we hold
-         * rs_lock, which we do right next.
-         */
+       /* The disk commit callback holds exp_uncommitted_replies_lock while it
+        * iterates over newly committed replies, removing them from
+        * exp_uncommitted_replies.  It then drops this lock and schedules the
+        * replies it found for handling here.
+        *
+        * We can avoid contention for exp_uncommitted_replies_lock between the
+        * HRT threads and further commit callbacks by checking rs_committed
+        * which is set in the commit callback while it holds both
+        * rs_lock and exp_uncommitted_reples.
+        *
+        * If we see rs_committed clear, the commit callback _may_ not have
+        * handled this reply yet and we race with it to grab
+        * exp_uncommitted_replies_lock before removing the reply from
+        * exp_uncommitted_replies.  Note that if we lose the race and the
+        * reply has already been removed, list_del_init() is a noop.
+        *
+        * If we see rs_committed set, we know the commit callback is handling,
+        * or has handled this reply since store reordering might allow us to
+        * see rs_committed set out of sequence.  But since this is done
+        * holding rs_lock, we can be sure it has all completed once we hold
+        * rs_lock, which we do right next.
+        */
        if (!rs->rs_committed) {
+               /* if rs was commited, no need to convert locks, don't check
+                * rs_committed here because rs may never be added into
+                * exp_uncommitted_replies and this flag never be set, see
+                * target_send_reply() */
+               if (rs->rs_convert_lock &&
+                   rs->rs_transno > exp->exp_last_committed) {
+                       struct ldlm_lock *lock;
+                       struct ldlm_lock *ack_locks[RS_MAX_LOCKS] = { NULL };
+
+                       spin_lock(&rs->rs_lock);
+                       if (rs->rs_convert_lock &&
+                           rs->rs_transno > exp->exp_last_committed) {
+                               nlocks = rs->rs_nlocks;
+                               while (nlocks-- > 0) {
+                                       /*
+                                        * NB don't assume rs is always handled
+                                        * by the same service thread (see
+                                        * ptlrpc_hr_select, so REP-ACK hr may
+                                        * race with trans commit, while the
+                                        * latter will release locks, get locks
+                                        * here early to convert to COS mode
+                                        * safely.
+                                        */
+                                       lock = ldlm_handle2lock(
+                                                       &rs->rs_locks[nlocks]);
+                                       LASSERT(lock);
+                                       ack_locks[nlocks] = lock;
+                                       rs->rs_modes[nlocks] = LCK_COS;
+                               }
+                               nlocks = rs->rs_nlocks;
+                               rs->rs_convert_lock = 0;
+                               /* clear rs_scheduled so that commit callback
+                                * can schedule again */
+                               rs->rs_scheduled = 0;
+                               spin_unlock(&rs->rs_lock);
+
+                               while (nlocks-- > 0) {
+                                       lock = ack_locks[nlocks];
+                                       ldlm_lock_downgrade(lock, LCK_COS);
+                                       LDLM_LOCK_PUT(lock);
+                               }
+                               RETURN(0);
+                       }
+                       spin_unlock(&rs->rs_lock);
+               }
+
                spin_lock(&exp->exp_uncommitted_replies_lock);
                list_del_init(&rs->rs_obd_list);
                spin_unlock(&exp->exp_uncommitted_replies_lock);
        }
 
+       spin_lock(&exp->exp_lock);
+       /* Noop if removed already */
+       list_del_init(&rs->rs_exp_list);
+       spin_unlock(&exp->exp_lock);
+
        spin_lock(&rs->rs_lock);
 
-        been_handled = rs->rs_handled;
-        rs->rs_handled = 1;
+       been_handled = rs->rs_handled;
+       rs->rs_handled = 1;
 
-        nlocks = rs->rs_nlocks;                 /* atomic "steal", but */
-        rs->rs_nlocks = 0;                      /* locks still on rs_locks! */
+       nlocks = rs->rs_nlocks;                 /* atomic "steal", but */
+       rs->rs_nlocks = 0;                      /* locks still on rs_locks! */
 
-        if (nlocks == 0 && !been_handled) {
-                /* If we see this, we should already have seen the warning
-                 * in mds_steal_ack_locks()  */
+       if (nlocks == 0 && !been_handled) {
+               /* If we see this, we should already have seen the warning
+                * in mds_steal_ack_locks()  */
                CDEBUG(D_HA, "All locks stolen from rs %p x%lld.t%lld"
                       " o%d NID %s\n",
                       rs,
                       rs->rs_xid, rs->rs_transno, rs->rs_opc,
                       libcfs_nid2str(exp->exp_connection->c_peer.nid));
-        }
+       }
 
-        if ((!been_handled && rs->rs_on_net) || nlocks > 0) {
+       if ((!been_handled && rs->rs_on_net) || nlocks > 0) {
                spin_unlock(&rs->rs_lock);
 
                if (!been_handled && rs->rs_on_net) {
@@ -2234,6 +2294,7 @@ ptlrpc_handle_rs(struct ptlrpc_reply_state *rs)
        }
 
        rs->rs_scheduled = 0;
+       rs->rs_convert_lock = 0;
 
        if (!rs->rs_on_net) {
                /* Off the net */
@@ -2897,8 +2958,8 @@ int ptlrpc_hr_init(void)
        struct ptlrpc_hr_partition      *hrp;
        struct ptlrpc_hr_thread         *hrt;
        int                             rc;
+       int                             cpt;
        int                             i;
-       int                             j;
        int                             weight;
        ENTRY;
 
@@ -2912,29 +2973,28 @@ int ptlrpc_hr_init(void)
 
        init_waitqueue_head(&ptlrpc_hr.hr_waitq);
 
-       weight = cfs_cpu_ht_nsiblings(smp_processor_id());
+       weight = cpumask_weight(topology_sibling_cpumask(smp_processor_id()));
 
-       cfs_percpt_for_each(hrp, i, ptlrpc_hr.hr_partitions) {
-               hrp->hrp_cpt = i;
+       cfs_percpt_for_each(hrp, cpt, ptlrpc_hr.hr_partitions) {
+               hrp->hrp_cpt = cpt;
 
                atomic_set(&hrp->hrp_nstarted, 0);
                atomic_set(&hrp->hrp_nstopped, 0);
 
-               hrp->hrp_nthrs = cfs_cpt_weight(ptlrpc_hr.hr_cpt_table, i);
-
+               hrp->hrp_nthrs = cfs_cpt_weight(ptlrpc_hr.hr_cpt_table, cpt);
                hrp->hrp_nthrs /= weight;
                if (hrp->hrp_nthrs == 0)
                        hrp->hrp_nthrs = 1;
 
-               OBD_CPT_ALLOC(hrp->hrp_thrs, ptlrpc_hr.hr_cpt_table, i,
+               OBD_CPT_ALLOC(hrp->hrp_thrs, ptlrpc_hr.hr_cpt_table, cpt,
                              hrp->hrp_nthrs * sizeof(*hrt));
                if (hrp->hrp_thrs == NULL)
                        GOTO(out, rc = -ENOMEM);
 
-               for (j = 0; j < hrp->hrp_nthrs; j++) {
-                       hrt = &hrp->hrp_thrs[j];
+               for (i = 0; i < hrp->hrp_nthrs; i++) {
+                       hrt = &hrp->hrp_thrs[i];
 
-                       hrt->hrt_id = j;
+                       hrt->hrt_id = i;
                        hrt->hrt_partition = hrp;
                        init_waitqueue_head(&hrt->hrt_waitq);
                        spin_lock_init(&hrt->hrt_lock);
@@ -2952,14 +3012,14 @@ out:
 void ptlrpc_hr_fini(void)
 {
        struct ptlrpc_hr_partition      *hrp;
-       int                             i;
+       int                             cpt;
 
        if (ptlrpc_hr.hr_partitions == NULL)
                return;
 
        ptlrpc_stop_hr_threads();
 
-       cfs_percpt_for_each(hrp, i, ptlrpc_hr.hr_partitions) {
+       cfs_percpt_for_each(hrp, cpt, ptlrpc_hr.hr_partitions) {
                if (hrp->hrp_thrs != NULL) {
                        OBD_FREE(hrp->hrp_thrs,
                                 hrp->hrp_nthrs * sizeof(hrp->hrp_thrs[0]));
@@ -3186,6 +3246,7 @@ int ptlrpc_unregister_service(struct ptlrpc_service *service)
        ptlrpc_service_nrs_cleanup(service);
 
        ptlrpc_lprocfs_unregister_service(service);
+       ptlrpc_sysfs_unregister_service(service);
 
        ptlrpc_service_free(service);
 
@@ -3201,11 +3262,11 @@ EXPORT_SYMBOL(ptlrpc_unregister_service);
  * to be shot, so it's intentionally non-aggressive. */
 static int ptlrpc_svcpt_health_check(struct ptlrpc_service_part *svcpt)
 {
-       struct ptlrpc_request           *request = NULL;
-       struct timeval                  right_now;
-       long                            timediff;
+       struct ptlrpc_request *request = NULL;
+       struct timespec64 right_now;
+       struct timespec64 timediff;
 
-       do_gettimeofday(&right_now);
+       ktime_get_real_ts64(&right_now);
 
        spin_lock(&svcpt->scp_req_lock);
        /* How long has the next entry been waiting? */
@@ -3219,13 +3280,13 @@ static int ptlrpc_svcpt_health_check(struct ptlrpc_service_part *svcpt)
                return 0;
        }
 
-       timediff = cfs_timeval_sub(&right_now, &request->rq_arrival_time, NULL);
+       timediff = timespec64_sub(right_now, request->rq_arrival_time);
        spin_unlock(&svcpt->scp_req_lock);
 
-       if ((timediff / ONE_MILLION) >
+       if ((timediff.tv_sec) >
            (AT_OFF ? obd_timeout * 3 / 2 : at_max)) {
-               CERROR("%s: unhealthy - request has been waiting %lds\n",
-                      svcpt->scp_service->srv_name, timediff / ONE_MILLION);
+               CERROR("%s: unhealthy - request has been waiting %llds\n",
+                      svcpt->scp_service->srv_name, (s64)timediff.tv_sec);
                return -1;
        }