Whamcloud - gitweb
LU-11300 lnet: remove lnd_query interface.
[fs/lustre-release.git] / lnet / klnds / o2iblnd / o2iblnd_cb.c
index eba63b1..82c4509 100644 (file)
@@ -86,12 +86,17 @@ kiblnd_tx_done(struct kib_tx *tx)
                if (lntmsg[i] == NULL)
                        continue;
 
+               /* propagate health status to LNet for requests */
+               if (i == 0 && lntmsg[i])
+                       lntmsg[i]->msg_health_status = tx->tx_hstatus;
+
                lnet_finalize(lntmsg[i], rc);
        }
 }
 
 void
-kiblnd_txlist_done(struct list_head *txlist, int status)
+kiblnd_txlist_done(struct list_head *txlist, int status,
+                  enum lnet_msg_hstatus hstatus)
 {
        struct kib_tx *tx;
 
@@ -102,6 +107,8 @@ kiblnd_txlist_done(struct list_head *txlist, int status)
                /* complete now */
                tx->tx_waiting = 0;
                tx->tx_status = status;
+               if (hstatus != LNET_MSG_STATUS_OK)
+                       tx->tx_hstatus = hstatus;
                kiblnd_tx_done(tx);
        }
 }
@@ -131,6 +138,7 @@ kiblnd_get_idle_tx(struct lnet_ni *ni, lnet_nid_t target)
         LASSERT (tx->tx_nfrags == 0);
 
        tx->tx_gaps = false;
+       tx->tx_hstatus = LNET_MSG_STATUS_OK;
 
         return tx;
 }
@@ -195,7 +203,12 @@ kiblnd_post_rx(struct kib_rx *rx, int credit)
         * own this rx (and rx::rx_conn) anymore, LU-5678.
         */
        kiblnd_conn_addref(conn);
+#ifdef HAVE_IB_POST_SEND_RECV_CONST
+       rc = ib_post_recv(conn->ibc_cmid->qp, &rx->rx_wrq,
+                         (const struct ib_recv_wr **)&bad_wrq);
+#else
        rc = ib_post_recv(conn->ibc_cmid->qp, &rx->rx_wrq, &bad_wrq);
+#endif
        if (unlikely(rc != 0)) {
                CERROR("Can't post rx for %s: %d, bad_wrq: %p\n",
                       libcfs_nid2str(conn->ibc_peer->ibp_nid), rc, bad_wrq);
@@ -266,23 +279,24 @@ kiblnd_handle_completion(struct kib_conn *conn, int txtype, int status, u64 cook
                spin_unlock(&conn->ibc_lock);
 
                CWARN("Unmatched completion type %x cookie %#llx from %s\n",
-                      txtype, cookie, libcfs_nid2str(conn->ibc_peer->ibp_nid));
-                kiblnd_close_conn(conn, -EPROTO);
-                return;
-        }
+                     txtype, cookie, libcfs_nid2str(conn->ibc_peer->ibp_nid));
+               kiblnd_close_conn(conn, -EPROTO);
+               return;
+       }
 
-        if (tx->tx_status == 0) {               /* success so far */
-                if (status < 0) {               /* failed? */
-                        tx->tx_status = status;
-                } else if (txtype == IBLND_MSG_GET_REQ) {
-                        lnet_set_reply_msg_len(ni, tx->tx_lntmsg[1], status);
-                }
-        }
+       if (tx->tx_status == 0) {               /* success so far */
+               if (status < 0) {               /* failed? */
+                       tx->tx_status = status;
+                       tx->tx_hstatus = LNET_MSG_STATUS_REMOTE_ERROR;
+               } else if (txtype == IBLND_MSG_GET_REQ) {
+                       lnet_set_reply_msg_len(ni, tx->tx_lntmsg[1], status);
+               }
+       }
 
-        tx->tx_waiting = 0;
+       tx->tx_waiting = 0;
 
