mirror of
https://github.com/AuxXxilium/linux_dsm_epyc7002.git
synced 2024-12-22 21:47:04 +07:00
IB/hfi1: Add functions to receive TID RDMA WRITE request
This patch adds the functions to receive TID RDMA WRITE request. The request will be stored in the QP's s_ack_queue. This patch also adds code to handle duplicate TID RDMA WRITE request and a function to allocate TID resources for data receiving on the responder side. Signed-off-by: Mitko Haralanov <mitko.haralanov@intel.com> Signed-off-by: Mike Marciniszyn <mike.marciniszyn@intel.com> Signed-off-by: Ashutosh Dixit <ashutosh.dixit@intel.com> Signed-off-by: Kaike Wan <kaike.wan@intel.com> Signed-off-by: Dennis Dalessandro <dennis.dalessandro@intel.com> Signed-off-by: Doug Ledford <dledford@redhat.com>
This commit is contained in:
parent
4f9264d156
commit
07b923701e
@ -1512,6 +1512,7 @@ static int __init hfi1_mod_init(void)
|
||||
goto bail_dev;
|
||||
}
|
||||
|
||||
hfi1_compute_tid_rdma_flow_wt();
|
||||
/*
|
||||
* These must be called before the driver is registered with
|
||||
* the PCI subsystem.
|
||||
|
@ -2411,6 +2411,7 @@ void hfi1_rc_rcv(struct hfi1_packet *packet)
|
||||
void *data = packet->payload;
|
||||
u32 tlen = packet->tlen;
|
||||
struct rvt_qp *qp = packet->qp;
|
||||
struct hfi1_qp_priv *qpriv = qp->priv;
|
||||
struct hfi1_ibport *ibp = rcd_to_iport(rcd);
|
||||
struct ib_other_headers *ohdr = packet->ohdr;
|
||||
u32 opcode = packet->opcode;
|
||||
@ -2716,6 +2717,7 @@ void hfi1_rc_rcv(struct hfi1_packet *packet)
|
||||
qp->r_state = opcode;
|
||||
qp->r_nak_state = 0;
|
||||
qp->r_head_ack_queue = next;
|
||||
qpriv->r_tid_alloc = qp->r_head_ack_queue;
|
||||
|
||||
/* Schedule the send engine. */
|
||||
qp->s_flags |= RVT_S_RESP_PENDING;
|
||||
@ -2789,6 +2791,7 @@ void hfi1_rc_rcv(struct hfi1_packet *packet)
|
||||
qp->r_state = opcode;
|
||||
qp->r_nak_state = 0;
|
||||
qp->r_head_ack_queue = next;
|
||||
qpriv->r_tid_alloc = qp->r_head_ack_queue;
|
||||
|
||||
/* Schedule the send engine. */
|
||||
qp->s_flags |= RVT_S_RESP_PENDING;
|
||||
|
@ -109,12 +109,15 @@ static u32 mask_generation(u32 a)
|
||||
* C - Capcode
|
||||
*/
|
||||
|
||||
static u32 tid_rdma_flow_wt;
|
||||
|
||||
static void tid_rdma_trigger_resume(struct work_struct *work);
|
||||
static void hfi1_kern_exp_rcv_free_flows(struct tid_rdma_request *req);
|
||||
static int hfi1_kern_exp_rcv_alloc_flows(struct tid_rdma_request *req,
|
||||
gfp_t gfp);
|
||||
static void hfi1_init_trdma_req(struct rvt_qp *qp,
|
||||
struct tid_rdma_request *req);
|
||||
static void hfi1_tid_write_alloc_resources(struct rvt_qp *qp, bool intr_ctx);
|
||||
|
||||
static u64 tid_rdma_opfn_encode(struct tid_rdma_params *p)
|
||||
{
|
||||
@ -313,6 +316,11 @@ int hfi1_qp_priv_init(struct rvt_dev_info *rdi, struct rvt_qp *qp,
|
||||
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->rnr_nak_state = TID_RNR_NAK_INIT;
|
||||
qpriv->r_tid_head = HFI1_QP_WQE_INVALID;
|
||||
qpriv->r_tid_tail = HFI1_QP_WQE_INVALID;
|
||||
qpriv->r_tid_ack = HFI1_QP_WQE_INVALID;
|
||||
qpriv->r_tid_alloc = HFI1_QP_WQE_INVALID;
|
||||
INIT_LIST_HEAD(&qpriv->tid_wait);
|
||||
|
||||
if (init_attr->qp_type == IB_QPT_RC && HFI1_CAP_IS_KSET(TID_RDMA)) {
|
||||
@ -1959,6 +1967,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;
|
||||
@ -1982,7 +1992,8 @@ static int tid_rdma_rcv_error(struct hfi1_packet *packet,
|
||||
|
||||
spin_lock_irqsave(&qp->s_lock, flags);
|
||||
e = find_prev_entry(qp, psn, &prev, NULL, &old_req);
|
||||
if (!e || e->opcode != TID_OP(READ_REQ))
|
||||
if (!e || (e->opcode != TID_OP(READ_REQ) &&
|
||||
e->opcode != TID_OP(WRITE_REQ)))
|
||||
goto unlock;
|
||||
|
||||
req = ack_to_tid_req(e);
|
||||
@ -2042,6 +2053,114 @@ static int tid_rdma_rcv_error(struct hfi1_packet *packet,
|
||||
*/
|
||||
if (old_req)
|
||||
goto unlock;
|
||||
} else {
|
||||
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, MAX_FLOWS)) {
|
||||
fstate = &req->flows[req->clear_tail].flow_state;
|
||||
qpriv->pending_tid_w_segs -=
|
||||
CIRC_CNT(req->flow_idx, req->clear_tail,
|
||||
MAX_FLOWS);
|
||||
req->flow_idx =
|
||||
CIRC_ADD(req->clear_tail,
|
||||
delta_psn(psn, fstate->resp_ib_psn),
|
||||
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,
|
||||
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);
|
||||
trace_hfi1_tid_req_rcv_err(qp, 0, e->opcode, e->psn,
|
||||
e->lpsn, req);
|
||||
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,
|
||||
MAX_FLOWS);
|
||||
req->flow_idx = req->clear_tail;
|
||||
req->state = TID_REQUEST_RESEND;
|
||||
req->cur_seg = req->comp_seg;
|
||||
}
|
||||
}
|
||||
/* Re-process old requests.*/
|
||||
if (qp->s_acked_ack_queue == qp->s_tail_ack_queue)
|
||||
@ -2054,6 +2173,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;
|
||||
@ -2164,6 +2295,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);
|
||||
@ -3015,3 +3154,432 @@ u32 hfi1_build_tid_rdma_write_req(struct rvt_qp *qp, struct rvt_swqe *wqe,
|
||||
rcu_read_unlock();
|
||||
return sizeof(ohdr->u.tid_rdma.w_req) / sizeof(u32);
|
||||
}
|
||||
|
||||
void hfi1_compute_tid_rdma_flow_wt(void)
|
||||
{
|
||||
/*
|
||||
* Heuristic for computing the RNR timeout when waiting on the flow
|
||||
* queue. Rather than a computationaly expensive exact estimate of when
|
||||
* a flow will be available, we assume that if a QP is at position N in
|
||||
* the flow queue it has to wait approximately (N + 1) * (number of
|
||||
* segments between two sync points), assuming PMTU of 4K. The rationale
|
||||
* for this is that flows are released and recycled at each sync point.
|
||||
*/
|
||||
tid_rdma_flow_wt = MAX_TID_FLOW_PSN * enum_to_mtu(OPA_MTU_4096) /
|
||||
TID_RDMA_MAX_SEGMENT_SIZE;
|
||||
}
|
||||
|
||||
static u32 position_in_queue(struct hfi1_qp_priv *qpriv,
|
||||
struct tid_queue *queue)
|
||||
{
|
||||
return qpriv->tid_enqueue - queue->dequeue;
|
||||
}
|
||||
|
||||
/*
|
||||
* @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;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 up to 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
|
||||
* (MAX_FLOWS / 2) segments
|
||||
*/
|
||||
if (!CIRC_SPACE(req->setup_head, req->acked_tail,
|
||||
MAX_FLOWS)) {
|
||||
ret = -EAGAIN;
|
||||
to_seg = MAX_FLOWS >> 1;
|
||||
qpriv->s_flags |= RVT_S_ACK_PENDING;
|
||||
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:
|
||||
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);
|
||||
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);
|
||||
}
|
||||
|
@ -26,7 +26,9 @@
|
||||
*
|
||||
* HFI1_S_TID_WAIT_INTERLCK - QP is waiting for requester interlock
|
||||
*/
|
||||
/* BIT(4) reserved for RVT_S_ACK_PENDING. */
|
||||
#define HFI1_S_TID_WAIT_INTERLCK BIT(5)
|
||||
#define HFI1_R_TID_SW_PSN BIT(19)
|
||||
|
||||
/*
|
||||
* Unlike regular IB RDMA VERBS, which do not require an entry
|
||||
@ -89,10 +91,12 @@ struct tid_rdma_request {
|
||||
} e;
|
||||
|
||||
struct tid_rdma_flow *flows; /* array of tid flows */
|
||||
struct rvt_sge_state ss; /* SGE state for TID RDMA requests */
|
||||
u16 n_flows; /* size of the flow buffer window */
|
||||
u16 setup_head; /* flow index we are setting up */
|
||||
u16 clear_tail; /* flow index we are clearing */
|
||||
u16 flow_idx; /* flow index most recently set up */
|
||||
u16 acked_tail;
|
||||
|
||||
u32 seg_len;
|
||||
u32 total_len;
|
||||
@ -103,6 +107,7 @@ struct tid_rdma_request {
|
||||
u32 cur_seg; /* index of current segment */
|
||||
u32 comp_seg; /* index of last completed segment */
|
||||
u32 ack_seg; /* index of last ack'ed segment */
|
||||
u32 alloc_seg; /* index of next segment to be allocated */
|
||||
u32 isge; /* index of "current" sge */
|
||||
u32 ack_pending; /* num acks pending for this request */
|
||||
|
||||
@ -174,6 +179,12 @@ struct tid_rdma_flow {
|
||||
u32 tid_entry[TID_RDMA_MAX_PAGES];
|
||||
};
|
||||
|
||||
enum tid_rnr_nak_state {
|
||||
TID_RNR_NAK_INIT = 0,
|
||||
TID_RNR_NAK_SEND,
|
||||
TID_RNR_NAK_SENT,
|
||||
};
|
||||
|
||||
bool tid_rdma_conn_req(struct rvt_qp *qp, u64 *data);
|
||||
bool tid_rdma_conn_reply(struct rvt_qp *qp, u64 data);
|
||||
bool tid_rdma_conn_resp(struct rvt_qp *qp, u64 *data);
|
||||
@ -247,4 +258,9 @@ static inline void hfi1_setup_tid_rdma_wqe(struct rvt_qp *qp,
|
||||
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);
|
||||
|
||||
void hfi1_compute_tid_rdma_flow_wt(void);
|
||||
|
||||
void hfi1_rc_rcv_tid_rdma_write_req(struct hfi1_packet *packet);
|
||||
|
||||
#endif /* HFI1_TID_RDMA_H */
|
||||
|
@ -172,7 +172,15 @@ struct hfi1_qp_priv {
|
||||
unsigned long tid_timer_timeout_jiffies;
|
||||
|
||||
/* variables for the TID RDMA SE state machine */
|
||||
u8 rnr_nak_state; /* RNR NAK state */
|
||||
u32 s_flags;
|
||||
u32 r_tid_head; /* Most recently added TID RDMA request */
|
||||
u32 r_tid_tail; /* the last completed TID RDMA request */
|
||||
u32 r_tid_ack; /* the TID RDMA request to be ACK'ed */
|
||||
u32 r_tid_alloc; /* Request for which we are allocating resources */
|
||||
u32 pending_tid_w_segs; /* Num of pending tid write segments */
|
||||
u32 alloc_w_segs; /* Number of segments for which write */
|
||||
/* resources have been allocated for this QP */
|
||||
|
||||
/* For TID RDMA READ */
|
||||
u32 tid_r_reqs; /* Num of tid reads requested */
|
||||
@ -180,8 +188,12 @@ struct hfi1_qp_priv {
|
||||
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 */
|
||||
|
||||
u8 sync_pt; /* Set when QP reaches sync point */
|
||||
};
|
||||
|
||||
#define HFI1_QP_WQE_INVALID ((u32)-1)
|
||||
|
||||
struct hfi1_swqe_priv {
|
||||
struct tid_rdma_request tid_req;
|
||||
struct rvt_sge_state ss; /* Used for TID RDMA READ Request */
|
||||
|
Loading…
Reference in New Issue
Block a user