Whamcloud - gitweb
LU-11304 misc: update all url links to whamcloud
[fs/lustre-release.git] / lnet / klnds / socklnd / socklnd_cb.c
index f8b0f06..01f9629 100644 (file)
@@ -1,14 +1,14 @@
 /*
  * Copyright (c) 2003, 2010, Oracle and/or its affiliates. All rights reserved.
  *
- * Copyright (c) 2011, 2014, Intel Corporation.
+ * Copyright (c) 2011, 2017, Intel Corporation.
  *
  *   Author: Zach Brown <zab@zabbo.net>
  *   Author: Peter J. Braam <braam@clusterfs.com>
  *   Author: Phil Schwan <phil@clusterfs.com>
  *   Author: Eric Barton <eric@bartonsoftware.com>
  *
- *   This file is part of Lustre, https://wiki.hpdd.intel.com/
+ *   This file is part of Lustre, https://wiki.whamcloud.com/
  *
  *   Portals is free software; you can redistribute it and/or
  *   modify it under the terms of version 2 of the GNU General Public
@@ -57,6 +57,7 @@ ksocknal_alloc_tx(int type, int size)
        tx->tx_zc_aborted = 0;
        tx->tx_zc_capable = 0;
        tx->tx_zc_checked = 0;
+       tx->tx_hstatus = LNET_MSG_STATUS_OK;
        tx->tx_desc_size  = size;
 
        atomic_inc(&ksocknal_data.ksnd_nactive_txs);
@@ -222,7 +223,7 @@ ksocknal_transmit(struct ksock_conn *conn, struct ksock_tx *tx)
                        /* allocated send buffer bytes < computed; infer
                         * something got ACKed */
                        conn->ksnc_tx_deadline = ktime_get_seconds() +
-                                                *ksocknal_tunables.ksnd_timeout;
+                                                lnet_get_lnd_timeout();
                        conn->ksnc_peer->ksnp_last_alive = ktime_get_seconds();
                        conn->ksnc_tx_bufnob = bufnob;
                        smp_mb();
@@ -271,7 +272,7 @@ ksocknal_recv_iov(struct ksock_conn *conn)
 
        conn->ksnc_peer->ksnp_last_alive = ktime_get_seconds();
        conn->ksnc_rx_deadline = ktime_get_seconds() +
-                                *ksocknal_tunables.ksnd_timeout;
+                                lnet_get_lnd_timeout();
        smp_mb();                       /* order with setting rx_started */
        conn->ksnc_rx_started = 1;
 
@@ -315,7 +316,7 @@ ksocknal_recv_kiov(struct ksock_conn *conn)
 
        conn->ksnc_peer->ksnp_last_alive = ktime_get_seconds();
        conn->ksnc_rx_deadline = ktime_get_seconds() +
-                                *ksocknal_tunables.ksnd_timeout;
+                                lnet_get_lnd_timeout();
        smp_mb();                       /* order with setting rx_started */
        conn->ksnc_rx_started = 1;
 
@@ -392,19 +393,28 @@ void
 ksocknal_tx_done(struct lnet_ni *ni, struct ksock_tx *tx, int rc)
 {
        struct lnet_msg *lnetmsg = tx->tx_lnetmsg;
+       enum lnet_msg_hstatus hstatus = tx->tx_hstatus;
         ENTRY;
 
        LASSERT(ni != NULL || tx->tx_conn != NULL);
 
-       if (!rc && (tx->tx_resid != 0 || tx->tx_zc_aborted))
+       if (!rc && (tx->tx_resid != 0 || tx->tx_zc_aborted)) {
                rc = -EIO;
+               if (hstatus == LNET_MSG_STATUS_OK)
+                       hstatus = LNET_MSG_STATUS_LOCAL_ERROR;
+       }
 
        if (tx->tx_conn != NULL)
                ksocknal_conn_decref(tx->tx_conn);
 
        ksocknal_free_tx(tx);
-       if (lnetmsg != NULL) /* KSOCK_MSG_NOOP go without lnetmsg */
+       if (lnetmsg != NULL) { /* KSOCK_MSG_NOOP go without lnetmsg */
+               if (rc)
+                       CERROR("tx failure rc = %d, hstatus = %d\n", rc,
+                              hstatus);
+               lnetmsg->msg_health_status = hstatus;
                lnet_finalize(lnetmsg, rc);
+       }
 
        EXIT;
 }
@@ -429,6 +439,22 @@ ksocknal_txlist_done(struct lnet_ni *ni, struct list_head *txlist, int error)
 
                list_del(&tx->tx_list);
 
