[PATCH for-next 09/24] IB/hfi1: Integrate TID RDMA READ protocol into RC protocol

[Date Prev][Date Next][Thread Prev][Thread Next][Date Index][Thread Index]

 



From: Kaike Wan <kaike.wan@xxxxxxxxx>

This patch integrates the TID RDMA READ protocol into the IB RC protocol.
This protocol is an end-to-end protocol between the hfi1 drivers on two
OPA nodes that converts a qualified RDMA READ request into a TID RDMA
READ request to avoid data copying on the requester side.

Reviewed-by: Mike Marciniszyn <mike.marciniszyn@xxxxxxxxx>
Signed-off-by: Kaike Wan <kaike.wan@xxxxxxxxx>
Signed-off-by: Dennis Dalessandro <dennis.dalessandro@xxxxxxxxx>
---
 drivers/infiniband/hw/hfi1/rc.c       |  370 ++++++++++++++++++++++++++++++---
 drivers/infiniband/hw/hfi1/tid_rdma.c |  186 +++++++++++++++++
 drivers/infiniband/hw/hfi1/tid_rdma.h |   14 +
 drivers/infiniband/hw/hfi1/verbs.c    |    1 
 drivers/infiniband/hw/hfi1/verbs.h    |   21 ++
 drivers/infiniband/sw/rdmavt/qp.c     |   11 +
 include/rdma/rdma_vt.h                |   14 +
 include/rdma/rdmavt_qp.h              |    8 +
 8 files changed, 581 insertions(+), 44 deletions(-)