-        idle = !tx->tx_queued && (tx->tx_sending == 0);
-        if (idle)
+       idle = !tx->tx_queued && (tx->tx_sending == 0);
+       if (idle)
                list_del(&tx->tx_list);
 
        spin_unlock(&conn->ibc_lock);
@@ -857,6 +871,7 @@ __must_hold(&conn->ibc_lock)
 {
        struct kib_msg *msg = tx->tx_msg;
        struct kib_peer_ni *peer_ni = conn->ibc_peer;
+       struct lnet_ni *ni = peer_ni->ibp_ni;
        int ver = conn->ibc_version;
        int rc;
        int done;
@@ -873,7 +888,7 @@ __must_hold(&conn->ibc_lock)
        LASSERT(conn->ibc_credits <= conn->ibc_queue_depth);
 
        if (conn->ibc_nsends_posted ==
-           conn->ibc_queue_depth) {
+           kiblnd_concurrent_sends(ver, ni)) {
                /* tx completions outstanding... */
                CDEBUG(D_NET, "%s: posted enough\n",
                       libcfs_nid2str(peer_ni->ibp_nid));
@@ -906,6 +921,7 @@ __must_hold(&conn->ibc_lock)
                 * kiblnd_check_sends_locked will queue NOOP again when
                 * posted NOOPs complete */
                spin_unlock(&conn->ibc_lock);
+               tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_ERROR;
                kiblnd_tx_done(tx);
                spin_lock(&conn->ibc_lock);
                 CDEBUG(D_NET, "%s(%d): redundant or enough NOOP\n",
@@ -960,7 +976,15 @@ __must_hold(&conn->ibc_lock)
                         libcfs_nid2str(conn->ibc_peer->ibp_nid));
 
                bad = NULL;
-               rc = ib_post_send(conn->ibc_cmid->qp, wr, &bad);
+               if (lnet_send_error_simulation(tx->tx_lntmsg[0], &tx->tx_hstatus))
+                       rc = -EINVAL;
+               else
+#ifdef HAVE_IB_POST_SEND_RECV_CONST
+                       rc = ib_post_send(conn->ibc_cmid->qp, wr,
+                                         (const struct ib_send_wr **)&bad);
+#else
+                       rc = ib_post_send(conn->ibc_cmid->qp, wr, &bad);
+#endif
        }
 
        conn->ibc_last_send = ktime_get();
@@ -1018,7 +1042,7 @@ kiblnd_check_sends_locked(struct kib_conn *conn)
         }
 
        LASSERT(conn->ibc_nsends_posted <=
-               conn->ibc_queue_depth);
+               kiblnd_concurrent_sends(ver, ni));
         LASSERT (!IBLND_OOB_CAPABLE(ver) ||
                  conn->ibc_noops_posted <= IBLND_OOB_MSGS(ver));
         LASSERT (conn->ibc_reserved_credits >= 0);
@@ -1027,8 +1051,7 @@ kiblnd_check_sends_locked(struct kib_conn *conn)
               !list_empty(&conn->ibc_tx_queue_rsrvd)) {
                tx = list_entry(conn->ibc_tx_queue_rsrvd.next,
                                struct kib_tx, tx_list);
-               list_del(&tx->tx_list);
-               list_add_tail(&tx->tx_list, &conn->ibc_tx_queue);
+               list_move_tail(&tx->tx_list, &conn->ibc_tx_queue);
                 conn->ibc_reserved_credits--;
         }
 
@@ -1101,6 +1124,7 @@ kiblnd_tx_complete(struct kib_tx *tx, int status)
                 conn->ibc_noops_posted--;
 
         if (failed) {
+               tx->tx_hstatus = LNET_MSG_STATUS_REMOTE_DROPPED;
                 tx->tx_waiting = 0;             /* don't wait for peer_ni */
                 tx->tx_status = -EIO;
         }
@@ -1268,7 +1292,22 @@ kiblnd_queue_tx_locked(struct kib_tx *tx, struct kib_conn *conn)
        LASSERT(!tx->tx_queued);        /* not queued for sending already */
        LASSERT(conn->ibc_state >= IBLND_CONN_ESTABLISHED);
 