+               if (tx->tx_hstatus == LNET_MSG_STATUS_OK) {
+                       if (error == -ETIMEDOUT)
+                               tx->tx_hstatus =
+                                 LNET_MSG_STATUS_LOCAL_TIMEOUT;
+                       else if (error == -ENETDOWN ||
+                                error == -EHOSTUNREACH ||
+                                error == -ENETUNREACH)
+                               tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_DROPPED;
+                       /*
+                        * for all other errors we don't want to
+                        * retransmit
+                        */
+                       else if (error)
+                               tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_ERROR;
+               }
+
                LASSERT(atomic_read(&tx->tx_refcount) == 1);
                ksocknal_tx_done(ni, tx, error);
        }
@@ -464,7 +490,7 @@ ksocknal_check_zc_req(struct ksock_tx *tx)
 
         /* ZC_REQ is going to be pinned to the peer_ni */
        tx->tx_deadline = ktime_get_seconds() +
-                         *ksocknal_tunables.ksnd_timeout;
+                         lnet_get_lnd_timeout();
 
         LASSERT (tx->tx_msg.ksm_zc_cookies[0] == 0);
 
@@ -508,6 +534,13 @@ static int
 ksocknal_process_transmit(struct ksock_conn *conn, struct ksock_tx *tx)
 {
        int rc;
+       bool error_sim = false;
+
+       if (lnet_send_error_simulation(tx->tx_lnetmsg, &tx->tx_hstatus)) {
+               error_sim = true;
+               rc = -EINVAL;
+               goto simulate_error;
+       }
 
         if (tx->tx_zc_capable && !tx->tx_zc_checked)
                 ksocknal_check_zc_req(tx);
@@ -546,39 +579,58 @@ ksocknal_process_transmit(struct ksock_conn *conn, struct ksock_tx *tx)
                        wake_up(&ksocknal_data.ksnd_reaper_waitq);
 
                spin_unlock_bh(&ksocknal_data.ksnd_reaper_lock);
+
+               /*
+                * set the health status of the message which determines
+                * whether we should retry the transmit
+                */
+               tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_ERROR;
                return (rc);
        }
 
-        /* Actual error */
-        LASSERT (rc < 0);
+simulate_error:
 
-        if (!conn->ksnc_closing) {
-                switch (rc) {
-                case -ECONNRESET:
+       /* Actual error */
+       LASSERT(rc < 0);
+
+       if (!error_sim) {
+               /*
+               * set the health status of the message which determines
+               * whether we should retry the transmit
+               */
+               if (rc == -ETIMEDOUT)
+                       tx->tx_hstatus = LNET_MSG_STATUS_REMOTE_TIMEOUT;
+               else
+                       tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_ERROR;
+       }
+
+       if (!conn->ksnc_closing) {
+               switch (rc) {
+               case -ECONNRESET:
                        LCONSOLE_WARN("Host %pI4h reset our connection "
-                                      "while we were sending data; it may have "
-                                      "rebooted.\n",
+                                     "while we were sending data; it may have "
+                                     "rebooted.\n",
                                      &conn->ksnc_ipaddr);
-                        break;
-                default:
-                        LCONSOLE_WARN("There was an unexpected network error "
+                       break;
+               default:
+                       LCONSOLE_WARN("There was an unexpected network error "
                                      "while writing to %pI4h: %d.\n",
                                      &conn->ksnc_ipaddr, rc);
-                        break;
-                }
+                       break;
+               }
                CDEBUG(D_NET, "[%p] Error %d on write to %s ip %pI4h:%d\n",
                       conn, rc, libcfs_id2str(conn->ksnc_peer->ksnp_id),
                       &conn->ksnc_ipaddr, conn->ksnc_port);
-        }
+       }
 
-        if (tx->tx_zc_checked)
-                ksocknal_uncheck_zc_req(tx);
+       if (tx->tx_zc_checked)
+               ksocknal_uncheck_zc_req(tx);
 
-        /* it's not an error if conn is being closed */
-        ksocknal_close_conn_and_siblings (conn,
-                                          (conn->ksnc_closing) ? 0 : rc);
+       /* it's not an error if conn is being closed */
+       ksocknal_close_conn_and_siblings(conn,
+                                         (conn->ksnc_closing) ? 0 : rc);
 
-        return (rc);
+       return rc;
 }
 
 static void