diff --git a/drivers/infiniband/hw/hfi1/rc.c b/drivers/infiniband/hw/hfi1/rc.c
index 4e1abd3..dee631f 100644
--- a/drivers/infiniband/hw/hfi1/rc.c
+++ b/drivers/infiniband/hw/hfi1/rc.c
@@ -112,12 +112,14 @@ static int make_rc_ack(struct hfi1_ibdev *dev, struct rvt_qp *qp,
 {
 	struct rvt_ack_entry *e;
 	u32 hwords;
-	u32 len;
-	u32 bth0, bth2;
+	u32 len = 0;
+	u32 bth0 = 0, bth2 = 0;
 	u32 bth1 = qp->remote_qpn | (HFI1_CAP_IS_KSET(OPFN) << IB_BTHE_E_SHIFT);
 	int middle = 0;
 	u32 pmtu = qp->pmtu;
 	struct hfi1_qp_priv *priv = qp->priv;
+	bool last_pkt;
+	u32 delta;
 
 	lockdep_assert_held(&qp->s_lock);
 	/* Don't send an ACK if we aren't supposed to. */
@@ -189,6 +191,26 @@ static int make_rc_ack(struct hfi1_ibdev *dev, struct rvt_qp *qp,
 			hwords++;
 			qp->s_ack_rdma_psn = e->psn;
 			bth2 = mask_psn(qp->s_ack_rdma_psn++);
+		} else if (e->opcode == TID_OP(READ_REQ)) {
+			/*
+			 * If a TID RDMA read response is being resent and
+			 * we haven't seen the duplicate request yet,
+			 * then stop sending the remaining responses the
+			 * responder has seen until the requester re-sends it.
+			 */
+			len = e->rdma_sge.sge_length;
+			if (len && !e->rdma_sge.mr) {
+				qp->s_tail_ack_queue = qp->r_head_ack_queue;
+				goto bail;
+			}
+			/* Copy SGE state in case we need to resend */
+			ps->s_txreq->mr = e->rdma_sge.mr;
+			if (ps->s_txreq->mr)
+				rvt_get_mr(ps->s_txreq->mr);
+			qp->s_ack_rdma_sge.sge = e->rdma_sge;
+			qp->s_ack_rdma_sge.num_sge = 1;
+			qp->s_ack_state = TID_OP(READ_RESP);
+			goto read_resp;
 		} else {
 			/* COMPARE_SWAP or FETCH_ADD */
 			ps->s_txreq->ss = NULL;
@@ -226,6 +248,28 @@ static int make_rc_ack(struct hfi1_ibdev *dev, struct rvt_qp *qp,
 		bth2 = mask_psn(qp->s_ack_rdma_psn++);
 		break;
 
+	case TID_OP(READ_RESP):
+read_resp:
+		e = &qp->s_ack_queue[qp->s_tail_ack_queue];
+		ps->s_txreq->ss = &qp->s_ack_rdma_sge;
+		delta = hfi1_build_tid_rdma_read_resp(qp, e, ohdr, &bth0,
+						      &bth1, &bth2, &len,
+						      &last_pkt);
+		if (delta == 0)
+			goto error_qp;
+		hwords += delta;
+		if (last_pkt) {
+			e->sent = 1;
+			/*
+			 * Increment qp->s_tail_ack_queue through s_ack_state
+			 * transition.
+			 */
+			qp->s_ack_state = OP(RDMA_READ_RESPONSE_LAST);
+		}
+		break;
+	case TID_OP(READ_REQ):
+		goto bail;
+
 	default:
 normal:
 		/*
@@ -255,7 +299,14 @@ static int make_rc_ack(struct hfi1_ibdev *dev, struct rvt_qp *qp,
 	ps->s_txreq->hdr_dwords = hwords;
 	hfi1_make_ruc_header(qp, ohdr, bth0, bth1, bth2, middle, ps);
 	return 1;
-
+error_qp:
+	spin_unlock_irqrestore(&qp->s_lock, ps->flags);
+	spin_lock_irqsave(&qp->r_lock, ps->flags);
+	spin_lock(&qp->s_lock);
+	rvt_error_qp(qp, IB_WC_WR_FLUSH_ERR);
+	spin_unlock(&qp->s_lock);
+	spin_unlock_irqrestore(&qp->r_lock, ps->flags);
+	spin_lock_irqsave(&qp->s_lock, ps->flags);
 bail:
 	qp->s_ack_state = OP(ACKNOWLEDGE);
 	/*
@@ -282,16 +333,20 @@ int hfi1_make_rc_req(struct rvt_qp *qp, struct hfi1_pkt_state *ps)
 	struct hfi1_qp_priv *priv = qp->priv;
 	struct hfi1_ibdev *dev = to_idev(qp->ibqp.device);
 	struct ib_other_headers *ohdr;
-	struct rvt_sge_state *ss;
+	struct rvt_sge_state *ss = NULL;
 	struct rvt_swqe *wqe;
-	u32 hwords;
-	u32 len;
-	u32 bth0 = 0, bth2;
+	struct hfi1_swqe_priv *wpriv;
+	struct tid_rdma_request *req = NULL;
+	/* header size in 32-bit words LRH+BTH = (8+12)/4. */
+	u32 hwords = 5;
+	u32 len = 0;
+	u32 bth0 = 0, bth2 = 0;
 	u32 bth1 = qp->remote_qpn | (HFI1_CAP_IS_KSET(OPFN) << IB_BTHE_E_SHIFT);
 	u32 pmtu = qp->pmtu;
 	char newreq;
 	int middle = 0;
 	int delta;
+	struct tid_rdma_flow *flow = NULL;
 
 	lockdep_assert_held(&qp->s_lock);
 	ps->s_txreq = get_txreq(ps->dev, qp);
@@ -353,6 +408,7 @@ int hfi1_make_rc_req(struct rvt_qp *qp, struct hfi1_pkt_state *ps)
 
 	/* Send a request. */
 	wqe = rvt_get_swqe_ptr(qp, qp->s_cur);
+check_s_state:
 	switch (qp->s_state) {
 	default:
 		if (!(ib_rvt_state_ops[qp->state] & RVT_PROCESS_NEXT_SEND_OK))
@@ -374,9 +430,13 @@ int hfi1_make_rc_req(struct rvt_qp *qp, struct hfi1_pkt_state *ps)
 			/*
 			 * If a fence is requested, wait for previous
 			 * RDMA read and atomic operations to finish.
+			 * However, this is no need to guard against
+			 * TID RDMA READ after TID RDMA READ.
 			 */
 			if ((wqe->wr.send_flags & IB_SEND_FENCE) &&
-			    qp->s_num_rd_atomic) {
+			    qp->s_num_rd_atomic &&
+			    (wqe->wr.opcode != IB_WR_TID_RDMA_READ ||
+			     priv->pending_tid_r_segs < qp->s_num_rd_atomic)) {
 				qp->s_flags |= RVT_S_WAIT_FENCE;
 				goto bail;
 			}
@@ -502,16 +562,14 @@ int hfi1_make_rc_req(struct rvt_qp *qp, struct hfi1_pkt_state *ps)
 			 * Don't allow more operations to be started
 			 * than the QP limits allow.
 			 */
-			if (newreq) {
-				if (qp->s_num_rd_atomic >=
-				    qp->s_max_rd_atomic) {
-					qp->s_flags |= RVT_S_WAIT_RDMAR;
-					goto bail;
-				}
-				qp->s_num_rd_atomic++;
-				if (!(qp->s_flags & RVT_S_UNLIMITED_CREDIT))
-					qp->s_lsn++;
+			if (qp->s_num_rd_atomic >=
+			    qp->s_max_rd_atomic) {
+				qp->s_flags |= RVT_S_WAIT_RDMAR;
+				goto bail;
 			}
+			qp->s_num_rd_atomic++;
+			if (newreq && !(qp->s_flags & RVT_S_UNLIMITED_CREDIT))
+				qp->s_lsn++;
 			put_ib_reth_vaddr(
 				wqe->rdma_wr.remote_addr,
 				&ohdr->u.rc.reth);
@@ -527,6 +585,75 @@ int hfi1_make_rc_req(struct rvt_qp *qp, struct hfi1_pkt_state *ps)
 				qp->s_cur = 0;
 			break;
 
+		case IB_WR_TID_RDMA_READ:
+			wpriv = wqe->priv;
+			req = wqe_to_tid_req(wqe);
+			delta = cmp_psn(qp->s_psn, wqe->psn);
+
+			/*
+			 * Don't allow more operations to be started
+			 * than the QP limits allow. We could get here under
+			 * three conditions; (1) It's a new request; (2) We are
+			 * sending the second or later segment of a request,
+			 * but the qp->s_state is set to OP(RDMA_READ_REQUEST)
+			 * when the last segment of a previous request is
+			 * received just before this; (3) We are re-sending a
+			 * request.
+			 */
+			if (qp->s_num_rd_atomic >= qp->s_max_rd_atomic) {
+				qp->s_flags |= RVT_S_WAIT_RDMAR;
+				goto bail;
+			}
+			if (newreq) {
+				struct tid_rdma_flow *flow =
+					&req->flows[req->setup_head];
+
+				/*
+				 * Set up s_sge as it is needed for TID
+				 * allocation. However, if the pages have been
+				 * walked and mapped, skip it. An earlier try
+				 * has failed to allocate the TID entries.
+				 */
+				if (!flow->npagesets) {
+					qp->s_sge.sge = wqe->sg_list[0];
+					qp->s_sge.sg_list = wqe->sg_list + 1;
+					qp->s_sge.num_sge = wqe->wr.num_sge;
+					qp->s_sge.total_len = wqe->length;
+					qp->s_len = wqe->length;
+					req->isge = 0;
+					req->clear_tail = req->setup_head;
+					req->flow_idx = req->setup_head;
+					req->state = TID_REQUEST_ACTIVE;
+				}
+			} else if (delta == 0) {
+				/* Re-send a request */
+				req->cur_seg = 0;
+				req->comp_seg = 0;
+				req->ack_pending = 0;
+				req->flow_idx = req->clear_tail;
+				req->state = TID_REQUEST_RESEND;
+			}
+			req->s_next_psn = qp->s_psn;
+			/* Read one segment at a time */
+			len = min_t(u32, req->seg_len,
+				    wqe->length - req->seg_len * req->cur_seg);
+			delta = hfi1_build_tid_rdma_read_req(qp, wqe, ohdr,
+							     &bth1, &bth2,
+							     &len);
+			if (delta <= 0) {
+				/* Wait for TID space */
+				goto bail;
+			}
+			if (newreq && !(qp->s_flags & RVT_S_UNLIMITED_CREDIT))
+				qp->s_lsn++;
+			hwords += delta;
+			ss = &wpriv->ss;
+			/* Check if this is the last segment */
+			if (req->cur_seg >= req->total_segs &&
+			    ++qp->s_cur == qp->s_size)
+				qp->s_cur = 0;
+			break;
+
 		case IB_WR_ATOMIC_CMP_AND_SWP:
 		case IB_WR_ATOMIC_FETCH_AND_ADD:
 			/*
@@ -572,11 +699,13 @@ int hfi1_make_rc_req(struct rvt_qp *qp, struct hfi1_pkt_state *ps)
 		default:
 			goto bail;
 		}
-		qp->s_sge.sge = wqe->sg_list[0];
-		qp->s_sge.sg_list = wqe->sg_list + 1;
-		qp->s_sge.num_sge = wqe->wr.num_sge;
-		qp->s_sge.total_len = wqe->length;
-		qp->s_len = wqe->length;
+		if (wqe->wr.opcode != IB_WR_TID_RDMA_READ) {
+			qp->s_sge.sge = wqe->sg_list[0];
+			qp->s_sge.sg_list = wqe->sg_list + 1;
+			qp->s_sge.num_sge = wqe->wr.num_sge;
+			qp->s_sge.total_len = wqe->length;
+			qp->s_len = wqe->length;
+		}
 		if (newreq) {
 			qp->s_tail++;
 			if (qp->s_tail >= qp->s_size)
@@ -584,6 +713,8 @@ int hfi1_make_rc_req(struct rvt_qp *qp, struct hfi1_pkt_state *ps)
 		}
 		if (wqe->wr.opcode == IB_WR_RDMA_READ)
 			qp->s_psn = wqe->lpsn + 1;
+		else if (wqe->wr.opcode == IB_WR_TID_RDMA_READ)
+			qp->s_psn = req->s_next_psn;
 		else
 			qp->s_psn++;
 		break;
@@ -700,6 +831,100 @@ int hfi1_make_rc_req(struct rvt_qp *qp, struct hfi1_pkt_state *ps)
 		if (qp->s_cur == qp->s_size)
 			qp->s_cur = 0;
 		break;
+
+	case TID_OP(READ_RESP):
+		if (wqe->wr.opcode != IB_WR_TID_RDMA_READ)
+			goto bail;
+		/* This is used to restart a TID read request */
+		req = wqe_to_tid_req(wqe);
+		wpriv = wqe->priv;
+		/*
+		 * Back down. The field qp->s_psn has been set to the psn with
+		 * which the request should be restart. It's OK to use division
+		 * as this is on the retry path.
+		 */
+		req->cur_seg = delta_psn(qp->s_psn, wqe->psn) / priv->pkts_ps;
+
+		/*
+		 * The following function need to be redefined to return the
+		 * status to make sure that we find the flow. At the same
+		 * time, we can use the req->state change to check if the
+		 * call succeeds or not.
+		 */
+		req->state = TID_REQUEST_RESEND;
+		hfi1_tid_rdma_restart_req(qp, wqe, &bth2);
+		if (req->state != TID_REQUEST_ACTIVE) {
+			/*
+			 * Failed to find the flow. Release all allocated tid
+			 * resources.
+			 */
+			hfi1_kern_exp_rcv_clear_all(req);
+			hfi1_kern_clear_hw_flow(priv->rcd, qp);
+
+			hfi1_send_complete(qp, wqe, IB_WC_LOC_QP_OP_ERR);
+			goto bail;
+		}
+		req->state = TID_REQUEST_RESEND;
+		len = min_t(u32, req->seg_len,
+			    wqe->length - req->seg_len * req->cur_seg);
+		flow = &req->flows[req->flow_idx];
+		len -= flow->sent;
+		req->s_next_psn = flow->flow_state.ib_lpsn + 1;
+		delta = hfi1_build_tid_rdma_read_packet(wqe, ohdr, &bth1,
+							&bth2, &len);
+		if (delta <= 0) {
+			/* Wait for TID space */
+			goto bail;
+		}
+		hwords += delta;
+		ss = &wpriv->ss;
+		/* Check if this is the last segment */
+		if (req->cur_seg >= req->total_segs &&
+		    ++qp->s_cur == qp->s_size)
+			qp->s_cur = 0;
+		qp->s_psn = req->s_next_psn;
+		break;
+	case TID_OP(READ_REQ):
+		req = wqe_to_tid_req(wqe);
+		delta = cmp_psn(qp->s_psn, wqe->psn);
+		/*
+		 * If the current WR is not TID RDMA READ, or this is the start
+		 * of a new request, we need to change the qp->s_state so that
+		 * the request can be set up properly.
+		 */
+		if (wqe->wr.opcode != IB_WR_TID_RDMA_READ || delta == 0 ||
+		    qp->s_cur == qp->s_tail) {
+			qp->s_state = OP(RDMA_READ_REQUEST);
+			if (delta == 0 || qp->s_cur == qp->s_tail)
+				goto check_s_state;
+			else
+				goto bail;
+		}
+
+		/* Rate limiting */
+		if (qp->s_num_rd_atomic >= qp->s_max_rd_atomic) {
+			qp->s_flags |= RVT_S_WAIT_RDMAR;
+			goto bail;
+		}
+
+		wpriv = wqe->priv;
+		/* Read one segment at a time */
+		len = min_t(u32, req->seg_len,
+			    wqe->length - req->seg_len * req->cur_seg);
+		delta = hfi1_build_tid_rdma_read_req(qp, wqe, ohdr, &bth1,
+						     &bth2, &len);
+		if (delta <= 0) {
+			/* Wait for TID space */
+			goto bail;
+		}
+		hwords += delta;
+		ss = &wpriv->ss;
+		/* Check if this is the last segment */
+		if (req->cur_seg >= req->total_segs &&
+		    ++qp->s_cur == qp->s_size)
+			qp->s_cur = 0;
+		qp->s_psn = req->s_next_psn;
+		break;
 	}
 	qp->s_sending_hpsn = bth2;
 	delta = delta_psn(bth2, wqe->psn);
@@ -968,6 +1193,43 @@ void hfi1_send_rc_ack(struct hfi1_packet *packet, bool is_fecn)
 }
 
 /**
+ * update_num_rd_atomic - update the qp->s_num_rd_atomic
+ * @qp: the QP
+ * @psn: the packet sequence number to restart at
+ * @wqe: the wqe
+ *
+ * This is called from reset_psn() to update qp->s_num_rd_atomic
+ * for the current wqe.
+ * Called at interrupt level with the QP s_lock held.
+ */
+static void update_num_rd_atomic(struct rvt_qp *qp, u32 psn,
+				 struct rvt_swqe *wqe)
+{
+	u32 opcode = wqe->wr.opcode;
+
+	if (opcode == IB_WR_RDMA_READ ||
+	    opcode == IB_WR_ATOMIC_CMP_AND_SWP ||
+	    opcode == IB_WR_ATOMIC_FETCH_AND_ADD) {
+		qp->s_num_rd_atomic++;
+	} else if (opcode == IB_WR_TID_RDMA_READ) {
+		struct tid_rdma_request *req = wqe_to_tid_req(wqe);
+		struct hfi1_qp_priv *priv = qp->priv;
+
+		if (cmp_psn(psn, wqe->lpsn) <= 0) {
+			u32 cur_seg;
+
+			cur_seg = (psn - wqe->psn) / priv->pkts_ps;
+			req->ack_pending = cur_seg - req->comp_seg;
+			priv->pending_tid_r_segs += req->ack_pending;
+			qp->s_num_rd_atomic += req->ack_pending;
+		} else {
+			priv->pending_tid_r_segs += req->total_segs;
+			qp->s_num_rd_atomic += req->total_segs;
+		}
+	}
+}
+
+/**
  * reset_psn - reset the QP state to send starting from PSN
  * @qp: the QP
  * @psn: the packet sequence number to restart at
@@ -981,9 +1243,12 @@ static void reset_psn(struct rvt_qp *qp, u32 psn)
 	u32 n = qp->s_acked;
 	struct rvt_swqe *wqe = rvt_get_swqe_ptr(qp, n);
 	u32 opcode;
+	struct hfi1_qp_priv *priv = qp->priv;
 
 	lockdep_assert_held(&qp->s_lock);
 	qp->s_cur = n;
+	priv->pending_tid_r_segs = 0;
+	qp->s_num_rd_atomic = 0;
 
 	/*
 	 * If we are starting the request from the beginning,
@@ -993,9 +1258,9 @@ static void reset_psn(struct rvt_qp *qp, u32 psn)
 		qp->s_state = OP(SEND_LAST);
 		goto done;
 	}
+	update_num_rd_atomic(qp, psn, wqe);
 
 	/* Find the work request opcode corresponding to the given PSN. */
-	opcode = wqe->wr.opcode;
 	for (;;) {
 		int diff;
 
@@ -1005,8 +1270,11 @@ static void reset_psn(struct rvt_qp *qp, u32 psn)
 			break;
 		wqe = rvt_get_swqe_ptr(qp, n);
 		diff = cmp_psn(psn, wqe->psn);
-		if (diff < 0)
+		if (diff < 0) {
+			/* Point wqe back to the previous one*/
+			wqe = rvt_get_swqe_ptr(qp, qp->s_cur);
 			break;
+		}
 		qp->s_cur = n;
 		/*
 		 * If we are starting the request from the beginning,
@@ -1016,8 +1284,10 @@ static void reset_psn(struct rvt_qp *qp, u32 psn)
 			qp->s_state = OP(SEND_LAST);
 			goto done;
 		}
-		opcode = wqe->wr.opcode;
+
+		update_num_rd_atomic(qp, psn, wqe);
 	}
+	opcode = wqe->wr.opcode;
 
 	/*
 	 * Set the state to restart in the middle of a request.
@@ -1039,6 +1309,10 @@ static void reset_psn(struct rvt_qp *qp, u32 psn)
 		qp->s_state = OP(RDMA_READ_RESPONSE_MIDDLE);
 		break;
 
+	case IB_WR_TID_RDMA_READ:
+		qp->s_state = TID_OP(READ_RESP);
+		break;
+
 	default:
 		/*
 		 * This case shouldn't happen since its only
@@ -1092,6 +1366,14 @@ void hfi1_restart_rc(struct rvt_qp *qp, u32 psn, int wait)
 				wqe = do_rc_completion(qp, wqe, ibp);
 				qp->s_flags &= ~RVT_S_WAIT_ACK;
 			} else {
+				if (wqe->wr.opcode == IB_WR_TID_RDMA_READ) {
+					struct tid_rdma_request *req;
+
+					req = wqe_to_tid_req(wqe);
+					hfi1_kern_exp_rcv_clear_all(req);
+					hfi1_kern_clear_hw_flow(priv->rcd, qp);
+				}
+
 				hfi1_send_complete(qp, wqe,
 						   IB_WC_RETRY_EXC_ERR);
 				rvt_error_qp(qp, IB_WC_WR_FLUSH_ERR);
@@ -1105,7 +1387,8 @@ void hfi1_restart_rc(struct rvt_qp *qp, u32 psn, int wait)
 	}
 
 	ibp = to_iport(qp->ibqp.device, qp->port_num);
-	if (wqe->wr.opcode == IB_WR_RDMA_READ)
+	if (wqe->wr.opcode == IB_WR_RDMA_READ ||
+	    wqe->wr.opcode == IB_WR_TID_RDMA_READ)
 		ibp->rvp.n_rc_resends++;
 	else
 		ibp->rvp.n_rc_resends += delta_psn(qp->s_psn, psn);
@@ -1132,7 +1415,8 @@ static void reset_sending_psn(struct rvt_qp *qp, u32 psn)
 	for (;;) {
 		wqe = rvt_get_swqe_ptr(qp, n);
 		if (cmp_psn(psn, wqe->lpsn) <= 0) {
-			if (wqe->wr.opcode == IB_WR_RDMA_READ)
+			if (wqe->wr.opcode == IB_WR_RDMA_READ ||
+			    wqe->wr.opcode == IB_WR_TID_RDMA_READ)
 				qp->s_sending_psn = wqe->lpsn + 1;
 			else
 				qp->s_sending_psn = psn + 1;
@@ -1181,8 +1465,9 @@ void hfi1_rc_send_complete(struct rvt_qp *qp, struct hfi1_opa_header *opah)
 	}
 
 	opcode = ib_bth_get_opcode(ohdr);
-	if (opcode >= OP(RDMA_READ_RESPONSE_FIRST) &&
-	    opcode <= OP(ATOMIC_ACKNOWLEDGE)) {
+	if ((opcode >= OP(RDMA_READ_RESPONSE_FIRST) &&
+	     opcode <= OP(ATOMIC_ACKNOWLEDGE)) ||
+	    opcode == TID_OP(READ_RESP)) {
 		WARN_ON(!qp->s_rdma_ack_cnt);
 		qp->s_rdma_ack_cnt--;
 		return;
@@ -1198,8 +1483,12 @@ void hfi1_rc_send_complete(struct rvt_qp *qp, struct hfi1_opa_header *opah)
 	if ((psn & IB_BTH_REQ_ACK) && qp->s_acked != qp->s_tail &&
 	    !(qp->s_flags &
 		(RVT_S_TIMER | RVT_S_WAIT_RNR | RVT_S_WAIT_PSN)) &&
-		(ib_rvt_state_ops[qp->state] & RVT_PROCESS_RECV_OK))
-		rvt_add_retry_timer(qp);
+		(ib_rvt_state_ops[qp->state] & RVT_PROCESS_RECV_OK)) {
+		if (opcode == TID_OP(READ_REQ))
+			rvt_add_retry_timer_ext(qp, priv->timeout_shift);
+		else
+			rvt_add_retry_timer(qp);
+	}
 
 	while (qp->s_last != qp->s_acked) {
 		u32 s_last;
@@ -1334,6 +1623,7 @@ int do_rc_ack(struct rvt_qp *qp, u32 aeth, u32 psn, int opcode,
 {
 	struct hfi1_ibport *ibp;
 	enum ib_wc_status status;
+	struct hfi1_qp_priv *qpriv = qp->priv;
 	struct rvt_swqe *wqe;
 	int ret = 0;
 	u32 ack_psn;
@@ -1380,6 +1670,8 @@ int do_rc_ack(struct rvt_qp *qp, u32 aeth, u32 psn, int opcode,
 		 */
 		if ((wqe->wr.opcode == IB_WR_RDMA_READ &&
 		     (opcode != OP(RDMA_READ_RESPONSE_LAST) || diff != 0)) ||
+		    (wqe->wr.opcode == IB_WR_TID_RDMA_READ &&
+		     (opcode != TID_OP(READ_RESP) || diff != 0)) ||
 		    ((wqe->wr.opcode == IB_WR_ATOMIC_CMP_AND_SWP ||
 		      wqe->wr.opcode == IB_WR_ATOMIC_FETCH_AND_ADD) &&
 		     (opcode != OP(ATOMIC_ACKNOWLEDGE) || diff != 0))) {
@@ -1433,7 +1725,13 @@ int do_rc_ack(struct rvt_qp *qp, u32 aeth, u32 psn, int opcode,
 	switch (aeth >> IB_AETH_NAK_SHIFT) {
 	case 0:         /* ACK */
 		this_cpu_inc(*ibp->rvp.rc_acks);
-		if (qp->s_acked != qp->s_tail) {
+		if (wqe->wr.opcode == IB_WR_TID_RDMA_READ) {
+			if (wqe_to_tid_req(wqe)->ack_pending)
+				rvt_mod_retry_timer_ext(qp,
+							qpriv->timeout_shift);
+			else
+				rvt_stop_rc_timers(qp);
+		} else if (qp->s_acked != qp->s_tail) {
 			/*
 			 * We are expecting more ACKs so
 			 * mod the retry timer.
@@ -1522,6 +1820,9 @@ int do_rc_ack(struct rvt_qp *qp, u32 aeth, u32 psn, int opcode,
 			ibp->rvp.n_other_naks++;
 class_b:
 			if (qp->s_last == qp->s_acked) {
+				if (wqe->wr.opcode == IB_WR_TID_RDMA_READ)
+					hfi1_kern_read_tid_flow_free(qp);
+
 				hfi1_send_complete(qp, wqe, status);
 				rvt_error_qp(qp, IB_WC_WR_FLUSH_ERR);
 			}
@@ -1563,6 +1864,7 @@ static void rdma_seq_err(struct rvt_qp *qp, struct hfi1_ibport *ibp, u32 psn,
 
 	while (cmp_psn(psn, wqe->lpsn) > 0) {
 		if (wqe->wr.opcode == IB_WR_RDMA_READ ||
+		    wqe->wr.opcode == IB_WR_TID_RDMA_READ ||
 		    wqe->wr.opcode == IB_WR_ATOMIC_CMP_AND_SWP ||
 		    wqe->wr.opcode == IB_WR_ATOMIC_FETCH_AND_ADD)
 			break;
@@ -2315,7 +2617,7 @@ void hfi1_rc_rcv(struct hfi1_packet *packet)
 			update_ack_queue(qp, next);
 		}
 		e = &qp->s_ack_queue[qp->r_head_ack_queue];
-		if (e->opcode == OP(RDMA_READ_REQUEST) && e->rdma_sge.mr) {
+		if (e->rdma_sge.mr) {
 			rvt_put_mr(e->rdma_sge.mr);
 			e->rdma_sge.mr = NULL;
 		}
@@ -2392,7 +2694,7 @@ void hfi1_rc_rcv(struct hfi1_packet *packet)
 			update_ack_queue(qp, next);
 		}
 		e = &qp->s_ack_queue[qp->r_head_ack_queue];
-		if (e->opcode == OP(RDMA_READ_REQUEST) && e->rdma_sge.mr) {
+		if (e->rdma_sge.mr) {
 			rvt_put_mr(e->rdma_sge.mr);
 			e->rdma_sge.mr = NULL;
 		}
diff --git a/drivers/infiniband/hw/hfi1/tid_rdma.c b/drivers/infiniband/hw/hfi1/tid_rdma.c
index fcc1ab7..4e483c5 100644
--- a/drivers/infiniband/hw/hfi1/tid_rdma.c
+++ b/drivers/infiniband/hw/hfi1/tid_rdma.c
@@ -49,6 +49,22 @@
 #include "hfi.h"
 #include "verbs.h"
 #include "tid_rdma.h"
+#include "user_exp_rcv.h"
+
+/**
+ * DOC: TID RDMA READ protocol
+ *
+ * This is an end-to-end protocol at the hfi1 level between two nodes that
+ * improves performance by avoiding data copy on the requester side. It
+ * converts a qualified RDMA READ request into a TID RDMA READ request on
+ * the requester side and thereafter handles the request and response
+ * differently. To be qualified, the RDMA READ request should meet the
+ * following:
+ * -- The total data length should be greater than 256K;
+ * -- The total data length should be a multiple of 4K page size;
+ * -- Each local scatter-gather entry should be 4K page aligned;
+ * -- Each local scatter-gather entry should be a multiple of 4K page size;
+ */
 
 #define MAX_EXPECTED_PAGES     (MAX_EXPECTED_BUFFER / PAGE_SIZE)
 
@@ -130,6 +146,27 @@ static inline u32 mask_generation(u32 a)
  * C - Capcode
  */
 
+static struct tid_rdma_flow *find_flow_ib(struct tid_rdma_request *req,
+					  u32 psn, u16 *fidx)
+{
+	u16 head, tail;
+	struct tid_rdma_flow *flow;
+
+	head = req->setup_head;
+	tail = req->clear_tail;
+	for ( ; CIRC_CNT(head, tail, req->n_max_flows);
+	     tail = CIRC_NEXT(tail, req->n_max_flows)) {
+		flow = &req->flows[tail];
+		if (cmp_psn(psn, flow->flow_state.ib_spsn) >= 0 &&
+		    cmp_psn(psn, flow->flow_state.ib_lpsn) <= 0) {
+			if (fidx)
+				*fidx = tail;
+			return flow;
+		}
+	}
+	return NULL;
+}
+
 static u64 tid_rdma_opfn_encode(struct tid_rdma_params *p)
 {
 	return
@@ -279,6 +316,15 @@ static void tid_rdma_trigger_resume(struct work_struct *work)
 {
 }
 
+void hfi1_kern_clear_hw_flow(struct hfi1_ctxtdata *rcd, struct rvt_qp *qp)
+{
+}
+
+void hfi1_kern_exp_rcv_clear_all(struct tid_rdma_request *req)
+	__must_hold(&req->qp->s_lock)
+{
+}
+
 void hfi1_rc_rcv_tid_rdma_write_req(struct hfi1_packet *packet)
 {
 }
@@ -307,6 +353,11 @@ void hfi1_rc_rcv_tid_rdma_ack(struct hfi1_packet *packet)
 {
 }
 
+void hfi1_kern_read_tid_flow_free(struct rvt_qp *qp)
+	__must_hold(&qp->s_lock)
+{
+}
+
 bool hfi1_handle_kdeth_eflags(struct hfi1_ctxtdata *rcd,
 			      struct hfi1_pportdata *ppd,
 			      struct hfi1_packet *packet)
@@ -347,6 +398,7 @@ int hfi1_qp_priv_init(struct rvt_dev_info *rdi, struct rvt_qp *qp,
 		      struct ib_qp_init_attr *init_attr)
 {
 	struct hfi1_qp_priv *qpriv = qp->priv;
+	int i;
 
 	qpriv->rcd = qp_to_rcd(rdi, qp);
 
@@ -366,6 +418,41 @@ int hfi1_qp_priv_init(struct rvt_dev_info *rdi, struct rvt_qp *qp,
 					    GFP_KERNEL, dd->node);
 		if (!qpriv->pages)
 			return -ENOMEM;
+		for (i = 0; i < qp->s_size; i++) {
+			struct hfi1_swqe_priv *priv;
+			struct rvt_swqe *wqe = rvt_get_swqe_ptr(qp, i);
+
+			priv = kzalloc(sizeof(*priv), GFP_KERNEL);
+			if (!priv)
+				return -ENOMEM;
+
+			/*
+			 * Initialize various TID RDMA request variables.
+			 * These variables are "static", which is why they
+			 * can be pre-initialized here before the WRs has
+			 * even been submitted.
+			 * However, non-NULL values for these variables do not
+			 * imply that this WQE has been enabled for TID RDMA.
+			 * Drivers should check the WQE's opcode to determine
+			 * if a request is a TID RDMA one or not.
+			 */
+			priv->tid_req.qp = qp;
+			priv->tid_req.rcd = qpriv->rcd;
+			priv->tid_req.e.swqe = wqe;
+			wqe->priv = priv;
+		}
+		for (i = 0; i < rvt_max_atomic(rdi); i++) {
+			struct hfi1_ack_priv *priv;
+
+			priv = kzalloc(sizeof(*priv), GFP_KERNEL);
+			if (!priv)
+				return -ENOMEM;
+
+			priv->tid_req.qp = qp;
+			priv->tid_req.rcd = qpriv->rcd;
+			priv->tid_req.e.ack = &qp->s_ack_queue[i];
+			qp->s_ack_queue[i].priv = priv;
+		}
 	}
 
 	return 0;
@@ -374,10 +461,109 @@ int hfi1_qp_priv_init(struct rvt_dev_info *rdi, struct rvt_qp *qp,
 void hfi1_qp_priv_tid_free(struct rvt_dev_info *rdi, struct rvt_qp *qp)
 {
 	struct hfi1_qp_priv *priv = qp->priv;
+	struct rvt_swqe *wqe;
+	u32 i;
 
 	if (qp->ibqp.qp_type == IB_QPT_RC && HFI1_CAP_IS_KSET(TID_RDMA)) {
+		for (i = 0; i < qp->s_size; i++) {
+			struct hfi1_swqe_priv *priv;
+
+			wqe = rvt_get_swqe_ptr(qp, i);
+			priv = wqe->priv;
+			kfree(priv);
+			wqe->priv = NULL;
+		}
+		for (i = 0; i < rvt_max_atomic(rdi); i++) {
+			struct hfi1_ack_priv *priv = qp->s_ack_queue[i].priv;
+
+			kfree(priv);
+			qp->s_ack_queue[i].priv = NULL;
+		}
 		cancel_work_sync(&priv->opfn.opfn_work);
 		kfree(priv->pages);
 		priv->pages = NULL;
 	}
 }
+
+/*
+ *  "Rewind" the TID request information.
+ *  This means that we reset the state back to ACTIVE,
+ *  find the proper flow, set the flow index to that flow,
+ *  and reset the flow information.
+ */
+void hfi1_tid_rdma_restart_req(struct rvt_qp *qp, struct rvt_swqe *wqe,
+			       u32 *bth2)
+{
+	struct tid_rdma_request *req = wqe_to_tid_req(wqe);
+	struct tid_rdma_flow *flow;
+	int diff;
+	u32 tididx = 0;
+	u16 fidx;
+
+	if (wqe->wr.opcode == IB_WR_TID_RDMA_READ) {
+		*bth2 = mask_psn(qp->s_psn);
+		flow = find_flow_ib(req, *bth2, &fidx);
+		if (!flow)
+			return;
+	} else {
+		return;
+	}
+
+	diff = delta_psn(*bth2, flow->flow_state.ib_spsn);
+
+	flow->sent = 0;
+	flow->pkt = 0;
+	flow->tid_idx = 0;
+	flow->tid_offset = 0;
+	if (diff) {
+		for (tididx = 0; tididx < flow->tidcnt; tididx++) {
+			u32 tidentry = flow->fstate->tid_entry[tididx], tidlen,
+			tidnpkts, npkts;
+
+			flow->tid_offset = 0;
+			tidlen = EXP_TID_GET(tidentry, LEN) * PAGE_SIZE;
+			tidnpkts = rvt_div_round_up_mtu(qp, tidlen);
+			npkts = min_t(u32, diff, tidnpkts);
+			flow->pkt += npkts;
+			flow->sent += (npkts == tidnpkts ? tidlen :
+				       npkts * qp->pmtu);
+			flow->tid_offset += npkts * qp->pmtu;
+			diff -= npkts;
+			if (!diff)
+				break;
+		}
+	}
+
+	if (flow->tid_offset ==
+		EXP_TID_GET(flow->fstate->tid_entry[tididx], LEN) * PAGE_SIZE) {
+		tididx++;
+		flow->tid_offset = 0;
+	}
+	flow->tid_idx = tididx;
+	/* Move flow_idx to correct index */
+	req->flow_idx = fidx;
+
+	req->state = TID_REQUEST_ACTIVE;
+}
+
+u32 hfi1_build_tid_rdma_read_packet(struct rvt_swqe *wqe,
+				    struct ib_other_headers *ohdr,
+				    u32 *bth1, u32 *bth2, u32 *len)
+{
+	return 0;
+}
+
+u32 hfi1_build_tid_rdma_read_req(struct rvt_qp *qp, struct rvt_swqe *wqe,
+				 struct ib_other_headers *ohdr, u32 *bth1,
+				 u32 *bth2, u32 *len)
+	 __must_hold(&qp->s_lock)
+{
+	return 0;
+}
+
+u32 hfi1_build_tid_rdma_read_resp(struct rvt_qp *qp, struct rvt_ack_entry *e,
+				  struct ib_other_headers *ohdr, u32 *bth0,
+				  u32 *bth1, u32 *bth2, u32 *len, bool *last)
+{
+	return 0;
+}
diff --git a/drivers/infiniband/hw/hfi1/tid_rdma.h b/drivers/infiniband/hw/hfi1/tid_rdma.h
index 8797621..8cdf6ca 100644
--- a/drivers/infiniband/hw/hfi1/tid_rdma.h
+++ b/drivers/infiniband/hw/hfi1/tid_rdma.h
@@ -260,6 +260,7 @@ struct trdma_flow_state {
 void hfi1_kern_clear_hw_flow(struct hfi1_ctxtdata *rcd, struct rvt_qp *qp);
 int hfi1_kern_exp_rcv_init(struct hfi1_ctxtdata *rcd, int reinit);
 void hfi1_kern_exp_rcv_clear_all(struct tid_rdma_request *req);
+void hfi1_kern_read_tid_flow_free(struct rvt_qp *qp);
 
 void hfi1_rc_rcv_tid_rdma_write_req(struct hfi1_packet *packet);
 
@@ -283,6 +284,19 @@ int hfi1_qp_priv_init(struct rvt_dev_info *rdi, struct rvt_qp *qp,
 		      struct ib_qp_init_attr *init_attr);
 void hfi1_qp_priv_tid_free(struct rvt_dev_info *rdi, struct rvt_qp *qp);
 
+void hfi1_tid_rdma_restart_req(struct rvt_qp *qp, struct rvt_swqe *wqe,
+			       u32 *bth2);
+
 void tid_rdma_opfn_init(struct rvt_qp *qp, struct tid_rdma_params *p);
 
+u32 hfi1_build_tid_rdma_read_packet(struct rvt_swqe *wqe,
+				    struct ib_other_headers *ohdr,
+				    u32 *bth1, u32 *bth2, u32 *len);
+u32 hfi1_build_tid_rdma_read_req(struct rvt_qp *qp, struct rvt_swqe *wqe,
+				 struct ib_other_headers *ohdr, u32 *bth1,
+				 u32 *bth2, u32 *len);
+u32 hfi1_build_tid_rdma_read_resp(struct rvt_qp *qp, struct rvt_ack_entry *e,
+				  struct ib_other_headers *ohdr, u32 *bth0,
+				  u32 *bth1, u32 *bth2, u32 *len, bool *last);
+
 #endif /* HFI1_TID_RDMA_H */
diff --git a/drivers/infiniband/hw/hfi1/verbs.c b/drivers/infiniband/hw/hfi1/verbs.c
index 4e7e885..47648f3 100644
--- a/drivers/infiniband/hw/hfi1/verbs.c
+++ b/drivers/infiniband/hw/hfi1/verbs.c
@@ -315,6 +315,7 @@ static inline bool wss_exceeds_threshold(void)
 	[IB_WR_SEND] = IB_WC_SEND,
 	[IB_WR_SEND_WITH_IMM] = IB_WC_SEND,
 	[IB_WR_RDMA_READ] = IB_WC_RDMA_READ,
+	[IB_WR_TID_RDMA_READ] = IB_WC_RDMA_READ,
 	[IB_WR_ATOMIC_CMP_AND_SWP] = IB_WC_COMP_SWAP,
 	[IB_WR_ATOMIC_FETCH_AND_ADD] = IB_WC_FETCH_ADD,
 	[IB_WR_SEND_WITH_INV] = IB_WC_SEND,
diff --git a/drivers/infiniband/hw/hfi1/verbs.h b/drivers/infiniband/hw/hfi1/verbs.h
index e308ebd..f2fb40c 100644
--- a/drivers/infiniband/hw/hfi1/verbs.h
+++ b/drivers/infiniband/hw/hfi1/verbs.h
@@ -173,10 +173,21 @@ struct hfi1_qp_priv {
 	unsigned long tid_timer_timeout_jiffies;
 	unsigned long tid_retry_timeout_jiffies;
 	u8 s_retry;
+	u32 pending_tid_r_segs; /* Num of pending tid read segments */
 	u16 pkts_ps;            /* packets per segment */
 	u8 timeout_shift;       /* account for number of packets per segment */
 };
 
+struct hfi1_swqe_priv {
+	struct tid_rdma_request tid_req;
+	u32 flags;
+	struct rvt_sge_state ss;  /* Used for TID RDMA READ Request */
+};
+
+struct hfi1_ack_priv {
+	struct tid_rdma_request tid_req;
+};
+
 /*
  * This structure is used to hold commonly lookedup and computed values during
  * the send engine progress.
@@ -323,6 +334,16 @@ static inline u32 delta_psn(u32 a, u32 b)
 	return (((int)a - (int)b) << PSN_SHIFT) >> PSN_SHIFT;
 }
 
+static inline struct tid_rdma_request *wqe_to_tid_req(struct rvt_swqe *wqe)
+{
+	return &((struct hfi1_swqe_priv *)wqe->priv)->tid_req;
+}
+
+static inline struct tid_rdma_request *ack_to_tid_req(struct rvt_ack_entry *e)
+{
+	return &((struct hfi1_ack_priv *)e->priv)->tid_req;
+}
+
 /*
  * Look through all the active flows for a TID RDMA request and find
  * the one (if it exists) that contains the specified PSN.
diff --git a/drivers/infiniband/sw/rdmavt/qp.c b/drivers/infiniband/sw/rdmavt/qp.c
index e55ecbb..cade920 100644
--- a/drivers/infiniband/sw/rdmavt/qp.c
+++ b/drivers/infiniband/sw/rdmavt/qp.c
@@ -2201,11 +2201,12 @@ static inline unsigned long rvt_aeth_to_usec(u32 aeth)
 }
 
 /*
- *  rvt_add_retry_timer - add/start a retry timer
+ *  rvt_add_retry_timer_ext - add/start a retry timer
  *  @qp - the QP
+ *  @shift - timeout shift to wait for multiple packets
  *  add a retry timer on the QP
  */
-void rvt_add_retry_timer(struct rvt_qp *qp)
+void rvt_add_retry_timer_ext(struct rvt_qp *qp, u8 shift)
 {
 	struct ib_qp *ibqp = &qp->ibqp;
 	struct rvt_dev_info *rdi = ib_to_rvt(ibqp->device);
@@ -2213,11 +2214,11 @@ void rvt_add_retry_timer(struct rvt_qp *qp)
 	lockdep_assert_held(&qp->s_lock);
 	qp->s_flags |= RVT_S_TIMER;
        /* 4.096 usec. * (1 << qp->timeout) */
-	qp->s_timer.expires = jiffies + qp->timeout_jiffies +
-			     rdi->busy_jiffies;
+	qp->s_timer.expires = jiffies + rdi->busy_jiffies +
+			      (qp->timeout_jiffies << shift);
 	add_timer(&qp->s_timer);
 }
-EXPORT_SYMBOL(rvt_add_retry_timer);
+EXPORT_SYMBOL(rvt_add_retry_timer_ext);
 
 /**
  * rvt_add_rnr_timer - add/start an rnr timer
diff --git a/include/rdma/rdma_vt.h b/include/rdma/rdma_vt.h
index d187621..33032fc 100644
--- a/include/rdma/rdma_vt.h
+++ b/include/rdma/rdma_vt.h
@@ -533,11 +533,12 @@ static inline u16 rvt_get_pkey(struct rvt_dev_info *rdi,
 }
 
 /**
- * rvt_mod_retry_timer - mod a retry timer
+ * rvt_mod_retry_timer_ext - mod a retry timer
  * @qp - the QP
+ * @shift - timeout shift to wait for multiple packets
  * Modify a potentially already running retry timer
  */
-static inline void rvt_mod_retry_timer(struct rvt_qp *qp)
+static inline void rvt_mod_retry_timer_ext(struct rvt_qp *qp, u8 shift)
 {
 	struct ib_qp *ibqp = &qp->ibqp;
 	struct rvt_dev_info *rdi = ib_to_rvt(ibqp->device);
@@ -545,8 +546,13 @@ static inline void rvt_mod_retry_timer(struct rvt_qp *qp)
 	lockdep_assert_held(&qp->s_lock);
 	qp->s_flags |= RVT_S_TIMER;
 	/* 4.096 usec. * (1 << qp->timeout) */
-	mod_timer(&qp->s_timer, jiffies + qp->timeout_jiffies +
-		  rdi->busy_jiffies);
+	mod_timer(&qp->s_timer, jiffies + rdi->busy_jiffies +
+		  (qp->timeout_jiffies << shift));
+}
+
+static inline void rvt_mod_retry_timer(struct rvt_qp *qp)
+{
+	return rvt_mod_retry_timer_ext(qp, 0);
 }
 
 struct rvt_dev_info *rvt_alloc_device(size_t size, int nports);
diff --git a/include/rdma/rdmavt_qp.h b/include/rdma/rdmavt_qp.h
index 7e32aa4..3b2c7d3 100644
--- a/include/rdma/rdmavt_qp.h
+++ b/include/rdma/rdmavt_qp.h
@@ -174,6 +174,7 @@ struct rvt_swqe {
 	u32 lpsn;               /* last packet sequence number */
 	u32 ssn;                /* send sequence number */
 	u32 length;             /* total length of data in sg_list */
+	void *priv;		/* driver dependent field */
 	struct rvt_sge sg_list[0];
 };
 
@@ -235,6 +236,7 @@ struct rvt_ack_entry {
 	u32 lpsn;
 	u8 opcode;
 	u8 sent;
+	void *priv;
 };
 
 #define	RC_QP_SCALING_INTERVAL	5
@@ -679,7 +681,11 @@ static inline unsigned long rvt_timeout_to_jiffies(u8 timeout)
 void rvt_add_rnr_timer(struct rvt_qp *qp, u32 aeth);
 void rvt_del_timers_sync(struct rvt_qp *qp);
 void rvt_stop_rc_timers(struct rvt_qp *qp);
-void rvt_add_retry_timer(struct rvt_qp *qp);
+void rvt_add_retry_timer_ext(struct rvt_qp *qp, u8 shift);
+static inline void rvt_add_retry_timer(struct rvt_qp *qp)
+{
+	rvt_add_retry_timer_ext(qp, 0);
+}
 
 /**
  * struct rvt_qp_iter - the iterator for QPs

--
To unsubscribe from this list: send the line "unsubscribe linux-rdma" in
the body of a message to majordomo@xxxxxxxxxxxxxxx
More majordomo info at  http://vger.kernel.org/majordomo-info.html



[Index of Archives]     [Linux USB Devel]     [Video for Linux]     [Linux Audio Users]     [Photo]     [Yosemite News]     [Yosemite Photos]     [Linux Kernel]     [Linux SCSI]     [XFree86]

  Powered by Linux