@@ -164,9 +164,12 @@ static u32 mask_generation(u32 a)
* C - Capcode
*/
+#define KDETH_INVALID_PSN 0x80000000
+
static u32 tid_rdma_flow_wt;
static void hfi1_do_tid_send(struct rvt_qp *qp);
+static void hfi1_tid_write_alloc_resources(struct rvt_qp *qp, bool intr_ctx);
static struct tid_rdma_flow *find_flow_ib(struct tid_rdma_request *req,
u32 psn, u16 *fidx)
@@ -189,6 +192,27 @@ static u32 mask_generation(u32 a)
return NULL;
}
+static struct tid_rdma_flow *
+__find_flow_ranged(struct tid_rdma_request *req, u16 head, u16 tail,
+ u32 psn, u16 *fidx)
+{
+ for ( ; CIRC_CNT(head, tail, req->n_max_flows);
+ tail = CIRC_NEXT(tail, req->n_max_flows)) {
+ struct tid_rdma_flow *flow = &req->flows[tail];
+ u32 spsn, lpsn;
+
+ spsn = full_flow_psn(flow, flow->flow_state.spsn);
+ lpsn = full_flow_psn(flow, flow->flow_state.lpsn);
+
+ if (cmp_psn(psn, spsn) >= 0 && cmp_psn(psn, lpsn) <= 0) {
+ if (fidx)
+ *fidx = tail;
+ return flow;
+ }
+ }
+ return NULL;
+}
+
static u8 trdma_pset_order(struct tid_rdma_pageset *s)
{
u8 count = s->count;
@@ -207,16 +231,126 @@ static int hfi1_send_tid_ok(struct rvt_qp *qp)
!(qp->s_flags & HFI1_S_ANY_TID_WAIT_SEND));
}
+/*
+ * @qp: points to rvt_qp context.
+ * @to_seg: desired RNR timeout in segments.
+ * Return: index of the next highest timeout in the ib_hfi1_rnr_table[]
+ */
+static u32 hfi1_compute_tid_rnr_timeout(struct rvt_qp *qp, u32 to_seg)
+{
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ u64 timeout;
+ u32 bytes_per_us;
+ u8 i;
+
+ bytes_per_us = active_egress_rate(qpriv->rcd->ppd) / 8;
+ timeout = (to_seg * TID_RDMA_MAX_SEGMENT_SIZE) / bytes_per_us;
+ /*
+ * Find the next highest value in the RNR table to the required
+ * timeout. This gives the responder some padding.
+ */
+ for (i = 1; i <= IB_AETH_CREDIT_MASK; i++)
+ if (rvt_rnr_tbl_to_usec(i) >= timeout)
+ return i;
+ return 0;
+}
+
+static void hfi1_add_tid_reap_timer(struct rvt_qp *qp)
+{
+ struct hfi1_qp_priv *qpriv = qp->priv;
+
+ lockdep_assert_held(&qp->s_lock);
+ if (!(qpriv->s_flags & HFI1_R_TID_RSC_TIMER)) {
+ qpriv->s_flags |= HFI1_R_TID_RSC_TIMER;
+ qpriv->s_tid_timer.expires = jiffies +
+ qpriv->tid_timer_timeout_jiffies;
+ add_timer(&qpriv->s_tid_timer);
+ }
+}
+
+static void hfi1_mod_tid_reap_timer(struct rvt_qp *qp)
+{
+ struct hfi1_qp_priv *qpriv = qp->priv;
+
+ lockdep_assert_held(&qp->s_lock);
+ qpriv->s_flags |= HFI1_R_TID_RSC_TIMER;
+ mod_timer(&qpriv->s_tid_timer, jiffies +
+ qpriv->tid_timer_timeout_jiffies);
+}
+
+static int hfi1_stop_tid_reap_timer(struct rvt_qp *qp)
+{
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ int rval = 0;
+
+ lockdep_assert_held(&qp->s_lock);
+ if (qpriv->s_flags & HFI1_R_TID_RSC_TIMER) {
+ rval = del_timer(&qpriv->s_tid_timer);
+ qpriv->s_flags &= ~HFI1_R_TID_RSC_TIMER;
+ }
+ return rval;
+}
+
void hfi1_del_tid_reap_timer(struct rvt_qp *qp)
{
+ struct hfi1_qp_priv *qpriv = qp->priv;
+
+ del_timer_sync(&qpriv->s_tid_timer);
+ qpriv->s_flags &= ~HFI1_R_TID_RSC_TIMER;
}
void hfi1_add_tid_retry_timer(struct rvt_qp *qp)
{
+ struct hfi1_qp_priv *priv = qp->priv;
+ struct ib_qp *ibqp = &qp->ibqp;
+ struct rvt_dev_info *rdi = ib_to_rvt(ibqp->device);
+
+ lockdep_assert_held(&qp->s_lock);
+ if (!(priv->s_flags & HFI1_S_TID_RETRY_TIMER)) {
+ priv->s_flags |= HFI1_S_TID_RETRY_TIMER;
+ priv->s_tid_retry_timer.expires = jiffies +
+ priv->tid_retry_timeout_jiffies + rdi->busy_jiffies;
+ add_timer(&priv->s_tid_retry_timer);
+ }
+}
+
+static void hfi1_mod_tid_retry_timer(struct rvt_qp *qp)
+{
+ struct hfi1_qp_priv *priv = qp->priv;
+ struct ib_qp *ibqp = &qp->ibqp;
+ struct rvt_dev_info *rdi = ib_to_rvt(ibqp->device);
+
+ lockdep_assert_held(&qp->s_lock);
+ priv->s_flags |= HFI1_S_TID_RETRY_TIMER;
+ mod_timer(&priv->s_tid_retry_timer, jiffies +
+ priv->tid_retry_timeout_jiffies + rdi->busy_jiffies);
+}
+
+static int hfi1_stop_tid_retry_timer(struct rvt_qp *qp)
+{
+ struct hfi1_qp_priv *priv = qp->priv;
+ int rval = 0;
+
+ lockdep_assert_held(&qp->s_lock);
+ if (priv->s_flags & HFI1_S_TID_RETRY_TIMER) {
+ rval = del_timer(&priv->s_tid_retry_timer);
+ priv->s_flags &= ~HFI1_S_TID_RETRY_TIMER;
+ }
+ return rval;
}
void hfi1_del_tid_retry_timer(struct rvt_qp *qp)
{
+ struct hfi1_qp_priv *priv = qp->priv;
+
+ del_timer_sync(&priv->s_tid_retry_timer);
+ priv->s_flags &= ~HFI1_S_TID_RETRY_TIMER;
+}
+
+static bool hfi1_tid_rdma_is_resync_psn(u32 psn)
+{
+ return (bool)((psn & HFI1_KDETH_BTH_SEQ_MASK) ==
+ HFI1_KDETH_BTH_SEQ_MASK);
}
static u64 tid_rdma_opfn_encode(struct tid_rdma_params *p)
@@ -1698,6 +1832,8 @@ static int tid_rdma_rcv_error(struct hfi1_packet *packet,
{
struct hfi1_ibport *ibp = to_iport(qp->ibqp.device, qp->port_num);
struct hfi1_ctxtdata *rcd = ((struct hfi1_qp_priv *)qp->priv)->rcd;
+ struct hfi1_ibdev *dev = to_idev(qp->ibqp.device);
+ struct hfi1_qp_priv *qpriv = qp->priv;
struct rvt_ack_entry *e;
struct tid_rdma_request *req;
unsigned long flags;
@@ -1726,6 +1862,113 @@ static int tid_rdma_rcv_error(struct hfi1_packet *packet,
req = ack_to_tid_req(e);
req->r_flow_psn = psn;
if (e->opcode == TID_OP(WRITE_REQ)) {
+ struct flow_state *fstate;
+ bool schedule = false;
+ u8 i;
+
+ if (req->state == TID_REQUEST_RESEND) {
+ req->state = TID_REQUEST_RESEND_ACTIVE;
+ } else if (req->state == TID_REQUEST_INIT_RESEND) {
+ req->state = TID_REQUEST_INIT;
+ schedule = true;
+ }
+
+ /*
+ * True if the request is already scheduled (between
+ * qp->s_tail_ack_queue and qp->r_head_ack_queue).
+ * Also, don't change requests, which are at the SYNC
+ * point and haven't generated any responses yet.
+ * There is nothing to retransmit for them yet.
+ */
+ if (old_req || req->state == TID_REQUEST_INIT ||
+ (req->state == TID_REQUEST_SYNC && !req->cur_seg)) {
+ for (i = prev + 1; ; i++) {
+ if (i > rvt_size_atomic(&dev->rdi))
+ i = 0;
+ if (i == qp->r_head_ack_queue)
+ break;
+ e = &qp->s_ack_queue[i];
+ req = ack_to_tid_req(e);
+ if (e->opcode == TID_OP(WRITE_REQ) &&
+ req->state == TID_REQUEST_INIT)
+ req->state = TID_REQUEST_INIT_RESEND;
+ }
+ /*
+ * If the state of the request has been changed,
+ * the first leg needs to get scheduled in order to
+ * pick up the change. Otherwise, normal response
+ * processing should take care of it.
+ */
+ if (!schedule)
+ goto unlock;
+ }
+
+ /*
+ * If there is no more allocated segment, just schedule the qp
+ * without changing any state.
+ */
+ if (req->clear_tail == req->setup_head)
+ goto schedule;
+ /*
+ * If this request has sent responses for segments, which have
+ * not received data yet (flow_idx != clear_tail), the flow_idx
+ * pointer needs to be adjusted so the same responses can be
+ * re-sent.
+ */
+ if (CIRC_CNT(req->flow_idx, req->clear_tail,
+ req->n_max_flows)) {
+ fstate = &req->flows[req->clear_tail].flow_state;
+ qpriv->pending_tid_w_segs -=
+ CIRC_CNT(req->flow_idx, req->clear_tail,
+ req->n_max_flows);
+ req->flow_idx =
+ CIRC_ADD(req->clear_tail,
+ delta_psn(psn, fstate->resp_ib_psn),
+ req->n_max_flows);
+ qpriv->pending_tid_w_segs +=
+ delta_psn(psn, fstate->resp_ib_psn);
+ /*
+ * When flow_idx == setup_head, we've gotten a duplicate
+ * request for a segment, which has not been allocated
+ * yet. In that case, don't adjust this request.
+ * However, we still want to go through the loop below
+ * to adjust all subsequent requests.
+ */
+ if (CIRC_CNT(req->setup_head, req->flow_idx,
+ req->n_max_flows)) {
+ req->cur_seg = delta_psn(psn, e->psn);
+ req->state = TID_REQUEST_RESEND_ACTIVE;
+ }
+ }
+
+ for (i = prev + 1; ; i++) {
+ /*
+ * Look at everything up to and including
+ * s_tail_ack_queue
+ */
+ if (i > rvt_size_atomic(&dev->rdi))
+ i = 0;
+ if (i == qp->r_head_ack_queue)
+ break;
+ e = &qp->s_ack_queue[i];
+ req = ack_to_tid_req(e);
+ if (e->opcode != TID_OP(WRITE_REQ) ||
+ req->cur_seg == req->comp_seg ||
+ req->state == TID_REQUEST_INIT ||
+ req->state == TID_REQUEST_INIT_RESEND) {
+ if (req->state == TID_REQUEST_INIT)
+ req->state = TID_REQUEST_INIT_RESEND;
+ continue;
+ }
+ qpriv->pending_tid_w_segs -=
+ CIRC_CNT(req->flow_idx,
+ req->clear_tail,
+ req->n_max_flows);
+ req->flow_idx = req->clear_tail;
+ req->state = TID_REQUEST_RESEND;
+ req->cur_seg = req->comp_seg;
+ }
+ qpriv->s_flags &= ~HFI1_R_TID_WAIT_INTERLCK;
} else {
struct ib_reth *reth;
u32 offset;
@@ -1775,26 +2018,12 @@ static int tid_rdma_rcv_error(struct hfi1_packet *packet,
goto unlock;
/*
- * If all the response packets for the current request have
- * been sent out and this request is complete (old_request
- * == false) and the TID flow may be unusable (the
- * req->clear_tail is advanced). However, when an earlier
- * request is received, this request will not be complete any
- * more (qp->s_tail_ack_queue is moved back, see below).
- * Consequently, we need to update the TID flow info everytime
- * a duplicate request is received.
- */
- bth0 = be32_to_cpu(ohdr->bth[0]);
- if (tid_rdma_rcv_read_request(qp, e, packet, ohdr, bth0, psn,
- vaddr, len))
- goto unlock;
-
- /*
* True if the request is already scheduled (between
* qp->s_tail_ack_queue and qp->r_head_ack_queue);
*/
if (old_req)
goto unlock;
+
}
/* Re-process old requests.*/
if (qp->s_acked_ack_queue == qp->s_tail_ack_queue)
@@ -1807,6 +2036,18 @@ static int tid_rdma_rcv_error(struct hfi1_packet *packet,
* wrong memory region.
*/
qp->s_ack_state = OP(ACKNOWLEDGE);
+schedule:
+ /*
+ * It's possible to receive a retry psn that is earlier than an RNRNAK
+ * psn. In this case, the rnrnak state should be cleared.
+ */
+ if (qpriv->rnr_nak_state) {
+ qp->s_nak_state = 0;
+ qpriv->rnr_nak_state = TID_RNR_NAK_INIT;
+ qp->r_psn = e->lpsn + 1;
+ hfi1_tid_write_alloc_resources(qp, true);
+ }
+
qp->r_state = e->opcode;
qp->r_nak_state = 0;
qp->s_flags |= RVT_S_RESP_PENDING;
@@ -1817,16 +2058,678 @@ static int tid_rdma_rcv_error(struct hfi1_packet *packet,
return 1;
}
+static u32 position_in_queue(struct hfi1_qp_priv *qpriv,
+ struct tid_queue *queue)
+{
+ return qpriv->tid_enqueue - queue->dequeue;
+}
+
+/**
+ * Central place for resource allocation at TID write responder, is called from
+ * write_req and write_data interrupt handlers as well as the send thread when a
+ * queued QP is scheduled for resource allocation.
+ *
+ * Iterates over (a) segments of a request and then (b) queued requests
+ * themselves to allocate resources for upto local->max_write segments across
+ * multiple requests. Stop allocating when we hit a sync point, resume
+ * allocating after data packets at sync point have been received.
+ *
+ * Resource allocation and sending of responses is decoupled. The
+ * request/segment which are being allocated and sent are as follows.
+ * Resources are allocated for:
+ * [request: qpriv->r_tid_alloc, segment: req->alloc_seg]
+ * The send thread sends:
+ * [request: qp->s_tail_ack_queue, segment:req->cur_seg]
+ */
+static void hfi1_tid_write_alloc_resources(struct rvt_qp *qp, bool intr_ctx)
+{
+ struct tid_rdma_request *req;
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ struct hfi1_ctxtdata *rcd = qpriv->rcd;
+ struct tid_rdma_params *local = &qpriv->tid_rdma.local;
+ struct rvt_ack_entry *e;
+ u32 npkts, to_seg;
+ bool last;
+ int ret = 0;
+
+ lockdep_assert_held(&qp->s_lock);
+
+ while (1) {
+ /*
+ * Don't allocate more segments if a RNR NAK has already been
+ * scheduled to avoid messing up qp->r_psn: the RNR NAK will
+ * be sent only when all allocated segments have been sent.
+ * However, if more segments are allocated before that, TID RDMA
+ * WRITE RESP packets will be sent out for these new segments
+ * before the RNR NAK packet. When the requester receives the
+ * RNR NAK packet, it will restart with qp->s_last_psn + 1,
+ * which does not match qp->r_psn and will be dropped.
+ * Consequently, the requester will exhaust its retries and
+ * put the qp into error state.
+ */
+ if (qpriv->rnr_nak_state == TID_RNR_NAK_SEND)
+ break;
+
+ /* No requests left to process */
+ if (qpriv->r_tid_alloc == qpriv->r_tid_head) {
+ /* If all data has been received, clear the flow */
+ if (qpriv->flow_state.index < RXE_NUM_TID_FLOWS &&
+ !qpriv->alloc_w_segs)
+ hfi1_kern_clear_hw_flow(rcd, qp);
+ break;
+ }
+
+ e = &qp->s_ack_queue[qpriv->r_tid_alloc];
+ if (e->opcode != TID_OP(WRITE_REQ))
+ goto next_req;
+ req = ack_to_tid_req(e);
+ /* Finished allocating for all segments of this request */
+ if (req->alloc_seg >= req->total_segs)
+ goto next_req;
+
+ /* Can allocate only a maximum of local->max_write for a QP */
+ if (qpriv->alloc_w_segs >= local->max_write)
+ break;
+
+ /* Don't allocate at a sync point with data packets pending */
+ if (qpriv->sync_pt && qpriv->alloc_w_segs)
+ break;
+
+ /* All data received at the sync point, continue */
+ if (qpriv->sync_pt && !qpriv->alloc_w_segs) {
+ hfi1_kern_clear_hw_flow(rcd, qp);
+ qpriv->sync_pt = false;
+ if (qpriv->s_flags & HFI1_R_TID_SW_PSN)
+ qpriv->s_flags &= ~HFI1_R_TID_SW_PSN;
+ }
+
+ /* Allocate flow if we don't have one */
+ if (qpriv->flow_state.index >= RXE_NUM_TID_FLOWS) {
+ ret = hfi1_kern_setup_hw_flow(qpriv->rcd, qp);
+ if (ret) {
+ to_seg = tid_rdma_flow_wt *
+ position_in_queue(qpriv,
+ &rcd->flow_queue);
+ break;
+ }
+ }
+
+ npkts = rvt_div_round_up_mtu(qp, req->seg_len);
+
+ /*
+ * We are at a sync point if we run out of KDETH PSN space.
+ * Last PSN of every generation is reserved for RESYNC.
+ */
+ if (qpriv->flow_state.psn + npkts > MAX_TID_FLOW_PSN - 1) {
+ qpriv->sync_pt = true;
+ break;
+ }
+
+ /*
+ * If overtaking req->acked_tail, send an RNR NAK. Because the
+ * QP is not queued in this case, and the issue can only be
+ * caused due a delay in scheduling the second leg which we
+ * cannot estimate, we use a rather arbitrary RNR timeout of
+ * (req->n_max_flows / 2) segments
+ */
+ if (!CIRC_SPACE(req->setup_head, req->acked_tail,
+ req->n_max_flows)) {
+ ret = -EAGAIN;
+ to_seg = req->n_max_flows >> 1;
+ qpriv->s_flags |= RVT_S_ACK_PENDING;
+ hfi1_schedule_tid_send(qp);
+ break;
+ }
+
+ /* Try to allocate rcv array / TID entries */
+ ret = hfi1_kern_exp_rcv_setup(req, &req->ss, &last);
+ if (ret == -EAGAIN)
+ to_seg = position_in_queue(qpriv, &rcd->rarr_queue);
+ if (ret)
+ break;
+
+ qpriv->alloc_w_segs++;
+ req->alloc_seg++;
+ continue;
+next_req:
+ /* Begin processing the next request */
+ if (++qpriv->r_tid_alloc >
+ rvt_size_atomic(ib_to_rvt(qp->ibqp.device)))
+ qpriv->r_tid_alloc = 0;
+ }
+
+ /*
+ * Schedule an RNR NAK to be sent if (a) flow or rcv array allocation
+ * has failed (b) we are called from the rcv handler interrupt context
+ * (c) an RNR NAK has not already been scheduled
+ */
+ if (ret == -EAGAIN && intr_ctx && !qp->r_nak_state)
+ goto send_rnr_nak;
+
+ return;
+
+send_rnr_nak:
+ /*
+ * NO_RNR_NAK turns off sending RNR NAK's. RNR NAK's will not be sent
+ * but everything else will work as expected. The remote QP might time
+ * out and resend the request if it doesn't receive a response. The
+ * resent request will be discarded as duplicate. This works as long as
+ * the remote QP does not run out of retries
+ */
+#ifdef NO_RNR_NAK
+ return;
+#endif
+ lockdep_assert_held(&qp->r_lock);
+
+ /* Set r_nak_state to prevent unrelated events from generating NAK's */
+ qp->r_nak_state = hfi1_compute_tid_rnr_timeout(qp, to_seg) | IB_RNR_NAK;
+
+ /* Pull back r_psn to the segment being RNR NAK'd */
+ qp->r_psn = e->psn + req->alloc_seg;
+ qp->r_ack_psn = qp->r_psn;
+ /*
+ * Pull back r_head_ack_queue to the ack entry following the request
+ * being RNR NAK'd. This allows resources to be allocated to the request
+ * if the queued QP is scheduled.
+ */
+ qp->r_head_ack_queue = qpriv->r_tid_alloc + 1;
+ if (qp->r_head_ack_queue > rvt_size_atomic(ib_to_rvt(qp->ibqp.device)))
+ qp->r_head_ack_queue = 0;
+ qpriv->r_tid_head = qp->r_head_ack_queue;
+ /*
+ * These send side fields are used in make_rc_ack(). They are set in
+ * hfi1_send_rc_ack() but must be set here before dropping qp->s_lock
+ * for consistency
+ */
+ qp->s_nak_state = qp->r_nak_state;
+ qp->s_ack_psn = qp->r_ack_psn;
+ /*
+ * Clear the ACK PENDING flag to prevent unwanted ACK because we
+ * have modified qp->s_ack_psn here.
+ */
+ qp->s_flags &= ~(RVT_S_ACK_PENDING);
+ /*
+ * qpriv->rnr_nak_state is used to determine when the scheduled RNR NAK
+ * has actually been sent. qp->s_flags RVT_S_ACK_PENDING bit cannot be
+ * used for this because qp->s_lock is dropped before calling
+ * hfi1_send_rc_ack() leading to inconsistency between the receive
+ * interrupt handlers and the send thread in make_rc_ack()
+ */
+ qpriv->rnr_nak_state = TID_RNR_NAK_SEND;
+
+ /*
+ * Schedule RNR NAK to be sent. RNR NAK's are scheduled from the receive
+ * interrupt handlers but will be sent from the send engine behind any
+ * previous responses that may have been scheduled
+ */
+ rc_defered_ack(rcd, qp);
+}
+
void hfi1_rc_rcv_tid_rdma_write_req(struct hfi1_packet *packet)
{
+ /* HANDLER FOR TID RDMA WRITE REQUEST packet (Responder side)*/
+
+ /*
+ * 1. Verify TID RDMA WRITE REQ as per IB_OPCODE_RC_RDMA_WRITE_FIRST
+ * (see hfi1_rc_rcv())
+ * - Don't allow 0-length requests.
+ * 2. Put TID RDMA WRITE REQ into the response queueu (s_ack_queue)
+ * - Setup struct tid_rdma_req with request info
+ * - Prepare struct tid_rdma_flow array?
+ * 3. Set the qp->s_ack_state as state diagram in design doc.
+ * 4. Set RVT_S_RESP_PENDING in s_flags.
+ * 5. Kick the send engine (hfi1_schedule_send())
+ */
+ struct hfi1_ctxtdata *rcd = packet->rcd;
+ struct rvt_qp *qp = packet->qp;
+ struct hfi1_ibport *ibp = to_iport(qp->ibqp.device, qp->port_num);
+ struct ib_other_headers *ohdr = packet->ohdr;
+ struct rvt_ack_entry *e;
+ unsigned long flags;
+ struct ib_reth *reth;
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ struct tid_rdma_request *req;
+ u32 bth0, psn, len, rkey, num_segs;
+ bool is_fecn;
+ u8 next;
+ u64 vaddr;
+ int diff;
+
+ bth0 = be32_to_cpu(ohdr->bth[0]);
+ if (hfi1_ruc_check_hdr(ibp, packet))
+ return;
+
+ is_fecn = process_ecn(qp, packet, false);
+ psn = mask_psn(be32_to_cpu(ohdr->bth[2]));
+
+ if (qp->state == IB_QPS_RTR && !(qp->r_flags & RVT_R_COMM_EST))
+ rvt_comm_est(qp);
+
+ if (unlikely(!(qp->qp_access_flags & IB_ACCESS_REMOTE_WRITE)))
+ goto nack_inv;
+
+ reth = &ohdr->u.tid_rdma.w_req.reth;
+ vaddr = be64_to_cpu(reth->vaddr);
+ len = be32_to_cpu(reth->length);
+
+ num_segs = DIV_ROUND_UP(len, qpriv->tid_rdma.local.max_len);
+ diff = delta_psn(psn, qp->r_psn);
+ if (unlikely(diff)) {
+ if (tid_rdma_rcv_error(packet, ohdr, qp, psn, diff))
+ return;
+ goto send_ack;
+ }
+
+ /*
+ * The resent request which was previously RNR NAK'd is inserted at the
+ * location of the original request, which is one entry behind
+ * r_head_ack_queue
+ */
+ if (qpriv->rnr_nak_state)
+ qp->r_head_ack_queue = qp->r_head_ack_queue ?
+ qp->r_head_ack_queue - 1 :
+ rvt_size_atomic(ib_to_rvt(qp->ibqp.device));
+
+ /* We've verified the request, insert it into the ack queue. */
+ next = qp->r_head_ack_queue + 1;
+ if (next > rvt_size_atomic(ib_to_rvt(qp->ibqp.device)))
+ next = 0;
+ spin_lock_irqsave(&qp->s_lock, flags);
+ if (unlikely(next == qp->s_acked_ack_queue)) {
+ if (!qp->s_ack_queue[next].sent)
+ goto nack_inv_unlock;
+ update_ack_queue(qp, next);
+ }
+ e = &qp->s_ack_queue[qp->r_head_ack_queue];
+ req = ack_to_tid_req(e);
+
+ /* Bring previously RNR NAK'd request back to life */
+ if (qpriv->rnr_nak_state) {
+ qp->r_nak_state = 0;
+ qp->s_nak_state = 0;
+ qpriv->rnr_nak_state = TID_RNR_NAK_INIT;
+ qp->r_psn = e->lpsn + 1;
+ req->state = TID_REQUEST_INIT;
+ goto update_head;
+ }
+
+ if (e->rdma_sge.mr) {
+ rvt_put_mr(e->rdma_sge.mr);
+ e->rdma_sge.mr = NULL;
+ }
+
+ /* The length needs to be in multiples of PAGE_SIZE */
+ if (!len || len & ~PAGE_MASK)
+ goto nack_inv_unlock;
+
+ rkey = be32_to_cpu(reth->rkey);
+ qp->r_len = len;
+
+ if (e->opcode == TID_OP(WRITE_REQ) &&
+ (req->setup_head != req->clear_tail ||
+ req->clear_tail != req->acked_tail))
+ goto nack_inv_unlock;
+
+ if (unlikely(!rvt_rkey_ok(qp, &e->rdma_sge, qp->r_len, vaddr,
+ rkey, IB_ACCESS_REMOTE_WRITE)))
+ goto nack_acc;
+
+ qp->r_psn += num_segs - 1;
+
+ e->opcode = (bth0 >> 24) & 0xff;
+ e->psn = psn;
+ e->lpsn = qp->r_psn;
+ e->sent = 0;
+
+ req->n_flows = min_t(u16, num_segs, qpriv->tid_rdma.local.max_write);
+ req->state = TID_REQUEST_INIT;
+ req->cur_seg = 0;
+ req->comp_seg = 0;
+ req->ack_seg = 0;
+ req->alloc_seg = 0;
+ req->isge = 0;
+ req->seg_len = qpriv->tid_rdma.local.max_len;
+ req->total_len = len;
+ req->total_segs = num_segs;
+ req->r_flow_psn = e->psn;
+ req->ss.sge = e->rdma_sge;
+ req->ss.num_sge = 1;
+
+ req->flow_idx = req->setup_head;
+ req->clear_tail = req->setup_head;
+ req->acked_tail = req->setup_head;
+
+ qp->r_state = e->opcode;
+ qp->r_nak_state = 0;
+ /*
+ * We need to increment the MSN here instead of when we
+ * finish sending the result since a duplicate request would
+ * increment it more than once.
+ */
+ qp->r_msn++;
+ qp->r_psn++;
+
+ if (qpriv->r_tid_tail == HFI1_QP_WQE_INVALID) {
+ qpriv->r_tid_tail = qp->r_head_ack_queue;
+ } else if (qpriv->r_tid_tail == qpriv->r_tid_head) {
+ struct tid_rdma_request *ptr;
+
+ e = &qp->s_ack_queue[qpriv->r_tid_tail];
+ ptr = ack_to_tid_req(e);
+
+ if (e->opcode != TID_OP(WRITE_REQ) ||
+ ptr->comp_seg == ptr->total_segs) {
+ if (qpriv->r_tid_tail == qpriv->r_tid_ack)
+ qpriv->r_tid_ack = qp->r_head_ack_queue;
+ qpriv->r_tid_tail = qp->r_head_ack_queue;
+ }
+ }
+update_head:
+ qp->r_head_ack_queue = next;
+ qpriv->r_tid_head = qp->r_head_ack_queue;
+
+ hfi1_tid_write_alloc_resources(qp, true);
+
+ /* Schedule the send tasklet. */
+ qp->s_flags |= RVT_S_RESP_PENDING;
+ hfi1_schedule_send(qp);
+
+ spin_unlock_irqrestore(&qp->s_lock, flags);
+ if (is_fecn)
+ goto send_ack;
+ return;
+
+nack_inv_unlock:
+ spin_unlock_irqrestore(&qp->s_lock, flags);
+nack_inv:
+ rvt_rc_error(qp, IB_WC_LOC_QP_OP_ERR);
+ qp->r_nak_state = IB_NAK_INVALID_REQUEST;
+ qp->r_ack_psn = qp->r_psn;
+ /* Queue NAK for later */
+ rc_defered_ack(rcd, qp);
+ return;
+nack_acc:
+ spin_unlock_irqrestore(&qp->s_lock, flags);
+ rvt_rc_error(qp, IB_WC_LOC_PROT_ERR);
+ qp->r_nak_state = IB_NAK_REMOTE_ACCESS_ERROR;
+ qp->r_ack_psn = qp->r_psn;
+send_ack:
+ hfi1_send_rc_ack(packet, is_fecn);
}
void hfi1_rc_rcv_tid_rdma_write_data(struct hfi1_packet *packet)
{
+ struct rvt_qp *qp = packet->qp;
+ struct hfi1_qp_priv *priv = qp->priv;
+ struct hfi1_ctxtdata *rcd = priv->rcd;
+ struct ib_other_headers *ohdr = packet->ohdr;
+ struct rvt_ack_entry *e;
+ struct tid_rdma_request *req;
+ struct tid_rdma_flow *flow;
+ struct hfi1_ibdev *dev = to_idev(qp->ibqp.device);
+ unsigned long flags;
+ u32 psn, next;
+ u8 opcode;
+
+ psn = mask_psn(be32_to_cpu(ohdr->bth[2]));
+ opcode = (be32_to_cpu(ohdr->bth[0]) >> 24) & 0xff;
+
+ /*
+ * All error handling should be done by now. If we are here, the packet
+ * is either good or been accepted by the error handler.
+ */
+ spin_lock_irqsave(&qp->s_lock, flags);
+ e = &qp->s_ack_queue[priv->r_tid_tail];
+ req = ack_to_tid_req(e);
+ flow = &req->flows[req->clear_tail];
+ if (cmp_psn(psn, full_flow_psn(flow, flow->flow_state.lpsn))) {
+ if (cmp_psn(psn, flow->flow_state.r_next_psn))
+ goto send_nak;
+ flow->flow_state.r_next_psn++;
+ goto exit;
+ }
+ flow->flow_state.r_next_psn = mask_psn(psn + 1);
+ hfi1_kern_exp_rcv_clear(req);
+ priv->alloc_w_segs--;
+ rcd->flows[flow->idx].psn = psn & HFI1_KDETH_BTH_SEQ_MASK;
+ req->comp_seg++;
+ priv->s_nak_state = 0;
+
+ /*
+ * Release the flow if one of the following conditions has been met:
+ * - The request has reached a sync point AND all outstanding
+ * segments have been completed, or
+ * - The entire request is complete and there are no more requests
+ * (of any kind) in the queue.
+ */
+
+ if (priv->r_tid_ack == HFI1_QP_WQE_INVALID)
+ priv->r_tid_ack = priv->r_tid_tail;
+
+ if (opcode == TID_OP(WRITE_DATA_LAST)) {
+ for (next = priv->r_tid_tail + 1; ; next++) {
+ if (next > rvt_size_atomic(&dev->rdi))
+ next = 0;
+ if (next == priv->r_tid_head)
+ break;
+ e = &qp->s_ack_queue[next];
+ if (e->opcode == TID_OP(WRITE_REQ))
+ break;
+ }
+ priv->r_tid_tail = next;
+ if (++qp->s_acked_ack_queue > rvt_size_atomic(&dev->rdi))
+ qp->s_acked_ack_queue = 0;
+ }
+
+ hfi1_tid_write_alloc_resources(qp, true);
+
+ /*
+ * If we need to generate more responses, schedule the
+ * send engine.
+ */
+ if (req->cur_seg < req->total_segs ||
+ qp->s_tail_ack_queue != qp->r_head_ack_queue) {
+ qp->s_flags |= RVT_S_RESP_PENDING;
+ hfi1_schedule_send(qp);
+ }
+
+ priv->pending_tid_w_segs--;
+ if (priv->s_flags & HFI1_R_TID_RSC_TIMER) {
+ if (priv->pending_tid_w_segs)
+ hfi1_mod_tid_reap_timer(req->qp);
+ else
+ hfi1_stop_tid_reap_timer(req->qp);
+ }
+
+done:
+ priv->s_flags |= RVT_S_ACK_PENDING;
+ hfi1_schedule_tid_send(qp);
+exit:
+ priv->r_next_psn_kdeth = flow->flow_state.r_next_psn;
+ spin_unlock_irqrestore(&qp->s_lock, flags);
+ return;
+
+send_nak:
+ if (!priv->s_nak_state) {
+ priv->s_nak_state = IB_NAK_PSN_ERROR;
+ priv->s_nak_psn = flow->flow_state.r_next_psn;
+ priv->s_flags |= RVT_S_ACK_PENDING;
+ if (priv->r_tid_ack == HFI1_QP_WQE_INVALID)
+ priv->r_tid_ack = priv->r_tid_tail;
+ hfi1_schedule_tid_send(qp);
+ }
+ goto done;
}
void hfi1_rc_rcv_tid_rdma_write_resp(struct hfi1_packet *packet)
{
+ /* HANDLER FOR TID RDMA WRITE RESPONSE packet (Requestor side */
+
+ /*
+ * 1. Find matching SWQE
+ * 2. Check that TIDENTRY array has enough space for a complete
+ * segment. If not, put QP in error state.
+ * 3. Save response data in struct tid_rdma_req and struct tid_rdma_flow
+ * 4. Remove HFI1_S_WAIT_TID_RESP from s_flags.
+ * 5. Set qp->s_state as per state diagram in design doc
+ * 6. Kick the send engine (hfi1_schedule_send())
+ */
+ struct ib_other_headers *ohdr = packet->ohdr;
+ struct rvt_qp *qp = packet->qp;
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ struct hfi1_ctxtdata *rcd = packet->rcd;
+ struct rvt_swqe *wqe;
+ struct tid_rdma_request *req;
+ struct tid_rdma_flow *flow;
+ enum ib_wc_status status;
+ u32 opcode, aeth, psn, flow_psn, i, tidlen = 0, pktlen;
+ bool is_fecn;
+ unsigned long flags;
+
+ is_fecn = process_ecn(qp, packet, false);
+ psn = mask_psn(be32_to_cpu(ohdr->bth[2]));
+ aeth = be32_to_cpu(ohdr->u.tid_rdma.w_rsp.aeth);
+ opcode = (be32_to_cpu(ohdr->bth[0]) >> 24) & 0xff;
+
+ spin_lock_irqsave(&qp->s_lock, flags);
+
+ /* Ignore invalid responses */
+ if (cmp_psn(psn, qp->s_next_psn) >= 0)
+ goto ack_done;
+
+ /* Ignore duplicate responses. */
+ /* XXX Check with TID RDMA spec if this is possible */
+ if (unlikely(cmp_psn(psn, qp->s_last_psn) <= 0))
+ goto ack_done;
+
+ if (unlikely(qp->s_acked == qp->s_tail))
+ goto ack_done;
+
+ /*
+ * If we are waiting for a particular packet sequence number
+ * due to a request being resent, check for it. Otherwise,
+ * ensure that we haven't missed anything.
+ */
+ if (qp->r_flags & RVT_R_RDMAR_SEQ) {
+ if (cmp_psn(psn, qp->s_last_psn + 1) != 0)
+ goto ack_done;
+ qp->r_flags &= ~RVT_R_RDMAR_SEQ;
+ }
+
+ wqe = rvt_get_swqe_ptr(qp, qpriv->s_tid_cur);
+ if (unlikely(wqe->wr.opcode != IB_WR_TID_RDMA_WRITE))
+ goto ack_op_err;
+
+ req = wqe_to_tid_req(wqe);
+ /*
+ * If we've lost ACKs and our acked_tail pointer is too far
+ * behind, don't overwrite segments. Just drop the packet and
+ * let the reliability protocol take care of it.
+ */
+ if (!CIRC_SPACE(req->setup_head, req->acked_tail,
+ req->n_max_flows))
+ goto ack_done;
+
+ /*
+ * The call to do_rc_ack() should be last in the chain of
+ * packet checks because it will end up updating the QP state.
+ * Therefore, anything that would prevent the packet from
+ * being accepted as a successful response should be prior
+ * to it.
+ */
+ if (!do_rc_ack(qp, aeth, psn, opcode, 0, rcd))
+ goto ack_done;
+
+ trace_hfi1_ack(qp, psn);
+
+ flow = &req->flows[req->setup_head];
+ flow->req = req;
+ flow->pkt = 0;
+ flow->tid_idx = 0;
+ flow->sent = 0;
+ flow->resync_npkts = 0;
+ flow->tid_qpn = be32_to_cpu(ohdr->u.tid_rdma.w_rsp.tid_flow_qp);
+ flow->idx = (flow->tid_qpn >> TID_RDMA_DESTQP_FLOW_SHIFT) &
+ TID_RDMA_DESTQP_FLOW_MASK;
+ flow_psn = mask_psn(be32_to_cpu(ohdr->u.tid_rdma.w_rsp.tid_flow_psn));
+ flow->flow_state.generation = flow_psn >> HFI1_KDETH_BTH_SEQ_SHIFT;
+ flow->flow_state.spsn = flow_psn & HFI1_KDETH_BTH_SEQ_MASK;
+ flow->flow_state.resp_ib_psn = psn;
+ flow->length = min_t(u32, req->seg_len,
+ (wqe->length - (req->comp_seg * req->seg_len)));
+
+ flow->npkts = rvt_div_round_up_mtu(qp, flow->length);
+ flow->flow_state.lpsn = flow->flow_state.spsn +
+ flow->npkts - 1;
+ /* payload length = packet length - (header length + ICRC length) */
+ pktlen = packet->tlen - (packet->hlen + 4);
+ memcpy(flow->fstate->tid_entry, packet->ebuf, pktlen);
+ flow->tidcnt = pktlen / sizeof(*flow->fstate->tid_entry);
+
+ req->comp_seg++;
+ /*
+ * Walk the TID_ENTRY list to make sure we have enough space for a
+ * complete segment.
+ */
+ for (i = 0; i < flow->tidcnt; i++) {
+ if (!EXP_TID_GET(flow->fstate->tid_entry[i], LEN)) {
+ status = IB_WC_LOC_LEN_ERR;
+ goto ack_err;
+ }
+ tidlen += EXP_TID_GET(flow->fstate->tid_entry[i], LEN);
+ }
+ if (tidlen * PAGE_SIZE < flow->length) {
+ status = IB_WC_LOC_LEN_ERR;
+ goto ack_err;
+ }
+
+ /*
+ * If this is the first response for this request, set the initial
+ * flow index to the current flow.
+ */
+ if (!cmp_psn(psn, wqe->psn)) {
+ req->r_last_acked = mask_psn(wqe->psn - 1);
+ /* Set acked flow index to head index */
+ req->acked_tail = req->setup_head;
+ }
+
+ /* advance circular buffer head */
+ req->setup_head = CIRC_NEXT(req->setup_head, req->n_max_flows);
+ req->state = TID_REQUEST_ACTIVE;
+
+ /*
+ * If all responses for this TID RDMA WRITE request have been received
+ * advance the pointer to the next one.
+ * Since TID RDMA requests could be mixed in with regular IB requests,
+ * they might not appear sequentially in the queue. Therefore, the
+ * next request needs to be "found".
+ */
+ if (qpriv->s_tid_cur != qpriv->s_tid_head &&
+ req->comp_seg == req->total_segs) {
+ for (i = qpriv->s_tid_cur + 1; ; i++) {
+ if (i == qp->s_size)
+ i = 0;
+ wqe = rvt_get_swqe_ptr(qp, i);
+ if (i == qpriv->s_tid_head)
+ break;
+ if (wqe->wr.opcode == IB_WR_TID_RDMA_WRITE)
+ break;
+ }
+ qpriv->s_tid_cur = i;
+ }
+ qp->s_flags &= ~HFI1_S_WAIT_TID_RESP;
+
+ hfi1_schedule_tid_send(qp);
+ goto ack_done;
+
+ack_op_err:
+ status = IB_WC_LOC_QP_OP_ERR;
+ack_err:
+ rvt_error_qp(qp, status);
+ack_done:
+ spin_unlock_irqrestore(&qp->s_lock, flags);
+ if (is_fecn)
+ hfi1_send_rc_ack(packet, is_fecn);
}
void hfi1_rc_rcv_tid_rdma_read_req(struct hfi1_packet *packet)
@@ -1928,6 +2831,14 @@ void hfi1_rc_rcv_tid_rdma_read_req(struct hfi1_packet *packet)
qp->r_head_ack_queue = next;
+ /*
+ * For all requests other than TID WRITE which are added to the ack
+ * queue, qpriv->r_tid_alloc follows qp->r_head_ack_queue. It is ok to
+ * do this because of interlocks between these and TID WRITE
+ * requests. The same change has also been made in hfi1_rc_rcv().
+ */
+ qpriv->r_tid_alloc = qp->r_head_ack_queue;
+
/* Schedule the send tasklet. */
qp->s_flags |= RVT_S_RESP_PENDING;
hfi1_schedule_send(qp);
@@ -1988,7 +2899,7 @@ void hfi1_rc_rcv_tid_rdma_read_resp(struct hfi1_packet *packet)
/*
* 1. Find matching SWQE
* 2. Check that the entire segment has been read.
- * 3. Remove RVT_S_WAIT_TID_RESP from s_flags.
+ * 3. Remove HFI1_S_WAIT_TID_RESP from s_flags.
* 4. Free the TID flow resources.
* 5. Kick the send engine (hfi1_schedule_send())
*/
@@ -2082,10 +2993,420 @@ void hfi1_rc_rcv_tid_rdma_read_resp(struct hfi1_packet *packet)
void hfi1_rc_rcv_tid_rdma_resync(struct hfi1_packet *packet)
{
+ struct ib_other_headers *ohdr = packet->ohdr;
+ struct rvt_qp *qp = packet->qp;
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ struct hfi1_ctxtdata *rcd = qpriv->rcd;
+ struct hfi1_ibdev *dev = to_idev(qp->ibqp.device);
+ struct rvt_ack_entry *e;
+ struct tid_rdma_request *req;
+ struct tid_rdma_flow *flow;
+ struct tid_flow_state *fs = &qpriv->flow_state;
+ u32 psn, generation, idx, gen_next;
+ bool is_fecn;
+ unsigned long flags;
+
+ is_fecn = process_ecn(qp, packet, false);
+ psn = mask_psn(be32_to_cpu(ohdr->bth[2]));
+
+ generation = mask_psn(psn + 1) >> HFI1_KDETH_BTH_SEQ_SHIFT;
+ spin_lock_irqsave(&qp->s_lock, flags);
+
+ gen_next = (fs->generation == KERN_GENERATION_RESERVED) ?
+ generation : kern_flow_generation_next(fs->generation);
+ /*
+ * RESYNC packet contains the "next" generation and can only be
+ * from the current or previous generations
+ */
+ if (generation != mask_generation(gen_next - 1) &&
+ generation != gen_next)
+ goto bail;
+ /* Already processing a resync */
+ if (qpriv->resync)
+ goto bail;
+
+ spin_lock(&rcd->exp_lock);
+ if (fs->index >= RXE_NUM_TID_FLOWS) {
+ /*
+ * If we don't have a flow, save the generation so it can be
+ * applied when a new flow is allocated
+ */
+ fs->generation = generation;
+ } else {
+ /* Reprogram the QP flow with new generation */
+ rcd->flows[fs->index].generation = generation;
+ fs->generation = kern_setup_hw_flow(rcd, fs->index);
+ }
+ fs->psn = 0;
+ /*
+ * Disable SW PSN checking since a RESYNC is equivalent to a
+ * sync point and the flow has/will be reprogrammed
+ */
+ qpriv->s_flags &= ~HFI1_R_TID_SW_PSN;
+ /*
+ * Reset all TID flow information with the new generation.
+ * This is done for all requests and segments after the
+ * last received segment
+ */
+ for (idx = qpriv->r_tid_tail; ; idx++) {
+ u16 flow_idx;
+
+ if (idx > rvt_size_atomic(&dev->rdi))
+ idx = 0;
+ e = &qp->s_ack_queue[idx];
+ if (e->opcode == TID_OP(WRITE_REQ)) {
+ req = ack_to_tid_req(e);
+ /* start from last unacked segment */
+ for (flow_idx = req->clear_tail;
+ CIRC_CNT(req->setup_head, flow_idx,
+ req->n_max_flows);
+ flow_idx = CIRC_NEXT(flow_idx, req->n_max_flows)) {
+ u32 lpsn;
+ u32 next;
+
+ flow = &req->flows[flow_idx];
+ lpsn = full_flow_psn(flow,
+ flow->flow_state.lpsn);
+ next = flow->flow_state.r_next_psn;
+ flow->npkts = delta_psn(lpsn, next - 1);
+ flow->flow_state.generation = fs->generation;
+ flow->flow_state.spsn = fs->psn;
+ flow->flow_state.lpsn =
+ flow->flow_state.spsn + flow->npkts - 1;
+ flow->flow_state.r_next_psn =
+ full_flow_psn(flow,
+ flow->flow_state.spsn);
+ fs->psn += flow->npkts;
+ }
+ }
+ if (idx == qp->s_tail_ack_queue)
+ break;
+ }
+
+ spin_unlock(&rcd->exp_lock);
+ qpriv->resync = true;
+ /* RESYNC request always gets a TID RDMA ACK. */
+ qpriv->s_nak_state = 0;
+ qpriv->s_flags |= RVT_S_ACK_PENDING;
+ hfi1_schedule_tid_send(qp);
+bail:
+ spin_unlock_irqrestore(&qp->s_lock, flags);
}
void hfi1_rc_rcv_tid_rdma_ack(struct hfi1_packet *packet)
{
+ struct ib_other_headers *ohdr = packet->ohdr;
+ struct rvt_qp *qp = packet->qp;
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ struct rvt_swqe *wqe;
+ struct tid_rdma_request *req;
+ struct tid_rdma_flow *flow;
+ u32 aeth, psn, req_psn, ack_psn, fspsn, resync_psn, ack_kpsn;
+ bool is_fecn;
+ unsigned long flags;
+ u16 fidx;
+
+ is_fecn = process_ecn(qp, packet, false);
+ psn = mask_psn(be32_to_cpu(ohdr->bth[2]));
+ aeth = be32_to_cpu(ohdr->u.tid_rdma.ack.aeth);
+ req_psn = mask_psn(be32_to_cpu(ohdr->u.tid_rdma.ack.verbs_psn));
+ resync_psn = mask_psn(be32_to_cpu(ohdr->u.tid_rdma.ack.tid_flow_psn));
+
+ spin_lock_irqsave(&qp->s_lock, flags);
+ /* If we are waiting for an ACK to RESYNC, drop any other packets */
+ if ((qp->s_flags & HFI1_S_WAIT_HALT) &&
+ cmp_psn(psn, qpriv->s_resync_psn))
+ goto ack_op_err;
+
+ ack_psn = req_psn;
+ if (hfi1_tid_rdma_is_resync_psn(psn))
+ ack_kpsn = resync_psn;
+ else
+ ack_kpsn = psn;
+ if (aeth >> 29) {
+ ack_psn--;
+ ack_kpsn--;
+ }
+
+ wqe = rvt_get_swqe_ptr(qp, qp->s_acked);
+
+ if (wqe->wr.opcode != IB_WR_TID_RDMA_WRITE)
+ goto ack_op_err;
+
+ req = wqe_to_tid_req(wqe);
+ flow = &req->flows[req->acked_tail];
+
+ /* Drop stale ACK/NAK */
+ if (cmp_psn(psn, full_flow_psn(flow, flow->flow_state.spsn)) < 0)
+ goto ack_op_err;
+
+ while (cmp_psn(ack_kpsn,
+ full_flow_psn(flow, flow->flow_state.lpsn)) >= 0 &&
+ req->ack_seg < req->cur_seg) {
+ req->ack_seg++;
+ /* advance acked segment pointer */
+ req->acked_tail = CIRC_NEXT(req->acked_tail, req->n_max_flows);
+ req->r_last_acked = flow->flow_state.resp_ib_psn;
+ if (req->ack_seg == req->total_segs) {
+ req->state = TID_REQUEST_COMPLETE;
+ wqe = do_rc_completion(qp, wqe,
+ to_iport(qp->ibqp.device,
+ qp->port_num));
+ atomic_dec(&qpriv->n_tid_requests);
+ if (qp->s_acked == qp->s_tail)
+ break;
+ if (wqe->wr.opcode != IB_WR_TID_RDMA_WRITE)
+ break;
+ req = wqe_to_tid_req(wqe);
+ }
+ flow = &req->flows[req->acked_tail];
+ }
+
+ switch (aeth >> 29) {
+ case 0: /* ACK */
+ if (qpriv->s_flags & RVT_S_WAIT_ACK)
+ qpriv->s_flags &= ~RVT_S_WAIT_ACK;
+ if (!hfi1_tid_rdma_is_resync_psn(psn)) {
+ /* Check if there is any pending TID ACK */
+ if (wqe->wr.opcode == IB_WR_TID_RDMA_WRITE &&
+ req->ack_seg < req->cur_seg)
+ hfi1_mod_tid_retry_timer(qp);
+ else
+ hfi1_stop_tid_retry_timer(qp);
+ hfi1_schedule_send(qp);
+ } else {
+ u32 spsn, fpsn, last_acked, generation;
+ struct tid_rdma_request *rptr;
+
+ /* ACK(RESYNC) */
+ hfi1_stop_tid_retry_timer(qp);
+ /* Allow new requests (see hfi1_make_tid_rdma_pkt) */
+ qp->s_flags &= ~HFI1_S_WAIT_HALT;
+ /*
+ * Clear RVT_S_SEND_ONE flag in case that the TID RDMA
+ * ACK is received after the TID retry timer is fired
+ * again. In this case, do not send any more TID
+ * RESYNC request or wait for any more TID ACK packet.
+ */
+ qpriv->s_flags &= ~RVT_S_SEND_ONE;
+ hfi1_schedule_send(qp);
+
+ if ((qp->s_acked == qpriv->s_tid_tail &&
+ req->ack_seg == req->total_segs) ||
+ qp->s_acked == qp->s_tail) {
+ qpriv->s_state = TID_OP(WRITE_DATA_LAST);
+ goto done;
+ }
+
+ if (req->ack_seg == req->comp_seg) {
+ qpriv->s_state = TID_OP(WRITE_DATA);
+ goto done;
+ }
+
+ /*
+ * The PSN to start with is the next PSN after the
+ * RESYNC PSN.
+ */
+ psn = mask_psn(psn + 1);
+ generation = psn >> HFI1_KDETH_BTH_SEQ_SHIFT;
+ spsn = 0;
+
+ /*
+ * Update to the correct WQE when we get an ACK(RESYNC)
+ * in the middle of a request.
+ */
+ if (delta_psn(ack_psn, wqe->lpsn))
+ wqe = rvt_get_swqe_ptr(qp, qp->s_acked);
+ req = wqe_to_tid_req(wqe);
+ flow = &req->flows[req->acked_tail];
+ /*
+ * RESYNC re-numbers the PSN ranges of all remaining
+ * segments. Also, PSN's start from 0 in the middle of a
+ * segment and the first segment size is less than the
+ * default number of packets. flow->resync_npkts is used
+ * to track the number of packets from the start of the
+ * real segment to the point of 0 PSN after the RESYNC
+ * in order to later correctly rewind the SGE.
+ */
+ fpsn = full_flow_psn(flow, flow->flow_state.spsn);
+ req->r_ack_psn = psn;
+ flow->resync_npkts +=
+ delta_psn(mask_psn(resync_psn + 1), fpsn);
+ /*
+ * Renumber all packet sequence number ranges
+ * based on the new generation.
+ */
+ last_acked = qp->s_acked;
+ rptr = req;
+ while (1) {
+ /* start from last acked segment */
+ for (fidx = rptr->acked_tail;
+ CIRC_CNT(rptr->setup_head, fidx,
+ rptr->n_max_flows);
+ fidx = CIRC_NEXT(fidx,
+ rptr->n_max_flows)) {
+ u32 lpsn;
+ u32 gen;
+
+ flow = &rptr->flows[fidx];
+ gen = flow->flow_state.generation;
+ if (WARN_ON(gen == generation &&
+ flow->flow_state.spsn !=
+ spsn))
+ continue;
+ lpsn = flow->flow_state.lpsn;
+ lpsn = full_flow_psn(flow, lpsn);
+ flow->npkts =
+ delta_psn(lpsn,
+ mask_psn(resync_psn)
+ );
+ flow->flow_state.generation =
+ generation;
+ flow->flow_state.spsn = spsn;
+ flow->flow_state.lpsn =
+ flow->flow_state.spsn +
+ flow->npkts - 1;
+ flow->pkt = 0;
+ spsn += flow->npkts;
+ resync_psn += flow->npkts;
+ }
+ if (++last_acked == qpriv->s_tid_cur + 1)
+ break;
+ if (last_acked == qp->s_size)
+ last_acked = 0;
+ wqe = rvt_get_swqe_ptr(qp, last_acked);
+ rptr = wqe_to_tid_req(wqe);
+ }
+ req->cur_seg = req->ack_seg;
+ qpriv->s_tid_tail = qp->s_acked;
+ qpriv->s_state = TID_OP(WRITE_REQ);
+ hfi1_schedule_tid_send(qp);
+ }
+done:
+ qpriv->s_retry = qp->s_retry_cnt;
+ break;
+
+ case 3: /* NAK */
+ hfi1_stop_tid_retry_timer(qp);
+ switch ((aeth >> IB_AETH_CREDIT_SHIFT) &
+ IB_AETH_CREDIT_MASK) {
+ case 0: /* PSN sequence error */
+ flow = &req->flows[req->acked_tail];
+ fspsn = full_flow_psn(flow, flow->flow_state.spsn);
+ req->r_ack_psn = mask_psn(be32_to_cpu(ohdr->bth[2]));
+ req->cur_seg = req->ack_seg;
+ qpriv->s_tid_tail = qp->s_acked;
+ qpriv->s_state = TID_OP(WRITE_REQ);
+ qpriv->s_retry = qp->s_retry_cnt;
+ hfi1_schedule_tid_send(qp);
+ break;
+
+ default:
+ break;
+ }
+ break;
+
+ default:
+ break;
+ }
+
+ack_op_err:
+ spin_unlock_irqrestore(&qp->s_lock, flags);
+}
+
+static void hfi1_tid_timeout(struct timer_list *t)
+{
+ struct hfi1_qp_priv *qpriv = from_timer(qpriv, t, s_tid_timer);
+ struct rvt_qp *qp = qpriv->owner;
+ struct rvt_dev_info *rdi = ib_to_rvt(qp->ibqp.device);
+ unsigned long flags;
+ u32 i;
+
+ spin_lock_irqsave(&qp->r_lock, flags);
+ spin_lock(&qp->s_lock);
+ if (qpriv->s_flags & HFI1_R_TID_RSC_TIMER) {
+ dd_dev_warn(dd_from_ibdev(qp->ibqp.device), "[QP%u] %s %d\n",
+ qp->ibqp.qp_num, __func__, __LINE__);
+ hfi1_stop_tid_reap_timer(qp);
+ /*
+ * Go though the entire ack queue and clear any outstanding
+ * HW flow and RcvArray resources.
+ */
+ hfi1_kern_clear_hw_flow(qpriv->rcd, qp);
+ for (i = 0; i < rvt_max_atomic(rdi); i++) {
+ struct tid_rdma_request *req =
+ ack_to_tid_req(&qp->s_ack_queue[i]);
+
+ hfi1_kern_exp_rcv_clear_all(req);
+ }
+ spin_unlock(&qp->s_lock);
+ if (qp->ibqp.event_handler) {
+ struct ib_event ev;
+
+ ev.device = qp->ibqp.device;
+ ev.element.qp = &qp->ibqp;
+ ev.event = IB_EVENT_QP_FATAL;
+ qp->ibqp.event_handler(&ev, qp->ibqp.qp_context);
+ }
+ rvt_rc_error(qp, IB_WC_RESP_TIMEOUT_ERR);
+ goto unlock_r_lock;
+ }
+ spin_unlock(&qp->s_lock);
+unlock_r_lock:
+ spin_unlock_irqrestore(&qp->r_lock, flags);
+}
+
+static inline struct rvt_ack_entry *
+find_prev_entry_kdeth(struct rvt_qp *qp, u32 psn, u8 *prev, u8 *prev_ack,
+ bool *scheduled, u16 *fidx)
+ __must_hold(&qp->s_lock)
+{
+ struct rvt_ack_entry *e = NULL;
+ struct tid_rdma_request *req;
+ struct tid_rdma_flow *flow;
+ u32 ibpsn = KDETH_INVALID_PSN;
+ u16 idx = 0;
+ u8 i, p;
+ bool s = true;
+
+ for (i = qp->r_head_ack_queue; ; i = p) {
+ if (i == qp->s_tail_ack_queue)
+ s = false;
+ if (i)
+ p = i - 1;
+ else
+ p = rvt_size_atomic(ib_to_rvt(qp->ibqp.device));
+ if (p == qp->r_head_ack_queue) {
+ e = NULL;
+ break;
+ }
+ e = &qp->s_ack_queue[p];
+ if (!e->opcode) {
+ e = NULL;
+ break;
+ }
+ req = ack_to_tid_req(e);
+ flow = __find_flow_ranged(req, req->n_max_flows - 1, 0,
+ psn, &idx);
+ if (flow) {
+ ibpsn = flow->flow_state.resp_ib_psn;
+ if (cmp_psn(ibpsn, e->psn) >= 0) {
+ if (p == qp->s_tail_ack_queue &&
+ cmp_psn(ibpsn, e->lpsn) <= 0)
+ s = false;
+ break;
+ }
+ }
+ }
+ if (prev)
+ *prev = p;
+ if (prev_ack)
+ *prev_ack = i;
+ if (scheduled)
+ *scheduled = s;
+ if (fidx)
+ *fidx = idx;
+ return e;
}
void hfi1_kern_read_tid_flow_free(struct rvt_qp *qp)
@@ -2100,6 +3421,44 @@ bool hfi1_handle_kdeth_eflags(struct hfi1_ctxtdata *rcd,
return true;
}
+static void hfi1_tid_retry_timeout(struct timer_list *t)
+{
+ struct hfi1_qp_priv *priv = from_timer(priv, t, s_tid_retry_timer);
+ struct rvt_qp *qp = priv->owner;
+ struct rvt_swqe *wqe;
+ unsigned long flags;
+ struct tid_rdma_request *req;
+
+ spin_lock_irqsave(&qp->r_lock, flags);
+ spin_lock(&qp->s_lock);
+ if (priv->s_flags & HFI1_S_TID_RETRY_TIMER) {
+ hfi1_stop_tid_retry_timer(qp);
+ if (!priv->s_retry) {
+ wqe = rvt_get_swqe_ptr(qp, qp->s_acked);
+ hfi1_send_complete(qp, wqe,
+ IB_WC_RETRY_EXC_ERR);
+ rvt_error_qp(qp, IB_WC_WR_FLUSH_ERR);
+ } else {
+ wqe = rvt_get_swqe_ptr(qp, qp->s_acked);
+ req = wqe_to_tid_req(wqe);
+
+ priv->s_flags &= ~RVT_S_WAIT_ACK;
+ /* Only send one packet (the RESYNC) */
+ priv->s_flags |= RVT_S_SEND_ONE;
+ /*
+ * No additional request shall be made by this QP until
+ * the RESYNC has been complete.
+ */
+ qp->s_flags |= HFI1_S_WAIT_HALT;
+ priv->s_state = TID_OP(RESYNC);
+ priv->s_retry--;
+ hfi1_schedule_tid_send(qp);
+ }
+ }
+ spin_unlock(&qp->s_lock);
+ spin_unlock_irqrestore(&qp->r_lock, flags);
+}
+
static bool _hfi1_schedule_tid_send(struct rvt_qp *qp)
{
struct hfi1_qp_priv *priv = qp->priv;
@@ -2131,12 +3490,12 @@ bool hfi1_schedule_tid_send(struct rvt_qp *qp)
{
lockdep_assert_held(&qp->s_lock);
if (hfi1_send_tid_ok(qp)) {
- /*
- * The following call returns true if the qp is not on the
- * queue and false if the qp is already on the queue before
- * this call. Either way, the qp will be on the queue when the
- * call returns.
- */
+ /*
+ * The following call returns true if the qp is not on the
+ * queue and false if the qp is already on the queue before
+ * this call. Either way, the qp will be on the queue when the
+ * call returns.
+ */
_hfi1_schedule_tid_send(qp);
return true;
}
@@ -2186,10 +3545,21 @@ int hfi1_qp_priv_init(struct rvt_dev_info *rdi, struct rvt_qp *qp,
spin_lock_init(&qpriv->opfn.lock);
INIT_WORK(&qpriv->opfn.opfn_work, opfn_send_conn_request);
INIT_WORK(&qpriv->tid_rdma.trigger_work, tid_rdma_trigger_resume);
+ qpriv->r_tid_tail = qp->s_tail_ack_queue;
qpriv->flow_state.psn = 0;
qpriv->flow_state.index = RXE_NUM_TID_FLOWS;
qpriv->flow_state.last_index = RXE_NUM_TID_FLOWS;
qpriv->flow_state.generation = KERN_GENERATION_RESERVED;
+ qpriv->s_state = TID_OP(WRITE_RESP);
+ qpriv->s_tid_cur = HFI1_QP_WQE_INVALID;
+ qpriv->s_tid_tail = HFI1_QP_WQE_INVALID;
+ qpriv->r_tid_tail = HFI1_QP_WQE_INVALID;
+ qpriv->r_tid_ack = HFI1_QP_WQE_INVALID;
+ atomic_set(&qpriv->n_requests, 0);
+ atomic_set(&qpriv->n_tid_requests, 0);
+ timer_setup(&qpriv->s_tid_timer, hfi1_tid_timeout, 0);
+ timer_setup(&qpriv->s_tid_retry_timer, hfi1_tid_retry_timeout, 0);
+ INIT_LIST_HEAD(&qpriv->tid_wait);
if (init_attr->qp_type == IB_QPT_RC && HFI1_CAP_IS_KSET(TID_RDMA)) {
struct hfi1_devdata *dd = qpriv->rcd->dd;
@@ -2199,7 +3569,6 @@ 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);
@@ -2285,6 +3654,49 @@ void hfi1_qp_priv_tid_free(struct rvt_dev_info *rdi, struct rvt_qp *qp)
void hfi1_qp_kern_exp_rcv_clear_all(struct rvt_qp *qp)
{
+ int i, ret;
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ struct tid_flow_state *fs;
+
+ if (qp->ibqp.qp_type != IB_QPT_RC || !HFI1_CAP_IS_KSET(TID_RDMA))
+ return;
+
+ /*
+ * First, clear the flow to help prevent any delayed packets from
+ * being delivered.
+ */
+ fs = &qpriv->flow_state;
+ if (fs->index != RXE_NUM_TID_FLOWS)
+ hfi1_kern_clear_hw_flow(qpriv->rcd, qp);
+
+ for (i = qp->s_acked; i != qp->s_head;) {
+ struct rvt_swqe *wqe = rvt_get_swqe_ptr(qp, i);
+
+ if (++i == qp->s_size)
+ i = 0;
+ /* Free only locally allocated TID entries */
+ if (wqe->wr.opcode != IB_WR_TID_RDMA_READ)
+ continue;
+ do {
+ struct hfi1_swqe_priv *priv = wqe->priv;
+
+ ret = hfi1_kern_exp_rcv_clear(&priv->tid_req);
+ } while (!ret);
+ }
+ for (i = qp->s_acked_ack_queue; i != qp->r_head_ack_queue;) {
+ struct rvt_ack_entry *e = &qp->s_ack_queue[i];
+
+ if (++i == rvt_max_atomic(ib_to_rvt(qp->ibqp.device)))
+ i = 0;
+ /* Free only locally allocated TID entries */
+ if (e->opcode != TID_OP(WRITE_REQ))
+ continue;
+ do {
+ struct hfi1_ack_priv *priv = e->priv;
+
+ ret = hfi1_kern_exp_rcv_clear(&priv->tid_req);
+ } while (!ret);
+ }
}
u64 hfi1_access_sw_tid_wait(const struct cntr_entry *entry,
@@ -2295,25 +3707,527 @@ u64 hfi1_access_sw_tid_wait(const struct cntr_entry *entry,
return dd->verbs_dev.n_tidwait;
}
+static bool build_tid_rdma_packet(struct rvt_swqe *wqe,
+ struct ib_other_headers *ohdr,
+ u32 *bth1, u32 *bth2, u32 *len)
+{
+ struct tid_rdma_request *req = wqe_to_tid_req(wqe);
+ struct tid_rdma_flow *flow = &req->flows[req->clear_tail];
+ struct tid_rdma_params *remote;
+ struct rvt_qp *qp = req->qp;
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ u32 tidentry = flow->fstate->tid_entry[flow->tid_idx];
+ u32 tidlen = EXP_TID_GET(tidentry, LEN) << PAGE_SHIFT;
+ struct tid_rdma_write_data *wd = &ohdr->u.tid_rdma.w_data;
+ u32 next_offset, om = KDETH_OM_LARGE;
+ bool last_pkt;
+
+ if (!tidlen) {
+ hfi1_send_complete(qp, wqe, IB_WC_REM_INV_RD_REQ_ERR);
+ rvt_error_qp(qp, IB_WC_REM_INV_RD_REQ_ERR);
+ }
+
+ *len = min_t(u32, qp->pmtu, tidlen - flow->tid_offset);
+ flow->sent += *len;
+ next_offset = flow->tid_offset + *len;
+ last_pkt = (flow->tid_idx == (flow->tidcnt - 1) &&
+ next_offset >= tidlen) || (flow->sent >= flow->length);
+
+ rcu_read_lock();
+ remote = rcu_dereference(qpriv->tid_rdma.remote);
+ KDETH_RESET(wd->kdeth0, KVER, 0x1);
+ KDETH_SET(wd->kdeth0, SH, !last_pkt);
+ KDETH_SET(wd->kdeth0, INTR, !!(!last_pkt && remote->urg));
+ KDETH_SET(wd->kdeth0, TIDCTRL, EXP_TID_GET(tidentry, CTRL));
+ KDETH_SET(wd->kdeth0, TID, EXP_TID_GET(tidentry, IDX));
+ KDETH_SET(wd->kdeth0, OM, om == KDETH_OM_LARGE);
+ KDETH_SET(wd->kdeth0, OFFSET, flow->tid_offset / om);
+ KDETH_RESET(wd->kdeth1, JKEY, remote->jkey);
+ wd->verbs_qp = cpu_to_be32(qp->remote_qpn);
+ rcu_read_unlock();
+
+ *bth1 = flow->tid_qpn;
+ *bth2 = mask_psn(((flow->flow_state.spsn + flow->pkt++) &
+ HFI1_KDETH_BTH_SEQ_MASK) |
+ (flow->flow_state.generation <<
+ HFI1_KDETH_BTH_SEQ_SHIFT));
+ if (last_pkt) {
+ /* PSNs are zero-based, so +1 to count number of packets */
+ if (flow->flow_state.lpsn + 1 +
+ rvt_div_round_up_mtu(qp, req->seg_len) >
+ MAX_TID_FLOW_PSN)
+ req->state = TID_REQUEST_SYNC;
+ *bth2 |= IB_BTH_REQ_ACK;
+ }
+
+ if (next_offset >= tidlen) {
+ flow->tid_offset = 0;
+ flow->tid_idx++;
+ } else {
+ flow->tid_offset = next_offset;
+ }
+ return last_pkt;
+}
+
+static u32 build_tid_rdma_write_ack(struct rvt_qp *qp, struct rvt_ack_entry *e,
+ struct ib_other_headers *ohdr, u16 iflow,
+ u32 *bth1, u32 *bth2)
+{
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ struct tid_flow_state *fs = &qpriv->flow_state;
+ struct tid_rdma_request *req = ack_to_tid_req(e);
+ struct tid_rdma_flow *flow = &req->flows[iflow];
+ struct tid_rdma_params *remote;
+
+ rcu_read_lock();
+ remote = rcu_dereference(qpriv->tid_rdma.remote);
+ KDETH_RESET(ohdr->u.tid_rdma.ack.kdeth1, JKEY, remote->jkey);
+ ohdr->u.tid_rdma.ack.verbs_qp = cpu_to_be32(qp->remote_qpn);
+ *bth1 = remote->qp;
+ rcu_read_unlock();
+
+ if (qpriv->resync) {
+ *bth2 = mask_psn((fs->generation <<
+ HFI1_KDETH_BTH_SEQ_SHIFT) - 1);
+ ohdr->u.tid_rdma.ack.aeth = rvt_compute_aeth(qp);
+ } else if (qpriv->s_nak_state) {
+ *bth2 = mask_psn(qpriv->s_nak_psn);
+ ohdr->u.tid_rdma.ack.aeth =
+ cpu_to_be32((qp->r_msn & IB_MSN_MASK) |
+ (qpriv->s_nak_state <<
+ IB_AETH_CREDIT_SHIFT));
+ } else {
+ *bth2 = full_flow_psn(flow, flow->flow_state.lpsn);
+ ohdr->u.tid_rdma.ack.aeth = rvt_compute_aeth(qp);
+ }
+ KDETH_RESET(ohdr->u.tid_rdma.ack.kdeth0, KVER, 0x1);
+ ohdr->u.tid_rdma.ack.tid_flow_qp =
+ cpu_to_be32(qpriv->tid_rdma.local.qp |
+ ((flow->idx & TID_RDMA_DESTQP_FLOW_MASK) <<
+ TID_RDMA_DESTQP_FLOW_SHIFT) |
+ qpriv->rcd->ctxt);
+
+ ohdr->u.tid_rdma.ack.tid_flow_psn = 0;
+ ohdr->u.tid_rdma.ack.verbs_psn =
+ cpu_to_be32(flow->flow_state.resp_ib_psn);
+
+ if (qpriv->resync) {
+ /*
+ * If the PSN before the current expect KDETH PSN is the
+ * RESYNC PSN, then we never received a good TID RDMA WRITE
+ * DATA packet after a previous RESYNC.
+ * In this case, the next expected KDETH PSN stays the same.
+ */
+ if (hfi1_tid_rdma_is_resync_psn(qpriv->r_next_psn_kdeth - 1)) {
+ ohdr->u.tid_rdma.ack.tid_flow_psn =
+ cpu_to_be32(qpriv->r_next_psn_kdeth_save);
+ } else {
+ /*
+ * Because the KDETH PSNs jump during a RESYNC, it's
+ * not possible to infer (or compute) the previous value
+ * of r_next_psn_kdeth in the case of back-to-back
+ * RESYNC packets. Therefore, we save it.
+ */
+ qpriv->r_next_psn_kdeth_save =
+ qpriv->r_next_psn_kdeth - 1;
+ ohdr->u.tid_rdma.ack.tid_flow_psn =
+ cpu_to_be32(qpriv->r_next_psn_kdeth_save);
+ qpriv->r_next_psn_kdeth = mask_psn(*bth2 + 1);
+ }
+ qpriv->resync = false;
+ }
+
+ return sizeof(ohdr->u.tid_rdma.ack) / sizeof(u32);
+}
+
+static int make_tid_rdma_ack(struct rvt_qp *qp,
+ struct ib_other_headers *ohdr,
+ struct hfi1_pkt_state *ps)
+{
+ struct rvt_ack_entry *e;
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ struct hfi1_ibdev *dev = to_idev(qp->ibqp.device);
+ u32 hwords, next;
+ u32 len = 0;
+ u32 bth1 = 0, bth2 = 0;
+ int middle = 0;
+ u16 flow;
+ struct tid_rdma_request *req, *nreq;
+
+ /* Don't send an ACK if we aren't supposed to. */
+ if (!(ib_rvt_state_ops[qp->state] & RVT_PROCESS_RECV_OK))
+ goto bail;
+
+ /* header size in 32-bit words LRH+BTH = (8+12)/4. */
+ hwords = 5;
+
+ e = &qp->s_ack_queue[qpriv->r_tid_ack];
+ req = ack_to_tid_req(e);
+ /*
+ * In the RESYNC case, we are exactly one segment past the
+ * previously sent ack or at the previously sent NAK. So to send
+ * the resync ack, we go back one segment (which might be part of
+ * the previous request) and let the do-while loop execute again.
+ * The advantage of executing the do-while loop is that any data
+ * received after the previous ack is automatically acked in the
+ * RESYNC ack. It turns out that for the do-while loop we only need
+ * to pull back qpriv->r_tid_ack, not the segment
+ * indices/counters. The scheme works even if the previous request
+ * was not a TID WRITE request.
+ */
+ if (qpriv->resync) {
+ if (!req->ack_seg || req->ack_seg == req->total_segs)
+ qpriv->r_tid_ack = !qpriv->r_tid_ack ?
+ rvt_size_atomic(&dev->rdi) :
+ qpriv->r_tid_ack - 1;
+ e = &qp->s_ack_queue[qpriv->r_tid_ack];
+ req = ack_to_tid_req(e);
+ }
+
+ /*
+ * If we've sent all the ACKs that we can, we are done
+ * until we get more segments...
+ */
+ if (!qpriv->s_nak_state && !qpriv->resync &&
+ req->ack_seg == req->comp_seg)
+ goto bail;
+
+ do {
+ /*
+ * To deal with coalesced ACKs, the acked_tail pointer
+ * into the flow array is used. The distance between it
+ * and the clear_tail is the number of flows that are
+ * being ACK'ed.
+ */
+ req->ack_seg +=
+ /* Get up-to-date value */
+ CIRC_CNT(req->clear_tail, req->acked_tail,
+ req->n_max_flows);
+ /* Advance acked index */
+ req->acked_tail = req->clear_tail;
+
+ /*
+ * req->clear_tail points to the segment currently being
+ * received. So, when sending an ACK, the previous
+ * segment is being ACK'ed.
+ */
+ flow = CIRC_PREV(req->acked_tail, req->n_max_flows);
+ if (req->ack_seg != req->total_segs)
+ break;
+ req->state = TID_REQUEST_COMPLETE;
+
+ next = qpriv->r_tid_ack + 1;
+ if (next > rvt_size_atomic(&dev->rdi))
+ next = 0;
+ qpriv->r_tid_ack = next;
+ if (qp->s_ack_queue[next].opcode != TID_OP(WRITE_REQ))
+ break;
+ nreq = ack_to_tid_req(&qp->s_ack_queue[next]);
+ if (!nreq->comp_seg || nreq->ack_seg == nreq->comp_seg)
+ break;
+
+ /* Move to the next ack entry now */
+ e = &qp->s_ack_queue[qpriv->r_tid_ack];
+ req = ack_to_tid_req(e);
+ } while (1);
+
+ /*
+ * At this point qpriv->r_tid_ack == qpriv->r_tid_tail but e and
+ * req could be pointing at the previous ack queue entry
+ */
+ if (qpriv->s_nak_state ||
+ (qpriv->resync &&
+ !hfi1_tid_rdma_is_resync_psn(qpriv->r_next_psn_kdeth - 1) &&
+ (cmp_psn(qpriv->r_next_psn_kdeth - 1,
+ full_flow_psn(&req->flows[flow],
+ req->flows[flow].flow_state.lpsn)) > 0))) {
+ /*
+ * A NAK will implicitly acknowledge all previous TID RDMA
+ * requests. Therefore, we NAK with the req->acked_tail
+ * segment for the request at qpriv->r_tid_ack (same at
+ * this point as the req->clear_tail segment for the
+ * qpriv->r_tid_tail request)
+ */
+ e = &qp->s_ack_queue[qpriv->r_tid_ack];
+ req = ack_to_tid_req(e);
+ flow = req->acked_tail;
+ } else if (req->ack_seg == req->total_segs &&
+ qpriv->s_flags & HFI1_R_TID_WAIT_INTERLCK)
+ qpriv->s_flags &= ~HFI1_R_TID_WAIT_INTERLCK;
+
+ hwords += build_tid_rdma_write_ack(qp, e, ohdr, flow, &bth1,
+ &bth2);
+ len = 0;
+ qpriv->s_flags &= ~RVT_S_ACK_PENDING;
+ ps->s_txreq->hdr_dwords = hwords;
+ ps->s_txreq->sde = qpriv->s_sde;
+ ps->s_txreq->s_cur_size = len;
+ ps->s_txreq->ss = NULL;
+ hfi1_make_ruc_header(qp, ohdr, (TID_OP(ACK) << 24), bth1, bth2, middle,
+ ps);
+ return 1;
+bail:
+ /*
+ * Ensure s_rdma_ack_cnt changes are committed prior to resetting
+ * RVT_S_RESP_PENDING
+ */
+ smp_wmb();
+ qpriv->s_flags &= ~RVT_S_ACK_PENDING;
+ return 0;
+}
+
+/*
+ * Call this function when the last TID RDMA WRITE DATA packet for a request
+ * is built.
+ */
+static void update_tid_tail(struct rvt_qp *qp)
+ __must_hold(&qp->s_lock)
+{
+ struct hfi1_qp_priv *priv = qp->priv;
+ u32 i;
+ struct rvt_swqe *wqe;
+
+ lockdep_assert_held(&qp->s_lock);
+ /* Can't move beyond s_tid_cur */
+ if (priv->s_tid_tail == priv->s_tid_cur)
+ return;
+ for (i = priv->s_tid_tail + 1; ; i++) {
+ if (i == qp->s_size)
+ i = 0;
+
+ if (i == priv->s_tid_cur)
+ break;
+ wqe = rvt_get_swqe_ptr(qp, i);
+ if (wqe->wr.opcode == IB_WR_TID_RDMA_WRITE)
+ break;
+ }
+ priv->s_tid_tail = i;
+ priv->s_state = TID_OP(WRITE_RESP);
+}
+
+static u32 build_tid_rdma_resync(struct rvt_qp *qp, struct rvt_swqe *wqe,
+ struct ib_other_headers *ohdr, u32 *bth1,
+ u32 *bth2, u16 fidx)
+{
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ struct tid_rdma_params *remote;
+ struct tid_rdma_request *req = wqe_to_tid_req(wqe);
+ struct tid_rdma_flow *flow = &req->flows[fidx];
+ u32 generation;
+
+ rcu_read_lock();
+ remote = rcu_dereference(qpriv->tid_rdma.remote);
+ KDETH_RESET(ohdr->u.tid_rdma.ack.kdeth1, JKEY, remote->jkey);
+ ohdr->u.tid_rdma.ack.verbs_qp = cpu_to_be32(qp->remote_qpn);
+ *bth1 = remote->qp;
+ rcu_read_unlock();
+
+ generation = kern_flow_generation_next(flow->flow_state.generation);
+ *bth2 = mask_psn((generation << HFI1_KDETH_BTH_SEQ_SHIFT) - 1);
+ qpriv->s_resync_psn = *bth2;
+ *bth2 |= IB_BTH_REQ_ACK;
+ KDETH_RESET(ohdr->u.tid_rdma.ack.kdeth0, KVER, 0x1);
+
+ return sizeof(ohdr->u.tid_rdma.resync) / sizeof(u32);
+}
+
static int hfi1_make_tid_rdma_pkt(struct rvt_qp *qp, struct hfi1_pkt_state *ps)
__must_hold(&qp->s_lock)
{
+ struct hfi1_qp_priv *priv = qp->priv;
+ struct rvt_swqe *wqe;
+ u32 bth1 = 0, bth2 = 0, hwords = 5, len, middle = 0;
+ struct ib_other_headers *ohdr;
+ struct rvt_sge_state *ss = &qp->s_sge;
+ struct rvt_ack_entry *e = &qp->s_ack_queue[qp->s_tail_ack_queue];
+ struct tid_rdma_request *req = ack_to_tid_req(e);
+ bool last = false;
+ u8 opcode = TID_OP(WRITE_DATA);
+
+ lockdep_assert_held(&qp->s_lock);
+ /*
+ * Prioritize the sending of the requests and responses over the
+ * sending of the TID RDMA data packets.
+ */
+ if (((atomic_read(&priv->n_tid_requests) < HFI1_TID_RDMA_WRITE_CNT) &&
+ atomic_read(&priv->n_requests) &&
+ !(qp->s_flags & (RVT_S_BUSY | RVT_S_WAIT_ACK |
+ HFI1_S_ANY_WAIT_IO))) ||
+ (e->opcode == TID_OP(WRITE_REQ) && req->cur_seg < req->alloc_seg &&
+ !(qp->s_flags & (RVT_S_BUSY | HFI1_S_ANY_WAIT_IO)))) {
+ struct iowait_work *iowork;
+
+ iowork = iowait_get_ib_work(&priv->s_iowait);
+ ps->s_txreq = get_waiting_verbs_txreq(iowork);
+ if (ps->s_txreq || hfi1_make_rc_req(qp, ps)) {
+ priv->s_flags |= HFI1_S_TID_BUSY_SET;
+ return 1;
+ }
+ }
+
+ ps->s_txreq = get_txreq(ps->dev, qp);
+ if (!ps->s_txreq)
+ goto bail_no_tx;
+
+ ohdr = &ps->s_txreq->phdr.hdr.ibh.u.oth;
+
+ if ((priv->s_flags & RVT_S_ACK_PENDING) &&
+ make_tid_rdma_ack(qp, ohdr, ps))
+ return 1;
+
+ if (!(ib_rvt_state_ops[qp->state] & RVT_PROCESS_SEND_OK)) {
+ if (!(ib_rvt_state_ops[qp->state] & RVT_FLUSH_SEND))
+ goto bail;
+ /* We are in the error state, flush the work request. */
+ if (qp->s_last == READ_ONCE(qp->s_head))
+ goto bail;
+ /* If DMAs are in progress, we can't flush immediately. */
+ if (iowait_sdma_pending(&priv->s_iowait)) {
+ qp->s_flags |= RVT_S_WAIT_DMA;
+ goto bail;
+ }
+ clear_ahg(qp);
+ wqe = rvt_get_swqe_ptr(qp, qp->s_last);
+ hfi1_send_complete(qp, wqe, qp->s_last != qp->s_acked ?
+ IB_WC_SUCCESS : IB_WC_WR_FLUSH_ERR);
+ /* will get called again */
+ goto done_free_tx;
+ }
+
+ if (priv->s_flags & RVT_S_WAIT_ACK)
+ goto bail;
+
+ /* Check whether there is anything to do. */
+ if (priv->s_tid_tail == HFI1_QP_WQE_INVALID)
+ goto bail;
+ wqe = rvt_get_swqe_ptr(qp, priv->s_tid_tail);
+ req = wqe_to_tid_req(wqe);
+ switch (priv->s_state) {
+ case TID_OP(WRITE_REQ):
+ case TID_OP(WRITE_RESP):
+ priv->tid_ss.sge = wqe->sg_list[0];
+ priv->tid_ss.sg_list = wqe->sg_list + 1;
+ priv->tid_ss.num_sge = wqe->wr.num_sge;
+ priv->tid_ss.total_len = wqe->length;
+
+ if (priv->s_state == TID_OP(WRITE_REQ))
+ hfi1_tid_rdma_restart_req(qp, wqe, &bth2);
+ priv->s_state = TID_OP(WRITE_DATA);
+ /* fall through */
+
+ case TID_OP(WRITE_DATA):
+ /*
+ * 1. Check whether TID RDMA WRITE RESP available.
+ * 2. If no:
+ * 2.1 If have more segments and no TID RDMA WRITE RESP,
+ * set HFI1_S_WAIT_TID_RESP
+ * 2.2 Return indicating no progress made.
+ * 3. If yes:
+ * 3.1 Build TID RDMA WRITE DATA packet.
+ * 3.2 If last packet in segment:
+ * 3.2.1 Change KDETH header bits
+ * 3.2.2 Advance RESP pointers.
+ * 3.3 Return indicating progress made.
+ */
+ wqe = rvt_get_swqe_ptr(qp, priv->s_tid_tail);
+ req = wqe_to_tid_req(wqe);
+ len = wqe->length;
+
+ if (!req->comp_seg || req->cur_seg == req->comp_seg)
+ goto bail;
+
+ last = build_tid_rdma_packet(wqe, ohdr, &bth1, &bth2, &len);
+
+ if (last) {
+ /* move pointer to next flow */
+ req->clear_tail = CIRC_NEXT(req->clear_tail,
+ req->n_max_flows);
+ if (++req->cur_seg < req->total_segs) {
+ if (!CIRC_CNT(req->setup_head, req->clear_tail,
+ req->n_max_flows))
+ qp->s_flags |= HFI1_S_WAIT_TID_RESP;
+ } else {
+ priv->s_state = TID_OP(WRITE_DATA_LAST);
+ opcode = TID_OP(WRITE_DATA_LAST);
+
+ /* Advance the s_tid_tail now */
+ update_tid_tail(qp);
+ }
+ }
+ hwords += sizeof(ohdr->u.tid_rdma.w_data) / sizeof(u32);
+ ss = &priv->tid_ss;
+ break;
+
+ case TID_OP(RESYNC):
+ /* Use generation from the most recently received response */
+ wqe = rvt_get_swqe_ptr(qp, priv->s_tid_cur);
+ req = wqe_to_tid_req(wqe);
+ /* If no responses for this WQE look at the previous one */
+ if (!req->comp_seg) {
+ wqe = rvt_get_swqe_ptr(qp,
+ (!priv->s_tid_cur ? qp->s_size :
+ priv->s_tid_cur) - 1);
+ req = wqe_to_tid_req(wqe);
+ }
+ hwords += build_tid_rdma_resync(qp, wqe, ohdr, &bth1, &bth2,
+ CIRC_PREV(req->setup_head,
+ req->n_max_flows));
+ ss = NULL;
+ len = 0;
+ opcode = TID_OP(RESYNC);
+ break;
+
+ default:
+ goto bail;
+ }
+ if (priv->s_flags & RVT_S_SEND_ONE) {
+ priv->s_flags &= ~RVT_S_SEND_ONE;
+ priv->s_flags |= RVT_S_WAIT_ACK;
+ bth2 |= IB_BTH_REQ_ACK;
+ }
+ qp->s_len -= len;
+ ps->s_txreq->hdr_dwords = hwords;
+ ps->s_txreq->sde = priv->s_sde;
+ ps->s_txreq->ss = ss;
+ ps->s_txreq->s_cur_size = len;
+ hfi1_make_ruc_header(qp, ohdr, (opcode << 24), bth1, bth2,
+ middle, ps);
+ return 1;
+done_free_tx:
+ hfi1_put_txreq(ps->s_txreq);
+ ps->s_txreq = NULL;
+ return 1;
+
+bail:
+ hfi1_put_txreq(ps->s_txreq);
+bail_no_tx:
+ ps->s_txreq = NULL;
+ priv->s_flags &= ~RVT_S_BUSY;
+ /*
+ * If we didn't get a txreq, the QP will be woken up later to try
+ * again, set the flags to the the wake up which work item to wake
+ * up.
+ * (A better algorithm should be found to do this and generalize the
+ * sleep/wakeup flags.)
+ */
+ iowait_set_flag(&priv->s_iowait, IOWAIT_PENDING_TID);
return 0;
}
/*
- * "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.
+ * "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;
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ int diff, delta_pkts;
+ u32 tididx = 0, i;
u16 fidx;
if (wqe->wr.opcode == IB_WR_TID_RDMA_READ) {
@@ -2322,10 +4236,19 @@ void hfi1_tid_rdma_restart_req(struct rvt_qp *qp, struct rvt_swqe *wqe,
if (!flow)
return;
} else {
- return;
+ fidx = req->acked_tail;
+ flow = &req->flows[fidx];
+ *bth2 = mask_psn(req->r_ack_psn);
}
- diff = delta_psn(*bth2, flow->flow_state.ib_spsn);
+ if (wqe->wr.opcode == IB_WR_TID_RDMA_READ)
+ delta_pkts = delta_psn(*bth2, flow->flow_state.ib_spsn);
+ else
+ delta_pkts = delta_psn(*bth2,
+ full_flow_psn(flow,
+ flow->flow_state.spsn));
+
+ diff = delta_pkts + flow->resync_npkts;
flow->sent = 0;
flow->pkt = 0;
@@ -2334,7 +4257,7 @@ void hfi1_tid_rdma_restart_req(struct rvt_qp *qp, struct rvt_swqe *wqe,
if (diff) {
for (tididx = 0; tididx < flow->tidcnt; tididx++) {
u32 tidentry = flow->fstate->tid_entry[tididx], tidlen,
- tidnpkts, npkts;
+ tidnpkts, npkts;
flow->tid_offset = 0;
tidlen = EXP_TID_GET(tidentry, LEN) * PAGE_SIZE;
@@ -2349,17 +4272,59 @@ void hfi1_tid_rdma_restart_req(struct rvt_qp *qp, struct rvt_swqe *wqe,
break;
}
}
+ if (wqe->wr.opcode == IB_WR_TID_RDMA_WRITE) {
+ rvt_skip_sge(&qpriv->tid_ss, (req->cur_seg * req->seg_len) +
+ flow->sent, 0);
+ /*
+ * Packet PSN is based on flow_state.spsn + flow->pkt. However,
+ * during a RESYNC, the generation is incremented and the
+ * sequence is reset to 0. Since we've adjusted the npkts in the
+ * flow and the SGE has been sufficiently advanced, we have to
+ * adjust flow->pkt in order to calculate the correct PSN.
+ */
+ flow->pkt -= flow->resync_npkts;
+ }
if (flow->tid_offset ==
- EXP_TID_GET(flow->fstate->tid_entry[tididx], LEN) * PAGE_SIZE) {
+ 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;
+ if (wqe->wr.opcode == IB_WR_TID_RDMA_READ)
+ /* Move flow_idx to correct index */
+ req->flow_idx = fidx;
+ else
+ req->clear_tail = fidx;
req->state = TID_REQUEST_ACTIVE;
+ if (wqe->wr.opcode == IB_WR_TID_RDMA_WRITE) {
+ /* Reset all the flows that we are going to resend */
+ fidx = CIRC_NEXT(fidx, req->n_max_flows);
+ i = qpriv->s_tid_tail;
+ do {
+ for (;
+ CIRC_CNT(req->setup_head, fidx, req->n_max_flows);
+ fidx = CIRC_NEXT(fidx, req->n_max_flows)) {
+ req->flows[fidx].sent = 0;
+ req->flows[fidx].pkt = 0;
+ req->flows[fidx].tid_idx = 0;
+ req->flows[fidx].tid_offset = 0;
+ req->flows[fidx].resync_npkts = 0;
+ }
+ if (i == qpriv->s_tid_cur)
+ break;
+ do {
+ i = (++i == qp->s_size ? 0 : i);
+ wqe = rvt_get_swqe_ptr(qp, i);
+ } while (wqe->wr.opcode != IB_WR_TID_RDMA_WRITE);
+ req = wqe_to_tid_req(wqe);
+ req->cur_seg = req->ack_seg;
+ fidx = req->acked_tail;
+ /* Pull req->clear_tail back */
+ req->clear_tail = fidx;
+ } while (1);
+ }
}
void _hfi1_do_tid_send(struct work_struct *work)
@@ -2442,7 +4407,36 @@ u32 hfi1_build_tid_rdma_write_req(struct rvt_qp *qp, struct rvt_swqe *wqe,
struct ib_other_headers *ohdr,
u32 *bth1, u32 *bth2, u32 *len)
{
- return 0;
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ struct tid_rdma_request *req = wqe_to_tid_req(wqe);
+ struct tid_rdma_params *remote;
+
+ rcu_read_lock();
+ remote = rcu_dereference(qpriv->tid_rdma.remote);
+ /*
+ * Set the number of flow to be used based on negotiated
+ * parameters.
+ */
+ req->n_flows = remote->max_write;
+ req->state = TID_REQUEST_ACTIVE;
+
+ KDETH_RESET(ohdr->u.tid_rdma.w_req.kdeth0, KVER, 0x1);
+ KDETH_RESET(ohdr->u.tid_rdma.w_req.kdeth1, JKEY, remote->jkey);
+ ohdr->u.tid_rdma.w_req.reth.vaddr =
+ cpu_to_be64(wqe->rdma_wr.remote_addr + (wqe->length - *len));
+ ohdr->u.tid_rdma.w_req.reth.rkey =
+ cpu_to_be32(wqe->rdma_wr.rkey);
+ ohdr->u.tid_rdma.w_req.reth.length = cpu_to_be32(*len);
+ ohdr->u.tid_rdma.w_req.verbs_qp = cpu_to_be32(qp->remote_qpn);
+ *bth1 &= ~RVT_QPN_MASK;
+ *bth1 |= remote->qp;
+ qp->s_state = TID_OP(WRITE_REQ);
+ qp->s_flags |= HFI1_S_WAIT_TID_RESP;
+ *bth2 |= IB_BTH_REQ_ACK;
+ *len = 0;
+
+ rcu_read_unlock();
+ return sizeof(ohdr->u.tid_rdma.w_req) / sizeof(u32);
}
u32 hfi1_build_tid_rdma_write_resp(struct rvt_qp *qp, struct rvt_ack_entry *e,
@@ -2450,7 +4444,97 @@ u32 hfi1_build_tid_rdma_write_resp(struct rvt_qp *qp, struct rvt_ack_entry *e,
u32 bth2, u32 *len,
struct rvt_sge_state **ss)
{
- return 0;
+ struct hfi1_ack_priv *epriv = e->priv;
+ struct tid_rdma_request *req = &epriv->tid_req;
+ struct hfi1_qp_priv *qpriv = qp->priv;
+ struct tid_rdma_flow *flow = NULL;
+ u32 resp_len = 0, hdwords = 0;
+ void *resp_addr = NULL;
+ struct tid_rdma_params *remote;
+
+ flow = &req->flows[req->flow_idx];
+ switch (req->state) {
+ default:
+ /*
+ * Try to allocate resources here in case QP was queued and was
+ * later scheduled when resources became available
+ */
+ hfi1_tid_write_alloc_resources(qp, false);
+
+ /* We've already sent everything which is ready */
+ if (req->cur_seg >= req->alloc_seg)
+ goto done;
+
+ /*
+ * Resources can be assigned but responses cannot be sent in
+ * rnr_nak state, till the resent request is received
+ */
+ if (qpriv->rnr_nak_state == TID_RNR_NAK_SENT)
+ goto done;
+
+ req->state = TID_REQUEST_ACTIVE;
+ req->flow_idx = CIRC_NEXT(req->flow_idx, req->n_max_flows);
+ hfi1_add_tid_reap_timer(qp);
+ break;
+
+ case TID_REQUEST_RESEND_ACTIVE:
+ case TID_REQUEST_RESEND:
+ req->flow_idx = CIRC_NEXT(req->flow_idx, req->n_max_flows);
+ if (!CIRC_CNT(req->setup_head, req->flow_idx,
+ req->n_max_flows))
+ req->state = TID_REQUEST_ACTIVE;
+
+ hfi1_mod_tid_reap_timer(qp);
+ break;
+ }
+ flow->flow_state.resp_ib_psn = bth2;
+ resp_addr = (void *)flow->fstate->tid_entry;
+ resp_len = sizeof(*flow->fstate->tid_entry) * flow->tidcnt;
+ req->cur_seg++;
+
+ memset(&ohdr->u.tid_rdma.w_rsp, 0, sizeof(ohdr->u.tid_rdma.w_rsp));
+ epriv->ss.sge.vaddr = resp_addr;
+ epriv->ss.sge.sge_length = resp_len;
+ epriv->ss.sge.length = epriv->ss.sge.sge_length;
+ /*
+ * We can safely zero these out. Since the first SGE covers the
+ * entire packet, nothing else should even look at the MR.
+ */
+ epriv->ss.sge.mr = NULL;
+ epriv->ss.sge.m = 0;
+ epriv->ss.sge.n = 0;
+
+ epriv->ss.sg_list = NULL;
+ epriv->ss.total_len = epriv->ss.sge.sge_length;
+ epriv->ss.num_sge = 1;
+
+ *ss = &epriv->ss;
+ *len = epriv->ss.total_len;
+
+ /* Construct the TID RDMA WRITE RESP packet header */
+ rcu_read_lock();
+ remote = rcu_dereference(qpriv->tid_rdma.remote);
+
+ KDETH_RESET(ohdr->u.tid_rdma.w_rsp.kdeth0, KVER, 0x1);
+ KDETH_RESET(ohdr->u.tid_rdma.w_rsp.kdeth1, JKEY, remote->jkey);
+ ohdr->u.tid_rdma.w_rsp.aeth = rvt_compute_aeth(qp);
+ ohdr->u.tid_rdma.w_rsp.tid_flow_psn =
+ cpu_to_be32((flow->flow_state.generation <<
+ HFI1_KDETH_BTH_SEQ_SHIFT) |
+ (flow->flow_state.spsn &
+ HFI1_KDETH_BTH_SEQ_MASK));
+ ohdr->u.tid_rdma.w_rsp.tid_flow_qp =
+ cpu_to_be32(qpriv->tid_rdma.local.qp |
+ ((flow->idx & TID_RDMA_DESTQP_FLOW_MASK) <<
+ TID_RDMA_DESTQP_FLOW_SHIFT) |
+ qpriv->rcd->ctxt);
+ ohdr->u.tid_rdma.w_rsp.verbs_qp = cpu_to_be32(qp->remote_qpn);
+ *bth1 = remote->qp;
+ rcu_read_unlock();
+ hdwords = sizeof(ohdr->u.tid_rdma.w_rsp) / sizeof(u32);
+ qpriv->pending_tid_w_segs++;
+done:
+ return hdwords;
}
u32 hfi1_build_tid_rdma_read_packet(struct rvt_swqe *wqe,
@@ -2539,7 +4623,7 @@ u32 hfi1_build_tid_rdma_read_packet(struct rvt_swqe *wqe,
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)
+ __must_hold(&qp->s_lock)
{
struct hfi1_qp_priv *qpriv = qp->priv;
struct tid_rdma_request *req = wqe_to_tid_req(wqe);
@@ -182,6 +182,7 @@ struct hfi1_qp_priv {
u8 s_state;
u8 s_nak_state;
u8 s_retry;
+ u32 s_nak_psn;
u32 s_flags;
u32 s_tid_cur;
u32 s_tid_head;
@@ -203,6 +204,8 @@ struct hfi1_qp_priv {
u8 rnr_nak_state; /* RNR NAK state */
u32 r_next_psn_ib;
u32 r_next_psn_kdeth;
+ u32 r_next_psn_kdeth_save;
+ u32 s_resync_psn;
bool sync_pt; /* Set when QP reaches sync point */
bool resync;
};