c9358de193
The hfi1 user SDMA pinned-page cache will leave a stale cache entry when the cache-entry's virtual address range is invalidated but that cache entry is in-use by an outstanding SDMA request. Subsequent user SDMA requests with buffers in or spanning the virtual address range of the stale cache entry will result in packets constructed from the wrong memory, the physical pages pointed to by the stale cache entry. To fix this, remove mmu_rb_node cache entries from the mmu_rb_handler cache independent of the cache entry's refcount. Add 'struct kref refcount' to struct mmu_rb_node and manage mmu_rb_node lifetime with kref_get() and kref_put(). mmu_rb_node.refcount makes sdma_mmu_node.refcount redundant. Remove 'atomic_t refcount' from struct sdma_mmu_node and change sdma_mmu_node code to use mmu_rb_node.refcount. Move the mmu_rb_handler destructor call after a wait-for-SDMA-request-completion call so mmu_rb_nodes that need mmu_rb_handler's workqueue to queue themselves up for destruction from an interrupt context may do so. Fixes: f48ad614c100 ("IB/hfi1: Move driver out of staging") Fixes: 00cbce5cbf88 ("IB/hfi1: Fix bugs with non-PAGE_SIZE-end multi-iovec user SDMA requests") Link: https://lore.kernel.org/r/168451393605.3700681.13493776139032178861.stgit@awfm-02.cornelisnetworks.com Reviewed-by: Dean Luick <dean.luick@cornelisnetworks.com> Signed-off-by: Brendan Cunningham <bcunningham@cornelisnetworks.com> Signed-off-by: Dennis Dalessandro <dennis.dalessandro@cornelisnetworks.com> Signed-off-by: Jason Gunthorpe <jgg@nvidia.com>
1654 lines
45 KiB
C
1654 lines
45 KiB
C
// SPDX-License-Identifier: GPL-2.0 or BSD-3-Clause
|
|
/*
|
|
* Copyright(c) 2020 - Cornelis Networks, Inc.
|
|
* Copyright(c) 2015 - 2018 Intel Corporation.
|
|
*/
|
|
|
|
#include <linux/mm.h>
|
|
#include <linux/types.h>
|
|
#include <linux/device.h>
|
|
#include <linux/dmapool.h>
|
|
#include <linux/slab.h>
|
|
#include <linux/list.h>
|
|
#include <linux/highmem.h>
|
|
#include <linux/io.h>
|
|
#include <linux/uio.h>
|
|
#include <linux/rbtree.h>
|
|
#include <linux/spinlock.h>
|
|
#include <linux/delay.h>
|
|
#include <linux/kthread.h>
|
|
#include <linux/mmu_context.h>
|
|
#include <linux/module.h>
|
|
#include <linux/vmalloc.h>
|
|
#include <linux/string.h>
|
|
|
|
#include "hfi.h"
|
|
#include "sdma.h"
|
|
#include "user_sdma.h"
|
|
#include "verbs.h" /* for the headers */
|
|
#include "common.h" /* for struct hfi1_tid_info */
|
|
#include "trace.h"
|
|
|
|
static uint hfi1_sdma_comp_ring_size = 128;
|
|
module_param_named(sdma_comp_size, hfi1_sdma_comp_ring_size, uint, S_IRUGO);
|
|
MODULE_PARM_DESC(sdma_comp_size, "Size of User SDMA completion ring. Default: 128");
|
|
|
|
static unsigned initial_pkt_count = 8;
|
|
|
|
static int user_sdma_send_pkts(struct user_sdma_request *req, u16 maxpkts);
|
|
static void user_sdma_txreq_cb(struct sdma_txreq *txreq, int status);
|
|
static inline void pq_update(struct hfi1_user_sdma_pkt_q *pq);
|
|
static void user_sdma_free_request(struct user_sdma_request *req);
|
|
static int check_header_template(struct user_sdma_request *req,
|
|
struct hfi1_pkt_header *hdr, u32 lrhlen,
|
|
u32 datalen);
|
|
static int set_txreq_header(struct user_sdma_request *req,
|
|
struct user_sdma_txreq *tx, u32 datalen);
|
|
static int set_txreq_header_ahg(struct user_sdma_request *req,
|
|
struct user_sdma_txreq *tx, u32 len);
|
|
static inline void set_comp_state(struct hfi1_user_sdma_pkt_q *pq,
|
|
struct hfi1_user_sdma_comp_q *cq,
|
|
u16 idx, enum hfi1_sdma_comp_state state,
|
|
int ret);
|
|
static inline u32 set_pkt_bth_psn(__be32 bthpsn, u8 expct, u32 frags);
|
|
static inline u32 get_lrh_len(struct hfi1_pkt_header, u32 len);
|
|
|
|
static int defer_packet_queue(
|
|
struct sdma_engine *sde,
|
|
struct iowait_work *wait,
|
|
struct sdma_txreq *txreq,
|
|
uint seq,
|
|
bool pkts_sent);
|
|
static void activate_packet_queue(struct iowait *wait, int reason);
|
|
static bool sdma_rb_filter(struct mmu_rb_node *node, unsigned long addr,
|
|
unsigned long len);
|
|
static int sdma_rb_evict(void *arg, struct mmu_rb_node *mnode,
|
|
void *arg2, bool *stop);
|
|
static void sdma_rb_remove(void *arg, struct mmu_rb_node *mnode);
|
|
|
|
static struct mmu_rb_ops sdma_rb_ops = {
|
|
.filter = sdma_rb_filter,
|
|
.evict = sdma_rb_evict,
|
|
.remove = sdma_rb_remove,
|
|
};
|
|
|
|
static int add_system_pages_to_sdma_packet(struct user_sdma_request *req,
|
|
struct user_sdma_txreq *tx,
|
|
struct user_sdma_iovec *iovec,
|
|
u32 *pkt_remaining);
|
|
|
|
static int defer_packet_queue(
|
|
struct sdma_engine *sde,
|
|
struct iowait_work *wait,
|
|
struct sdma_txreq *txreq,
|
|
uint seq,
|
|
bool pkts_sent)
|
|
{
|
|
struct hfi1_user_sdma_pkt_q *pq =
|
|
container_of(wait->iow, struct hfi1_user_sdma_pkt_q, busy);
|
|
|
|
write_seqlock(&sde->waitlock);
|
|
trace_hfi1_usdma_defer(pq, sde, &pq->busy);
|
|
if (sdma_progress(sde, seq, txreq))
|
|
goto eagain;
|
|
/*
|
|
* We are assuming that if the list is enqueued somewhere, it
|
|
* is to the dmawait list since that is the only place where
|
|
* it is supposed to be enqueued.
|
|
*/
|
|
xchg(&pq->state, SDMA_PKT_Q_DEFERRED);
|
|
if (list_empty(&pq->busy.list)) {
|
|
pq->busy.lock = &sde->waitlock;
|
|
iowait_get_priority(&pq->busy);
|
|
iowait_queue(pkts_sent, &pq->busy, &sde->dmawait);
|
|
}
|
|
write_sequnlock(&sde->waitlock);
|
|
return -EBUSY;
|
|
eagain:
|
|
write_sequnlock(&sde->waitlock);
|
|
return -EAGAIN;
|
|
}
|
|
|
|
static void activate_packet_queue(struct iowait *wait, int reason)
|
|
{
|
|
struct hfi1_user_sdma_pkt_q *pq =
|
|
container_of(wait, struct hfi1_user_sdma_pkt_q, busy);
|
|
|
|
trace_hfi1_usdma_activate(pq, wait, reason);
|
|
xchg(&pq->state, SDMA_PKT_Q_ACTIVE);
|
|
wake_up(&wait->wait_dma);
|
|
};
|
|
|
|
int hfi1_user_sdma_alloc_queues(struct hfi1_ctxtdata *uctxt,
|
|
struct hfi1_filedata *fd)
|
|
{
|
|
int ret = -ENOMEM;
|
|
char buf[64];
|
|
struct hfi1_devdata *dd;
|
|
struct hfi1_user_sdma_comp_q *cq;
|
|
struct hfi1_user_sdma_pkt_q *pq;
|
|
|
|
if (!uctxt || !fd)
|
|
return -EBADF;
|
|
|
|
if (!hfi1_sdma_comp_ring_size)
|
|
return -EINVAL;
|
|
|
|
dd = uctxt->dd;
|
|
|
|
pq = kzalloc(sizeof(*pq), GFP_KERNEL);
|
|
if (!pq)
|
|
return -ENOMEM;
|
|
pq->dd = dd;
|
|
pq->ctxt = uctxt->ctxt;
|
|
pq->subctxt = fd->subctxt;
|
|
pq->n_max_reqs = hfi1_sdma_comp_ring_size;
|
|
atomic_set(&pq->n_reqs, 0);
|
|
init_waitqueue_head(&pq->wait);
|
|
atomic_set(&pq->n_locked, 0);
|
|
|
|
iowait_init(&pq->busy, 0, NULL, NULL, defer_packet_queue,
|
|
activate_packet_queue, NULL, NULL);
|
|
pq->reqidx = 0;
|
|
|
|
pq->reqs = kcalloc(hfi1_sdma_comp_ring_size,
|
|
sizeof(*pq->reqs),
|
|
GFP_KERNEL);
|
|
if (!pq->reqs)
|
|
goto pq_reqs_nomem;
|
|
|
|
pq->req_in_use = bitmap_zalloc(hfi1_sdma_comp_ring_size, GFP_KERNEL);
|
|
if (!pq->req_in_use)
|
|
goto pq_reqs_no_in_use;
|
|
|
|
snprintf(buf, 64, "txreq-kmem-cache-%u-%u-%u", dd->unit, uctxt->ctxt,
|
|
fd->subctxt);
|
|
pq->txreq_cache = kmem_cache_create(buf,
|
|
sizeof(struct user_sdma_txreq),
|
|
L1_CACHE_BYTES,
|
|
SLAB_HWCACHE_ALIGN,
|
|
NULL);
|
|
if (!pq->txreq_cache) {
|
|
dd_dev_err(dd, "[%u] Failed to allocate TxReq cache\n",
|
|
uctxt->ctxt);
|
|
goto pq_txreq_nomem;
|
|
}
|
|
|
|
cq = kzalloc(sizeof(*cq), GFP_KERNEL);
|
|
if (!cq)
|
|
goto cq_nomem;
|
|
|
|
cq->comps = vmalloc_user(PAGE_ALIGN(sizeof(*cq->comps)
|
|
* hfi1_sdma_comp_ring_size));
|
|
if (!cq->comps)
|
|
goto cq_comps_nomem;
|
|
|
|
cq->nentries = hfi1_sdma_comp_ring_size;
|
|
|
|
ret = hfi1_mmu_rb_register(pq, &sdma_rb_ops, dd->pport->hfi1_wq,
|
|
&pq->handler);
|
|
if (ret) {
|
|
dd_dev_err(dd, "Failed to register with MMU %d", ret);
|
|
goto pq_mmu_fail;
|
|
}
|
|
|
|
rcu_assign_pointer(fd->pq, pq);
|
|
fd->cq = cq;
|
|
|
|
return 0;
|
|
|
|
pq_mmu_fail:
|
|
vfree(cq->comps);
|
|
cq_comps_nomem:
|
|
kfree(cq);
|
|
cq_nomem:
|
|
kmem_cache_destroy(pq->txreq_cache);
|
|
pq_txreq_nomem:
|
|
bitmap_free(pq->req_in_use);
|
|
pq_reqs_no_in_use:
|
|
kfree(pq->reqs);
|
|
pq_reqs_nomem:
|
|
kfree(pq);
|
|
|
|
return ret;
|
|
}
|
|
|
|
static void flush_pq_iowait(struct hfi1_user_sdma_pkt_q *pq)
|
|
{
|
|
unsigned long flags;
|
|
seqlock_t *lock = pq->busy.lock;
|
|
|
|
if (!lock)
|
|
return;
|
|
write_seqlock_irqsave(lock, flags);
|
|
if (!list_empty(&pq->busy.list)) {
|
|
list_del_init(&pq->busy.list);
|
|
pq->busy.lock = NULL;
|
|
}
|
|
write_sequnlock_irqrestore(lock, flags);
|
|
}
|
|
|
|
int hfi1_user_sdma_free_queues(struct hfi1_filedata *fd,
|
|
struct hfi1_ctxtdata *uctxt)
|
|
{
|
|
struct hfi1_user_sdma_pkt_q *pq;
|
|
|
|
trace_hfi1_sdma_user_free_queues(uctxt->dd, uctxt->ctxt, fd->subctxt);
|
|
|
|
spin_lock(&fd->pq_rcu_lock);
|
|
pq = srcu_dereference_check(fd->pq, &fd->pq_srcu,
|
|
lockdep_is_held(&fd->pq_rcu_lock));
|
|
if (pq) {
|
|
rcu_assign_pointer(fd->pq, NULL);
|
|
spin_unlock(&fd->pq_rcu_lock);
|
|
synchronize_srcu(&fd->pq_srcu);
|
|
/* at this point there can be no more new requests */
|
|
iowait_sdma_drain(&pq->busy);
|
|
/* Wait until all requests have been freed. */
|
|
wait_event_interruptible(
|
|
pq->wait,
|
|
!atomic_read(&pq->n_reqs));
|
|
kfree(pq->reqs);
|
|
if (pq->handler)
|
|
hfi1_mmu_rb_unregister(pq->handler);
|
|
bitmap_free(pq->req_in_use);
|
|
kmem_cache_destroy(pq->txreq_cache);
|
|
flush_pq_iowait(pq);
|
|
kfree(pq);
|
|
} else {
|
|
spin_unlock(&fd->pq_rcu_lock);
|
|
}
|
|
if (fd->cq) {
|
|
vfree(fd->cq->comps);
|
|
kfree(fd->cq);
|
|
fd->cq = NULL;
|
|
}
|
|
return 0;
|
|
}
|
|
|
|
static u8 dlid_to_selector(u16 dlid)
|
|
{
|
|
static u8 mapping[256];
|
|
static int initialized;
|
|
static u8 next;
|
|
int hash;
|
|
|
|
if (!initialized) {
|
|
memset(mapping, 0xFF, 256);
|
|
initialized = 1;
|
|
}
|
|
|
|
hash = ((dlid >> 8) ^ dlid) & 0xFF;
|
|
if (mapping[hash] == 0xFF) {
|
|
mapping[hash] = next;
|
|
next = (next + 1) & 0x7F;
|
|
}
|
|
|
|
return mapping[hash];
|
|
}
|
|
|
|
/**
|
|
* hfi1_user_sdma_process_request() - Process and start a user sdma request
|
|
* @fd: valid file descriptor
|
|
* @iovec: array of io vectors to process
|
|
* @dim: overall iovec array size
|
|
* @count: number of io vector array entries processed
|
|
*/
|
|
int hfi1_user_sdma_process_request(struct hfi1_filedata *fd,
|
|
struct iovec *iovec, unsigned long dim,
|
|
unsigned long *count)
|
|
{
|
|
int ret = 0, i;
|
|
struct hfi1_ctxtdata *uctxt = fd->uctxt;
|
|
struct hfi1_user_sdma_pkt_q *pq =
|
|
srcu_dereference(fd->pq, &fd->pq_srcu);
|
|
struct hfi1_user_sdma_comp_q *cq = fd->cq;
|
|
struct hfi1_devdata *dd = pq->dd;
|
|
unsigned long idx = 0;
|
|
u8 pcount = initial_pkt_count;
|
|
struct sdma_req_info info;
|
|
struct user_sdma_request *req;
|
|
u8 opcode, sc, vl;
|
|
u16 pkey;
|
|
u32 slid;
|
|
u16 dlid;
|
|
u32 selector;
|
|
|
|
if (iovec[idx].iov_len < sizeof(info) + sizeof(req->hdr)) {
|
|
hfi1_cdbg(
|
|
SDMA,
|
|
"[%u:%u:%u] First vector not big enough for header %lu/%lu",
|
|
dd->unit, uctxt->ctxt, fd->subctxt,
|
|
iovec[idx].iov_len, sizeof(info) + sizeof(req->hdr));
|
|
return -EINVAL;
|
|
}
|
|
ret = copy_from_user(&info, iovec[idx].iov_base, sizeof(info));
|
|
if (ret) {
|
|
hfi1_cdbg(SDMA, "[%u:%u:%u] Failed to copy info QW (%d)",
|
|
dd->unit, uctxt->ctxt, fd->subctxt, ret);
|
|
return -EFAULT;
|
|
}
|
|
|
|
trace_hfi1_sdma_user_reqinfo(dd, uctxt->ctxt, fd->subctxt,
|
|
(u16 *)&info);
|
|
if (info.comp_idx >= hfi1_sdma_comp_ring_size) {
|
|
hfi1_cdbg(SDMA,
|
|
"[%u:%u:%u:%u] Invalid comp index",
|
|
dd->unit, uctxt->ctxt, fd->subctxt, info.comp_idx);
|
|
return -EINVAL;
|
|
}
|
|
|
|
/*
|
|
* Sanity check the header io vector count. Need at least 1 vector
|
|
* (header) and cannot be larger than the actual io vector count.
|
|
*/
|
|
if (req_iovcnt(info.ctrl) < 1 || req_iovcnt(info.ctrl) > dim) {
|
|
hfi1_cdbg(SDMA,
|
|
"[%u:%u:%u:%u] Invalid iov count %d, dim %ld",
|
|
dd->unit, uctxt->ctxt, fd->subctxt, info.comp_idx,
|
|
req_iovcnt(info.ctrl), dim);
|
|
return -EINVAL;
|
|
}
|
|
|
|
if (!info.fragsize) {
|
|
hfi1_cdbg(SDMA,
|
|
"[%u:%u:%u:%u] Request does not specify fragsize",
|
|
dd->unit, uctxt->ctxt, fd->subctxt, info.comp_idx);
|
|
return -EINVAL;
|
|
}
|
|
|
|
/* Try to claim the request. */
|
|
if (test_and_set_bit(info.comp_idx, pq->req_in_use)) {
|
|
hfi1_cdbg(SDMA, "[%u:%u:%u] Entry %u is in use",
|
|
dd->unit, uctxt->ctxt, fd->subctxt,
|
|
info.comp_idx);
|
|
return -EBADSLT;
|
|
}
|
|
/*
|
|
* All safety checks have been done and this request has been claimed.
|
|
*/
|
|
trace_hfi1_sdma_user_process_request(dd, uctxt->ctxt, fd->subctxt,
|
|
info.comp_idx);
|
|
req = pq->reqs + info.comp_idx;
|
|
req->data_iovs = req_iovcnt(info.ctrl) - 1; /* subtract header vector */
|
|
req->data_len = 0;
|
|
req->pq = pq;
|
|
req->cq = cq;
|
|
req->ahg_idx = -1;
|
|
req->iov_idx = 0;
|
|
req->sent = 0;
|
|
req->seqnum = 0;
|
|
req->seqcomp = 0;
|
|
req->seqsubmitted = 0;
|
|
req->tids = NULL;
|
|
req->has_error = 0;
|
|
INIT_LIST_HEAD(&req->txps);
|
|
|
|
memcpy(&req->info, &info, sizeof(info));
|
|
|
|
/* The request is initialized, count it */
|
|
atomic_inc(&pq->n_reqs);
|
|
|
|
if (req_opcode(info.ctrl) == EXPECTED) {
|
|
/* expected must have a TID info and at least one data vector */
|
|
if (req->data_iovs < 2) {
|
|
SDMA_DBG(req,
|
|
"Not enough vectors for expected request");
|
|
ret = -EINVAL;
|
|
goto free_req;
|
|
}
|
|
req->data_iovs--;
|
|
}
|
|
|
|
if (!info.npkts || req->data_iovs > MAX_VECTORS_PER_REQ) {
|
|
SDMA_DBG(req, "Too many vectors (%u/%u)", req->data_iovs,
|
|
MAX_VECTORS_PER_REQ);
|
|
ret = -EINVAL;
|
|
goto free_req;
|
|
}
|
|
|
|
/* Copy the header from the user buffer */
|
|
ret = copy_from_user(&req->hdr, iovec[idx].iov_base + sizeof(info),
|
|
sizeof(req->hdr));
|
|
if (ret) {
|
|
SDMA_DBG(req, "Failed to copy header template (%d)", ret);
|
|
ret = -EFAULT;
|
|
goto free_req;
|
|
}
|
|
|
|
/* If Static rate control is not enabled, sanitize the header. */
|
|
if (!HFI1_CAP_IS_USET(STATIC_RATE_CTRL))
|
|
req->hdr.pbc[2] = 0;
|
|
|
|
/* Validate the opcode. Do not trust packets from user space blindly. */
|
|
opcode = (be32_to_cpu(req->hdr.bth[0]) >> 24) & 0xff;
|
|
if ((opcode & USER_OPCODE_CHECK_MASK) !=
|
|
USER_OPCODE_CHECK_VAL) {
|
|
SDMA_DBG(req, "Invalid opcode (%d)", opcode);
|
|
ret = -EINVAL;
|
|
goto free_req;
|
|
}
|
|
/*
|
|
* Validate the vl. Do not trust packets from user space blindly.
|
|
* VL comes from PBC, SC comes from LRH, and the VL needs to
|
|
* match the SC look up.
|
|
*/
|
|
vl = (le16_to_cpu(req->hdr.pbc[0]) >> 12) & 0xF;
|
|
sc = (((be16_to_cpu(req->hdr.lrh[0]) >> 12) & 0xF) |
|
|
(((le16_to_cpu(req->hdr.pbc[1]) >> 14) & 0x1) << 4));
|
|
if (vl >= dd->pport->vls_operational ||
|
|
vl != sc_to_vlt(dd, sc)) {
|
|
SDMA_DBG(req, "Invalid SC(%u)/VL(%u)", sc, vl);
|
|
ret = -EINVAL;
|
|
goto free_req;
|
|
}
|
|
|
|
/* Checking P_KEY for requests from user-space */
|
|
pkey = (u16)be32_to_cpu(req->hdr.bth[0]);
|
|
slid = be16_to_cpu(req->hdr.lrh[3]);
|
|
if (egress_pkey_check(dd->pport, slid, pkey, sc, PKEY_CHECK_INVALID)) {
|
|
ret = -EINVAL;
|
|
goto free_req;
|
|
}
|
|
|
|
/*
|
|
* Also should check the BTH.lnh. If it says the next header is GRH then
|
|
* the RXE parsing will be off and will land in the middle of the KDETH
|
|
* or miss it entirely.
|
|
*/
|
|
if ((be16_to_cpu(req->hdr.lrh[0]) & 0x3) == HFI1_LRH_GRH) {
|
|
SDMA_DBG(req, "User tried to pass in a GRH");
|
|
ret = -EINVAL;
|
|
goto free_req;
|
|
}
|
|
|
|
req->koffset = le32_to_cpu(req->hdr.kdeth.swdata[6]);
|
|
/*
|
|
* Calculate the initial TID offset based on the values of
|
|
* KDETH.OFFSET and KDETH.OM that are passed in.
|
|
*/
|
|
req->tidoffset = KDETH_GET(req->hdr.kdeth.ver_tid_offset, OFFSET) *
|
|
(KDETH_GET(req->hdr.kdeth.ver_tid_offset, OM) ?
|
|
KDETH_OM_LARGE : KDETH_OM_SMALL);
|
|
trace_hfi1_sdma_user_initial_tidoffset(dd, uctxt->ctxt, fd->subctxt,
|
|
info.comp_idx, req->tidoffset);
|
|
idx++;
|
|
|
|
/* Save all the IO vector structures */
|
|
for (i = 0; i < req->data_iovs; i++) {
|
|
req->iovs[i].offset = 0;
|
|
INIT_LIST_HEAD(&req->iovs[i].list);
|
|
memcpy(&req->iovs[i].iov,
|
|
iovec + idx++,
|
|
sizeof(req->iovs[i].iov));
|
|
if (req->iovs[i].iov.iov_len == 0) {
|
|
ret = -EINVAL;
|
|
goto free_req;
|
|
}
|
|
req->data_len += req->iovs[i].iov.iov_len;
|
|
}
|
|
trace_hfi1_sdma_user_data_length(dd, uctxt->ctxt, fd->subctxt,
|
|
info.comp_idx, req->data_len);
|
|
if (pcount > req->info.npkts)
|
|
pcount = req->info.npkts;
|
|
/*
|
|
* Copy any TID info
|
|
* User space will provide the TID info only when the
|
|
* request type is EXPECTED. This is true even if there is
|
|
* only one packet in the request and the header is already
|
|
* setup. The reason for the singular TID case is that the
|
|
* driver needs to perform safety checks.
|
|
*/
|
|
if (req_opcode(req->info.ctrl) == EXPECTED) {
|
|
u16 ntids = iovec[idx].iov_len / sizeof(*req->tids);
|
|
u32 *tmp;
|
|
|
|
if (!ntids || ntids > MAX_TID_PAIR_ENTRIES) {
|
|
ret = -EINVAL;
|
|
goto free_req;
|
|
}
|
|
|
|
/*
|
|
* We have to copy all of the tids because they may vary
|
|
* in size and, therefore, the TID count might not be
|
|
* equal to the pkt count. However, there is no way to
|
|
* tell at this point.
|
|
*/
|
|
tmp = memdup_user(iovec[idx].iov_base,
|
|
ntids * sizeof(*req->tids));
|
|
if (IS_ERR(tmp)) {
|
|
ret = PTR_ERR(tmp);
|
|
SDMA_DBG(req, "Failed to copy %d TIDs (%d)",
|
|
ntids, ret);
|
|
goto free_req;
|
|
}
|
|
req->tids = tmp;
|
|
req->n_tids = ntids;
|
|
req->tididx = 0;
|
|
idx++;
|
|
}
|
|
|
|
dlid = be16_to_cpu(req->hdr.lrh[1]);
|
|
selector = dlid_to_selector(dlid);
|
|
selector += uctxt->ctxt + fd->subctxt;
|
|
req->sde = sdma_select_user_engine(dd, selector, vl);
|
|
|
|
if (!req->sde || !sdma_running(req->sde)) {
|
|
ret = -ECOMM;
|
|
goto free_req;
|
|
}
|
|
|
|
/* We don't need an AHG entry if the request contains only one packet */
|
|
if (req->info.npkts > 1 && HFI1_CAP_IS_USET(SDMA_AHG))
|
|
req->ahg_idx = sdma_ahg_alloc(req->sde);
|
|
|
|
set_comp_state(pq, cq, info.comp_idx, QUEUED, 0);
|
|
pq->state = SDMA_PKT_Q_ACTIVE;
|
|
|
|
/*
|
|
* This is a somewhat blocking send implementation.
|
|
* The driver will block the caller until all packets of the
|
|
* request have been submitted to the SDMA engine. However, it
|
|
* will not wait for send completions.
|
|
*/
|
|
while (req->seqsubmitted != req->info.npkts) {
|
|
ret = user_sdma_send_pkts(req, pcount);
|
|
if (ret < 0) {
|
|
int we_ret;
|
|
|
|
if (ret != -EBUSY)
|
|
goto free_req;
|
|
we_ret = wait_event_interruptible_timeout(
|
|
pq->busy.wait_dma,
|
|
pq->state == SDMA_PKT_Q_ACTIVE,
|
|
msecs_to_jiffies(
|
|
SDMA_IOWAIT_TIMEOUT));
|
|
trace_hfi1_usdma_we(pq, we_ret);
|
|
if (we_ret <= 0)
|
|
flush_pq_iowait(pq);
|
|
}
|
|
}
|
|
*count += idx;
|
|
return 0;
|
|
free_req:
|
|
/*
|
|
* If the submitted seqsubmitted == npkts, the completion routine
|
|
* controls the final state. If sequbmitted < npkts, wait for any
|
|
* outstanding packets to finish before cleaning up.
|
|
*/
|
|
if (req->seqsubmitted < req->info.npkts) {
|
|
if (req->seqsubmitted)
|
|
wait_event(pq->busy.wait_dma,
|
|
(req->seqcomp == req->seqsubmitted - 1));
|
|
user_sdma_free_request(req);
|
|
pq_update(pq);
|
|
set_comp_state(pq, cq, info.comp_idx, ERROR, ret);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
static inline u32 compute_data_length(struct user_sdma_request *req,
|
|
struct user_sdma_txreq *tx)
|
|
{
|
|
/*
|
|
* Determine the proper size of the packet data.
|
|
* The size of the data of the first packet is in the header
|
|
* template. However, it includes the header and ICRC, which need
|
|
* to be subtracted.
|
|
* The minimum representable packet data length in a header is 4 bytes,
|
|
* therefore, when the data length request is less than 4 bytes, there's
|
|
* only one packet, and the packet data length is equal to that of the
|
|
* request data length.
|
|
* The size of the remaining packets is the minimum of the frag
|
|
* size (MTU) or remaining data in the request.
|
|
*/
|
|
u32 len;
|
|
|
|
if (!req->seqnum) {
|
|
if (req->data_len < sizeof(u32))
|
|
len = req->data_len;
|
|
else
|
|
len = ((be16_to_cpu(req->hdr.lrh[2]) << 2) -
|
|
(sizeof(tx->hdr) - 4));
|
|
} else if (req_opcode(req->info.ctrl) == EXPECTED) {
|
|
u32 tidlen = EXP_TID_GET(req->tids[req->tididx], LEN) *
|
|
PAGE_SIZE;
|
|
/*
|
|
* Get the data length based on the remaining space in the
|
|
* TID pair.
|
|
*/
|
|
len = min(tidlen - req->tidoffset, (u32)req->info.fragsize);
|
|
/* If we've filled up the TID pair, move to the next one. */
|
|
if (unlikely(!len) && ++req->tididx < req->n_tids &&
|
|
req->tids[req->tididx]) {
|
|
tidlen = EXP_TID_GET(req->tids[req->tididx],
|
|
LEN) * PAGE_SIZE;
|
|
req->tidoffset = 0;
|
|
len = min_t(u32, tidlen, req->info.fragsize);
|
|
}
|
|
/*
|
|
* Since the TID pairs map entire pages, make sure that we
|
|
* are not going to try to send more data that we have
|
|
* remaining.
|
|
*/
|
|
len = min(len, req->data_len - req->sent);
|
|
} else {
|
|
len = min(req->data_len - req->sent, (u32)req->info.fragsize);
|
|
}
|
|
trace_hfi1_sdma_user_compute_length(req->pq->dd,
|
|
req->pq->ctxt,
|
|
req->pq->subctxt,
|
|
req->info.comp_idx,
|
|
len);
|
|
return len;
|
|
}
|
|
|
|
static inline u32 pad_len(u32 len)
|
|
{
|
|
if (len & (sizeof(u32) - 1))
|
|
len += sizeof(u32) - (len & (sizeof(u32) - 1));
|
|
return len;
|
|
}
|
|
|
|
static inline u32 get_lrh_len(struct hfi1_pkt_header hdr, u32 len)
|
|
{
|
|
/* (Size of complete header - size of PBC) + 4B ICRC + data length */
|
|
return ((sizeof(hdr) - sizeof(hdr.pbc)) + 4 + len);
|
|
}
|
|
|
|
static int user_sdma_txadd_ahg(struct user_sdma_request *req,
|
|
struct user_sdma_txreq *tx,
|
|
u32 datalen)
|
|
{
|
|
int ret;
|
|
u16 pbclen = le16_to_cpu(req->hdr.pbc[0]);
|
|
u32 lrhlen = get_lrh_len(req->hdr, pad_len(datalen));
|
|
struct hfi1_user_sdma_pkt_q *pq = req->pq;
|
|
|
|
/*
|
|
* Copy the request header into the tx header
|
|
* because the HW needs a cacheline-aligned
|
|
* address.
|
|
* This copy can be optimized out if the hdr
|
|
* member of user_sdma_request were also
|
|
* cacheline aligned.
|
|
*/
|
|
memcpy(&tx->hdr, &req->hdr, sizeof(tx->hdr));
|
|
if (PBC2LRH(pbclen) != lrhlen) {
|
|
pbclen = (pbclen & 0xf000) | LRH2PBC(lrhlen);
|
|
tx->hdr.pbc[0] = cpu_to_le16(pbclen);
|
|
}
|
|
ret = check_header_template(req, &tx->hdr, lrhlen, datalen);
|
|
if (ret)
|
|
return ret;
|
|
ret = sdma_txinit_ahg(&tx->txreq, SDMA_TXREQ_F_AHG_COPY,
|
|
sizeof(tx->hdr) + datalen, req->ahg_idx,
|
|
0, NULL, 0, user_sdma_txreq_cb);
|
|
if (ret)
|
|
return ret;
|
|
ret = sdma_txadd_kvaddr(pq->dd, &tx->txreq, &tx->hdr, sizeof(tx->hdr));
|
|
if (ret)
|
|
sdma_txclean(pq->dd, &tx->txreq);
|
|
return ret;
|
|
}
|
|
|
|
static int user_sdma_send_pkts(struct user_sdma_request *req, u16 maxpkts)
|
|
{
|
|
int ret = 0;
|
|
u16 count;
|
|
unsigned npkts = 0;
|
|
struct user_sdma_txreq *tx = NULL;
|
|
struct hfi1_user_sdma_pkt_q *pq = NULL;
|
|
struct user_sdma_iovec *iovec = NULL;
|
|
|
|
if (!req->pq)
|
|
return -EINVAL;
|
|
|
|
pq = req->pq;
|
|
|
|
/* If tx completion has reported an error, we are done. */
|
|
if (READ_ONCE(req->has_error))
|
|
return -EFAULT;
|
|
|
|
/*
|
|
* Check if we might have sent the entire request already
|
|
*/
|
|
if (unlikely(req->seqnum == req->info.npkts)) {
|
|
if (!list_empty(&req->txps))
|
|
goto dosend;
|
|
return ret;
|
|
}
|
|
|
|
if (!maxpkts || maxpkts > req->info.npkts - req->seqnum)
|
|
maxpkts = req->info.npkts - req->seqnum;
|
|
|
|
while (npkts < maxpkts) {
|
|
u32 datalen = 0;
|
|
|
|
/*
|
|
* Check whether any of the completions have come back
|
|
* with errors. If so, we are not going to process any
|
|
* more packets from this request.
|
|
*/
|
|
if (READ_ONCE(req->has_error))
|
|
return -EFAULT;
|
|
|
|
tx = kmem_cache_alloc(pq->txreq_cache, GFP_KERNEL);
|
|
if (!tx)
|
|
return -ENOMEM;
|
|
|
|
tx->flags = 0;
|
|
tx->req = req;
|
|
INIT_LIST_HEAD(&tx->list);
|
|
|
|
/*
|
|
* For the last packet set the ACK request
|
|
* and disable header suppression.
|
|
*/
|
|
if (req->seqnum == req->info.npkts - 1)
|
|
tx->flags |= (TXREQ_FLAGS_REQ_ACK |
|
|
TXREQ_FLAGS_REQ_DISABLE_SH);
|
|
|
|
/*
|
|
* Calculate the payload size - this is min of the fragment
|
|
* (MTU) size or the remaining bytes in the request but only
|
|
* if we have payload data.
|
|
*/
|
|
if (req->data_len) {
|
|
iovec = &req->iovs[req->iov_idx];
|
|
if (READ_ONCE(iovec->offset) == iovec->iov.iov_len) {
|
|
if (++req->iov_idx == req->data_iovs) {
|
|
ret = -EFAULT;
|
|
goto free_tx;
|
|
}
|
|
iovec = &req->iovs[req->iov_idx];
|
|
WARN_ON(iovec->offset);
|
|
}
|
|
|
|
datalen = compute_data_length(req, tx);
|
|
|
|
/*
|
|
* Disable header suppression for the payload <= 8DWS.
|
|
* If there is an uncorrectable error in the receive
|
|
* data FIFO when the received payload size is less than
|
|
* or equal to 8DWS then the RxDmaDataFifoRdUncErr is
|
|
* not reported.There is set RHF.EccErr if the header
|
|
* is not suppressed.
|
|
*/
|
|
if (!datalen) {
|
|
SDMA_DBG(req,
|
|
"Request has data but pkt len is 0");
|
|
ret = -EFAULT;
|
|
goto free_tx;
|
|
} else if (datalen <= 32) {
|
|
tx->flags |= TXREQ_FLAGS_REQ_DISABLE_SH;
|
|
}
|
|
}
|
|
|
|
if (req->ahg_idx >= 0) {
|
|
if (!req->seqnum) {
|
|
ret = user_sdma_txadd_ahg(req, tx, datalen);
|
|
if (ret)
|
|
goto free_tx;
|
|
} else {
|
|
int changes;
|
|
|
|
changes = set_txreq_header_ahg(req, tx,
|
|
datalen);
|
|
if (changes < 0) {
|
|
ret = changes;
|
|
goto free_tx;
|
|
}
|
|
}
|
|
} else {
|
|
ret = sdma_txinit(&tx->txreq, 0, sizeof(req->hdr) +
|
|
datalen, user_sdma_txreq_cb);
|
|
if (ret)
|
|
goto free_tx;
|
|
/*
|
|
* Modify the header for this packet. This only needs
|
|
* to be done if we are not going to use AHG. Otherwise,
|
|
* the HW will do it based on the changes we gave it
|
|
* during sdma_txinit_ahg().
|
|
*/
|
|
ret = set_txreq_header(req, tx, datalen);
|
|
if (ret)
|
|
goto free_txreq;
|
|
}
|
|
|
|
req->koffset += datalen;
|
|
if (req_opcode(req->info.ctrl) == EXPECTED)
|
|
req->tidoffset += datalen;
|
|
req->sent += datalen;
|
|
while (datalen) {
|
|
ret = add_system_pages_to_sdma_packet(req, tx, iovec,
|
|
&datalen);
|
|
if (ret)
|
|
goto free_txreq;
|
|
iovec = &req->iovs[req->iov_idx];
|
|
}
|
|
list_add_tail(&tx->txreq.list, &req->txps);
|
|
/*
|
|
* It is important to increment this here as it is used to
|
|
* generate the BTH.PSN and, therefore, can't be bulk-updated
|
|
* outside of the loop.
|
|
*/
|
|
tx->seqnum = req->seqnum++;
|
|
npkts++;
|
|
}
|
|
dosend:
|
|
ret = sdma_send_txlist(req->sde,
|
|
iowait_get_ib_work(&pq->busy),
|
|
&req->txps, &count);
|
|
req->seqsubmitted += count;
|
|
if (req->seqsubmitted == req->info.npkts) {
|
|
/*
|
|
* The txreq has already been submitted to the HW queue
|
|
* so we can free the AHG entry now. Corruption will not
|
|
* happen due to the sequential manner in which
|
|
* descriptors are processed.
|
|
*/
|
|
if (req->ahg_idx >= 0)
|
|
sdma_ahg_free(req->sde, req->ahg_idx);
|
|
}
|
|
return ret;
|
|
|
|
free_txreq:
|
|
sdma_txclean(pq->dd, &tx->txreq);
|
|
free_tx:
|
|
kmem_cache_free(pq->txreq_cache, tx);
|
|
return ret;
|
|
}
|
|
|
|
static u32 sdma_cache_evict(struct hfi1_user_sdma_pkt_q *pq, u32 npages)
|
|
{
|
|
struct evict_data evict_data;
|
|
struct mmu_rb_handler *handler = pq->handler;
|
|
|
|
evict_data.cleared = 0;
|
|
evict_data.target = npages;
|
|
hfi1_mmu_rb_evict(handler, &evict_data);
|
|
return evict_data.cleared;
|
|
}
|
|
|
|
static int check_header_template(struct user_sdma_request *req,
|
|
struct hfi1_pkt_header *hdr, u32 lrhlen,
|
|
u32 datalen)
|
|
{
|
|
/*
|
|
* Perform safety checks for any type of packet:
|
|
* - transfer size is multiple of 64bytes
|
|
* - packet length is multiple of 4 bytes
|
|
* - packet length is not larger than MTU size
|
|
*
|
|
* These checks are only done for the first packet of the
|
|
* transfer since the header is "given" to us by user space.
|
|
* For the remainder of the packets we compute the values.
|
|
*/
|
|
if (req->info.fragsize % PIO_BLOCK_SIZE || lrhlen & 0x3 ||
|
|
lrhlen > get_lrh_len(*hdr, req->info.fragsize))
|
|
return -EINVAL;
|
|
|
|
if (req_opcode(req->info.ctrl) == EXPECTED) {
|
|
/*
|
|
* The header is checked only on the first packet. Furthermore,
|
|
* we ensure that at least one TID entry is copied when the
|
|
* request is submitted. Therefore, we don't have to verify that
|
|
* tididx points to something sane.
|
|
*/
|
|
u32 tidval = req->tids[req->tididx],
|
|
tidlen = EXP_TID_GET(tidval, LEN) * PAGE_SIZE,
|
|
tididx = EXP_TID_GET(tidval, IDX),
|
|
tidctrl = EXP_TID_GET(tidval, CTRL),
|
|
tidoff;
|
|
__le32 kval = hdr->kdeth.ver_tid_offset;
|
|
|
|
tidoff = KDETH_GET(kval, OFFSET) *
|
|
(KDETH_GET(req->hdr.kdeth.ver_tid_offset, OM) ?
|
|
KDETH_OM_LARGE : KDETH_OM_SMALL);
|
|
/*
|
|
* Expected receive packets have the following
|
|
* additional checks:
|
|
* - offset is not larger than the TID size
|
|
* - TIDCtrl values match between header and TID array
|
|
* - TID indexes match between header and TID array
|
|
*/
|
|
if ((tidoff + datalen > tidlen) ||
|
|
KDETH_GET(kval, TIDCTRL) != tidctrl ||
|
|
KDETH_GET(kval, TID) != tididx)
|
|
return -EINVAL;
|
|
}
|
|
return 0;
|
|
}
|
|
|
|
/*
|
|
* Correctly set the BTH.PSN field based on type of
|
|
* transfer - eager packets can just increment the PSN but
|
|
* expected packets encode generation and sequence in the
|
|
* BTH.PSN field so just incrementing will result in errors.
|
|
*/
|
|
static inline u32 set_pkt_bth_psn(__be32 bthpsn, u8 expct, u32 frags)
|
|
{
|
|
u32 val = be32_to_cpu(bthpsn),
|
|
mask = (HFI1_CAP_IS_KSET(EXTENDED_PSN) ? 0x7fffffffull :
|
|
0xffffffull),
|
|
psn = val & mask;
|
|
if (expct)
|
|
psn = (psn & ~HFI1_KDETH_BTH_SEQ_MASK) |
|
|
((psn + frags) & HFI1_KDETH_BTH_SEQ_MASK);
|
|
else
|
|
psn = psn + frags;
|
|
return psn & mask;
|
|
}
|
|
|
|
static int set_txreq_header(struct user_sdma_request *req,
|
|
struct user_sdma_txreq *tx, u32 datalen)
|
|
{
|
|
struct hfi1_user_sdma_pkt_q *pq = req->pq;
|
|
struct hfi1_pkt_header *hdr = &tx->hdr;
|
|
u8 omfactor; /* KDETH.OM */
|
|
u16 pbclen;
|
|
int ret;
|
|
u32 tidval = 0, lrhlen = get_lrh_len(*hdr, pad_len(datalen));
|
|
|
|
/* Copy the header template to the request before modification */
|
|
memcpy(hdr, &req->hdr, sizeof(*hdr));
|
|
|
|
/*
|
|
* Check if the PBC and LRH length are mismatched. If so
|
|
* adjust both in the header.
|
|
*/
|
|
pbclen = le16_to_cpu(hdr->pbc[0]);
|
|
if (PBC2LRH(pbclen) != lrhlen) {
|
|
pbclen = (pbclen & 0xf000) | LRH2PBC(lrhlen);
|
|
hdr->pbc[0] = cpu_to_le16(pbclen);
|
|
hdr->lrh[2] = cpu_to_be16(lrhlen >> 2);
|
|
/*
|
|
* Third packet
|
|
* This is the first packet in the sequence that has
|
|
* a "static" size that can be used for the rest of
|
|
* the packets (besides the last one).
|
|
*/
|
|
if (unlikely(req->seqnum == 2)) {
|
|
/*
|
|
* From this point on the lengths in both the
|
|
* PBC and LRH are the same until the last
|
|
* packet.
|
|
* Adjust the template so we don't have to update
|
|
* every packet
|
|
*/
|
|
req->hdr.pbc[0] = hdr->pbc[0];
|
|
req->hdr.lrh[2] = hdr->lrh[2];
|
|
}
|
|
}
|
|
/*
|
|
* We only have to modify the header if this is not the
|
|
* first packet in the request. Otherwise, we use the
|
|
* header given to us.
|
|
*/
|
|
if (unlikely(!req->seqnum)) {
|
|
ret = check_header_template(req, hdr, lrhlen, datalen);
|
|
if (ret)
|
|
return ret;
|
|
goto done;
|
|
}
|
|
|
|
hdr->bth[2] = cpu_to_be32(
|
|
set_pkt_bth_psn(hdr->bth[2],
|
|
(req_opcode(req->info.ctrl) == EXPECTED),
|
|
req->seqnum));
|
|
|
|
/* Set ACK request on last packet */
|
|
if (unlikely(tx->flags & TXREQ_FLAGS_REQ_ACK))
|
|
hdr->bth[2] |= cpu_to_be32(1UL << 31);
|
|
|
|
/* Set the new offset */
|
|
hdr->kdeth.swdata[6] = cpu_to_le32(req->koffset);
|
|
/* Expected packets have to fill in the new TID information */
|
|
if (req_opcode(req->info.ctrl) == EXPECTED) {
|
|
tidval = req->tids[req->tididx];
|
|
/*
|
|
* If the offset puts us at the end of the current TID,
|
|
* advance everything.
|
|
*/
|
|
if ((req->tidoffset) == (EXP_TID_GET(tidval, LEN) *
|
|
PAGE_SIZE)) {
|
|
req->tidoffset = 0;
|
|
/*
|
|
* Since we don't copy all the TIDs, all at once,
|
|
* we have to check again.
|
|
*/
|
|
if (++req->tididx > req->n_tids - 1 ||
|
|
!req->tids[req->tididx]) {
|
|
return -EINVAL;
|
|
}
|
|
tidval = req->tids[req->tididx];
|
|
}
|
|
omfactor = EXP_TID_GET(tidval, LEN) * PAGE_SIZE >=
|
|
KDETH_OM_MAX_SIZE ? KDETH_OM_LARGE_SHIFT :
|
|
KDETH_OM_SMALL_SHIFT;
|
|
/* Set KDETH.TIDCtrl based on value for this TID. */
|
|
KDETH_SET(hdr->kdeth.ver_tid_offset, TIDCTRL,
|
|
EXP_TID_GET(tidval, CTRL));
|
|
/* Set KDETH.TID based on value for this TID */
|
|
KDETH_SET(hdr->kdeth.ver_tid_offset, TID,
|
|
EXP_TID_GET(tidval, IDX));
|
|
/* Clear KDETH.SH when DISABLE_SH flag is set */
|
|
if (unlikely(tx->flags & TXREQ_FLAGS_REQ_DISABLE_SH))
|
|
KDETH_SET(hdr->kdeth.ver_tid_offset, SH, 0);
|
|
/*
|
|
* Set the KDETH.OFFSET and KDETH.OM based on size of
|
|
* transfer.
|
|
*/
|
|
trace_hfi1_sdma_user_tid_info(
|
|
pq->dd, pq->ctxt, pq->subctxt, req->info.comp_idx,
|
|
req->tidoffset, req->tidoffset >> omfactor,
|
|
omfactor != KDETH_OM_SMALL_SHIFT);
|
|
KDETH_SET(hdr->kdeth.ver_tid_offset, OFFSET,
|
|
req->tidoffset >> omfactor);
|
|
KDETH_SET(hdr->kdeth.ver_tid_offset, OM,
|
|
omfactor != KDETH_OM_SMALL_SHIFT);
|
|
}
|
|
done:
|
|
trace_hfi1_sdma_user_header(pq->dd, pq->ctxt, pq->subctxt,
|
|
req->info.comp_idx, hdr, tidval);
|
|
return sdma_txadd_kvaddr(pq->dd, &tx->txreq, hdr, sizeof(*hdr));
|
|
}
|
|
|
|
static int set_txreq_header_ahg(struct user_sdma_request *req,
|
|
struct user_sdma_txreq *tx, u32 datalen)
|
|
{
|
|
u32 ahg[AHG_KDETH_ARRAY_SIZE];
|
|
int idx = 0;
|
|
u8 omfactor; /* KDETH.OM */
|
|
struct hfi1_user_sdma_pkt_q *pq = req->pq;
|
|
struct hfi1_pkt_header *hdr = &req->hdr;
|
|
u16 pbclen = le16_to_cpu(hdr->pbc[0]);
|
|
u32 val32, tidval = 0, lrhlen = get_lrh_len(*hdr, pad_len(datalen));
|
|
size_t array_size = ARRAY_SIZE(ahg);
|
|
|
|
if (PBC2LRH(pbclen) != lrhlen) {
|
|
/* PBC.PbcLengthDWs */
|
|
idx = ahg_header_set(ahg, idx, array_size, 0, 0, 12,
|
|
(__force u16)cpu_to_le16(LRH2PBC(lrhlen)));
|
|
if (idx < 0)
|
|
return idx;
|
|
/* LRH.PktLen (we need the full 16 bits due to byte swap) */
|
|
idx = ahg_header_set(ahg, idx, array_size, 3, 0, 16,
|
|
(__force u16)cpu_to_be16(lrhlen >> 2));
|
|
if (idx < 0)
|
|
return idx;
|
|
}
|
|
|
|
/*
|
|
* Do the common updates
|
|
*/
|
|
/* BTH.PSN and BTH.A */
|
|
val32 = (be32_to_cpu(hdr->bth[2]) + req->seqnum) &
|
|
(HFI1_CAP_IS_KSET(EXTENDED_PSN) ? 0x7fffffff : 0xffffff);
|
|
if (unlikely(tx->flags & TXREQ_FLAGS_REQ_ACK))
|
|
val32 |= 1UL << 31;
|
|
idx = ahg_header_set(ahg, idx, array_size, 6, 0, 16,
|
|
(__force u16)cpu_to_be16(val32 >> 16));
|
|
if (idx < 0)
|
|
return idx;
|
|
idx = ahg_header_set(ahg, idx, array_size, 6, 16, 16,
|
|
(__force u16)cpu_to_be16(val32 & 0xffff));
|
|
if (idx < 0)
|
|
return idx;
|
|
/* KDETH.Offset */
|
|
idx = ahg_header_set(ahg, idx, array_size, 15, 0, 16,
|
|
(__force u16)cpu_to_le16(req->koffset & 0xffff));
|
|
if (idx < 0)
|
|
return idx;
|
|
idx = ahg_header_set(ahg, idx, array_size, 15, 16, 16,
|
|
(__force u16)cpu_to_le16(req->koffset >> 16));
|
|
if (idx < 0)
|
|
return idx;
|
|
if (req_opcode(req->info.ctrl) == EXPECTED) {
|
|
__le16 val;
|
|
|
|
tidval = req->tids[req->tididx];
|
|
|
|
/*
|
|
* If the offset puts us at the end of the current TID,
|
|
* advance everything.
|
|
*/
|
|
if ((req->tidoffset) == (EXP_TID_GET(tidval, LEN) *
|
|
PAGE_SIZE)) {
|
|
req->tidoffset = 0;
|
|
/*
|
|
* Since we don't copy all the TIDs, all at once,
|
|
* we have to check again.
|
|
*/
|
|
if (++req->tididx > req->n_tids - 1 ||
|
|
!req->tids[req->tididx])
|
|
return -EINVAL;
|
|
tidval = req->tids[req->tididx];
|
|
}
|
|
omfactor = ((EXP_TID_GET(tidval, LEN) *
|
|
PAGE_SIZE) >=
|
|
KDETH_OM_MAX_SIZE) ? KDETH_OM_LARGE_SHIFT :
|
|
KDETH_OM_SMALL_SHIFT;
|
|
/* KDETH.OM and KDETH.OFFSET (TID) */
|
|
idx = ahg_header_set(
|
|
ahg, idx, array_size, 7, 0, 16,
|
|
((!!(omfactor - KDETH_OM_SMALL_SHIFT)) << 15 |
|
|
((req->tidoffset >> omfactor)
|
|
& 0x7fff)));
|
|
if (idx < 0)
|
|
return idx;
|
|
/* KDETH.TIDCtrl, KDETH.TID, KDETH.Intr, KDETH.SH */
|
|
val = cpu_to_le16(((EXP_TID_GET(tidval, CTRL) & 0x3) << 10) |
|
|
(EXP_TID_GET(tidval, IDX) & 0x3ff));
|
|
|
|
if (unlikely(tx->flags & TXREQ_FLAGS_REQ_DISABLE_SH)) {
|
|
val |= cpu_to_le16((KDETH_GET(hdr->kdeth.ver_tid_offset,
|
|
INTR) <<
|
|
AHG_KDETH_INTR_SHIFT));
|
|
} else {
|
|
val |= KDETH_GET(hdr->kdeth.ver_tid_offset, SH) ?
|
|
cpu_to_le16(0x1 << AHG_KDETH_SH_SHIFT) :
|
|
cpu_to_le16((KDETH_GET(hdr->kdeth.ver_tid_offset,
|
|
INTR) <<
|
|
AHG_KDETH_INTR_SHIFT));
|
|
}
|
|
|
|
idx = ahg_header_set(ahg, idx, array_size,
|
|
7, 16, 14, (__force u16)val);
|
|
if (idx < 0)
|
|
return idx;
|
|
}
|
|
|
|
trace_hfi1_sdma_user_header_ahg(pq->dd, pq->ctxt, pq->subctxt,
|
|
req->info.comp_idx, req->sde->this_idx,
|
|
req->ahg_idx, ahg, idx, tidval);
|
|
sdma_txinit_ahg(&tx->txreq,
|
|
SDMA_TXREQ_F_USE_AHG,
|
|
datalen, req->ahg_idx, idx,
|
|
ahg, sizeof(req->hdr),
|
|
user_sdma_txreq_cb);
|
|
|
|
return idx;
|
|
}
|
|
|
|
/**
|
|
* user_sdma_txreq_cb() - SDMA tx request completion callback.
|
|
* @txreq: valid sdma tx request
|
|
* @status: success/failure of request
|
|
*
|
|
* Called when the SDMA progress state machine gets notification that
|
|
* the SDMA descriptors for this tx request have been processed by the
|
|
* DMA engine. Called in interrupt context.
|
|
* Only do work on completed sequences.
|
|
*/
|
|
static void user_sdma_txreq_cb(struct sdma_txreq *txreq, int status)
|
|
{
|
|
struct user_sdma_txreq *tx =
|
|
container_of(txreq, struct user_sdma_txreq, txreq);
|
|
struct user_sdma_request *req;
|
|
struct hfi1_user_sdma_pkt_q *pq;
|
|
struct hfi1_user_sdma_comp_q *cq;
|
|
enum hfi1_sdma_comp_state state = COMPLETE;
|
|
|
|
if (!tx->req)
|
|
return;
|
|
|
|
req = tx->req;
|
|
pq = req->pq;
|
|
cq = req->cq;
|
|
|
|
if (status != SDMA_TXREQ_S_OK) {
|
|
SDMA_DBG(req, "SDMA completion with error %d",
|
|
status);
|
|
WRITE_ONCE(req->has_error, 1);
|
|
state = ERROR;
|
|
}
|
|
|
|
req->seqcomp = tx->seqnum;
|
|
kmem_cache_free(pq->txreq_cache, tx);
|
|
|
|
/* sequence isn't complete? We are done */
|
|
if (req->seqcomp != req->info.npkts - 1)
|
|
return;
|
|
|
|
user_sdma_free_request(req);
|
|
set_comp_state(pq, cq, req->info.comp_idx, state, status);
|
|
pq_update(pq);
|
|
}
|
|
|
|
static inline void pq_update(struct hfi1_user_sdma_pkt_q *pq)
|
|
{
|
|
if (atomic_dec_and_test(&pq->n_reqs))
|
|
wake_up(&pq->wait);
|
|
}
|
|
|
|
static void user_sdma_free_request(struct user_sdma_request *req)
|
|
{
|
|
if (!list_empty(&req->txps)) {
|
|
struct sdma_txreq *t, *p;
|
|
|
|
list_for_each_entry_safe(t, p, &req->txps, list) {
|
|
struct user_sdma_txreq *tx =
|
|
container_of(t, struct user_sdma_txreq, txreq);
|
|
list_del_init(&t->list);
|
|
sdma_txclean(req->pq->dd, t);
|
|
kmem_cache_free(req->pq->txreq_cache, tx);
|
|
}
|
|
}
|
|
|
|
kfree(req->tids);
|
|
clear_bit(req->info.comp_idx, req->pq->req_in_use);
|
|
}
|
|
|
|
static inline void set_comp_state(struct hfi1_user_sdma_pkt_q *pq,
|
|
struct hfi1_user_sdma_comp_q *cq,
|
|
u16 idx, enum hfi1_sdma_comp_state state,
|
|
int ret)
|
|
{
|
|
if (state == ERROR)
|
|
cq->comps[idx].errcode = -ret;
|
|
smp_wmb(); /* make sure errcode is visible first */
|
|
cq->comps[idx].status = state;
|
|
trace_hfi1_sdma_user_completion(pq->dd, pq->ctxt, pq->subctxt,
|
|
idx, state, ret);
|
|
}
|
|
|
|
static void unpin_vector_pages(struct mm_struct *mm, struct page **pages,
|
|
unsigned int start, unsigned int npages)
|
|
{
|
|
hfi1_release_user_pages(mm, pages + start, npages, false);
|
|
kfree(pages);
|
|
}
|
|
|
|
static void free_system_node(struct sdma_mmu_node *node)
|
|
{
|
|
if (node->npages) {
|
|
unpin_vector_pages(mm_from_sdma_node(node), node->pages, 0,
|
|
node->npages);
|
|
atomic_sub(node->npages, &node->pq->n_locked);
|
|
}
|
|
kfree(node);
|
|
}
|
|
|
|
/*
|
|
* kref_get()'s an additional kref on the returned rb_node to prevent rb_node
|
|
* from being released until after rb_node is assigned to an SDMA descriptor
|
|
* (struct sdma_desc) under add_system_iovec_to_sdma_packet(), even if the
|
|
* virtual address range for rb_node is invalidated between now and then.
|
|
*/
|
|
static struct sdma_mmu_node *find_system_node(struct mmu_rb_handler *handler,
|
|
unsigned long start,
|
|
unsigned long end)
|
|
{
|
|
struct mmu_rb_node *rb_node;
|
|
unsigned long flags;
|
|
|
|
spin_lock_irqsave(&handler->lock, flags);
|
|
rb_node = hfi1_mmu_rb_get_first(handler, start, (end - start));
|
|
if (!rb_node) {
|
|
spin_unlock_irqrestore(&handler->lock, flags);
|
|
return NULL;
|
|
}
|
|
|
|
/* "safety" kref to prevent release before add_system_iovec_to_sdma_packet() */
|
|
kref_get(&rb_node->refcount);
|
|
spin_unlock_irqrestore(&handler->lock, flags);
|
|
|
|
return container_of(rb_node, struct sdma_mmu_node, rb);
|
|
}
|
|
|
|
static int pin_system_pages(struct user_sdma_request *req,
|
|
uintptr_t start_address, size_t length,
|
|
struct sdma_mmu_node *node, int npages)
|
|
{
|
|
struct hfi1_user_sdma_pkt_q *pq = req->pq;
|
|
int pinned, cleared;
|
|
struct page **pages;
|
|
|
|
pages = kcalloc(npages, sizeof(*pages), GFP_KERNEL);
|
|
if (!pages)
|
|
return -ENOMEM;
|
|
|
|
retry:
|
|
if (!hfi1_can_pin_pages(pq->dd, current->mm, atomic_read(&pq->n_locked),
|
|
npages)) {
|
|
SDMA_DBG(req, "Evicting: nlocked %u npages %u",
|
|
atomic_read(&pq->n_locked), npages);
|
|
cleared = sdma_cache_evict(pq, npages);
|
|
if (cleared >= npages)
|
|
goto retry;
|
|
}
|
|
|
|
SDMA_DBG(req, "Acquire user pages start_address %lx node->npages %u npages %u",
|
|
start_address, node->npages, npages);
|
|
pinned = hfi1_acquire_user_pages(current->mm, start_address, npages, 0,
|
|
pages);
|
|
|
|
if (pinned < 0) {
|
|
kfree(pages);
|
|
SDMA_DBG(req, "pinned %d", pinned);
|
|
return pinned;
|
|
}
|
|
if (pinned != npages) {
|
|
unpin_vector_pages(current->mm, pages, node->npages, pinned);
|
|
SDMA_DBG(req, "npages %u pinned %d", npages, pinned);
|
|
return -EFAULT;
|
|
}
|
|
node->rb.addr = start_address;
|
|
node->rb.len = length;
|
|
node->pages = pages;
|
|
node->npages = npages;
|
|
atomic_add(pinned, &pq->n_locked);
|
|
SDMA_DBG(req, "done. pinned %d", pinned);
|
|
return 0;
|
|
}
|
|
|
|
/*
|
|
* kref refcount on *node_p will be 2 on successful addition: one kref from
|
|
* kref_init() for mmu_rb_handler and one kref to prevent *node_p from being
|
|
* released until after *node_p is assigned to an SDMA descriptor (struct
|
|
* sdma_desc) under add_system_iovec_to_sdma_packet(), even if the virtual
|
|
* address range for *node_p is invalidated between now and then.
|
|
*/
|
|
static int add_system_pinning(struct user_sdma_request *req,
|
|
struct sdma_mmu_node **node_p,
|
|
unsigned long start, unsigned long len)
|
|
|
|
{
|
|
struct hfi1_user_sdma_pkt_q *pq = req->pq;
|
|
struct sdma_mmu_node *node;
|
|
int ret;
|
|
|
|
node = kzalloc(sizeof(*node), GFP_KERNEL);
|
|
if (!node)
|
|
return -ENOMEM;
|
|
|
|
/* First kref "moves" to mmu_rb_handler */
|
|
kref_init(&node->rb.refcount);
|
|
|
|
/* "safety" kref to prevent release before add_system_iovec_to_sdma_packet() */
|
|
kref_get(&node->rb.refcount);
|
|
|
|
node->pq = pq;
|
|
ret = pin_system_pages(req, start, len, node, PFN_DOWN(len));
|
|
if (ret == 0) {
|
|
ret = hfi1_mmu_rb_insert(pq->handler, &node->rb);
|
|
if (ret)
|
|
free_system_node(node);
|
|
else
|
|
*node_p = node;
|
|
|
|
return ret;
|
|
}
|
|
|
|
kfree(node);
|
|
return ret;
|
|
}
|
|
|
|
static int get_system_cache_entry(struct user_sdma_request *req,
|
|
struct sdma_mmu_node **node_p,
|
|
size_t req_start, size_t req_len)
|
|
{
|
|
struct hfi1_user_sdma_pkt_q *pq = req->pq;
|
|
u64 start = ALIGN_DOWN(req_start, PAGE_SIZE);
|
|
u64 end = PFN_ALIGN(req_start + req_len);
|
|
struct mmu_rb_handler *handler = pq->handler;
|
|
int ret;
|
|
|
|
if ((end - start) == 0) {
|
|
SDMA_DBG(req,
|
|
"Request for empty cache entry req_start %lx req_len %lx start %llx end %llx",
|
|
req_start, req_len, start, end);
|
|
return -EINVAL;
|
|
}
|
|
|
|
SDMA_DBG(req, "req_start %lx req_len %lu", req_start, req_len);
|
|
|
|
while (1) {
|
|
struct sdma_mmu_node *node =
|
|
find_system_node(handler, start, end);
|
|
u64 prepend_len = 0;
|
|
|
|
SDMA_DBG(req, "node %p start %llx end %llu", node, start, end);
|
|
if (!node) {
|
|
ret = add_system_pinning(req, node_p, start,
|
|
end - start);
|
|
if (ret == -EEXIST) {
|
|
/*
|
|
* Another execution context has inserted a
|
|
* conficting entry first.
|
|
*/
|
|
continue;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
if (node->rb.addr <= start) {
|
|
/*
|
|
* This entry covers at least part of the region. If it doesn't extend
|
|
* to the end, then this will be called again for the next segment.
|
|
*/
|
|
*node_p = node;
|
|
return 0;
|
|
}
|
|
|
|
SDMA_DBG(req, "prepend: node->rb.addr %lx, node->rb.refcount %d",
|
|
node->rb.addr, kref_read(&node->rb.refcount));
|
|
prepend_len = node->rb.addr - start;
|
|
|
|
/*
|
|
* This node will not be returned, instead a new node
|
|
* will be. So release the reference.
|
|
*/
|
|
kref_put(&node->rb.refcount, hfi1_mmu_rb_release);
|
|
|
|
/* Prepend a node to cover the beginning of the allocation */
|
|
ret = add_system_pinning(req, node_p, start, prepend_len);
|
|
if (ret == -EEXIST) {
|
|
/* Another execution context has inserted a conficting entry first. */
|
|
continue;
|
|
}
|
|
return ret;
|
|
}
|
|
}
|
|
|
|
static void sdma_mmu_rb_node_get(void *ctx)
|
|
{
|
|
struct mmu_rb_node *node = ctx;
|
|
|
|
kref_get(&node->refcount);
|
|
}
|
|
|
|
static void sdma_mmu_rb_node_put(void *ctx)
|
|
{
|
|
struct sdma_mmu_node *node = ctx;
|
|
|
|
kref_put(&node->rb.refcount, hfi1_mmu_rb_release);
|
|
}
|
|
|
|
static int add_mapping_to_sdma_packet(struct user_sdma_request *req,
|
|
struct user_sdma_txreq *tx,
|
|
struct sdma_mmu_node *cache_entry,
|
|
size_t start,
|
|
size_t from_this_cache_entry)
|
|
{
|
|
struct hfi1_user_sdma_pkt_q *pq = req->pq;
|
|
unsigned int page_offset;
|
|
unsigned int from_this_page;
|
|
size_t page_index;
|
|
void *ctx;
|
|
int ret;
|
|
|
|
/*
|
|
* Because the cache may be more fragmented than the memory that is being accessed,
|
|
* it's not strictly necessary to have a descriptor per cache entry.
|
|
*/
|
|
|
|
while (from_this_cache_entry) {
|
|
page_index = PFN_DOWN(start - cache_entry->rb.addr);
|
|
|
|
if (page_index >= cache_entry->npages) {
|
|
SDMA_DBG(req,
|
|
"Request for page_index %zu >= cache_entry->npages %u",
|
|
page_index, cache_entry->npages);
|
|
return -EINVAL;
|
|
}
|
|
|
|
page_offset = start - ALIGN_DOWN(start, PAGE_SIZE);
|
|
from_this_page = PAGE_SIZE - page_offset;
|
|
|
|
if (from_this_page < from_this_cache_entry) {
|
|
ctx = NULL;
|
|
} else {
|
|
/*
|
|
* In the case they are equal the next line has no practical effect,
|
|
* but it's better to do a register to register copy than a conditional
|
|
* branch.
|
|
*/
|
|
from_this_page = from_this_cache_entry;
|
|
ctx = cache_entry;
|
|
}
|
|
|
|
ret = sdma_txadd_page(pq->dd, &tx->txreq,
|
|
cache_entry->pages[page_index],
|
|
page_offset, from_this_page,
|
|
ctx,
|
|
sdma_mmu_rb_node_get,
|
|
sdma_mmu_rb_node_put);
|
|
if (ret) {
|
|
/*
|
|
* When there's a failure, the entire request is freed by
|
|
* user_sdma_send_pkts().
|
|
*/
|
|
SDMA_DBG(req,
|
|
"sdma_txadd_page failed %d page_index %lu page_offset %u from_this_page %u",
|
|
ret, page_index, page_offset, from_this_page);
|
|
return ret;
|
|
}
|
|
start += from_this_page;
|
|
from_this_cache_entry -= from_this_page;
|
|
}
|
|
return 0;
|
|
}
|
|
|
|
static int add_system_iovec_to_sdma_packet(struct user_sdma_request *req,
|
|
struct user_sdma_txreq *tx,
|
|
struct user_sdma_iovec *iovec,
|
|
size_t from_this_iovec)
|
|
{
|
|
while (from_this_iovec > 0) {
|
|
struct sdma_mmu_node *cache_entry;
|
|
size_t from_this_cache_entry;
|
|
size_t start;
|
|
int ret;
|
|
|
|
start = (uintptr_t)iovec->iov.iov_base + iovec->offset;
|
|
ret = get_system_cache_entry(req, &cache_entry, start,
|
|
from_this_iovec);
|
|
if (ret) {
|
|
SDMA_DBG(req, "pin system segment failed %d", ret);
|
|
return ret;
|
|
}
|
|
|
|
from_this_cache_entry = cache_entry->rb.len - (start - cache_entry->rb.addr);
|
|
if (from_this_cache_entry > from_this_iovec)
|
|
from_this_cache_entry = from_this_iovec;
|
|
|
|
ret = add_mapping_to_sdma_packet(req, tx, cache_entry, start,
|
|
from_this_cache_entry);
|
|
|
|
/*
|
|
* Done adding cache_entry to zero or more sdma_desc. Can
|
|
* kref_put() the "safety" kref taken under
|
|
* get_system_cache_entry().
|
|
*/
|
|
kref_put(&cache_entry->rb.refcount, hfi1_mmu_rb_release);
|
|
|
|
if (ret) {
|
|
SDMA_DBG(req, "add system segment failed %d", ret);
|
|
return ret;
|
|
}
|
|
|
|
iovec->offset += from_this_cache_entry;
|
|
from_this_iovec -= from_this_cache_entry;
|
|
}
|
|
|
|
return 0;
|
|
}
|
|
|
|
static int add_system_pages_to_sdma_packet(struct user_sdma_request *req,
|
|
struct user_sdma_txreq *tx,
|
|
struct user_sdma_iovec *iovec,
|
|
u32 *pkt_data_remaining)
|
|
{
|
|
size_t remaining_to_add = *pkt_data_remaining;
|
|
/*
|
|
* Walk through iovec entries, ensure the associated pages
|
|
* are pinned and mapped, add data to the packet until no more
|
|
* data remains to be added.
|
|
*/
|
|
while (remaining_to_add > 0) {
|
|
struct user_sdma_iovec *cur_iovec;
|
|
size_t from_this_iovec;
|
|
int ret;
|
|
|
|
cur_iovec = iovec;
|
|
from_this_iovec = iovec->iov.iov_len - iovec->offset;
|
|
|
|
if (from_this_iovec > remaining_to_add) {
|
|
from_this_iovec = remaining_to_add;
|
|
} else {
|
|
/* The current iovec entry will be consumed by this pass. */
|
|
req->iov_idx++;
|
|
iovec++;
|
|
}
|
|
|
|
ret = add_system_iovec_to_sdma_packet(req, tx, cur_iovec,
|
|
from_this_iovec);
|
|
if (ret)
|
|
return ret;
|
|
|
|
remaining_to_add -= from_this_iovec;
|
|
}
|
|
*pkt_data_remaining = remaining_to_add;
|
|
|
|
return 0;
|
|
}
|
|
|
|
static bool sdma_rb_filter(struct mmu_rb_node *node, unsigned long addr,
|
|
unsigned long len)
|
|
{
|
|
return (bool)(node->addr == addr);
|
|
}
|
|
|
|
/*
|
|
* Return 1 to remove the node from the rb tree and call the remove op.
|
|
*
|
|
* Called with the rb tree lock held.
|
|
*/
|
|
static int sdma_rb_evict(void *arg, struct mmu_rb_node *mnode,
|
|
void *evict_arg, bool *stop)
|
|
{
|
|
struct sdma_mmu_node *node =
|
|
container_of(mnode, struct sdma_mmu_node, rb);
|
|
struct evict_data *evict_data = evict_arg;
|
|
|
|
/* this node will be evicted, add its pages to our count */
|
|
evict_data->cleared += node->npages;
|
|
|
|
/* have enough pages been cleared? */
|
|
if (evict_data->cleared >= evict_data->target)
|
|
*stop = true;
|
|
|
|
return 1; /* remove this node */
|
|
}
|
|
|
|
static void sdma_rb_remove(void *arg, struct mmu_rb_node *mnode)
|
|
{
|
|
struct sdma_mmu_node *node =
|
|
container_of(mnode, struct sdma_mmu_node, rb);
|
|
|
|
free_system_node(node);
|
|
}
|