@@ -730,7 +782,7 @@ ksocknal_queue_tx_locked(struct ksock_tx *tx, struct ksock_conn *conn)
        if (list_empty(&conn->ksnc_tx_queue) && bufnob == 0) {
                /* First packet starts the timeout */
                conn->ksnc_tx_deadline = ktime_get_seconds() +
-                                        *ksocknal_tunables.ksnd_timeout;
+                                        lnet_get_lnd_timeout();
                if (conn->ksnc_tx_bufnob > 0) /* something got ACKed */
                        conn->ksnc_peer->ksnp_last_alive = ktime_get_seconds();
                conn->ksnc_tx_bufnob = 0;
@@ -907,7 +959,7 @@ ksocknal_launch_packet(struct lnet_ni *ni, struct ksock_tx *tx,
             ksocknal_find_connecting_route_locked (peer_ni) != NULL) {
                 /* the message is going to be pinned to the peer_ni */
                tx->tx_deadline = ktime_get_seconds() +
-                                 *ksocknal_tunables.ksnd_timeout;
+                                 lnet_get_lnd_timeout();
 
                 /* Queue the message until a connection is established */
                list_add_tail(&tx->tx_list, &peer_ni->ksnp_tx_queue);
@@ -1722,7 +1774,7 @@ ksocknal_recv_hello(struct lnet_ni *ni, struct ksock_conn *conn,
        /* socket type set on active connections - not set on passive */
        LASSERT(!active == !(conn->ksnc_type != SOCKLND_CONN_NONE));
 
-       timeout = active ? *ksocknal_tunables.ksnd_timeout :
+       timeout = active ? lnet_get_lnd_timeout() :
                            lnet_acceptor_timeout();
 
        rc = lnet_sock_read(sock, &hello->kshm_magic,
@@ -1857,7 +1909,7 @@ ksocknal_connect(struct ksock_route *route)
         int               retry_later = 0;
         int               rc = 0;
 
-       deadline = ktime_get_seconds() + *ksocknal_tunables.ksnd_timeout;
+       deadline = ktime_get_seconds() + lnet_get_lnd_timeout();
 
        write_lock_bh(&ksocknal_data.ksnd_global_lock);
 
@@ -2132,7 +2184,7 @@ ksocknal_connd(void *arg)
 {
        spinlock_t *connd_lock = &ksocknal_data.ksnd_connd_lock;
        struct ksock_connreq *cr;
-       wait_queue_t wait;
+       wait_queue_entry_t wait;
        int nloops = 0;
        int cons_retry = 0;
 
@@ -2246,6 +2298,7 @@ ksocknal_find_timed_out_conn(struct ksock_peer_ni *peer_ni)
         /* We're called with a shared lock on ksnd_global_lock */
        struct ksock_conn *conn;
        struct list_head *ctmp;
+       struct ksock_tx *tx;
 
        list_for_each(ctmp, &peer_ni->ksnp_conns) {
                int error;
@@ -2310,6 +2363,10 @@ ksocknal_find_timed_out_conn(struct ksock_peer_ni *peer_ni)
                         /* Timed out messages queued for sending or
                          * buffered in the socket's send buffer */
                         ksocknal_conn_addref(conn);
+                       list_for_each_entry(tx, &conn->ksnc_tx_queue,
+                                           tx_list)
+                               tx->tx_hstatus =
+                                       LNET_MSG_STATUS_LOCAL_TIMEOUT;
                        CNETERR("Timeout sending data to %s (%pI4h:%d) "
                                 "the network or that node may be down.\n",
                                 libcfs_id2str(peer_ni->ksnp_id),
@@ -2336,6 +2393,8 @@ ksocknal_flush_stale_txs(struct ksock_peer_ni *peer_ni)
                if (ktime_get_seconds() < tx->tx_deadline)
                        break;
 
+               tx->tx_hstatus = LNET_MSG_STATUS_LOCAL_TIMEOUT;
+
                list_del(&tx->tx_list);
                list_add_tail(&tx->tx_list, &stale_txs);
        }
@@ -2510,7 +2569,7 @@ ksocknal_check_peer_timeouts(int idx)
 
 int ksocknal_reaper(void *arg)
 {
-       wait_queue_t wait;
+       wait_queue_entry_t wait;
        struct ksock_conn *conn;
        struct ksock_sched *sched;
        struct list_head enomem_conns;
@@ -2589,6 +2648,7 @@ int ksocknal_reaper(void *arg)
                         const int n = 4;
                         const int p = 1;
                         int       chunk = ksocknal_data.ksnd_peer_hash_size;
+                       unsigned int lnd_timeout;
 
                         /* Time to check for timeouts on a few more peers: I do
                          * checks every 'p' seconds on a proportion of the peer_ni
@@ -2597,11 +2657,11 @@ int ksocknal_reaper(void *arg)
                          * timeout on any connection within (n+1)/n times the
                          * timeout interval. */
 
-                        if (*ksocknal_tunables.ksnd_timeout > n * p)
-                                chunk = (chunk * n * p) /
-                                        *ksocknal_tunables.ksnd_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++) {
                                 ksocknal_check_peer_timeouts (peer_index);
@@ -2627,7 +2687,7 @@ int ksocknal_reaper(void *arg)
                if (!ksocknal_data.ksnd_shuttingdown &&
                    list_empty(&ksocknal_data.ksnd_deathrow_conns) &&
                    list_empty(&ksocknal_data.ksnd_zombie_conns))
-                       schedule_timeout(cfs_duration_sec(timeout));
+                       schedule_timeout(cfs_time_seconds(timeout));
 
                set_current_state(TASK_RUNNING);
                remove_wait_queue(&ksocknal_data.ksnd_reaper_waitq, &wait);