-       timeout_ns = *kiblnd_tunables.kib_timeout * NSEC_PER_SEC;
+       if (conn->ibc_state >= IBLND_CONN_DISCONNECTED) {
+               tx->tx_status = -ECONNABORTED;
+               tx->tx_waiting = 0;
+               if (tx->tx_conn != NULL) {
+                       /* PUT_DONE first attached to conn as a PUT_REQ */
+                       LASSERT(tx->tx_conn == conn);
+                       LASSERT(tx->tx_msg->ibm_type == IBLND_MSG_PUT_DONE);
+                       tx->tx_conn = NULL;
+                       kiblnd_conn_decref(conn);
+               }
+               list_add(&tx->tx_list, &conn->ibc_zombie_txs);
+
+               return;
+       }
+
+       timeout_ns = lnet_get_lnd_timeout() * NSEC_PER_SEC;
        tx->tx_queued = 1;
        tx->tx_deadline = ktime_add_ns(ktime_get(), timeout_ns);
 
@@ -1372,8 +1411,9 @@ kiblnd_connect_peer(struct kib_peer_ni *peer_ni)
         LASSERT (net != NULL);
         LASSERT (peer_ni->ibp_connecting > 0);
 
-        cmid = kiblnd_rdma_create_id(kiblnd_cm_callback, peer_ni, RDMA_PS_TCP,
-                                     IB_QPT_RC);
+       cmid = kiblnd_rdma_create_id(peer_ni->ibp_ni->ni_net_ns,
+                                    kiblnd_cm_callback, peer_ni,
+                                    RDMA_PS_TCP, IB_QPT_RC);
 
         if (IS_ERR(cmid)) {
                 CERROR("Can't create CMID for %s: %ld\n",
@@ -1394,21 +1434,21 @@ kiblnd_connect_peer(struct kib_peer_ni *peer_ni)
 
         kiblnd_peer_addref(peer_ni);               /* cmid's ref */
 
-        if (*kiblnd_tunables.kib_use_priv_port) {
-                rc = kiblnd_resolve_addr(cmid, &srcaddr, &dstaddr,
-                                         *kiblnd_tunables.kib_timeout * 1000);
-        } else {
-                rc = rdma_resolve_addr(cmid,
-                                       (struct sockaddr *)&srcaddr,
-                                       (struct sockaddr *)&dstaddr,
-                                       *kiblnd_tunables.kib_timeout * 1000);
-        }
-        if (rc != 0) {
-                /* Can't initiate address resolution:  */
-                CERROR("Can't resolve addr for %s: %d\n",
-                       libcfs_nid2str(peer_ni->ibp_nid), rc);
-                goto failed2;
-        }
+       if (*kiblnd_tunables.kib_use_priv_port) {
+               rc = kiblnd_resolve_addr(cmid, &srcaddr, &dstaddr,
+                                        lnet_get_lnd_timeout() * 1000);
+       } else {
+               rc = rdma_resolve_addr(cmid,
+                                      (struct sockaddr *)&srcaddr,
+                                      (struct sockaddr *)&dstaddr,
+                                      lnet_get_lnd_timeout() * 1000);
+       }
+       if (rc != 0) {
+               /* Can't initiate address resolution:  */
+               CERROR("Can't resolve addr for %s: %d\n",
+                      libcfs_nid2str(peer_ni->ibp_nid), rc);
+               goto failed2;
+       }
 
        return;
 
@@ -1424,12 +1464,10 @@ kiblnd_connect_peer(struct kib_peer_ni *peer_ni)
 bool
 kiblnd_reconnect_peer(struct kib_peer_ni *peer_ni)
 {
-       rwlock_t         *glock = &kiblnd_data.kib_global_lock;
-       char             *reason = NULL;
-       struct list_head  txs;
-       unsigned long     flags;
-
-       INIT_LIST_HEAD(&txs);
+       rwlock_t *glock = &kiblnd_data.kib_global_lock;
+       char *reason = NULL;
+       LIST_HEAD(txs);
+       unsigned long flags;
 
        write_lock_irqsave(glock, flags);
        if (peer_ni->ibp_reconnecting == 0) {
@@ -1468,7 +1506,8 @@ kiblnd_reconnect_peer(struct kib_peer_ni *peer_ni)
 
        CWARN("Abort reconnection of %s: %s\n",
              libcfs_nid2str(peer_ni->ibp_nid), reason);
-       kiblnd_txlist_done(&txs, -ECONNABORTED);
+       kiblnd_txlist_done(&txs, -ECONNABORTED,
+                          LNET_MSG_STATUS_LOCAL_ABORTED);
        return false;
 }
 
@@ -1543,6 +1582,7 @@ kiblnd_launch_tx(struct lnet_ni *ni, struct kib_tx *tx, lnet_nid_t nid)
                if (tx != NULL) {
                        tx->tx_status = -EHOSTUNREACH;
                        tx->tx_waiting = 0;
+                       tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_ERROR;
                        kiblnd_tx_done(tx);
                }
                return;
@@ -1667,6 +1707,7 @@ kiblnd_send(struct lnet_ni *ni, void *private, struct lnet_msg *lntmsg)
                if (rc != 0) {
                        CERROR("Can't setup GET sink for %s: %d\n",
                               libcfs_nid2str(target.nid), rc);
+                       tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_ERROR;
                        kiblnd_tx_done(tx);
                        return -EIO;
                }
@@ -1821,9 +1862,11 @@ kiblnd_reply(struct lnet_ni *ni, struct kib_rx *rx, struct lnet_msg *lntmsg)
         kiblnd_queue_tx(tx, rx->rx_conn);
         return;
 
- failed_1:
+
+failed_1:
+       tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_ERROR;
        kiblnd_tx_done(tx);
- failed_0:
+failed_0:
        lnet_finalize(lntmsg, -EIO);
 }
 
@@ -1905,6 +1948,7 @@ kiblnd_recv(struct lnet_ni *ni, void *private, struct lnet_msg *lntmsg,
                if (rc != 0) {
                        CERROR("Can't setup PUT sink for %s: %d\n",
                               libcfs_nid2str(conn->ibc_peer->ibp_nid), rc);
+                       tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_ERROR;
                        kiblnd_tx_done(tx);
                        /* tell peer_ni it's over */
                        kiblnd_send_completion(rx->rx_conn, IBLND_MSG_PUT_NAK,
@@ -1973,24 +2017,24 @@ kiblnd_peer_alive(struct kib_peer_ni *peer_ni)
 static void
 kiblnd_peer_notify(struct kib_peer_ni *peer_ni)
 {
-        int           error = 0;
+       int           error = 0;
        time64_t last_alive = 0;
-        unsigned long flags;
+       unsigned long flags;
 
        read_lock_irqsave(&kiblnd_data.kib_global_lock, flags);
 
        if (kiblnd_peer_idle(peer_ni) && peer_ni->ibp_error != 0) {
-                error = peer_ni->ibp_error;
-                peer_ni->ibp_error = 0;
+               error = peer_ni->ibp_error;
+               peer_ni->ibp_error = 0;
 
-                last_alive = peer_ni->ibp_last_alive;
-        }
+               last_alive = peer_ni->ibp_last_alive;
+       }
 
        read_unlock_irqrestore(&kiblnd_data.kib_global_lock, flags);
 
-        if (error != 0)
-                lnet_notify(peer_ni->ibp_ni,
-                            peer_ni->ibp_nid, 0, last_alive);
+       if (error != 0)
+               lnet_notify(peer_ni->ibp_ni,
+                           peer_ni->ibp_nid, false, false, last_alive);
 }
 
 void
@@ -2101,10 +2145,10 @@ kiblnd_handle_early_rxs(struct kib_conn *conn)
        write_unlock_irqrestore(&kiblnd_data.kib_global_lock, flags);
 }
 
-static void
+void
 kiblnd_abort_txs(struct kib_conn *conn, struct list_head *txs)
 {
-       struct list_head         zombies = LIST_HEAD_INIT(zombies);
+       LIST_HEAD(zombies);
        struct list_head        *tmp;
        struct list_head        *nxt;
        struct kib_tx *tx;
@@ -2118,23 +2162,50 @@ kiblnd_abort_txs(struct kib_conn *conn, struct list_head *txs)
                        LASSERT(!tx->tx_queued);
                        LASSERT(tx->tx_waiting ||
                                tx->tx_sending != 0);
+                       if (conn->ibc_comms_error == -ETIMEDOUT) {
+                               if (tx->tx_waiting && !tx->tx_sending)
+                                       tx->tx_hstatus =
+                                         LNET_MSG_STATUS_REMOTE_TIMEOUT;
+                               else if (tx->tx_sending)
+                                       tx->tx_hstatus =
+                                         LNET_MSG_STATUS_NETWORK_TIMEOUT;
+                       }
                } else {
                        LASSERT(tx->tx_queued);
+                       if (conn->ibc_comms_error == -ETIMEDOUT)
+                               tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_TIMEOUT;
+                       else
+                               tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_ERROR;
                }
 
                tx->tx_status = -ECONNABORTED;
                tx->tx_waiting = 0;
 
+               /*
+                * TODO: This makes an assumption that
+                * kiblnd_tx_complete() will be called for each tx. If
+                * that event is dropped we could end up with stale
+                * connections floating around. We'd like to deal with
+                * that in a better way.
+                *
+                * Also that means we can exceed the timeout by many
+                * seconds.
+                */
                if (tx->tx_sending == 0) {
                        tx->tx_queued = 0;
-                       list_del(&tx->tx_list);
-                       list_add(&tx->tx_list, &zombies);
+                       list_move(&tx->tx_list, &zombies);
                }
        }
 
        spin_unlock(&conn->ibc_lock);
 
-       kiblnd_txlist_done(&zombies, -ECONNABORTED);
+       /*
+        * aborting transmits occurs when finalizing the connection.
+        * The connection is finalized on error.
+        * Passing LNET_MSG_STATUS_OK to txlist_done() will not
+        * override the value already set in tx->tx_hstatus above.
+        */
+       kiblnd_txlist_done(&zombies, -ECONNABORTED, LNET_MSG_STATUS_OK);
 }
 
 static void
@@ -2143,13 +2214,13 @@ kiblnd_finalise_conn(struct kib_conn *conn)
        LASSERT (!in_interrupt());
        LASSERT (conn->ibc_state > IBLND_CONN_INIT);
 
-       kiblnd_set_conn_state(conn, IBLND_CONN_DISCONNECTED);
-
        /* abort_receives moves QP state to IB_QPS_ERR.  This is only required
         * for connections that didn't get as far as being connected, because
         * rdma_disconnect() does this for free. */
        kiblnd_abort_receives(conn);
 
+       kiblnd_set_conn_state(conn, IBLND_CONN_DISCONNECTED);
+
        /* Complete all tx descs not waiting for sends to complete.
         * NB we should be safe from RDMA now that the QP has changed state */
 
@@ -2166,7 +2237,7 @@ static void
 kiblnd_peer_connect_failed(struct kib_peer_ni *peer_ni, int active,
                           int error)
 {
-       struct list_head zombies = LIST_HEAD_INIT(zombies);
+       LIST_HEAD(zombies);
        unsigned long   flags;
 
        LASSERT (error != 0);
@@ -2213,7 +2284,8 @@ kiblnd_peer_connect_failed(struct kib_peer_ni *peer_ni, int active,
        CNETERR("Deleting messages for %s: connection failed\n",
                libcfs_nid2str(peer_ni->ibp_nid));
 
-       kiblnd_txlist_done(&zombies, -EHOSTUNREACH);
+       kiblnd_txlist_done(&zombies, error,
+                          LNET_MSG_STATUS_LOCAL_DROPPED);
 }
 
 static void
@@ -2221,7 +2293,7 @@ kiblnd_connreq_done(struct kib_conn *conn, int status)
 {
        struct kib_peer_ni *peer_ni = conn->ibc_peer;
        struct kib_tx *tx;
-       struct list_head txs;
+       LIST_HEAD(txs);
        unsigned long    flags;
        int              active;
 
@@ -2278,7 +2350,6 @@ kiblnd_connreq_done(struct kib_conn *conn, int status)
         }
 
        /* grab pending txs while I have the lock */
-       INIT_LIST_HEAD(&txs);
        list_splice_init(&peer_ni->ibp_tx_queue, &txs);
 
         if (!kiblnd_peer_active(peer_ni) ||        /* peer_ni has been deleted */
@@ -2288,7 +2359,8 @@ kiblnd_connreq_done(struct kib_conn *conn, int status)
                 kiblnd_close_conn_locked(conn, -ECONNABORTED);
                write_unlock_irqrestore(&kiblnd_data.kib_global_lock, flags);
 
-               kiblnd_txlist_done(&txs, -ECONNABORTED);
+               kiblnd_txlist_done(&txs, -ECONNABORTED,
+                                  LNET_MSG_STATUS_LOCAL_ERROR);
 
                return;
        }
@@ -2679,11 +2751,12 @@ kiblnd_check_reconnect(struct kib_conn *conn, int version,
        }
 
        write_lock_irqsave(glock, flags);
-        /* retry connection if it's still needed and no other connection
-         * attempts (active or passive) are in progress
-         * NB: reconnect is still needed even when ibp_tx_queue is
-         * empty if ibp_version != version because reconnect may be
-         * initiated by kiblnd_query() */
+       /* retry connection if it's still needed and no other connection
+        * attempts (active or passive) are in progress
+        * NB: reconnect is still needed even when ibp_tx_queue is
+        * empty if ibp_version != version because reconnect may be
+        * initiated.
+        */
        reconnect = (!list_empty(&peer_ni->ibp_tx_queue) ||
                     peer_ni->ibp_version != version) &&
                    peer_ni->ibp_connecting &&
@@ -3109,9 +3182,9 @@ kiblnd_cm_callback(struct rdma_cm_id *cmid, struct rdma_cm_event *event)
                         CNETERR("Can't resolve address for %s: %d\n",
                                 libcfs_nid2str(peer_ni->ibp_nid), event->status);
                         rc = event->status;
-                } else {
-                        rc = rdma_resolve_route(
-                                cmid, *kiblnd_tunables.kib_timeout * 1000);
+               } else {
+                       rc = rdma_resolve_route(
+                               cmid, lnet_get_lnd_timeout() * 1000);
                        if (rc == 0) {
                                struct kib_net *net = peer_ni->ibp_ni->ni_data;
                                struct kib_dev *dev = net->ibn_dev;
@@ -3292,9 +3365,9 @@ kiblnd_conn_timed_out_locked(struct kib_conn *conn)
 static void
 kiblnd_check_conns (int idx)
 {
-       struct list_head  closes = LIST_HEAD_INIT(closes);
-       struct list_head  checksends = LIST_HEAD_INIT(checksends);
-       struct list_head  timedout_txs = LIST_HEAD_INIT(timedout_txs);
+       LIST_HEAD(closes);
+       LIST_HEAD(checksends);
+       LIST_HEAD(timedout_txs);
        struct list_head *peers = &kiblnd_data.kib_peers[idx];
        struct list_head *ptmp;
        struct kib_peer_ni *peer_ni;
@@ -3361,7 +3434,8 @@ kiblnd_check_conns (int idx)
        write_unlock_irqrestore(&kiblnd_data.kib_global_lock, flags);
 
        if (!list_empty(&timedout_txs))
-               kiblnd_txlist_done(&timedout_txs, -ETIMEDOUT);
+               kiblnd_txlist_done(&timedout_txs, -ETIMEDOUT,
+                                  LNET_MSG_STATUS_LOCAL_TIMEOUT);
 
        /* Handle timeout by closing the whole
         * connection. We can only be sure RDMA activity
@@ -3515,6 +3589,7 @@ kiblnd_connd (void *arg)
                         const int n = 4;
                         const int p = 1;
                         int       chunk = kiblnd_data.kib_peer_hash_size;
+                       unsigned int lnd_timeout;
 
                        spin_unlock_irqrestore(lock, flags);
                         dropped_lock = 1;
@@ -3527,11 +3602,11 @@ kiblnd_connd (void *arg)
                          * connection within (n+1)/n times the timeout
                          * interval. */
 
-                        if (*kiblnd_tunables.kib_timeout > n * p)
-                                chunk = (chunk * n * p) /
-                                        *kiblnd_tunables.kib_timeout;
-                        if (chunk == 0)
-                                chunk = 1;
+                       lnd_timeout = lnet_get_lnd_timeout();
+                       if (lnd_timeout > n * p)
+                               chunk = (chunk * n * p) / lnd_timeout;
+                       if (chunk == 0)
+                               chunk = 1;
 
                        for (i = 0; i < chunk; i++) {
                                kiblnd_check_conns(peer_index);
@@ -3539,7 +3614,7 @@ kiblnd_connd (void *arg)
                                             kiblnd_data.kib_peer_hash_size;
                        }
 
-                       deadline += msecs_to_jiffies(p * MSEC_PER_SEC);
+                       deadline += cfs_time_seconds(p);
                        spin_lock_irqsave(lock, flags);
                }
 
@@ -3569,21 +3644,34 @@ kiblnd_qp_event(struct ib_event *event, void *arg)
 {
        struct kib_conn *conn = arg;
 
-        switch (event->event) {
-        case IB_EVENT_COMM_EST:
-                CDEBUG(D_NET, "%s established\n",
-                       libcfs_nid2str(conn->ibc_peer->ibp_nid));
+       switch (event->event) {
+       case IB_EVENT_COMM_EST:
+               CDEBUG(D_NET, "%s established\n",
+                      libcfs_nid2str(conn->ibc_peer->ibp_nid));
                /* We received a packet but connection isn't established
                 * probably handshake packet was lost, so free to
                 * force make connection established */
                rdma_notify(conn->ibc_cmid, IB_EVENT_COMM_EST);
-                return;
+               return;
 
-        default:
-                CERROR("%s: Async QP event type %d\n",
-                       libcfs_nid2str(conn->ibc_peer->ibp_nid), event->event);
-                return;
-        }
+       case IB_EVENT_PORT_ERR:
+       case IB_EVENT_DEVICE_FATAL:
+               CERROR("Fatal device error for NI %s\n",
+                      libcfs_nid2str(conn->ibc_peer->ibp_ni->ni_nid));
+               atomic_set(&conn->ibc_peer->ibp_ni->ni_fatal_error_on, 1);
+               return;
+
+       case IB_EVENT_PORT_ACTIVE:
+               CERROR("Port reactivated for NI %s\n",
+                      libcfs_nid2str(conn->ibc_peer->ibp_ni->ni_nid));
+               atomic_set(&conn->ibc_peer->ibp_ni->ni_fatal_error_on, 0);
+               return;
+
+       default:
+               CERROR("%s: Async QP event type %d\n",
+                      libcfs_nid2str(conn->ibc_peer->ibp_nid), event->event);
+               return;
+       }
 }
 
 static void
@@ -3810,6 +3898,7 @@ kiblnd_failover_thread(void *arg)
 {
        rwlock_t        *glock = &kiblnd_data.kib_global_lock;
        struct kib_dev *dev;
+       struct net *ns = arg;
        wait_queue_entry_t wait;
        unsigned long    flags;
        int              rc;
@@ -3838,7 +3927,7 @@ kiblnd_failover_thread(void *arg)
                         dev->ibd_failover = 1;
                        write_unlock_irqrestore(glock, flags);
 
-                       rc = kiblnd_dev_failover(dev);
+                       rc = kiblnd_dev_failover(dev, ns);
 
                        write_lock_irqsave(glock, flags);