All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH RFC 0/9] A rendezvous module
@ 2021-03-19 12:56 kaike.wan
  2021-03-19 12:56 ` [PATCH RFC 1/9] RDMA/rv: Public interferce for the RDMA Rendezvous module kaike.wan
                   ` (9 more replies)
  0 siblings, 10 replies; 52+ messages in thread
From: kaike.wan @ 2021-03-19 12:56 UTC (permalink / raw)
  To: dledford, jgg; +Cc: linux-rdma, todd.rimmer, Kaike Wan

From: Kaike Wan <kaike.wan@intel.com>

RDMA transactions on RC QPs have a high demand for memory for HPC
applications such as MPI, especially on nodes with high cpu core
count, where a process is normally dispatched to each core and an RC QP
is created for each remote process. For a 100-node fabric,
about 10 GB - 100 GB memory is required for WQEs/Buffers/QP states on
each server node. This high demand imposes a severe restriction
on the scalability of HPC fabric.

A number of solutions have been implemented over the years. UD based
solutions solve the scalability problem by requiring only one UD
QP per process that can send a message to or receive a message from any
other process. However, it does not have the performance of an
RC QP and the application has to manage the segmentation for large
messages.

SRQ reduces the memory demand by sharing a receive queue among multiple
QPs. However, it still requires an RC QP for each remote
process and each RC QP still requires a send queue. In addition, it is
an optional feature.

XRC further reduces the memory demand by allowing a single QP per
process to communicate with any process on a remote node. In this
mode, each process requires only one QP for each remote node. Again,
this is optional and not all vendors support it.

Dynamically Connected transport minimizes the memory usage, but requires
vendor specific hardware and changes in applications.
Addtionally, all of these mechanisms can induce latency jitter due to
use of more QPs, more QP state, and hence additional
PCIe transactions at scale where NIC QP/CQ caches overflow.

Based on this, we are here proposing a generic, vendor-agnostic approach
to address the RC scalalibity and potentially improve RDMA
performance for larger messages which uses RDMA Write as part of a
rendezvous protocol.

Here are the key points of the proposal:
- For each device used by a job on a given node, there are a fixed
  number of RC connections (default: 4) between this node and any
  other remote node, no matter how many processes are running on each
  node for this job. That is, for a given job and device, all the
  processes on a given node will communicate with all the processes of
  the same job on a remote node through the same fixed number of
  connections. This eliminates the increased memory demand caused by
  core count increase and reduces the overall RC connection setup
  costs.
- Memory region cache is added to reduce the MR
  registration/deregistration costs, as the same buffers tend to be used
  repeatedly by applications. The mr cache is per process and can be
  accessed only by processes in the same job.

Here is how we are proposing to implement this application enabling
rendezvous module that take advantage of the various features
of the RDMA subsystem:
- Create a rendezvous module (rv) under drivers/infiniband/ulp/rv/. This
  can be changed if a better location is recommended.
- The rv modules adds a char device file /dev/rv that user
  applications/middlewares can open to communicate with the rv module
  (PSM3 OFI provider in libfabric 1.12.0 being the primary use case).
  The file interface is crucial for associating the job/process
  with the mr, the connection endpoints, and RDMA requests.
  rv_file_open - Open a handle to the rv module for the current user
                 process.
  rv_file_close - Close the handle and clean up.
  rv_file_ioctl - The main communication methods: to attach to a device,
                  to register/deregister mr, to create connection
                  endpoint, to connect, to query, to get stats, to do
                  RDMA transactions, etc.
  rv_file_mmap - Mmap an event ring into user space for the current user
                 process.
- Basic mode of operations (PSM3 is used as an example for user
  applications):
  - A middleware (like MPI) has out-of-band communication channels
    between any two nodes, which are used to establish high performance
    communications for providers such as PSM3.
  - On each node, the PSM3 provider opens the rv module, and issues an
    attach request to bind to a specific local device and specifies
    information specific to the job. This associates the user process
    with the RDMA device and the job.
  - On each node, the PSM3 provider will establish user space low
    latency eager and control message communications, typically via user
    space UD QPs.
  - On each node, the PSM3 provider will mmap a ring buffer for events
    from the rv module.
  - On each node, the PSM3 provider creates a set of connection
    endpoints by passing the destination info to the rv module. If the
    local node is chosen as the listener for the connection (based on
    address comparison between the two nodes), it will start to listen
    for any incoming IB CM connection requests and accept them when whey
    arrive. This step associates the shared connection endpoints with
    the job and device.
  - On each node, the PSM3 provider will request connection
    establishment through the rv module. On the client node, rv sends an
    IB CM request to the remote node. On the listener node, nothing
    additional needs to be done for the connect request from the PSM3
    provider.
  - On each node, the PSM3 provider will wait until the RC connection is
    established. The PSM3 provider will query the rv module about the
    connection status periodically.
  - As large MPI messages are requested, the PSM3 provider will request
    rv to register MRs for the MPI application’s send and receive
    buffers.  The rv module makes use of its MR cache to limit when such
    requests need to interact with verbs MR calls for the NIC.
    The PSM3 provider control message mechanisms are used to exchange IO
    virtual addresses and rkeys for such buffers and MRs.
  - For RDMA transactions, the RDMA WRITE WITH IMMED request will be
    used. The immediate data will contain info about the target
    process on the remote node  and which outstanding rendezvous message
    this RDMA is for. For send completion and receive completion,
    an event will be posted into the event ring buffer and the PSM3
    provider can poll for completion.
  - As RDMA transactions complete, the PSM3 provider will indicate
    completion of the corresponding MPI send/receive to the MPI
    middleware/application. The PSM3 provider will also inform the rv
    module it may deregister the MR.  The deregistration allows rv
    to cache the MR for use in future requests by the PSM3 provider to
    register the same memory for use in another MPI message. The
    cached MR will be removed if the user buffer is freed and a MMU
    notifier event is received by the rv module.

Please comment,

Thank you,

Kaike

Kaike Wan (9):
  RDMA/rv: Public interferce for the RDMA Rendezvous module
  RDMA/rv: Add the internal header files
  RDMA/rv: Add the rv module
  RDMA/rv: Add functions for memory region cache
  RDMA/rv: Add function to register/deregister memory region
  RDMA/rv: Add connection management functions
  RDMA/rv: Add functions for RDMA transactions
  RDMA/rv: Add functions for file operations
  RDMA/rv: Integrate the file operations into the rv module

 MAINTAINERS                                |    6 +
 drivers/infiniband/Kconfig                 |    1 +
 drivers/infiniband/core/core_priv.h        |    4 +
 drivers/infiniband/core/rdma_core.c        |  237 ++
 drivers/infiniband/core/uverbs_cmd.c       |   54 +-
 drivers/infiniband/ulp/Makefile            |    1 +
 drivers/infiniband/ulp/rv/Kconfig          |   11 +
 drivers/infiniband/ulp/rv/Makefile         |    9 +
 drivers/infiniband/ulp/rv/rv.h             |  892 ++++++
 drivers/infiniband/ulp/rv/rv_conn.c        | 3037 ++++++++++++++++++++
 drivers/infiniband/ulp/rv/rv_file.c        | 1196 ++++++++
 drivers/infiniband/ulp/rv/rv_main.c        |  271 ++
 drivers/infiniband/ulp/rv/rv_mr.c          |  393 +++
 drivers/infiniband/ulp/rv/rv_mr_cache.c    |  428 +++
 drivers/infiniband/ulp/rv/rv_mr_cache.h    |  152 +
 drivers/infiniband/ulp/rv/rv_rdma.c        |  416 +++
 drivers/infiniband/ulp/rv/trace.c          |    7 +
 drivers/infiniband/ulp/rv/trace.h          |   10 +
 drivers/infiniband/ulp/rv/trace_conn.h     |  547 ++++
 drivers/infiniband/ulp/rv/trace_dev.h      |   82 +
 drivers/infiniband/ulp/rv/trace_mr.h       |  109 +
 drivers/infiniband/ulp/rv/trace_mr_cache.h |  181 ++
 drivers/infiniband/ulp/rv/trace_rdma.h     |  254 ++
 drivers/infiniband/ulp/rv/trace_user.h     |  321 +++
 include/rdma/uverbs_types.h                |   10 +
 include/uapi/rdma/rv_user_ioctls.h         |  558 ++++
 26 files changed, 9139 insertions(+), 48 deletions(-)
 create mode 100644 drivers/infiniband/ulp/rv/Kconfig
 create mode 100644 drivers/infiniband/ulp/rv/Makefile
 create mode 100644 drivers/infiniband/ulp/rv/rv.h
 create mode 100644 drivers/infiniband/ulp/rv/rv_conn.c
 create mode 100644 drivers/infiniband/ulp/rv/rv_file.c
 create mode 100644 drivers/infiniband/ulp/rv/rv_main.c
 create mode 100644 drivers/infiniband/ulp/rv/rv_mr.c
 create mode 100644 drivers/infiniband/ulp/rv/rv_mr_cache.c
 create mode 100644 drivers/infiniband/ulp/rv/rv_mr_cache.h
 create mode 100644 drivers/infiniband/ulp/rv/rv_rdma.c
 create mode 100644 drivers/infiniband/ulp/rv/trace.c
 create mode 100644 drivers/infiniband/ulp/rv/trace.h
 create mode 100644 drivers/infiniband/ulp/rv/trace_conn.h
 create mode 100644 drivers/infiniband/ulp/rv/trace_dev.h
 create mode 100644 drivers/infiniband/ulp/rv/trace_mr.h
 create mode 100644 drivers/infiniband/ulp/rv/trace_mr_cache.h
 create mode 100644 drivers/infiniband/ulp/rv/trace_rdma.h
 create mode 100644 drivers/infiniband/ulp/rv/trace_user.h
 create mode 100644 include/uapi/rdma/rv_user_ioctls.h

-- 
2.18.1


^ permalink raw reply	[flat|nested] 52+ messages in thread

* [PATCH RFC 1/9] RDMA/rv: Public interferce for the RDMA Rendezvous module
  2021-03-19 12:56 [PATCH RFC 0/9] A rendezvous module kaike.wan
@ 2021-03-19 12:56 ` kaike.wan
  2021-03-19 16:00   ` Jason Gunthorpe
  2021-03-19 18:42   ` kernel test robot
  2021-03-19 12:56 ` [PATCH RFC 2/9] RDMA/rv: Add the internal header files kaike.wan
                   ` (8 subsequent siblings)
  9 siblings, 2 replies; 52+ messages in thread
From: kaike.wan @ 2021-03-19 12:56 UTC (permalink / raw)
  To: dledford, jgg; +Cc: linux-rdma, todd.rimmer, Kaike Wan

From: Kaike Wan <kaike.wan@intel.com>

The RDMA Rendezvous (rv) module provides an interface for HPC
middlewares to improve performance by caching memory region
registration, and improve the scalibity of RDMA transaction
through connection managements between nodes. This mechanism
is implemented through the following ioctl requests:
- ATTACH: to attach to an RDMA device.
- REG_MEM: to register a user/kernel memory region.
- DEREG_MEM: to release application use of MR, allowing it to
             remain in cache.
- GET_CACHE_STATS: to get cache statistics.
- CONN_CREATE: to create an RC connection.
- CONN_CONNECT: to start the connection.
- CONN_GET_CONN_COUNT: to use as part of error recovery from lost
                       messages in the application.
- CONN_GET_STATS: to get connection statistics.
- GET_EVENT_STATS: to get the RDMA event statistics.
- POST_RDMA_WR_IMMED: to post an RDMA WRITE WITH IMMED request.

Signed-off-by: Todd Rimmer <todd.rimmer@intel.com>
Signed-off-by: Kaike Wan <kaike.wan@intel.com>
---
 include/uapi/rdma/rv_user_ioctls.h | 558 +++++++++++++++++++++++++++++
 1 file changed, 558 insertions(+)
 create mode 100644 include/uapi/rdma/rv_user_ioctls.h

diff --git a/include/uapi/rdma/rv_user_ioctls.h b/include/uapi/rdma/rv_user_ioctls.h
new file mode 100644
index 000000000000..97e35b722443
--- /dev/null
+++ b/include/uapi/rdma/rv_user_ioctls.h
@@ -0,0 +1,558 @@
+/* SPDX-License-Identifier: ((GPL-2.0 WITH Linux-syscall-note) OR BSD-3-Clause) */
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+#ifndef __RV_USER_IOCTL_H__
+#define __RV_USER_IOCTL_H__
+#include <rdma/rdma_user_ioctl.h>
+#include <rdma/ib_user_sa.h>
+#include <rdma/ib_user_verbs.h>
+
+/* Checking /Documentation/userspace-api/ioctl/ioctl-number.rst */
+#define RV_MAGIC RDMA_IOCTL_MAGIC
+#define RV_FILE_NAME "/dev/rv"
+
+/*
+ * Handles are opaque to application; they are meaningful only to the
+ * RV driver
+ */
+
+/* this version of ABI */
+#define RV_ABI_VER_MAJOR 1
+#define RV_ABI_VER_MINOR 0
+
+struct rv_query_params_out {
+		/* ABI version */
+	__u16 major_rev;
+	__u16 minor_rev;
+	__u32 resv1;
+	__aligned_u64 capability;
+	__aligned_u64 resv2[6];
+};
+
+#define RV_IOCTL_QUERY _IOR(RV_MAGIC, 0xFC, struct rv_query_params_out)
+
+/* Mode for use of rv module by PSM */
+#define RV_RDMA_MODE_USER 0	/* user MR caching only */
+#define RV_RDMA_MODE_KERNEL 1	/* + kernel RC QPs with kernel MR caching */
+#define RV_RDMA_MODE_MAX 1
+
+#define RV_MAX_DEV_NAME_LEN IB_DEVICE_NAME_MAX
+#define RV_MAX_NUM_CONN 16
+#define RV_MAX_INDEX_BITS 12
+#define RV_MAX_JOB_KEY_LEN 16
+#define RV_MAX_CQ_ENTRIES 10000
+
+/*
+ * mr_cache_size is in MBs and if 0 will use module param as default
+ * num_conn - number of QPs between each pair of nodes
+ * loc_addr - used to select client/listen vs rem_addr
+ * index_bits - num high bits of immed data with rv index
+ * loc_gid_index - SGID for client connections
+ * loc_gid[16] - to double check gid_index unchanged
+ * job_key[RV_MAX_JOB_KEY_LEN] = unique uuid per job
+ * job_key_len - len, if 0 matches jobs with len==0 only
+ * q_depth - size of QP and per QP CQs
+ * reconnect_timeout - in seconds from loss to restoration
+ * hb_interval - in milliseconds between heartbeats
+ */
+struct rv_attach_params_in {
+	char dev_name[RV_MAX_DEV_NAME_LEN];
+	__u32 mr_cache_size;
+	__u8 rdma_mode;
+
+	/* additional information for RV_RDMA_MODE_KERNEL */
+	__u8 port_num;
+	__u8 num_conn;
+	__u32 loc_addr;
+	__u8 index_bits;
+	__u16 loc_gid_index;
+	__u8 loc_gid[16];
+	__u8 job_key[RV_MAX_JOB_KEY_LEN];
+	__u8 job_key_len;
+	__aligned_u64 service_id;
+	__aligned_u64 context;
+	__u32 cq_entries;
+	__u32 q_depth;
+	__u32 reconnect_timeout;
+	__u32 hb_interval;
+};
+
+/*
+ * rv_index - unique within job on given NIC
+ * mr_cache_size - in MBs
+ * q_depth - size of QP and per QP CQs
+ * reconnect_timeout - value being used
+ */
+struct rv_attach_params_out {
+	__u32 rv_index;
+	__u32 mr_cache_size;
+	__u32 q_depth;
+	__u32 reconnect_timeout;
+};
+
+struct rv_attach_params {
+	union {
+		struct rv_attach_params_in in;
+		struct rv_attach_params_out out;
+	};
+};
+
+#define RV_IOCTL_ATTACH		_IOWR(RV_MAGIC, 0xF1, struct rv_attach_params)
+
+/* The buffer is used to register a kernel mr */
+#define IBV_ACCESS_KERNEL 0x80000000
+
+/*
+ * ibv_pd_handle - user space appl allocated pd
+ * ulen - driver_udata inlen
+ * *udata - driver_updata inbuf
+ */
+struct rv_mem_params_in {
+	__u32 ibv_pd_handle;
+	__u32 cmd_fd_int;
+	__aligned_u64 addr;
+	__aligned_u64 length;
+	__u32 access;
+	size_t ulen;
+	void *udata;
+};
+
+struct rv_mem_params_out {
+	__aligned_u64 mr_handle;
+	__aligned_u64 iova;
+	__u32 lkey;
+	__u32 rkey;
+};
+
+struct rv_mem_params {
+	union {
+		struct rv_mem_params_in in;
+		struct rv_mem_params_out out;
+	};
+};
+
+#define RV_IOCTL_REG_MEM	_IOWR(RV_MAGIC, 0xF3, struct rv_mem_params)
+
+struct rv_dereg_params_in {
+	__aligned_u64 mr_handle;
+	__aligned_u64 addr;
+	__aligned_u64 length;
+	__u32 access;
+};
+
+#define RV_IOCTL_DEREG_MEM	_IOW(RV_MAGIC, 0xF4, struct rv_dereg_params_in)
+
+/*
+ * MR cache statistics are as follows:
+ *   cache_size - Current memory in the cache in bytes
+ *   max_cache_size - Maximum of cache_size in bytes
+ *   limit_cache_size - Maximum allowed cache_size in MB
+ *   count - Current number of MRs in the cache
+ *   max_count - Maximum of count
+ *   inuse - Current number of MRs in the cache with refcount > 0
+ *   max_inuse - Maximum of inuse
+ *   inuse_bytes - Current number of bytes in cache for MRs with refcount > 0
+ *   max_inuse_bytes - Maximum of inuse_bytes
+ *   max_refcount - Maximum of refcount for an MR
+ *   Event counts:
+ *      hit - Cache hit
+ *      miss - Cache miss and added
+ *      full - Cache miss but can't add since full
+ *      failed - Cache miss but can't add since reg_mr failed
+ *      remove - Refcount == 0 and removed by mmu notifier event or cache deinit
+ *      evict - Removed from cache due to lack of cache space
+ *   Number of valid IOCTL_REG_MEM calls = hit+miss+full+failed
+ *      (counts omit some EINVAL and EFAULT REG_MEM use cases)
+ */
+
+struct rv_cache_stats_params_out {
+	__aligned_u64 cache_size;
+	__aligned_u64 max_cache_size;
+	__u32 limit_cache_size;
+	__u32 count;
+	__u32 max_count;
+	__u32 inuse;
+	__u32 max_inuse;
+	__aligned_u64 inuse_bytes;
+	__aligned_u64 max_inuse_bytes;
+	__u32 max_refcount;
+	__aligned_u64 hit;
+	__aligned_u64 miss;
+	__aligned_u64 full;
+	__aligned_u64 failed;
+	__aligned_u64 remove;
+	__aligned_u64 evict;
+};
+
+#define RV_IOCTL_GET_CACHE_STATS _IOR(RV_MAGIC, 0xF7, \
+				      struct rv_cache_stats_params_out)
+
+/*
+ * The create provides an ah_attr.  Field use is as follows:
+ * sanity checked with attach:
+ *	grh.sgid_index, port_num
+ *	possibly in future: src_path_bits
+ * identify QPs which can be shared
+ *	rem_addr, ah.is_global, grh.dgid or dlid
+ *	possibly in future: sl, traffic_class, flow_label, maybe static_rate
+ * sanity checked with connect path:
+ *	dlid, grh.dgid
+ *	could check: sl
+ *	don't want to check: static_rate (could negotiate down in path)
+ * validated with inbound REQ
+ *	port_num, grh.dgid or dlid
+ * Not used: hop_limit (locally resolve)
+ * rem_addr - used to select client/listen vs loc_addr
+ */
+struct rv_conn_create_params_in {
+	struct ib_uverbs_ah_attr ah;
+	__u32 rem_addr;
+	__aligned_u64 context;
+};
+
+/*
+ * handle - rv_user_conn for future conn/discon calls
+ * conn_handle - rv_conn for completions only
+ */
+struct rv_conn_create_params_out {
+	__aligned_u64 handle;
+	__aligned_u64 conn_handle;
+};
+
+struct rv_conn_create_params {
+	union {
+		struct rv_conn_create_params_in in;
+		struct rv_conn_create_params_out out;
+	};
+};
+
+#define RV_IOCTL_CONN_CREATE	_IOWR(RV_MAGIC, 0xF8, \
+				      struct rv_conn_create_params)
+
+struct rv_conn_connect_params_in {
+	__aligned_u64 handle;
+	struct ib_user_path_rec path;
+};
+
+#define RV_IOCTL_CONN_CONNECT	_IOW(RV_MAGIC, 0xF9, \
+				     struct rv_conn_connect_params_in)
+
+struct rv_conn_connected_params_in {
+	__aligned_u64 handle;
+};
+
+#define RV_IOCTL_CONN_CONNECTED	_IOW(RV_MAGIC, 0xFA, \
+				     struct rv_conn_connected_params_in)
+
+/*
+ * get connection count for a specific sconn
+ * returns:
+ *	0 - count returned
+ *	EIO - connection lost and unrecoverable
+ *	EINVAL - invalid handle and/or index
+ * A 32b count is sufficient to handle constant RV reconnects, with a
+ * 100ms delay between each established connection, for up to 13 years.
+ */
+struct rv_conn_get_conn_count_params_in {
+	__aligned_u64 handle;
+	__u8 index;
+};
+
+/* we return count as an output parameter (vs ioctl ret) so can use full 32b */
+struct rv_conn_get_conn_count_params_out {
+	__u32 count;
+};
+
+struct rv_conn_get_conn_count_params {
+	union {
+		struct rv_conn_get_conn_count_params_in in;
+		struct rv_conn_get_conn_count_params_out out;
+	};
+};
+
+#define RV_IOCTL_CONN_GET_CONN_COUNT	_IOWR(RV_MAGIC, 0xFF, \
+					   struct rv_conn_get_conn_count_params)
+
+/* index to get agg of sconn in given conn */
+#define RV_CONN_STATS_AGGREGATE 255
+
+/*
+ * handle - if 0, aggregate of all rv's sconn returned
+ * index - ignored if !handle, otherwise specific sconn index
+ */
+struct rv_conn_get_stats_params_in {
+	__aligned_u64 handle;
+	__u8 index;
+};
+
+/*
+ * flags can be combined when get aggregate results
+ * so separate bits for client vs server
+ */
+#define RV_CONN_STAT_FLAG_SERVER 0x01
+#define RV_CONN_STAT_FLAG_CLIENT 0x02
+#define RV_CONN_STAT_FLAG_WAS_CONNECTED 0x04
+
+/*
+ * index - mimics input value
+ * flags
+ * num_conn - total QPs included
+ * CM events
+ *	req_error - IB_CM_REQ_ERROR
+ *	req_recv - IB_CM_REQ_RECEIVED
+ *	rep_error - IB_CM_REP_ERROR
+ *	rep_recv - IB_CM_REP_RECEIVED
+ *	rtu_recv - IB_CM_RTU_RECEIVED
+ *	established - IB_CM_USER_ESTABLISHED - via ib_cm_notify
+ *	dreq_error - IB_CM_DREQ_ERROR
+ *	dreq_recv - IB_CM_DREQ_RECEIVED
+ *	drep_recv - IB_CM_DREP_RECEIVED
+ *	timewait - IB_CM_TIMEWAIT_EXIT
+ *	mra_recv - IB_CM_MRA_RECEIVED
+ *	rej_recv - IB_CM_REJ_RECEIVED
+ *	lap_error - IB_CM_LAP_ERROR
+ *	lap_recv - IB_CM_LAP_RECEIVED
+ *	apr_recv - IB_CM_APR_RECEIVED
+ *	unexp_event - SIDR and any others
+ * outbound CM messages
+ *	req_sent - CM REQ
+ *	rep_sent - CM REP
+ *	rtu_sent - CM RTU
+ *	rej_sent - CM REJ
+ *	dreq_sent - CM DREQ
+ *	drep_sent - CM DREP
+ *	(re)connect time does not include wait nor resolver time
+ *	wait_time - microseconds for initial connect
+ *	resolve_time - microseconds for initial connect
+ *	connect_time - microseconds for initial connect
+ *	connected_time - microseconds were connected
+ *	resolve - attempts at resolving
+ *	resolve_fail - hard failures
+ *	conn_recovery - # times recovered connection
+ *	rewait_time - microseconds for connect recovery
+ *	reresolve_time - microseconds for connect recovery
+ *	reconnect_time - microseconds for connect recovery
+ *	max_rewait_time - microseconds for connect recovery
+ *	max_reresolve_time - microseconds for connect recovery
+ *	max_reconnect_time - microseconds for connect recovery
+ *	reresolve - attempts at resolving
+ *	reresolve_fail - hard failures
+ *	post_write - successful post_rdma_write
+ *	post_write_fail - failed at time of posting
+ *	post_write_bytes - for successful post
+ *	outstand_send_write - sent RDMA Write waiting for CQE
+ *	send_write_cqe - successful sent RDMA Write CQE
+ *	send_write_cqe_fail - sent RDMA Write CQE with bad status
+ *	recv_write_cqe - successful recv RDMA Write CQE
+ *	recv_write_bytes - successful recv RDMA Write
+ *	recv_cqe_fail - recv CQE with bad status
+ *	post_hb - successful post of heartbeat
+ *	post_hb_fail - failed at time of posting
+ *	send_hb_cqe - successful sent heartbeat CQE
+ *	send_hb_cqe_fail - sent heartbeat CQE with bad status
+ *	recv_hb_cqe - successful recv heartbeat CQE
+ */
+struct rv_conn_get_stats_params_out {
+	__u8 index;
+	__u8 flags;
+	__u32 num_conn;
+
+	/* CM events */
+	__u32 req_error;
+	__u32 req_recv;
+	__u32 rep_error;
+	__u32 rep_recv;
+	__u32 rtu_recv;
+	__u32 established;
+	__u32 dreq_error;
+	__u32 dreq_recv;
+	__u32 drep_recv;
+	__u32 timewait;
+	__u32 mra_recv;
+	__u32 rej_recv;
+	__u32 lap_error;
+	__u32 lap_recv;
+	__u32 apr_recv;
+	__u32 unexp_event;
+
+	/* outbound CM messages */
+	__u32 req_sent;
+	__u32 rep_sent;
+	__u32 rtu_sent;
+	__u32 rej_sent;
+	__u32 dreq_sent;
+	__u32 drep_sent;
+	__aligned_u64 wait_time;
+	__aligned_u64 resolve_time;
+	__aligned_u64 connect_time;
+	__aligned_u64 connected_time;
+	__u32 resolve;
+	__u32 resolve_fail;
+	__u32 conn_recovery;
+	__aligned_u64 rewait_time;
+	__aligned_u64 reresolve_time;
+	__aligned_u64 reconnect_time;
+	__aligned_u64 max_rewait_time;
+	__aligned_u64 max_reresolve_time;
+	__aligned_u64 max_reconnect_time;
+	__u32 reresolve;
+	__u32 reresolve_fail;
+
+	__aligned_u64 post_write;
+	__aligned_u64 post_write_fail;
+	__aligned_u64 post_write_bytes;
+	__u32 outstand_send_write;
+	__aligned_u64 send_write_cqe;
+	__aligned_u64 send_write_cqe_fail;
+
+	__aligned_u64 recv_write_cqe;
+	__aligned_u64 recv_write_bytes;
+	__aligned_u64 recv_cqe_fail;
+
+	__aligned_u64 post_hb;
+	__aligned_u64 post_hb_fail;
+	__aligned_u64 send_hb_cqe;
+	__aligned_u64 send_hb_cqe_fail;
+	__aligned_u64 recv_hb_cqe;
+};
+
+struct rv_conn_get_stats_params {
+	union {
+		struct rv_conn_get_stats_params_in in;
+		struct rv_conn_get_stats_params_out out;
+	};
+};
+
+#define RV_IOCTL_CONN_GET_STATS	_IOWR(RV_MAGIC, 0xFD, \
+				      struct rv_conn_get_stats_params)
+
+/*
+ * send_write_cqe - successful sent RDMA Write CQE
+ * send_write_cqe_fail - sent RDMA Write CQE with bad status
+ * send_write_bytes - for successful send
+ * recv_write_cqe - successful recv RDMA Write CQE
+ * recv_write_cqe_fail - recv RDMA Write CQE with bad status
+ * recv_write_bytes - successful recv RDMA Write
+ */
+struct rv_event_stats_params_out {
+	__aligned_u64 send_write_cqe;
+	__aligned_u64 send_write_cqe_fail;
+	__aligned_u64 send_write_bytes;
+
+	__aligned_u64 recv_write_cqe;
+	__aligned_u64 recv_write_cqe_fail;
+	__aligned_u64 recv_write_bytes;
+};
+
+#define RV_IOCTL_GET_EVENT_STATS _IOR(RV_MAGIC, 0xFE, \
+				      struct rv_event_stats_params_out)
+
+/*
+ * handle - from create_conn
+ * application source buffer and a kernel lkey for it
+ *	loc_addr
+ *	loc_mr_handle
+ * local MR - selected by 3-tuple (addr, len, access)
+ *	loc_mr_addr
+ *	loc_mr_length
+ *	loc_mr_access
+ * remote application dest buffer and a kernel rkey for it
+ *	rem_addr
+ *	rkey
+ * length
+ * wr_id - application context, included in RV SQ completion events
+ * immed
+ */
+struct rv_post_write_params_in {
+	__aligned_u64 handle;
+	__aligned_u64 loc_addr;
+	__aligned_u64 loc_mr_handle;
+	__aligned_u64 loc_mr_addr;
+	__aligned_u64 loc_mr_length;
+	__u32 loc_mr_access;
+	__u32 rkey;
+	__aligned_u64 rem_addr;
+	__aligned_u64 length;
+	__aligned_u64 wr_id;
+	__u32 immed;
+};
+
+struct rv_post_write_params_out {
+	__u8 sconn_index;
+	__u32 conn_count;
+};
+
+struct rv_post_write_params {
+	union {
+		struct rv_post_write_params_in in;
+		struct rv_post_write_params_out out;
+	};
+};
+
+#define RV_IOCTL_POST_RDMA_WR_IMMED _IOWR(RV_MAGIC, 0xFB, \
+					  struct rv_post_write_params)
+
+enum rv_event_type {
+	RV_WC_RDMA_WRITE,		/* send RDMA Write CQE */
+	RV_WC_RECV_RDMA_WITH_IMM,	/* recv RDMA Write w/immed CQE */
+};
+
+/*
+ * events placed on ring buffer for delivery to user space.
+ * Carefully sized to be a multiple of 64 bytes for cache alignment.
+ * Must pack to get good field alignment and desired 64B overall size
+ * Unlike verbs, all rv_event fields are defined even when
+ * rv_event.wc.status != IB_WC_SUCCESS. Only sent writes can report bad status.
+ * event_type - enum rv_event_type
+ * wc - send or recv work completions
+ *	status - ib_wc_status
+ *	resv1 - alignment
+ *	imm_data - for RV_WC_RECV_RDMA_WITH_IMM only
+ *	wr_id - PSM wr_id for RV_WC_RDMA_WRITE only
+ *	conn_handle - conn handle. For efficiency in completion processing, this
+ *		handle is the rv_conn handle, not the rv_user_conn.
+ *		Main use is sanity checks.  On Recv PSM must use imm_data to
+ *		efficiently identify source.
+ *	byte_len - unlike verbs API, this is always valid
+ *	resv2 - alignment
+ * cache_align -  not used, but forces overall struct to 64B size
+ */
+struct rv_event {
+	__u8		event_type;
+	union {
+		struct {
+			__u8		status;
+			__u16	resv1;
+			__u32	imm_data;
+			__aligned_u64	wr_id;
+			__aligned_u64	conn_handle;
+			__u32	byte_len;
+			__u32	resv2;
+		} __attribute__((__packed__)) wc;
+		struct {
+			__u8 pad[7];
+			uint64_t pad2[7];
+		} __attribute__((__packed__)) cache_align;
+	};
+} __attribute__((__packed__));
+
+/*
+ * head - consumer removes here
+ * tail - producer adds here
+ * overflow_cnt - number of times producer overflowed ring and discarded
+ * pad - 64B cache alignment for entries
+ */
+struct rv_ring_header {
+	volatile __u32 head;
+	volatile __u32 tail;
+	volatile __u64 overflow_cnt;
+	__aligned_u64 pad[6];
+	struct rv_event entries[];
+};
+
+#define RV_RING_ALLOC_LEN(num_entries) \
+	((__u32)(((num_entries) * sizeof(struct rv_event)) + \
+	       sizeof(struct rv_ring_header)))
+
+#endif /* __RV_USER_IOCTL_H__ */
-- 
2.18.1


^ permalink raw reply related	[flat|nested] 52+ messages in thread

* [PATCH RFC 2/9] RDMA/rv: Add the internal header files
  2021-03-19 12:56 [PATCH RFC 0/9] A rendezvous module kaike.wan
  2021-03-19 12:56 ` [PATCH RFC 1/9] RDMA/rv: Public interferce for the RDMA Rendezvous module kaike.wan
@ 2021-03-19 12:56 ` kaike.wan
  2021-03-19 16:02   ` Jason Gunthorpe
  2021-03-19 12:56 ` [PATCH RFC 3/9] RDMA/rv: Add the rv module kaike.wan
                   ` (7 subsequent siblings)
  9 siblings, 1 reply; 52+ messages in thread
From: kaike.wan @ 2021-03-19 12:56 UTC (permalink / raw)
  To: dledford, jgg; +Cc: linux-rdma, todd.rimmer, Kaike Wan

From: Kaike Wan <kaike.wan@intel.com>

The two header files include the defines, structures, MACROs,
function prototypes, and functions used in the module.

Signed-off-by: Todd Rimmer <todd.rimmer@intel.com>
Signed-off-by: Kaike Wan <kaike.wan@intel.com>
---
 drivers/infiniband/ulp/rv/rv.h          | 892 ++++++++++++++++++++++++
 drivers/infiniband/ulp/rv/rv_mr_cache.h | 152 ++++
 2 files changed, 1044 insertions(+)
 create mode 100644 drivers/infiniband/ulp/rv/rv.h
 create mode 100644 drivers/infiniband/ulp/rv/rv_mr_cache.h

diff --git a/drivers/infiniband/ulp/rv/rv.h b/drivers/infiniband/ulp/rv/rv.h
new file mode 100644
index 000000000000..20ca65387d22
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/rv.h
@@ -0,0 +1,892 @@
+/* SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause) */
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+
+#ifndef __RV_H__
+#define __RV_H__
+#include <linux/types.h>
+#include <linux/list.h>
+#include <linux/mutex.h>
+#include <linux/scatterlist.h>
+#include <rdma/ib_verbs.h>
+#include <rdma/ib_sa.h>
+#include <rdma/ib_cm.h>
+#include <rdma/rdma_cm.h>
+#include <linux/mutex.h>
+#include <linux/socket.h>
+#include <linux/timer.h>
+#include <rdma/ib_addr.h>
+#include <rdma/ib_cm.h>
+#include <linux/moduleparam.h>
+
+#include "rv_mr_cache.h"
+#include <rdma/rv_user_ioctls.h>
+
+/*
+ * Lock Heirachy
+ * In order that locks can be acquired:
+ * mm->mmap_lock - held during mmap call and mmu_notifier_register
+ * rv_user.mutex
+ * rv_job_dev_list_mutex
+ * rv_job_dev.conn_list_mutex
+ * rv_device.listener_mutex
+ * rv_sconn.mutex - we can never hold this while calling destroy_cm_id
+ *	because destroy cm_id will wait for handlers via it's mutex and deadlock
+ * rv_job_dev_list rcu
+ * rv_dev_list_lock
+ * rv_job_dev.conn_list rcu - no sub locks
+ * rv_device.listener_lock - no sub locks
+ * rv_user.umrs.cache.lock - no sub locks
+ * rv_job_dev.user_array_lock - no sub locks
+ * ring.lock - no sub locks
+ * mr_pd_uobject_lock - no sub locks
+ * rv_conn.next_lock - no sub locks
+ * rv_sconn.drain_lock - no sub locks
+ */
+
+/*
+ * Our goal is to allocate shared resources, but all processes are equal peers.
+ * So we use a get_alloc approach where processes attempt to get a resource
+ * reference (lookup and get) and if it's not on the list, it is allocated and
+ * added.  This is used for rv_job_dev, rv_listener and rv_conn.
+ * Each such list is protected by a mutex to prevent duplicate additions
+ * and an RCU for actual list access.  The mutex also protects
+ * get_alloc/list_del races.  All get_alloc calls are in premptable
+ * app ioctl call context.
+ * rv_listener list is rarely searched, so it simply uses a mutex and spinlock.
+ */
+
+/*
+ * When working with IB CM, Async events and timeouts we can't predict if and
+ * when their callbacks will occur so they can't have a reference held in
+ * advance. This presents opportunities for the callback to race with rv_conn
+ * destruction. To solve this problem, we use rv_conn_get_check to get a
+ * reference to rv_conn only if rv_conn.kref != 0. rv_conn and rv_sconn
+ * destruction uses destroy_cm_id and del_timer_sync to stop all callbacks (and
+ * allow any outstanding handlers to finish) before it proceeds to free rv_conn.
+ * This ensures the cm_id, etc can remain valid enough for the callback to
+ * call rv_conn_get_check and decide whether to simply ignore the event.
+ * It also allows the rv_sconn to be the ib_cm, CQ, QP and timer context and
+ * avoid searches. Once the callback has the reference, it is protected
+ * from other threads calling rv_conn_release.
+ */
+
+#define DRIVER_NAME "rv"
+
+#define RV_INVALID -1
+
+/* For errors to surface on the console */
+#define rv_err(idx, fmt, ...) \
+	pr_err("[%s:%s %d]: " fmt, DRIVER_NAME, __func__, idx, ##__VA_ARGS__)
+
+#define rv_ptr_err(prefix, ptr, fmt, ...) \
+	pr_err("[%s:%s %s 0x%p]: " fmt, DRIVER_NAME, __func__, prefix, ptr, \
+	       ##__VA_ARGS__)
+
+/* For debugging*/
+#define rv_dbg(idx, fmt, ...) \
+	pr_debug("[%s:%s %d]: " fmt, DRIVER_NAME, __func__, idx, ##__VA_ARGS__)
+
+/* For general console info */
+#define rv_info(idx, fmt, ...) \
+	pr_info("[%s:%s %d]: " fmt, DRIVER_NAME, __func__, idx, ##__VA_ARGS__)
+
+/* For debugging with any pointer */
+#define rv_ptr_dbg(prefix, ptr, fmt, ...) \
+	pr_debug("[%s:%s %s 0x%p]: " fmt, DRIVER_NAME, __func__, prefix, ptr, \
+		 ##__VA_ARGS__)
+
+#define rv_ptr_info(prefix, ptr, fmt, ...) \
+	pr_info("[%s:%s %s 0x%p]: " fmt, DRIVER_NAME, __func__, prefix, ptr, \
+		##__VA_ARGS__)
+
+/* For debugging with rv_conn_info */
+#define rv_conn_err(ptr, fmt, ...) rv_ptr_err("sconn", ptr, fmt, ##__VA_ARGS__)
+#define rv_conn_info(ptr, fmt, ...) rv_ptr_info("sconn", ptr, fmt,\
+						##__VA_ARGS__)
+#define rv_conn_dbg(ptr, fmt, ...) rv_ptr_dbg("sconn", ptr, fmt, ##__VA_ARGS__)
+
+/* For debugging with rv_device */
+#define rv_dev_err(ptr, fmt, ...) rv_ptr_err("dev", ptr, fmt, ##__VA_ARGS__)
+#define rv_dev_info(ptr, fmt, ...) rv_ptr_info("dev", ptr, fmt, ##__VA_ARGS__)
+#define rv_dev_dbg(ptr, fmt, ...) rv_ptr_dbg("dev", ptr, fmt, ##__VA_ARGS__)
+
+/* For debugging with ib cm id */
+#define rv_cm_err(ptr, fmt, ...) rv_ptr_err("cm_id", ptr, fmt, ##__VA_ARGS__)
+#define rv_cm_info(ptr, fmt, ...) rv_ptr_info("cm_id", ptr, fmt, ##__VA_ARGS__)
+#define rv_cm_dbg(ptr, fmt, ...) rv_ptr_dbg("cm_id", ptr, fmt, ##__VA_ARGS__)
+
+struct rv_device;
+
+/*
+ * A listener can handle more than 1 job on a given dev.
+ * There is a listener for each unique service_id on each dev and
+ * could be as few as 1 even for multiple jobs.
+ * For IB, a listener services all ports on a given HCA.
+ * listener_entry - entry on rv_device.listener_list
+ * dev - device being listened on
+ * cm_id - actual IB CM listener, stores service_id here
+ */
+struct rv_listener {
+	struct list_head listener_entry;
+	struct rv_device *dev;
+	struct ib_cm_id *cm_id;
+	struct kref kref;
+};
+
+/*
+ * For each physical device, RV establishes a single rv_device which
+ * is shared across all jobs, listeners, etc.
+ * ib_dev - underlying RDMA device
+ * dev_entry - entry on rv_dev_list
+ *
+ * IB cm listener management:
+ *	listener_mutex, listener_lock, listener_list
+ *
+ * user_list - list of rv_user (protected by rv_dev_list_lock)
+ */
+struct rv_device {
+	struct ib_device *ib_dev;
+	struct list_head dev_entry;
+	struct kref kref;
+	struct ib_event_handler event_handler;
+
+	struct mutex listener_mutex; /* avoid duplicate add in get_alloc */
+	spinlock_t listener_lock; /* protect list search, add, remove */
+	struct list_head listener_list;
+
+	struct list_head user_list;
+};
+
+#define RV_CONN_MAX_ACTIVE_WQ_ENTRIES 100 /* used for conn handling */
+
+#define RV_RESOLVER_RETRY 5		/* max retries (0 == try once) */
+#define RV_RESOLVER_TIMEOUT 10000	/* in milliseconds */
+
+/* duration client spends in RV_DELAY before re-attempt reconnect */
+#define RV_RECONNECT_DELAY msecs_to_jiffies(100)
+
+/* private data in REQ and REP is used to negotiate features and exchange
+ * additional informaton.
+ * REQ.ver is the version of the sender and defines the priv_data structure.
+ * Newer versions of struct should be a superset of older versions.
+ * REP.ver is the version the listener is accepting and defines REP priv_data.
+ * New client, old listerner:
+ *	listener accepts REQ >= it's version, responds with it's version in REP
+ * Old client, new listener:
+ *	listener accepts REQ, responds with REP matching client's version
+ * Protocol ver is now 2.  For simplicity protocol 0 and 1 (pre-releases) are
+ * not accepted.
+ */
+
+#define RV_PRIVATE_DATA_MAGIC 0x00125550534d2121ULL
+#define RV_PRIVATE_DATA_VER 2
+
+/*
+ * Private data used in CM REQ
+ * laid out for good alignment of fields without packing
+ * index indicates index of rv_sconn within it's rv_conn and differentiates
+ * the multiple connection REQs for num_conn > 1
+ * The job_key selects the rv_job_dev on receiver side
+ * uid is 32b
+ */
+struct rv_req_priv_data {
+	u64 magic;
+	u32 ver;
+	u16 resv1;
+	u8 index;
+	u8 job_key_len;
+	u8 job_key[RV_MAX_JOB_KEY_LEN];
+	uid_t uid;
+};
+
+/*
+ * Private data used in CM REP
+ * laid out for good alignment of fields without packing
+ */
+struct rv_rep_priv_data {
+	u64 magic;
+	u32 ver;
+};
+
+/*
+ * RV_INIT - client side create_conn done
+ * RV_WAITING - server side create_conn done, listening
+ * RV_RESOLVING - client side resolving ethernet dmac via ARP
+ * RV_CONNECTING - client or server side going through CM states
+ * RV_CONNECTED - connection established
+ * RV_DISCONNECTING - connection teardown
+ * RV_DELAY- RV delay before restarting client connection
+ * RV_ERROR - connection lost or failed
+ * destroying - ref count == 0
+ */
+enum rv_sconn_state {
+	RV_INIT = 1,
+	RV_WAITING,
+	RV_RESOLVING,
+	RV_CONNECTING,
+	RV_CONNECTED,
+	RV_DISCONNECTING,
+	RV_DELAY,
+	RV_ERROR,
+};
+
+/*
+ * Meaning of each state and status of key fields once in state and
+ * have unlocked mutex.
+ * In all states (except while destroying) immutable fields in rv_conn valid.
+ *
+ * WAS_CONNECTED flag essentially creates a superstate indicating connection
+ * recovery for WAITING, RESOLVING, and CONNECTING.  During connection
+ * recovery, conn_timer is running for reconnect_timeout and once it fires,
+ * the connect recovery is aborted and moved to ERROR
+ *
+ * If WAS_CONNECTED, post write in wrong state returns EAGAIN instead of EINVAL
+ *
+ * A complication is identifying when a connection is down for a receiving
+ * end of a traffic pattern.  The receiver may see no packets and can't tell
+ * a path down from an idle QP.  To address this periodic RDMA Write zero
+ * can be sent if no traffic has been sent or received for a while.  This
+ * situation is no worse that user space UD PSM as the job failure may
+ * occur over an out of band network to kill the job.  A gap is the client side
+ * of a connection desiring recovery, which requires the heatbeat to recognize.
+ *
+ * If the receiver happens to also be the server side of rv_sconn, we may
+ * get a REQ while in connected because the sender may get a QP timeout
+ * long before the receiver heartbeat notices.  We treat this as disconnect
+ * and if appropriate (likely) begin connection recovery.
+ *
+ * RV_INIT: initial state for client side connections (after 1st create_conn)
+ *	cm_id, primary_path are NULL
+ *	dev_addr, resolver_retry_left  uninitialized
+ *	qp in RESET state, no outstanding CQEs nor WQEs
+ *	conn_timer not running (no reconnect_timeout)
+ *	delay_timer not running
+ *	hb_timer not running
+ *	Next States:
+ *		RESOLVING - user calls cm_connect
+ *		destroying - user close
+ * RV_WAITING: initial state for server side connections (after 1st create_conn)
+ *	a listener exists at rv_job_dev level, rv_cm_server_handler
+ *	cm_id NULL
+ *	start_time is set when enter state
+ *	qp in RESET state, no outstanding CQEs nor WQEs
+ *	if WAS_CONNECTED, conn_timer is running for reconnect_timeout
+ *	delay_timer not running
+ *	hb_timer not running
+ *	DRAINING, SQ_DRAINED and RQ_DRAINED flags clear
+ *	Next States:
+ *		CONNECTING - inbound REQ
+ *		ERROR - reconnect_timeout expires
+ *		destroying - user close
+ * RV_RESOLVING: 1st step in establishing a client connection (ARP)
+ *	For non-ethernet, this is a brief transient state (only inside mutex)
+ *	cm_id established (client side), rv_cm_handler
+ *	resolver_retry_left valid
+ *	primary_path != NULL, but contents incomplete
+ *		dmac, route_resolved, hop__limit uninitialized
+ *	dev_addr undefined
+ *	start_time is set when enter state
+ *	a rdma_resolve_ip callback is scheduled
+ *	qp in RESET state, no outstanding CQEs nor WQEs
+ *	if WAS_CONNECTED, conn_timer is running for reconnect_timeout
+ *	delay_timer not running
+ *	hb_timer not running
+ *	DRAINING, SQ_DRAINED and RQ_DRAINED flags clear
+ *	Next States:
+ *		CONNECTING - resolution successfully complete
+ *		ERROR - resolving hard fail or retry exceeded or connect timeout
+ *		RESOLVING - cb error and < retry limit
+ *		DELAY - cb error > retrylimit and WAS_CONNECTED and reconnect
+ *		destroying - user close
+ * RV_CONNECTING: client or server connection in hands of IB CM
+ *	cm_id established (either side), rv_cm_handler
+ *	primary_path NULL
+ *	dev_addr valid on client side
+ *	resolver_retry_left undefined
+ *	start_time is set when enter state
+ *	CM may have a rep or rtu outstanding
+ *	qp progressing through RESET->INIT->RTR->RTS via IB CM states/events
+ *	if WAS_CONNECTED, conn_timer is running for reconnect_timeout
+ *	delay_timer not running
+ *	hb_timer not running
+ *	DRAINING, SQ_DRAINED and RQ_DRAINED flags clear
+ *	Next States:
+ *		CONNECTED - client gets REP, server gets RTU or Established
+ *		ERROR - REJ, REQ err, REP err, DREQ, QP event or connect timeout
+ *		DISCONNECTING - WAS_CONNECTED and REQ err, REJ, REP err, QP evt
+ *				or REQ
+ *		destroying - user close
+ * RV_CONNECTED: client or server connection usable
+ *	cm_id established (either side), rv_cm_handler
+ *	primary_path NULL
+ *	dev_addr valid on client side
+ *	resolver_retry_left undefined
+ *	no outbound CM operations outstanding
+ *	start_time is set when enter state
+ *	qp in RTS, has recv WQEs posted
+ *	qp may have outstanding CQEs and WQEs
+ *	conn_timer not running (no reconnect_timeout)
+ *	delay_timer not running
+ *	hb_timer may be running (client side only)
+ *	WAS_CONNECTED is set on entry
+ *	DRAINING, SQ_DRAINED and RQ_DRAINED flags clear
+ *	heartbeat timer running
+ *		record post SQ and RQ CQE count when start timer
+ *		when fires, if counts same, send 0 len RDMA Write
+ *		TBD - set larger than timeout*retry or simply track SQ so
+ *			don't overflow SQ
+ *		TBD - stagger heartbeat sends on different sconns
+ *	Next States:
+ *		ERROR - no reconnect and get DREQ, QP event, REQ
+ *		DISCONNECTING - reconnect and get DREQ, QP event (send DREQ),
+ *				get REQ (remote end reconnecting)
+ *		destroying - user close
+ * RV_DISCONNECTING: client or server connection cleanup in prep for reconnect
+ *	cm_id established (either side), rv_cm_handler
+ *	primary_path NULL
+ *	dev_addr valid on client side
+ *	resolver_retry_left undefined
+ *	DREP or DREQ outbound CM operations may be outstanding
+ *	start_time is set when enter state
+ *	qp in ERROR, SCONN_DRAINING flag set,
+ *		- waiting for "drain indicator" WQEs' CQEs on RQ and SQ
+ *		- drain indicator CQEs set SQ_DRAINED or RQ_DRAINED
+ *	qp may have other outstanding CQEs and WQEs
+ *	if WAS_CONNECTED, conn_timer may be is running for reconnect_timeout
+ *	delay_timer not running
+ *	hb_timer not running
+ *	Note: we do not depend on DREP and ignore DREQ Err or DREP Err
+ *		path to remote node may be down
+ *	Next States:
+ *		ERROR - unable to queue CM_id destroy or reset QP
+ *			or connect timeout
+ *		DELAY - SQ & RQ drain finish for client
+ *		WAITING - SQ & RQ drain finish for server (reset QP,
+ *				cm_id NULL and queued for destroy)
+ *		destroying - user close
+ * RV_DELAY: client reconnection delay
+ *	cm_id established, rv_cm_handler
+ *	primary_path NULL
+ *	dev_addr valid on client side
+ *	resolver_retry_left undefined
+ *	DREP or DREQ outbound CM operations may be outstanding on prior cm_id
+ *	start_time is set when enter state
+ *	qp in RESET
+ *	prior qp has no outstanding CQEs and WQEs
+ *	if WAS_CONNECTED, conn_timer may be is running for reconnect_timeout
+ *	delay_timer is running
+ *	hb_timer not running
+ *	DRAINING, SQ_DRAINED and RQ_DRAINED flags clear
+ *	while in this state all other errors are ignored
+ *	Next States:
+ *		RESOLVING - timer expires
+ *		ERROR - reconnect timeout
+ *		destroying - user close
+ * RV_ERROR: terminal state for lost/failed connection
+ *	may enter from any state
+ *	cm_id may be NULL or established with rv_cm_handler
+ *	CM may have an outstanding message
+ *		typically REJ, DREQ or DREP, but could be REQ, REP or RTU
+ *	primary_path NULL
+ *	dev_addr, resolver_retry_left undefined
+ *	no rdma_resolve_ip callback scheduled (cancel when enter RV_ERROR)
+ *	qp in ERROR, SCONN_DRAINING flag set or qp in RESET or no QP
+ *	SQ_DRAINED and RQ_DRAINED flags may be progressing on drain
+ *	qp may have outstanding CQEs and WQEs
+ *	conn_timer not running (no reconnect_timeout)
+ *	delay_timer not running
+ *	hb_timer not running
+ *	Next States:
+ *		destroying - user close
+ * destroying: rv_conn kref is 0
+ *	may enter from any state
+ *	on entry:
+ *		cm_id may be NULL or established with rv_cm_handler
+ *		a rdma_resolve_ip callback may be scheduled
+ *		qp may have outstanding CQEs and WQEs and callbacks active
+ *		conn_timer may be running (reconnect_timeout or reconnect delay)
+ *	all fields (including immitable) will be released via rv_sconn_deinit
+ *	no way out other than kfree of parent rv_conn
+ *	may prempt, does not require mutex (kref==0 protects races with cb)
+ *	Next States:
+ *		N/A - conn/sconn freed
+ *
+ * typical client side state sequence:
+ *	RV_INIT->RV_RESOLVING->RV_CONNECTING->RV_CONNECTED->destroying
+ * typical server side state sequence:
+ *	RV_WAITING->RV_CONNECTING->RV_CONNECTED->destroying
+ *
+ * typical client side recovery state sequence:
+ *	...->RV_CONNECTED->DISCONNECTING->DELAY->RV_RESOLVING->...
+ * typical server side recovery state sequence:
+ *	...->RV_CONNECTED->DISCONNECTING->WAITING->...
+ */
+
+/*
+ * rv_sconn.flags bit numbers
+ *	RV_SCONN_SERVER	- server vs client side (immutable)
+ *	RV_SCONN_WAS_CONNECTED - was RV_CONNECTED at least once
+ *	RV_SCONN_DRAINING - started draining in DISCONNECTING
+ *	RV_SCONN_SQ_DRAINED - SQ drained in DISCONNECTING
+ *	RV_SCONN_RQ_DRAINED - RQ drained in DISCONNECTING
+ *	RV_SCONN_ASYNC - got async event
+ */
+#define RV_SCONN_SERVER		0
+#define RV_SCONN_WAS_CONNECTED	1
+#define RV_SCONN_DRAINING	2
+#define RV_SCONN_SQ_DRAINED	3
+#define RV_SCONN_RQ_DRAINED	4
+
+/*
+ * rv_sconn.stats.cm_evt_cnt[] classify events > CM_APR_RECEIVED as unexpected.
+ * Beyond that are only SIDR events
+ */
+#define RV_CM_EVENT_MAX IB_CM_APR_RECEIVED
+#define RV_CM_EVENT_UNEXP ((enum ib_cm_event_type)(RV_CM_EVENT_MAX + 1))
+
+/*
+ * a single QP/connection
+ * mutex - protects state driven fields
+ *
+ * These are set once at create and can be read without lock
+ *	index - unique index for connection within rv_conn
+ *	qp, send_cq, recv_cq, max_send_wr (from ib_qp_cap)
+ *	cqe - for recv completions
+ *
+ * Fields below require the mutex and their validity depends on the current
+ * value of rv_sconn.state and the RV_SCONN_SERVER flag.
+ *
+ * Basic fields:
+ *	state, flags
+ *	start_time - when started waiting, resolving or connecting
+ *	cm_id - our actual connection
+ *	path - from PSM.  Consistency checked on listener, for client connect
+ *		we use path.dlid != 0 to identify if path has been initialized
+ *
+ * Client only fields:
+ *	resolver_retry_left, primary_path, dev_addr
+ *	For RoCE our resolver step fills in dev_addr with resolved RoCE HW
+ *	addresses (aka MAC address)
+ *
+ * Async QP Draining,
+ *	drain_lock -  protects these and enter;test of RC_SCONN_*DRAIN* flags
+ *	drain_lock, rdrain_cqe, sdrain_cqe, drain_work (for drain done handler)
+ *	drain_timer - for drain timeout
+ *	done_wr_list - most recent completed pend_wr's
+ *	done_wr_count - number of entries on list
+ *
+ * reconnect_timeout timer: conn_timer, timer_work
+ * RV_DELAY timer: delay_timer, delay_work
+ * Heartbeat: hb_timer, act_count (activity since last hb), hb_cqe, hb_work
+ *
+ * Stats:
+ *	all but atomic CQE stats protected by rv_sconn.mutex
+ * connection:
+ *	*_time values are in microseconds
+ *	max_* is the largest observed for all reconnects on this sconn
+ *	cm_evt_cnt - extra +1 to include EVENT_MAX and +1 for UNEXP
+ *	stats for each CM packet explicitly send (req, rep, rtu, dreq, drep)
+ *	initial connect: wait_time, resolve_time, connect_time
+ *		connect time does not include wait_time nor resolve_time
+ *		resolve - attempts, resolve_fail - unexpected local issues
+ *	connected_time - total time connected (after initial + after recovery)
+ *	conn_recovery - # times recovered connection
+ *	connect recovery: rewait_time, reresolve_time, reconnect_time
+ *		reconnect time does not include wait_time nor resolve_time
+ *		reresolve - attempts, reresolve_fail - unexpected local issues
+ * data movement:
+ *	post_* is SQ posts (success, fail, payload byte count)
+ *	outstand_send_write - current send writes waiting for CQE
+ *	send_write_cqe_* is SQ cqes (RDMA Write w/Immediate)
+ *	recv_write_cqe_* is RQ cqes (RDMA Write w/Immediate)
+ *	recv_cqe_fail - RQ CQEs with bad status (opcode undefined)
+ *	*_hb_* - heartbeat
+ */
+
+struct rv_sconn {
+	struct mutex mutex; /* lock for state driven fields */
+	u8 index;
+	struct ib_qp *qp;
+	struct ib_cq *send_cq;
+	struct ib_cq *recv_cq;
+	struct ib_cqe cqe;
+	struct rv_conn *parent;
+
+	unsigned long flags;
+	enum rv_sconn_state state;
+	ktime_t start_time;
+	struct ib_cm_id *cm_id;
+	struct ib_user_path_rec path;
+
+	u32 resolver_retry_left;
+	struct sa_path_rec *primary_path;
+	struct rdma_dev_addr dev_addr;
+
+	/* protects these & enter;test RC_SCONN_*DRAIN* flags */
+	spinlock_t drain_lock;
+	struct ib_cqe rdrain_cqe;
+	struct ib_cqe sdrain_cqe;
+	struct work_struct drain_work;
+	struct timer_list drain_timer;
+
+	struct timer_list conn_timer;
+	struct work_struct timer_work;
+
+	struct timer_list delay_timer;
+	struct work_struct delay_work;
+
+	struct timer_list hb_timer;
+	u64 act_count;
+	struct ib_cqe hb_cqe;
+	struct work_struct hb_work;
+
+	struct {
+		u32 cm_evt_cnt[RV_CM_EVENT_MAX + 2];
+		u32 req_sent;
+		u32 rep_sent;
+		u32 rtu_sent;
+		u32 rej_sent;
+		u32 dreq_sent;
+		u32 drep_sent;
+		u64 wait_time;
+		u64 resolve_time;
+		u64 connect_time;
+		u64 connected_time;
+		u32 resolve;
+		u32 resolve_fail;
+		u32 conn_recovery;
+		u64 rewait_time;
+		u64 reresolve_time;
+		u64 reconnect_time;
+		u64 max_rewait_time;
+		u64 max_reresolve_time;
+		u64 max_reconnect_time;
+		u32 reresolve;
+		u32 reresolve_fail;
+		u64 post_write;
+		u64 post_write_fail;
+		u64 post_write_bytes;
+		u64 post_hb;
+		u64 post_hb_fail;
+
+		atomic_t outstand_send_write;
+		atomic64_t send_write_cqe;
+		atomic64_t send_write_cqe_fail;
+		atomic64_t recv_write_cqe;
+		atomic64_t recv_write_bytes;
+		atomic64_t recv_cqe_fail;
+		atomic64_t send_hb_cqe;
+		atomic64_t send_hb_cqe_fail;
+		atomic64_t recv_hb_cqe;
+	} stats;
+};
+
+/*
+ * A load balanced multi QP connection using multiple underlying connections
+ * and is shared by multiple rv_user's.
+ * num_conn, jdev, ah and rem_addr are immutable (set once at create)
+ * Entry in rv_job_dev.conn_list: conn_entry, rcu
+ * sconn round robin IO: next_lock, next
+ *	next_lock also protects read;inc of sconn->stats.outstand_send_write
+ */
+struct rv_conn {
+	u8 num_conn;
+	struct rv_job_dev *jdev;
+	struct ib_uverbs_ah_attr ah;
+	u32 rem_addr;
+
+	struct list_head conn_entry;
+	struct rcu_head rcu;
+
+	struct kref kref;
+	struct work_struct put_work;
+
+	spinlock_t next_lock; /* protect rv_conn.next & read;inc outstand_wr */
+	u8 next;
+	struct work_struct free_work;
+
+	struct rv_sconn sconn_arr[];
+};
+
+/*
+ * from IBTA 1.4 Vol1 section A3.2.3.4.  Externally Admined Service IDs
+ * 1h:OUI:OUI:OUI:hh:hh:hh:hh
+ * using Intel 00:12:55 OUI
+ */
+#define RV_DFLT_SERVICE_ID 0x1000125500000001ULL
+
+/*
+ * the set of MRs registered by a single rv_user (on a single NIC)
+ * These are cached for efficiency.
+ * When using kernel rendezvous QPs (eg. rv_conn) these MRs are
+ * registered with the rv_job_dev.pd.
+ * When using user space QPs, we register with the user supplied pd
+ * cache has it's own lock
+ * jdev, rv_inx , CQs, QP set once at alloc
+ * need parent rv_user.mutex for: cqe, post REG_MR to QP (doit_reg_mem), stats
+ */
+struct rv_user_mrs {
+	struct rv_mr_cache cache;
+	struct rv_job_dev *jdev;
+	int rv_inx; /* our creator, for logging */
+
+	struct kref kref;
+	struct work_struct put_work;
+
+	struct {
+		u64 failed;	/* cache miss and failed to register */
+	} stats;
+};
+
+/*
+ * Resources shared among all rv_user's using a given NIC port within a job
+ * for a given kuid.  On the wire we use uid to limit jobs to a single user.
+ * This approach has similarities to the hfi1 and Omni-Path jkey concept
+ * but we are not restricted to the 16b HW jkey field here, so we use all
+ * 32b of uid on the wire.
+ *
+ * There will typically be 1-8 NICs per node and 1-2 concurrent jobs
+ * with one rv_user per CPU core per NIC.
+ *
+ * kref tracks total jdev references while user_kref is the subset of
+ * references representing attached rv_user objects. user_kref <= kref.
+ * To get_alloc a jdev we must successfully get both. Once user_kref
+ * is zero, the jdev is removed from rv_job_dev_list and future attach
+ * or inbound REQ processing will not find it anymore.  At this point it is
+ * destined for destruction as soon as the remaining callbacks with a reference
+ * occur/finish. jdev get_alloc searches which see (and skip) jdevs with
+ * user_kref==0 but kfef>0 only happen when a new job starts without a unique
+ * job key, as the previous job is cleaning up. Such cases are rare.
+ * By decoupling the jobs into different jdev's the new job may also have
+ * different attach and connection params.
+ *
+ * These fields are immutable and can be accessed without a lock:
+ *	kuid, uid, dev, pd, dev_name, port_num,
+ *	num_conn - number of rv_sconn per rv_conn
+ *	index_bits - number of high bits of RDMA immed data to hold rv index
+ *	loc_gid_index - SGID - only used on client rv_sconn
+ *	loc_addr - abstraction of address compared with rem_addr to select
+ *		client/server mode for each rv_sconn at conn_create time.
+ *	log_gid - SGID - to double check loc_gid_index still is same GID
+ *	service_id, q_depth
+ *	qp_depth - max send and recv WQEs to use (N/A space for drain)
+ *	reconnect_timeout (seconds), hb_interval (milliseconds),
+ *	sgid_attr - local NIC gid & address for use by resolver
+ *	max_users - 1<<index_bits
+ *
+ * job_dev_entry, rcu - entry on rv_job_dev_list
+ * conn_list - list of shared rv_conn, protected by RCU and conn_list_mutex
+ *	conn_list_mutex prevents duplicate add in get_alloc
+ * listener - created and shared when 1st server rv_conn is added
+ * user_array[max_users] - rv_users sharing this jdev. user_array_lock protects
+ *	rv_user.index - subscript to this array for given rv_user so
+ *		RDMA recv immediate data can simply index to find which
+ *		rv_user to deliver the completion to.
+ *	user_array_next - where to start next search for free slot
+ */
+struct rv_job_dev {
+	uid_t uid;
+	kuid_t kuid;
+	struct rv_device *dev;
+	struct ib_pd *pd;
+	char dev_name[RV_MAX_DEV_NAME_LEN];
+	u8 port_num;
+	u8 num_conn;
+	u8 index_bits;
+	u16 loc_gid_index;
+	u32 loc_addr;
+	u8 loc_gid[16];
+	u8 job_key[RV_MAX_JOB_KEY_LEN];
+	u8 job_key_len;
+	u64 service_id;
+	u32 q_depth;
+	u32 qp_depth;
+	u32 reconnect_timeout;
+	u32 hb_interval;
+	const struct ib_gid_attr *sgid_attr;
+	int max_users;
+
+	struct kref kref;
+	struct list_head job_dev_entry;
+	struct rcu_head rcu;
+
+	struct mutex conn_list_mutex; /* prevent duplicate add in get_alloc */
+	struct list_head conn_list;
+	struct rv_listener *listener;
+
+	spinlock_t user_array_lock;/* protects add/remove from user_array */
+	u32 user_array_next;
+	struct kref user_kref;
+	struct rv_user *user_array[];
+};
+
+/*
+ * rv_user represents a single open fd from a user
+ * In multi-rail a process may have multiple rv_user (separate open/close)
+ *
+ * mutex - prevents concurrent ATTACH ioctl and protects conn_list
+ *	also protects doit_reg vs self and vs doit_dreg races
+ * attached - set last during ATTACH after dev/jdev, rdma_mode and cq_entries
+ *	have been set.  We have no detach.
+ * inx - immutable ID assignd to rv_user strictly for logging
+ * rdma_mode - indicates USER (MRs only) or KERNEL (jdev, conns, etc) attach
+ *	For rdma_mode KERNEL these are also valid:
+ *		context, cq_entries
+ *		index - rv_user index within rv_job_dev.user_array[]
+ *		cqr - event ring to deliver send and recv RDMA completion
+ *			events to PSM (only if ATTACH.cq_entries!=0)
+ * rv_user_mrs  - MRs registered by this rv_user
+ * conn_xa - ID indexed list of rv_conn (entries assigned on create_conn)
+ *	ID 0 is reserved, PSM uses this value to indicate uninitialized rv intf
+ *	rv_user.mutex protects, so no need to use xas_lock.
+ * user_entry - entry in rv_device.user_list
+ * compl - completion for detach
+ */
+struct rv_user {
+	struct mutex mutex; /* single thread most actions for a user */
+
+	int inx;
+
+	u8 rdma_mode;
+	u8 attached;
+	u8 was_attached;
+	union {
+		struct rv_device *dev;
+		struct rv_job_dev *jdev;
+	};
+
+	u64 context;
+	u32 cq_entries;
+	u16 index;
+	struct rv_user_ring *cqr;
+	struct rv_user_mrs *umrs;
+
+	struct xarray conn_xa;
+
+	struct list_head user_entry;
+	struct completion compl;
+};
+
+/*
+ * an event ring for use by a single rv_user
+ * allows events to be efficiently passed from rv to PSM for PSM polling
+ * Immutable fields set on alloc:
+ *	rv_inx - our creator's rv_user.inx (for logging)
+ *	num_entries, page, order
+ * lock - protects kernel posting to ring and stats
+ * hdr - mmapped into PSM
+ * stats - index for each is rv_event_type excluding RV_TEST_EVENT
+ */
+struct rv_user_ring {
+	int rv_inx;
+	u32 num_entries;
+	unsigned long page;
+	unsigned int order;
+
+	spinlock_t lock; /* protects posting to ring and stats*/
+	struct rv_ring_header *hdr;
+	struct {
+		u64 cqe[2];
+		u64 cqe_fail[2];
+		u64 bytes[2];
+	} stats;
+};
+
+/*
+ * an inflight RDMA write on behalf of an rv_user
+ *
+ * user_index - rv_user index within rv_job_dev.user_array[]
+ * umrs - MR cache holding the local MR
+ * mrc, loc_addr - local MR (with lkey), source memory address for RDMA Write
+ * rkey, rem_addr - dest memory address for RDMA Write
+ * wr_id - for completion event to PSM (supplied by PSM)
+ * done_wr_entry - Entry in rv_sconn.done_wr_list
+ */
+struct rv_pend_write {
+	struct ib_cqe cqe;
+	u16 user_index;
+	struct rv_user_mrs *umrs;
+	struct rv_sconn *sconn;
+
+	struct rv_mr_cached *mrc;
+	u64 loc_addr;
+	u32 rkey;
+	u64 rem_addr;
+	size_t length;
+	u32 immed;
+	u64 wr_id;
+};
+
+extern unsigned int enable_user_mr;
+
+/* Prototypes */
+struct rv_device *rv_device_get_add_user(char *dev_name, struct rv_user *rv);
+void rv_device_get(struct rv_device *dev);
+void rv_device_put(struct rv_device *dev);
+int rv_device_del_user(struct rv_user *rv);
+
+void rv_listener_get(struct rv_listener *listener);
+void rv_listener_put(struct rv_listener *listener);
+struct rv_listener *rv_listener_get_alloc(struct rv_device *dev,
+					  u64 service_id,
+					  ib_cm_handler handler);
+
+int rv_file_init(void);
+void rv_file_uninit(void);
+void rv_queue_work(struct work_struct *work);
+void rv_queue_work2(struct work_struct *work);
+void rv_flush_work2(void);
+void rv_queue_work3(struct work_struct *work);
+
+int doit_reg_mem(struct rv_user *rv, unsigned long arg);
+int doit_dereg_mem(struct rv_user *rv, unsigned long arg);
+void rv_user_mrs_get(struct rv_user_mrs *umrs);
+void rv_user_mrs_put(struct rv_user_mrs *umrs);
+struct rv_user_mrs *rv_user_mrs_alloc(struct rv_user *rv, u32 cache_size);
+void rv_user_mrs_attach(struct rv_user *rv);
+int rv_drv_api_dereg_mem(struct mr_info *mr);
+
+int rv_drv_prepost_recv(struct rv_sconn *sconn);
+void rv_recv_done(struct ib_cq *cq, struct ib_wc *wc);
+void rv_report_cqe_error(struct ib_cq *cq, struct ib_wc *wc,
+			 struct rv_sconn *sconn, const char *opname);
+int doit_post_rdma_write(struct rv_user *rv, unsigned long arg);
+
+static inline struct rv_conn *user_conn_find(struct rv_user *rv, u64 handle)
+{
+	return xa_load(&rv->conn_xa, handle);
+}
+
+void rv_conn_put(struct rv_conn *conn);
+int rv_conn_get_check(struct rv_conn *conn);
+void rv_conn_get(struct rv_conn *conn);
+int doit_conn_create(struct rv_user *rv, unsigned long arg);
+int doit_conn_connect(struct rv_user *rv, unsigned long arg);
+int doit_conn_connected(struct rv_user *rv, unsigned long arg);
+int doit_conn_get_conn_count(struct rv_user *rv, unsigned long arg);
+int doit_conn_get_stats(struct rv_user *rv, unsigned long arg);
+int cmp_gid(const void *gid1, const void *gid2);
+
+void rv_job_dev_get(struct rv_job_dev *jdev);
+void rv_job_dev_put(struct rv_job_dev *jdev);
+
+static inline bool rv_job_dev_has_users(struct rv_job_dev *jdev)
+{
+	return kref_read(&jdev->user_kref);
+}
+
+static inline bool rv_jdev_protocol_roce(const struct rv_job_dev *jdev)
+{
+	return rdma_protocol_roce(jdev->dev->ib_dev, jdev->port_num);
+}
+
+struct rv_sconn *
+rv_find_sconn_from_req(struct ib_cm_id *id,
+		       const struct ib_cm_req_event_param *param,
+		       struct rv_req_priv_data *priv_data);
+
+void rv_detach_user(struct rv_user *rv);
+
+#endif /* __RV_H__ */
diff --git a/drivers/infiniband/ulp/rv/rv_mr_cache.h b/drivers/infiniband/ulp/rv/rv_mr_cache.h
new file mode 100644
index 000000000000..6d36ff11b5b6
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/rv_mr_cache.h
@@ -0,0 +1,152 @@
+/* SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause) */
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+
+#ifndef __RV_MR_CACHE_H__
+#define __RV_MR_CACHE_H__
+
+#include <linux/types.h>
+#include <linux/file.h>
+#include <linux/list.h>
+#include <linux/rculist.h>
+#include <linux/mmu_notifier.h>
+#include <linux/interval_tree_generic.h>
+
+#define MAX_RB_SIZE 256 /* This is MB */
+#define RV_RB_MAX_ACTIVE_WQ_ENTRIES 5
+
+/*
+ * The MR cache holds registered MRs and tracks reference counts for each.
+ * Entries with a refcount==0 may remain in the cache and on an lru_list.
+ * If the MMU notifier indicates pages would like to be freed, the entry
+ * will be removed from the cache if it's refcount==0.  Otherwise there
+ * are IOs in flight (app should not free memory for buffers with IOs in flight)
+ * and the MMU notifier is not allowed to free the pages.
+ * If a new cache entry is needed (cache miss), entries will be evicted, oldest
+ * to newest based on the lru_list, until there is space for the new entry.
+ *
+ * max_size - limit allowed for total_size in bytes, immutable
+ * ops_arg - owner context for all ops calls, immutable
+ * mn - MMU notifier
+ * lock - protects the RB-tree, lru_list, del_list, total_size, and stats
+ * root - an RB-tree with an interval based lookup
+ * total_size - current bytes in the cache
+ * ops - owner callbacks for major cache events
+ * mm - for MMU notifier
+ * lru_list - ordered list, most recently used to least recently used
+ * del_work, del_list, wq - handle deletes on a work queue
+ *
+ * Statistics:
+ *	max_cache_size - max bytes in the cache
+ *	count - Current number of MRs in the cache
+ *	max_count - Maximum of count
+ *	inuse - Current number of MRs with refcount > 0
+ *	max_inuse - Maximum of inuse
+ *	inuse_bytes - number of bytes with refcount > 0
+ *	max_inuse_bytes - of inuse_bytes
+ *	max_refcount - Maximum of refcount for any MR
+ *	hit - Cache hit
+ *	miss - Cache miss and added
+ *	full - Cache miss and can't add since full
+ *	evict - Removed due to lack of cache space
+ *	remove - Refcount==0 & remove by mmu notifier event
+ */
+struct rv_mr_cache {
+	u64 max_size;
+	void *ops_arg;
+	struct mmu_notifier mn;
+	spinlock_t lock; /* See above */
+	struct rb_root_cached root;
+	u64 total_size;
+	const struct rv_mr_cache_ops *ops;
+	struct mm_struct *mm;
+	struct list_head lru_list;
+	struct work_struct del_work;
+	struct list_head del_list;
+	struct workqueue_struct *wq;
+
+	struct {
+		u64 max_cache_size;
+		u32 count;
+		u32 max_count;
+		u32 inuse;
+		u32 max_inuse;
+		u64 inuse_bytes;
+		u64 max_inuse_bytes;
+		u32 max_refcount;
+		u64 hit;
+		u64 miss;
+		u64 full;
+		u64 evict;
+		u64 remove;
+	} stats;
+};
+
+/*
+ * basic info about an MR
+ * ib_pd - converted from user version
+ * fd - converted from user provided cmd_fd
+ */
+struct mr_info {
+	struct ib_mr *ib_mr;
+	struct ib_pd *ib_pd;
+	struct fd fd;
+};
+
+/* an MR entry in the MR cache RB-tree */
+struct rv_mr_cached {
+	struct mr_info mr;
+	u64 addr;
+	u64 len;
+	u32 access;
+	u64 __last;
+	atomic_t refcount;
+	struct rb_node node;
+	struct list_head list;
+};
+
+/* callbacks for each major cache event */
+struct rv_mr_cache_ops {
+	bool (*filter)(struct rv_mr_cached *mrc, u64 addr, u64 len, u32 acc);
+	void (*get)(struct rv_mr_cache *cache,
+		    void *ops_arg, struct rv_mr_cached *mrc);
+	int (*put)(struct rv_mr_cache *cache,
+		   void *ops_arg, struct rv_mr_cached *mrc);
+	int (*invalidate)(struct rv_mr_cache *cache,
+			  void *ops_arg, struct rv_mr_cached *mrc);
+	int (*evict)(struct rv_mr_cache *cache,
+		     void *ops_arg, struct rv_mr_cached *mrc,
+		     void *evict_arg, bool *stop);
+};
+
+void rv_mr_cache_update_stats_max(struct rv_mr_cache *cache,
+				  int refcount);
+
+int rv_mr_cache_insert(struct rv_mr_cache *cache, struct rv_mr_cached *mrc);
+
+void rv_mr_cache_evict(struct rv_mr_cache *cache, void *evict_arg);
+
+struct rv_mr_cached *rv_mr_cache_search_get(struct rv_mr_cache *cache,
+					    u64 addr, u64 len, u32 acc,
+					    bool update_hit);
+struct rv_mr_cached *rv_mr_cache_search_put(struct rv_mr_cache *cache,
+					    u64 addr, u64 len, u32 acc);
+void rv_mr_cache_put(struct rv_mr_cache *cache, struct rv_mr_cached *mrc);
+
+int rv_mr_cache_init(int rv_inx, struct rv_mr_cache *cache,
+		     const struct rv_mr_cache_ops *ops, void *priv,
+		     struct mm_struct *mm, u32 cache_size);
+void rv_mr_cache_deinit(int rv_inx, struct rv_mr_cache *cache);
+
+/*
+ * evict operation argument
+ * cleared - count evicted so far in bytes
+ * target - target count to evict in bytes
+ */
+struct evict_data {
+	u64 cleared;
+	u64 target;
+};
+
+#endif /* __RV_MR_CACHE_H__ */
-- 
2.18.1


^ permalink raw reply related	[flat|nested] 52+ messages in thread

* [PATCH RFC 3/9] RDMA/rv: Add the rv module
  2021-03-19 12:56 [PATCH RFC 0/9] A rendezvous module kaike.wan
  2021-03-19 12:56 ` [PATCH RFC 1/9] RDMA/rv: Public interferce for the RDMA Rendezvous module kaike.wan
  2021-03-19 12:56 ` [PATCH RFC 2/9] RDMA/rv: Add the internal header files kaike.wan
@ 2021-03-19 12:56 ` kaike.wan
  2021-03-19 12:56 ` [PATCH RFC 4/9] RDMA/rv: Add functions for memory region cache kaike.wan
                   ` (6 subsequent siblings)
  9 siblings, 0 replies; 52+ messages in thread
From: kaike.wan @ 2021-03-19 12:56 UTC (permalink / raw)
  To: dledford, jgg; +Cc: linux-rdma, todd.rimmer, Kaike Wan

From: Kaike Wan <kaike.wan@intel.com>

Add the rv module, the Makefile, and Kconfig file.

Also add the functions to manage IB devices.

Signed-off-by: Todd Rimmer <todd.rimmer@intel.com>
Signed-off-by: Kaike Wan <kaike.wan@intel.com>
---
 MAINTAINERS                           |   6 +
 drivers/infiniband/Kconfig            |   1 +
 drivers/infiniband/ulp/Makefile       |   1 +
 drivers/infiniband/ulp/rv/Kconfig     |  11 ++
 drivers/infiniband/ulp/rv/Makefile    |   9 +
 drivers/infiniband/ulp/rv/rv_main.c   | 266 ++++++++++++++++++++++++++
 drivers/infiniband/ulp/rv/trace.c     |   7 +
 drivers/infiniband/ulp/rv/trace.h     |   5 +
 drivers/infiniband/ulp/rv/trace_dev.h |  82 ++++++++
 9 files changed, 388 insertions(+)
 create mode 100644 drivers/infiniband/ulp/rv/Kconfig
 create mode 100644 drivers/infiniband/ulp/rv/Makefile
 create mode 100644 drivers/infiniband/ulp/rv/rv_main.c
 create mode 100644 drivers/infiniband/ulp/rv/trace.c
 create mode 100644 drivers/infiniband/ulp/rv/trace.h
 create mode 100644 drivers/infiniband/ulp/rv/trace_dev.h

diff --git a/MAINTAINERS b/MAINTAINERS
index d92f85ca831d..ba50affec9bc 100644
--- a/MAINTAINERS
+++ b/MAINTAINERS
@@ -15547,6 +15547,12 @@ L:	linux-rdma@vger.kernel.org
 S:	Maintained
 F:	drivers/infiniband/ulp/rtrs/
 
+RV DRIVER
+M:	Kaike Wan <kaike.wan@intel.com>
+L:	linux-rdma@vger.kernel.org
+S:	Supported
+F:	drivers/infiniband/ulp/rv
+
 RXRPC SOCKETS (AF_RXRPC)
 M:	David Howells <dhowells@redhat.com>
 L:	linux-afs@lists.infradead.org
diff --git a/drivers/infiniband/Kconfig b/drivers/infiniband/Kconfig
index 04a78d9f8fe3..5086164c836f 100644
--- a/drivers/infiniband/Kconfig
+++ b/drivers/infiniband/Kconfig
@@ -107,5 +107,6 @@ source "drivers/infiniband/ulp/isert/Kconfig"
 source "drivers/infiniband/ulp/rtrs/Kconfig"
 
 source "drivers/infiniband/ulp/opa_vnic/Kconfig"
+source "drivers/infiniband/ulp/rv/Kconfig"
 
 endif # INFINIBAND
diff --git a/drivers/infiniband/ulp/Makefile b/drivers/infiniband/ulp/Makefile
index 4d0004b58377..f925deb9241c 100644
--- a/drivers/infiniband/ulp/Makefile
+++ b/drivers/infiniband/ulp/Makefile
@@ -6,3 +6,4 @@ obj-$(CONFIG_INFINIBAND_ISER)		+= iser/
 obj-$(CONFIG_INFINIBAND_ISERT)		+= isert/
 obj-$(CONFIG_INFINIBAND_OPA_VNIC)	+= opa_vnic/
 obj-$(CONFIG_INFINIBAND_RTRS)		+= rtrs/
+obj-$(CONFIG_INFINIBAND_RV)		+= rv/
diff --git a/drivers/infiniband/ulp/rv/Kconfig b/drivers/infiniband/ulp/rv/Kconfig
new file mode 100644
index 000000000000..32a0523ff8ce
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/Kconfig
@@ -0,0 +1,11 @@
+# SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause)
+#
+# Copyright(c) 2020 - 2021 Intel Corporation.
+#
+config INFINIBAND_RV
+	tristate "InfiniBand Rendezvous Module"
+	depends on X86_64 && INFINIBAND
+	help
+	  The rendezvous module provides mechanisms for HPC middlewares
+	  to cache memory region registration, to manage connections
+	  between nodes, and improve the scability of RDMA transactions.
diff --git a/drivers/infiniband/ulp/rv/Makefile b/drivers/infiniband/ulp/rv/Makefile
new file mode 100644
index 000000000000..07a7a7dd9c3b
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/Makefile
@@ -0,0 +1,9 @@
+# SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause)
+#
+# Copyright(c) 2020 - 2021 Intel Corporation.
+#
+obj-$(CONFIG_INFINIBAND_RV) += rv.o
+
+rv-y := rv_main.o trace.o
+
+CFLAGS_trace.o = -I$(src)
diff --git a/drivers/infiniband/ulp/rv/rv_main.c b/drivers/infiniband/ulp/rv/rv_main.c
new file mode 100644
index 000000000000..7f81f97a01f0
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/rv_main.c
@@ -0,0 +1,266 @@
+// SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause)
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+
+/* This file contains the base of the rendezvous RDMA driver */
+
+#include <linux/module.h>
+#include <linux/init.h>
+#include <linux/err.h>
+#include <linux/parser.h>
+
+#include <rdma/ib_user_sa.h>
+
+#include "rv.h"
+#include "trace.h"
+
+MODULE_AUTHOR("Kaike Wan");
+MODULE_DESCRIPTION("Rendezvous Module");
+MODULE_LICENSE("Dual BSD/GPL");
+
+static int rv_add_one(struct ib_device *device);
+static void rv_remove_one(struct ib_device *device, void *client_data);
+static void rv_rename_dev(struct ib_device *device, void *client_data);
+
+static struct ib_client rv_client = {
+	.name = "rv",
+	.add = rv_add_one,
+	.remove = rv_remove_one,
+	.rename = rv_rename_dev
+};
+
+static struct list_head rv_dev_list;	/* list of rv_device */
+static spinlock_t rv_dev_list_lock;
+
+/* get a device reference and add an rv_user to rv_device.user_list */
+struct rv_device *rv_device_get_add_user(char *dev_name, struct rv_user *rv)
+{
+	struct rv_device *dev;
+	unsigned long flags;
+
+	spin_lock_irqsave(&rv_dev_list_lock, flags);
+	list_for_each_entry(dev, &rv_dev_list, dev_entry) {
+		if (strcmp(dev->ib_dev->name, dev_name) == 0) {
+			if (!kref_get_unless_zero(&dev->kref))
+				continue; /* skip, going away */
+			list_add_tail(&rv->user_entry, &dev->user_list);
+			spin_unlock_irqrestore(&rv_dev_list_lock, flags);
+			trace_rv_dev_get(dev_name, kref_read(&dev->kref));
+			return dev;
+		}
+	}
+	spin_unlock_irqrestore(&rv_dev_list_lock, flags);
+	rv_err(RV_INVALID, "Could not find IB dev %s\n", dev_name);
+	return NULL;
+}
+
+static void rv_device_release(struct kref *kref)
+{
+	struct rv_device *dev = container_of(kref, struct rv_device, kref);
+
+	ib_unregister_event_handler(&dev->event_handler); /* may need sooner */
+	kfree(dev);
+}
+
+void rv_device_get(struct rv_device *dev)
+{
+	kref_get(&dev->kref);
+}
+
+void rv_device_put(struct rv_device *dev)
+{
+	trace_rv_dev_put(dev->ib_dev ? dev->ib_dev->name : "nil",
+			 kref_read(&dev->kref));
+	kref_put(&dev->kref, rv_device_release);
+}
+
+/*
+ * Remove a rv_user from rv_device.user_list
+ *
+ * @rv - The rv_user to remove
+ *
+ * Return:
+ *   0 - The rv_user is in rv_device.user_list and removed;
+ *   1 - The rv_user is already not in rv_device.user_list.
+ */
+int rv_device_del_user(struct rv_user *rv)
+{
+	unsigned long flags;
+	int ret = 0;
+
+	spin_lock_irqsave(&rv_dev_list_lock, flags);
+	if (list_empty(&rv->user_entry))
+		ret = 1;
+	else
+		list_del_init(&rv->user_entry);
+	spin_unlock_irqrestore(&rv_dev_list_lock, flags);
+
+	return ret;
+}
+
+/* verbs device level async events */
+static void rv_device_event_handler(struct ib_event_handler *handler,
+				    struct ib_event *event)
+{
+	struct rv_device *dev;
+
+	dev = ib_get_client_data(event->device, &rv_client);
+	if (!dev || dev->ib_dev != event->device)
+		return;
+
+	trace_rv_device_event(dev->ib_dev->name, ib_event_msg(event->event));
+	switch (event->event) {
+	case IB_EVENT_DEVICE_FATAL:
+	case IB_EVENT_PORT_ERR:
+	case IB_EVENT_PORT_ACTIVE:
+	case IB_EVENT_LID_CHANGE:
+	case IB_EVENT_PKEY_CHANGE:
+	case IB_EVENT_SM_CHANGE:
+	case IB_EVENT_CLIENT_REREGISTER:
+	case IB_EVENT_GID_CHANGE:
+	default:
+		break;
+	}
+}
+
+static int rv_add_one(struct ib_device *device)
+{
+	struct rv_device *dev;
+	unsigned long flags;
+
+	dev = kzalloc(sizeof(*dev), GFP_KERNEL);
+	if (!dev)
+		return -ENOMEM;
+	dev->ib_dev = device;
+	kref_init(&dev->kref);
+	mutex_init(&dev->listener_mutex);
+	spin_lock_init(&dev->listener_lock);
+	INIT_LIST_HEAD(&dev->listener_list);
+	INIT_LIST_HEAD(&dev->user_list);
+	spin_lock_irqsave(&rv_dev_list_lock, flags);
+	list_add(&dev->dev_entry, &rv_dev_list);
+	spin_unlock_irqrestore(&rv_dev_list_lock, flags);
+	trace_rv_dev_add(device->name, kref_read(&dev->kref));
+	ib_set_client_data(device, &rv_client, dev);
+
+	INIT_IB_EVENT_HANDLER(&dev->event_handler, device,
+			      rv_device_event_handler);
+	ib_register_event_handler(&dev->event_handler);
+
+	return 0;
+}
+
+/*
+ * Called on device removal, gets users off the device
+ *
+ * At the same time, applications will get device async events which should
+ * trigger them to start user space cleanup and close.
+ *
+ * We remove the rv_user from the user_list so that the user application knows
+ * that the remove_one handler is cleaning up this rv_user. After this,
+ * the rv->user_entry itself is an empty list, an indicator that the
+ * remove_one handler owns this rv_user.
+ *
+ * To comply with lock heirarchy, we must release rv_dev_list_lock so
+ * rv_detach_user can get rv->mutex.  The empty rv->user_entry will prevent
+ * a race with rv_user starting its own detach.
+ */
+static void rv_device_detach_users(struct rv_device *dev)
+{
+	unsigned long flags;
+	struct rv_user *rv;
+
+	spin_lock_irqsave(&rv_dev_list_lock, flags);
+	while (!list_empty(&dev->user_list)) {
+		rv = list_first_entry(&dev->user_list, struct rv_user,
+				      user_entry);
+		list_del_init(&rv->user_entry);
+
+		spin_unlock_irqrestore(&rv_dev_list_lock, flags);
+		/* Detach user here */
+		spin_lock_irqsave(&rv_dev_list_lock, flags);
+	}
+	spin_unlock_irqrestore(&rv_dev_list_lock, flags);
+}
+
+/*
+ * device removal handler
+ *
+ * we allow a wait_time of 2 seconds for applications to cleanup themselves
+ * and close.  Typically they will get an async event and react quickly.
+ * After which we begin forcibly removing the remaining users and
+ * then wait for the internal references to get releaseed by their callbacks
+ */
+static void rv_remove_one(struct ib_device *device, void *client_data)
+{
+	struct rv_device *dev = client_data;
+	unsigned long flags;
+	unsigned long wait_time = 2000; /* 2 seconds */
+	unsigned long sleep_time = msecs_to_jiffies(100);
+	unsigned long end;
+
+	trace_rv_dev_remove(device->name, kref_read(&dev->kref));
+	spin_lock_irqsave(&rv_dev_list_lock, flags);
+	list_del(&dev->dev_entry);
+	spin_unlock_irqrestore(&rv_dev_list_lock, flags);
+
+	end = jiffies + msecs_to_jiffies(wait_time);
+	while (time_before(jiffies, end) && !list_empty(&dev->user_list))
+		schedule_timeout_interruptible(sleep_time);
+
+	rv_device_detach_users(dev);
+
+	while (kref_read(&dev->kref) > 1)
+		schedule_timeout_interruptible(sleep_time);
+
+	rv_device_put(dev);
+}
+
+static void rv_rename_dev(struct ib_device *device, void *client_data)
+{
+}
+
+static void rv_init_devices(void)
+{
+	spin_lock_init(&rv_dev_list_lock);
+	INIT_LIST_HEAD(&rv_dev_list);
+}
+
+/* uses syncrhnoize_rcu to ensure previous kfree_rcu of references are done */
+static void rv_deinit_devices(void)
+{
+	struct rv_device *dev, *temp;
+	unsigned long flags;
+
+	synchronize_rcu();
+	spin_lock_irqsave(&rv_dev_list_lock, flags);
+	list_for_each_entry_safe(dev, temp, &rv_dev_list, dev_entry) {
+		list_del(&dev->dev_entry);
+		rv_device_put(dev);
+	}
+	spin_unlock_irqrestore(&rv_dev_list_lock, flags);
+}
+
+static int __init rv_init_module(void)
+{
+	pr_info("Loading rendezvous module");
+
+	rv_init_devices();
+
+	if (ib_register_client(&rv_client)) {
+		rv_err(RV_INVALID, "Failed to register with the IB core\n");
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+static void __exit rv_cleanup_module(void)
+{
+	ib_unregister_client(&rv_client);
+	rv_deinit_devices();
+}
+
+module_init(rv_init_module);
+module_exit(rv_cleanup_module);
diff --git a/drivers/infiniband/ulp/rv/trace.c b/drivers/infiniband/ulp/rv/trace.c
new file mode 100644
index 000000000000..b27536056c60
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/trace.c
@@ -0,0 +1,7 @@
+// SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause)
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+#define CREATE_TRACE_POINTS
+#include <rdma/rv_user_ioctls.h>
+#include "trace.h"
diff --git a/drivers/infiniband/ulp/rv/trace.h b/drivers/infiniband/ulp/rv/trace.h
new file mode 100644
index 000000000000..cb1d1d087e16
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/trace.h
@@ -0,0 +1,5 @@
+/* SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause) */
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+#include "trace_dev.h"
diff --git a/drivers/infiniband/ulp/rv/trace_dev.h b/drivers/infiniband/ulp/rv/trace_dev.h
new file mode 100644
index 000000000000..2bfc6b07d518
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/trace_dev.h
@@ -0,0 +1,82 @@
+/* SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause) */
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+#if !defined(__RV_TRACE_DEV_H) || defined(TRACE_HEADER_MULTI_READ)
+#define __RV_TRACE_DEV_H
+
+#include <linux/tracepoint.h>
+#include <linux/trace_seq.h>
+
+#undef TRACE_SYSTEM
+#define TRACE_SYSTEM rv_dev
+
+DECLARE_EVENT_CLASS(/* dev */
+	rv_dev_template,
+	TP_PROTO(const char *dev_name, u32 refcount),
+	TP_ARGS(dev_name, refcount),
+	TP_STRUCT__entry(/* entry */
+		__string(name, dev_name)
+		__field(u32, refcount)
+	),
+	TP_fast_assign(/* assign */
+		__assign_str(name, dev_name);
+		__entry->refcount = refcount;
+	),
+	TP_printk(/* print */
+		"name %s, refcount %u",
+		__get_str(name),
+		__entry->refcount
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_dev_template, rv_dev_add,
+	TP_PROTO(const char *dev_name, u32 refcount),
+	TP_ARGS(dev_name, refcount)
+);
+
+DEFINE_EVENT(/* event */
+	rv_dev_template, rv_dev_remove,
+	TP_PROTO(const char *dev_name, u32 refcount),
+	TP_ARGS(dev_name, refcount)
+);
+
+DEFINE_EVENT(/* event */
+	rv_dev_template, rv_dev_get,
+	TP_PROTO(const char *dev_name, u32 refcount),
+	TP_ARGS(dev_name, refcount)
+);
+
+DEFINE_EVENT(/* event */
+	rv_dev_template, rv_dev_put,
+	TP_PROTO(const char *dev_name, u32 refcount),
+	TP_ARGS(dev_name, refcount)
+);
+
+TRACE_EVENT(/* event */
+	rv_device_event,
+	TP_PROTO(const char *dev_name, const char *evt_name),
+	TP_ARGS(dev_name, evt_name),
+	TP_STRUCT__entry(/* entry */
+		__string(device, dev_name)
+		__string(event, evt_name)
+	),
+	TP_fast_assign(/* assign */
+		__assign_str(device, dev_name);
+		__assign_str(event, evt_name);
+	),
+	TP_printk(/* print */
+		"Device %s Event %s",
+		__get_str(device),
+		__get_str(event)
+	)
+);
+
+#endif /* __RV_TRACE_DEV_H */
+
+#undef TRACE_INCLUDE_PATH
+#undef TRACE_INCLUDE_FILE
+#define TRACE_INCLUDE_PATH .
+#define TRACE_INCLUDE_FILE trace_dev
+#include <trace/define_trace.h>
-- 
2.18.1


^ permalink raw reply related	[flat|nested] 52+ messages in thread

* [PATCH RFC 4/9] RDMA/rv: Add functions for memory region cache
  2021-03-19 12:56 [PATCH RFC 0/9] A rendezvous module kaike.wan
                   ` (2 preceding siblings ...)
  2021-03-19 12:56 ` [PATCH RFC 3/9] RDMA/rv: Add the rv module kaike.wan
@ 2021-03-19 12:56 ` kaike.wan
  2021-03-19 12:56 ` [PATCH RFC 5/9] RDMA/rv: Add function to register/deregister memory region kaike.wan
                   ` (5 subsequent siblings)
  9 siblings, 0 replies; 52+ messages in thread
From: kaike.wan @ 2021-03-19 12:56 UTC (permalink / raw)
  To: dledford, jgg; +Cc: linux-rdma, todd.rimmer, Kaike Wan

From: Kaike Wan <kaike.wan@intel.com>

The MR cache is implemented through an rb tree. Each node is indexed
by a simple (address, length, access_flags) tuple, without any
consideration of buffer overlapping. When a node's refcount goes
down to 0, it is not removed from the cache. Instead, it is put into
an LRU list that could be evicted if the cache memory limit is reached.
However, if the user buffer for the memory region is freed, the node
will be removed when the MMU notice is received.

Signed-off-by: Todd Rimmer <todd.rimmer@intel.com>
Signed-off-by: Kaike Wan <kaike.wan@intel.com>
---
 drivers/infiniband/ulp/rv/Makefile         |   2 +-
 drivers/infiniband/ulp/rv/rv_mr_cache.c    | 428 +++++++++++++++++++++
 drivers/infiniband/ulp/rv/trace.h          |   1 +
 drivers/infiniband/ulp/rv/trace_mr_cache.h | 181 +++++++++
 4 files changed, 611 insertions(+), 1 deletion(-)
 create mode 100644 drivers/infiniband/ulp/rv/rv_mr_cache.c
 create mode 100644 drivers/infiniband/ulp/rv/trace_mr_cache.h

diff --git a/drivers/infiniband/ulp/rv/Makefile b/drivers/infiniband/ulp/rv/Makefile
index 07a7a7dd9c3b..01e93dc25f1d 100644
--- a/drivers/infiniband/ulp/rv/Makefile
+++ b/drivers/infiniband/ulp/rv/Makefile
@@ -4,6 +4,6 @@
 #
 obj-$(CONFIG_INFINIBAND_RV) += rv.o
 
-rv-y := rv_main.o trace.o
+rv-y := rv_main.o rv_mr_cache.o trace.o
 
 CFLAGS_trace.o = -I$(src)
diff --git a/drivers/infiniband/ulp/rv/rv_mr_cache.c b/drivers/infiniband/ulp/rv/rv_mr_cache.c
new file mode 100644
index 000000000000..48ea7c958f74
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/rv_mr_cache.c
@@ -0,0 +1,428 @@
+// SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause)
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+
+#include <linux/types.h>
+#include <linux/list.h>
+#include <linux/mutex.h>
+#include <linux/scatterlist.h>
+#include <linux/debugfs.h>
+#include <linux/interval_tree_generic.h>
+
+#include <rdma/ib_user_sa.h>
+
+#include "rv.h"
+#include "trace.h"
+
+static unsigned int mr_cache_size = MAX_RB_SIZE;
+
+module_param(mr_cache_size, uint, 0444);
+MODULE_PARM_DESC(mr_cache_size, "Size of mr cache (in MB)");
+
+static void handle_remove(struct work_struct *work);
+static void do_remove(struct rv_mr_cache *cache, struct list_head *del_list);
+static u32 rv_cache_evict(struct rv_mr_cache *cache, u64 mbytes);
+static int mmu_notifier_range_start(struct mmu_notifier *,
+				    const struct mmu_notifier_range *);
+static struct rv_mr_cached *rv_mr_cache_search(struct rv_mr_cache *cache,
+					       u64 addr, u64 len, u32 acc);
+static void rv_update_mrc_stats_add(struct rv_mr_cache *cache,
+				    struct rv_mr_cached *mrc);
+static void rv_update_mrc_stats_remove(struct rv_mr_cache *cache,
+				       struct rv_mr_cached *mrc);
+
+static const struct mmu_notifier_ops mn_opts = {
+	.invalidate_range_start = mmu_notifier_range_start,
+};
+
+static u64 mrc_start(struct rv_mr_cached *mrc)
+{
+	return mrc->addr;
+}
+
+static u64 mrc_last(struct rv_mr_cached *mrc)
+{
+	return mrc->addr + mrc->len - 1;
+}
+
+INTERVAL_TREE_DEFINE(struct rv_mr_cached, node, u64, __last,
+		     mrc_start, mrc_last, static, rv_int_rb);
+
+/*
+ * MMU notifier callback
+ *
+ * If the address range overlaps an MR which is in use (refcount>0)
+ * we refuse to remove it.  Otherwise we remove it from the MR cache
+ * by getting it off the LRU list and RB-tree and schedule the
+ * MR deregistration.
+ */
+static int mmu_notifier_range_start(struct mmu_notifier *mn,
+				    const struct mmu_notifier_range *range)
+{
+	struct rv_mr_cache *cache = container_of(mn, struct rv_mr_cache, mn);
+	struct rb_root_cached *root = &cache->root;
+	struct rv_mr_cached *mrc, *ptr = NULL;
+	unsigned long flags;
+	bool added = false;
+
+	spin_lock_irqsave(&cache->lock, flags);
+	for (mrc = rv_int_rb_iter_first(root, range->start, range->end - 1);
+	     mrc; mrc = ptr) {
+		ptr = rv_int_rb_iter_next(mrc, range->start, range->end - 1);
+		if (cache->ops->invalidate(cache, cache->ops_arg, mrc)) {
+			trace_rv_mr_cache_notifier(mrc->addr, mrc->len,
+						   mrc->access);
+			rv_int_rb_remove(mrc, root);
+			list_move(&mrc->list, &cache->del_list);
+			cache->stats.remove++;
+			rv_update_mrc_stats_remove(cache, mrc);
+			added = true;
+		}
+	}
+	spin_unlock_irqrestore(&cache->lock, flags);
+
+	if (added)
+		queue_work(cache->wq, &cache->del_work);
+	return 0;
+}
+
+/* MR deregistration is done on a per rv_user work queue.  */
+int rv_mr_cache_init(int rv_inx, struct rv_mr_cache *cache,
+		     const struct rv_mr_cache_ops *ops, void *priv,
+		     struct mm_struct *mm, u32 cache_size)
+{
+	char wq_name[25];
+	int ret = 0;
+
+	sprintf(wq_name, "rv-%d\n", rv_inx);
+	cache->wq = alloc_workqueue(wq_name,
+				    WQ_SYSFS | WQ_HIGHPRI | WQ_CPU_INTENSIVE
+					| WQ_MEM_RECLAIM,
+				    RV_RB_MAX_ACTIVE_WQ_ENTRIES);
+	if (!cache->wq)
+		return -ENOMEM;
+
+	trace_rv_mr_cache_wq_alloc(wq_name);
+	cache->root = RB_ROOT_CACHED;
+	cache->ops = ops;
+	cache->ops_arg = priv;
+
+	INIT_HLIST_NODE(&cache->mn.hlist);
+	spin_lock_init(&cache->lock);
+
+	cache->mn.ops = &mn_opts;
+	cache->mm = mm;
+
+	INIT_WORK(&cache->del_work, handle_remove);
+	INIT_LIST_HEAD(&cache->del_list);
+	INIT_LIST_HEAD(&cache->lru_list);
+
+	if (cache_size)
+		cache->max_size = (u64)cache_size * 1024 * 1024;
+	else
+		cache->max_size = (u64)mr_cache_size * 1024 * 1024;
+
+	if (mm) {
+		ret = mmu_notifier_register(&cache->mn, cache->mm);
+		if (ret)
+			goto bail_wq;
+	}
+
+	return ret;
+
+bail_wq:
+	destroy_workqueue(cache->wq);
+	cache->wq = NULL;
+	return ret;
+}
+
+/* All remaining entries in the cache are deregistered */
+void rv_mr_cache_deinit(int rv_inx, struct rv_mr_cache *cache)
+{
+	struct rv_mr_cached *mrc;
+	struct rb_node *node;
+	unsigned long flags;
+	struct list_head del_list;
+
+	if (cache->mm)
+		mmu_notifier_unregister(&cache->mn, cache->mm);
+
+	INIT_LIST_HEAD(&del_list);
+
+	spin_lock_irqsave(&cache->lock, flags);
+	while ((node = rb_first_cached(&cache->root))) {
+		mrc = rb_entry(node, struct rv_mr_cached, node);
+		trace_rv_mr_cache_deinit(mrc->addr, mrc->len, mrc->access,
+					 atomic_read(&mrc->refcount));
+		rb_erase_cached(node, &cache->root);
+		list_move(&mrc->list, &del_list);
+		cache->stats.remove++;
+		rv_update_mrc_stats_remove(cache, mrc);
+	}
+	WARN_ON(cache->total_size);
+
+	spin_unlock_irqrestore(&cache->lock, flags);
+
+	do_remove(cache, &del_list);
+
+	if (cache->wq) {
+		char wq_name[25];
+
+		sprintf(wq_name, "rv-%d\n", rv_inx);
+		trace_rv_mr_cache_wq_destroy(wq_name);
+		flush_workqueue(cache->wq);
+		destroy_workqueue(cache->wq);
+	}
+	cache->wq = NULL;
+	cache->mm = NULL;
+}
+
+/* called with cache->lock */
+void rv_mr_cache_update_stats_max(struct rv_mr_cache *cache, int refcount)
+{
+	if ((u32)refcount > cache->stats.max_refcount)
+		cache->stats.max_refcount = (u32)refcount;
+	if (cache->stats.inuse > cache->stats.max_inuse)
+		cache->stats.max_inuse = cache->stats.inuse;
+	if (cache->stats.inuse_bytes > cache->stats.max_inuse_bytes)
+		cache->stats.max_inuse_bytes = cache->stats.inuse_bytes;
+	if (cache->stats.count > cache->stats.max_count)
+		cache->stats.max_count = cache->stats.count;
+	if (cache->total_size > cache->stats.max_cache_size)
+		cache->stats.max_cache_size = cache->total_size;
+}
+
+/* gets a reference to mrc on behalf of caller */
+int rv_mr_cache_insert(struct rv_mr_cache *cache,
+		       struct rv_mr_cached *mrc)
+{
+	struct rv_mr_cached *existing;
+	unsigned long flags;
+	u64 new_len, evict_len;
+	int ret = 0;
+
+again:
+	trace_rv_mr_cache_insert(mrc->addr, mrc->len, mrc->access);
+
+	spin_lock_irqsave(&cache->lock, flags);
+	existing = rv_mr_cache_search(cache, mrc->addr, mrc->len, mrc->access);
+	if (existing) {
+		ret = -EINVAL;
+		goto unlock;
+	}
+	new_len = cache->total_size + mrc->len;
+	if (new_len > cache->max_size) {
+		spin_unlock_irqrestore(&cache->lock, flags);
+
+		trace_rv_mr_cache_cache_full(cache->max_size, cache->total_size,
+					     mrc->len);
+
+		evict_len = new_len - cache->max_size;
+		if (rv_cache_evict(cache, evict_len) >= evict_len)
+			goto again;
+		spin_lock_irqsave(&cache->lock, flags);
+		cache->stats.full++;
+		spin_unlock_irqrestore(&cache->lock, flags);
+		return -ENOMEM;
+	}
+
+	rv_int_rb_insert(mrc, &cache->root);
+	INIT_LIST_HEAD(&mrc->list);
+
+	cache->ops->get(cache, cache->ops_arg, mrc);
+	cache->stats.miss++;
+	rv_update_mrc_stats_add(cache, mrc);
+unlock:
+	spin_unlock_irqrestore(&cache->lock, flags);
+	return ret;
+}
+
+/* Caller must hold cache->lock */
+static struct rv_mr_cached *rv_mr_cache_search(struct rv_mr_cache *cache,
+					       u64 addr, u64 len, u32 acc)
+{
+	struct rv_mr_cached *mrc = NULL;
+
+	trace_rv_mr_cache_search_enter(addr, len, acc);
+
+	if (!cache->ops->filter) {
+		mrc = rv_int_rb_iter_first(&cache->root, addr,
+					   (addr + len) - 1);
+		if (mrc)
+			trace_rv_mr_cache_search_mrc(mrc->addr, mrc->len,
+						     mrc->access);
+	} else {
+		for (mrc = rv_int_rb_iter_first(&cache->root, addr,
+						(addr + len) - 1);
+		     mrc;
+		     mrc = rv_int_rb_iter_next(mrc, addr, (addr + len) - 1)) {
+			trace_rv_mr_cache_search_mrc(mrc->addr, mrc->len,
+						     mrc->access);
+			if (cache->ops->filter(mrc, addr, len, acc))
+				return mrc;
+		}
+	}
+	return mrc;
+}
+
+/* look for a cache hit.  If get a hit, make sure removed from LRU list */
+struct rv_mr_cached *rv_mr_cache_search_get(struct rv_mr_cache *cache,
+					    u64 addr, u64 len, u32 acc,
+					    bool update_hit)
+{
+	unsigned long flags;
+	struct rv_mr_cached *mrc;
+
+	spin_lock_irqsave(&cache->lock, flags);
+	mrc =  rv_mr_cache_search(cache, addr, len, acc);
+	if (mrc) {
+		cache->ops->get(cache, cache->ops_arg, mrc);
+		if (update_hit)
+			cache->stats.hit++;
+		list_del_init(&mrc->list);
+	}
+	spin_unlock_irqrestore(&cache->lock, flags);
+
+	return mrc;
+}
+
+/*
+ * release a cache reference by address.
+ * This is called from user ioctl, so we must make sure they don't
+ * dereg twice yielding a negative refcount.
+ * The released entry goes on our LRU list to prioritize evictions.
+ */
+struct rv_mr_cached *rv_mr_cache_search_put(struct rv_mr_cache *cache,
+					    u64 addr, u64 len, u32 acc)
+{
+	unsigned long flags;
+	struct rv_mr_cached *mrc;
+
+	spin_lock_irqsave(&cache->lock, flags);
+	mrc =  rv_mr_cache_search(cache, addr, len, acc);
+	if (mrc) {
+		if (!atomic_read(&mrc->refcount)) {
+			mrc = NULL;
+			goto unlock;
+		}
+		if (!cache->ops->put(cache, cache->ops_arg, mrc))
+			list_add(&mrc->list, &cache->lru_list);
+	}
+unlock:
+	spin_unlock_irqrestore(&cache->lock, flags);
+
+	return mrc;
+}
+
+/* Simple release, the entry goes on our LRU list to prioritize evictions. */
+void rv_mr_cache_put(struct rv_mr_cache *cache, struct rv_mr_cached *mrc)
+{
+	unsigned long flags;
+
+	spin_lock_irqsave(&cache->lock, flags);
+	if (!cache->ops->put(cache, cache->ops_arg, mrc))
+		list_add(&mrc->list, &cache->lru_list);
+	spin_unlock_irqrestore(&cache->lock, flags);
+}
+
+/*
+ * evict entries from the cache, least recently used first.
+ * We evict until we reach the goal or LRU_list is empty. Evicted
+ * entries are removed from the cache and deregistered.
+ */
+void rv_mr_cache_evict(struct rv_mr_cache *cache, void *evict_arg)
+{
+	struct rv_mr_cached *mrc, *temp;
+	struct list_head del_list;
+	unsigned long flags;
+	bool stop = false;
+
+	INIT_LIST_HEAD(&del_list);
+
+	spin_lock_irqsave(&cache->lock, flags);
+	list_for_each_entry_safe_reverse(mrc, temp, &cache->lru_list, list) {
+		if (cache->ops->evict(cache, cache->ops_arg, mrc, evict_arg,
+				      &stop)) {
+			trace_rv_mr_cache_evict_evict(mrc->addr, mrc->len,
+						      mrc->access);
+			rv_int_rb_remove(mrc, &cache->root);
+			list_move(&mrc->list, &del_list);
+			cache->stats.evict++;
+			rv_update_mrc_stats_remove(cache, mrc);
+		} else {
+			trace_rv_mr_cache_evict_keep(mrc->addr, mrc->len,
+						     mrc->access);
+		}
+		if (stop)
+			break;
+	}
+	spin_unlock_irqrestore(&cache->lock, flags);
+
+	do_remove(cache, &del_list);
+}
+
+/*
+ * Call the remove function for the given cache and the list.  This
+ * is expected to be called with a delete list extracted from cache.
+ * The caller does NOT need the cache->lock.
+ */
+static void do_remove(struct rv_mr_cache *cache, struct list_head *del_list)
+{
+	struct rv_mr_cached *mrc;
+
+	while (!list_empty(del_list)) {
+		mrc = list_first_entry(del_list, struct rv_mr_cached, list);
+		list_del(&mrc->list);
+		/* Deregister the mr here */
+		kfree(mrc);
+	}
+}
+
+/*
+ * Work queue function to remove all nodes that have been queued up to
+ * be removed.	The key feature is that mm->mmap_lock is not being held
+ * and the remove callback can sleep while taking it, if needed.
+ */
+static void handle_remove(struct work_struct *work)
+{
+	struct rv_mr_cache *cache = container_of(work, struct rv_mr_cache,
+						 del_work);
+	struct list_head del_list;
+	unsigned long flags;
+
+	spin_lock_irqsave(&cache->lock, flags);
+	list_replace_init(&cache->del_list, &del_list);
+	spin_unlock_irqrestore(&cache->lock, flags);
+
+	do_remove(cache, &del_list);
+}
+
+static u32 rv_cache_evict(struct rv_mr_cache *cache, u64 mbytes)
+{
+	struct evict_data evict_data;
+
+	evict_data.cleared = 0;
+	evict_data.target = mbytes;
+	trace_rv_mr_cache_cache_evict(evict_data.cleared, evict_data.target,
+				      cache->total_size);
+	rv_mr_cache_evict(cache, &evict_data);
+	trace_rv_mr_cache_cache_evict(evict_data.cleared, evict_data.target,
+				      cache->total_size);
+	return evict_data.cleared;
+}
+
+static void rv_update_mrc_stats_add(struct rv_mr_cache *cache,
+				    struct rv_mr_cached *mrc)
+{
+	cache->total_size += mrc->len;
+	cache->stats.count++;
+	rv_mr_cache_update_stats_max(cache, atomic_read(&mrc->refcount));
+}
+
+static void rv_update_mrc_stats_remove(struct rv_mr_cache *cache,
+				       struct rv_mr_cached *mrc)
+{
+	cache->total_size -= mrc->len;
+	cache->stats.count--;
+}
diff --git a/drivers/infiniband/ulp/rv/trace.h b/drivers/infiniband/ulp/rv/trace.h
index cb1d1d087e16..7a4cc4919693 100644
--- a/drivers/infiniband/ulp/rv/trace.h
+++ b/drivers/infiniband/ulp/rv/trace.h
@@ -2,4 +2,5 @@
 /*
  * Copyright(c) 2020 - 2021 Intel Corporation.
  */
+#include "trace_mr_cache.h"
 #include "trace_dev.h"
diff --git a/drivers/infiniband/ulp/rv/trace_mr_cache.h b/drivers/infiniband/ulp/rv/trace_mr_cache.h
new file mode 100644
index 000000000000..4b1f668d4b87
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/trace_mr_cache.h
@@ -0,0 +1,181 @@
+/* SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause) */
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+#if !defined(__RV_TRACE_MR_CACHE_H) || defined(TRACE_HEADER_MULTI_READ)
+#define __RV_TRACE_MR_CACHE_H
+
+#include <linux/tracepoint.h>
+#include <linux/trace_seq.h>
+
+#undef TRACE_SYSTEM
+#define TRACE_SYSTEM rv_mr_cache
+
+DECLARE_EVENT_CLASS(/* rv_mr_cache */
+	rv_mr_cache_template,
+	TP_PROTO(u64 addr, u64 len, u32 acc),
+	TP_ARGS(addr, len, acc),
+	TP_STRUCT__entry(/* entry */
+		__field(u64, addr)
+		__field(u64, len)
+		__field(u32, acc)
+	),
+	TP_fast_assign(/* assign */
+		__entry->addr = addr;
+		__entry->len = len;
+		__entry->acc = acc;
+	),
+	TP_printk(/* print */
+		"addr 0x%llx, len %llu acc 0x%x",
+		__entry->addr,
+		__entry->len,
+		__entry->acc
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_cache_template, rv_mr_cache_insert,
+	TP_PROTO(u64 addr, u64 len, u32 acc),
+	TP_ARGS(addr, len, acc)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_cache_template, rv_mr_cache_search_enter,
+	TP_PROTO(u64 addr, u64 len, u32 acc),
+	TP_ARGS(addr, len, acc)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_cache_template, rv_mr_cache_search_mrc,
+	TP_PROTO(u64 addr, u64 len, u32 acc),
+	TP_ARGS(addr, len, acc)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_cache_template, rv_mr_cache_remove,
+	TP_PROTO(u64 addr, u64 len, u32 acc),
+	TP_ARGS(addr, len, acc)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_cache_template, rv_mr_cache_evict_evict,
+	TP_PROTO(u64 addr, u64 len, u32 acc),
+	TP_ARGS(addr, len, acc)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_cache_template, rv_mr_cache_evict_keep,
+	TP_PROTO(u64 addr, u64 len, u32 acc),
+	TP_ARGS(addr, len, acc)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_cache_template, rv_mr_cache_notifier,
+	TP_PROTO(u64 addr, u64 len, u32 acc),
+	TP_ARGS(addr, len, acc)
+);
+
+TRACE_EVENT(/* event */
+	rv_mr_cache_cache_full,
+	TP_PROTO(u64 max, u64 total, u64 cur),
+	TP_ARGS(max, total, cur),
+	TP_STRUCT__entry(/* entry */
+		__field(u64, max)
+		__field(u64, total)
+		__field(u64, cur)
+	),
+	TP_fast_assign(/* assign */
+		__entry->max = max;
+		__entry->total = total;
+		__entry->cur = cur;
+	),
+	TP_printk(/* print */
+		"Cache Full max %llu, total %llu, cur %llu",
+		__entry->max,
+		__entry->total,
+		__entry->cur
+	)
+);
+
+TRACE_EVENT(/* event */
+	rv_mr_cache_deinit,
+	TP_PROTO(u64 addr, u64 len, u32 acc, int cnt),
+	TP_ARGS(addr, len, acc, cnt),
+	TP_STRUCT__entry(/* entry */
+		__field(u64, addr)
+		__field(u64, len)
+		__field(u32, acc)
+		__field(int, cnt)
+	),
+	TP_fast_assign(/* assign */
+		__entry->addr = addr;
+		__entry->len = len;
+		__entry->acc = acc;
+		__entry->cnt = cnt;
+	),
+	TP_printk(/* print */
+		"addr 0x%llx, len %llu, acc %u refcnt %d",
+		__entry->addr,
+		__entry->len,
+		__entry->acc,
+		__entry->cnt
+	)
+);
+
+DECLARE_EVENT_CLASS(/* rv_mr_cache_wq */
+	rv_mr_cache_wq_template,
+	TP_PROTO(const char *wq_name),
+	TP_ARGS(wq_name),
+	TP_STRUCT__entry(/* entry */
+		__string(name, wq_name)
+	),
+	TP_fast_assign(/* assign */
+		__assign_str(name, wq_name);
+	),
+	TP_printk(/* print */
+		"Workqueue %s",
+		__get_str(name)
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_cache_wq_template, rv_mr_cache_wq_alloc,
+	TP_PROTO(const char *wq_name),
+	TP_ARGS(wq_name)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_cache_wq_template, rv_mr_cache_wq_destroy,
+	TP_PROTO(const char *wq_name),
+	TP_ARGS(wq_name)
+);
+
+TRACE_EVENT(/* event */
+	rv_mr_cache_cache_evict,
+	TP_PROTO(u64 cleared, u64 target, u64 total_size),
+	TP_ARGS(cleared, target, total_size),
+	TP_STRUCT__entry(/* entry */
+		__field(u64, cleared)
+		__field(u64, target)
+		__field(u64, total_size)
+	),
+	TP_fast_assign(/* assign */
+		__entry->cleared = cleared;
+		__entry->target = target;
+		__entry->total_size = total_size;
+	),
+	TP_printk(/* print */
+		"cleared 0x%llx target 0x%llx total_size 0x%llx",
+		__entry->cleared,
+		__entry->target,
+		__entry->total_size
+	)
+);
+
+#endif /* __RV_TRACE_MR_CACHE_H */
+
+#undef TRACE_INCLUDE_PATH
+#undef TRACE_INCLUDE_FILE
+#define TRACE_INCLUDE_PATH .
+#define TRACE_INCLUDE_FILE trace_mr_cache
+#include <trace/define_trace.h>
-- 
2.18.1


^ permalink raw reply related	[flat|nested] 52+ messages in thread

* [PATCH RFC 5/9] RDMA/rv: Add function to register/deregister memory region
  2021-03-19 12:56 [PATCH RFC 0/9] A rendezvous module kaike.wan
                   ` (3 preceding siblings ...)
  2021-03-19 12:56 ` [PATCH RFC 4/9] RDMA/rv: Add functions for memory region cache kaike.wan
@ 2021-03-19 12:56 ` kaike.wan
  2021-03-19 12:56 ` [PATCH RFC 6/9] RDMA/rv: Add connection management functions kaike.wan
                   ` (4 subsequent siblings)
  9 siblings, 0 replies; 52+ messages in thread
From: kaike.wan @ 2021-03-19 12:56 UTC (permalink / raw)
  To: dledford, jgg; +Cc: linux-rdma, todd.rimmer, Kaike Wan

From: Kaike Wan <kaike.wan@intel.com>

This patch adds functions for user application to register/deregister
memory region (mr) for user buffers. Two types of mrs are supported:
user mrs and kernel mrs.

User mrs are used soley by the user application. The reason that
the user mrs are cached in the rv module instead of in the user
application is that a middleware application may not known when a user
buffer (allocated by a upper lay application) is freed in order to free
any stale nodes. On the other hand, the rv module can register an MMU
notifier callback so that it can promptly remove any stale cache nodes.

Kernel mrs are used by the rv module for any RDMA transactions between
nodes.

A user mr is registered in a way similar to that in the verbs
interface. A kernel mr is registered similar to that in
ib_reg_user_mr() for on-demand paging. An RDMA hardware may have to
be qualified for this mechanism.

Signed-off-by: Todd Rimmer <todd.rimmer@intel.com>
Signed-off-by: Kaike Wan <kaike.wan@intel.com>
---
 drivers/infiniband/core/core_priv.h     |   4 +
 drivers/infiniband/core/rdma_core.c     | 237 ++++++++++++++
 drivers/infiniband/core/uverbs_cmd.c    |  54 +---
 drivers/infiniband/ulp/rv/Makefile      |   2 +-
 drivers/infiniband/ulp/rv/rv_file.c     |  31 ++
 drivers/infiniband/ulp/rv/rv_mr.c       | 393 ++++++++++++++++++++++++
 drivers/infiniband/ulp/rv/rv_mr_cache.c |   2 +-
 drivers/infiniband/ulp/rv/trace.h       |   2 +
 drivers/infiniband/ulp/rv/trace_mr.h    | 109 +++++++
 drivers/infiniband/ulp/rv/trace_user.h  |  66 ++++
 include/rdma/uverbs_types.h             |  10 +
 11 files changed, 860 insertions(+), 50 deletions(-)
 create mode 100644 drivers/infiniband/ulp/rv/rv_file.c
 create mode 100644 drivers/infiniband/ulp/rv/rv_mr.c
 create mode 100644 drivers/infiniband/ulp/rv/trace_mr.h
 create mode 100644 drivers/infiniband/ulp/rv/trace_user.h

diff --git a/drivers/infiniband/core/core_priv.h b/drivers/infiniband/core/core_priv.h
index 315f7a297eee..678fb9a3d77b 100644
--- a/drivers/infiniband/core/core_priv.h
+++ b/drivers/infiniband/core/core_priv.h
@@ -404,4 +404,8 @@ void rdma_umap_priv_init(struct rdma_umap_priv *priv,
 
 void ib_cq_pool_cleanup(struct ib_device *dev);
 
+struct ib_mr *uverbs_reg_mr(struct uverbs_attr_bundle *attrs, u32 pd_handle,
+			    u64 start, u64 length, u64 hca_va,
+			    u32 access_flags, struct ib_udata *driver_udata);
+
 #endif /* _CORE_PRIV_H */
diff --git a/drivers/infiniband/core/rdma_core.c b/drivers/infiniband/core/rdma_core.c
index 75eafd9208aa..22120adf0ef2 100644
--- a/drivers/infiniband/core/rdma_core.c
+++ b/drivers/infiniband/core/rdma_core.c
@@ -1013,3 +1013,240 @@ void uverbs_finalize_object(struct ib_uobject *uobj,
 		WARN_ON(true);
 	}
 }
+
+static struct ib_uverbs_file *get_ufile_from_fd(struct fd *fd)
+{
+	return fd->file ? (struct ib_uverbs_file *)fd->file->private_data :
+			  NULL;
+}
+
+static struct ib_uverbs_file *get_ufile(u32 cmd_fd, struct fd *fd)
+{
+	struct fd f;
+	struct ib_uverbs_file *file;
+
+	/* fd to "struct fd" */
+	f = fdget(cmd_fd);
+
+	file = get_ufile_from_fd(&f);
+	if (!file)
+		goto bail;
+
+	memcpy(fd, &f, sizeof(f));
+	return file;
+bail:
+	fdput(f);
+	return NULL;
+}
+
+struct ib_mr *uverbs_reg_mr(struct uverbs_attr_bundle *attrs, u32 pd_handle,
+			    u64 start, u64 length, u64 hca_va,
+			    u32 access_flags, struct ib_udata *driver_udata)
+{
+	struct ib_uobject           *uobj;
+	struct ib_pd                *pd;
+	struct ib_mr                *mr;
+	int                          ret;
+	struct ib_device            *ib_dev;
+
+	uobj = uobj_alloc(UVERBS_OBJECT_MR, attrs, &ib_dev);
+	if (IS_ERR(uobj))
+		return (struct ib_mr *)uobj;
+
+	ret = ib_check_mr_access(ib_dev, access_flags);
+	if (ret)
+		goto err_free;
+
+	pd = uobj_get_obj_read(pd, UVERBS_OBJECT_PD, pd_handle, attrs);
+	if (!pd) {
+		ret = -EINVAL;
+		goto err_free;
+	}
+
+	mr = pd->device->ops.reg_user_mr(pd, start, length, hca_va,
+					 access_flags, driver_udata);
+	if (IS_ERR(mr)) {
+		ret = PTR_ERR(mr);
+		goto err_put;
+	}
+
+	mr->device  = pd->device;
+	mr->pd      = pd;
+	mr->type    = IB_MR_TYPE_USER;
+	mr->dm      = NULL;
+	mr->sig_attrs = NULL;
+	mr->uobject = uobj;
+	atomic_inc(&pd->usecnt);
+	mr->iova = hca_va;
+
+	rdma_restrack_new(&mr->res, RDMA_RESTRACK_MR);
+	rdma_restrack_set_name(&mr->res, NULL);
+	rdma_restrack_add(&mr->res);
+
+	uobj->object = mr;
+	uobj_put_obj_read(pd);
+	uobj_finalize_uobj_create(uobj, attrs);
+
+	return mr;
+
+err_put:
+	uobj_put_obj_read(pd);
+err_free:
+	uobj_alloc_abort(uobj, attrs);
+	return ERR_PTR(ret);
+}
+
+struct ib_mr *rdma_reg_user_mr(struct ib_device *ib_dev, u32 cmd_fd,
+			       u32 pd_handle, u64 start, u64 length,
+			       u32 access_flags, size_t ulen, void *udata,
+			       struct fd *fd)
+{
+	struct ib_mr *mr;
+	struct ib_uverbs_file *ufile;
+	struct uverbs_attr_bundle attrs;
+	int srcu_key;
+	struct ib_device *device;
+
+	if (!fd) {
+		mr = ERR_PTR(-EINVAL);
+		goto out;
+	}
+
+	memset(&attrs, 0, sizeof(attrs));
+	ufile = get_ufile(cmd_fd, fd);
+	if (!ufile) {
+		mr = ERR_PTR(-EINVAL);
+		goto out;
+	}
+
+	srcu_key = srcu_read_lock(&ufile->device->disassociate_srcu);
+	/* Validate the target ib_device */
+	device = srcu_dereference(ufile->device->ib_dev,
+				  &ufile->device->disassociate_srcu);
+	if (device != ib_dev) {
+		mr = ERR_PTR(-EINVAL);
+		goto out_fd;
+	}
+	attrs.ufile = ufile;
+	attrs.driver_udata.inlen = ulen;
+	if (ulen)
+		attrs.driver_udata.inbuf = udata;
+
+	mr = uverbs_reg_mr(&attrs, pd_handle, start, length, start,
+			   access_flags, &attrs.driver_udata);
+	if (IS_ERR(mr))
+		goto out_fd;
+	else
+		uverbs_finalize_object(attrs.uobject, UVERBS_ACCESS_NEW, true,
+				       true, &attrs);
+
+	goto out_unlock;
+
+out_fd:
+	fdput(*fd);
+out_unlock:
+	srcu_read_unlock(&ufile->device->disassociate_srcu, srcu_key);
+out:
+	return mr;
+}
+EXPORT_SYMBOL(rdma_reg_user_mr);
+
+int rdma_dereg_user_mr(struct ib_mr *mr, struct fd *fd)
+{
+	struct ib_uverbs_file *ufile;
+	struct uverbs_attr_bundle attrs;
+	int srcu_key;
+	int ret;
+
+	if (!fd || !mr || !mr->uobject)
+		return -EINVAL;
+
+	memset(&attrs, 0, sizeof(attrs));
+	ufile = get_ufile_from_fd(fd);
+	if (!ufile || !ufile->device)
+		return -EINVAL;
+
+	srcu_key = srcu_read_lock(&ufile->device->disassociate_srcu);
+	attrs.ufile = ufile;
+
+	ret = uobj_perform_destroy(UVERBS_OBJECT_MR, (u32)mr->uobject->id,
+				   &attrs);
+	fdput(*fd);
+
+	srcu_read_unlock(&ufile->device->disassociate_srcu, srcu_key);
+	return ret;
+}
+EXPORT_SYMBOL(rdma_dereg_user_mr);
+
+struct ib_mr *rdma_reg_kernel_mr(u32 cmd_fd, struct ib_pd *kern_pd, u64 start,
+				 u64 length, u32 access_flags, size_t ulen,
+				 void *udata, struct fd *fd)
+{
+	struct ib_mr *mr;
+	struct ib_uverbs_file *ufile;
+	struct uverbs_attr_bundle attrs;
+	int srcu_key;
+
+	if (!fd) {
+		mr = ERR_PTR(-EINVAL);
+		goto out;
+	}
+
+	memset(&attrs, 0, sizeof(attrs));
+	ufile = get_ufile(cmd_fd, fd);
+	if (!ufile) {
+		mr = ERR_PTR(-EINVAL);
+		goto out;
+	}
+	srcu_key = srcu_read_lock(&ufile->device->disassociate_srcu);
+	attrs.ufile = ufile;
+	attrs.driver_udata.inlen = ulen;
+	if (ulen)
+		attrs.driver_udata.inbuf = udata;
+
+	mr = kern_pd->device->ops.reg_user_mr(kern_pd, start, length, start,
+					      access_flags,
+					      &attrs.driver_udata);
+	if (IS_ERR(mr)) {
+		fdput(*fd);
+		goto out_unlock;
+	}
+
+	mr->device  = kern_pd->device;
+	mr->pd      = kern_pd;
+	mr->dm      = NULL;
+	atomic_inc(&kern_pd->usecnt);
+
+out_unlock:
+	srcu_read_unlock(&ufile->device->disassociate_srcu, srcu_key);
+out:
+	return mr;
+}
+EXPORT_SYMBOL(rdma_reg_kernel_mr);
+
+int rdma_dereg_kernel_mr(struct ib_mr *mr, struct fd *fd)
+{
+	struct ib_pd *pd;
+	struct ib_uverbs_file *ufile;
+	int srcu_key;
+	int ret = 0;
+	struct ib_udata udata;
+
+	if (!fd || !mr)
+		return -EINVAL;
+
+	ufile = get_ufile_from_fd(fd);
+	if (!ufile)
+		return -EINVAL;
+
+	srcu_key = srcu_read_lock(&ufile->device->disassociate_srcu);
+	memset(&udata, 0, sizeof(udata));
+	pd = mr->pd;
+	ret = pd->device->ops.dereg_mr(mr, &udata);
+	atomic_dec(&pd->usecnt);
+	fdput(*fd);
+
+	srcu_read_unlock(&ufile->device->disassociate_srcu, srcu_key);
+	return ret;
+}
+EXPORT_SYMBOL(rdma_dereg_kernel_mr);
diff --git a/drivers/infiniband/core/uverbs_cmd.c b/drivers/infiniband/core/uverbs_cmd.c
index f5b8be3bedde..83751801d217 100644
--- a/drivers/infiniband/core/uverbs_cmd.c
+++ b/drivers/infiniband/core/uverbs_cmd.c
@@ -696,11 +696,8 @@ static int ib_uverbs_reg_mr(struct uverbs_attr_bundle *attrs)
 {
 	struct ib_uverbs_reg_mr_resp resp = {};
 	struct ib_uverbs_reg_mr      cmd;
-	struct ib_uobject           *uobj;
-	struct ib_pd                *pd;
 	struct ib_mr                *mr;
 	int                          ret;
-	struct ib_device *ib_dev;
 
 	ret = uverbs_request(attrs, &cmd, sizeof(cmd));
 	if (ret)
@@ -709,55 +706,16 @@ static int ib_uverbs_reg_mr(struct uverbs_attr_bundle *attrs)
 	if ((cmd.start & ~PAGE_MASK) != (cmd.hca_va & ~PAGE_MASK))
 		return -EINVAL;
 
-	uobj = uobj_alloc(UVERBS_OBJECT_MR, attrs, &ib_dev);
-	if (IS_ERR(uobj))
-		return PTR_ERR(uobj);
-
-	ret = ib_check_mr_access(ib_dev, cmd.access_flags);
-	if (ret)
-		goto err_free;
-
-	pd = uobj_get_obj_read(pd, UVERBS_OBJECT_PD, cmd.pd_handle, attrs);
-	if (!pd) {
-		ret = -EINVAL;
-		goto err_free;
-	}
-
-	mr = pd->device->ops.reg_user_mr(pd, cmd.start, cmd.length, cmd.hca_va,
-					 cmd.access_flags,
-					 &attrs->driver_udata);
-	if (IS_ERR(mr)) {
-		ret = PTR_ERR(mr);
-		goto err_put;
-	}
-
-	mr->device  = pd->device;
-	mr->pd      = pd;
-	mr->type    = IB_MR_TYPE_USER;
-	mr->dm	    = NULL;
-	mr->sig_attrs = NULL;
-	mr->uobject = uobj;
-	atomic_inc(&pd->usecnt);
-	mr->iova = cmd.hca_va;
-
-	rdma_restrack_new(&mr->res, RDMA_RESTRACK_MR);
-	rdma_restrack_set_name(&mr->res, NULL);
-	rdma_restrack_add(&mr->res);
-
-	uobj->object = mr;
-	uobj_put_obj_read(pd);
-	uobj_finalize_uobj_create(uobj, attrs);
+	mr = uverbs_reg_mr(attrs, cmd.pd_handle, cmd.start, cmd.length,
+			   cmd.hca_va, cmd.access_flags,
+			   &attrs->driver_udata);
+	if (IS_ERR(mr))
+		return PTR_ERR(mr);
 
 	resp.lkey = mr->lkey;
 	resp.rkey = mr->rkey;
-	resp.mr_handle = uobj->id;
+	resp.mr_handle = mr->uobject->id;
 	return uverbs_response(attrs, &resp, sizeof(resp));
-
-err_put:
-	uobj_put_obj_read(pd);
-err_free:
-	uobj_alloc_abort(uobj, attrs);
-	return ret;
 }
 
 static int ib_uverbs_rereg_mr(struct uverbs_attr_bundle *attrs)
diff --git a/drivers/infiniband/ulp/rv/Makefile b/drivers/infiniband/ulp/rv/Makefile
index 01e93dc25f1d..677b113c0666 100644
--- a/drivers/infiniband/ulp/rv/Makefile
+++ b/drivers/infiniband/ulp/rv/Makefile
@@ -4,6 +4,6 @@
 #
 obj-$(CONFIG_INFINIBAND_RV) += rv.o
 
-rv-y := rv_main.o rv_mr_cache.o trace.o
+rv-y := rv_main.o rv_mr_cache.o rv_file.o rv_mr.o trace.o
 
 CFLAGS_trace.o = -I$(src)
diff --git a/drivers/infiniband/ulp/rv/rv_file.c b/drivers/infiniband/ulp/rv/rv_file.c
new file mode 100644
index 000000000000..3625a9c1681a
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/rv_file.c
@@ -0,0 +1,31 @@
+// SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause)
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+
+#include "rv.h"
+
+/* A workqueue for all */
+static struct workqueue_struct *rv_wq;
+
+void rv_queue_work(struct work_struct *work)
+{
+	queue_work(rv_wq, work);
+}
+
+void rv_job_dev_get(struct rv_job_dev *jdev)
+{
+	kref_get(&jdev->kref);
+}
+
+static void rv_job_dev_release(struct kref *kref)
+{
+	struct rv_job_dev *jdev = container_of(kref, struct rv_job_dev, kref);
+
+	kfree_rcu(jdev, rcu);
+}
+
+void rv_job_dev_put(struct rv_job_dev *jdev)
+{
+	kref_put(&jdev->kref, rv_job_dev_release);
+}
diff --git a/drivers/infiniband/ulp/rv/rv_mr.c b/drivers/infiniband/ulp/rv/rv_mr.c
new file mode 100644
index 000000000000..76786bd11502
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/rv_mr.c
@@ -0,0 +1,393 @@
+// SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause)
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+
+#include <rdma/uverbs_std_types.h>
+#include <rdma/uverbs_ioctl.h>
+
+#include "rv.h"
+#include "trace.h"
+
+unsigned int enable_user_mr;
+
+module_param(enable_user_mr, uint, 0444);
+MODULE_PARM_DESC(enable_user_mr, "Enable user mode MR caching");
+
+static void rv_handle_user_mrs_put(struct work_struct *work);
+
+static bool rv_cache_mrc_filter(struct rv_mr_cached *mrc, u64 addr,
+				u64 len, u32 acc);
+static void rv_cache_mrc_get(struct rv_mr_cache *cache,
+			     void *arg, struct rv_mr_cached *mrc);
+static int rv_cache_mrc_put(struct rv_mr_cache *cache,
+			    void *arg, struct rv_mr_cached *mrc);
+static int rv_cache_mrc_invalidate(struct rv_mr_cache *cache,
+				   void *arg, struct rv_mr_cached *mrc);
+static int rv_cache_mrc_evict(struct rv_mr_cache *cache,
+			      void *arg, struct rv_mr_cached *mrc,
+			      void *evict_arg, bool *stop);
+
+static const struct rv_mr_cache_ops rv_cache_ops = {
+	.filter = rv_cache_mrc_filter,
+	.get = rv_cache_mrc_get,
+	.put = rv_cache_mrc_put,
+	.invalidate = rv_cache_mrc_invalidate,
+	.evict = rv_cache_mrc_evict
+};
+
+/* given an rv, find the proper ib_dev to use when registering user MRs */
+static struct ib_device *rv_ib_dev(struct rv_user *rv)
+{
+	struct rv_device *dev = rv->rdma_mode == RV_RDMA_MODE_USER ? rv->dev :
+				rv->jdev->dev;
+
+	return dev->ib_dev;
+}
+
+/* caller must hold rv->mutex */
+static int rv_drv_api_reg_mem(struct rv_user *rv,
+			      struct rv_mem_params_in *minfo,
+			      struct mr_info *mr)
+{
+	struct ib_mr *ib_mr;
+
+	mr->ib_mr = NULL;
+	mr->ib_pd = NULL;
+
+	/*
+	 * Check if the buffer is for kernel use. It should be noted that
+	 * the ibv_pd_handle value "0" is a valid user space pd handle.
+	 */
+	if (minfo->access & IBV_ACCESS_KERNEL)
+		ib_mr = rdma_reg_kernel_mr(minfo->cmd_fd_int, rv->jdev->pd,
+					   minfo->addr, minfo->length,
+					   minfo->access & ~IBV_ACCESS_KERNEL,
+					   minfo->ulen, minfo->udata,
+					   &mr->fd);
+	else
+		ib_mr = rdma_reg_user_mr(rv_ib_dev(rv), minfo->cmd_fd_int,
+					 minfo->ibv_pd_handle,
+					 minfo->addr, minfo->length,
+					 minfo->access, minfo->ulen,
+					 minfo->udata, &mr->fd);
+	if (IS_ERR(ib_mr)) {
+		rv_err(rv->inx, "reg_user_mr failed\n");
+		return  PTR_ERR(ib_mr);
+	}
+	/* A hardware driver may not set the iova field */
+	if (!ib_mr->iova)
+		ib_mr->iova = minfo->addr;
+
+	trace_rv_mr_info_reg(minfo->addr, minfo->length, minfo->access,
+			     ib_mr->lkey, ib_mr->rkey, ib_mr->iova,
+			     atomic_read(&ib_mr->pd->usecnt));
+	mr->ib_mr = ib_mr;
+	mr->ib_pd = ib_mr->pd;
+
+	return 0;
+}
+
+int rv_drv_api_dereg_mem(struct mr_info *mr)
+{
+	int ret;
+	struct rv_mr_cached *mrc = container_of(mr, struct rv_mr_cached, mr);
+
+	trace_rv_mr_info_dereg(mrc->addr, mrc->len, mrc->access,
+			       mr->ib_mr->lkey, mr->ib_mr->rkey,
+			       mr->ib_mr->iova,
+			       atomic_read(&mr->ib_pd->usecnt));
+
+	if (mrc->access & IBV_ACCESS_KERNEL)
+		ret = rdma_dereg_kernel_mr(mr->ib_mr, &mr->fd);
+	else
+		ret = rdma_dereg_user_mr(mr->ib_mr, &mr->fd);
+	if (!ret) {
+		mr->ib_mr = NULL;
+		mr->ib_pd = NULL;
+	}
+	return ret;
+}
+
+/* Cannot hold rv->mutex */
+struct rv_user_mrs *rv_user_mrs_alloc(struct rv_user *rv, u32 cache_size)
+{
+	int ret;
+	struct rv_user_mrs *umrs;
+
+	umrs = kzalloc(sizeof(*umrs), GFP_KERNEL);
+	if (!umrs)
+		return ERR_PTR(-ENOMEM);
+
+	umrs->rv_inx = rv->inx;
+	ret = rv_mr_cache_init(rv->inx, &umrs->cache, &rv_cache_ops, NULL,
+			       current->mm, cache_size);
+	if (ret)
+		goto bail_free;
+	kref_init(&umrs->kref); /* refcount now 1 */
+	INIT_WORK(&umrs->put_work, rv_handle_user_mrs_put);
+	return umrs;
+
+bail_free:
+	kfree(umrs);
+	return ERR_PTR(ret);
+}
+
+/* called with rv->mutex */
+void rv_user_mrs_attach(struct rv_user *rv)
+{
+	struct rv_user_mrs *umrs = rv->umrs;
+
+	if (rv->rdma_mode == RV_RDMA_MODE_KERNEL) {
+		/*
+		 * for mode KERNEL the user_mrs object may survive past the
+		 * rv_user close, so we need our own jdev reference to dereg
+		 * MRs while outstanding send IOs complete.
+		 * For mode USER, the MRs are using the user's pd
+		 * and rv_user will free all MRs during close
+		 *
+		 * the jdev->pd we will use for MRs and QP needs ref to jdev
+		 */
+		rv_job_dev_get(rv->jdev);
+		umrs->jdev = rv->jdev;
+	}
+	trace_rv_user_mrs_attach(umrs->rv_inx, umrs->jdev,
+				 umrs->cache.total_size, umrs->cache.max_size,
+				 kref_read(&umrs->kref));
+}
+
+static void rv_user_mrs_release(struct rv_user_mrs *umrs)
+{
+	trace_rv_user_mrs_release(umrs->rv_inx, umrs->jdev,
+				  umrs->cache.total_size, umrs->cache.max_size,
+				  kref_read(&umrs->kref));
+	rv_mr_cache_deinit(umrs->rv_inx, &umrs->cache);
+	if (umrs->jdev)
+		rv_job_dev_put(umrs->jdev);
+	kfree(umrs);
+}
+
+static void rv_handle_user_mrs_put(struct work_struct *work)
+{
+	struct rv_user_mrs *umrs = container_of(work, struct rv_user_mrs,
+						put_work);
+
+	rv_user_mrs_release(umrs);
+}
+
+static void rv_user_mrs_schedule_release(struct kref *kref)
+{
+	struct rv_user_mrs *umrs = container_of(kref, struct rv_user_mrs, kref);
+
+	/*
+	 * Since this function may be called from rv_write_done(),
+	 * we can't call rv_user_mrs_release() directly to
+	 * destroy it's rc QP and rv_mr_cache_deinit (and wait for completion)
+	 * Instead, put the cleanup on a workqueue thread.
+	 */
+	rv_queue_work(&umrs->put_work);
+}
+
+void rv_user_mrs_get(struct rv_user_mrs *umrs)
+{
+	kref_get(&umrs->kref);
+}
+
+void rv_user_mrs_put(struct rv_user_mrs *umrs)
+{
+	kref_put(&umrs->kref, rv_user_mrs_schedule_release);
+}
+
+int doit_reg_mem(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_mem_params mparams;
+	struct rv_mr_cached *mrc;
+	int ret;
+	struct rv_user_mrs *umrs = rv->umrs;
+
+	if (copy_from_user(&mparams.in, (void __user *)arg,
+			   sizeof(mparams.in)))
+		return -EFAULT;
+
+	if (!enable_user_mr && !(mparams.in.access & IBV_ACCESS_KERNEL))
+		return -EINVAL;
+
+	/*
+	 * rv->mutex protects use of umrs QP for REG_MR, also
+	 * protects between rb_search and rb_insert vs races with other
+	 * doit_reg_mem and doit_dereg_mem calls
+	 */
+	mutex_lock(&rv->mutex);
+	if (!rv->attached) {
+		ret = rv->was_attached ? -ENXIO : -EINVAL;
+		goto bail_unlock;
+	}
+	if (rv->rdma_mode != RV_RDMA_MODE_KERNEL &&
+	    (mparams.in.access & IBV_ACCESS_KERNEL)) {
+		ret = -EINVAL;
+		goto bail_unlock;
+	}
+
+	trace_rv_mr_reg(rv->rdma_mode, mparams.in.addr,
+			mparams.in.length, mparams.in.access);
+	/* get reference,  if found update hit stats */
+	mrc = rv_mr_cache_search_get(&umrs->cache, mparams.in.addr,
+				     mparams.in.length, mparams.in.access,
+				     true);
+	if (mrc)
+		goto cont;
+
+	/* create a new mrc for rb tree */
+	mrc = kzalloc(sizeof(*mrc), GFP_KERNEL);
+	if (!mrc) {
+		ret = -ENOMEM;
+		umrs->stats.failed++;
+		goto bail_unlock;
+	}
+
+	/* register using verbs callback */
+	ret = rv_drv_api_reg_mem(rv, &mparams.in, &mrc->mr);
+	if (ret) {
+		umrs->stats.failed++;
+		goto bail_free;
+	}
+	mrc->addr = mparams.in.addr;
+	mrc->len = mparams.in.length;
+	mrc->access = mparams.in.access;
+
+	ret = rv_mr_cache_insert(&umrs->cache, mrc);
+	if (ret)
+		goto bail_dereg;
+
+cont:
+	/* return the mr handle, lkey & rkey */
+	mparams.out.mr_handle = (uint64_t)mrc;
+	mparams.out.iova = mrc->mr.ib_mr->iova;
+	mparams.out.lkey = mrc->mr.ib_mr->lkey;
+	mparams.out.rkey = mrc->mr.ib_mr->rkey;
+
+	if (copy_to_user((void __user *)arg, &mparams.out,
+			 sizeof(mparams.out))) {
+		ret = -EFAULT;
+		goto bail_put;
+	}
+
+	mutex_unlock(&rv->mutex);
+
+	return 0;
+
+bail_dereg:
+	if (rv_drv_api_dereg_mem(&mrc->mr))
+		rv_err(rv->inx, "dereg_mem failed during cleanup\n");
+bail_free:
+	kfree(mrc);
+bail_unlock:
+	mutex_unlock(&rv->mutex);
+	return ret;
+
+bail_put:
+	rv_mr_cache_put(&umrs->cache, mrc);
+	mutex_unlock(&rv->mutex);
+	return ret;
+}
+
+int doit_dereg_mem(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_mr_cached *mrc;
+	struct rv_dereg_params_in dparams;
+	int ret = -EINVAL;
+
+	if (copy_from_user(&dparams, (void __user *)arg, sizeof(dparams)))
+		return -EFAULT;
+
+	/* rv->mutex protects possible race with doit_reg_mem */
+	mutex_lock(&rv->mutex);
+	if (!rv->attached) {
+		ret = rv->was_attached ? -ENXIO : -EINVAL;
+		goto bail_unlock;
+	}
+
+	mrc = rv_mr_cache_search_put(&rv->umrs->cache, dparams.addr,
+				     dparams.length, dparams.access);
+	if (!mrc)
+		goto bail_unlock;
+
+	mutex_unlock(&rv->mutex);
+	trace_rv_mr_dereg(rv->rdma_mode, dparams.addr,
+			  dparams.length, dparams.access);
+
+	return 0;
+
+bail_unlock:
+	mutex_unlock(&rv->mutex);
+	return ret;
+}
+
+/* called with cache->lock */
+static bool rv_cache_mrc_filter(struct rv_mr_cached *mrc, u64 addr,
+				u64 len, u32 acc)
+{
+	return mrc->addr == addr && mrc->len == len && mrc->access == acc;
+}
+
+/* called with cache->lock */
+static void rv_cache_mrc_get(struct rv_mr_cache *cache,
+			     void *arg, struct rv_mr_cached *mrc)
+{
+	int refcount;
+
+	refcount = atomic_inc_return(&mrc->refcount);
+	if (refcount == 1) {
+		cache->stats.inuse++;
+		cache->stats.inuse_bytes += mrc->len;
+	}
+	rv_mr_cache_update_stats_max(cache, refcount);
+}
+
+/* called with cache->lock */
+static int rv_cache_mrc_put(struct rv_mr_cache *cache,
+			    void *arg, struct rv_mr_cached *mrc)
+{
+	int refcount;
+
+	refcount = atomic_dec_return(&mrc->refcount);
+	if (!refcount) {
+		cache->stats.inuse--;
+		cache->stats.inuse_bytes -= mrc->len;
+	}
+	return refcount;
+}
+
+/* called with cache->lock */
+static int rv_cache_mrc_invalidate(struct rv_mr_cache *cache,
+				   void *arg, struct rv_mr_cached *mrc)
+{
+	if (!atomic_read(&mrc->refcount))
+		return 1;
+	return 0;
+}
+
+/*
+ * Return 1 if the mrc can be evicted from the cache
+ *
+ * Called with cache->lock
+ */
+static int rv_cache_mrc_evict(struct rv_mr_cache *cache,
+			      void *arg, struct rv_mr_cached *mrc,
+			      void *evict_arg, bool *stop)
+{
+	struct evict_data *evict_data = evict_arg;
+
+	/* is this mrc still being used? */
+	if (atomic_read(&mrc->refcount))
+		return 0; /* keep this mrc */
+
+	/* this mrc will be evicted, add its size to our count */
+	evict_data->cleared += mrc->len;
+
+	/* have enough bytes been cleared? */
+	if (evict_data->cleared >= evict_data->target)
+		*stop = true;
+
+	return 1; /* remove this mrc */
+}
diff --git a/drivers/infiniband/ulp/rv/rv_mr_cache.c b/drivers/infiniband/ulp/rv/rv_mr_cache.c
index 48ea7c958f74..830ee246006b 100644
--- a/drivers/infiniband/ulp/rv/rv_mr_cache.c
+++ b/drivers/infiniband/ulp/rv/rv_mr_cache.c
@@ -374,7 +374,7 @@ static void do_remove(struct rv_mr_cache *cache, struct list_head *del_list)
 	while (!list_empty(del_list)) {
 		mrc = list_first_entry(del_list, struct rv_mr_cached, list);
 		list_del(&mrc->list);
-		/* Deregister the mr here */
+		rv_drv_api_dereg_mem(&mrc->mr);
 		kfree(mrc);
 	}
 }
diff --git a/drivers/infiniband/ulp/rv/trace.h b/drivers/infiniband/ulp/rv/trace.h
index 7a4cc4919693..d2827582be05 100644
--- a/drivers/infiniband/ulp/rv/trace.h
+++ b/drivers/infiniband/ulp/rv/trace.h
@@ -4,3 +4,5 @@
  */
 #include "trace_mr_cache.h"
 #include "trace_dev.h"
+#include "trace_mr.h"
+#include "trace_user.h"
diff --git a/drivers/infiniband/ulp/rv/trace_mr.h b/drivers/infiniband/ulp/rv/trace_mr.h
new file mode 100644
index 000000000000..158c1b106b77
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/trace_mr.h
@@ -0,0 +1,109 @@
+/* SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause) */
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+#if !defined(__RV_TRACE_MR_H) || defined(TRACE_HEADER_MULTI_READ)
+#define __RV_TRACE_MR_H
+
+#include <linux/tracepoint.h>
+#include <linux/trace_seq.h>
+
+#undef TRACE_SYSTEM
+#define TRACE_SYSTEM rv_mr
+
+#define MR_INFO_PRN "addr 0x%llx len 0x%llx acc 0x%x lkey 0x%x rkey 0x%x " \
+		    "iova 0x%llx pd_usecnt %u"
+
+DECLARE_EVENT_CLASS(/* mr */
+	rv_mr_template,
+	TP_PROTO(u8 mode, u64 addr, u64 len, u32 acc),
+	TP_ARGS(mode, addr, len, acc),
+	TP_STRUCT__entry(/* entry */
+		__field(u8, mode)
+		__field(u64, addr)
+		__field(u64, len)
+		__field(u32, acc)
+	),
+	TP_fast_assign(/* assign */
+		__entry->mode = mode;
+		__entry->addr = addr;
+		__entry->len = len;
+		__entry->acc = acc;
+	),
+	TP_printk(/* print */
+		"mode 0x%x addr 0x%llx, len %llu acc 0x%x",
+		__entry->mode,
+		__entry->addr,
+		__entry->len,
+		__entry->acc
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_template, rv_mr_reg,
+	TP_PROTO(u8 mode, u64 addr, u64 len, u32 acc),
+	TP_ARGS(mode, addr, len, acc)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_template, rv_mr_dereg,
+	TP_PROTO(u8 mode, u64 addr, u64 len, u32 acc),
+	TP_ARGS(mode, addr, len, acc)
+);
+
+DECLARE_EVENT_CLASS(/* mr_info */
+	rv_mr_info_template,
+	TP_PROTO(u64 addr, u64 len, u32 acc, u32 lkey,
+		 u32 rkey, u64 iova, u32 pd_usecnt),
+	TP_ARGS(addr, len, acc, lkey, rkey, iova, pd_usecnt),
+	TP_STRUCT__entry(/* entry */
+		__field(u64, addr)
+		__field(u64, len)
+		__field(u32, acc)
+		__field(u32, lkey)
+		__field(u32, rkey)
+		__field(u64, iova)
+		__field(u32, cnt)
+	),
+	TP_fast_assign(/* assign */
+		__entry->addr = addr;
+		__entry->len = len;
+		__entry->acc = acc;
+		__entry->lkey = lkey;
+		__entry->rkey = rkey;
+		__entry->iova = iova;
+		__entry->cnt = pd_usecnt;
+	),
+	TP_printk(/* print */
+		MR_INFO_PRN,
+		__entry->addr,
+		__entry->len,
+		__entry->acc,
+		__entry->lkey,
+		__entry->rkey,
+		__entry->iova,
+		__entry->cnt
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_info_template, rv_mr_info_reg,
+	TP_PROTO(u64 addr, u64 len, u32 acc, u32 lkey,
+		 u32 rkey, u64 iova, u32 pd_usecnt),
+	TP_ARGS(addr, len, acc, lkey, rkey, iova, pd_usecnt)
+);
+
+DEFINE_EVENT(/* event */
+	rv_mr_info_template, rv_mr_info_dereg,
+	TP_PROTO(u64 addr, u64 len, u32 acc, u32 lkey,
+		 u32 rkey, u64 iova, u32 pd_usecnt),
+	TP_ARGS(addr, len, acc, lkey, rkey, iova, pd_usecnt)
+);
+
+#endif /* __RV_TRACE_MR_H */
+
+#undef TRACE_INCLUDE_PATH
+#undef TRACE_INCLUDE_FILE
+#define TRACE_INCLUDE_PATH .
+#define TRACE_INCLUDE_FILE trace_mr
+#include <trace/define_trace.h>
diff --git a/drivers/infiniband/ulp/rv/trace_user.h b/drivers/infiniband/ulp/rv/trace_user.h
new file mode 100644
index 000000000000..2707e39bdfd6
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/trace_user.h
@@ -0,0 +1,66 @@
+/* SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause) */
+/*
+ * Copyright(c) 2020 Intel Corporation.
+ */
+#if !defined(__RV_TRACE_USER_H) || defined(TRACE_HEADER_MULTI_READ)
+#define __RV_TRACE_USER_H
+
+#include <linux/tracepoint.h>
+#include <linux/trace_seq.h>
+
+#undef TRACE_SYSTEM
+#define TRACE_SYSTEM rv_user
+
+#define RV_USER_MRS_PRN "rv_nx %d jdev %p total_size 0x%llx max_size 0x%llx " \
+			"refcount %u"
+
+DECLARE_EVENT_CLASS(/* user_mrs */
+	rv_user_mrs_template,
+	TP_PROTO(int rv_inx, void *jdev, u64 total_size, u64 max_size,
+		 u32 refcount),
+	TP_ARGS(rv_inx, jdev, total_size, max_size, refcount),
+	TP_STRUCT__entry(/* entry */
+		__field(int, rv_inx)
+		__field(void *, jdev)
+		__field(u64, total_size)
+		__field(u64, max_size)
+		__field(u32, refcount)
+	),
+	TP_fast_assign(/* assign */
+		__entry->rv_inx = rv_inx;
+		__entry->jdev = jdev;
+		__entry->total_size = total_size;
+		__entry->max_size = max_size;
+		__entry->refcount = refcount;
+	),
+	TP_printk(/* print */
+		RV_USER_MRS_PRN,
+		__entry->rv_inx,
+		__entry->jdev,
+		__entry->total_size,
+		__entry->max_size,
+		__entry->refcount
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_mrs_template, rv_user_mrs_attach,
+	TP_PROTO(int rv_inx, void *jdev, u64 total_size, u64 max_size,
+		 u32 refcount),
+	TP_ARGS(rv_inx, jdev, total_size, max_size, refcount)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_mrs_template, rv_user_mrs_release,
+	TP_PROTO(int rv_inx, void *jdev, u64 total_size, u64 max_size,
+		 u32 refcount),
+	TP_ARGS(rv_inx, jdev, total_size, max_size, refcount)
+);
+
+#endif /* __RV_TRACE_USER_H */
+
+#undef TRACE_INCLUDE_PATH
+#undef TRACE_INCLUDE_FILE
+#define TRACE_INCLUDE_PATH .
+#define TRACE_INCLUDE_FILE trace_user
+#include <trace/define_trace.h>
diff --git a/include/rdma/uverbs_types.h b/include/rdma/uverbs_types.h
index ccd11631c167..93aefd4d085f 100644
--- a/include/rdma/uverbs_types.h
+++ b/include/rdma/uverbs_types.h
@@ -134,6 +134,16 @@ static inline void uverbs_uobject_get(struct ib_uobject *uobject)
 }
 void uverbs_uobject_put(struct ib_uobject *uobject);
 
+struct ib_mr *rdma_reg_user_mr(struct ib_device *ib_dev, u32 cmd_fd,
+			       u32 pd_handle, u64 start, u64 length,
+			       u32 access_flags, size_t ulen, void *udata,
+			       struct fd *fd);
+int rdma_dereg_user_mr(struct ib_mr *mr, struct fd *fd);
+struct ib_mr *rdma_reg_kernel_mr(u32 cmd_fd, struct ib_pd *kern_pd, u64 start,
+				 u64 length, u32 access_flags, size_t ulen,
+				 void *udata, struct fd *fd);
+int rdma_dereg_kernel_mr(struct ib_mr *mr, struct fd *fd);
+
 struct uverbs_obj_fd_type {
 	/*
 	 * In fd based objects, uverbs_obj_type_ops points to generic
-- 
2.18.1


^ permalink raw reply related	[flat|nested] 52+ messages in thread

* [PATCH RFC 6/9] RDMA/rv: Add connection management functions
  2021-03-19 12:56 [PATCH RFC 0/9] A rendezvous module kaike.wan
                   ` (4 preceding siblings ...)
  2021-03-19 12:56 ` [PATCH RFC 5/9] RDMA/rv: Add function to register/deregister memory region kaike.wan
@ 2021-03-19 12:56 ` kaike.wan
  2021-03-19 12:56 ` [PATCH RFC 7/9] RDMA/rv: Add functions for RDMA transactions kaike.wan
                   ` (3 subsequent siblings)
  9 siblings, 0 replies; 52+ messages in thread
From: kaike.wan @ 2021-03-19 12:56 UTC (permalink / raw)
  To: dledford, jgg; +Cc: linux-rdma, todd.rimmer, Kaike Wan

From: Kaike Wan <kaike.wan@intel.com>

To improve scalibity of RDMA transaction, there will be only one
connection between any two nodes. Within each node pair, one node will
be the client and the other node will be the server, depending the
lids/gids of the two nodes. However, to make best use of the bandwidth,
each connection could have multiple RC QPs to share among all of the
processes within a job. Connection is established through the IB CM
interface.

This patch adds the following functions:
- Listerner functions to wait for any IB CM requests from the same
  job.
- Client functions to send IB CM requests.
- Functions to manage the lifetime of the connection object.
- Functions to manage RC QPs.
- IB CM event handlers for client and server.

Signed-off-by: Todd Rimmer <todd.rimmer@intel.com>
Signed-off-by: Kaike Wan <kaike.wan@intel.com>
---
 drivers/infiniband/ulp/rv/Makefile     |    2 +-
 drivers/infiniband/ulp/rv/rv_conn.c    | 3037 ++++++++++++++++++++++++
 drivers/infiniband/ulp/rv/rv_file.c    |  125 +
 drivers/infiniband/ulp/rv/rv_rdma.c    |  103 +
 drivers/infiniband/ulp/rv/trace.h      |    2 +
 drivers/infiniband/ulp/rv/trace_conn.h |  529 +++++
 drivers/infiniband/ulp/rv/trace_rdma.h |  129 +
 drivers/infiniband/ulp/rv/trace_user.h |   57 +-
 8 files changed, 3982 insertions(+), 2 deletions(-)
 create mode 100644 drivers/infiniband/ulp/rv/rv_conn.c
 create mode 100644 drivers/infiniband/ulp/rv/rv_rdma.c
 create mode 100644 drivers/infiniband/ulp/rv/trace_conn.h
 create mode 100644 drivers/infiniband/ulp/rv/trace_rdma.h

diff --git a/drivers/infiniband/ulp/rv/Makefile b/drivers/infiniband/ulp/rv/Makefile
index 677b113c0666..7211e6975d1d 100644
--- a/drivers/infiniband/ulp/rv/Makefile
+++ b/drivers/infiniband/ulp/rv/Makefile
@@ -4,6 +4,6 @@
 #
 obj-$(CONFIG_INFINIBAND_RV) += rv.o
 
-rv-y := rv_main.o rv_mr_cache.o rv_file.o rv_mr.o trace.o
+rv-y := rv_main.o rv_mr_cache.o rv_file.o rv_mr.o trace.o rv_rdma.o rv_conn.o
 
 CFLAGS_trace.o = -I$(src)
diff --git a/drivers/infiniband/ulp/rv/rv_conn.c b/drivers/infiniband/ulp/rv/rv_conn.c
new file mode 100644
index 000000000000..842b0369125c
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/rv_conn.c
@@ -0,0 +1,3037 @@
+// SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause)
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+
+#define DRAIN_TIMEOUT 5 /* in seconds */
+
+#include <rdma/ib_marshall.h>
+#include <linux/nospec.h>
+
+#include "rv.h"
+#include "trace.h"
+
+static int rv_resolve_ip(struct rv_sconn *sconn);
+static int rv_err_qp(struct ib_qp *qp);
+static int rv_create_qp(int rv_inx, struct rv_sconn *sconn,
+			struct rv_job_dev *jdev);
+static void rv_destroy_qp(struct rv_sconn *sconn);
+static int rv_sconn_can_reconn(struct rv_sconn *sconn);
+static void rv_sconn_timeout_func(struct timer_list *timer);
+static void rv_sconn_timeout_work(struct work_struct *work);
+static void rv_sconn_delay_func(struct timer_list *timer);
+static void rv_sconn_delay_work(struct work_struct *work);
+static void rv_sconn_hb_func(struct timer_list *timer);
+static void rv_sconn_hb_work(struct work_struct *work);
+static void rv_hb_done(struct ib_cq *cq, struct ib_wc *wc);
+static void rv_sconn_enter_disconnecting(struct rv_sconn *sconn,
+					 const char *reason);
+static void rv_sconn_done_disconnecting(struct rv_sconn *sconn);
+static void rv_sconn_drain_timeout_func(struct timer_list *timer);
+
+static int rv_cm_handler(struct ib_cm_id *id, const struct ib_cm_event *evt);
+static void rv_send_req(struct rv_sconn *sconn);
+static void rv_qp_event(struct ib_event *event, void *context);
+static void rv_cq_event(struct ib_event *event, void *context);
+
+static void rv_sconn_free_primary_path(struct rv_sconn *sconn)
+{
+	kfree(sconn->primary_path);
+	sconn->primary_path = NULL;
+}
+
+/*
+ * cleanup a rv_sconn - a careful dance to shutdown all sconn activity
+ * rv_conn.kref is already 0, all sconn QP/resolve/CM callbacks
+ * will test sconn->parent->kref and return without starting new work
+ * also CM listener callback won't accept new REQ for rv_conn
+ * after rdma_addr_cancel - no resolver cb in flight nor scheduled
+ * after ib_destroy_cm_id - CM will ensure no callbacks active
+ * after rv_destroy_qp - QP empty/drained, no more QP events, no more CQEs
+ */
+static void rv_sconn_deinit(struct rv_sconn *sconn)
+{
+	trace_rv_sconn_deinit(sconn, sconn->index,
+			      sconn->qp ? sconn->qp->qp_num : 0,
+			      sconn->parent, sconn->flags, (u32)sconn->state,
+			      sconn->cm_id, sconn->resolver_retry_left);
+
+	del_timer_sync(&sconn->drain_timer);
+	del_timer_sync(&sconn->conn_timer);
+	del_timer_sync(&sconn->delay_timer);
+	del_timer_sync(&sconn->hb_timer);
+
+	if (sconn->state == RV_RESOLVING)
+		rdma_addr_cancel(&sconn->dev_addr);
+
+	if (sconn->cm_id) {
+		ib_destroy_cm_id(sconn->cm_id);
+		sconn->cm_id = NULL;
+	}
+
+	rv_destroy_qp(sconn);
+
+	rv_sconn_free_primary_path(sconn);
+}
+
+/*
+ * We flush wq2 to ensure all prior QP drain/destroy workitems items
+ * (especially those for sconn's in our conn) are done before
+ * we free the conn.  This avoids late RQ CQEs from dereferencing sconn
+ * after it has been freed.
+ */
+static void rv_handle_free_conn(struct work_struct *work)
+{
+	struct rv_conn *conn = container_of(work, struct rv_conn, free_work);
+
+	trace_rv_conn_release(conn, conn->rem_addr, conn->ah.is_global,
+			      conn->ah.dlid,
+			      be64_to_cpu(*((__be64 *)&conn->ah.grh.dgid[0])),
+			      be64_to_cpu(*((__be64 *)&conn->ah.grh.dgid[8])),
+			      conn->num_conn, conn->next,
+			      conn->jdev, kref_read(&conn->kref));
+	rv_flush_work2();
+	kfree_rcu(conn, rcu);
+}
+
+static void rv_conn_release(struct rv_conn *conn)
+{
+	int i;
+
+	trace_rv_conn_release(conn, conn->rem_addr, conn->ah.is_global,
+			      conn->ah.dlid,
+			      be64_to_cpu(*((__be64 *)&conn->ah.grh.dgid[0])),
+			      be64_to_cpu(*((__be64 *)&conn->ah.grh.dgid[8])),
+			      conn->num_conn, conn->next,
+			      conn->jdev, kref_read(&conn->kref));
+
+	mutex_lock(&conn->jdev->conn_list_mutex);
+	list_del_rcu(&conn->conn_entry);
+	mutex_unlock(&conn->jdev->conn_list_mutex);
+
+	for (i = 0; i < conn->num_conn; i++)
+		rv_sconn_deinit(&conn->sconn_arr[i]);
+	rv_job_dev_put(conn->jdev);
+	rv_queue_work3(&conn->free_work);
+}
+
+/*
+ * Since this function may be called from rv_cm_handler(), we can't call
+ * rv_conn_release() directly to destroy the cm_id (and wait on cm handler
+ * mutex).
+ * Instead, put the cleanup on a workqueue thread.
+ */
+static void rv_conn_schedule_release(struct kref *kref)
+{
+	struct rv_conn *conn = container_of(kref, struct rv_conn, kref);
+
+	rv_queue_work(&conn->put_work);
+}
+
+void rv_conn_put(struct rv_conn *conn)
+{
+	kref_put(&conn->kref, rv_conn_schedule_release);
+}
+
+/* return 0 if successful get, return -ENXIO if object going away */
+int rv_conn_get_check(struct rv_conn *conn)
+{
+	return kref_get_unless_zero(&conn->kref) ? 0 : -ENXIO;
+}
+
+void rv_conn_get(struct rv_conn *conn)
+{
+	kref_get(&conn->kref);
+}
+
+/* we can get away with a quick read of state without the mutex */
+static int rv_sconn_connected(struct rv_sconn *sconn)
+{
+	switch (sconn->state) {
+	case RV_CONNECTED:
+		return 1;
+	case RV_ERROR:
+		return -EIO;
+	default:
+		return test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags);
+	}
+}
+
+static int rv_conn_connected(struct rv_conn *conn)
+{
+	int i;
+	int ret;
+
+	for (i = 0; i < conn->num_conn; i++) {
+		ret = rv_sconn_connected(&conn->sconn_arr[i]);
+		if (ret <= 0)
+			return ret;
+	}
+	return 1;
+}
+
+/*
+ * Returns 1 if gid1 > gid 2
+ * Returns 0 if the same
+ * Returns -1 if gid1 < gid2
+ */
+int cmp_gid(const void *gid1, const void *gid2)
+{
+	u64 *subn1, *subn2, *ifid1, *ifid2;
+
+	subn1 = (u64 *)gid1;
+	ifid1 = (u64 *)gid1 + 1;
+
+	subn2 = (u64 *)gid2;
+	ifid2 = (u64 *)gid2 + 1;
+
+	if (*subn1 != *subn2) {
+		if (*subn1 > *subn2)
+			return 1;
+		else
+			return -1;
+	}
+	if (*ifid1 > *ifid2)
+		return 1;
+	else if (*ifid2 > *ifid1)
+		return -1;
+	else
+		return 0;
+}
+
+/* in microseconds */
+static u64 rv_sconn_time_elapsed(struct rv_sconn *sconn)
+{
+	return ktime_us_delta(ktime_get(), sconn->start_time);
+}
+
+/*
+ * LID is 10 characters + \0
+ * IPv4 is 15 + \0
+ * IPv6 is 39 + \0
+ */
+#define RV_MAX_ADDR_STR 40
+
+static char *show_gid(char *buf, size_t size, const u8 *gid)
+{
+	if (ipv6_addr_v4mapped((struct in6_addr *)gid))
+		snprintf(buf, size, "%pI4", &gid[12]);
+	else
+		snprintf(buf, size, "%pI6", gid);
+	return buf;
+}
+
+static char *show_rem_addr(char *buf, size_t size, struct rv_sconn *sconn)
+{
+	struct rv_conn *conn = sconn->parent;
+
+	if (!conn->ah.is_global)
+		snprintf(buf, size, "LID 0x%x", conn->ah.dlid);
+	else
+		show_gid(buf, size, conn->ah.grh.dgid);
+	return buf;
+}
+
+static const char *get_device_name(struct rv_sconn *sconn)
+{
+	struct ib_device *ib_dev = sconn->parent->jdev->dev->ib_dev;
+
+	if (ib_dev)
+		return ib_dev->name;
+	else
+		return "unknown";
+}
+
+/*
+ * Move to the new state and handle basic transition activities
+ *
+ * rv_sconn.mutex must be held
+ * reason - used in log messages for transitions out of RV_CONNECTED
+ *	or to RV_ERROR
+ */
+static void rv_sconn_set_state(struct rv_sconn *sconn, enum rv_sconn_state new,
+			       const char *reason)
+{
+	enum rv_sconn_state old = sconn->state;
+	char buf[RV_MAX_ADDR_STR];
+
+	/* some log messages for major transitions */
+	if (old == RV_CONNECTED && new != RV_CONNECTED)
+		rv_conn_err(sconn,
+			    "Conn Lost to %s via %s: sconn inx %u qp %u: %s\n",
+			    show_rem_addr(buf, sizeof(buf), sconn),
+			    get_device_name(sconn), sconn->index,
+			    sconn->qp ? sconn->qp->qp_num : 0, reason);
+	if (old != RV_CONNECTED && new == RV_CONNECTED &&
+	    test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags))
+		rv_conn_err(sconn,
+			    "Reconnected to %s via %s: sconn index %u qp %u\n",
+			    show_rem_addr(buf, sizeof(buf), sconn),
+			    get_device_name(sconn), sconn->index,
+			    sconn->qp ? sconn->qp->qp_num : 0);
+	else if (old != RV_ERROR && new == RV_ERROR) {
+		if (test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags))
+			rv_conn_err(sconn,
+				    "Unable to Reconn to %s via %s: sconn %u qp %u: %s\n",
+				    show_rem_addr(buf, sizeof(buf), sconn),
+				    get_device_name(sconn), sconn->index,
+				    sconn->qp ? sconn->qp->qp_num : 0, reason);
+		else
+			rv_conn_err(sconn,
+				    "Unable to Connect to %s via %s: sconn %u qp %u: %s\n",
+				    show_rem_addr(buf, sizeof(buf), sconn),
+				    get_device_name(sconn), sconn->index,
+				    sconn->qp ? sconn->qp->qp_num : 0, reason);
+	}
+
+	/*
+	 * process exit from old state
+	 * elapsed time measured for success or failure
+	 */
+	if (old == RV_WAITING && new != RV_WAITING) {
+		if (test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags)) {
+			u64 elapsed = rv_sconn_time_elapsed(sconn);
+
+			sconn->stats.rewait_time += elapsed;
+			if (elapsed > sconn->stats.max_rewait_time)
+				sconn->stats.max_rewait_time = elapsed;
+		} else {
+			sconn->stats.wait_time = rv_sconn_time_elapsed(sconn);
+		}
+	} else if (old == RV_RESOLVING && new != RV_RESOLVING) {
+		if (test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags)) {
+			u64 elapsed = rv_sconn_time_elapsed(sconn);
+
+			sconn->stats.reresolve_time += elapsed;
+			if (elapsed > sconn->stats.max_reresolve_time)
+				sconn->stats.max_reresolve_time = elapsed;
+		} else {
+			sconn->stats.resolve_time =
+				rv_sconn_time_elapsed(sconn);
+		}
+	} else if (old == RV_CONNECTING && new != RV_CONNECTING) {
+		if (test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags)) {
+			u64 elapsed = rv_sconn_time_elapsed(sconn);
+
+			sconn->stats.reconnect_time += elapsed;
+			if (elapsed > sconn->stats.max_reconnect_time)
+				sconn->stats.max_reconnect_time = elapsed;
+		} else {
+			sconn->stats.connect_time =
+				rv_sconn_time_elapsed(sconn);
+		}
+	} else if (old == RV_CONNECTED && new != RV_CONNECTED) {
+		del_timer_sync(&sconn->hb_timer);
+		sconn->stats.connected_time += rv_sconn_time_elapsed(sconn);
+		if (new != RV_ERROR) {
+			/* reconnect starts on 1st exit from CONNECTED */
+			sconn->conn_timer.expires = jiffies +
+				       sconn->parent->jdev->reconnect_timeout *
+				       HZ;
+			add_timer(&sconn->conn_timer);
+		}
+	} else if (old == RV_DISCONNECTING && new != RV_DISCONNECTING) {
+		del_timer_sync(&sconn->drain_timer);
+	}
+
+	/* process entry to new state */
+	if (old != RV_WAITING && new == RV_WAITING) {
+		sconn->start_time = ktime_get();
+	} else if (old != RV_RESOLVING && new == RV_RESOLVING) {
+		sconn->start_time = ktime_get();
+	} else if (old != RV_CONNECTING && new == RV_CONNECTING) {
+		sconn->start_time = ktime_get();
+	} else if (old != RV_CONNECTED && new == RV_CONNECTED) {
+		if (test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags))
+			sconn->stats.conn_recovery++;
+		sconn->start_time = ktime_get();
+		set_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags);
+		del_timer_sync(&sconn->conn_timer);
+	} else if (old != RV_DELAY && new == RV_DELAY) {
+		sconn->delay_timer.expires = jiffies + RV_RECONNECT_DELAY;
+		add_timer(&sconn->delay_timer);
+	} else if (old != RV_ERROR && new == RV_ERROR) {
+		del_timer_sync(&sconn->hb_timer);
+		del_timer_sync(&sconn->conn_timer);
+		del_timer_sync(&sconn->delay_timer);
+		if (sconn->qp) {
+			/* this will trigger the QP to self drain */
+			rv_err_qp(sconn->qp);
+			set_bit(RV_SCONN_DRAINING, &sconn->flags);
+		}
+	}
+
+	sconn->state = new;
+	trace_rv_sconn_set_state(sconn, sconn->index,
+				 sconn->qp ? sconn->qp->qp_num : 0,
+				 sconn->parent, sconn->flags,
+				 (u32)sconn->state, sconn->cm_id,
+				 sconn->resolver_retry_left);
+}
+
+static int rv_sconn_move_qp_to_rtr(struct rv_sconn *sconn, u32 *psn)
+{
+	struct ib_qp_attr qp_attr;
+	int attr_mask = 0;
+	int ret;
+
+	/* move QP to INIT */
+	memset(&qp_attr, 0, sizeof(qp_attr));
+	qp_attr.qp_state = IB_QPS_INIT;
+	ret = ib_cm_init_qp_attr(sconn->cm_id, &qp_attr, &attr_mask);
+	if (ret) {
+		rv_conn_err(sconn, "Failed to init qp_attr for INIT: %d\n",
+			    ret);
+		return ret;
+	}
+	trace_rv_msg_qp_rtr(sconn, sconn->index, "pkey_index + sconn",
+			    (u64)qp_attr.pkey_index, (u64)sconn);
+	ret = ib_modify_qp(sconn->qp, &qp_attr, attr_mask);
+	if (ret) {
+		rv_conn_err(sconn, "Failed to move qp %u into INIT: %d\n",
+			    sconn->qp ? sconn->qp->qp_num : 0,  ret);
+		return ret;
+	}
+
+	/* move QP to RTR */
+	memset(&qp_attr, 0, sizeof(qp_attr));
+	qp_attr.qp_state = IB_QPS_RTR;
+	ret = ib_cm_init_qp_attr(sconn->cm_id, &qp_attr, &attr_mask);
+	if (ret) {
+		rv_conn_err(sconn, "Failed to init qp_attr for RTR: %d\n", ret);
+		return ret;
+	}
+	if (psn) {
+		*psn = prandom_u32() & 0xffffff;
+		qp_attr.rq_psn = *psn;
+	}
+	trace_rv_msg_qp_rtr(sconn, sconn->index, "dlid | dqp_num, mtu | rq_psn",
+			    (u64)(qp_attr.ah_attr.ib.dlid |
+				  ((u64)qp_attr.dest_qp_num) << 32),
+			    (u64)(qp_attr.path_mtu |
+				  ((u64)qp_attr.rq_psn) << 32));
+	ret = ib_modify_qp(sconn->qp, &qp_attr, attr_mask);
+	if (ret) {
+		rv_conn_err(sconn, "Failed to move qp %u into RTR: %d\n",
+			    sconn->qp ? sconn->qp->qp_num : 0,  ret);
+		return ret;
+	}
+
+	/* post recv WQEs */
+	ret = rv_drv_prepost_recv(sconn);
+	if (ret)
+		rv_conn_err(sconn, "Failed to prepost qp %u recv WQEs: %d\n",
+			    sconn->qp ? sconn->qp->qp_num : 0,  ret);
+
+	return ret;
+}
+
+static int rv_sconn_move_qp_to_rts(struct rv_sconn *sconn)
+{
+	struct ib_qp_attr qp_attr;
+	int attr_mask;
+	int ret;
+
+	memset(&qp_attr, 0, sizeof(qp_attr));
+	qp_attr.qp_state = IB_QPS_RTS;
+	ret = ib_cm_init_qp_attr(sconn->cm_id, &qp_attr, &attr_mask);
+	if (ret) {
+		rv_conn_err(sconn, "Failed to init qp_attr for RTS: %d\n", ret);
+		return ret;
+	}
+	ret = ib_modify_qp(sconn->qp, &qp_attr, attr_mask);
+	if (ret) {
+		rv_conn_err(sconn, "Failed to move qp %u into RTS: %d\n",
+			    sconn->qp ? sconn->qp->qp_num : 0,  ret);
+		return ret;
+	}
+	return ret;
+}
+
+/*
+ * validate REP basics
+ * - private_data format and version
+ *	rev must always be an exact version we support
+ *	reject rev 0 & 1, only support rev 2
+ * - SRQ
+ */
+static int rv_check_rep_basics(struct rv_sconn *sconn,
+			       const struct ib_cm_rep_event_param *param,
+			       struct rv_rep_priv_data *priv_data)
+{
+	if (priv_data->magic != RV_PRIVATE_DATA_MAGIC) {
+		rv_conn_err(sconn,
+			    "Inval CM REP recv: magic 0x%llx expected 0x%llx\n",
+			    priv_data->magic, RV_PRIVATE_DATA_MAGIC);
+		return -EINVAL;
+	}
+	if (priv_data->ver != RV_PRIVATE_DATA_VER) {
+		rv_conn_err(sconn,
+			    "Invalid CM REP recv: rv version %d expected %d\n",
+			    priv_data->ver, RV_PRIVATE_DATA_VER);
+		return -EINVAL;
+	}
+	if (param->srq) {
+		rv_conn_err(sconn, "Invalid srq %d\n", param->srq);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Client side inbound CM REP handler
+ * caller must hold a rv_conn reference and the sconn->mutex
+ * This func does not release that ref
+ */
+static void rv_cm_rep_handler(struct rv_sconn *sconn,
+			      const struct ib_cm_rep_event_param *param,
+			      const void *private_data)
+{
+	int ret;
+	struct rv_job_dev *jdev = sconn->parent->jdev;
+	struct rv_rep_priv_data *priv_data =
+				 (struct rv_rep_priv_data *)private_data;
+	const char *reason;
+
+	if (rv_check_rep_basics(sconn, param, priv_data)) {
+		reason = "invalid REP";
+		goto rej;
+	}
+
+	if (sconn->state != RV_CONNECTING) {
+		reason = "unexpected REP";
+		goto rej;
+	}
+
+	if (rv_sconn_move_qp_to_rtr(sconn, NULL))
+		goto err;
+
+	if (rv_sconn_move_qp_to_rts(sconn))
+		goto err;
+
+	ret = ib_send_cm_rtu(sconn->cm_id, NULL, 0);
+	if (ret) {
+		rv_conn_err(sconn, "Failed to send cm RTU: %d\n", ret);
+		goto err;
+	}
+	sconn->stats.rtu_sent++;
+	trace_rv_msg_cm_rep_handler(sconn, sconn->index, "Sending RTU", 0,
+				    (u64)sconn);
+	if (jdev->hb_interval) {
+		sconn->hb_timer.expires = jiffies +
+					  msecs_to_jiffies(jdev->hb_interval);
+		add_timer(&sconn->hb_timer);
+	}
+	rv_sconn_set_state(sconn, RV_CONNECTED, "");
+	return;
+
+err:
+	/* do not try to retry/recover for fundamental QP errors */
+	if (!ib_send_cm_rej(sconn->cm_id, IB_CM_REJ_INSUFFICIENT_RESP_RESOURCES,
+			    NULL, 0, NULL, 0)) {
+		u64 val = (u64)IB_CM_REJ_INSUFFICIENT_RESP_RESOURCES;
+
+		sconn->stats.rej_sent++;
+		trace_rv_msg_cm_rep_handler(sconn, sconn->index,
+					    "Sending REJ reason",
+					    val, (u64)sconn);
+	}
+	rv_sconn_set_state(sconn, RV_ERROR, "local error handling REP");
+	return;
+
+rej:
+	if (!ib_send_cm_rej(sconn->cm_id, IB_CM_REJ_CONSUMER_DEFINED,
+			    NULL, 0, NULL, 0)) {
+		sconn->stats.rej_sent++;
+		trace_rv_msg_cm_rep_handler(sconn, sconn->index,
+					    "Sending REJ reason",
+					(u64)IB_CM_REJ_CONSUMER_DEFINED,
+					(u64)sconn);
+	}
+	rv_sconn_set_state(sconn, RV_ERROR, reason);
+}
+
+/*
+ * validate REQ basics
+ * - private_data format and version
+ *	reject rev 0 & 1
+ *	accept >=2, assume future versions will be forward compatible
+ * - QP type and APM
+ */
+static int rv_check_req_basics(struct ib_cm_id *id,
+			       const struct ib_cm_req_event_param *param,
+			       struct rv_req_priv_data *priv_data)
+{
+	if (priv_data->magic != RV_PRIVATE_DATA_MAGIC) {
+		rv_cm_err(id,
+			  "Inval CM REQ recv: magic 0x%llx expected 0x%llx\n",
+			  priv_data->magic, RV_PRIVATE_DATA_MAGIC);
+		return -EINVAL;
+	}
+	if (priv_data->ver < RV_PRIVATE_DATA_VER) {
+		rv_cm_err(id,
+			  "Invalid CM REQ recv: rv version %d expected %d\n",
+			  priv_data->ver, RV_PRIVATE_DATA_VER);
+		return -EINVAL;
+	}
+
+	if (param->qp_type != IB_QPT_RC || param->srq) {
+		rv_cm_err(id,
+			  "Invalid qp_type 0x%x or srq %d\n",
+			  param->qp_type, param->srq);
+		return -EINVAL;
+	}
+
+	if (param->alternate_path) {
+		rv_cm_err(id,
+			  "Invalid CM REQ recv: alt path not allowed\n");
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/* validate REQ primary_path against sconn->conn->ah from create_conn */
+static int rv_sconn_req_check_ah(const struct rv_sconn *sconn,
+				 const struct sa_path_rec *path)
+{
+	struct rv_conn *conn = sconn->parent;
+	int ret = -EINVAL;
+
+#define RV_CHECK(f1, f2) (path->f1 != conn->ah.f2)
+#define RV_CHECK_BE32(f1, f2) (be32_to_cpu(path->f1) != conn->ah.f2)
+#define RV_REPORT(f1, f2, text, format) \
+		rv_conn_err(sconn, "CM REQ inconsistent " text " " format \
+			    " with create_conn " format "\n", \
+			    path->f1, conn->ah.f2)
+
+	if (RV_CHECK(sl, sl))
+		RV_REPORT(sl, sl, "SL", "%u");
+	else if (conn->ah.is_global &&
+		 RV_CHECK(traffic_class, grh.traffic_class))
+		RV_REPORT(traffic_class, grh.traffic_class, "traffic_class",
+			  "%u");
+	else if (conn->ah.is_global &&
+		 RV_CHECK_BE32(flow_label, grh.flow_label))
+		RV_REPORT(flow_label, grh.flow_label, "flow_label", "0x%x");
+		/* for RoCE hop_limit overridden by resolver */
+	else if (conn->ah.is_global && !rv_jdev_protocol_roce(conn->jdev) &&
+		 RV_CHECK(hop_limit, grh.hop_limit))
+		RV_REPORT(hop_limit, grh.hop_limit, "hop_limit", "%u");
+	else if (RV_CHECK(rate, static_rate))
+		RV_REPORT(rate, static_rate, "rate", "%u");
+#undef RV_CHECK
+#undef RV_CHECK_BE32
+#undef RV_REPORT
+	else
+		ret = 0;
+	return ret;
+}
+
+/* validate REQ primary_path against sconn->path from cm_connect */
+static int rv_sconn_req_check_path(const struct rv_sconn *sconn,
+				   const struct sa_path_rec *path)
+{
+	int ret = -EINVAL;
+
+#define RV_CHECK(field) (path->field != sconn->path.field)
+#define RV_REPORT(field, text, format) \
+		     rv_conn_err(sconn, "CM REQ inconsistent " text " " format \
+				 " with connect " format "\n", \
+				 path->field, sconn->path.field)
+	if (RV_CHECK(pkey))
+		RV_REPORT(pkey, "pkey", "0x%x");
+	else if (RV_CHECK(mtu))
+		RV_REPORT(mtu, "mtu", "%u");
+	else if (RV_CHECK(sl))
+		RV_REPORT(sl, "SL", "%u");
+	else if (RV_CHECK(traffic_class))
+		RV_REPORT(traffic_class, "traffic_class", "%u");
+	else if (RV_CHECK(flow_label))
+		RV_REPORT(flow_label, "flow_label", "0x%x");
+	else if (RV_CHECK(rate))
+		RV_REPORT(rate, "rate", "%u");
+		/* for RoCE hop_limit overridden by resolver */
+	else if (!rv_jdev_protocol_roce(sconn->parent->jdev) &&
+		 RV_CHECK(hop_limit))
+		RV_REPORT(hop_limit, "hop_limit", "%u");
+	else if (path->packet_life_time < sconn->path.packet_life_time)
+		RV_REPORT(packet_life_time, "packet_life_time", "%u");
+#undef RV_CHECK
+#undef RV_REPORT
+	else
+		ret = 0;
+	return ret;
+}
+
+/*
+ * caller must hold a rv_conn reference and sconn->mutex.
+ * This func does not release the ref nor mutex
+ * The private data version must be <= version in REQ and reflect a
+ * version both client and listener support.
+ * We currently only support version 2.
+ */
+static void rv_send_rep(struct rv_sconn *sconn,
+			const struct ib_cm_req_event_param *param, u32 psn)
+{
+	struct ib_cm_rep_param rep;
+	struct rv_rep_priv_data priv_data = {
+			.magic = RV_PRIVATE_DATA_MAGIC,
+			.ver = RV_PRIVATE_DATA_VER,
+		};
+	int ret;
+
+	memset(&rep, 0, sizeof(rep));
+	rep.qp_num = sconn->qp->qp_num;
+	rep.rnr_retry_count = min_t(unsigned int, 7, param->rnr_retry_count);
+	rep.flow_control = 1;
+	rep.failover_accepted = 0;
+	rep.srq = !!(sconn->qp->srq);
+	rep.responder_resources = 0;
+	rep.initiator_depth = 0;
+	rep.starting_psn = psn;
+
+	rep.private_data = &priv_data;
+	rep.private_data_len = sizeof(priv_data);
+
+	ret = ib_send_cm_rep(sconn->cm_id, &rep);
+	if (ret) {
+		rv_conn_err(sconn, "Failed to send CM REP: %d\n", ret);
+		goto err;
+	}
+	sconn->stats.rep_sent++;
+	trace_rv_msg_cm_req_handler(sconn, sconn->index, "Sending REP", 0,
+				    (u64)sconn);
+	rv_sconn_set_state(sconn, RV_CONNECTING, "");
+	return;
+
+err:
+	if (!ib_send_cm_rej(sconn->cm_id, IB_CM_REJ_INSUFFICIENT_RESP_RESOURCES,
+			    NULL, 0, NULL, 0)) {
+		u64 val =  (u64)IB_CM_REJ_INSUFFICIENT_RESP_RESOURCES;
+
+		sconn->stats.rej_sent++;
+		trace_rv_msg_cm_req_handler(sconn, sconn->index,
+					    "Sending REJ reason",
+					    val, (u64)sconn);
+	}
+	rv_sconn_set_state(sconn, RV_ERROR, "local error sending REP");
+}
+
+/*
+ * Server side inbound CM REQ handler
+ * special cases:
+ *	if RV_CONNECTING - RTU got lost and remote trying again alraedy
+ *	if RV_CONNECTED - remote figured out connection is down 1st
+ * return:
+ *	0 - sconn has taken ownership of the cm_id
+ *	<0 - CM should destroy the id
+ * rv_find_sconn_from_req validates REQ against jdev:
+ *	job key, local port, local device, sconn index,
+ *	remote address (dgid or dlid), hb_interval
+ * For valid REQs, we establish a new IB CM handler for subsequent CM events
+ */
+static int rv_cm_req_handler(struct ib_cm_id *id,
+			     const struct ib_cm_req_event_param *param,
+			     void *private_data)
+{
+	struct rv_req_priv_data *priv_data =
+				 (struct rv_req_priv_data *)private_data;
+	struct rv_sconn *sconn = NULL;
+	u32 psn;
+
+	if (rv_check_req_basics(id, param, priv_data))
+		goto rej;
+
+	sconn = rv_find_sconn_from_req(id, param, priv_data);
+	if (!sconn) {
+		rv_cm_err(id, "Could not find conn for the request\n");
+		goto rej;
+	}
+
+	mutex_lock(&sconn->mutex);
+
+	sconn->stats.cm_evt_cnt[IB_CM_REQ_RECEIVED]++;
+	trace_rv_sconn_req_handler(sconn, sconn->index,
+				   sconn->qp ? sconn->qp->qp_num : 0,
+				   sconn->parent, sconn->flags,
+				   (u32)sconn->state, id,
+				   sconn->resolver_retry_left);
+
+	if (rv_sconn_req_check_ah(sconn, param->primary_path))
+		goto rej;
+	if (sconn->path.dlid &&
+	    rv_sconn_req_check_path(sconn, param->primary_path))
+		goto rej;
+
+	switch (sconn->state) {
+	case RV_WAITING:
+		break;
+	case RV_CONNECTING:
+	case RV_CONNECTED:
+		if (rv_sconn_can_reconn(sconn))
+			rv_sconn_enter_disconnecting(sconn,
+						     "remote reconnecting");
+		/* FALLSTHROUGH */
+	default:
+		goto rej;
+	}
+	if (!sconn->qp)
+		goto rej;
+
+	sconn->cm_id = id;
+	id->context = sconn;
+
+	id->cm_handler = rv_cm_handler;
+	if (rv_sconn_move_qp_to_rtr(sconn, &psn))
+		goto err;
+
+	rv_send_rep(sconn, param, psn);
+	mutex_unlock(&sconn->mutex);
+	rv_conn_put(sconn->parent);
+	return 0;
+
+err:
+	if (!ib_send_cm_rej(id, IB_CM_REJ_INSUFFICIENT_RESP_RESOURCES, NULL, 0,
+			    NULL, 0)) {
+		u64 val =  (u64)IB_CM_REJ_INSUFFICIENT_RESP_RESOURCES;
+
+		sconn->stats.rej_sent++;
+		trace_rv_msg_cm_req_handler(sconn, sconn->index,
+					    "Sending REJ reason",
+					    val, (u64)sconn);
+	}
+	rv_sconn_set_state(sconn, RV_ERROR, "local error handling REQ");
+	mutex_unlock(&sconn->mutex);
+	rv_conn_put(sconn->parent);
+	return 0;
+
+rej:
+	if (!ib_send_cm_rej(id, IB_CM_REJ_CONSUMER_DEFINED, NULL, 0, NULL, 0)) {
+		if (sconn)
+			sconn->stats.rej_sent++;
+		trace_rv_msg_cm_req_handler(sconn, sconn ? sconn->index : 0,
+					    "Sending REJ reason",
+					    (u64)IB_CM_REJ_CONSUMER_DEFINED,
+					    (u64)sconn);
+	}
+	if (sconn) {
+		mutex_unlock(&sconn->mutex);
+		rv_conn_put(sconn->parent);
+	}
+	return -EINVAL;
+}
+
+/* must hold sconn->mutex */
+static int rv_sconn_can_reconn(struct rv_sconn *sconn)
+{
+	return test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags) &&
+	       sconn->parent->jdev->reconnect_timeout &&
+	       rv_job_dev_has_users(sconn->parent->jdev);
+}
+
+/*
+ * Post a WR, rv_drain_done will handle when SQ is empty
+ * caller must hold a reference and QP must be in QPS_ERR
+ * an additional reference is established on behalf of the WR's CQ callback
+ */
+static int rv_start_drain_sq(struct rv_sconn *sconn)
+{
+	struct ib_rdma_wr swr = {
+		.wr = {
+			.wr_cqe	= &sconn->sdrain_cqe,
+			.opcode	= IB_WR_RDMA_WRITE,
+		},
+	};
+	int ret;
+
+	rv_conn_get(sconn->parent);
+	ret = ib_post_send(sconn->qp, &swr.wr, NULL);
+	if (ret) {
+		rv_conn_err(sconn, "failed to drain send queue: post %d\n",
+			    ret);
+		rv_conn_put(sconn->parent);
+	}
+	return ret;
+}
+
+/*
+ * Post a WR, rv_drain_done will handle when RQ is empty
+ * caller must hold a reference and QP must be in QPS_ERR
+ * an additional reference is established on behalf of the WR's CQ callback
+ */
+static int rv_start_drain_rq(struct rv_sconn *sconn)
+{
+	struct ib_recv_wr rwr = {
+		.wr_cqe	= &sconn->rdrain_cqe,
+	};
+	int ret;
+
+	rv_conn_get(sconn->parent);
+	ret = ib_post_recv(sconn->qp, &rwr, NULL);
+	if (ret) {
+		rv_conn_err(sconn, "failed to drain recv queue: post %d\n",
+			    ret);
+		rv_conn_put(sconn->parent);
+	}
+	return ret;
+}
+
+/* in soft IRQ context, a reference held on our behalf */
+static void rv_rq_drain_done(struct ib_cq *cq, struct ib_wc *wc)
+{
+	struct rv_sconn *sconn = container_of(wc->wr_cqe,
+					      struct rv_sconn, rdrain_cqe);
+
+	if (test_bit(RV_SCONN_DRAINING, &sconn->flags)) {
+		set_bit(RV_SCONN_RQ_DRAINED, &sconn->flags);
+		trace_rv_sconn_drain_done(sconn, sconn->index,
+					  sconn->qp ? sconn->qp->qp_num : 0,
+					  sconn->parent, sconn->flags,
+					  (u32)sconn->state, sconn->cm_id,
+					  sconn->resolver_retry_left);
+		if (test_bit(RV_SCONN_SQ_DRAINED, &sconn->flags)) {
+			del_timer_sync(&sconn->drain_timer);
+			rv_queue_work(&sconn->drain_work);
+			return;
+		}
+	}
+	rv_conn_put(sconn->parent);
+}
+
+/* in soft IRQ context, a reference held on our behalf */
+static void rv_sq_drain_done(struct ib_cq *cq, struct ib_wc *wc)
+{
+	struct rv_sconn *sconn = container_of(wc->wr_cqe,
+					      struct rv_sconn, sdrain_cqe);
+
+	if (test_bit(RV_SCONN_DRAINING, &sconn->flags)) {
+		set_bit(RV_SCONN_SQ_DRAINED, &sconn->flags);
+		trace_rv_sconn_drain_done(sconn, sconn->index,
+					  sconn->qp ? sconn->qp->qp_num : 0,
+					  sconn->parent, sconn->flags,
+					  (u32)sconn->state, sconn->cm_id,
+					  sconn->resolver_retry_left);
+		if (test_bit(RV_SCONN_RQ_DRAINED, &sconn->flags)) {
+			del_timer_sync(&sconn->drain_timer);
+			rv_queue_work(&sconn->drain_work);
+			return;
+		}
+	}
+	rv_conn_put(sconn->parent);
+}
+
+/*
+ * timeout exhausted on a drain CQE callback
+ * a rv_conn reference is held by the outstanding RQ and SQ drains
+ * we assume we have waited long enough that CQE callback is not coming
+ * and will not race with this func
+ */
+static void rv_sconn_drain_timeout_func(struct timer_list *timer)
+{
+	struct rv_sconn *sconn = container_of(timer, struct rv_sconn,
+					      drain_timer);
+
+	if (!sconn->parent)
+		return;
+	if (!test_bit(RV_SCONN_SQ_DRAINED, &sconn->flags) &&
+	    !test_bit(RV_SCONN_RQ_DRAINED, &sconn->flags))
+		rv_conn_put(sconn->parent);
+
+	if (!test_bit(RV_SCONN_RQ_DRAINED, &sconn->flags)) {
+		set_bit(RV_SCONN_RQ_DRAINED, &sconn->flags);
+		rv_conn_dbg(sconn,
+			    "drain recv queue sconn index %u qp %u conn %p\n",
+			    sconn->index, sconn->qp ? sconn->qp->qp_num : 0,
+			    sconn->parent);
+	}
+	if (!test_bit(RV_SCONN_SQ_DRAINED, &sconn->flags)) {
+		set_bit(RV_SCONN_SQ_DRAINED, &sconn->flags);
+		rv_conn_dbg(sconn,
+			    "drain send queue sconn index %u qp %u conn %p\n",
+			    sconn->index, sconn->qp ? sconn->qp->qp_num : 0,
+			    sconn->parent);
+	}
+	rv_queue_work(&sconn->drain_work);
+}
+
+/*
+ * must hold sconn->mutex and have a reference
+ * If QP is in QPS_RESET, nothing to do
+ * drain_lock makes sure no recv WQEs get reposted after our drain WQE
+ */
+static void rv_sconn_enter_disconnecting(struct rv_sconn *sconn,
+					 const char *reason)
+{
+	unsigned long flags;
+	int ret;
+
+	rv_sconn_set_state(sconn, RV_DISCONNECTING, reason);
+
+	ret = rv_err_qp(sconn->qp);
+	if (ret == 1) {
+		rv_sconn_done_disconnecting(sconn);
+		return;
+	} else if (ret) {
+		goto fail;
+	}
+
+	spin_lock_irqsave(&sconn->drain_lock, flags);
+	set_bit(RV_SCONN_DRAINING, &sconn->flags);
+	sconn->drain_timer.expires = jiffies + DRAIN_TIMEOUT * HZ;
+	add_timer(&sconn->drain_timer);
+
+	ret = rv_start_drain_rq(sconn);
+	ret |= rv_start_drain_sq(sconn);
+	spin_unlock_irqrestore(&sconn->drain_lock, flags);
+	if (!ret)
+		return;
+fail:
+	trace_rv_msg_enter_disconnect(sconn, sconn->index,
+				      "Unable to move QP to error", 0, 0);
+	rv_sconn_set_state(sconn, RV_ERROR, "unable to drain QP");
+}
+
+struct rv_dest_cm_work_item {
+	struct work_struct destroy_work;
+	struct rv_sconn *sconn;
+	struct ib_cm_id *cm_id;
+	struct ib_qp *qp;
+};
+
+/*
+ * destroy the CM_ID and the QP
+ * Once ib_destroy_cm_id returns, all CM callbacks are done
+ * Any WQEs/CQEs in flight must be drained before this handler is scheduled
+ */
+static void rv_handle_destroy_qp_cm(struct work_struct *work)
+{
+	struct rv_dest_cm_work_item *item = container_of(work,
+				struct rv_dest_cm_work_item, destroy_work);
+
+	ib_destroy_cm_id(item->cm_id);
+
+	ib_destroy_qp(item->qp);
+
+	rv_conn_put(item->sconn->parent);
+	kfree(item);
+}
+
+/*
+ * must hold sconn->mutex
+ * QP is now drained and no longer posting recv nor sends
+ * We start fresh with a new QP and cm_id.  This lets CM do its own
+ * timewait handling and avoids any stale packets arriving on our new QP.
+ * To conform to lock heirarchy, schedule actual destroy in WQ
+ * since can't destroy cm_id while holding sconn->mutex nor in CM callback.
+ */
+static void rv_sconn_done_disconnecting(struct rv_sconn *sconn)
+{
+	struct rv_dest_cm_work_item *item;
+	struct rv_job_dev *jdev = sconn->parent->jdev;
+	int ret;
+	struct ib_cm_id *id;
+
+	trace_rv_sconn_done_discon(sconn, sconn->index,
+				   sconn->qp ? sconn->qp->qp_num : 0,
+				   sconn->parent, sconn->flags,
+				   (u32)sconn->state, sconn->cm_id,
+				   sconn->resolver_retry_left);
+
+	item = kzalloc(sizeof(*item), GFP_KERNEL);
+	if (!item)
+		goto fail;
+	rv_conn_get(sconn->parent);
+	INIT_WORK(&item->destroy_work, rv_handle_destroy_qp_cm);
+	item->sconn = sconn;
+	item->cm_id = sconn->cm_id;
+	sconn->cm_id = NULL;
+	item->qp = sconn->qp;
+	sconn->qp = NULL;
+	rv_queue_work(&item->destroy_work);
+
+	clear_bit(RV_SCONN_DRAINING, &sconn->flags);
+	clear_bit(RV_SCONN_RQ_DRAINED, &sconn->flags);
+	clear_bit(RV_SCONN_SQ_DRAINED, &sconn->flags);
+
+	ret = rv_create_qp(RV_INVALID, sconn, jdev);
+	if (ret) {
+		rv_conn_err(sconn, "Failed to re-create qp: %d\n", ret);
+		goto fail;
+	}
+
+	if (test_bit(RV_SCONN_SERVER, &sconn->flags)) {
+		rv_sconn_set_state(sconn, RV_WAITING, "");
+		return;
+	}
+
+	id = ib_create_cm_id(jdev->dev->ib_dev, rv_cm_handler, sconn);
+	if (IS_ERR(id)) {
+		rv_conn_err(sconn, "Create CM ID failed\n");
+		goto fail;
+	}
+	sconn->cm_id = id;
+	rv_sconn_set_state(sconn, RV_DELAY, "");
+	return;
+
+fail:
+	rv_sconn_set_state(sconn, RV_ERROR, "local error disconnecting");
+}
+
+/* only allowed in RV_DISCONNECTING or RV_ERROR */
+static void rv_sconn_drain_work(struct work_struct *work)
+{
+	struct rv_sconn *sconn = container_of(work, struct rv_sconn,
+					      drain_work);
+
+	mutex_lock(&sconn->mutex);
+	if (sconn->state == RV_DISCONNECTING)
+		rv_sconn_done_disconnecting(sconn);
+	else
+		WARN_ON(sconn->state != RV_ERROR);
+	mutex_unlock(&sconn->mutex);
+
+	rv_conn_put(sconn->parent);
+}
+
+/*
+ * rv_cm_handler - The client Callback frunction from IB CM
+ * @cm_id: Handle for connection manager
+ * @event: The event it caught
+ * Be reminded that we can not destroy cm_id in this thread.
+ */
+static int rv_cm_handler(struct ib_cm_id *id, const struct ib_cm_event *evt)
+{
+	struct rv_sconn *sconn = id->context;
+
+	trace_rv_cm_event_handler((u32)evt->event, id, sconn);
+	if (!sconn || !sconn->parent)
+		return 0;
+	if (rv_conn_get_check(sconn->parent))
+		return 0;
+	trace_rv_sconn_cm_handler(sconn, sconn->index,
+				  sconn->qp ? sconn->qp->qp_num : 0,
+				  sconn->parent, sconn->flags,
+				  (u32)sconn->state, sconn->cm_id,
+				  sconn->resolver_retry_left);
+
+	mutex_lock(&sconn->mutex);
+	sconn->stats.cm_evt_cnt[min(evt->event, RV_CM_EVENT_UNEXP)]++;
+
+	if (sconn->cm_id != id)
+		goto unlock;
+
+	switch (evt->event) {
+	case IB_CM_REP_RECEIVED:
+		rv_cm_rep_handler(sconn, &evt->param.rep_rcvd,
+				  evt->private_data);
+		break;
+	case IB_CM_RTU_RECEIVED:
+	case IB_CM_USER_ESTABLISHED:
+		if (sconn->state != RV_CONNECTING) {
+			if (!ib_send_cm_dreq(id, NULL, 0)) {
+				sconn->stats.dreq_sent++;
+				trace_rv_msg_cm_handler(sconn, sconn->index,
+							"Sending DREQ", 0,
+							(u64)sconn);
+			}
+			rv_sconn_set_state(sconn, RV_ERROR, "unexpected RTU");
+		} else if (rv_sconn_move_qp_to_rts(sconn)) {
+			if (!ib_send_cm_dreq(id, NULL, 0)) {
+				sconn->stats.dreq_sent++;
+				trace_rv_msg_cm_handler(sconn, sconn->index,
+							"Sending DREQ", 0,
+							(u64)sconn);
+			}
+			rv_sconn_set_state(sconn, RV_ERROR,
+					   "local error handling RTU");
+		} else {
+			rv_sconn_set_state(sconn, RV_CONNECTED, "");
+		}
+		break;
+
+	case IB_CM_REQ_ERROR:
+		trace_rv_msg_cm_handler(sconn, sconn->index,
+					"Sending CM REQ failed, send_status",
+					(u64)evt->param.send_status,
+					(u64)sconn);
+		if (sconn->state == RV_CONNECTING && rv_sconn_can_reconn(sconn))
+			rv_sconn_enter_disconnecting(sconn, "no REQ response");
+		else
+			rv_sconn_set_state(sconn, RV_ERROR, "no REQ response");
+		break;
+	case IB_CM_REP_ERROR:
+		trace_rv_msg_cm_handler(sconn, sconn->index,
+					"Sending CM REP failed, send_status",
+					(u64)evt->param.send_status,
+					(u64)sconn);
+		if (sconn->state == RV_CONNECTING && rv_sconn_can_reconn(sconn))
+			rv_sconn_enter_disconnecting(sconn, "no REP response");
+		else
+			rv_sconn_set_state(sconn, RV_ERROR, "no REP response");
+		break;
+	case IB_CM_REJ_RECEIVED:
+		trace_rv_msg_cm_handler(sconn, sconn->index,
+					"CM REJ received reason",
+					(u64)evt->param.rej_rcvd.reason,
+					(u64)sconn);
+		if (sconn->state == RV_CONNECTING && rv_sconn_can_reconn(sconn))
+			rv_sconn_enter_disconnecting(sconn, "received REJ");
+		else
+			rv_sconn_set_state(sconn, RV_ERROR, "received REJ");
+		break;
+	case IB_CM_DREQ_RECEIVED:
+		if (!ib_send_cm_drep(id, NULL, 0)) {
+			sconn->stats.drep_sent++;
+			trace_rv_msg_cm_handler(sconn, sconn->index,
+						"Sending DREP", 0, (u64)sconn);
+		}
+
+		if (sconn->state != RV_DISCONNECTING) {
+			if ((sconn->state == RV_CONNECTED ||
+			     sconn->state == RV_CONNECTING) &&
+			    rv_sconn_can_reconn(sconn))
+				rv_sconn_enter_disconnecting(sconn,
+							     "received DREQ");
+			else
+				rv_sconn_set_state(sconn, RV_ERROR,
+						   "received DREQ");
+		}
+		break;
+
+	case IB_CM_TIMEWAIT_EXIT:
+		break;
+	case IB_CM_MRA_RECEIVED:
+		break;
+	case IB_CM_DREQ_ERROR:
+	case IB_CM_DREP_RECEIVED:
+		break;
+
+	case IB_CM_REQ_RECEIVED:
+	case IB_CM_LAP_ERROR:
+	case IB_CM_LAP_RECEIVED:
+	case IB_CM_APR_RECEIVED:
+		break;
+	case IB_CM_SIDR_REQ_ERROR:
+	case IB_CM_SIDR_REQ_RECEIVED:
+	case IB_CM_SIDR_REP_RECEIVED:
+		break;
+	default:
+		rv_conn_err(sconn, "Unhandled CM event %d\n", evt->event);
+		WARN_ON(1);
+		rv_sconn_set_state(sconn, RV_ERROR, "invalid CM event");
+		break;
+	}
+
+unlock:
+	mutex_unlock(&sconn->mutex);
+
+	rv_conn_put(sconn->parent);
+
+	return 0;
+}
+
+/*
+ * rv_cm_server_handler - The server callback function from IB CM
+ * @cm_id: Handle for connection manager. This is an newly created cm_id
+ *         For the new connection, which is different from the original
+ *         listener cm_id.
+ * @event: The event it caught
+ * It only handles incoming REQs. All other events should go to rv_cm_handler
+ */
+static int rv_cm_server_handler(struct ib_cm_id *id,
+				const struct ib_cm_event *evt)
+{
+	int ret = 0;
+
+	trace_rv_cm_event_server_handler((u32)evt->event, id, NULL);
+	switch (evt->event) {
+	case IB_CM_REQ_RECEIVED:
+		ret = rv_cm_req_handler(id, &evt->param.req_rcvd,
+					evt->private_data);
+		break;
+	default:
+		rv_cm_err(id, "Unhandled CM event %d\n", evt->event);
+		WARN_ON(1);
+		ret = -EINVAL;
+		break;
+	}
+
+	return ret;
+}
+
+static int rv_conn_cmp_params(int rv_inx, const struct rv_conn *conn,
+			      const struct rv_conn_create_params_in *param)
+{
+	if (param->rem_addr != conn->rem_addr) {
+		trace_rv_msg_cmp_params(rv_inx,
+					"rem_addr differ, skipping",
+					(u64)param->rem_addr,
+					(u64)conn->rem_addr);
+		return 1;
+	}
+
+	if (param->ah.is_global != conn->ah.is_global) {
+		trace_rv_msg_cmp_params(rv_inx,
+					"Global flags differ, skipping",
+					(u64)param->ah.is_global,
+					(u64)conn->ah.is_global);
+		return 1;
+	}
+
+	if (param->ah.is_global) {
+		if (cmp_gid(&param->ah.grh.dgid[0],
+			    &conn->ah.grh.dgid[0]) == 0) {
+			trace_rv_msg_cmp_params(rv_inx,
+						"Gid's are matching",
+						*(u64 *)&param->ah.grh.dgid[8],
+						*(u64 *)&conn->ah.grh.dgid[8]);
+			return 0;
+		}
+		trace_rv_msg_cmp_params(rv_inx, "Gid's do not match",
+					*(u64 *)&param->ah.grh.dgid[8],
+					*(u64 *)&conn->ah.grh.dgid[8]);
+		return 1;
+
+	} else {
+		if (param->ah.dlid == conn->ah.dlid) {
+			trace_rv_msg_cmp_params(rv_inx, "Found matching dlid",
+						(u64)param->ah.dlid,
+						(u64)conn->ah.dlid);
+			return 0;
+		}
+		trace_rv_msg_cmp_params(rv_inx, "DLID not matching",
+					(u64)param->ah.dlid,
+					(u64)conn->ah.dlid);
+		return 1;
+	}
+}
+
+/*
+ * Search the list for the GID or DLID in the AH
+ * Caller must hold rv_user.mutex
+ */
+static struct rv_conn *
+user_conn_exist(struct rv_user *rv, struct rv_conn_create_params_in *param)
+{
+	struct rv_conn *conn;
+
+	XA_STATE(xas, &rv->conn_xa, 0);
+
+	xas_for_each(&xas, conn, UINT_MAX) {
+		trace_rv_msg_conn_exist(rv->inx, "Conn found in list",
+					(u64)conn, 0);
+		if (!rv_conn_cmp_params(rv->inx, conn, param))
+			return conn;
+	}
+
+	return NULL;
+}
+
+struct rv_cq_event_work_item {
+	struct work_struct cq_event_work;
+	struct rv_sconn *sconn;
+	struct ib_event event;
+};
+
+/*
+ * CQ Async event callback worker
+ * Must make sure the CQs are still relevant as they could have changed
+ */
+static void rv_cq_event_work(struct work_struct *work)
+{
+	struct rv_cq_event_work_item *item = container_of(work,
+				struct rv_cq_event_work_item, cq_event_work);
+	struct rv_sconn *sconn = item->sconn;
+
+	trace_rv_sconn_cq_event(sconn, sconn->index,
+				sconn->qp ? sconn->qp->qp_num : 0,
+				sconn->parent, sconn->flags,
+				(u32)sconn->state, sconn->cm_id,
+				sconn->resolver_retry_left);
+
+	mutex_lock(&sconn->mutex);
+	if (sconn->send_cq != item->event.element.cq &&
+	    sconn->recv_cq != item->event.element.cq)
+		goto unlock;
+
+	switch (item->event.event) {
+	case IB_EVENT_CQ_ERR:
+		if (!ib_send_cm_dreq(sconn->cm_id, NULL, 0)) {
+			sconn->stats.dreq_sent++;
+			trace_rv_msg_cq_event(sconn, sconn->index,
+					      "Sending DREQ", 0, (u64)sconn);
+		}
+		rv_sconn_set_state(sconn, RV_ERROR, "CQ error");
+		break;
+	default:
+		break;
+	}
+unlock:
+	mutex_unlock(&sconn->mutex);
+	rv_conn_put(sconn->parent);
+	kfree(item);
+}
+
+/*
+ * CQ Async Event
+ * Non-premptable, so real work in WQ
+ */
+static void rv_cq_event(struct ib_event *event, void *context)
+{
+	struct rv_sconn *sconn = (struct rv_sconn *)context;
+	struct rv_cq_event_work_item *item;
+	char *cq_text;
+
+	if (!sconn || !sconn->parent)
+		return;
+	if (rv_conn_get_check(sconn->parent))
+		return;
+	if (sconn->send_cq == event->element.cq)
+		cq_text = "Send";
+	else if (sconn->recv_cq == event->element.cq)
+		cq_text = "Recv";
+	else
+		cq_text = "Unkn";
+
+	rv_conn_err(sconn,
+		    "%s CQ Event received: %s: sconn index %u qp %u\n",
+		    cq_text, ib_event_msg(event->event), sconn->index,
+		    event->element.qp->qp_num);
+
+	item = kzalloc(sizeof(*item), GFP_ATOMIC);
+	if (!item)
+		goto fail;
+	INIT_WORK(&item->cq_event_work, rv_cq_event_work);
+	item->sconn = sconn;
+	item->event = *event;
+	rv_queue_work(&item->cq_event_work);
+	return;
+
+fail:
+	rv_conn_err(sconn,
+		    "No mem for %s CQ Evt: %s: sconn index %u qp %u conn %p\n",
+		    cq_text, ib_event_msg(event->event), sconn->index,
+		    event->element.qp->qp_num, sconn->parent);
+	rv_conn_put(sconn->parent);
+}
+
+struct rv_qp_event_work_item {
+	struct work_struct qp_event_work;
+	struct rv_sconn *sconn;
+	struct ib_event event;
+};
+
+/*
+ * QP Async event callback worker
+ * Must make sure the QP is still relevant as it could have changed
+ * unfortnately only get LID_CHANGE, PORT_ERR, PORT_ACTIVE
+ * GID_CHANGE, at device level, but likely to get QP event soon after
+ */
+static void rv_qp_event_work(struct work_struct *work)
+{
+	struct rv_qp_event_work_item *item = container_of(work,
+				struct rv_qp_event_work_item, qp_event_work);
+	struct rv_sconn *sconn = item->sconn;
+
+	trace_rv_sconn_qp_event(sconn, sconn->index,
+				sconn->qp ? sconn->qp->qp_num : 0,
+				sconn->parent, sconn->flags,
+				(u32)sconn->state, sconn->cm_id,
+				sconn->resolver_retry_left);
+
+	mutex_lock(&sconn->mutex);
+	if (sconn->qp != item->event.element.qp)
+		goto unlock;
+
+	switch (item->event.event) {
+	case IB_EVENT_PATH_MIG:
+		if (sconn->state == RV_CONNECTED)
+			ib_cm_notify(sconn->cm_id, item->event.event);
+		break;
+	case IB_EVENT_COMM_EST:
+		if (sconn->state == RV_CONNECTING)
+			ib_cm_notify(sconn->cm_id, item->event.event);
+		break;
+	case IB_EVENT_QP_FATAL:
+	case IB_EVENT_QP_REQ_ERR:
+	case IB_EVENT_QP_ACCESS_ERR:
+		if (!ib_send_cm_dreq(sconn->cm_id, NULL, 0)) {
+			sconn->stats.dreq_sent++;
+			trace_rv_msg_qp_event(sconn, sconn->index,
+					      "Sending DREQ", 0, (u64)sconn);
+		}
+		if (sconn->state != RV_DISCONNECTING) {
+			if ((sconn->state == RV_CONNECTED ||
+			     sconn->state == RV_CONNECTING) &&
+			    rv_sconn_can_reconn(sconn))
+				rv_sconn_enter_disconnecting(sconn, "QP error");
+			else
+				rv_sconn_set_state(sconn, RV_ERROR, "QP error");
+		}
+		break;
+	default:
+		break;
+	}
+unlock:
+	mutex_unlock(&sconn->mutex);
+	rv_conn_put(sconn->parent);
+	kfree(item);
+}
+
+/*
+ * QP Async Event
+ * Non-premptable, so real work in WQ
+ */
+static void rv_qp_event(struct ib_event *event, void *context)
+{
+	struct rv_sconn *sconn = (struct rv_sconn *)context;
+	struct rv_qp_event_work_item *item;
+
+	if (!sconn || !sconn->parent)
+		return;
+	if (rv_conn_get_check(sconn->parent))
+		return;
+
+	rv_conn_err(sconn,
+		    "QP Event received: %s: sconn index %u qp %u\n",
+		    ib_event_msg(event->event), sconn->index,
+		    event->element.qp->qp_num);
+
+	item = kzalloc(sizeof(*item), GFP_ATOMIC);
+	if (!item)
+		goto fail;
+	INIT_WORK(&item->qp_event_work, rv_qp_event_work);
+	item->sconn = sconn;
+	item->event = *event;
+	rv_queue_work(&item->qp_event_work);
+	return;
+
+fail:
+	rv_conn_err(sconn,
+		    "No mem for QP Event: %s: sconn index %u qp %u conn %p\n",
+		    ib_event_msg(event->event), sconn->index,
+		    event->element.qp->qp_num, sconn->parent);
+	rv_conn_put(sconn->parent);
+}
+
+/*
+ * shared rv_conn QP create and re-create
+ * Allocate 2 extra WQEs and CQEs in each direction so room for error
+ * recovery drain and drain in release.
+ * In rare case of release during error recovery may need both.
+ * Plus one for heartbeat
+ * mlx5 driver requires recv_sge>0, even though we expect no data
+ * Returns:
+ *	0 on success
+ *	-ENOSPC - QP from device too small
+ *		(note can't be -ENOXIO since that means device removed
+ *	error from ib_create_qp
+ */
+static int rv_create_qp(int rv_inx, struct rv_sconn *sconn,
+			struct rv_job_dev *jdev)
+{
+	int ret = 0;
+	struct ib_qp_init_attr qp_attr;
+	int alloced_s_cq = 0;
+	int alloced_r_cq = 0;
+	u32 qp_depth;
+
+	qp_depth = jdev->qp_depth + 3;
+
+	if (!sconn->send_cq) {
+		sconn->send_cq = ib_alloc_cq(jdev->dev->ib_dev, sconn,
+					     qp_depth, 0, IB_POLL_SOFTIRQ);
+		if (IS_ERR(sconn->send_cq)) {
+			ret = PTR_ERR(sconn->send_cq);
+			rv_err(rv_inx, "Creating send cq failed %d\n", ret);
+			goto bail_out;
+		}
+		sconn->send_cq->event_handler = rv_cq_event;
+		alloced_s_cq = 1;
+	}
+
+	if (!sconn->recv_cq) {
+		sconn->recv_cq = ib_alloc_cq(jdev->dev->ib_dev, sconn,
+					     qp_depth, 0, IB_POLL_SOFTIRQ);
+		if (IS_ERR(sconn->recv_cq)) {
+			ret = PTR_ERR(sconn->recv_cq);
+			rv_err(rv_inx, "Creating recv cq failed %d\n", ret);
+			goto bail_s_cq;
+		}
+		sconn->recv_cq->event_handler = rv_cq_event;
+		alloced_r_cq = 1;
+	}
+
+	memset(&qp_attr, 0, sizeof(qp_attr));
+	qp_attr.event_handler = rv_qp_event;
+	qp_attr.qp_context = sconn;
+	qp_attr.cap.max_send_wr = qp_depth;
+	qp_attr.cap.max_recv_wr = qp_depth;
+	qp_attr.cap.max_recv_sge = 1;
+	qp_attr.cap.max_send_sge = 1;
+	qp_attr.sq_sig_type = IB_SIGNAL_REQ_WR;
+	qp_attr.qp_type = IB_QPT_RC;
+	qp_attr.send_cq = sconn->send_cq;
+	qp_attr.recv_cq = sconn->recv_cq;
+
+	sconn->qp = ib_create_qp(jdev->pd, &qp_attr);
+	if (IS_ERR(sconn->qp)) {
+		ret = PTR_ERR(sconn->qp);
+		sconn->qp = NULL;
+		goto bail_r_cq;
+	}
+	if (qp_attr.cap.max_recv_wr < qp_depth ||
+	    qp_attr.cap.max_send_wr < qp_depth) {
+		ret = -ENOSPC;
+		goto bail_qp;
+	}
+
+	return 0;
+
+bail_qp:
+	ib_destroy_qp(sconn->qp);
+	sconn->qp = NULL;
+bail_r_cq:
+	if (alloced_r_cq) {
+		ib_free_cq(sconn->recv_cq);
+		sconn->recv_cq = NULL;
+	}
+bail_s_cq:
+	if (alloced_r_cq) {
+		ib_free_cq(sconn->send_cq);
+		sconn->send_cq = NULL;
+	}
+bail_out:
+
+	return ret;
+}
+
+static int rv_query_qp_state(struct ib_qp *qp)
+{
+	struct ib_qp_attr attr;
+	struct ib_qp_init_attr qp_init_attr;
+	int ret;
+
+	ret = ib_query_qp(qp, &attr, IB_QP_STATE, &qp_init_attr);
+	if (ret) {
+		rv_err(RV_INVALID, "failed to query qp %u: %d\n",
+		       qp->qp_num, ret);
+		return ret;
+	}
+	trace_rv_msg_err_qp(RV_INVALID, "qp_state", (u64)qp->qp_num,
+			    (u64)attr.qp_state);
+
+	return attr.qp_state;
+}
+
+/*
+ * If QP is not in reset state, move it to error state.
+ *
+ * Return:  0  - Success;
+ *          1 - QP in RESET
+ *          <0 - Failure.
+ */
+static int rv_err_qp(struct ib_qp *qp)
+{
+	struct ib_qp_attr attr;
+	int ret;
+
+	ret = rv_query_qp_state(qp);
+	if (ret < 0)
+		return ret;
+
+	if (ret == IB_QPS_RESET)
+		return 1;
+
+	if (ret == IB_QPS_ERR)
+		return 0;
+
+	attr.qp_state = IB_QPS_ERR;
+	ret = ib_modify_qp(qp, &attr, IB_QP_STATE);
+
+	return ret;
+}
+
+struct rv_dest_qp_work_item {
+	struct work_struct destroy_work;
+	struct ib_qp *qp;
+	struct ib_cq *send_cq;
+	struct ib_cq *recv_cq;
+};
+
+/* only used if QP needs to be drained */
+static void rv_handle_destroy_qp(struct work_struct *work)
+{
+	struct rv_dest_qp_work_item *item = container_of(work,
+				struct rv_dest_qp_work_item, destroy_work);
+
+	trace_rv_msg_destroy_qp(NULL, RV_INVALID, "destroy qp",
+				item->qp ? (u64)item->qp->qp_num : 0, 0);
+	if (item->qp) {
+		ib_drain_qp(item->qp);
+		ib_destroy_qp(item->qp);
+	}
+	if (item->recv_cq)
+		ib_free_cq(item->recv_cq);
+
+	if (item->send_cq)
+		ib_free_cq(item->send_cq);
+	kfree(item);
+}
+
+/*
+ * destroy QP and CQs, cannot hold sconn->mutex
+ * Drain the qp before destroying it to avoid the race between QP destroy
+ * and completion handler. Timeout protects against CQ issues.
+ */
+static void rv_destroy_qp(struct rv_sconn *sconn)
+{
+	int qps = -1;
+	struct rv_dest_qp_work_item *item;
+
+	if (sconn->qp)
+		qps = rv_query_qp_state(sconn->qp);
+	if (qps >= 0 && qps != IB_QPS_RESET) {
+		item = kzalloc(sizeof(*item), GFP_KERNEL);
+		if (item) {
+			trace_rv_msg_destroy_qp(sconn, sconn->index,
+						"queue destroy qp",
+						(u64)sconn->qp->qp_num,
+						(u64)sconn);
+			INIT_WORK(&item->destroy_work, rv_handle_destroy_qp);
+			item->qp = sconn->qp;
+			item->recv_cq = sconn->recv_cq;
+			item->send_cq = sconn->send_cq;
+			sconn->qp = NULL;
+			sconn->recv_cq = NULL;
+			sconn->send_cq = NULL;
+
+			rv_queue_work2(&item->destroy_work);
+			return;
+		}
+	}
+	trace_rv_msg_destroy_qp(sconn, sconn->index, "destroy qp",
+				sconn->qp ? (u64)sconn->qp->qp_num : 0,
+				(u64)sconn);
+	if (sconn->qp) {
+		if (qps >= 0 && qps != IB_QPS_RESET)
+			ib_drain_qp(sconn->qp);
+		ib_destroy_qp(sconn->qp);
+		sconn->qp = NULL;
+	}
+
+	if (sconn->recv_cq) {
+		ib_free_cq(sconn->recv_cq);
+		sconn->recv_cq = NULL;
+	}
+
+	if (sconn->send_cq) {
+		ib_free_cq(sconn->send_cq);
+		sconn->send_cq = NULL;
+	}
+}
+
+/*
+ * only for use by rv_conn_alloc
+ * others use rv_conn_get_alloc or rv_conn_get
+ * must be called with jdev->conn_list_mutex held
+ * We create QP now to make sure we can create it before going further.
+ * Otherwise we really don't need it until REQ handler on server or
+ * connect on client.
+ */
+static int rv_sconn_init(struct rv_user *rv, struct rv_sconn *sconn,
+			 struct rv_conn_create_params_in *param,
+			 struct rv_conn *parent, u8 index)
+{
+	struct rv_job_dev *jdev = rv->jdev;
+	int ret;
+
+	sconn->index = index;
+	sconn->parent = parent;
+
+	mutex_init(&sconn->mutex);
+
+	spin_lock_init(&sconn->drain_lock);
+
+	INIT_WORK(&sconn->drain_work, rv_sconn_drain_work);
+	timer_setup(&sconn->drain_timer, rv_sconn_drain_timeout_func, 0);
+
+	timer_setup(&sconn->conn_timer, rv_sconn_timeout_func, 0);
+	INIT_WORK(&sconn->timer_work, rv_sconn_timeout_work);
+
+	timer_setup(&sconn->delay_timer, rv_sconn_delay_func, 0);
+	INIT_WORK(&sconn->delay_work, rv_sconn_delay_work);
+
+	timer_setup(&sconn->hb_timer, rv_sconn_hb_func, 0);
+	INIT_WORK(&sconn->hb_work, rv_sconn_hb_work);
+
+	sconn->cqe.done = rv_recv_done;
+	sconn->rdrain_cqe.done = rv_rq_drain_done;
+	sconn->sdrain_cqe.done = rv_sq_drain_done;
+	sconn->hb_cqe.done = rv_hb_done;
+
+	if (jdev->loc_addr < param->rem_addr)
+		set_bit(RV_SCONN_SERVER, &sconn->flags);
+
+	ret = rv_create_qp(rv->inx, sconn, jdev);
+	if (ret) {
+		rv_err(rv->inx, "Failed to create qp: %d\n", ret);
+		goto bail;
+	}
+
+	if (test_bit(RV_SCONN_SERVER, &sconn->flags)) {
+		if (!jdev->listener) {
+			jdev->listener = rv_listener_get_alloc(jdev->dev,
+							       jdev->service_id,
+							  rv_cm_server_handler);
+			if (!jdev->listener) {
+				rv_err(rv->inx,
+				       "Failed to get/allocate listener\n");
+				goto bail_qp;
+			}
+		}
+		sconn->state = RV_WAITING;
+		sconn->start_time = ktime_get();
+	} else {
+		sconn->state = RV_INIT;
+	}
+	atomic_set(&sconn->stats.outstand_send_write, 0);
+	atomic64_set(&sconn->stats.send_write_cqe, 0);
+	atomic64_set(&sconn->stats.send_write_cqe_fail, 0);
+	atomic64_set(&sconn->stats.recv_write_cqe, 0);
+	atomic64_set(&sconn->stats.recv_write_bytes, 0);
+	atomic64_set(&sconn->stats.recv_cqe_fail, 0);
+	atomic64_set(&sconn->stats.send_hb_cqe, 0);
+	atomic64_set(&sconn->stats.send_hb_cqe_fail, 0);
+	atomic64_set(&sconn->stats.recv_hb_cqe, 0);
+	trace_rv_sconn_init(sconn, sconn->index, sconn->qp->qp_num,
+			    sconn->parent, sconn->flags, (u32)sconn->state,
+			    sconn->cm_id, sconn->resolver_retry_left);
+	return 0;
+
+bail_qp:
+	rv_destroy_qp(sconn);
+bail:
+	return -ENOMEM;
+}
+
+static void rv_handle_conn_put(struct work_struct *work)
+{
+	struct rv_conn *conn = container_of(work, struct rv_conn, put_work);
+
+	rv_conn_release(conn);
+}
+
+/*
+ * only for use by rv_conn_get_alloc
+ * others use rv_conn_get_alloc or rv_conn_get
+ * must be called with jdev->conn_list_mutex held
+ */
+static struct rv_conn *rv_conn_alloc(struct rv_user *rv,
+				     struct rv_conn_create_params_in *param)
+{
+	struct rv_conn *conn;
+	struct rv_job_dev *jdev = rv->jdev;
+	int i;
+
+	conn = kzalloc(sizeof(*conn) +
+		       sizeof(conn->sconn_arr[0]) * jdev->num_conn,
+		       GFP_KERNEL);
+	if (!conn)
+		return NULL;
+
+	conn->num_conn = jdev->num_conn;
+	rv_job_dev_get(jdev);
+	conn->jdev = jdev;
+	memcpy(&conn->ah, &param->ah, sizeof(struct ib_uverbs_ah_attr));
+	conn->rem_addr = (u64)param->rem_addr;
+
+	kref_init(&conn->kref);
+	INIT_WORK(&conn->put_work, rv_handle_conn_put);
+	INIT_WORK(&conn->free_work, rv_handle_free_conn);
+
+	spin_lock_init(&conn->next_lock);
+	for (i = 0; i < conn->num_conn; i++) {
+		if (rv_sconn_init(rv, &conn->sconn_arr[i], param, conn, i))
+			goto bail_conn;
+	}
+	trace_rv_conn_alloc(conn, conn->rem_addr, conn->ah.is_global,
+			    conn->ah.dlid,
+			    be64_to_cpu(*((__be64 *)&conn->ah.grh.dgid[0])),
+			    be64_to_cpu(*((__be64 *)&conn->ah.grh.dgid[8])),
+			    conn->num_conn, conn->next,
+			    conn->jdev, kref_read(&conn->kref));
+
+	return conn;
+
+bail_conn:
+	for (--i; i >= 0; i--)
+		rv_sconn_deinit(&conn->sconn_arr[i]);
+	rv_job_dev_put(jdev);
+	kfree(conn);
+	return NULL;
+}
+
+/*
+ * get a reference to the matching rv_conn.  Allocate an rv_conn if no
+ * match found.  kref_get_unless_zero avoids race w/ release removing from list
+ */
+static struct rv_conn *rv_conn_get_alloc(struct rv_user *rv,
+					 struct rv_conn_create_params_in *param)
+{
+	struct rv_conn *conn;
+	struct rv_job_dev *jdev = rv->jdev;
+
+	mutex_lock(&jdev->conn_list_mutex);
+	rcu_read_lock();
+	list_for_each_entry_rcu(conn, &jdev->conn_list, conn_entry) {
+		if (rv_conn_cmp_params(rv->inx, conn, param))
+			continue;
+		if (!kref_get_unless_zero(&conn->kref))
+			continue;
+		rcu_read_unlock();
+		goto done;
+	}
+	rcu_read_unlock();
+	conn = rv_conn_alloc(rv, param);
+	if (!conn)
+		goto done;
+	list_add_rcu(&conn->conn_entry, &jdev->conn_list);
+done:
+	mutex_unlock(&jdev->conn_list_mutex);
+	return conn;
+}
+
+/* validate conn_create against jdev->ah */
+static int rv_jdev_check_create_ah(int rv_inx, const struct rv_job_dev *jdev,
+				   const struct rv_conn_create_params_in *param)
+{
+	if (!param->ah.dlid && !rv_jdev_protocol_roce(jdev)) {
+		rv_err(rv_inx, "create_conn: DLID must be non-zero\n");
+		return -EINVAL;
+	}
+	if (param->ah.is_global &&
+	    jdev->loc_gid_index != param->ah.grh.sgid_index) {
+		rv_err(rv_inx, "create_conn: incorrect sgid_index\n");
+		return -EINVAL;
+	}
+	if (jdev->port_num != param->ah.port_num) {
+		rv_err(rv_inx, "create_conn: port or sgid_index\n");
+		return -EINVAL;
+	}
+	if (jdev->loc_addr == param->rem_addr) {
+		rv_err(rv_inx, "create_conn: loopback not allowed\n");
+		return -EINVAL;
+	}
+	return 0;
+}
+
+/*
+ * validate conn_create ah against conn->ah
+ * Assumes caller has used rv_jdev_check_create_ah and
+ * assumes conn_get_alloc matched on rem_addr, is_global and (dgid or dlid).
+ * Confirms the rest of ah is consistent
+ */
+static int rv_conn_create_check_ah(int rv_inx, const struct rv_conn *conn,
+				   const struct ib_uverbs_ah_attr *ah)
+{
+	int ret = -EEXIST;
+
+#define RV_CHECK(field) (ah->field != conn->ah.field)
+#define RV_REPORT(field, text, format) \
+		     rv_err(rv_inx, "create_conn: inconsistent " text " " \
+			    format " with other processes " format "\n", \
+			    ah->field, conn->ah.field)
+	if (RV_CHECK(dlid))
+		RV_REPORT(dlid, "DLID", "0x%x");
+	else if (RV_CHECK(src_path_bits))
+		RV_REPORT(src_path_bits, "src_path_bits", "0x%x");
+	else if (RV_CHECK(sl))
+		RV_REPORT(sl, "SL", "%u");
+	else if (conn->ah.is_global && RV_CHECK(grh.traffic_class))
+		RV_REPORT(grh.traffic_class, "traffic_class", "%u");
+	else if (conn->ah.is_global && RV_CHECK(grh.flow_label))
+		RV_REPORT(grh.flow_label, "flow_label", "0x%x");
+	else if (RV_CHECK(static_rate))
+		RV_REPORT(static_rate, "rate", "%u");
+	else if (conn->ah.is_global && RV_CHECK(grh.hop_limit))
+		RV_REPORT(grh.hop_limit, "hop_limit", "%u");
+	else
+		ret = 0;
+#undef RV_CHECK
+#undef RV_REPORT
+
+	return ret;
+}
+
+int doit_conn_create(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_conn_create_params param;
+	struct rv_conn *conn;
+	struct rv_job_dev *jdev;
+	int ret = 0;
+	u32 id;
+
+	if (copy_from_user(&param.in, (void __user *)arg, sizeof(param.in)))
+		return -EFAULT;
+	trace_rv_conn_create_req(/* trace */
+		param.in.rem_addr, param.in.ah.is_global,
+		param.in.ah.grh.sgid_index, param.in.ah.port_num,
+		param.in.ah.dlid,
+		be64_to_cpu(*((__be64 *)&param.in.ah.grh.dgid[0])),
+		be64_to_cpu(*((__be64 *)&param.in.ah.grh.dgid[8])));
+
+	mutex_lock(&rv->mutex);
+	if (!rv->attached) {
+		ret = rv->was_attached ? -ENXIO : -EINVAL;
+		goto bail_unlock;
+	}
+	if (rv->rdma_mode != RV_RDMA_MODE_KERNEL) {
+		ret = -EINVAL;
+		goto bail_unlock;
+	}
+
+	jdev = rv->jdev;
+	trace_rv_jdev_conn_create(jdev, jdev->dev_name, jdev->num_conn,
+				  jdev->index_bits, jdev->loc_gid_index,
+				  jdev->loc_addr, jdev->job_key_len,
+				  jdev->job_key, jdev->service_id,
+				  jdev->q_depth, jdev->user_array_next,
+				  kref_read(&jdev->kref));
+	ret = rv_jdev_check_create_ah(rv->inx, jdev, &param.in);
+	if (ret)
+		goto bail_unlock;
+
+	conn = user_conn_exist(rv, &param.in);
+	if (conn) {
+		trace_rv_msg_conn_create(rv->inx, "User_conn exists",
+					 (u64)conn, 0);
+		ret = -EBUSY;
+		goto bail_unlock;
+	}
+
+	conn = rv_conn_get_alloc(rv, &param.in);
+	if (!conn) {
+		rv_err(rv->inx, "Failed to get/allocate conn\n");
+		ret = -ENOMEM;
+		goto bail_unlock;
+	}
+	trace_rv_conn_create(conn, conn->rem_addr, conn->ah.is_global,
+			     conn->ah.dlid,
+			     be64_to_cpu(*((__be64 *)&conn->ah.grh.dgid[0])),
+			     be64_to_cpu(*((__be64 *)&conn->ah.grh.dgid[8])),
+			     conn->num_conn, conn->next,
+			     conn->jdev, kref_read(&conn->kref));
+	ret = rv_conn_create_check_ah(rv->inx, conn, &param.in.ah);
+	if (ret)
+		goto bail_put;
+
+	ret = xa_alloc(&rv->conn_xa, &id, conn, XA_LIMIT(1, UINT_MAX),
+		       GFP_KERNEL);
+	if (ret)
+		goto bail_put;
+
+	param.out.handle = (u64)id;
+	param.out.conn_handle = (u64)conn;
+	ret = copy_to_user((void __user *)arg, &param.out, sizeof(param.out));
+	if (ret) {
+		ret = -EFAULT;
+		goto bail_xa;
+	}
+
+	trace_rv_msg_uconn_create(rv->inx, "rv_user create uconn", (u64)conn,
+				  0);
+	mutex_unlock(&rv->mutex);
+
+	return 0;
+
+bail_xa:
+	xa_erase(&rv->conn_xa, id);
+bail_put:
+	rv_conn_put(conn);
+bail_unlock:
+	mutex_unlock(&rv->mutex);
+	return ret;
+}
+
+/*
+ * Address Resolver callback.
+ * there is a slight chance the device bounced and changed mode
+ * from RoCE to IB or iWARP.  However then the gids we have
+ * are wrong anyway. So just let resolver struggle and hit retry limit
+ * instead of trying to redo rdma_protocol_roce(), etc.
+ * PSM will fail in this case anyway and close.
+ */
+static void rv_resolve_ip_cb(int status, struct sockaddr *src_addr,
+			     struct rdma_dev_addr *addr, void *context)
+{
+	struct rv_sconn *sconn = (struct rv_sconn *)context;
+
+	if (!sconn || !sconn->parent)
+		return;
+	if (rv_conn_get_check(sconn->parent))
+		return;
+
+	mutex_lock(&sconn->mutex);
+	trace_rv_sconn_resolve_cb(sconn, sconn->index,
+				  sconn->qp ? sconn->qp->qp_num : 0,
+				  sconn->parent, sconn->flags,
+				  (u32)sconn->state, sconn->cm_id,
+				  sconn->resolver_retry_left);
+	if (sconn->state != RV_RESOLVING)
+		goto unlock;
+
+	if (status) {
+		rv_conn_err(sconn, "failed to resolve_ip status %d\n", status);
+		goto retry;
+	}
+	if (addr != &sconn->dev_addr) {
+		rv_conn_err(sconn, "wrong dev_addr in callback\n");
+		goto fail;
+	}
+	if (addr->sgid_attr != sconn->parent->jdev->sgid_attr) {
+		rv_conn_err(sconn, "wrong sgid_attr in callback\n");
+		goto fail;
+	}
+	sconn->primary_path->roce.route_resolved = true;
+	sa_path_set_dmac(sconn->primary_path, addr->dst_dev_addr);
+	sconn->primary_path->hop_limit = addr->hoplimit;
+
+	rv_send_req(sconn);
+unlock:
+	mutex_unlock(&sconn->mutex);
+	rv_conn_put(sconn->parent);
+	return;
+
+retry:
+	if (sconn->resolver_retry_left) {
+		sconn->resolver_retry_left--;
+		if (!rv_resolve_ip(sconn)) {
+			mutex_unlock(&sconn->mutex);
+			rv_conn_put(sconn->parent);
+			return;
+		}
+	}
+	if (rv_sconn_can_reconn(sconn))
+		rv_sconn_set_state(sconn, RV_DELAY, "");
+fail:
+	if (test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags))
+		sconn->stats.reresolve_fail++;
+	else
+		sconn->stats.resolve_fail++;
+	rv_sconn_free_primary_path(sconn);
+	if (sconn->state != RV_DELAY)
+		rv_sconn_set_state(sconn, RV_ERROR,
+				   "unable to resolve address");
+	mutex_unlock(&sconn->mutex);
+	rv_conn_put(sconn->parent);
+}
+
+/*
+ * Algorithm based on roce_resolve_route_from_path
+ * Caller must hold a rv_conn reference. This func does not release that ref
+ * Caller holds mutex and has validated sconn->state, caller will release mutex
+ */
+static int rv_resolve_ip(struct rv_sconn *sconn)
+{
+	union {
+		struct sockaddr     _sockaddr;
+		struct sockaddr_in  _sockaddr_in;
+		struct sockaddr_in6 _sockaddr_in6;
+	} src_addr, dst_addr;
+
+	if (test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags))
+		sconn->stats.reresolve++;
+	else
+		sconn->stats.resolve++;
+	rdma_gid2ip(&src_addr._sockaddr, &sconn->primary_path->sgid);
+	rdma_gid2ip(&dst_addr._sockaddr, &sconn->primary_path->dgid);
+
+	if (src_addr._sockaddr.sa_family != dst_addr._sockaddr.sa_family)
+		return -EINVAL;
+
+	memset(&sconn->dev_addr, 0, sizeof(sconn->dev_addr));
+	sconn->dev_addr.net = &init_net; /* manditory, but will not be used */
+	sconn->dev_addr.sgid_attr = sconn->parent->jdev->sgid_attr;
+
+	return rdma_resolve_ip(&src_addr._sockaddr, &dst_addr._sockaddr,
+				&sconn->dev_addr, RV_RESOLVER_TIMEOUT,
+				rv_resolve_ip_cb, true, sconn);
+}
+
+/*
+ * Gets connection establishment ball rolling
+ * After this everything proceeds via callbacks or timeouts.
+ * Caller must hold a rv_conn reference. This func does not release that ref.
+ * Caller holds mutex and has validated sconn->state, caller will release mutex
+ * For IB/OPA, no need to resolve IP to dmac, so move to next step.
+ */
+static void rv_resolve_path(struct rv_sconn *sconn)
+{
+	int ret;
+	struct rv_job_dev *jdev = sconn->parent->jdev;
+
+	rv_sconn_set_state(sconn, RV_RESOLVING, "");
+
+	sconn->resolver_retry_left = RV_RESOLVER_RETRY;
+
+	trace_rv_sconn_resolve(sconn, sconn->index, sconn->qp->qp_num,
+			       sconn->parent, sconn->flags, (u32)sconn->state,
+			       sconn->cm_id, sconn->resolver_retry_left);
+	sconn->primary_path = kzalloc(sizeof(*sconn->primary_path), GFP_KERNEL);
+	if (!sconn->primary_path)
+		goto err;
+
+	/* this sets record type to IB or OPA, fix up below for RoCE */
+	ib_copy_path_rec_from_user(sconn->primary_path, &sconn->path);
+	sconn->primary_path->service_id = cpu_to_be64(jdev->service_id);
+
+	if (rv_jdev_protocol_roce(jdev)) {
+		sconn->primary_path->rec_type =
+			sa_conv_gid_to_pathrec_type(jdev->sgid_attr->gid_type);
+		if (unlikely(!sa_path_is_roce(sconn->primary_path)))
+			goto err;
+		ret = rv_resolve_ip(sconn);
+		if (ret)
+			goto err;
+		return;
+	}
+	rv_send_req(sconn);
+	return;
+
+err:
+	if (test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags))
+		sconn->stats.reresolve_fail++;
+	else
+		sconn->stats.resolve_fail++;
+	rv_sconn_free_primary_path(sconn);
+	rv_sconn_set_state(sconn, RV_ERROR, "local error resolving address");
+}
+
+/* caller must hold a rv_conn reference. This func does not release that ref */
+static void rv_send_req(struct rv_sconn *sconn)
+{
+	struct rv_job_dev *jdev = sconn->parent->jdev;
+	struct ib_cm_req_param req;
+	struct rv_req_priv_data priv_data = {
+			.magic = RV_PRIVATE_DATA_MAGIC,
+			.ver = RV_PRIVATE_DATA_VER,
+		};
+	int ret;
+
+	memset(&req, 0, sizeof(req));
+	req.ppath_sgid_attr = jdev->sgid_attr;
+	req.flow_control = 1;
+	req.retry_count = 7;
+	req.responder_resources = 0;
+	req.rnr_retry_count = 7;
+	req.max_cm_retries = 15;
+	req.primary_path = sconn->primary_path;
+	req.service_id = req.primary_path->service_id;
+	req.initiator_depth = 0;
+	req.remote_cm_response_timeout = 17;
+	req.local_cm_response_timeout = 17;
+	req.qp_num = sconn->qp->qp_num;
+	req.qp_type = sconn->qp->qp_type;
+	req.srq = !!(sconn->qp->srq);
+	req.starting_psn = prandom_u32() & 0xffffff;
+
+	req.private_data = &priv_data;
+	req.private_data_len = sizeof(priv_data);
+	priv_data.index = sconn->index;
+	priv_data.job_key_len = jdev->job_key_len;
+	memcpy(priv_data.job_key, jdev->job_key, sizeof(priv_data.job_key));
+	priv_data.uid = jdev->uid;
+	trace_rv_msg_send_req(sconn, sconn->index,
+			      "sending rec_type | route_resolved, dmac",
+			      (u64)(req.primary_path->rec_type |
+			      (((u64)req.primary_path->roce.route_resolved) <<
+			       31)),
+			      (u64)(req.primary_path->roce.dmac[0] |
+			      ((u64)req.primary_path->roce.dmac[1]) << 8 |
+			      ((u64)req.primary_path->roce.dmac[2]) << 16 |
+			      ((u64)req.primary_path->roce.dmac[3]) << 24 |
+			      ((u64)req.primary_path->roce.dmac[4]) << 32 |
+			      ((u64)req.primary_path->roce.dmac[5]) << 40));
+
+	ret = ib_send_cm_req(sconn->cm_id, &req);
+	rv_sconn_free_primary_path(sconn);
+	if (!ret) {
+		sconn->stats.req_sent++;
+		trace_rv_msg_send_req(sconn, sconn->index, "Sending REQ", 0,
+				      (u64)sconn);
+		rv_sconn_set_state(sconn, RV_CONNECTING, "");
+	} else {
+		rv_conn_err(sconn, "Failed to send cm req. %d\n", ret);
+		rv_sconn_set_state(sconn, RV_ERROR, "local error sending REQ");
+	}
+}
+
+/*
+ * called on work queue with rv_conn reference held on our behalf
+ * if in CONNECTING:
+ *	IB CM listener could have a rep outstanding, REJ cancels it,
+ *	or we could have sent or gotten RTU and raced with CM cb.
+ *	Tell IB CM to send REJ and DREQ, it will sort things out for us.
+ * if already in CONNECTED do nothing, we got in just under the timelimit.
+ */
+static void rv_sconn_timeout_work(struct work_struct *work)
+{
+	struct rv_sconn *sconn = container_of(work, struct rv_sconn,
+					      timer_work);
+
+	mutex_lock(&sconn->mutex);
+	trace_rv_sconn_timeout_work(sconn, sconn->index,
+				    sconn->qp ? sconn->qp->qp_num : 0,
+				    sconn->parent, sconn->flags,
+				    (u32)sconn->state, sconn->cm_id,
+				    sconn->resolver_retry_left);
+	switch (sconn->state) {
+	case RV_RESOLVING:
+		rv_sconn_free_primary_path(sconn);
+		rdma_addr_cancel(&sconn->dev_addr);
+		rv_sconn_set_state(sconn, RV_ERROR, "connection timeout");
+		break;
+	case RV_CONNECTING:
+		if (!ib_send_cm_rej(sconn->cm_id, IB_CM_REJ_TIMEOUT, NULL, 0,
+				    NULL, 0)) {
+			sconn->stats.rej_sent++;
+			trace_rv_msg_sconn_timeout_work(sconn, sconn->index,
+							"Sending REJ reason",
+							(u64)IB_CM_REJ_TIMEOUT,
+							(u64)sconn);
+		}
+		if (!ib_send_cm_dreq(sconn->cm_id, NULL, 0)) {
+			sconn->stats.dreq_sent++;
+			trace_rv_msg_sconn_timeout_work(sconn, sconn->index,
+							"Sending DREQ", 0,
+							(u64)sconn);
+		}
+		fallthrough;
+	case RV_WAITING:
+	case RV_DISCONNECTING:
+	case RV_DELAY:
+		rv_sconn_set_state(sconn, RV_ERROR, "connection timeout");
+		break;
+	case RV_CONNECTED:
+		break;
+	case RV_INIT:
+	case RV_ERROR:
+	default:
+		break;
+	}
+	mutex_unlock(&sconn->mutex);
+
+	rv_conn_put(sconn->parent);
+}
+
+/* called at SOFT IRQ,  so real work in WQ */
+static void rv_sconn_timeout_func(struct timer_list *timer)
+{
+	struct rv_sconn *sconn = container_of(timer, struct rv_sconn,
+					      conn_timer);
+
+	if (!sconn->parent)
+		return;
+	if (rv_conn_get_check(sconn->parent))
+		return;
+	rv_queue_work(&sconn->timer_work);
+}
+
+/* called on work queue with rv_conn reference held on our behalf */
+static void rv_sconn_delay_work(struct work_struct *work)
+{
+	struct rv_sconn *sconn = container_of(work, struct rv_sconn,
+					      delay_work);
+
+	mutex_lock(&sconn->mutex);
+	trace_rv_sconn_delay_work(sconn, sconn->index,
+				  sconn->qp ? sconn->qp->qp_num : 0,
+				  sconn->parent, sconn->flags,
+				  (u32)sconn->state, sconn->cm_id,
+				  sconn->resolver_retry_left);
+	if (sconn->state == RV_DELAY)
+		rv_resolve_path(sconn);
+	mutex_unlock(&sconn->mutex);
+
+	rv_conn_put(sconn->parent);
+}
+
+/* called at SOFT IRQ,  so real work in WQ */
+static void rv_sconn_delay_func(struct timer_list *timer)
+{
+	struct rv_sconn *sconn = container_of(timer, struct rv_sconn,
+					      delay_timer);
+
+	if (!sconn->parent)
+		return;
+	if (rv_conn_get_check(sconn->parent))
+		return;
+	rv_queue_work(&sconn->delay_work);
+}
+
+/*
+ * validate cm_connect path against sconn->path
+ */
+static int rv_sconn_connect_check_path(int rv_inx, const struct rv_sconn *sconn,
+				       const struct ib_user_path_rec *path)
+{
+	char buf1[RV_MAX_ADDR_STR];
+	char buf2[RV_MAX_ADDR_STR];
+	int ret = -EEXIST;
+
+#define RV_CHECK(field) (path->field != sconn->path.field)
+#define RV_REPORT(field, text, format) \
+		     rv_err(rv_inx, "connect: inconsistent " text " " format \
+			    " with other processes " format "\n", \
+			    path->field, sconn->path.field)
+
+	if (RV_CHECK(dlid))
+		RV_REPORT(dlid, "DLID", "0x%x");
+	else if (cmp_gid(path->dgid, sconn->path.dgid))
+		rv_err(rv_inx,
+		       "connect: inconsistent dest %s with other proc %s\n",
+		       show_gid(buf1, sizeof(buf1), path->dgid),
+		       show_gid(buf2, sizeof(buf2), sconn->path.dgid));
+	else if (RV_CHECK(slid))
+		RV_REPORT(slid, "SLID", "0x%x");
+	else if (cmp_gid(path->sgid, sconn->path.sgid))
+		rv_err(rv_inx,
+		       "connect: inconsistent src %s with other processes %s\n",
+		       show_gid(buf1, sizeof(buf1), path->sgid),
+		       show_gid(buf2, sizeof(buf2), sconn->path.sgid));
+	else if (RV_CHECK(pkey))
+		RV_REPORT(pkey, "pkey", "0x%x");
+	else if (RV_CHECK(mtu))
+		RV_REPORT(pkey, "mtu", "%u");
+	else if (RV_CHECK(sl))
+		RV_REPORT(sl, "SL", "%u");
+	else if (RV_CHECK(traffic_class))
+		RV_REPORT(traffic_class, "traffic_class", "%u");
+	else if (RV_CHECK(flow_label))
+		RV_REPORT(flow_label, "flow_label", "0x%x");
+	else if (RV_CHECK(rate))
+		RV_REPORT(rate, "rate", "%u");
+	else if (RV_CHECK(hop_limit))
+		RV_REPORT(hop_limit, "hop_limit", "%u");
+	else if (RV_CHECK(packet_life_time))
+		RV_REPORT(packet_life_time, "packet_life_time", "%u");
+#undef RV_CHECK
+#undef RV_REPORT
+	else
+		ret = 0;
+	return ret;
+}
+
+/*
+ * start connection and wait for client side to complete
+ * caller must hold a rv_conn reference. This func does not release that ref
+ * We use sconn->path.dlid to identify the 1st connect call for the given sconn
+ * On subsequent calls we only need to check params match existing.
+ */
+static int rv_sconn_connect(int rv_inx, struct rv_sconn *sconn,
+			    struct rv_conn_connect_params_in *params)
+{
+	struct rv_job_dev *jdev = sconn->parent->jdev;
+	int ret = 0;
+	struct ib_cm_id *id;
+
+	mutex_lock(&sconn->mutex);
+
+	if (!sconn->path.dlid)
+		sconn->path = params->path;
+
+	if (sconn->state != RV_INIT) {
+		ret = rv_sconn_connect_check_path(rv_inx, sconn,
+						  &params->path);
+		mutex_unlock(&sconn->mutex);
+		return ret;
+	}
+
+	sconn->path = params->path;
+
+	id = ib_create_cm_id(jdev->dev->ib_dev, rv_cm_handler, sconn);
+	if (IS_ERR(id)) {
+		rv_err(rv_inx, "Create CM ID failed\n");
+		rv_sconn_set_state(sconn, RV_ERROR,
+				   "local error preparing client");
+		mutex_unlock(&sconn->mutex);
+		return PTR_ERR(id);
+	}
+	sconn->cm_id = id;
+
+	rv_resolve_path(sconn);
+	mutex_unlock(&sconn->mutex);
+	return ret;
+}
+
+/*
+ * validate rv_user supplied path is consistent with conn->ah from create_conn
+ * sgid already checked against jdev in caller
+ */
+static int rv_conn_connect_check_ah(int rv_inx, const struct rv_conn *conn,
+				    const struct ib_user_path_rec *path)
+{
+	char buf1[RV_MAX_ADDR_STR];
+	char buf2[RV_MAX_ADDR_STR];
+	int ret = -EINVAL;
+
+#define RV_CHECK(f1, f2) (path->f1 != conn->ah.f2)
+#define RV_CHECK_BE32(f1, f2) (be32_to_cpu(path->f1) != conn->ah.f2)
+#define RV_REPORT(f1, f2, text, format) \
+		     rv_err(rv_inx, "connect: inconsistent " text " " \
+			    format " with create_conn " format "\n", \
+			    path->f1, conn->ah.f2)
+	if (be16_to_cpu(path->dlid) != conn->ah.dlid)
+		rv_err(rv_inx,
+		       "connect: inconsistent DLID 0x%x with create_conn 0x%x\n",
+		       be16_to_cpu(path->dlid), conn->ah.dlid);
+	else if (conn->ah.is_global &&
+		 cmp_gid(conn->ah.grh.dgid, path->dgid))
+		rv_err(rv_inx,
+		       "connect: inconsistent dest %s with other proc %s\n",
+		       show_gid(buf1, sizeof(buf1), path->dgid),
+		       show_gid(buf2, sizeof(buf2), conn->ah.grh.dgid));
+	else if (RV_CHECK(sl, sl))
+		RV_REPORT(sl, sl, "SL", "%u");
+	else if (conn->ah.is_global &&
+		 RV_CHECK(traffic_class, grh.traffic_class))
+		RV_REPORT(traffic_class, grh.traffic_class, "traffic_class",
+			  "%u");
+	else if (conn->ah.is_global &&
+		 RV_CHECK_BE32(flow_label, grh.flow_label))
+		RV_REPORT(flow_label, grh.flow_label, "flow_label", "0x%x");
+	else if (conn->ah.is_global && RV_CHECK(hop_limit, grh.hop_limit))
+		RV_REPORT(hop_limit, grh.hop_limit, "hop_limit", "%u");
+	else if (RV_CHECK(rate, static_rate))
+		RV_REPORT(rate, static_rate, "rate", "%u");
+	else
+		ret = 0;
+#undef RV_CHECK
+#undef RV_CHECK_BE32
+#undef RV_REPORT
+	return ret;
+}
+
+static int rv_conn_connect(int rv_inx, struct rv_conn *conn,
+			   struct rv_conn_connect_params_in *params)
+{
+	int i;
+	int ret;
+
+	trace_rv_conn_connect(conn, conn->rem_addr, conn->ah.is_global,
+			      conn->ah.dlid,
+			      be64_to_cpu(*((__be64 *)&conn->ah.grh.dgid[0])),
+			      be64_to_cpu(*((__be64 *)&conn->ah.grh.dgid[8])),
+			      conn->num_conn, conn->next,
+			      conn->jdev, kref_read(&conn->kref));
+
+	ret = rv_conn_connect_check_ah(rv_inx, conn, &params->path);
+	if (ret)
+		return ret;
+
+	for (i = 0; i < conn->num_conn; i++) {
+		ret = rv_sconn_connect(rv_inx, &conn->sconn_arr[i], params);
+		if (ret)
+			return ret;
+	}
+	return 0;
+}
+
+/* validate connect against jdev->ah */
+static int rv_jdev_check_connect_path(int rv_inx, const struct rv_job_dev *jdev,
+				      const struct ib_user_path_rec *path)
+{
+	char buf1[RV_MAX_ADDR_STR];
+	char buf2[RV_MAX_ADDR_STR];
+
+	if (cmp_gid(path->sgid, jdev->loc_gid)) {
+		rv_err(rv_inx, "connect: inconsistent src %s with attach %s\n",
+		       show_gid(buf1, sizeof(buf1), path->sgid),
+		       show_gid(buf2, sizeof(buf2), jdev->loc_gid));
+		return -EINVAL;
+	}
+	return 0;
+}
+
+/*
+ * PSM gurantees that both sides have created their connection prior to
+ * either trying to connect it.
+ */
+int doit_conn_connect(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_conn_connect_params_in params;
+	struct rv_conn *conn;
+	int ret = 0;
+
+	if (copy_from_user(&params, (void __user *)arg, sizeof(params)))
+		return -EFAULT;
+
+	mutex_lock(&rv->mutex);
+	if (!rv->attached) {
+		ret = rv->was_attached ? -ENXIO : -EINVAL;
+		goto unlock;
+	}
+	if (rv->rdma_mode != RV_RDMA_MODE_KERNEL) {
+		ret = -EINVAL;
+		goto unlock;
+	}
+	ret = rv_jdev_check_connect_path(rv->inx, rv->jdev, &params.path);
+	if (ret)
+		goto unlock;
+	conn = user_conn_find(rv, params.handle);
+	if (!conn) {
+		rv_err(rv->inx, "connect: No connection found\n");
+		ret = -EINVAL;
+		goto unlock;
+	}
+	trace_rv_msg_uconn_connect(rv->inx, "rv_user connect", (u64)conn, 0);
+
+	ret = rv_conn_connect(rv->inx, conn, &params);
+	if (ret) {
+		rv_err(rv->inx, "Failed to connect to server: %d\n", ret);
+		xa_erase(&rv->conn_xa, params.handle);
+		rv_conn_put(conn);
+	}
+unlock:
+	mutex_unlock(&rv->mutex);
+
+	return ret;
+}
+
+int doit_conn_connected(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_conn_connected_params_in params;
+	struct rv_conn *conn;
+	int ret = 0;
+
+	if (copy_from_user(&params, (void __user *)arg, sizeof(params)))
+		return -EFAULT;
+
+	mutex_lock(&rv->mutex);
+	conn = user_conn_find(rv, params.handle);
+	if (!conn) {
+		rv_err(rv->inx, "connect: No connection found\n");
+		ret = -EINVAL;
+		goto unlock;
+	}
+
+	ret = rv_conn_connected(conn);
+unlock:
+	mutex_unlock(&rv->mutex);
+
+	return ret;
+}
+
+int doit_conn_get_conn_count(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_conn_get_conn_count_params params;
+	struct rv_conn *conn;
+	struct rv_sconn *sconn;
+	int ret = 0;
+	u8 index;
+
+	if (copy_from_user(&params.in, (void __user *)arg, sizeof(params.in)))
+		return -EFAULT;
+
+	mutex_lock(&rv->mutex);
+	if (!rv->attached) {
+		ret = rv->was_attached ? -ENXIO : -EINVAL;
+		goto unlock;
+	}
+	if (rv->rdma_mode != RV_RDMA_MODE_KERNEL) {
+		ret = -EINVAL;
+		goto unlock;
+	}
+
+	conn = user_conn_find(rv, params.in.handle);
+	if (!conn) {
+		rv_err(rv->inx, "get_conn_count: No connection found\n");
+		ret = -EINVAL;
+		goto unlock;
+	}
+	if (params.in.index >= conn->num_conn) {
+		rv_err(rv->inx, "get_conn_count: Invalid index: %d\n",
+		       params.in.index);
+		ret = -EINVAL;
+		goto unlock;
+	}
+	index = array_index_nospec(params.in.index, conn->num_conn);
+
+	sconn = &conn->sconn_arr[index];
+
+	mutex_lock(&sconn->mutex);
+	if (sconn->state == RV_ERROR)
+		ret = -EIO;
+	else
+		params.out.count = sconn->stats.conn_recovery +
+				   (test_bit(RV_SCONN_WAS_CONNECTED,
+					     &sconn->flags) ? 1 : 0);
+	mutex_unlock(&sconn->mutex);
+	if (ret)
+		goto unlock;
+
+	if (copy_to_user((void __user *)arg, &params.out, sizeof(params.out)))
+		ret = -EFAULT;
+unlock:
+	mutex_unlock(&rv->mutex);
+
+	return ret;
+}
+
+static void rv_sconn_add_stats(struct rv_sconn *sconn,
+			       struct rv_conn_get_stats_params *params)
+{
+	mutex_lock(&sconn->mutex);
+	params->out.num_conn++;
+	if (test_bit(RV_SCONN_SERVER, &sconn->flags))
+		params->out.flags |= RV_CONN_STAT_FLAG_SERVER;
+	else
+		params->out.flags |= RV_CONN_STAT_FLAG_CLIENT;
+	if (!test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags))
+		params->out.flags &= ~RV_CONN_STAT_FLAG_WAS_CONNECTED;
+
+#define RV_ADD_CM_EVT_STAT(sconn, params, s, evt) \
+	((params)->out.s += (sconn)->stats.cm_evt_cnt[evt])
+
+#define RV_ADD_STAT(sconn, params, s) \
+	((params)->out.s += (sconn)->stats.s)
+
+#define RV_ADD_ATOMIC_STAT(sconn, params, s) \
+	((params)->out.s += atomic_read(&(sconn)->stats.s))
+
+#define RV_ADD_ATOMIC64_STAT(sconn, params, s) \
+	((params)->out.s += atomic64_read(&(sconn)->stats.s))
+
+#define RV_MAX_STAT(sconn, params, s) \
+	((params)->out.s = max((params)->out.s, (sconn)->stats.s))
+
+	RV_ADD_CM_EVT_STAT(sconn, params, req_error, IB_CM_REQ_ERROR);
+	RV_ADD_CM_EVT_STAT(sconn, params, rep_error, IB_CM_REP_ERROR);
+	RV_ADD_CM_EVT_STAT(sconn, params, rep_recv, IB_CM_REP_RECEIVED);
+	RV_ADD_CM_EVT_STAT(sconn, params, rtu_recv, IB_CM_RTU_RECEIVED);
+	RV_ADD_CM_EVT_STAT(sconn, params, established, IB_CM_USER_ESTABLISHED);
+	RV_ADD_CM_EVT_STAT(sconn, params, dreq_error, IB_CM_DREQ_ERROR);
+	RV_ADD_CM_EVT_STAT(sconn, params, dreq_recv, IB_CM_DREQ_RECEIVED);
+	RV_ADD_CM_EVT_STAT(sconn, params, drep_recv, IB_CM_DREP_RECEIVED);
+	RV_ADD_CM_EVT_STAT(sconn, params, timewait, IB_CM_TIMEWAIT_EXIT);
+	RV_ADD_CM_EVT_STAT(sconn, params, mra_recv, IB_CM_MRA_RECEIVED);
+	RV_ADD_CM_EVT_STAT(sconn, params, rej_recv, IB_CM_REJ_RECEIVED);
+	RV_ADD_CM_EVT_STAT(sconn, params, lap_error, IB_CM_LAP_ERROR);
+	RV_ADD_CM_EVT_STAT(sconn, params, lap_recv, IB_CM_LAP_RECEIVED);
+	RV_ADD_CM_EVT_STAT(sconn, params, apr_recv, IB_CM_APR_RECEIVED);
+	RV_ADD_CM_EVT_STAT(sconn, params, unexp_event, RV_CM_EVENT_UNEXP);
+
+	RV_ADD_STAT(sconn, params, req_sent);
+	RV_ADD_STAT(sconn, params, rep_sent);
+	RV_ADD_STAT(sconn, params, rtu_sent);
+	RV_ADD_STAT(sconn, params, rej_sent);
+	RV_ADD_STAT(sconn, params, dreq_sent);
+	RV_ADD_STAT(sconn, params, drep_sent);
+
+	RV_MAX_STAT(sconn, params, wait_time);
+	RV_MAX_STAT(sconn, params, resolve_time);
+	RV_MAX_STAT(sconn, params, connect_time);
+	RV_MAX_STAT(sconn, params, connected_time);
+	RV_ADD_STAT(sconn, params, resolve);
+	RV_ADD_STAT(sconn, params, resolve_fail);
+	RV_ADD_STAT(sconn, params, conn_recovery);
+	RV_MAX_STAT(sconn, params, rewait_time);
+	RV_MAX_STAT(sconn, params, reresolve_time);
+	RV_MAX_STAT(sconn, params, reconnect_time);
+	RV_MAX_STAT(sconn, params, max_rewait_time);
+	RV_MAX_STAT(sconn, params, max_reresolve_time);
+	RV_MAX_STAT(sconn, params, max_reconnect_time);
+	RV_ADD_STAT(sconn, params, reresolve);
+	RV_ADD_STAT(sconn, params, reresolve_fail);
+	RV_ADD_STAT(sconn, params, post_write);
+	RV_ADD_STAT(sconn, params, post_write_fail);
+	RV_ADD_STAT(sconn, params, post_write_bytes);
+	RV_ADD_STAT(sconn, params, post_hb);
+	RV_ADD_STAT(sconn, params, post_hb_fail);
+
+	RV_ADD_ATOMIC_STAT(sconn, params, outstand_send_write);
+	RV_ADD_ATOMIC64_STAT(sconn, params, send_write_cqe);
+	RV_ADD_ATOMIC64_STAT(sconn, params, send_write_cqe_fail);
+	RV_ADD_ATOMIC64_STAT(sconn, params, recv_write_cqe);
+	RV_ADD_ATOMIC64_STAT(sconn, params, recv_write_bytes);
+	RV_ADD_ATOMIC64_STAT(sconn, params, recv_cqe_fail);
+	RV_ADD_ATOMIC64_STAT(sconn, params, send_hb_cqe);
+	RV_ADD_ATOMIC64_STAT(sconn, params, send_hb_cqe_fail);
+	RV_ADD_ATOMIC64_STAT(sconn, params, recv_hb_cqe);
+#undef RV_ADD_CM_EVT_STAT
+#undef RV_ADD_STAT
+#undef RV_ADD_ATOMIC_STAT
+#undef RV_ADD_ATOMIC64_STAT
+#undef RV_MAX_STAT
+	mutex_unlock(&sconn->mutex);
+}
+
+/* add up all the stats for sconns in given conn */
+static void rv_conn_add_stats(struct rv_conn *conn,
+			      struct rv_conn_get_stats_params *params)
+{
+	int i;
+
+	for (i = 0; i < conn->num_conn; i++)
+		rv_sconn_add_stats(&conn->sconn_arr[i], params);
+}
+
+int doit_conn_get_stats(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_conn_get_stats_params params;
+	struct rv_conn *conn;
+	int ret = 0;
+	u8 index;
+
+	if (copy_from_user(&params.in, (void __user *)arg, sizeof(params.in)))
+		return -EFAULT;
+
+	mutex_lock(&rv->mutex);
+	if (!rv->attached) {
+		ret = rv->was_attached ? -ENXIO : -EINVAL;
+		goto unlock;
+	}
+	if (rv->rdma_mode != RV_RDMA_MODE_KERNEL) {
+		ret = -EINVAL;
+		goto unlock;
+	}
+
+	if (params.in.handle) {
+		conn = user_conn_find(rv, params.in.handle);
+		if (!conn) {
+			rv_err(rv->inx,
+			       "conn_get_stats: No connection found\n");
+			ret = -EINVAL;
+			goto unlock;
+		}
+		index = params.in.index;
+
+		memset(&params, 0, sizeof(params));
+		params.out.flags = RV_CONN_STAT_FLAG_WAS_CONNECTED;
+		params.out.index = index;
+
+		if (index == RV_CONN_STATS_AGGREGATE) {
+			rv_conn_add_stats(conn, &params);
+		} else if (index >= conn->num_conn) {
+			ret = -EINVAL;
+			goto unlock;
+		} else {
+			index = array_index_nospec(index, conn->num_conn);
+			rv_sconn_add_stats(&conn->sconn_arr[index], &params);
+		}
+	} else {
+		XA_STATE(xas, &rv->conn_xa, 0);
+
+		memset(&params, 0, sizeof(params));
+		params.out.flags = RV_CONN_STAT_FLAG_WAS_CONNECTED;
+		params.out.index =  RV_CONN_STATS_AGGREGATE;
+
+		xas_for_each(&xas, conn, UINT_MAX)
+			rv_conn_add_stats(conn, &params);
+	}
+
+	if (copy_to_user((void __user *)arg, &params.out, sizeof(params.out)))
+		ret = -EFAULT;
+unlock:
+	mutex_unlock(&rv->mutex);
+
+	return ret;
+}
+
+/*
+ * We have a rv_conn reference for the heartbeat cqe
+ * We let QP Async Event callback handle the errors for us.
+ * Note: rv_conn_put can put rv_job_dev and trigger whole job cleanup
+ */
+static void rv_hb_done(struct ib_cq *cq, struct ib_wc *wc)
+{
+	struct rv_sconn *sconn = container_of(wc->wr_cqe,
+					      struct rv_sconn, hb_cqe);
+
+	trace_rv_wc_hb_done((u64)sconn, wc->status, wc->opcode, wc->byte_len,
+			    0);
+	trace_rv_sconn_hb_done(sconn, sconn->index,
+			       sconn->qp ? sconn->qp->qp_num : 0,
+			       sconn->parent, sconn->flags,
+			       (u32)sconn->state, 0);
+
+	if (wc->status) {
+		rv_report_cqe_error(cq, wc, sconn, "Heartbeat");
+		atomic64_inc(&sconn->stats.send_hb_cqe_fail);
+	} else {
+		struct rv_job_dev *jdev = sconn->parent->jdev;
+
+		WARN_ON(wc->qp != sconn->qp);
+		atomic64_inc(&sconn->stats.send_hb_cqe);
+		sconn->hb_timer.expires = jiffies +
+					  msecs_to_jiffies(jdev->hb_interval);
+		add_timer(&sconn->hb_timer);
+	}
+
+	rv_conn_put(sconn->parent);
+}
+
+/*
+ * issue HB WQE as needed.
+ * if there has been activity, no need for a new HB packet
+ * called on work queue with rv_conn reference held on our behalf
+ */
+static void rv_sconn_hb_work(struct work_struct *work)
+{
+	struct rv_sconn *sconn = container_of(work, struct rv_sconn, hb_work);
+	struct ib_send_wr swr = {
+			.opcode	= IB_WR_SEND,
+			.wr_cqe	= &sconn->hb_cqe,
+			.send_flags = IB_SEND_SIGNALED,
+	};
+	u64 old_act_count;
+	int ret;
+
+	mutex_lock(&sconn->mutex);
+
+	if (sconn->state != RV_CONNECTED)
+		goto unlock;
+
+	old_act_count =  sconn->act_count;
+	sconn->act_count = sconn->stats.post_write +
+			   atomic64_read(&sconn->stats.recv_write_cqe) +
+			   atomic64_read(&sconn->stats.recv_hb_cqe);
+	if (sconn->act_count > old_act_count) {
+		struct rv_job_dev *jdev = sconn->parent->jdev;
+
+		sconn->hb_timer.expires = jiffies +
+					  msecs_to_jiffies(jdev->hb_interval);
+		add_timer(&sconn->hb_timer);
+		goto unlock;
+	}
+
+	trace_rv_sconn_hb_post(sconn, sconn->index, sconn->qp->qp_num,
+			       sconn->parent, sconn->flags,
+			       (u32)sconn->state, 0);
+	rv_conn_get(sconn->parent);
+	ret = ib_post_send(sconn->qp, &swr, NULL);
+	if (ret) {
+		sconn->stats.post_hb_fail++;
+		rv_conn_err(sconn, "failed to send hb: post %d\n", ret);
+		rv_conn_put(sconn->parent);
+	} else {
+		sconn->stats.post_hb++;
+	}
+
+unlock:
+	mutex_unlock(&sconn->mutex);
+
+	rv_conn_put(sconn->parent);
+}
+
+/* called at SOFT IRQ,  so real work in WQ */
+static void rv_sconn_hb_func(struct timer_list *timer)
+{
+	struct rv_sconn *sconn = container_of(timer, struct rv_sconn, hb_timer);
+
+	if (!sconn->parent)
+		return;
+	if (rv_conn_get_check(sconn->parent))
+		return;
+	rv_queue_work(&sconn->hb_work);
+}
+
+static void rv_listener_release(struct kref *kref)
+{
+	struct rv_listener *listener =
+		container_of(kref, struct rv_listener, kref);
+	struct rv_device *dev = listener->dev;
+	unsigned long flags;
+
+	spin_lock_irqsave(&dev->listener_lock, flags);
+	list_del(&listener->listener_entry);
+	spin_unlock_irqrestore(&dev->listener_lock, flags);
+
+	ib_destroy_cm_id(listener->cm_id);
+
+	rv_device_put(dev);
+	kfree(listener);
+}
+
+void rv_listener_put(struct rv_listener *listener)
+{
+	trace_rv_listener_put(listener->dev->ib_dev->name,
+			      be64_to_cpu(listener->cm_id->service_id),
+			      kref_read(&listener->kref));
+	kref_put(&listener->kref, rv_listener_release);
+}
+
+/*
+ * only for use by rv_listener_get_alloc
+ * all others must use rv_listener_get_alloc or rv_listener_get
+ */
+static struct rv_listener *rv_listener_alloc(struct rv_device *dev,
+					     u64 service_id,
+					     ib_cm_handler handler)
+{
+	struct rv_listener *listener;
+	int ret;
+
+	listener = kzalloc(sizeof(*listener), GFP_KERNEL);
+	if (!listener)
+		return NULL;
+
+	listener->cm_id = ib_create_cm_id(dev->ib_dev, handler, listener);
+	if (IS_ERR(listener->cm_id)) {
+		rv_ptr_err("listener", listener, "Failed to create CM ID\n");
+		goto err_free;
+	}
+
+	ret = ib_cm_listen(listener->cm_id, cpu_to_be64(service_id), 0);
+	if (ret) {
+		rv_ptr_err("listener", listener, "CM listen failed: %d\n", ret);
+		goto err_cm;
+	}
+	rv_device_get(dev);
+	listener->dev = dev;
+
+	kref_init(&listener->kref);
+
+	return listener;
+err_cm:
+	ib_destroy_cm_id(listener->cm_id);
+err_free:
+	kfree(listener);
+
+	return NULL;
+}
+
+struct rv_listener *rv_listener_get_alloc(struct rv_device *dev, u64 service_id,
+					  ib_cm_handler handler)
+{
+	unsigned long flags;
+	struct rv_listener *entry = NULL;
+	__be64 sid = cpu_to_be64(service_id);
+
+	mutex_lock(&dev->listener_mutex);
+	spin_lock_irqsave(&dev->listener_lock, flags);
+	list_for_each_entry(entry, &dev->listener_list, listener_entry) {
+		if (sid == entry->cm_id->service_id) {
+			if (!kref_get_unless_zero(&entry->kref))
+				continue;
+			goto done;
+		}
+	}
+	spin_unlock_irqrestore(&dev->listener_lock, flags);
+	entry = rv_listener_alloc(dev, service_id, handler);
+	if (!entry)
+		goto unlock_mutex;
+	trace_rv_listener_get(dev->ib_dev->name, service_id,
+			      kref_read(&entry->kref));
+	spin_lock_irqsave(&dev->listener_lock, flags);
+	list_add(&entry->listener_entry, &dev->listener_list);
+
+done:
+	spin_unlock_irqrestore(&dev->listener_lock, flags);
+unlock_mutex:
+	mutex_unlock(&dev->listener_mutex);
+	return entry;
+}
diff --git a/drivers/infiniband/ulp/rv/rv_file.c b/drivers/infiniband/ulp/rv/rv_file.c
index 3625a9c1681a..9d23503a30d9 100644
--- a/drivers/infiniband/ulp/rv/rv_file.c
+++ b/drivers/infiniband/ulp/rv/rv_file.c
@@ -3,16 +3,49 @@
  * Copyright(c) 2020 - 2021 Intel Corporation.
  */
 
+#include <rdma/ib_cache.h>
+#include <linux/cdev.h>
+
 #include "rv.h"
+#include "trace.h"
 
 /* A workqueue for all */
 static struct workqueue_struct *rv_wq;
+static struct workqueue_struct *rv_wq2;
+static struct workqueue_struct *rv_wq3;
+
+/*
+ * We expect relatively few jobs per node (typically 1)
+ * and relatively few devices per node (typically 1 to 8)
+ * so the list of job_dev's should be short and is only used
+ * at job launch and shutdown.
+ *
+ * search key is job_key, dev_name, port_num; short list linear search ok
+ * mutex avoids duplicate get_alloc adds, RCU protects list access.
+ * See rv.h comments about "get_alloc" for more information.
+ */
+static struct list_head rv_job_dev_list;
 
 void rv_queue_work(struct work_struct *work)
 {
 	queue_work(rv_wq, work);
 }
 
+void rv_queue_work2(struct work_struct *work)
+{
+	queue_work(rv_wq2, work);
+}
+
+void rv_queue_work3(struct work_struct *work)
+{
+	queue_work(rv_wq3, work);
+}
+
+void rv_flush_work2(void)
+{
+	flush_workqueue(rv_wq2);
+}
+
 void rv_job_dev_get(struct rv_job_dev *jdev)
 {
 	kref_get(&jdev->kref);
@@ -29,3 +62,95 @@ void rv_job_dev_put(struct rv_job_dev *jdev)
 {
 	kref_put(&jdev->kref, rv_job_dev_release);
 }
+
+/*
+ * confirm that we expected a REQ from this remote node on this port.
+ * Note CM swaps src vs dest so dest is remote node here
+ */
+static struct rv_sconn *
+rv_conn_match_req(struct rv_conn *conn,
+		  const struct ib_cm_req_event_param *param,
+		  struct rv_req_priv_data *priv_data)
+{
+	if (param->port != conn->ah.port_num)
+		return NULL;
+	if ((param->primary_path->rec_type == SA_PATH_REC_TYPE_IB &&
+	     be16_to_cpu(param->primary_path->ib.dlid) != conn->ah.dlid) ||
+	    (param->primary_path->rec_type == SA_PATH_REC_TYPE_OPA &&
+	     be32_to_cpu(param->primary_path->opa.dlid) != conn->ah.dlid) ||
+	    (conn->ah.is_global &&
+	     cmp_gid(&param->primary_path->dgid, conn->ah.grh.dgid)))
+		return NULL;
+
+	if (priv_data->index >= conn->num_conn)
+		return NULL;
+
+	return &conn->sconn_arr[priv_data->index];
+}
+
+/*
+ * Within an rv_job_dev, find the server rv_sconn which matches the incoming
+ * CM request
+ * We are holding the rv_job_dev_list rcu_read_lock
+ * If found, the refcount for the rv_conn_info will be incremented.
+ */
+static struct rv_sconn *
+rv_jdev_find_conn(struct rv_job_dev *jdev,
+		  const struct ib_cm_req_event_param *param,
+		  struct rv_req_priv_data *priv_data)
+{
+	struct rv_conn *conn;
+	struct rv_sconn *sconn = NULL;
+
+	rcu_read_lock();
+	list_for_each_entry_rcu(conn, &jdev->conn_list, conn_entry) {
+		WARN_ON(jdev != conn->jdev);
+		sconn = rv_conn_match_req(conn, param, priv_data);
+		if (!sconn)
+			continue;
+		if (!kref_get_unless_zero(&conn->kref))
+			continue;
+		break;
+	}
+	rcu_read_unlock();
+
+	return sconn;
+}
+
+/*
+ * Find the rv_sconn matching the received REQ
+ * listener may be shared by rv_job_dev's so filter on dev 1st
+ */
+struct rv_sconn *
+rv_find_sconn_from_req(struct ib_cm_id *id,
+		       const struct ib_cm_req_event_param *param,
+		       struct rv_req_priv_data *priv_data)
+{
+	struct rv_sconn *sconn = NULL;
+	struct rv_listener *listener = id->context;
+	struct rv_job_dev *jdev;
+
+	rcu_read_lock();
+	list_for_each_entry_rcu(jdev, &rv_job_dev_list, job_dev_entry) {
+		if (listener->dev != jdev->dev)
+			continue;
+		if (priv_data->uid != jdev->uid)
+			continue;
+		if (priv_data->job_key_len != jdev->job_key_len ||
+		    memcmp(priv_data->job_key, jdev->job_key,
+			   jdev->job_key_len))
+			continue;
+		if (param->port != jdev->port_num ||
+		    cmp_gid(&param->primary_path->sgid, jdev->loc_gid))
+			continue;
+		if (!rv_job_dev_has_users(jdev))
+			continue;
+
+		sconn = rv_jdev_find_conn(jdev, param, priv_data);
+		if (sconn)
+			break;
+	}
+	rcu_read_unlock();
+
+	return sconn;
+}
diff --git a/drivers/infiniband/ulp/rv/rv_rdma.c b/drivers/infiniband/ulp/rv/rv_rdma.c
new file mode 100644
index 000000000000..10334c0441a5
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/rv_rdma.c
@@ -0,0 +1,103 @@
+// SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause)
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+
+#include "rv.h"
+#include "trace.h"
+
+/*
+ * This is called in Soft IRQs for CQE handling.
+ * We just report errors here, let the QP Async Event deal with
+ * how the sconn will react to the QP moving to QPS_ERR
+ */
+void rv_report_cqe_error(struct ib_cq *cq, struct ib_wc *wc,
+			 struct rv_sconn *sconn, const char *opname)
+{
+	if (wc->status != IB_WC_WR_FLUSH_ERR)
+		rv_conn_err(sconn,
+			    "failed %s qp %u status %s (%d) for CQE %p\n",
+			    opname, wc->qp ? wc->qp->qp_num : 0,
+			    ib_wc_status_msg(wc->status), wc->status,
+			    wc->wr_cqe);
+}
+
+static int rv_drv_post_recv(struct rv_sconn *sconn)
+{
+	struct ib_recv_wr wr;
+	const struct ib_recv_wr *bad_wr;
+
+	trace_rv_sconn_recv_post(sconn, sconn->index, sconn->qp->qp_num,
+				 sconn->parent, sconn->flags,
+				 (u32)sconn->state, 0);
+
+	wr.next = NULL;
+	wr.wr_cqe = &sconn->cqe;
+	wr.sg_list = NULL;
+	wr.num_sge = 0; /* only expect inbound RDMA Write w/immed */
+	return ib_post_recv(sconn->qp, &wr, &bad_wr);
+}
+
+int rv_drv_prepost_recv(struct rv_sconn *sconn)
+{
+	int i;
+	int ret;
+	u32 qp_depth = sconn->parent->jdev->qp_depth;
+
+	trace_rv_msg_prepost_recv(sconn, sconn->index, "prepost recv",
+				  (u64)qp_depth, (u64)sconn);
+	for (i = 0; i < qp_depth; i++) {
+		ret = rv_drv_post_recv(sconn);
+		if (ret)
+			return ret;
+	}
+	return 0;
+}
+
+/* drain_lock makes sure no recv WQEs get reposted after a drain WQE */
+void rv_recv_done(struct ib_cq *cq, struct ib_wc *wc)
+{
+	struct rv_sconn *sconn = container_of(wc->wr_cqe,
+					      struct rv_sconn, cqe);
+	unsigned long flags;
+
+	trace_rv_wc_recv_done((u64)sconn, wc->status, wc->opcode, wc->byte_len,
+			      be32_to_cpu(wc->ex.imm_data));
+	if (!sconn->parent)
+		return;
+	if (rv_conn_get_check(sconn->parent))
+		return;
+	trace_rv_sconn_recv_done(sconn, sconn->index,
+				 wc->qp->qp_num, sconn->parent, sconn->flags,
+				 (u32)(sconn->state),
+				 be32_to_cpu(wc->ex.imm_data));
+	if (unlikely(wc->status != IB_WC_SUCCESS)) {
+		if (wc->status != IB_WC_WR_FLUSH_ERR) {
+			rv_report_cqe_error(cq, wc, sconn, "Recv bad status");
+			atomic64_inc(&sconn->stats.recv_cqe_fail);
+		}
+		goto put;
+	}
+	if (wc->qp != sconn->qp)
+		goto put;
+
+	if (unlikely(wc->opcode == IB_WC_RECV)) {
+		atomic64_inc(&sconn->stats.recv_hb_cqe);
+		goto repost;
+	}
+
+	/* use relaxed, no big deal if stats updated out of order */
+	atomic64_inc(&sconn->stats.recv_write_cqe);
+	atomic64_add_return_relaxed(wc->byte_len,
+				    &sconn->stats.recv_write_bytes);
+
+	if (unlikely(wc->opcode != IB_WC_RECV_RDMA_WITH_IMM))
+		rv_report_cqe_error(cq, wc, sconn, "Recv bad opcode");
+repost:
+	spin_lock_irqsave(&sconn->drain_lock, flags);
+	if (likely(!test_bit(RV_SCONN_DRAINING, &sconn->flags)))
+		rv_drv_post_recv(sconn);
+	spin_unlock_irqrestore(&sconn->drain_lock, flags);
+put:
+	rv_conn_put(sconn->parent);
+}
diff --git a/drivers/infiniband/ulp/rv/trace.h b/drivers/infiniband/ulp/rv/trace.h
index d2827582be05..8dc3313342e9 100644
--- a/drivers/infiniband/ulp/rv/trace.h
+++ b/drivers/infiniband/ulp/rv/trace.h
@@ -3,6 +3,8 @@
  * Copyright(c) 2020 - 2021 Intel Corporation.
  */
 #include "trace_mr_cache.h"
+#include "trace_conn.h"
 #include "trace_dev.h"
 #include "trace_mr.h"
 #include "trace_user.h"
+#include "trace_rdma.h"
diff --git a/drivers/infiniband/ulp/rv/trace_conn.h b/drivers/infiniband/ulp/rv/trace_conn.h
new file mode 100644
index 000000000000..2626545b0df6
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/trace_conn.h
@@ -0,0 +1,529 @@
+/* SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause) */
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+#if !defined(__RV_TRACE_CONN_H) || defined(TRACE_HEADER_MULTI_READ)
+#define __RV_TRACE_CONN_H
+
+#include <linux/tracepoint.h>
+#include <linux/trace_seq.h>
+
+#undef TRACE_SYSTEM
+#define TRACE_SYSTEM rv_conn
+
+#define RV_CONN_REQ_PRN  "rem_addr 0x%x global %u sgid_inx %u port_num %u " \
+			 "dlid 0x%x dgid 0x%llx %llx"
+
+#define RV_CONN_PRN  "Conn 0x%p rem_addr 0x%x global %u dlid 0x%x " \
+		     "dgid 0x%llx %llx num_conn %u next %u jdev 0x%p " \
+		     "refcount %u"
+
+#define RV_JDEV_PRN "jdev 0x%p dev %p num_conn %u index_bits %u " \
+		    "loc_gid_index %u loc addr 0x%x jkey_len %u " \
+		    "jkey 0x%s sid 0x%llx q_depth %u ua_next %u "\
+		    "refcount %u"
+
+#define RV_SCONN_PRN "sconn %p index %u qp 0x%x conn %p flags 0x%x state %u " \
+		     "cm_id %p retry %u"
+
+DECLARE_EVENT_CLASS(/* listener */
+	rv_listener_template,
+	TP_PROTO(const char *dev_name, u64 svc_id, u32 refcount),
+	TP_ARGS(dev_name, svc_id, refcount),
+	TP_STRUCT__entry(/* entry */
+		__string(name, dev_name)
+		__field(u64, sid)
+		__field(u32, count)
+	),
+	TP_fast_assign(/* assign */
+		__assign_str(name, dev_name);
+		__entry->sid = svc_id;
+		__entry->count = refcount;
+	),
+	TP_printk(/* print */
+		"Device %s sid 0x%llx refcount %u",
+		__get_str(name),
+		__entry->sid,
+		__entry->count
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_listener_template, rv_listener_get,
+	TP_PROTO(const char *dev_name, u64 svc_id, u32 refcount),
+	TP_ARGS(dev_name, svc_id, refcount)
+);
+
+DEFINE_EVENT(/* event */
+	rv_listener_template, rv_listener_put,
+	TP_PROTO(const char *dev_name, u64 svc_id, u32 refcount),
+	TP_ARGS(dev_name, svc_id, refcount)
+);
+
+TRACE_EVENT(/* event */
+	rv_conn_create_req,
+	TP_PROTO(u32 rem_addr, u8 global, u8 sgid_inx, u8 port_num, u16 dlid,
+		 u64 dgid1, u64 dgid2),
+	TP_ARGS(rem_addr, global, sgid_inx, port_num, dlid, dgid1, dgid2),
+	TP_STRUCT__entry(/* entry */
+		__field(u32, rem_addr)
+		__field(u8, global)
+		__field(u8, sgid_inx)
+		__field(u8, port_num)
+		__field(u16, dlid)
+		__field(u64, dgid1)
+		__field(u64, dgid2)
+	),
+	TP_fast_assign(/* assign */
+		__entry->rem_addr = rem_addr;
+		__entry->global = global;
+		__entry->sgid_inx = sgid_inx;
+		__entry->port_num = port_num;
+		__entry->dlid = dlid;
+		__entry->dgid1 = dgid1;
+		__entry->dgid2 = dgid2;
+	),
+	TP_printk(/* print */
+		RV_CONN_REQ_PRN,
+		__entry->rem_addr,
+		__entry->global,
+		__entry->sgid_inx,
+		__entry->port_num,
+		__entry->dlid,
+		__entry->dgid1,
+		__entry->dgid2
+	)
+);
+
+DECLARE_EVENT_CLASS(/* conn */
+	rv_conn_template,
+	TP_PROTO(void *ptr, u32 rem_addr, u8 global, u16 dlid, u64 dgid1,
+		 u64 dgid2, u8 num_conn, u32 next, void *jdev, u32 refcount),
+	TP_ARGS(ptr, rem_addr, global, dlid, dgid1, dgid2, num_conn, next,
+		jdev, refcount),
+	TP_STRUCT__entry(/* entry */
+		__field(void *, ptr)
+		__field(u32, rem_addr)
+		__field(u8, global)
+		__field(u16, dlid)
+		__field(u64, dgid1)
+		__field(u64, dgid2)
+		__field(u8, num_conn)
+		__field(u32, next)
+		__field(void *, jdev)
+		__field(u32, refcount)
+	),
+	TP_fast_assign(/* assign */
+		__entry->ptr = ptr;
+		__entry->rem_addr = rem_addr;
+		__entry->global = global;
+		__entry->dlid = dlid;
+		__entry->dgid1 = dgid1;
+		__entry->dgid2 = dgid2;
+		__entry->num_conn = num_conn;
+		__entry->next = next;
+		__entry->jdev = jdev;
+		__entry->refcount = refcount;
+	),
+	TP_printk(/* print */
+		RV_CONN_PRN,
+		__entry->ptr,
+		__entry->rem_addr,
+		__entry->global,
+		__entry->dlid,
+		__entry->dgid1,
+		__entry->dgid2,
+		__entry->num_conn,
+		__entry->next,
+		__entry->jdev,
+		__entry->refcount
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_conn_template, rv_conn_create,
+	TP_PROTO(void *ptr, u32 rem_addr, u8 global, u16 dlid, u64 dgid1,
+		 u64 dgid2, u8 num_conn, u32 next, void *jdev, u32 refcount),
+	TP_ARGS(ptr, rem_addr, global, dlid, dgid1, dgid2, num_conn, next,
+		jdev, refcount)
+);
+
+DEFINE_EVENT(/* event */
+	rv_conn_template, rv_conn_alloc,
+	TP_PROTO(void *ptr, u32 rem_addr, u8 global, u16 dlid, u64 dgid1,
+		 u64 dgid2, u8 num_conn, u32 next, void *jdev, u32 refcount),
+	TP_ARGS(ptr, rem_addr, global, dlid, dgid1, dgid2, num_conn, next,
+		jdev, refcount)
+);
+
+DEFINE_EVENT(/* event */
+	rv_conn_template, rv_conn_release,
+	TP_PROTO(void *ptr, u32 rem_addr, u8 global, u16 dlid, u64 dgid1,
+		 u64 dgid2, u8 num_conn, u32 next, void *jdev, u32 refcount),
+	TP_ARGS(ptr, rem_addr, global, dlid, dgid1, dgid2, num_conn, next,
+		jdev, refcount)
+);
+
+DEFINE_EVENT(/* event */
+	rv_conn_template, rv_conn_connect,
+	TP_PROTO(void *ptr, u32 rem_addr, u8 global, u16 dlid, u64 dgid1,
+		 u64 dgid2, u8 num_conn, u32 next, void *jdev, u32 refcount),
+	TP_ARGS(ptr, rem_addr, global, dlid, dgid1, dgid2, num_conn, next,
+		jdev, refcount)
+);
+
+DECLARE_EVENT_CLASS(/* jdev */
+	rv_jdev_template,
+	TP_PROTO(void *ptr, const char *dev_name, u8 num_conn, u8 index_bits,
+		 u16 loc_gid_index, u32 loc_addr, u8 jkey_len, u8 *jkey,
+		 u64 sid, u32 q_depth, u32 ua_next, u32 refcount),
+	TP_ARGS(ptr, dev_name, num_conn, index_bits, loc_gid_index, loc_addr,
+		jkey_len, jkey, sid, q_depth, ua_next, refcount),
+	TP_STRUCT__entry(/* entry */
+		__field(void *, ptr)
+		__string(name, dev_name)
+		__field(u8, num_conn)
+		__field(u8, index_bits)
+		__field(u16, loc_gid_index)
+		__field(u32, loc_addr)
+		__field(u8, jkey_len)
+		__array(u8, jkey, RV_MAX_JOB_KEY_LEN)
+		__field(u64, sid)
+		__field(u32, q_depth)
+		__field(u32, ua_next)
+		__field(u32, refcount)
+	),
+	TP_fast_assign(/* assign */
+		__entry->ptr = ptr;
+		__assign_str(name, dev_name);
+		__entry->num_conn = num_conn;
+		__entry->index_bits = index_bits;
+		__entry->loc_gid_index = loc_gid_index;
+		__entry->loc_addr = loc_addr;
+		__entry->jkey_len = jkey_len;
+		memcpy(__entry->jkey, jkey, RV_MAX_JOB_KEY_LEN);
+		__entry->sid = sid;
+		__entry->q_depth = q_depth;
+		__entry->ua_next = ua_next;
+		__entry->refcount = refcount;
+	),
+	TP_printk(/* print */
+		RV_JDEV_PRN,
+		__entry->ptr,
+		__get_str(name),
+		__entry->num_conn,
+		__entry->index_bits,
+		__entry->loc_gid_index,
+		__entry->loc_addr,
+		__entry->jkey_len,
+		__print_hex_str(__entry->jkey, RV_MAX_JOB_KEY_LEN),
+		__entry->sid,
+		__entry->q_depth,
+		__entry->ua_next,
+		__entry->refcount
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_jdev_template, rv_jdev_conn_create,
+	TP_PROTO(void *ptr, const char *dev_name, u8 num_conn, u8 index_bits,
+		 u16 loc_gid_index, u32 loc_addr, u8 jkey_len, u8 *jkey,
+		 u64 sid, u32 q_depth, u32 ua_next, u32 refcount),
+	TP_ARGS(ptr, dev_name, num_conn, index_bits, loc_gid_index, loc_addr,
+		jkey_len, jkey, sid, q_depth, ua_next, refcount)
+);
+
+DECLARE_EVENT_CLASS(/* sconn */
+	rv_sconn_template,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry),
+	TP_STRUCT__entry(/* entry */
+		__field(void *, ptr)
+		__field(u8, index)
+		__field(u32, qp_num)
+		__field(void *, conn)
+		__field(u32, flags)
+		__field(u32, state)
+		__field(void *, cm_id)
+		__field(u32, retry)
+	),
+	TP_fast_assign(/* assign */
+		__entry->ptr = ptr;
+		__entry->index = index;
+		__entry->qp_num = qp_num;
+		__entry->conn = conn;
+		__entry->flags = flags;
+		__entry->state = state;
+		__entry->cm_id = cm_id;
+		__entry->retry = retry;
+	),
+	TP_printk(/* print */
+		 RV_SCONN_PRN,
+		__entry->ptr,
+		__entry->index,
+		__entry->qp_num,
+		__entry->conn,
+		__entry->flags,
+		__entry->state,
+		__entry->cm_id,
+		__entry->retry
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_init,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_deinit,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_resolve,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_resolve_cb,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_cm_handler,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_set_state,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_req_handler,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_done_discon,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_drain_done,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_cq_event,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_qp_event,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_timeout_work,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_template, rv_sconn_delay_work,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, void *cm_id, u32 retry),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, cm_id, retry)
+);
+
+DECLARE_EVENT_CLASS(/* cm_event */
+	rv_cm_event_template,
+	TP_PROTO(u32 evt, void *cm_id, void *sconn),
+	TP_ARGS(evt, cm_id, sconn),
+	TP_STRUCT__entry(/* entry */
+		__field(u32, event)
+		__field(void *, cm_id)
+		__field(void *, sconn)
+	),
+	TP_fast_assign(/* assign */
+		__entry->event = evt;
+		__entry->cm_id = cm_id;
+		__entry->sconn = sconn;
+	),
+	TP_printk(/* print */
+		"Event %u cm_id %p sconn %p",
+		__entry->event,
+		__entry->cm_id,
+		__entry->sconn
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_cm_event_template, rv_cm_event_handler,
+	TP_PROTO(u32 evt, void *cm_id, void *sconn),
+	TP_ARGS(evt, cm_id, sconn)
+);
+
+DEFINE_EVENT(/* event */
+	rv_cm_event_template, rv_cm_event_server_handler,
+	TP_PROTO(u32 evt, void *cm_id, void *sconn),
+	TP_ARGS(evt, cm_id, sconn)
+);
+
+DECLARE_EVENT_CLASS(/* msg */
+	rv_sconn_msg_template,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2),
+	TP_STRUCT__entry(/* entry */
+		__field(void *, ptr)
+		__field(u8, index)
+		__string(msg, msg)
+		__field(u64, d1)
+		__field(u64, d2)
+	),
+	TP_fast_assign(/* assign */
+		__entry->ptr = ptr;
+		__entry->index = index;
+		__assign_str(msg, msg);
+		__entry->d1 = d1;
+		__entry->d2 = d2;
+	),
+	TP_printk(/* print */
+		"sconn %p index %u: %s 0x%llx 0x%llx",
+		__entry->ptr,
+		__entry->index,
+		__get_str(msg),
+		__entry->d1,
+		__entry->d2
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_msg_template, rv_msg_destroy_qp,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_msg_template, rv_msg_send_req,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_msg_template, rv_msg_qp_rtr,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_msg_template, rv_msg_cm_handler,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_msg_template, rv_msg_cm_rep_handler,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_msg_template, rv_msg_enter_disconnect,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_msg_template, rv_msg_cm_req_handler,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_msg_template, rv_msg_sconn_timeout_work,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_msg_template, rv_msg_cq_event,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_msg_template, rv_msg_qp_event,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_msg_template, rv_msg_prepost_recv,
+	TP_PROTO(void *ptr, u8 index, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(ptr, index, msg, d1, d2)
+);
+
+DECLARE_EVENT_CLASS(/* msg */
+	rv_msg_template,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2),
+	TP_STRUCT__entry(/* entry */
+		__field(int, inx)
+		__string(msg, msg)
+		__field(u64, d1)
+		__field(u64, d2)
+	),
+	TP_fast_assign(/* assign */
+		__entry->inx = inx;
+		__assign_str(msg, msg);
+		__entry->d1 = d1;
+		__entry->d2 = d2;
+	),
+	TP_printk(/* print */
+		"inx %u: %s 0x%llx 0x%llx",
+		__entry->inx,
+		__get_str(msg),
+		__entry->d1,
+		__entry->d2
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_msg_template, rv_msg_err_qp,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2)
+);
+#endif /* __RV_TRACE_CONN_H */
+
+#undef TRACE_INCLUDE_PATH
+#undef TRACE_INCLUDE_FILE
+#define TRACE_INCLUDE_PATH .
+#define TRACE_INCLUDE_FILE trace_conn
+#include <trace/define_trace.h>
diff --git a/drivers/infiniband/ulp/rv/trace_rdma.h b/drivers/infiniband/ulp/rv/trace_rdma.h
new file mode 100644
index 000000000000..54ea0cf2599c
--- /dev/null
+++ b/drivers/infiniband/ulp/rv/trace_rdma.h
@@ -0,0 +1,129 @@
+/* SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause) */
+/*
+ * Copyright(c) 2020 - 2021 Intel Corporation.
+ */
+#if !defined(__RV_TRACE_RDMA_H) || defined(TRACE_HEADER_MULTI_READ)
+#define __RV_TRACE_RDMA_H
+
+#include <linux/tracepoint.h>
+#include <linux/trace_seq.h>
+
+#undef TRACE_SYSTEM
+#define TRACE_SYSTEM rv_rdma
+
+#define RV_SCONN_RECV_PRN "sconn %p index %u qp 0x%x conn %p flags 0x%x " \
+			  " state %u immed 0x%x"
+
+DECLARE_EVENT_CLASS(/* recv */
+	rv_sconn_recv_template,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, u32 immed),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, immed),
+	TP_STRUCT__entry(/* entry */
+		__field(void *, ptr)
+		__field(u8, index)
+		__field(u32, qp_num)
+		__field(void *, conn)
+		__field(u32, flags)
+		__field(u32, state)
+		__field(u32, immed)
+	),
+	TP_fast_assign(/* assign */
+		__entry->ptr = ptr;
+		__entry->index = index;
+		__entry->qp_num = qp_num;
+		__entry->conn = conn;
+		__entry->flags = flags;
+		__entry->state = state;
+		__entry->immed = immed;
+	),
+	TP_printk(/* print */
+		 RV_SCONN_RECV_PRN,
+		__entry->ptr,
+		__entry->index,
+		__entry->qp_num,
+		__entry->conn,
+		__entry->flags,
+		__entry->state,
+		__entry->immed
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_recv_template, rv_sconn_recv_done,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, u32 immed),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, immed)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_recv_template, rv_sconn_recv_post,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, u32 immed),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, immed)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_recv_template, rv_sconn_hb_done,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, u32 immed),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, immed)
+);
+
+DEFINE_EVENT(/* event */
+	rv_sconn_recv_template, rv_sconn_hb_post,
+	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
+		 u32 state, u32 immed),
+	TP_ARGS(ptr, index, qp_num, conn, flags, state, immed)
+);
+
+DECLARE_EVENT_CLASS(/* wc */
+	rv_wc_template,
+	TP_PROTO(u64 wr_id, u32 status, u32 opcode, u32 byte_len,
+		 u32 imm_data),
+	TP_ARGS(wr_id, status, opcode, byte_len, imm_data),
+	TP_STRUCT__entry(/* entry */
+		__field(u64, wr_id)
+		__field(u32, status)
+		__field(u32, opcode)
+		__field(u32, byte_len)
+		__field(u32, imm_data)
+	),
+	TP_fast_assign(/* assign */
+		__entry->wr_id = wr_id;
+		__entry->status = status;
+		__entry->opcode = opcode;
+		__entry->byte_len = byte_len;
+		__entry->imm_data = imm_data;
+	),
+	TP_printk(/* print */
+		"wr_id 0x%llx status 0x%x opcode 0x%x byte_len 0x%x immed 0x%x",
+		__entry->wr_id,
+		__entry->status,
+		__entry->opcode,
+		__entry->byte_len,
+		__entry->imm_data
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_wc_template, rv_wc_recv_done,
+	TP_PROTO(u64 wr_id, u32 status, u32 opcode, u32 byte_len,
+		 u32 imm_data),
+	TP_ARGS(wr_id, status, opcode, byte_len, imm_data)
+);
+
+DEFINE_EVENT(/* event */
+	rv_wc_template, rv_wc_hb_done,
+	TP_PROTO(u64 wr_id, u32 status, u32 opcode, u32 byte_len,
+		 u32 imm_data),
+	TP_ARGS(wr_id, status, opcode, byte_len, imm_data)
+);
+
+#endif /* __RV_TRACE_RDMA_H */
+
+#undef TRACE_INCLUDE_PATH
+#undef TRACE_INCLUDE_FILE
+#define TRACE_INCLUDE_PATH .
+#define TRACE_INCLUDE_FILE trace_rdma
+#include <trace/define_trace.h>
diff --git a/drivers/infiniband/ulp/rv/trace_user.h b/drivers/infiniband/ulp/rv/trace_user.h
index 2707e39bdfd6..ce62c808ca10 100644
--- a/drivers/infiniband/ulp/rv/trace_user.h
+++ b/drivers/infiniband/ulp/rv/trace_user.h
@@ -1,6 +1,6 @@
 /* SPDX-License-Identifier: (GPL-2.0 OR BSD-3-Clause) */
 /*
- * Copyright(c) 2020 Intel Corporation.
+ * Copyright(c) 2020 - 2021 Intel Corporation.
  */
 #if !defined(__RV_TRACE_USER_H) || defined(TRACE_HEADER_MULTI_READ)
 #define __RV_TRACE_USER_H
@@ -14,6 +14,61 @@
 #define RV_USER_MRS_PRN "rv_nx %d jdev %p total_size 0x%llx max_size 0x%llx " \
 			"refcount %u"
 
+DECLARE_EVENT_CLASS(/* user msg */
+	rv_user_msg_template,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2),
+	TP_STRUCT__entry(/* entry */
+		__field(int, inx)
+		__string(msg, msg)
+		__field(u64, d1)
+		__field(u64, d2)
+	),
+	TP_fast_assign(/* assign */
+		__entry->inx = inx;
+		__assign_str(msg, msg);
+		__entry->d1 = d1;
+		__entry->d2 = d2;
+	),
+	TP_printk(/* print */
+		"rv_nx %d: %s 0x%llx 0x%llx",
+		__entry->inx,
+		__get_str(msg),
+		__entry->d1,
+		__entry->d2
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_msg_template, rv_msg_uconn_create,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_msg_template, rv_msg_uconn_connect,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_msg_template, rv_msg_cmp_params,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_msg_template, rv_msg_conn_exist,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_msg_template, rv_msg_conn_create,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2)
+);
+
 DECLARE_EVENT_CLASS(/* user_mrs */
 	rv_user_mrs_template,
 	TP_PROTO(int rv_inx, void *jdev, u64 total_size, u64 max_size,
-- 
2.18.1


^ permalink raw reply related	[flat|nested] 52+ messages in thread

* [PATCH RFC 7/9] RDMA/rv: Add functions for RDMA transactions
  2021-03-19 12:56 [PATCH RFC 0/9] A rendezvous module kaike.wan
                   ` (5 preceding siblings ...)
  2021-03-19 12:56 ` [PATCH RFC 6/9] RDMA/rv: Add connection management functions kaike.wan
@ 2021-03-19 12:56 ` kaike.wan
  2021-03-19 12:56 ` [PATCH RFC 8/9] RDMA/rv: Add functions for file operations kaike.wan
                   ` (2 subsequent siblings)
  9 siblings, 0 replies; 52+ messages in thread
From: kaike.wan @ 2021-03-19 12:56 UTC (permalink / raw)
  To: dledford, jgg; +Cc: linux-rdma, todd.rimmer, Kaike Wan

From: Kaike Wan <kaike.wan@intel.com>

The only RDMA request used by this module is the RDMA WRITE WITH
IMMEDIATE request. Part of the immediate data is used as a tag to
encode the intended receiving rv_user in the rv_conn object, and
remaining bits are reserved for the user application (eg. to associate
the inbound completion with a specific outstanding rendezvous IO).

This patch adds the following functions:
- Send RDMA write with immediate request.
- Handle the send completion event.
- Receive the RDMA write with immediate request.
- Post events to the event ring.

Signed-off-by: Todd Rimmer <todd.rimmer@intel.com>
Signed-off-by: Kaike Wan <kaike.wan@intel.com>
---
 drivers/infiniband/ulp/rv/rv_rdma.c    | 313 +++++++++++++++++++++++++
 drivers/infiniband/ulp/rv/trace_rdma.h | 125 ++++++++++
 drivers/infiniband/ulp/rv/trace_user.h |  31 +++
 3 files changed, 469 insertions(+)

diff --git a/drivers/infiniband/ulp/rv/rv_rdma.c b/drivers/infiniband/ulp/rv/rv_rdma.c
index 10334c0441a5..c30773e09ffe 100644
--- a/drivers/infiniband/ulp/rv/rv_rdma.c
+++ b/drivers/infiniband/ulp/rv/rv_rdma.c
@@ -6,6 +6,56 @@
 #include "rv.h"
 #include "trace.h"
 
+/*
+ * select next sconn to post and claim WQE by inc outstand_send_write
+ * if all sconn SQs are full, next is left back where it started
+ */
+static struct rv_sconn *rv_conn_next_sconn_to_post(struct rv_conn *conn)
+{
+	unsigned long flags;
+	struct rv_sconn *sconn;
+	u8 i;
+	u32 qp_depth = conn->jdev->qp_depth;
+
+	spin_lock_irqsave(&conn->next_lock, flags);
+	for (i = 0; i < conn->num_conn; i++) {
+		sconn = &conn->sconn_arr[conn->next];
+		conn->next = (conn->next + 1) % conn->num_conn;
+		if (atomic_read(&sconn->stats.outstand_send_write) < qp_depth) {
+			atomic_inc(&sconn->stats.outstand_send_write);
+			goto unlock;
+		}
+	}
+	sconn = NULL;
+unlock:
+	spin_unlock_irqrestore(&conn->next_lock, flags);
+	return sconn;
+}
+
+static int rv_drv_post_write_immed(struct rv_pend_write *pend_wr)
+{
+	struct ib_rdma_wr wr;
+	const struct ib_send_wr *bad_wr;
+	struct ib_sge list;
+	struct rv_mr_cached *mrc = pend_wr->mrc;
+
+	/* we xlat the user space loc_addr to an iova appropriate for the MR */
+	list.addr = mrc->mr.ib_mr->iova + (pend_wr->loc_addr - mrc->addr);
+	list.length = pend_wr->length;
+	list.lkey = mrc->mr.ib_mr->lkey;
+
+	wr.wr.next = NULL;
+	wr.wr.wr_cqe = &pend_wr->cqe;
+	wr.wr.sg_list = &list;
+	wr.wr.num_sge = 1;
+	wr.wr.opcode = IB_WR_RDMA_WRITE_WITH_IMM;
+	wr.wr.send_flags = IB_SEND_SIGNALED;
+	wr.wr.ex.imm_data = cpu_to_be32(pend_wr->immed);
+	wr.remote_addr = pend_wr->rem_addr;
+	wr.rkey = pend_wr->rkey;
+	return ib_post_send(pend_wr->sconn->qp, &wr.wr, &bad_wr);
+}
+
 /*
  * This is called in Soft IRQs for CQE handling.
  * We just report errors here, let the QP Async Event deal with
@@ -22,6 +72,246 @@ void rv_report_cqe_error(struct ib_cq *cq, struct ib_wc *wc,
 			    wc->wr_cqe);
 }
 
+static void rv_user_ring_post_event(struct rv_user_ring *ring,
+				    struct rv_event *ev)
+{
+	unsigned long flags;
+	struct rv_ring_header *hdr = ring->hdr;
+	int next;
+
+	trace_rv_user_ring_post_event(ring->rv_inx, ring->num_entries,
+				      ring->hdr->head, ring->hdr->tail);
+	trace_rv_event_post(ev->event_type, ev->wc.status, ev->wc.imm_data,
+			    ev->wc.wr_id, ev->wc.conn_handle,
+			    ev->wc.byte_len);
+	spin_lock_irqsave(&ring->lock, flags);
+	next = hdr->tail + 1;
+	if (next == ring->num_entries)
+		next = 0;
+	if (next == hdr->head)  {
+		hdr->overflow_cnt++;
+		rv_err(ring->rv_inx, "event ring full: head %u tail %u\n",
+		       hdr->head, hdr->tail);
+		goto unlock;
+	}
+
+	smp_rmb(); /* ensure we read tail before writing event */
+	hdr->entries[hdr->tail] = *ev;
+	smp_wmb(); /* ensure ev written before advance tail */
+
+	hdr->tail = next;
+	if (ev->wc.status) {
+		ring->stats.cqe_fail[ev->event_type]++;
+	} else {
+		ring->stats.cqe[ev->event_type]++;
+		ring->stats.bytes[ev->event_type] += ev->wc.byte_len;
+	}
+unlock:
+	spin_unlock_irqrestore(&ring->lock, flags);
+}
+
+static void rv_post_user_event_by_index(struct rv_job_dev *jdev, u16 index,
+					struct rv_event *ev)
+{
+	unsigned long flags;
+	struct rv_user *rv;
+
+	spin_lock_irqsave(&jdev->user_array_lock, flags);
+	if (index >= jdev->max_users)
+		goto unlock;
+	rv = jdev->user_array[index];
+	if (rv && rv->cqr)
+		rv_user_ring_post_event(rv->cqr, ev);
+unlock:
+	spin_unlock_irqrestore(&jdev->user_array_lock, flags);
+}
+
+/*
+ * We have a rv_conn reference for the pend_wr
+ * pass all failures to PSM to deal with.  We can't attempt
+ * to retry the write in rv since it might have succeeded on remote
+ * end (eg. ack lost) and remote end may be using buffer for something
+ * else already
+ */
+static void rv_rdma_write_done(struct ib_cq *cq, struct ib_wc *wc)
+{
+	struct rv_pend_write *pend_wr = container_of(wc->wr_cqe,
+						struct rv_pend_write, cqe);
+	struct rv_sconn *sconn = pend_wr->sconn;
+	struct rv_event ev = { 0 };
+
+	atomic_dec(&sconn->stats.outstand_send_write);
+	trace_rv_wc_write_done(pend_wr->wr_id, wc->status, wc->opcode,
+			       wc->byte_len, be32_to_cpu(wc->ex.imm_data));
+	trace_rv_pend_write_done(pend_wr->user_index, pend_wr->sconn, pend_wr,
+				 pend_wr->loc_addr, pend_wr->rkey,
+				 pend_wr->rem_addr, pend_wr->length,
+				 pend_wr->immed, pend_wr->wr_id);
+
+	if (unlikely(wc->status != IB_WC_SUCCESS))
+		rv_report_cqe_error(cq, wc, pend_wr->sconn, "RDMA Write");
+	else if (wc->qp != sconn->qp)
+		rv_report_cqe_error(cq, wc, pend_wr->sconn, "Stale RDMA Write");
+
+	ev.event_type = RV_WC_RDMA_WRITE;
+	ev.wc.status = wc->status;
+	ev.wc.wr_id = pend_wr->wr_id;
+	ev.wc.conn_handle = (u64)pend_wr->sconn->parent;
+	ev.wc.byte_len = pend_wr->length;
+	trace_rv_event_write_done(ev.event_type, ev.wc.status, ev.wc.imm_data,
+				  ev.wc.wr_id, ev.wc.conn_handle,
+				  ev.wc.byte_len);
+
+	rv_mr_cache_put(&pend_wr->umrs->cache, pend_wr->mrc);
+
+	rv_post_user_event_by_index(pend_wr->sconn->parent->jdev,
+				    pend_wr->user_index, &ev);
+
+	if (wc->status)
+		atomic64_inc(&sconn->stats.send_write_cqe_fail);
+	else
+		atomic64_inc(&sconn->stats.send_write_cqe);
+
+	/* our rv_conn ref prevents user_mrs_put from triggering job cleanup */
+	rv_user_mrs_put(pend_wr->umrs);
+
+	/* rv_conn_put can put rv_job_dev and trigger whole job cleanup */
+	rv_conn_put(sconn->parent);
+
+	kfree(pend_wr);
+}
+
+/*
+ * we do not need a queue inside rv of unposted writes.  If this fails
+ * PSM will try to repost later.
+ * We use loc_addr/length/access to lookup MR in cache and then verify RDMA is
+ * consistent with loc_addr and length
+ */
+int doit_post_rdma_write(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_post_write_params pparams;
+	struct rv_conn *conn;
+	struct rv_sconn *sconn;
+	struct rv_mr_cached *mrc;
+	struct rv_pend_write *pend_wr;
+	int ret;
+
+	if (copy_from_user(&pparams.in, (void __user *)arg,
+			   sizeof(pparams.in)))
+		return -EFAULT;
+
+	mutex_lock(&rv->mutex);
+
+	conn = user_conn_find(rv, pparams.in.handle);
+	if (!conn) {
+		rv_err(rv->inx, "post_write: No connection found\n");
+		ret = -EINVAL;
+		goto bail_unlock;
+	}
+	sconn = rv_conn_next_sconn_to_post(conn);
+	if (unlikely(!sconn)) {
+		ret = -ENOMEM;
+		goto bail_unlock;
+	}
+
+	mrc = rv_mr_cache_search_get(&rv->umrs->cache, pparams.in.loc_mr_addr,
+				     pparams.in.loc_mr_length,
+				     pparams.in.loc_mr_access, false);
+	if (!mrc) {
+		rv_err(rv->inx, "post_write: bad loc_mr\n");
+		ret = -EINVAL;
+		goto bail_dec;
+	}
+
+	if (mrc->addr > (u64)pparams.in.loc_addr ||
+	    mrc->addr + mrc->len <
+	    (u64)pparams.in.loc_addr + pparams.in.length) {
+		rv_err(rv->inx, "post_write: addr inconsistent with loc_mr\n");
+		ret = -EINVAL;
+		goto bail_put_mr;
+	}
+	if (!(mrc->access & IBV_ACCESS_KERNEL)) {
+		rv_err(rv->inx, "post_write: loc_mr not a kernel MR\n");
+		ret = -EINVAL;
+		goto bail_put_mr;
+	}
+
+	pend_wr = kzalloc(sizeof(*pend_wr), GFP_KERNEL);
+	if (!pend_wr) {
+		ret = -ENOMEM;
+		goto bail_put_mr;
+	}
+	pend_wr->cqe.done = rv_rdma_write_done;
+	pend_wr->user_index = rv->index;
+
+	rv_user_mrs_get(rv->umrs);
+	pend_wr->umrs = rv->umrs;
+
+	rv_conn_get(sconn->parent);
+	pend_wr->sconn = sconn;
+
+	pend_wr->mrc = mrc;
+	pend_wr->loc_addr = (u64)pparams.in.loc_addr;
+	pend_wr->rem_addr = pparams.in.rem_addr;
+	pend_wr->rkey = pparams.in.rkey;
+	pend_wr->length = pparams.in.length;
+	pend_wr->immed = pparams.in.immed;
+	pend_wr->wr_id = pparams.in.wr_id;
+
+	mutex_lock(&sconn->mutex);
+	if (sconn->state != RV_CONNECTED) {
+		if (sconn->state == RV_ERROR)
+			ret = -EIO;
+		else if (test_bit(RV_SCONN_WAS_CONNECTED, &sconn->flags))
+			ret = -EAGAIN;
+		else
+			ret = -EINVAL;
+		mutex_unlock(&sconn->mutex);
+		goto bail_free_pend;
+	}
+
+	trace_rv_pend_write_post(pend_wr->user_index, pend_wr->sconn, pend_wr,
+				 pend_wr->loc_addr, pend_wr->rkey,
+				 pend_wr->rem_addr, pend_wr->length,
+				 pend_wr->immed, pend_wr->wr_id);
+	ret = rv_drv_post_write_immed(pend_wr);
+	if (ret) {
+		sconn->stats.post_write_fail++;
+	} else {
+		sconn->stats.post_write++;
+		sconn->stats.post_write_bytes += pparams.in.length;
+	}
+
+	pparams.out.sconn_index = sconn->index;
+	pparams.out.conn_count = sconn->stats.conn_recovery + 1;
+
+	mutex_unlock(&sconn->mutex);
+	if (ret) {
+		rv_err(rv->inx, "post_write: failed: %d\n", ret);
+		goto bail_free_pend;
+	}
+
+	if (copy_to_user((void __user *)arg, &pparams.out, sizeof(pparams.out)))
+		ret = -EFAULT;
+
+	mutex_unlock(&rv->mutex);
+
+	return 0;
+
+bail_free_pend:
+	rv_conn_put(pend_wr->sconn->parent);
+	rv_user_mrs_put(pend_wr->umrs);
+	kfree(pend_wr);
+
+bail_put_mr:
+	rv_mr_cache_put(&rv->umrs->cache, mrc);
+bail_dec:
+	atomic_dec(&sconn->stats.outstand_send_write);
+bail_unlock:
+	mutex_unlock(&rv->mutex);
+	return ret;
+}
+
 static int rv_drv_post_recv(struct rv_sconn *sconn)
 {
 	struct ib_recv_wr wr;
@@ -54,6 +344,27 @@ int rv_drv_prepost_recv(struct rv_sconn *sconn)
 	return 0;
 }
 
+static void rv_recv_rdma_write(struct rv_sconn *sconn, struct ib_wc *wc)
+{
+	struct rv_job_dev *jdev = sconn->parent->jdev;
+	u32 index = be32_to_cpu(wc->ex.imm_data) >> (32 - jdev->index_bits);
+	struct rv_event ev = { 0 };
+
+	ev.event_type = RV_WC_RECV_RDMA_WITH_IMM;
+	ev.wc.status = wc->status;
+	ev.wc.resv1 = 0;
+	ev.wc.imm_data = be32_to_cpu(wc->ex.imm_data);
+	ev.wc.wr_id = 0;	/* N/A */
+	ev.wc.conn_handle = (u64)sconn->parent;
+	ev.wc.byte_len = wc->byte_len;
+	ev.wc.resv2 = 0;
+	trace_rv_event_recv_write(ev.event_type, ev.wc.status, ev.wc.imm_data,
+				  ev.wc.wr_id, ev.wc.conn_handle,
+				  ev.wc.byte_len);
+
+	rv_post_user_event_by_index(jdev, index, &ev);
+}
+
 /* drain_lock makes sure no recv WQEs get reposted after a drain WQE */
 void rv_recv_done(struct ib_cq *cq, struct ib_wc *wc)
 {
@@ -93,6 +404,8 @@ void rv_recv_done(struct ib_cq *cq, struct ib_wc *wc)
 
 	if (unlikely(wc->opcode != IB_WC_RECV_RDMA_WITH_IMM))
 		rv_report_cqe_error(cq, wc, sconn, "Recv bad opcode");
+	else
+		rv_recv_rdma_write(sconn, wc);
 repost:
 	spin_lock_irqsave(&sconn->drain_lock, flags);
 	if (likely(!test_bit(RV_SCONN_DRAINING, &sconn->flags)))
diff --git a/drivers/infiniband/ulp/rv/trace_rdma.h b/drivers/infiniband/ulp/rv/trace_rdma.h
index 54ea0cf2599c..e662b2246d8c 100644
--- a/drivers/infiniband/ulp/rv/trace_rdma.h
+++ b/drivers/infiniband/ulp/rv/trace_rdma.h
@@ -11,9 +11,74 @@
 #undef TRACE_SYSTEM
 #define TRACE_SYSTEM rv_rdma
 
+#define RV_PEND_WRITE_PRN "user_inx %d sconn %p pend_wr %p loc_addr 0x%llx" \
+			  " rkey 0x%x rem_addr 0x%llx len 0x%llx immed 0x%x" \
+			  " wr_id 0x%llx"
+
 #define RV_SCONN_RECV_PRN "sconn %p index %u qp 0x%x conn %p flags 0x%x " \
 			  " state %u immed 0x%x"
 
+#define RV_EVENT_PRN "type 0x%x status 0x%x immed 0x%x wr_id 0x%llx " \
+		     "conn_handle 0x%llx len 0x%x"
+
+DECLARE_EVENT_CLASS(/* pend_write */
+	rv_pend_write_template,
+	TP_PROTO(int user_inx, void *sconn, void *pend_wr, u64 loc_addr,
+		 u32 rkey, u64 rem_addr, u64 len, u32 immed, u64 wr_id),
+	TP_ARGS(user_inx, sconn, pend_wr, loc_addr, rkey, rem_addr, len, immed,
+		wr_id),
+	TP_STRUCT__entry(/* entry */
+		__field(int, user_inx)
+		__field(void *, sconn)
+		__field(void *, pend_wr)
+		__field(u64, loc_addr)
+		__field(u32, rkey)
+		__field(u64, rem_addr)
+		__field(u64, len)
+		__field(u32, immed)
+		__field(u64, wr_id)
+	),
+	TP_fast_assign(/* assign */
+		__entry->user_inx = user_inx;
+		__entry->sconn = sconn;
+		__entry->pend_wr = pend_wr;
+		__entry->loc_addr = loc_addr;
+		__entry->rkey = rkey;
+		__entry->rem_addr = rem_addr;
+		__entry->len = len;
+		__entry->immed = immed;
+		__entry->wr_id = wr_id;
+	),
+	TP_printk(/* print */
+		RV_PEND_WRITE_PRN,
+		__entry->user_inx,
+		__entry->sconn,
+		__entry->pend_wr,
+		__entry->loc_addr,
+		__entry->rkey,
+		__entry->rem_addr,
+		__entry->len,
+		__entry->immed,
+		__entry->wr_id
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_pend_write_template, rv_pend_write_post,
+	TP_PROTO(int user_inx, void *sconn, void *pend_wr, u64 loc_addr,
+		 u32 rkey, u64 rem_addr, u64 len, u32 immed, u64 wr_id),
+	TP_ARGS(user_inx, sconn, pend_wr, loc_addr, rkey, rem_addr, len, immed,
+		wr_id)
+);
+
+DEFINE_EVENT(/* event */
+	rv_pend_write_template, rv_pend_write_done,
+	TP_PROTO(int user_inx, void *sconn, void *pend_wr, u64 loc_addr,
+		 u32 rkey, u64 rem_addr, u64 len, u32 immed, u64 wr_id),
+	TP_ARGS(user_inx, sconn, pend_wr, loc_addr, rkey, rem_addr, len, immed,
+		wr_id)
+);
+
 DECLARE_EVENT_CLASS(/* recv */
 	rv_sconn_recv_template,
 	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
@@ -77,6 +142,59 @@ DEFINE_EVENT(/* event */
 	TP_ARGS(ptr, index, qp_num, conn, flags, state, immed)
 );
 
+DECLARE_EVENT_CLASS(/* event */
+	rv_event_template,
+	TP_PROTO(u8 type, u8 status, u32 immed, u64 wr_id, u64 conn_handle,
+		 u32 len),
+	TP_ARGS(type, status, immed, wr_id, conn_handle, len),
+	TP_STRUCT__entry(/* entry */
+		__field(u8, type)
+		__field(u8, status)
+		__field(u32, immed)
+		__field(u64, wr_id)
+		__field(u64, conn_handle)
+		__field(u32, len)
+	),
+	TP_fast_assign(/* assign */
+		__entry->type = type;
+		__entry->status = status;
+		__entry->immed = immed;
+		__entry->wr_id = wr_id;
+		__entry->conn_handle = conn_handle;
+		__entry->len = len;
+	),
+	TP_printk(/* print */
+		RV_EVENT_PRN,
+		__entry->type,
+		__entry->status,
+		__entry->immed,
+		__entry->wr_id,
+		__entry->conn_handle,
+		__entry->len
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_event_template, rv_event_write_done,
+	TP_PROTO(u8 type, u8 status, u32 immed, u64 wr_id, u64 conn_handle,
+		 u32 len),
+	TP_ARGS(type, status, immed, wr_id, conn_handle, len)
+);
+
+DEFINE_EVENT(/* event */
+	rv_event_template, rv_event_post,
+	TP_PROTO(u8 type, u8 status, u32 immed, u64 wr_id, u64 conn_handle,
+		 u32 len),
+	TP_ARGS(type, status, immed, wr_id, conn_handle, len)
+);
+
+DEFINE_EVENT(/* event */
+	rv_event_template, rv_event_recv_write,
+	TP_PROTO(u8 type, u8 status, u32 immed, u64 wr_id, u64 conn_handle,
+		 u32 len),
+	TP_ARGS(type, status, immed, wr_id, conn_handle, len)
+);
+
 DECLARE_EVENT_CLASS(/* wc */
 	rv_wc_template,
 	TP_PROTO(u64 wr_id, u32 status, u32 opcode, u32 byte_len,
@@ -113,6 +231,13 @@ DEFINE_EVENT(/* event */
 	TP_ARGS(wr_id, status, opcode, byte_len, imm_data)
 );
 
+DEFINE_EVENT(/* event */
+	rv_wc_template, rv_wc_write_done,
+	TP_PROTO(u64 wr_id, u32 status, u32 opcode, u32 byte_len,
+		 u32 imm_data),
+	TP_ARGS(wr_id, status, opcode, byte_len, imm_data)
+);
+
 DEFINE_EVENT(/* event */
 	rv_wc_template, rv_wc_hb_done,
 	TP_PROTO(u64 wr_id, u32 status, u32 opcode, u32 byte_len,
diff --git a/drivers/infiniband/ulp/rv/trace_user.h b/drivers/infiniband/ulp/rv/trace_user.h
index ce62c808ca10..90f67537d249 100644
--- a/drivers/infiniband/ulp/rv/trace_user.h
+++ b/drivers/infiniband/ulp/rv/trace_user.h
@@ -112,6 +112,37 @@ DEFINE_EVENT(/* event */
 	TP_ARGS(rv_inx, jdev, total_size, max_size, refcount)
 );
 
+DECLARE_EVENT_CLASS(/* user_ring */
+	rv_user_ring_template,
+	TP_PROTO(int rv_inx, u32 count, u32 hd, u32 tail),
+	TP_ARGS(rv_inx, count, hd, tail),
+	TP_STRUCT__entry(/* entry */
+		__field(int, rv_inx)
+		__field(u32, count)
+		__field(u32, head)
+		__field(u32, tail)
+	),
+	TP_fast_assign(/* assign */
+		__entry->rv_inx = rv_inx;
+		__entry->count = count;
+		__entry->head = hd;
+		__entry->tail = tail;
+	),
+	TP_printk(/* print */
+		"rv_inx %d entries %u head %u tail %u",
+		__entry->rv_inx,
+		__entry->count,
+		__entry->head,
+		__entry->tail
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_ring_template, rv_user_ring_post_event,
+	TP_PROTO(int rv_inx, u32 count, u32 hd, u32 tail),
+	TP_ARGS(rv_inx, count, hd, tail)
+);
+
 #endif /* __RV_TRACE_USER_H */
 
 #undef TRACE_INCLUDE_PATH
-- 
2.18.1


^ permalink raw reply related	[flat|nested] 52+ messages in thread

* [PATCH RFC 8/9] RDMA/rv: Add functions for file operations
  2021-03-19 12:56 [PATCH RFC 0/9] A rendezvous module kaike.wan
                   ` (6 preceding siblings ...)
  2021-03-19 12:56 ` [PATCH RFC 7/9] RDMA/rv: Add functions for RDMA transactions kaike.wan
@ 2021-03-19 12:56 ` kaike.wan
  2021-03-19 12:56 ` [PATCH RFC 9/9] RDMA/rv: Integrate the file operations into the rv module kaike.wan
  2021-03-19 13:53 ` [PATCH RFC 0/9] A rendezvous module Jason Gunthorpe
  9 siblings, 0 replies; 52+ messages in thread
From: kaike.wan @ 2021-03-19 12:56 UTC (permalink / raw)
  To: dledford, jgg; +Cc: linux-rdma, todd.rimmer, Kaike Wan

From: Kaike Wan <kaike.wan@intel.com>

A process communicates with the rv module through the file interface:
- The process opens the /dev/rv device file.
- The process sends an Attach request to bind to an RDMA device.
- The process registers a number of user/kernel memory regions (MR).
- The process sends Create Conn request to create connection between
  two nodes. If the local node is the server, it will start to listen
  to IB CM for any incoming connection request.
- The process sends Connect request to start connection. For a server
  node, it does nothing. However, for a client node, it will send the
  IB CM connection request.
- The process will wait for all connections to be established by
  polling the rv module. Receiving buffers will be posted.
- The process mmaps the event ring into user space.
- The process starts RDMA transaction by sending RDMA write with
  immediate requests to rv module. Send completion events will be
  posted to the event ring.
- On the responder side, RDMA write with immediate requests will be
  received and receive completion events will be posted to the event
  ring buffer.
- The process will poll the event ring for completion events.
- When RDMA transactions are done, the process deregisters the memory
  regions.
- The process closes the file. In this process, any connection will
  be torn down, and the RDMA device will be detached if there is no
  more user. Explicit detach and disconnection are not required.

Technically, the MR registration, RDMA transactions, and MR
deregistration occurs for every application IO.

This patch adds the functions for the file operations and integrates
the functions with memory region registration, connection management,
and RDMA transactions.

Signed-off-by: Todd Rimmer <todd.rimmer@intel.com>
Signed-off-by: Kaike Wan <kaike.wan@intel.com>
---
 drivers/infiniband/ulp/rv/rv_file.c    | 1040 ++++++++++++++++++++++++
 drivers/infiniband/ulp/rv/rv_main.c    |    2 +-
 drivers/infiniband/ulp/rv/trace_conn.h |   18 +
 drivers/infiniband/ulp/rv/trace_user.h |  169 ++++
 4 files changed, 1228 insertions(+), 1 deletion(-)

diff --git a/drivers/infiniband/ulp/rv/rv_file.c b/drivers/infiniband/ulp/rv/rv_file.c
index 9d23503a30d9..e10bcdee11f5 100644
--- a/drivers/infiniband/ulp/rv/rv_file.c
+++ b/drivers/infiniband/ulp/rv/rv_file.c
@@ -9,11 +9,58 @@
 #include "rv.h"
 #include "trace.h"
 
+static unsigned long service_id = RV_DFLT_SERVICE_ID;
+
+module_param(service_id, ulong, 0444);
+MODULE_PARM_DESC(service_id, "Default service_id for IB CM QP connections");
+
+static unsigned int num_conn = 4;
+
+module_param(num_conn, uint, 0444);
+MODULE_PARM_DESC(num_conn, "Default # QPs between each pair of nodes");
+
+/*
+ * these are per node to node connection.
+ *
+ * conservative for now, we should service CQEs fast enough that smaller numbers
+ * would work, however at 100 remote nodes (hence 100 connections), 400,000
+ * CQEs at say 64B each, is only 50MB including send and recv
+ *
+ * A given PSM receiver process
+ * will not allow any more than HFI_TF_NFLOWS (32) inflight RDMA's coming
+ * toward it. So if we have ~100 processes per node, that limits inflight to
+ * 3200 coming toward us.
+ * while a given sender can have many in flight, the total a given destination
+ * process will allow is HFI_TF_NFLOWS (32), so size this same as recv CQ
+ * same reasoning applies to QP sizes
+ */
+#define RV_Q_DEPTH 4000 /* some headroom */
+
+static unsigned int q_depth = RV_Q_DEPTH;
+
+module_param(q_depth, uint, 0444);
+MODULE_PARM_DESC(q_depth, "Default size of queues per remote node");
+
+static int rv_file_mmap(struct file *fp, struct vm_area_struct *vma);
+static void rv_user_ring_free(struct rv_user_ring *ring);
+static void rv_user_detach_all(struct rv_user *rv);
+
+static atomic_t seq;
 /* A workqueue for all */
 static struct workqueue_struct *rv_wq;
 static struct workqueue_struct *rv_wq2;
 static struct workqueue_struct *rv_wq3;
 
+/* Device file access  */
+struct rv_devdata {
+	struct class *class;
+	dev_t dev;
+	struct cdev user_cdev;
+	struct device user_device;
+};
+
+static struct rv_devdata *rv_dd;
+
 /*
  * We expect relatively few jobs per node (typically 1)
  * and relatively few devices per node (typically 1 to 8)
@@ -24,7 +71,9 @@ static struct workqueue_struct *rv_wq3;
  * mutex avoids duplicate get_alloc adds, RCU protects list access.
  * See rv.h comments about "get_alloc" for more information.
  */
+static struct mutex rv_job_dev_list_mutex;
 static struct list_head rv_job_dev_list;
+static atomic_t rv_job_dev_cnt;
 
 void rv_queue_work(struct work_struct *work)
 {
@@ -46,16 +95,268 @@ void rv_flush_work2(void)
 	flush_workqueue(rv_wq2);
 }
 
+static int doit_query(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_query_params_out params = { 0 };
+	int ret = 0;
+
+	params.major_rev = RV_ABI_VER_MAJOR;
+	params.minor_rev = RV_ABI_VER_MINOR;
+	params.capability = 0;
+
+	if (copy_to_user((void __user *)arg, &params, sizeof(params)))
+		ret = -EFAULT;
+
+	return ret;
+}
+
+/*
+ * only for use by rv_job_dev_get_alloc,
+ * all other callers must use rv_job_dev_get_alloc or rv_job_dev_get
+ * user_array is sized at end of rv_job_dev
+ * When num_conn>1 we will stripe across all sconn, so
+ * any given sconn's qp_depth can be smaller. However, this striping can be
+ * imperfect since not every IO is the same size. We allocate an
+ * extra 5% to minimize skipping sconns on send scheduling
+ */
+static struct rv_job_dev *
+rv_job_dev_alloc(int rv_inx, struct rv_device *dev,
+		 const struct rv_attach_params *params)
+{
+	int ret;
+	struct rv_job_dev *jdev;
+	int max_users = (1 << params->in.index_bits);
+
+	jdev = kzalloc(sizeof(*jdev) + sizeof(jdev->user_array[0]) * max_users,
+		       GFP_KERNEL);
+	if (!jdev) {
+		ret = -ENOMEM;
+		goto bail;
+	}
+
+	jdev->kuid = current_uid();
+	jdev->uid = from_kuid(current_user_ns(), jdev->kuid);
+	jdev->max_users = max_users;
+	spin_lock_init(&jdev->user_array_lock);
+	kref_init(&jdev->kref);
+	kref_init(&jdev->user_kref);
+	mutex_init(&jdev->conn_list_mutex);
+	INIT_LIST_HEAD(&jdev->conn_list);
+	jdev->dev = dev;
+	jdev->pd = ib_alloc_pd(jdev->dev->ib_dev, 0);
+	if (IS_ERR(jdev->pd)) {
+		rv_err(rv_inx, "Could not allocate PD\n");
+		ret = PTR_ERR(jdev->pd);
+		goto bail_free;
+	}
+
+	memcpy(jdev->dev_name, params->in.dev_name, sizeof(jdev->dev_name));
+	jdev->port_num = params->in.port_num;
+	jdev->num_conn = params->in.num_conn;
+	jdev->index_bits = params->in.index_bits;
+	jdev->loc_gid_index = params->in.loc_gid_index;
+	jdev->loc_addr = params->in.loc_addr;
+	memcpy(jdev->loc_gid, params->in.loc_gid, sizeof(jdev->loc_gid));
+	memcpy(jdev->job_key, params->in.job_key, sizeof(jdev->job_key));
+	jdev->job_key_len = params->in.job_key_len;
+	jdev->service_id =  params->in.service_id;
+	jdev->q_depth =  params->in.q_depth;
+	jdev->qp_depth = (jdev->q_depth + jdev->num_conn - 1) / jdev->num_conn;
+	if (jdev->num_conn > 1)
+		jdev->qp_depth += jdev->qp_depth / 20;
+	jdev->reconnect_timeout =  params->in.reconnect_timeout;
+	jdev->hb_interval =  params->in.hb_interval;
+	jdev->sgid_attr = rdma_get_gid_attr(jdev->dev->ib_dev, jdev->port_num,
+					    jdev->loc_gid_index);
+	if (!jdev->sgid_attr) {
+		rv_err(rv_inx, "can't resolve sgid_attr\n");
+		ret = -ENOENT;
+		goto bail_mr;
+	}
+	if (memcmp(&jdev->loc_gid, &jdev->sgid_attr->gid,
+		   sizeof(jdev->loc_gid))) {
+		rv_err(rv_inx, "sgid_attr gid and loc_gid mismatch\n");
+		ret = -ENOENT;
+		goto bail_put;
+	}
+	trace_rv_jdev_alloc(jdev, jdev->dev_name, jdev->num_conn,
+			    jdev->index_bits, jdev->loc_gid_index,
+			    jdev->loc_addr, jdev->job_key_len,
+			    jdev->job_key, jdev->service_id,
+			    jdev->q_depth, jdev->user_array_next,
+			    kref_read(&jdev->kref));
+	return jdev;
+
+bail_put:
+	rdma_put_gid_attr(jdev->sgid_attr);
+bail_mr:
+	ib_dealloc_pd(jdev->pd);
+bail_free:
+	kfree(jdev);
+bail:
+	return ERR_PTR(ret);
+}
+
+static int rv_job_dev_consistent(struct rv_job_dev *jdev,
+				 const struct rv_attach_params *params)
+{
+	return (params->in.num_conn == jdev->num_conn &&
+		params->in.index_bits == jdev->index_bits &&
+		params->in.loc_gid_index == jdev->loc_gid_index &&
+		params->in.loc_addr == jdev->loc_addr &&
+		!memcmp(jdev->loc_gid, params->in.loc_gid,
+			   sizeof(jdev->loc_gid)) &&
+		params->in.service_id == jdev->service_id &&
+		params->in.q_depth == jdev->q_depth &&
+		params->in.reconnect_timeout == jdev->reconnect_timeout &&
+		params->in.hb_interval == jdev->hb_interval);
+}
+
+static void rv_job_dev_list_init(void)
+{
+	mutex_init(&rv_job_dev_list_mutex);
+	INIT_LIST_HEAD(&rv_job_dev_list);
+	atomic_set(&rv_job_dev_cnt, 0);
+}
+
+static void rv_job_dev_user_release(struct kref *kref)
+{
+	struct rv_job_dev *jdev = container_of(kref, struct rv_job_dev,
+					       user_kref);
+
+	mutex_lock(&rv_job_dev_list_mutex);
+	list_del_rcu(&jdev->job_dev_entry);
+	mutex_unlock(&rv_job_dev_list_mutex);
+}
+
+static void rv_job_dev_put_user(struct rv_job_dev *jdev)
+{
+	kref_put(&jdev->user_kref, rv_job_dev_user_release);
+}
+
+/*
+ * get a job_dev matching the given ATTACH.  If none is found, create one
+ * The job_dev returned must be released with rv_job_dev_put when done using.
+ * Get device 1st to reduce lock nesting.  Device search should be quick.
+ * While searching for jdev, likely to have more devs than jobs, so filter on
+ * dev 1st.  job_key_len can be 0, which matches only jobs with job_key_len==0
+ * Ideally each job should have a unique job_key (really just a job identifer),
+ * but all jobs or processes with the same job_key must have the same params.
+ */
+static struct rv_job_dev *rv_job_dev_get_alloc(struct rv_user *rv,
+					       struct rv_attach_params *params)
+{
+	struct rv_job_dev *jdev;
+	struct rv_device *dev;
+
+	dev = rv_device_get_add_user(params->in.dev_name, rv);
+	if (!dev) {
+		rv_err(rv->inx, "attach: KERNEL ib_dev %s not found\n",
+		       params->in.dev_name);
+		return ERR_PTR(-ENODEV);
+	}
+
+	mutex_lock(&rv_job_dev_list_mutex);
+	rcu_read_lock();
+	list_for_each_entry_rcu(jdev, &rv_job_dev_list, job_dev_entry) {
+		if (!uid_eq(jdev->kuid, current_uid()) ||
+		    dev != jdev->dev ||
+		    params->in.port_num != jdev->port_num ||
+		    params->in.job_key_len != jdev->job_key_len ||
+		    (params->in.job_key_len &&
+		     memcmp(params->in.job_key, jdev->job_key,
+			    jdev->job_key_len)))
+			continue;
+		if (!kref_get_unless_zero(&jdev->kref))
+			continue;
+		if (!kref_get_unless_zero(&jdev->user_kref)) {
+			rv_job_dev_put(jdev);
+			continue;
+		}
+		rcu_read_unlock();
+		if (!rv_job_dev_consistent(jdev, params)) {
+			mutex_unlock(&rv_job_dev_list_mutex);
+			rv_job_dev_put_user(jdev);
+			rv_job_dev_put(jdev);
+			jdev = ERR_PTR(-EBUSY);
+			goto bail_put;
+		}
+		mutex_unlock(&rv_job_dev_list_mutex);
+		rv_device_put(dev);
+		return jdev;
+	}
+	rcu_read_unlock();
+	jdev = rv_job_dev_alloc(rv->inx, dev, params);
+	if (IS_ERR(jdev))
+		goto bail_unlock;
+
+	list_add_tail_rcu(&jdev->job_dev_entry, &rv_job_dev_list);
+	atomic_inc(&rv_job_dev_cnt);
+
+	mutex_unlock(&rv_job_dev_list_mutex);
+	return jdev;
+
+bail_unlock:
+	mutex_unlock(&rv_job_dev_list_mutex);
+bail_put:
+	rv_device_del_user(rv);
+	rv_device_put(dev);
+	return jdev;
+}
+
 void rv_job_dev_get(struct rv_job_dev *jdev)
 {
 	kref_get(&jdev->kref);
 }
 
+struct rv_dest_pd_work_item {
+	struct work_struct destroy_work;
+	struct ib_pd *pd;
+	struct rv_device *dev;
+};
+
+static void rv_handle_destroy_pd(struct work_struct *work)
+{
+	struct rv_dest_pd_work_item *item = container_of(work,
+				struct rv_dest_pd_work_item, destroy_work);
+
+	flush_workqueue(rv_wq2);
+	ib_dealloc_pd(item->pd);
+	rv_device_put(item->dev);
+	kfree(item);
+}
+
 static void rv_job_dev_release(struct kref *kref)
 {
 	struct rv_job_dev *jdev = container_of(kref, struct rv_job_dev, kref);
+	struct rv_dest_pd_work_item *item;
+
+	trace_rv_jdev_release(jdev, jdev->dev_name, jdev->num_conn,
+			      jdev->index_bits, jdev->loc_gid_index,
+			      jdev->loc_addr, jdev->job_key_len,
+			      jdev->job_key, jdev->service_id,
+			      jdev->q_depth, jdev->user_array_next,
+			      kref_read(&jdev->kref));
 
+	WARN_ON(!list_empty(&jdev->conn_list)); /* RCU safe */
+
+	if (jdev->listener)
+		rv_listener_put(jdev->listener);
+	rdma_put_gid_attr(jdev->sgid_attr);
+	item = kzalloc(sizeof(*item), GFP_KERNEL);
+	if (item) {
+		INIT_WORK(&item->destroy_work, rv_handle_destroy_pd);
+		item->pd = jdev->pd;
+		item->dev = jdev->dev;
+		jdev->pd = NULL;
+		jdev->dev = NULL;
+		rv_queue_work3(&item->destroy_work);
+	} else {
+		ib_dealloc_pd(jdev->pd);
+		rv_device_put(jdev->dev);
+	}
 	kfree_rcu(jdev, rcu);
+	atomic_dec(&rv_job_dev_cnt);
 }
 
 void rv_job_dev_put(struct rv_job_dev *jdev)
@@ -63,6 +364,316 @@ void rv_job_dev_put(struct rv_job_dev *jdev)
 	kref_put(&jdev->kref, rv_job_dev_release);
 }
 
+/*
+ * make a bi-directional linkage between rv_user and rv_job_dev
+ * Each rv_user is assigned a unique index within it's job_dev.
+ * This will be placed in RDMA immediate data on
+ * remote node so in recv CQE we can figure out which rv to deliver
+ * the RDMA w/immediate recv CQE event to
+ */
+static int rv_job_dev_add_user(struct rv_job_dev *jdev, struct rv_user *rv)
+{
+	unsigned long flags;
+	int i;
+	struct rv_user **jentry;
+	u32 next;
+
+	spin_lock_irqsave(&jdev->user_array_lock, flags);
+	next = jdev->user_array_next;
+	jentry = &jdev->user_array[next];
+	for (i = 0; i < jdev->max_users && *jentry; i++) {
+		if (++next >= jdev->max_users) {
+			next = 0;
+			jentry = &jdev->user_array[0];
+		} else {
+			jentry++;
+		}
+	}
+	if (i >= jdev->max_users) {
+		i = -ENOMEM;
+		goto unlock;
+	}
+	i = next;
+	if (++next >= jdev->max_users)
+		next = 0;
+	jdev->user_array_next = next;
+	*jentry = rv;
+	rv->index = i;
+
+unlock:
+	spin_unlock_irqrestore(&jdev->user_array_lock, flags);
+	return i;
+}
+
+/* break the bi-directional linkage between rv_user and rv_job_dev */
+static void rv_job_dev_del_user(struct rv_job_dev *jdev, struct rv_user *rv)
+{
+	unsigned long flags;
+
+	spin_lock_irqsave(&jdev->user_array_lock, flags);
+	WARN_ON(rv->index >= jdev->max_users);
+	WARN_ON(jdev->user_array[rv->index] != rv);
+	jdev->user_array[rv->index] = NULL;
+	rv->index = RV_INVALID;
+	spin_unlock_irqrestore(&jdev->user_array_lock, flags);
+}
+
+/* attach for rdma_mode KERNEL */
+static int rv_user_attach_kernel(struct rv_user *rv,
+				 struct rv_attach_params *params)
+{
+	struct rv_job_dev *jdev;
+	int ret;
+
+	jdev = rv_job_dev_get_alloc(rv, params);
+	if (IS_ERR(jdev))
+		return PTR_ERR(jdev);
+
+	rv->jdev = jdev;
+	rv->context = params->in.context;
+	rv->cq_entries = params->in.cq_entries;
+
+	ret = rv_job_dev_add_user(jdev, rv);
+	if (ret < 0) {
+		rv->jdev = NULL;
+		rv->context = 0;
+		rv->cq_entries = 0;
+		rv_job_dev_put_user(jdev);
+		rv_job_dev_put(jdev);
+		return ret;
+	}
+	return 0;
+}
+
+static void rv_user_detach_kernel(struct rv_user *rv)
+{
+	if (!rv->jdev)
+		return;
+	rv_job_dev_del_user(rv->jdev, rv);
+	rv_job_dev_put(rv->jdev);
+}
+
+/*
+ * Attach an rv_user to a jdev
+ * Note hb_interval must be less than reconnect_timeout otherwise listener
+ * could timeout before client side discovers it must reconnect
+ * To avoid deadlock rv_user_mrs_alloc must be called without rv->mutex
+ * because it will acquire mm->mmap_lock.
+ */
+static int doit_attach(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_attach_params params;
+	int ret;
+	struct rv_user_mrs *umrs;
+	u32 reconnect_timeout = 0;
+	u32 depth_out = 0;
+
+	if (copy_from_user(&params.in, (void __user *)arg, sizeof(params.in)))
+		return -EFAULT;
+
+	trace_rv_attach_req(rv->inx, params.in.dev_name, params.in.rdma_mode,
+			    params.in.port_num, params.in.loc_addr,
+			    params.in.job_key_len, params.in.job_key,
+			    params.in.service_id,
+			    params.in.cq_entries, params.in.q_depth,
+			    params.in.reconnect_timeout,
+			    params.in.hb_interval);
+	if (!params.in.dev_name[0] ||
+	    strnlen(params.in.dev_name, RV_MAX_DEV_NAME_LEN) >=
+		    RV_MAX_DEV_NAME_LEN) {
+		rv_err(rv->inx,
+		       "attach: dev_name empty or not nul terminated\n");
+		return -EINVAL;
+	}
+	if (params.in.rdma_mode > RV_RDMA_MODE_MAX) {
+		rv_err(rv->inx, "attach: rdma_mode invalid\n");
+		return -EINVAL;
+	}
+	if (params.in.rdma_mode == RV_RDMA_MODE_KERNEL) {
+		if (!params.in.port_num) {
+			rv_err(rv->inx, "attach: port_num invalid\n");
+			return -EINVAL;
+		}
+		if (params.in.num_conn > RV_MAX_NUM_CONN) {
+			rv_err(rv->inx,
+			       "attach: num_conn too large %d max %d\n",
+			       params.in.num_conn, RV_MAX_NUM_CONN);
+			return -EINVAL;
+		}
+		if (params.in.index_bits > RV_MAX_INDEX_BITS) {
+			rv_err(rv->inx,
+			       "attach: index_bits too large %d max %d\n",
+			       params.in.index_bits, RV_MAX_INDEX_BITS);
+			return -EINVAL;
+		}
+		if (params.in.job_key_len > RV_MAX_JOB_KEY_LEN) {
+			rv_err(rv->inx,
+			       "attach: job_key too large %u max %u\n",
+			       params.in.job_key_len, RV_MAX_JOB_KEY_LEN);
+			return -EINVAL;
+		}
+		if (params.in.cq_entries > RV_MAX_CQ_ENTRIES) {
+			rv_err(rv->inx,
+			       "attach: cq_entries too large %d max %d\n",
+			       params.in.cq_entries, RV_MAX_CQ_ENTRIES);
+			return -EINVAL;
+		}
+	} else if (!enable_user_mr) {
+		rv_err(rv->inx, "attach: rdma_mode user disabled\n");
+		return -EINVAL;
+	}
+
+	if (!params.in.num_conn)
+		params.in.num_conn = num_conn;
+	if (!params.in.num_conn)
+		params.in.num_conn = 1;
+	if (!params.in.service_id)
+		params.in.service_id = service_id;
+	if (!params.in.q_depth)
+		params.in.q_depth = q_depth;
+	if (params.in.reconnect_timeout &&
+	    params.in.reconnect_timeout * 1000 <= params.in.hb_interval) {
+		rv_err(rv->inx,
+		       "reconnect_timeout (%u secs) < hb_interval (%u msec)\n",
+		       params.in.reconnect_timeout, params.in.hb_interval);
+		return -EINVAL;
+	}
+
+	umrs = rv_user_mrs_alloc(rv, params.in.mr_cache_size);
+	if (IS_ERR(umrs))
+		return PTR_ERR(umrs);
+
+	mutex_lock(&rv->mutex);
+	if (rv->attached || rv->was_attached) {
+		rv_err(rv->inx, "attach: already attached to device\n");
+		ret = -EBUSY;
+		goto unlock;
+	}
+
+	rv->rdma_mode = params.in.rdma_mode;
+
+	switch (rv->rdma_mode) {
+	case RV_RDMA_MODE_USER:
+		rv->dev = rv_device_get_add_user(params.in.dev_name, rv);
+		if (!rv->dev) {
+			rv_err(rv->inx, "attach: USER ib_dev %s not found\n",
+			       params.in.dev_name);
+			ret = -ENODEV;
+			goto unlock;
+		}
+		rv->index = RV_INVALID; /* N/A */
+		depth_out = 0; /* N/A */
+		reconnect_timeout = 0; /* N/A */
+		break;
+	case RV_RDMA_MODE_KERNEL:
+		ret = rv_user_attach_kernel(rv, &params);
+		if (ret) {
+			rv_err(rv->inx, "attach: for kernel mode\n");
+			goto unlock;
+		}
+		depth_out = rv->jdev->q_depth;
+		reconnect_timeout = rv->jdev->reconnect_timeout;
+		break;
+	}
+
+	rv->umrs = umrs;
+	rv_user_mrs_attach(rv);
+	params.out.rv_index = rv->index;
+	params.out.mr_cache_size = umrs->cache.max_size / (1024 * 1024);
+	params.out.q_depth = depth_out;
+	params.out.reconnect_timeout = reconnect_timeout;
+	if (copy_to_user((void __user *)arg, &params.out, sizeof(params.out))) {
+		ret = -EFAULT;
+		goto bail_detach;
+	}
+
+	/* set attached last */
+	rv->attached = 1;
+	rv->was_attached = 1;
+	trace_rv_user_attach(rv->inx, rv->rdma_mode, rv->attached,
+			     params.in.dev_name, rv->cq_entries, rv->index);
+	mutex_unlock(&rv->mutex);
+
+	return 0;
+
+bail_detach:
+	rv_device_del_user(rv);
+	rv_user_detach_all(rv);
+	mutex_unlock(&rv->mutex);
+	return ret;
+unlock:
+	mutex_unlock(&rv->mutex);
+	rv_user_mrs_put(umrs);
+	return ret;
+}
+
+/*
+ * detach everything we find, for USER or KERNEL rdma_mode.
+ * must hold rv->mutex before calling.
+ * Once rv_user.attached is 0, the rv_user.dev/jdev is not valid
+ * we are paranoid and detach everything we find even for USER mode
+ * We wait for umrs->kref to be 1, to ensure all pending writes have put
+ * their MRs.  This way our final umrs_put will free the mr_cache.
+ * Note: other callers of rv_user_mrs_put don't get rv->mutex
+ */
+static void rv_user_detach_all(struct rv_user *rv)
+{
+	struct rv_conn *conn;
+	unsigned long sleep_time = msecs_to_jiffies(100);
+
+	XA_STATE(xas, &rv->conn_xa, 0);
+
+	trace_rv_msg_detach_all(rv->inx, "rv_user_detach_all", 0, 0);
+	rv->attached = 0;
+	if (rv->rdma_mode == RV_RDMA_MODE_KERNEL && rv->jdev)
+		rv_job_dev_put_user(rv->jdev);
+	xas_for_each(&xas, conn, UINT_MAX) {
+		trace_rv_msg_uconn_remove(rv->inx, "rv_user remove uconn",
+					  (u64)conn, 0);
+		xas_store(&xas, NULL);
+		rv_conn_put(conn);
+	}
+
+	if (rv->umrs) {
+		while (kref_read(&rv->umrs->kref) > 1)
+			schedule_timeout_interruptible(sleep_time);
+
+		rv_user_mrs_put(rv->umrs);
+		rv->umrs = NULL;
+		flush_workqueue(rv_wq);
+	}
+
+	if (rv->rdma_mode == RV_RDMA_MODE_USER && rv->dev) {
+		rv_device_put(rv->dev);
+		rv->dev = NULL;
+	} else if (rv->rdma_mode == RV_RDMA_MODE_KERNEL && rv->jdev) {
+		rv_user_detach_kernel(rv);
+		rv->jdev = NULL;
+	}
+
+	rv->rdma_mode = RV_RDMA_MODE_USER;
+
+	complete(&rv->compl);
+}
+
+/* Other cleanup at file close time. Must hold rv->mutex. */
+static void rv_user_cleanup(struct rv_user *rv)
+{
+	if (rv->cqr) {
+		trace_rv_msg_cleanup(rv->inx, "freeing event ring",
+				     (u64)rv->cqr, 0);
+		rv_user_ring_free(rv->cqr);
+		rv->cqr = NULL;
+	}
+}
+
+void rv_detach_user(struct rv_user *rv)
+{
+	mutex_lock(&rv->mutex);
+	rv_user_detach_all(rv);
+	mutex_unlock(&rv->mutex);
+}
+
 /*
  * confirm that we expected a REQ from this remote node on this port.
  * Note CM swaps src vs dest so dest is remote node here
@@ -154,3 +765,432 @@ rv_find_sconn_from_req(struct ib_cm_id *id,
 
 	return sconn;
 }
+
+/* Technically don't need rv_user.mutex, but play it safe */
+static int doit_get_cache_stats(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_cache_stats_params_out params;
+	int ret = 0;
+	unsigned long flags;
+
+	mutex_lock(&rv->mutex);
+
+	memset(&params, 0, sizeof(params));
+	if (rv->attached && rv->umrs) {
+		struct rv_mr_cache *cache = &rv->umrs->cache;
+
+		spin_lock_irqsave(&cache->lock, flags);
+
+		params.cache_size = cache->total_size;
+		params.max_cache_size = cache->stats.max_cache_size;
+		params.limit_cache_size = cache->max_size / (1024 * 1024);
+		params.count = cache->stats.count;
+		params.max_count = cache->stats.max_count;
+		params.inuse = cache->stats.inuse;
+		params.max_inuse = cache->stats.max_inuse;
+		params.inuse_bytes = cache->stats.inuse_bytes;
+		params.max_inuse_bytes = cache->stats.max_inuse_bytes;
+		params.max_refcount = cache->stats.max_refcount;
+		params.hit = cache->stats.hit;
+		params.miss = cache->stats.miss;
+		params.full = cache->stats.full;
+		params.failed = rv->umrs->stats.failed;
+		params.remove = cache->stats.remove;
+		params.evict = cache->stats.evict;
+
+		spin_unlock_irqrestore(&cache->lock, flags);
+	}
+
+	if (copy_to_user((void __user *)arg, &params, sizeof(params)))
+		ret = -EFAULT;
+
+	mutex_unlock(&rv->mutex);
+
+	return ret;
+}
+
+/* Technically don't need rv_user.mutex, but play it safe */
+static int doit_get_event_stats(struct rv_user *rv, unsigned long arg)
+{
+	struct rv_event_stats_params_out params;
+	int ret = 0;
+	unsigned long flags;
+
+	mutex_lock(&rv->mutex);
+
+	memset(&params, 0, sizeof(params));
+	if (rv->attached && rv->cqr) {
+		struct rv_user_ring *cqr = rv->cqr;
+
+		spin_lock_irqsave(&cqr->lock, flags);
+
+		params.send_write_cqe = cqr->stats.cqe[RV_WC_RDMA_WRITE];
+		params.send_write_cqe_fail =
+			cqr->stats.cqe_fail[RV_WC_RDMA_WRITE];
+		params.send_write_bytes = cqr->stats.bytes[RV_WC_RDMA_WRITE];
+
+		params.recv_write_cqe =
+			cqr->stats.cqe[RV_WC_RECV_RDMA_WITH_IMM];
+		params.recv_write_cqe_fail =
+			cqr->stats.cqe_fail[RV_WC_RECV_RDMA_WITH_IMM];
+		params.recv_write_bytes =
+			cqr->stats.bytes[RV_WC_RECV_RDMA_WITH_IMM];
+
+		spin_unlock_irqrestore(&cqr->lock, flags);
+	}
+
+	if (copy_to_user((void __user *)arg, &params, sizeof(params)))
+		ret = -EFAULT;
+
+	mutex_unlock(&rv->mutex);
+
+	return ret;
+}
+
+static int rv_file_open(struct inode *inode, struct file *fp)
+{
+	struct rv_user *rv;
+
+	rv = kzalloc(sizeof(*rv), GFP_KERNEL);
+	if (!rv)
+		return -ENOMEM;
+	mutex_init(&rv->mutex);
+	xa_init_flags(&rv->conn_xa, XA_FLAGS_ALLOC);
+	INIT_LIST_HEAD(&rv->user_entry);
+	init_completion(&rv->compl);
+	rv->inx = atomic_inc_return(&seq);
+	rv->index = RV_INVALID;
+	fp->private_data = rv;
+	trace_rv_user_open(rv->inx, rv->rdma_mode, rv->attached, "n/a",
+			   rv->cq_entries, rv->index);
+
+	return 0;
+}
+
+/*
+ * Remove rv_user from dev->user_list first. If it is already
+ * removed from dev->user_list by remove_one(), wait for the
+ * detach to finish.  Otherwise detach_all here will remove it.
+ */
+static int rv_file_close(struct inode *inode, struct file *fp)
+{
+	struct rv_user *rv = fp->private_data;
+
+	trace_rv_user_close(rv->inx, rv->rdma_mode, rv->attached, "n/a",
+			    rv->cq_entries, rv->index);
+
+	fp->private_data = NULL;
+
+	mutex_lock(&rv->mutex);
+	if (rv->attached && rv_device_del_user(rv))
+		goto unlock;
+
+	rv_user_detach_all(rv);
+unlock:
+	mutex_unlock(&rv->mutex);
+	wait_for_completion(&rv->compl);
+
+	mutex_lock(&rv->mutex);
+	rv_user_cleanup(rv);
+	mutex_unlock(&rv->mutex);
+	xa_destroy(&rv->conn_xa);
+	kfree(rv);
+
+	return 0;
+}
+
+static long rv_file_ioctl(struct file *fp, unsigned int cmd, unsigned long arg)
+{
+	struct rv_user *rv = fp->private_data;
+
+	trace_rv_ioctl(rv->inx, cmd);
+	switch (cmd) {
+	case RV_IOCTL_QUERY:
+		return doit_query(rv, arg);
+
+	case RV_IOCTL_ATTACH:
+		return doit_attach(rv, arg);
+
+	case RV_IOCTL_REG_MEM:
+		return doit_reg_mem(rv, arg);
+
+	case RV_IOCTL_DEREG_MEM:
+		return doit_dereg_mem(rv, arg);
+
+	case RV_IOCTL_GET_CACHE_STATS:
+		return doit_get_cache_stats(rv, arg);
+
+	case RV_IOCTL_CONN_CREATE:
+		return doit_conn_create(rv, arg);
+
+	case RV_IOCTL_CONN_CONNECT:
+		return doit_conn_connect(rv, arg);
+
+	case RV_IOCTL_CONN_CONNECTED:
+		return doit_conn_connected(rv, arg);
+
+	case RV_IOCTL_CONN_GET_CONN_COUNT:
+		return doit_conn_get_conn_count(rv, arg);
+
+	case RV_IOCTL_CONN_GET_STATS:
+		return doit_conn_get_stats(rv, arg);
+
+	case RV_IOCTL_GET_EVENT_STATS:
+		return doit_get_event_stats(rv, arg);
+
+	case RV_IOCTL_POST_RDMA_WR_IMMED:
+		return doit_post_rdma_write(rv, arg);
+
+	default:
+		return -EINVAL;
+	}
+}
+
+static const struct file_operations rv_file_ops = {
+	.owner = THIS_MODULE,
+	.open = rv_file_open,
+	.release = rv_file_close,
+	.unlocked_ioctl = rv_file_ioctl,
+	.llseek = noop_llseek,
+	.mmap = rv_file_mmap,
+};
+
+static void rv_dev_release(struct device *dev)
+{
+	kfree(rv_dd);
+	rv_dd = NULL;
+}
+
+static char *rv_devnode(struct device *dev, umode_t *mode)
+{
+	if (mode)
+		*mode = 0666;
+	return kasprintf(GFP_KERNEL, "%s", dev_name(dev));
+}
+
+int rv_file_init(void)
+{
+	int ret;
+	struct device *device;
+	struct cdev *cdev;
+
+	atomic_set(&seq, 0);
+	rv_job_dev_list_init();
+	rv_wq = alloc_workqueue("rv_wq",
+				WQ_SYSFS | WQ_HIGHPRI | WQ_CPU_INTENSIVE,
+				RV_CONN_MAX_ACTIVE_WQ_ENTRIES);
+	if (!rv_wq)
+		return -ENOMEM;
+
+	rv_wq2 = alloc_workqueue("rv_wq2",
+				 WQ_SYSFS | WQ_HIGHPRI | WQ_CPU_INTENSIVE,
+				 RV_CONN_MAX_ACTIVE_WQ_ENTRIES);
+	if (!rv_wq2) {
+		ret = -ENOMEM;
+		goto fail_wq;
+	}
+	rv_wq3 = alloc_workqueue("rv_wq3",
+				 WQ_SYSFS | WQ_HIGHPRI | WQ_CPU_INTENSIVE,
+				 10);
+	if (!rv_wq3) {
+		ret = -ENOMEM;
+		goto fail_wq;
+	}
+
+	rv_dd = kzalloc(sizeof(*rv_dd), GFP_KERNEL);
+	if (!rv_dd) {
+		ret = -ENOMEM;
+		goto fail_wq;
+	}
+
+	rv_dd->class = class_create(THIS_MODULE, DRIVER_NAME);
+	if (IS_ERR(rv_dd->class)) {
+		ret = PTR_ERR(rv_dd->class);
+		pr_err("Could not create device class: %d\n", ret);
+		goto fail_free;
+	}
+	rv_dd->class->devnode = rv_devnode;
+
+	/* Allocate the dev_t */
+	ret = alloc_chrdev_region(&rv_dd->dev, 0, 1, DRIVER_NAME);
+	if (ret < 0) {
+		pr_err("Could not allocate chrdev region (err %d)\n", -ret);
+		goto fail_destroy;
+	}
+
+	/* Add the char device to both sysfs and devfs */
+	device = &rv_dd->user_device;
+	device_initialize(device);
+	device->class = rv_dd->class;
+	device->parent = NULL;
+	device->devt = rv_dd->dev;
+	device->release = rv_dev_release;
+	dev_set_name(device, "%s", DRIVER_NAME);
+	cdev = &rv_dd->user_cdev;
+	cdev_init(cdev, &rv_file_ops);
+	cdev->owner = THIS_MODULE;
+	ret = cdev_device_add(cdev, device);
+	if (ret < 0) {
+		pr_err("Could not add cdev for %s\n", DRIVER_NAME);
+		goto fail_release;
+	}
+
+	return 0;
+
+fail_release:
+	unregister_chrdev_region(rv_dd->dev, 1);
+fail_destroy:
+	class_destroy(rv_dd->class);
+fail_free:
+	kfree(rv_dd);
+	rv_dd = NULL;
+fail_wq:
+	if (rv_wq3) {
+		destroy_workqueue(rv_wq3);
+		rv_wq3 = NULL;
+	}
+	if (rv_wq2) {
+		destroy_workqueue(rv_wq2);
+		rv_wq2 = NULL;
+	}
+	destroy_workqueue(rv_wq);
+	rv_wq = NULL;
+
+	return ret;
+}
+
+/*
+ * We wait for all job devs to finish. At this time, there are no more
+ * users. Please be reminded that in RV_RDMA_MODE_USER mode,
+ * no job_dev is allocated.
+ */
+void rv_file_uninit(void)
+{
+	unsigned long timeout = msecs_to_jiffies(100);
+
+	if (rv_dd) {
+		cdev_device_del(&rv_dd->user_cdev, &rv_dd->user_device);
+		unregister_chrdev_region(rv_dd->dev, 1);
+		class_destroy(rv_dd->class);
+		put_device(&rv_dd->user_device);
+	}
+
+	while (atomic_read(&rv_job_dev_cnt) > 0)
+		schedule_timeout_interruptible(timeout);
+
+	if (rv_wq3) {
+		flush_workqueue(rv_wq3);
+		destroy_workqueue(rv_wq3);
+		rv_wq3 = NULL;
+	}
+	if (rv_wq2) {
+		flush_workqueue(rv_wq2);
+		destroy_workqueue(rv_wq2);
+		rv_wq2 = NULL;
+	}
+	if (rv_wq) {
+		flush_workqueue(rv_wq);
+		destroy_workqueue(rv_wq);
+		rv_wq = NULL;
+	}
+}
+
+/* rv event reporting ring, allocated by mmap */
+static struct rv_user_ring *rv_user_ring_alloc(int rv_inx,
+					       u32 num_entries,
+					       struct vm_area_struct *vma)
+{
+	struct rv_user_ring *ring;
+	unsigned long len;
+	unsigned long pfn;
+	int ret;
+
+	len = RV_RING_ALLOC_LEN(num_entries);
+	len = ALIGN(len, SMP_CACHE_BYTES);
+	if (len > vma->vm_end - vma->vm_start) {
+		rv_err(rv_inx, "mmap too small for ring\n");
+		return ERR_PTR(-EINVAL);
+	}
+
+	ring = kzalloc(sizeof(*ring), GFP_KERNEL);
+	if (!ring)
+		return ERR_PTR(-ENOMEM);
+
+	ring->rv_inx = rv_inx;
+	ring->num_entries = num_entries;
+	ring->order = get_order(len);
+	ring->page =  __get_free_pages(GFP_KERNEL | __GFP_ZERO, ring->order);
+	if (!ring->page) {
+		rv_err(rv_inx, "ring alloc failure\n");
+		ret = -ENOMEM;
+		goto fail;
+	}
+	pfn = virt_to_phys((void *)ring->page) >> PAGE_SHIFT;
+
+	/* remap kernel memory to userspace */
+	ret = remap_pfn_range(vma, vma->vm_start, pfn, len, vma->vm_page_prot);
+	if (ret < 0) {
+		rv_err(rv_inx, "remap failed page:0x%lx pfn:0x%lx\n",
+		       (unsigned long)ring->page, pfn);
+		goto fail;
+	}
+	spin_lock_init(&ring->lock);
+	ring->hdr = (struct rv_ring_header *)ring->page;
+	ring->hdr->head = 0;
+	ring->hdr->tail = 0;
+	ring->hdr->overflow_cnt = 0;
+	trace_rv_user_ring_alloc(ring->rv_inx, ring->num_entries,
+				 ring->hdr->head, ring->hdr->tail);
+	return ring;
+
+fail:
+	if (ring->page)
+		free_pages(ring->page, ring->order);
+	kfree(ring);
+	return ERR_PTR(ret);
+}
+
+/* We sanity check ring->page, should always be != 0 here */
+static void rv_user_ring_free(struct rv_user_ring *ring)
+{
+	trace_rv_user_ring_free(ring->rv_inx, ring->num_entries,
+				ring->hdr->head, ring->hdr->tail);
+	if (ring->page)
+		free_pages(ring->page, ring->order);
+	kfree(ring);
+}
+
+/*
+ * we ignore offset, but if we decide we need multiple queues in future we
+ * can use that to identify which queue is being requested
+ */
+static int rv_file_mmap(struct file *fp, struct vm_area_struct *vma)
+{
+	struct rv_user *rv = fp->private_data;
+	struct rv_user_ring *ring;
+	int ret = -EINVAL;
+
+	trace_rv_msg_mmap(rv->inx, "vma", vma->vm_start, vma->vm_end);
+
+	mutex_lock(&rv->mutex);
+	if (rv->cqr) {
+		rv_err(rv->inx, "ring already allocated\n");
+		goto unlock;
+	}
+	if (!rv->cq_entries) {
+		rv_err(rv->inx, "ring disabled at attach time\n");
+		goto unlock;
+	}
+
+	ring  = rv_user_ring_alloc(rv->inx, rv->cq_entries, vma);
+	if (IS_ERR(ring)) {
+		ret = PTR_ERR(ring);
+		goto unlock;
+	}
+	rv->cqr = ring;
+	ret = 0;
+
+unlock:
+	mutex_unlock(&rv->mutex);
+	return ret;
+}
diff --git a/drivers/infiniband/ulp/rv/rv_main.c b/drivers/infiniband/ulp/rv/rv_main.c
index 7f81f97a01f0..c40ee36c6f9e 100644
--- a/drivers/infiniband/ulp/rv/rv_main.c
+++ b/drivers/infiniband/ulp/rv/rv_main.c
@@ -178,7 +178,7 @@ static void rv_device_detach_users(struct rv_device *dev)
 		list_del_init(&rv->user_entry);
 
 		spin_unlock_irqrestore(&rv_dev_list_lock, flags);
-		/* Detach user here */
+		rv_detach_user(rv);
 		spin_lock_irqsave(&rv_dev_list_lock, flags);
 	}
 	spin_unlock_irqrestore(&rv_dev_list_lock, flags);
diff --git a/drivers/infiniband/ulp/rv/trace_conn.h b/drivers/infiniband/ulp/rv/trace_conn.h
index 2626545b0df6..5ac7ce72dbaf 100644
--- a/drivers/infiniband/ulp/rv/trace_conn.h
+++ b/drivers/infiniband/ulp/rv/trace_conn.h
@@ -233,6 +233,24 @@ DEFINE_EVENT(/* event */
 		jkey_len, jkey, sid, q_depth, ua_next, refcount)
 );
 
+DEFINE_EVENT(/* event */
+	rv_jdev_template, rv_jdev_alloc,
+	TP_PROTO(void *ptr, const char *dev_name, u8 num_conn, u8 index_bits,
+		 u16 loc_gid_index, u32 loc_addr, u8 jkey_len, u8 *jkey,
+		 u64 sid, u32 q_depth, u32 ua_next, u32 refcount),
+	TP_ARGS(ptr, dev_name, num_conn, index_bits, loc_gid_index, loc_addr,
+		jkey_len, jkey, sid, q_depth, ua_next, refcount)
+);
+
+DEFINE_EVENT(/* event */
+	rv_jdev_template, rv_jdev_release,
+	TP_PROTO(void *ptr, const char *dev_name, u8 num_conn, u8 index_bits,
+		 u16 loc_gid_index, u32 loc_addr, u8 jkey_len, u8 *jkey,
+		 u64 sid, u32 q_depth, u32 ua_next, u32 refcount),
+	TP_ARGS(ptr, dev_name, num_conn, index_bits, loc_gid_index, loc_addr,
+		jkey_len, jkey, sid, q_depth, ua_next, refcount)
+);
+
 DECLARE_EVENT_CLASS(/* sconn */
 	rv_sconn_template,
 	TP_PROTO(void *ptr, u8 index, u32 qp_num, void *conn, u32 flags,
diff --git a/drivers/infiniband/ulp/rv/trace_user.h b/drivers/infiniband/ulp/rv/trace_user.h
index 90f67537d249..d03a53f89cc6 100644
--- a/drivers/infiniband/ulp/rv/trace_user.h
+++ b/drivers/infiniband/ulp/rv/trace_user.h
@@ -11,9 +11,70 @@
 #undef TRACE_SYSTEM
 #define TRACE_SYSTEM rv_user
 
+#define RV_USER_PRN "Inx %d rdma_mode %u attached %u dev_name %s " \
+		    "cq_entries %u index %u"
+
 #define RV_USER_MRS_PRN "rv_nx %d jdev %p total_size 0x%llx max_size 0x%llx " \
 			"refcount %u"
 
+#define RV_ATTACH_REQ_PRN "inx %d Device %s rdma_mode %u port_num %u " \
+			  "loc_addr 0x%x jkey_len %u jkey 0x%s " \
+			  " service_id 0x%llx cq_entries %u q_depth %u " \
+			  " timeout %u hb_timeout %u"
+
+DECLARE_EVENT_CLASS(/* user */
+	rv_user_template,
+	TP_PROTO(int inx, u8 rdma_mode, u8 attached, const char *dev_name,
+		 u32 cq_entries, u16 index),
+	TP_ARGS(inx, rdma_mode, attached, dev_name, cq_entries, index),
+	TP_STRUCT__entry(/* entry */
+		__field(int, inx)
+		__field(u8, rdma_mode)
+		__field(u8, attached)
+		__string(name, dev_name)
+		__field(u32, cq_entries)
+		__field(u16, index)
+	),
+	TP_fast_assign(/* assign */
+		__entry->inx = inx;
+		__entry->rdma_mode = rdma_mode;
+		__entry->attached = attached;
+		__assign_str(name, dev_name);
+		__entry->cq_entries = cq_entries;
+		__entry->index = index;
+	),
+	TP_printk(/* print */
+		RV_USER_PRN,
+		__entry->inx,
+		__entry->rdma_mode,
+		__entry->attached,
+		__get_str(name),
+		__entry->cq_entries,
+		__entry->index
+	)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_template, rv_user_open,
+	TP_PROTO(int inx, u8 rdma_mode, u8 attached, const char *dev_name,
+		 u32 cq_entries, u16 index),
+	TP_ARGS(inx, rdma_mode, attached, dev_name, cq_entries, index)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_template, rv_user_close,
+	TP_PROTO(int inx, u8 rdma_mode, u8 attached, const char *dev_name,
+		 u32 cq_entries, u16 index),
+	TP_ARGS(inx, rdma_mode, attached, dev_name, cq_entries, index)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_template, rv_user_attach,
+	TP_PROTO(int inx, u8 rdma_mode, u8 attached, const char *dev_name,
+		 u32 cq_entries, u16 index),
+	TP_ARGS(inx, rdma_mode, attached, dev_name, cq_entries, index)
+);
+
 DECLARE_EVENT_CLASS(/* user msg */
 	rv_user_msg_template,
 	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
@@ -51,6 +112,24 @@ DEFINE_EVENT(/* event */
 	TP_ARGS(inx, msg, d1, d2)
 );
 
+DEFINE_EVENT(/* event */
+	rv_user_msg_template, rv_msg_detach_all,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_msg_template, rv_msg_uconn_remove,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_msg_template, rv_msg_cleanup,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2)
+);
+
 DEFINE_EVENT(/* event */
 	rv_user_msg_template, rv_msg_cmp_params,
 	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
@@ -69,6 +148,12 @@ DEFINE_EVENT(/* event */
 	TP_ARGS(inx, msg, d1, d2)
 );
 
+DEFINE_EVENT(/* event */
+	rv_user_msg_template, rv_msg_mmap,
+	TP_PROTO(int inx, const char *msg, u64 d1, u64 d2),
+	TP_ARGS(inx, msg, d1, d2)
+);
+
 DECLARE_EVENT_CLASS(/* user_mrs */
 	rv_user_mrs_template,
 	TP_PROTO(int rv_inx, void *jdev, u64 total_size, u64 max_size,
@@ -112,6 +197,59 @@ DEFINE_EVENT(/* event */
 	TP_ARGS(rv_inx, jdev, total_size, max_size, refcount)
 );
 
+TRACE_EVENT(/* event */
+	rv_attach_req,
+	TP_PROTO(int inx, char *dev_name, u8 rdma_mode, u8 port_num,
+		 u32 loc_addr, u8 jkey_len, u8 *jkey, u64 service_id,
+		 u32 cq_entries, u32 q_depth, u32 timeout, u32 hb_timeout),
+	TP_ARGS(inx, dev_name, rdma_mode, port_num, loc_addr, jkey_len, jkey,
+		service_id, cq_entries, q_depth, timeout, hb_timeout),
+	TP_STRUCT__entry(/* entry */
+		__field(int, inx)
+		__string(device, dev_name)
+		__field(u8, rdma_mode)
+		__field(u8, port_num)
+		__field(u32, loc_addr)
+		__field(u8, jkey_len)
+		__array(u8, jkey, RV_MAX_JOB_KEY_LEN)
+		__field(u64, service_id)
+		__field(u32, cq_entries)
+		__field(u32, q_depth)
+		__field(u32, timeout)
+		__field(u32, hb_timeout)
+	),
+	TP_fast_assign(/* assign */
+		__entry->inx = inx;
+		__assign_str(device, dev_name);
+		__entry->inx = inx;
+		__entry->rdma_mode = rdma_mode;
+		__entry->port_num = port_num;
+		__entry->loc_addr = loc_addr;
+		__entry->jkey_len = jkey_len;
+		memcpy(__entry->jkey, jkey, RV_MAX_JOB_KEY_LEN);
+		__entry->service_id = service_id;
+		__entry->cq_entries = cq_entries;
+		__entry->q_depth = q_depth;
+		__entry->timeout = timeout;
+		__entry->hb_timeout = hb_timeout;
+	),
+	TP_printk(/* print */
+		RV_ATTACH_REQ_PRN,
+		__entry->inx,
+		__get_str(device),
+		__entry->rdma_mode,
+		__entry->port_num,
+		__entry->loc_addr,
+		__entry->jkey_len,
+		__print_hex_str(__entry->jkey, RV_MAX_JOB_KEY_LEN),
+		__entry->service_id,
+		__entry->cq_entries,
+		__entry->q_depth,
+		__entry->timeout,
+		__entry->hb_timeout
+	)
+);
+
 DECLARE_EVENT_CLASS(/* user_ring */
 	rv_user_ring_template,
 	TP_PROTO(int rv_inx, u32 count, u32 hd, u32 tail),
@@ -137,12 +275,43 @@ DECLARE_EVENT_CLASS(/* user_ring */
 	)
 );
 
+DEFINE_EVENT(/* event */
+	rv_user_ring_template, rv_user_ring_alloc,
+	TP_PROTO(int rv_inx, u32 count, u32 hd, u32 tail),
+	TP_ARGS(rv_inx, count, hd, tail)
+);
+
+DEFINE_EVENT(/* event */
+	rv_user_ring_template, rv_user_ring_free,
+	TP_PROTO(int rv_inx, u32 count, u32 hd, u32 tail),
+	TP_ARGS(rv_inx, count, hd, tail)
+);
+
 DEFINE_EVENT(/* event */
 	rv_user_ring_template, rv_user_ring_post_event,
 	TP_PROTO(int rv_inx, u32 count, u32 hd, u32 tail),
 	TP_ARGS(rv_inx, count, hd, tail)
 );
 
+TRACE_EVENT(/* event */
+	rv_ioctl,
+	TP_PROTO(int inx, u32 cmd),
+	TP_ARGS(inx, cmd),
+	TP_STRUCT__entry(/* entry */
+		__field(int, inx)
+		__field(u32, cmd)
+	),
+	TP_fast_assign(/* assign */
+		__entry->inx = inx;
+		__entry->cmd = cmd;
+	),
+	TP_printk(/* print */
+		"inx %d cmd 0x%x",
+		__entry->inx,
+		__entry->cmd
+	)
+);
+
 #endif /* __RV_TRACE_USER_H */
 
 #undef TRACE_INCLUDE_PATH
-- 
2.18.1


^ permalink raw reply related	[flat|nested] 52+ messages in thread

* [PATCH RFC 9/9] RDMA/rv: Integrate the file operations into the rv module
  2021-03-19 12:56 [PATCH RFC 0/9] A rendezvous module kaike.wan
                   ` (7 preceding siblings ...)
  2021-03-19 12:56 ` [PATCH RFC 8/9] RDMA/rv: Add functions for file operations kaike.wan
@ 2021-03-19 12:56 ` kaike.wan
  2021-03-19 13:53 ` [PATCH RFC 0/9] A rendezvous module Jason Gunthorpe
  9 siblings, 0 replies; 52+ messages in thread
From: kaike.wan @ 2021-03-19 12:56 UTC (permalink / raw)
  To: dledford, jgg; +Cc: linux-rdma, todd.rimmer, Kaike Wan

From: Kaike Wan <kaike.wan@intel.com>

Integrate the file operations into the module_init and module_exit
functions so that user applications can access the rv module.

Signed-off-by: Todd Rimmer <todd.rimmer@intel.com>
Signed-off-by: Kaike Wan <kaike.wan@intel.com>
---
 drivers/infiniband/ulp/rv/rv_main.c | 5 +++++
 1 file changed, 5 insertions(+)

diff --git a/drivers/infiniband/ulp/rv/rv_main.c b/drivers/infiniband/ulp/rv/rv_main.c
index c40ee36c6f9e..04c5a8606598 100644
--- a/drivers/infiniband/ulp/rv/rv_main.c
+++ b/drivers/infiniband/ulp/rv/rv_main.c
@@ -253,11 +253,16 @@ static int __init rv_init_module(void)
 		return -EINVAL;
 	}
 
+	if (rv_file_init()) {
+		ib_unregister_client(&rv_client);
+		return -EINVAL;
+	}
 	return 0;
 }
 
 static void __exit rv_cleanup_module(void)
 {
+	rv_file_uninit();
 	ib_unregister_client(&rv_client);
 	rv_deinit_devices();
 }
-- 
2.18.1


^ permalink raw reply related	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 12:56 [PATCH RFC 0/9] A rendezvous module kaike.wan
                   ` (8 preceding siblings ...)
  2021-03-19 12:56 ` [PATCH RFC 9/9] RDMA/rv: Integrate the file operations into the rv module kaike.wan
@ 2021-03-19 13:53 ` Jason Gunthorpe
  2021-03-19 14:49   ` Wan, Kaike
  9 siblings, 1 reply; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-19 13:53 UTC (permalink / raw)
  To: kaike.wan; +Cc: dledford, linux-rdma, todd.rimmer

On Fri, Mar 19, 2021 at 08:56:26AM -0400, kaike.wan@intel.com wrote:

> - Basic mode of operations (PSM3 is used as an example for user
>   applications):
>   - A middleware (like MPI) has out-of-band communication channels
>     between any two nodes, which are used to establish high performance
>     communications for providers such as PSM3.

Huh? Doesn't PSM3 already use it's own special non-verbs char devices
that already have memory caches and other stuff? Now you want to throw
that all away and do yet another char dev just for HFI? Why?

I also don't know why you picked the name rv, this looks like it has
little to do with the usual MPI rendezvous protocol. This is all about
bulk transfers. It is actually a lot like RDS. Maybe you should be
using RDS?

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 13:53 ` [PATCH RFC 0/9] A rendezvous module Jason Gunthorpe
@ 2021-03-19 14:49   ` Wan, Kaike
  2021-03-19 15:48     ` Jason Gunthorpe
  0 siblings, 1 reply; 52+ messages in thread
From: Wan, Kaike @ 2021-03-19 14:49 UTC (permalink / raw)
  To: Jason Gunthorpe; +Cc: dledford, linux-rdma, Rimmer, Todd

> From: Jason Gunthorpe <jgg@nvidia.com>
> Sent: Friday, March 19, 2021 9:53 AM
> To: Wan, Kaike <kaike.wan@intel.com>
> Cc: dledford@redhat.com; linux-rdma@vger.kernel.org; Rimmer, Todd
> <todd.rimmer@intel.com>
> Subject: Re: [PATCH RFC 0/9] A rendezvous module
> 
> On Fri, Mar 19, 2021 at 08:56:26AM -0400, kaike.wan@intel.com wrote:
> 
> > - Basic mode of operations (PSM3 is used as an example for user
> >   applications):
> >   - A middleware (like MPI) has out-of-band communication channels
> >     between any two nodes, which are used to establish high performance
> >     communications for providers such as PSM3.
> 
> Huh? Doesn't PSM3 already use it's own special non-verbs char devices that
> already have memory caches and other stuff? Now you want to throw that
> all away and do yet another char dev just for HFI? Why?
[Wan, Kaike] I think that you are referring to PSM2, which uses the OPA hfi1 driver that is specific to the OPA hardware.
PSM3 uses standard verbs drivers and supports standard RoCE.  A focus is the Intel RDMA Ethernet NICs. As such it cannot use the hfi1 driver through the special PSM2 interface. Rather it works with the hfi1 driver through standard verbs interface. The rv module was a new design to bring these concepts to standard transports and hardware.

> 
> I also don't know why you picked the name rv, this looks like it has little to do
> with the usual MPI rendezvous protocol. This is all about bulk transfers. It is
> actually a lot like RDS. Maybe you should be using RDS?
[Wan, Kaike] While there are similarities in concepts, details are different.  Quite frankly this could be viewed as an application accelerator much like RDS served that purpose for Oracle, which continues to be its main use case. The rv module is currently targeting to enables the MPI/OFI/PSM3 application.

The name "rv" is chosen simply because this module is designed to enable the rendezvous protocol of the MPI/OFI/PSM3 application stack for large messages. Short messages are handled by eager transfer through UDP in PSM3.

FYI, there is an OFA presentation from Thurs reviewing PSM3 and RV and covering much of the architecture and rationale.
> 
> Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 14:49   ` Wan, Kaike
@ 2021-03-19 15:48     ` Jason Gunthorpe
  2021-03-19 19:22       ` Dennis Dalessandro
  2021-03-19 20:34       ` Hefty, Sean
  0 siblings, 2 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-19 15:48 UTC (permalink / raw)
  To: Wan, Kaike; +Cc: dledford, linux-rdma, Rimmer, Todd

On Fri, Mar 19, 2021 at 02:49:29PM +0000, Wan, Kaike wrote:
> > From: Jason Gunthorpe <jgg@nvidia.com>
> > Sent: Friday, March 19, 2021 9:53 AM
> > To: Wan, Kaike <kaike.wan@intel.com>
> > Cc: dledford@redhat.com; linux-rdma@vger.kernel.org; Rimmer, Todd
> > <todd.rimmer@intel.com>
> > Subject: Re: [PATCH RFC 0/9] A rendezvous module
> > 
> > On Fri, Mar 19, 2021 at 08:56:26AM -0400, kaike.wan@intel.com wrote:
> > 
> > > - Basic mode of operations (PSM3 is used as an example for user
> > >   applications):
> > >   - A middleware (like MPI) has out-of-band communication channels
> > >     between any two nodes, which are used to establish high performance
> > >     communications for providers such as PSM3.
> > 
> > Huh? Doesn't PSM3 already use it's own special non-verbs char devices that
> > already have memory caches and other stuff? Now you want to throw that
> > all away and do yet another char dev just for HFI? Why?

> [Wan, Kaike] I think that you are referring to PSM2, which uses the
> OPA hfi1 driver that is specific to the OPA hardware.  PSM3 uses
> standard verbs drivers and supports standard RoCE.  

Uhhh.. "PSM" has always been about the ipath special char device, and
if I recall properly the library was semi-discontinued and merged into
libfabric.

So here you are talking about a libfabric verbs provider that doesn't
use the ipath style char interface but uses verbs and this rv thing so
we call it a libfabric PSM3 provider because thats not confusing to
anyone at all..

> A focus is the Intel RDMA Ethernet NICs. As such it cannot use the
> hfi1 driver through the special PSM2 interface. 

These are the drivers that aren't merged yet, I see. So why are you
sending this now? I'm not interested to look at even more Intel code
when their driver saga is still ongoing for years.

> Rather it works with the hfi1 driver through standard verbs
> interface.

But nobody would do that right? You'd get better results using the
hif1 native interfaces instead of their slow fake verbs stuff.

> > I also don't know why you picked the name rv, this looks like it has little to do
> > with the usual MPI rendezvous protocol. This is all about bulk transfers. It is
> > actually a lot like RDS. Maybe you should be using RDS?

> [Wan, Kaike] While there are similarities in concepts, details are
> different.  

You should list these differences.

> Quite frankly this could be viewed as an application accelerator
> much like RDS served that purpose for Oracle, which continues to be
> its main use case.

Obviously, except it seems to be doing the same basic acceleration
technique as RDS.

> The name "rv" is chosen simply because this module is designed to
> enable the rendezvous protocol of the MPI/OFI/PSM3 application stack
> for large messages. Short messages are handled by eager transfer
> through UDP in PSM3.

A bad name seems like it will further limit potential re-use of this
code.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 1/9] RDMA/rv: Public interferce for the RDMA Rendezvous module
  2021-03-19 12:56 ` [PATCH RFC 1/9] RDMA/rv: Public interferce for the RDMA Rendezvous module kaike.wan
@ 2021-03-19 16:00   ` Jason Gunthorpe
  2021-03-19 18:42   ` kernel test robot
  1 sibling, 0 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-19 16:00 UTC (permalink / raw)
  To: kaike.wan; +Cc: dledford, linux-rdma, todd.rimmer

On Fri, Mar 19, 2021 at 08:56:27AM -0400, kaike.wan@intel.com wrote:
> From: Kaike Wan <kaike.wan@intel.com>
> 
> The RDMA Rendezvous (rv) module provides an interface for HPC
> middlewares to improve performance by caching memory region
> registration, and improve the scalibity of RDMA transaction
> through connection managements between nodes. This mechanism
> is implemented through the following ioctl requests:
> - ATTACH: to attach to an RDMA device.
> - REG_MEM: to register a user/kernel memory region.
> - DEREG_MEM: to release application use of MR, allowing it to
>              remain in cache.
> - GET_CACHE_STATS: to get cache statistics.
> - CONN_CREATE: to create an RC connection.
> - CONN_CONNECT: to start the connection.
> - CONN_GET_CONN_COUNT: to use as part of error recovery from lost
>                        messages in the application.
> - CONN_GET_STATS: to get connection statistics.
> - GET_EVENT_STATS: to get the RDMA event statistics.
> - POST_RDMA_WR_IMMED: to post an RDMA WRITE WITH IMMED request.
> 
> Signed-off-by: Todd Rimmer <todd.rimmer@intel.com>
> Signed-off-by: Kaike Wan <kaike.wan@intel.com>
>  include/uapi/rdma/rv_user_ioctls.h | 558 +++++++++++++++++++++++++++++
>  1 file changed, 558 insertions(+)
>  create mode 100644 include/uapi/rdma/rv_user_ioctls.h
> 
> diff --git a/include/uapi/rdma/rv_user_ioctls.h b/include/uapi/rdma/rv_user_ioctls.h
> new file mode 100644
> index 000000000000..97e35b722443
> +++ b/include/uapi/rdma/rv_user_ioctls.h
> @@ -0,0 +1,558 @@
> +/* SPDX-License-Identifier: ((GPL-2.0 WITH Linux-syscall-note) OR BSD-3-Clause) */
> +/*
> + * Copyright(c) 2020 - 2021 Intel Corporation.
> + */
> +#ifndef __RV_USER_IOCTL_H__
> +#define __RV_USER_IOCTL_H__
> +#include <rdma/rdma_user_ioctl.h>
> +#include <rdma/ib_user_sa.h>
> +#include <rdma/ib_user_verbs.h>
> +
> +/* Checking /Documentation/userspace-api/ioctl/ioctl-number.rst */
> +#define RV_MAGIC RDMA_IOCTL_MAGIC

No.

> +#define RV_FILE_NAME "/dev/rv"

No.

> +
> +/*
> + * Handles are opaque to application; they are meaningful only to the
> + * RV driver
> + */
> +
> +/* this version of ABI */
> +#define RV_ABI_VER_MAJOR 1
> +#define RV_ABI_VER_MINOR 0

No, see my remarks to your other intel colleagues doing the ioasid
stuff.

> +struct rv_query_params_out {
> +		/* ABI version */
> +	__u16 major_rev;
> +	__u16 minor_rev;
> +	__u32 resv1;
> +	__aligned_u64 capability;
> +	__aligned_u64 resv2[6];

No pre-adding reserved stuff

> +};
> +
> +#define RV_IOCTL_QUERY _IOR(RV_MAGIC, 0xFC, struct rv_query_params_out)
> +
> +/* Mode for use of rv module by PSM */
> +#define RV_RDMA_MODE_USER 0	/* user MR caching only */
> +#define RV_RDMA_MODE_KERNEL 1	/* + kernel RC QPs with kernel MR caching */

Huh? Mode sounds bad.

> +/*
> + * mr_cache_size is in MBs and if 0 will use module param as default
> + * num_conn - number of QPs between each pair of nodes
> + * loc_addr - used to select client/listen vs rem_addr
> + * index_bits - num high bits of immed data with rv index
> + * loc_gid_index - SGID for client connections
> + * loc_gid[16] - to double check gid_index unchanged
> + * job_key[RV_MAX_JOB_KEY_LEN] = unique uuid per job
> + * job_key_len - len, if 0 matches jobs with len==0 only
> + * q_depth - size of QP and per QP CQs
> + * reconnect_timeout - in seconds from loss to restoration
> + * hb_interval - in milliseconds between heartbeats
> + */
> +struct rv_attach_params_in {
> +	char dev_name[RV_MAX_DEV_NAME_LEN];

Guessing this is a no too.

> +	__u32 mr_cache_size;
> +	__u8 rdma_mode;
> +
> +	/* additional information for RV_RDMA_MODE_KERNEL */
> +	__u8 port_num;
> +	__u8 num_conn;

Lots of alignment holes, don't do that either.

> +struct rv_attach_params {
> +	union {
> +		struct rv_attach_params_in in;
> +		struct rv_attach_params_out out;
> +	};
> +};

Yikes, no

> +/* The buffer is used to register a kernel mr */
> +#define IBV_ACCESS_KERNEL 0x80000000

Huh? WTF on on many levels

> +/*
> + * ibv_pd_handle - user space appl allocated pd
> + * ulen - driver_udata inlen
> + * *udata - driver_updata inbuf
> + */
> +struct rv_mem_params_in {
> +	__u32 ibv_pd_handle;
> +	__u32 cmd_fd_int;

Really? You want to reach into the command FD from a ULP and extract
objects? Double yikes. Did you do this properly, taking care of every
lifetime rule and handling disassociation?

> +	__aligned_u64 addr;
> +	__aligned_u64 length;
> +	__u32 access;
> +	size_t ulen;
> +	void *udata;

'void *' is wrong for ioctls.

> +struct rv_conn_get_stats_params_out {

Too many stats, don't you think? Most of the header seems to be stats
of one thing or another.

> +/*
> + * events placed on ring buffer for delivery to user space.
> + * Carefully sized to be a multiple of 64 bytes for cache alignment.
> + * Must pack to get good field alignment and desired 64B overall size
> + * Unlike verbs, all rv_event fields are defined even when
> + * rv_event.wc.status != IB_WC_SUCCESS. Only sent writes can report bad status.
> + * event_type - enum rv_event_type
> + * wc - send or recv work completions
> + *	status - ib_wc_status
> + *	resv1 - alignment
> + *	imm_data - for RV_WC_RECV_RDMA_WITH_IMM only
> + *	wr_id - PSM wr_id for RV_WC_RDMA_WRITE only
> + *	conn_handle - conn handle. For efficiency in completion processing, this
> + *		handle is the rv_conn handle, not the rv_user_conn.
> + *		Main use is sanity checks.  On Recv PSM must use imm_data to
> + *		efficiently identify source.
> + *	byte_len - unlike verbs API, this is always valid
> + *	resv2 - alignment
> + * cache_align -  not used, but forces overall struct to 64B size
> + */
> +struct rv_event {
> +	__u8		event_type;
> +	union {
> +		struct {
> +			__u8		status;
> +			__u16	resv1;
> +			__u32	imm_data;
> +			__aligned_u64	wr_id;
> +			__aligned_u64	conn_handle;
> +			__u32	byte_len;
> +			__u32	resv2;
> +		} __attribute__((__packed__)) wc;
> +		struct {
> +			__u8 pad[7];
> +			uint64_t pad2[7];
> +		} __attribute__((__packed__)) cache_align;
> +	};
> +} __attribute__((__packed__));

Uhh, mixing packed and aligned_u64 is pretty silly. I don't think this
needs to be packed or written in this tortured way.

> +
> +/*
> + * head - consumer removes here
> + * tail - producer adds here
> + * overflow_cnt - number of times producer overflowed ring and discarded
> + * pad - 64B cache alignment for entries
> + */
> +struct rv_ring_header {
> +	volatile __u32 head;
> +	volatile __u32 tail;
> +	volatile __u64 overflow_cnt;

No on volatile, and missed a __aligned here

This uapi needs to be much better. It looks like the mess from the PSM
char dev just re-imported here.

At the very least split the caching from the other operations and
follow normal ioctl design

And you need to rethink the uverbs stuff.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 2/9] RDMA/rv: Add the internal header files
  2021-03-19 12:56 ` [PATCH RFC 2/9] RDMA/rv: Add the internal header files kaike.wan
@ 2021-03-19 16:02   ` Jason Gunthorpe
  0 siblings, 0 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-19 16:02 UTC (permalink / raw)
  To: kaike.wan; +Cc: dledford, linux-rdma, todd.rimmer

On Fri, Mar 19, 2021 at 08:56:28AM -0400, kaike.wan@intel.com wrote:

> +struct rv_mr_cache {
> +	u64 max_size;
> +	void *ops_arg;
> +	struct mmu_notifier mn;

Nope on using raw mmu_notifier, this code isn't done right in hfi1,
I'm not looking at it again here. Use the interval notifier.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 1/9] RDMA/rv: Public interferce for the RDMA Rendezvous module
  2021-03-19 12:56 ` [PATCH RFC 1/9] RDMA/rv: Public interferce for the RDMA Rendezvous module kaike.wan
  2021-03-19 16:00   ` Jason Gunthorpe
@ 2021-03-19 18:42   ` kernel test robot
  1 sibling, 0 replies; 52+ messages in thread
From: kernel test robot @ 2021-03-19 18:42 UTC (permalink / raw)
  To: kbuild-all

[-- Attachment #1: Type: text/plain, Size: 1793 bytes --]

Hi,

[FYI, it's a private test report for your RFC patch.]
[auto build test ERROR on rdma/for-next]
[also build test ERROR on linus/master v5.12-rc3 next-20210319]
[If your patch is applied to the wrong git tree, kindly drop us a note.
And when submitting patch, we suggest to use '--base' as documented in
https://git-scm.com/docs/git-format-patch]

url:    https://github.com/0day-ci/linux/commits/kaike-wan-intel-com/A-rendezvous-module/20210319-210005
base:   https://git.kernel.org/pub/scm/linux/kernel/git/rdma/rdma.git for-next
config: i386-randconfig-s002-20210318 (attached as .config)
compiler: gcc-9 (Debian 9.3.0-22) 9.3.0
reproduce:
        # apt-get install sparse
        # sparse version: v0.6.3-277-gc089cd2d-dirty
        # https://github.com/0day-ci/linux/commit/96b4d25534a5c1ebaab65a35fd9cd242613cd6d1
        git remote add linux-review https://github.com/0day-ci/linux
        git fetch --no-tags linux-review kaike-wan-intel-com/A-rendezvous-module/20210319-210005
        git checkout 96b4d25534a5c1ebaab65a35fd9cd242613cd6d1
        # save the attached .config to linux build tree
        make W=1 C=1 CF='-fdiagnostic-prefix -D__CHECK_ENDIAN__' ARCH=i386 

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>

All errors (new ones prefixed by >>):

   In file included from <command-line>:32:
>> ./usr/include/rdma/rv_user_ioctls.h:117:2: error: unknown type name 'size_t'
     117 |  size_t ulen;
         |  ^~~~~~
>> ./usr/include/rdma/rv_user_ioctls.h:535:4: error: unknown type name 'uint64_t'
     535 |    uint64_t pad2[7];
         |    ^~~~~~~~

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org

[-- Attachment #2: config.gz --]
[-- Type: application/gzip, Size: 36429 bytes --]

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 15:48     ` Jason Gunthorpe
@ 2021-03-19 19:22       ` Dennis Dalessandro
  2021-03-19 19:44         ` Jason Gunthorpe
  2021-03-19 20:34       ` Hefty, Sean
  1 sibling, 1 reply; 52+ messages in thread
From: Dennis Dalessandro @ 2021-03-19 19:22 UTC (permalink / raw)
  To: Jason Gunthorpe, Wan, Kaike; +Cc: dledford, linux-rdma, Rimmer, Todd

On 3/19/2021 11:48 AM, Jason Gunthorpe wrote:
> On Fri, Mar 19, 2021 at 02:49:29PM +0000, Wan, Kaike wrote:
>>> From: Jason Gunthorpe <jgg@nvidia.com>
>>> Sent: Friday, March 19, 2021 9:53 AM
>>> To: Wan, Kaike <kaike.wan@intel.com>
>>> Cc: dledford@redhat.com; linux-rdma@vger.kernel.org; Rimmer, Todd
>>> <todd.rimmer@intel.com>
>>> Subject: Re: [PATCH RFC 0/9] A rendezvous module
>>>
>>> On Fri, Mar 19, 2021 at 08:56:26AM -0400, kaike.wan@intel.com wrote:
>>>
>>>> - Basic mode of operations (PSM3 is used as an example for user
>>>>    applications):
>>>>    - A middleware (like MPI) has out-of-band communication channels
>>>>      between any two nodes, which are used to establish high performance
>>>>      communications for providers such as PSM3.
>>>
>>> Huh? Doesn't PSM3 already use it's own special non-verbs char devices that
>>> already have memory caches and other stuff? Now you want to throw that
>>> all away and do yet another char dev just for HFI? Why?
> 
>> [Wan, Kaike] I think that you are referring to PSM2, which uses the
>> OPA hfi1 driver that is specific to the OPA hardware.  PSM3 uses
>> standard verbs drivers and supports standard RoCE.
> 
> Uhhh.. "PSM" has always been about the ipath special char device, and
> if I recall properly the library was semi-discontinued and merged into
> libfabric.

This driver is intended to work with a fork of the PSM2 library. The 
PSM2 library which is for Omni-Path is now maintained by Cornelis 
Networks on our GitHub. PSM3 is something from Intel for Ethernet. I 
know it's a bit confusing.

> So here you are talking about a libfabric verbs provider that doesn't
> use the ipath style char interface but uses verbs and this rv thing so
> we call it a libfabric PSM3 provider because thats not confusing to
> anyone at all..
> 
>> A focus is the Intel RDMA Ethernet NICs. As such it cannot use the
>> hfi1 driver through the special PSM2 interface.
> 
> These are the drivers that aren't merged yet, I see. So why are you
> sending this now? I'm not interested to look at even more Intel code
> when their driver saga is still ongoing for years.
> 
>> Rather it works with the hfi1 driver through standard verbs
>> interface.
> 
> But nobody would do that right? You'd get better results using the
> hif1 native interfaces instead of their slow fake verbs stuff.

I can't imagine why. I'm not sure what you mean by our slow fake verbs 
stuff? We support verbs just fine. It's certainly not fake.

>>> I also don't know why you picked the name rv, this looks like it has little to do
>>> with the usual MPI rendezvous protocol. This is all about bulk transfers. It is
>>> actually a lot like RDS. Maybe you should be using RDS?
.
.
>> The name "rv" is chosen simply because this module is designed to
>> enable the rendezvous protocol of the MPI/OFI/PSM3 application stack
>> for large messages. Short messages are handled by eager transfer
>> through UDP in PSM3.
> 
> A bad name seems like it will further limit potential re-use of this
> code.
  As to the name, I started this driver when I was still at Intel before 
handing the reins to Kaike when we spun out. I plucked the name rv out 
of the air without much consideration. While the code seems to have 
changed a lot, the name seems to have stuck. It was as Kaike mentions 
because it helps enable the rendezvous side of PSM3.

-Denny

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 19:22       ` Dennis Dalessandro
@ 2021-03-19 19:44         ` Jason Gunthorpe
  2021-03-19 20:12           ` Rimmer, Todd
  2021-03-19 20:18           ` Dennis Dalessandro
  0 siblings, 2 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-19 19:44 UTC (permalink / raw)
  To: Dennis Dalessandro; +Cc: Wan, Kaike, dledford, linux-rdma, Rimmer, Todd

On Fri, Mar 19, 2021 at 03:22:45PM -0400, Dennis Dalessandro wrote:

> > > [Wan, Kaike] I think that you are referring to PSM2, which uses the
> > > OPA hfi1 driver that is specific to the OPA hardware.  PSM3 uses
> > > standard verbs drivers and supports standard RoCE.
> > 
> > Uhhh.. "PSM" has always been about the ipath special char device, and
> > if I recall properly the library was semi-discontinued and merged into
> > libfabric.
> 
> This driver is intended to work with a fork of the PSM2 library. The PSM2
> library which is for Omni-Path is now maintained by Cornelis Networks on our
> GitHub. PSM3 is something from Intel for Ethernet. I know it's a bit
> confusing.

"a bit" huh?

> > So here you are talking about a libfabric verbs provider that doesn't
> > use the ipath style char interface but uses verbs and this rv thing so
> > we call it a libfabric PSM3 provider because thats not confusing to
> > anyone at all..
> > 
> > > A focus is the Intel RDMA Ethernet NICs. As such it cannot use the
> > > hfi1 driver through the special PSM2 interface.
> > 
> > These are the drivers that aren't merged yet, I see. So why are you
> > sending this now? I'm not interested to look at even more Intel code
> > when their driver saga is still ongoing for years.
> > 
> > > Rather it works with the hfi1 driver through standard verbs
> > > interface.
> > 
> > But nobody would do that right? You'd get better results using the
> > hif1 native interfaces instead of their slow fake verbs stuff.
> 
> I can't imagine why. I'm not sure what you mean by our slow fake verbs
> stuff? We support verbs just fine. It's certainly not fake.

hfi1 calls to the kernel for data path operations - that is "fake" in
my book. Verbs was always about avoiding that kernel transition, to
put it back in betrays the spirit. So a kernel call for rv, or the hfi
cdev, or the verbs post-send is really all a wash.

I didn't understand your answer, do you see using this with hfi1 or
not? 

It looks a lot copy&pasted from the hfi1 driver, so now we are on our
third copy of this code :(

And why did it suddenly become a ULP that somehow shares uverbs
resources?? I'm pretty skeptical that can be made to work correctly..

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 19:44         ` Jason Gunthorpe
@ 2021-03-19 20:12           ` Rimmer, Todd
  2021-03-19 20:26             ` Jason Gunthorpe
  2021-03-19 20:18           ` Dennis Dalessandro
  1 sibling, 1 reply; 52+ messages in thread
From: Rimmer, Todd @ 2021-03-19 20:12 UTC (permalink / raw)
  To: Jason Gunthorpe, Dennis Dalessandro
  Cc: Wan, Kaike, dledford, linux-rdma, Rimmer, Todd

> hfi1 calls to the kernel for data path operations - that is "fake" in my book. Verbs was always about avoiding that kernel transition, to put it back in betrays the spirit. > So a kernel call for rv, or the hfi cdev, or the verbs post-send is really all a wash.

To be clear, different vendors have different priorities and hence different HW designs and approaches.  hfi1 approached the HPC latency needs with a uniquely scalable approach with very low latency @scale. Ironically, other vendors have since replicated some of those mechanisms with their own proprietary mechanisms, such as UD-X.  hfi1 approached storage needs and large message HPC transfers with a direct data placement mechanism (aka RDMA).  It fully supported the verbs API and met the performance needs of it's customers with attractive price and power for its target markets.

Kaike sited hfi1 as just one example of a RDMA verbs device which rv can function for.  Various network and server vendors will each make their own recommendations for software configs and various end users will each have their own preferences and requirements.

Todd

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 19:44         ` Jason Gunthorpe
  2021-03-19 20:12           ` Rimmer, Todd
@ 2021-03-19 20:18           ` Dennis Dalessandro
  2021-03-19 20:30             ` Jason Gunthorpe
  1 sibling, 1 reply; 52+ messages in thread
From: Dennis Dalessandro @ 2021-03-19 20:18 UTC (permalink / raw)
  To: Jason Gunthorpe; +Cc: Wan, Kaike, dledford, linux-rdma, Rimmer, Todd

On 3/19/2021 3:44 PM, Jason Gunthorpe wrote:
> On Fri, Mar 19, 2021 at 03:22:45PM -0400, Dennis Dalessandro wrote:
> 
>>>> [Wan, Kaike] I think that you are referring to PSM2, which uses the
>>>> OPA hfi1 driver that is specific to the OPA hardware.  PSM3 uses
>>>> standard verbs drivers and supports standard RoCE.
>>>
>>> Uhhh.. "PSM" has always been about the ipath special char device, and
>>> if I recall properly the library was semi-discontinued and merged into
>>> libfabric.
>>
>> This driver is intended to work with a fork of the PSM2 library. The PSM2
>> library which is for Omni-Path is now maintained by Cornelis Networks on our
>> GitHub. PSM3 is something from Intel for Ethernet. I know it's a bit
>> confusing.
> 
> "a bit" huh?

Just a bit. :)

> 
>>> So here you are talking about a libfabric verbs provider that doesn't
>>> use the ipath style char interface but uses verbs and this rv thing so
>>> we call it a libfabric PSM3 provider because thats not confusing to
>>> anyone at all..
>>>
>>>> A focus is the Intel RDMA Ethernet NICs. As such it cannot use the
>>>> hfi1 driver through the special PSM2 interface.
>>>
>>> These are the drivers that aren't merged yet, I see. So why are you
>>> sending this now? I'm not interested to look at even more Intel code
>>> when their driver saga is still ongoing for years.
>>>
>>>> Rather it works with the hfi1 driver through standard verbs
>>>> interface.
>>>
>>> But nobody would do that right? You'd get better results using the
>>> hif1 native interfaces instead of their slow fake verbs stuff.
>>
>> I can't imagine why. I'm not sure what you mean by our slow fake verbs
>> stuff? We support verbs just fine. It's certainly not fake.
> 
> hfi1 calls to the kernel for data path operations - that is "fake" in
> my book. Verbs was always about avoiding that kernel transition, to
> put it back in betrays the spirit. So a kernel call for rv, or the hfi
> cdev, or the verbs post-send is really all a wash.

Probably better to argue that in another thread I guess.

> I didn't understand your answer, do you see using this with hfi1 or
> not?

I don't see how this could ever use hfi1. So no.

> It looks a lot copy&pasted from the hfi1 driver, so now we are on our
> third copy of this code :(

I haven't had a chance to look beyond the cover letter in depth at how 
things have changed. I really hope it's not that bad.

> And why did it suddenly become a ULP that somehow shares uverbs
> resources?? I'm pretty skeptical that can be made to work correctly..

I see your point.

I was just providing some background/clarification. Bottom line this has 
nothing to do with hfi1 or OPA or PSM2.

-Denny


^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 20:12           ` Rimmer, Todd
@ 2021-03-19 20:26             ` Jason Gunthorpe
  2021-03-19 20:46               ` Rimmer, Todd
  0 siblings, 1 reply; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-19 20:26 UTC (permalink / raw)
  To: Rimmer, Todd; +Cc: Dennis Dalessandro, Wan, Kaike, dledford, linux-rdma

On Fri, Mar 19, 2021 at 08:12:18PM +0000, Rimmer, Todd wrote:
> > hfi1 calls to the kernel for data path operations - that is "fake" in my book. Verbs was always about avoiding that kernel transition, to put it back in betrays the spirit. > So a kernel call for rv, or the hfi cdev, or the verbs post-send is really all a wash.
> 
> To be clear, different vendors have different priorities and hence
> different HW designs and approaches.  hfi1 approached the HPC
> latency needs with a uniquely scalable approach with very low
> latency @scale. 

Yes, we all know the marketing spin the vendors like to use here. A
kernel transition is a kernel transition, and the one in the HFI verbs
path through all the common code is particularly expensive.

I'm suprirsed to hear someone advocate that is a good thing when we
were all told that the hfi1 cdev *must* exist because the kernel
transition through verbs was far to expensive.

> Ironically, other vendors have since replicated some
> of those mechanisms with their own proprietary mechanisms, such as
> UD-X.  

What is a UD-X?

> Kaike sited hfi1 as just one example of a RDMA verbs device which rv
> can function for.

rv seems to completely destroy alot of the HPC performance offloads
that vendors are layering on RC QPs (see what hns and mlx5 are doing),
so I have doubt about this.

It would be better if you could get at least one other device to agree
this is reasonable.

Or work with something that already exists for bulk messaging like
RDS.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 20:18           ` Dennis Dalessandro
@ 2021-03-19 20:30             ` Jason Gunthorpe
  0 siblings, 0 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-19 20:30 UTC (permalink / raw)
  To: Dennis Dalessandro; +Cc: Wan, Kaike, dledford, linux-rdma, Rimmer, Todd

On Fri, Mar 19, 2021 at 04:18:56PM -0400, Dennis Dalessandro wrote:

> I was just providing some background/clarification. Bottom line this has
> nothing to do with hfi1 or OPA or PSM2.

Well, I hope your two companies can agree on naming if Cornelis
someday needs a PSM3 for it's HW.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 15:48     ` Jason Gunthorpe
  2021-03-19 19:22       ` Dennis Dalessandro
@ 2021-03-19 20:34       ` Hefty, Sean
  2021-03-21 12:08         ` Jason Gunthorpe
  1 sibling, 1 reply; 52+ messages in thread
From: Hefty, Sean @ 2021-03-19 20:34 UTC (permalink / raw)
  To: Jason Gunthorpe, Wan, Kaike; +Cc: dledford, linux-rdma, Rimmer, Todd

> > > I also don't know why you picked the name rv, this looks like it has little to do
> > > with the usual MPI rendezvous protocol. This is all about bulk transfers. It is
> > > actually a lot like RDS. Maybe you should be using RDS?
> 
> > [Wan, Kaike] While there are similarities in concepts, details are
> > different.
> 
> You should list these differences.
> 
> > Quite frankly this could be viewed as an application accelerator
> > much like RDS served that purpose for Oracle, which continues to be
> > its main use case.
> 
> Obviously, except it seems to be doing the same basic acceleration
> technique as RDS.

A better name for this might be "scalable RDMA service", with RDMA meaning the transport operation.  My understanding is this is intended to be usable over any IB/RoCE device.

I'm not familiar enough with the details of RDS or this service to know the differences.

- Sean

^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 20:26             ` Jason Gunthorpe
@ 2021-03-19 20:46               ` Rimmer, Todd
  2021-03-19 20:54                 ` Jason Gunthorpe
  0 siblings, 1 reply; 52+ messages in thread
From: Rimmer, Todd @ 2021-03-19 20:46 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Dennis Dalessandro, Wan, Kaike, dledford, linux-rdma, Rimmer, Todd

> I'm suprirsed to hear someone advocate that is a good thing when we were
> all told that the hfi1 cdev *must* exist because the kernel transition through
> verbs was far to expensive.
It depends on the goal vendors have with verbs vs other APIs such as libfabric.  hfi1's verbs goal was focused on storage bandwidth and the cdev was focused on HPC latency and bandwidth for MPI via PSM2 and libfabric.  I'm unclear why we are debating hfi1 here, seems it should be in another thread.

> What is a UD-X?
UD-X is a vendor specific set of HW interfaces and wire protocols implemented in UCX for nVidia Connect-X series of network devices.  Many of it's concepts are very similar to those which ipath and hfi1 HW and software implemented.

> rv seems to completely destroy alot of the HPC performance offloads that
> vendors are layering on RC QPs
Different vendors have different approaches to performance and chose different design trade-offs.

Todd


^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 20:46               ` Rimmer, Todd
@ 2021-03-19 20:54                 ` Jason Gunthorpe
  2021-03-19 20:59                   ` Wan, Kaike
  0 siblings, 1 reply; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-19 20:54 UTC (permalink / raw)
  To: Rimmer, Todd; +Cc: Dennis Dalessandro, Wan, Kaike, dledford, linux-rdma

On Fri, Mar 19, 2021 at 08:46:56PM +0000, Rimmer, Todd wrote:
> > I'm suprirsed to hear someone advocate that is a good thing when we were
> > all told that the hfi1 cdev *must* exist because the kernel transition through
> > verbs was far to expensive.
>
> It depends on the goal vendors have with verbs vs other APIs such as
> libfabric.  hfi1's verbs goal was focused on storage bandwidth and
> the cdev was focused on HPC latency and bandwidth for MPI via PSM2
> and libfabric.  I'm unclear why we are debating hfi1 here, seems it
> should be in another thread.

Because this is copied from hfi1?

> > What is a UD-X?
> UD-X is a vendor specific set of HW interfaces and wire protocols
> implemented in UCX for nVidia Connect-X series of network devices.
> Many of it's concepts are very similar to those which ipath and hfi1
> HW and software implemented.

Oh, there is lots of stuff in UCX, I'm not surprised you similarities
to what PSM did since psm/libfabric/ucx are all solving the same
problems.

> > rv seems to completely destroy alot of the HPC performance offloads that
> > vendors are layering on RC QPs

> Different vendors have different approaches to performance and chose
> different design trade-offs.

That isn't my point, by limiting the usability you also restrict the
drivers where this would meaningfully be useful.

So far we now know that it is not useful for mlx5 or hfi1, that leaves
only hns unknown and still in the HPC arena.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 20:54                 ` Jason Gunthorpe
@ 2021-03-19 20:59                   ` Wan, Kaike
  2021-03-19 21:28                     ` Dennis Dalessandro
  0 siblings, 1 reply; 52+ messages in thread
From: Wan, Kaike @ 2021-03-19 20:59 UTC (permalink / raw)
  To: Jason Gunthorpe, Rimmer, Todd; +Cc: Dennis Dalessandro, dledford, linux-rdma

> From: Jason Gunthorpe <jgg@nvidia.com>
> Sent: Friday, March 19, 2021 4:55 PM
> To: Rimmer, Todd <todd.rimmer@intel.com>
> Cc: Dennis Dalessandro <dennis.dalessandro@cornelisnetworks.com>; Wan,
> Oh, there is lots of stuff in UCX, I'm not surprised you similarities to what
> PSM did since psm/libfabric/ucx are all solving the same problems.
> 
> > > rv seems to completely destroy alot of the HPC performance offloads
> > > that vendors are layering on RC QPs
> 
> > Different vendors have different approaches to performance and chose
> > different design trade-offs.
> 
> That isn't my point, by limiting the usability you also restrict the drivers where
> this would meaningfully be useful.
> 
> So far we now know that it is not useful for mlx5 or hfi1, that leaves only hns
> unknown and still in the HPC arena.
[Wan, Kaike] Incorrect. The rv module works with hfi1.
> 
> Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 20:59                   ` Wan, Kaike
@ 2021-03-19 21:28                     ` Dennis Dalessandro
  2021-03-19 21:58                       ` Wan, Kaike
  2021-03-19 22:57                       ` Rimmer, Todd
  0 siblings, 2 replies; 52+ messages in thread
From: Dennis Dalessandro @ 2021-03-19 21:28 UTC (permalink / raw)
  To: Wan, Kaike, Jason Gunthorpe, Rimmer, Todd; +Cc: dledford, linux-rdma

On 3/19/2021 4:59 PM, Wan, Kaike wrote:
>> From: Jason Gunthorpe <jgg@nvidia.com>
>> Sent: Friday, March 19, 2021 4:55 PM
>> To: Rimmer, Todd <todd.rimmer@intel.com>
>> Cc: Dennis Dalessandro <dennis.dalessandro@cornelisnetworks.com>; Wan,
>> Oh, there is lots of stuff in UCX, I'm not surprised you similarities to what
>> PSM did since psm/libfabric/ucx are all solving the same problems.
>>
>>>> rv seems to completely destroy alot of the HPC performance offloads
>>>> that vendors are layering on RC QPs
>>
>>> Different vendors have different approaches to performance and chose
>>> different design trade-offs.
>>
>> That isn't my point, by limiting the usability you also restrict the drivers where
>> this would meaningfully be useful.
>>
>> So far we now know that it is not useful for mlx5 or hfi1, that leaves only hns
>> unknown and still in the HPC arena.
> [Wan, Kaike] Incorrect. The rv module works with hfi1.

Interesting. I was thinking the opposite. So what's the benefit? When 
would someone want to do that?

-Denny

^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 21:28                     ` Dennis Dalessandro
@ 2021-03-19 21:58                       ` Wan, Kaike
  2021-03-19 22:35                         ` Jason Gunthorpe
  2021-03-19 22:57                       ` Rimmer, Todd
  1 sibling, 1 reply; 52+ messages in thread
From: Wan, Kaike @ 2021-03-19 21:58 UTC (permalink / raw)
  To: Dennis Dalessandro, Jason Gunthorpe, Rimmer, Todd; +Cc: dledford, linux-rdma


> From: Dennis Dalessandro <dennis.dalessandro@cornelisnetworks.com>
> Sent: Friday, March 19, 2021 5:28 PM
> To: Wan, Kaike <kaike.wan@intel.com>; Jason Gunthorpe <jgg@nvidia.com>;
> Rimmer, Todd <todd.rimmer@intel.com>
> Cc: dledford@redhat.com; linux-rdma@vger.kernel.org
> Subject: Re: [PATCH RFC 0/9] A rendezvous module
> 
> On 3/19/2021 4:59 PM, Wan, Kaike wrote:
> >> From: Jason Gunthorpe <jgg@nvidia.com>
> >> Sent: Friday, March 19, 2021 4:55 PM
> >> To: Rimmer, Todd <todd.rimmer@intel.com>
> >> Cc: Dennis Dalessandro <dennis.dalessandro@cornelisnetworks.com>;
> >> Wan, Oh, there is lots of stuff in UCX, I'm not surprised you
> >> similarities to what PSM did since psm/libfabric/ucx are all solving the
> same problems.
> >>
> >>>> rv seems to completely destroy alot of the HPC performance offloads
> >>>> that vendors are layering on RC QPs
> >>
> >>> Different vendors have different approaches to performance and chose
> >>> different design trade-offs.
> >>
> >> That isn't my point, by limiting the usability you also restrict the
> >> drivers where this would meaningfully be useful.
> >>
> >> So far we now know that it is not useful for mlx5 or hfi1, that
> >> leaves only hns unknown and still in the HPC arena.
> > [Wan, Kaike] Incorrect. The rv module works with hfi1.
> 
> Interesting. I was thinking the opposite. So what's the benefit? When would
> someone want to do that?
[Wan, Kaike] This is only because rv works with the generic Verbs interface and hfi1 happens to be one of the devices that implements the Verbs interface.
> 
> -Denny

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 21:58                       ` Wan, Kaike
@ 2021-03-19 22:35                         ` Jason Gunthorpe
  0 siblings, 0 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-19 22:35 UTC (permalink / raw)
  To: Wan, Kaike; +Cc: Dennis Dalessandro, Rimmer, Todd, dledford, linux-rdma

On Fri, Mar 19, 2021 at 09:58:21PM +0000, Wan, Kaike wrote:

> > On 3/19/2021 4:59 PM, Wan, Kaike wrote:
> > >> From: Jason Gunthorpe <jgg@nvidia.com>
> > >> Sent: Friday, March 19, 2021 4:55 PM
> > >> To: Rimmer, Todd <todd.rimmer@intel.com>
> > >> Cc: Dennis Dalessandro <dennis.dalessandro@cornelisnetworks.com>;
> > >> Wan, Oh, there is lots of stuff in UCX, I'm not surprised you
> > >> similarities to what PSM did since psm/libfabric/ucx are all solving the
> > same problems.
> > >>
> > >>>> rv seems to completely destroy alot of the HPC performance offloads
> > >>>> that vendors are layering on RC QPs
> > >>
> > >>> Different vendors have different approaches to performance and chose
> > >>> different design trade-offs.
> > >>
> > >> That isn't my point, by limiting the usability you also restrict the
> > >> drivers where this would meaningfully be useful.
> > >>
> > >> So far we now know that it is not useful for mlx5 or hfi1, that
> > >> leaves only hns unknown and still in the HPC arena.
> > > [Wan, Kaike] Incorrect. The rv module works with hfi1.
> > 
> > Interesting. I was thinking the opposite. So what's the benefit? When would
> > someone want to do that?

> [Wan, Kaike] This is only because rv works with the generic Verbs
> interface and hfi1 happens to be one of the devices that implements
> the Verbs interface.

Well, it works with everything (except EFA), but that is not my point.

I said it is not useful.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 21:28                     ` Dennis Dalessandro
  2021-03-19 21:58                       ` Wan, Kaike
@ 2021-03-19 22:57                       ` Rimmer, Todd
  2021-03-19 23:06                         ` Jason Gunthorpe
                                           ` (2 more replies)
  1 sibling, 3 replies; 52+ messages in thread
From: Rimmer, Todd @ 2021-03-19 22:57 UTC (permalink / raw)
  To: Dennis Dalessandro, Wan, Kaike, Jason Gunthorpe
  Cc: dledford, linux-rdma, Rimmer, Todd

> > [Wan, Kaike] Incorrect. The rv module works with hfi1.
> 
> Interesting. I was thinking the opposite. So what's the benefit? When would
> someone want to do that?
The more interesting scenario is for customers who would like to run libfabric and other Open Fabrics Alliance software over various verbs capable hardware.
Today PSM2 is a good choice for OPA hardware.  However for some other devices without existing libfabric providers, rxm and rxd are the best choices.
As was presented in Open Fabrics workshop today by James Erwin, PSM3 offers noticeable benefits over existing libfabric rxm and rxd providers
and the rv module offers noticeable performance benefits when using PSM3.

> This driver is intended to work with a fork of the PSM2 library. The
> PSM2 library which is for Omni-Path is now maintained by Cornelis
> Networks on our GitHub. PSM3 is something from Intel for Ethernet. I
> know it's a bit confusing.
Intel retains various IP and trademark rights.  Intel's marketing team analyzed and chose the name PSM3.  Obviously plusses and minuses to any name choice.

This is not unlike other industry software history where new major revisions often add and remove support for various HW generations.
PSM(1) - supported infinipath IB adapters, was a standalone API (various forms).
PSM2 - dropped support for infinipath and IB and added support for Omni-Path, along with various features, also added libfabric support
PSM3 - dropped support for Omni-Path, added support for RoCE and verbs capable devices, along with other features,
	also dropped PSM2 API and standardized on libfabric.
All three have similar strategies of onload protocols for eager messages and shared kernel/HW resources for large messages
and direct data placement (RDMA).  So the name Performance Scaled Messaging is meant to reflect the concept and approach
as opposed to reflecting a specific HW implementation or even API.

PSM3 is only available as a libfabric provider.

> I haven't had a chance to look beyond the cover letter in depth at how things
> have changed. I really hope it's not that bad.
While a few stylistic elements got carried forward, as you noticed.  This is much different from hfi1 as it doesn't directly access hardware and is hence smaller.
We carefully looked at overlap with features in ib_core and the patch set contains a couple minor API additions to ib_core to simplify some operations
which others may find useful.

> I also don't know why you picked the name rv, this looks like it has little to do with the usual MPI rendezvous protocol.
The focus of the design was to support the bulk transfer part of the MPI rendezvous protocol, hence the name rv.
We'd welcome other name suggestions, wanted to keep the name simple and brief.

> No pre-adding reserved stuff
> Lots of alignment holes, don't do that either.
We'd like advise on a challenging situation.  Some customers desire NICs to support nVidia GPUs in some environments.
Unfortunately the nVidia GPU drivers are not upstream, and have not been for years.  So we are forced to have both out of tree
and upstream versions of the code.  We need the same applications to be able to work over both, so we would like the
GPU enabled versions of the code to have the same ABI as the upstream code as this greatly simplifies things.
We have removed all GPU specific code from the upstream submission, but used both the "alignment holes" and the "reserved"
mechanisms to hold places for GPU specific fields which can't be upstreamed.

Todd

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 22:57                       ` Rimmer, Todd
@ 2021-03-19 23:06                         ` Jason Gunthorpe
  2021-03-20 16:39                         ` Dennis Dalessandro
  2021-03-23 15:30                         ` Christoph Hellwig
  2 siblings, 0 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-19 23:06 UTC (permalink / raw)
  To: Rimmer, Todd; +Cc: Dennis Dalessandro, Wan, Kaike, dledford, linux-rdma

On Fri, Mar 19, 2021 at 10:57:20PM +0000, Rimmer, Todd wrote:

> We'd like advise on a challenging situation.  Some customers desire
> NICs to support nVidia GPUs in some environments. Unfortunately the
> nVidia GPU drivers are not upstream, and have not been for years.

Yep.

> So we are forced to have both out of tree and upstream versions of
> the code.

Sure

>  We need the same applications to be able to work over
> both, so we would like the GPU enabled versions of the code to have
> the same ABI as the upstream code as this greatly simplifies things.

The only answer is to get it upstream past community review before you
ship anything to customers.

> We have removed all GPU specific code from the upstream submission,
> but used both the "alignment holes" and the "reserved" mechanisms to
> hold places for GPU specific fields which can't be upstreamed.

Sorry, nope, you have to do something else. We will *not* have garbage
in upstream or be restricted in any way based on out of tree code.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 22:57                       ` Rimmer, Todd
  2021-03-19 23:06                         ` Jason Gunthorpe
@ 2021-03-20 16:39                         ` Dennis Dalessandro
  2021-03-21  8:56                           ` Leon Romanovsky
  2021-03-23 15:30                         ` Christoph Hellwig
  2 siblings, 1 reply; 52+ messages in thread
From: Dennis Dalessandro @ 2021-03-20 16:39 UTC (permalink / raw)
  To: Rimmer, Todd, Wan, Kaike, Jason Gunthorpe; +Cc: dledford, linux-rdma

On 3/19/2021 6:57 PM, Rimmer, Todd wrote:
>>> [Wan, Kaike] Incorrect. The rv module works with hfi1.
>>
>> Interesting. I was thinking the opposite. So what's the benefit? When would
>> someone want to do that?
> The more interesting scenario is for customers who would like to run libfabric and other Open Fabrics Alliance software over various verbs capable hardware.

Ah ok that makes sense. Not that running it over hfi1 is the goal but 
being able to run over verbs devices. Makes sense to me now.

> Today PSM2 is a good choice for OPA hardware.  However for some other devices without existing libfabric providers, rxm and rxd are the best choices.
> As was presented in Open Fabrics workshop today by James Erwin, PSM3 offers noticeable benefits over existing libfabric rxm and rxd providers
> and the rv module offers noticeable performance benefits when using PSM3.

For those that haven't seen it the talks will be posted to YouTube 
and/or OpenFabrics.org web page. There are actually two talks on this 
stuff. The first of which is by Todd is available now [1], James' talk 
will be up soon I'm sure.

>> I haven't had a chance to look beyond the cover letter in depth at how things
>> have changed. I really hope it's not that bad.
> While a few stylistic elements got carried forward, as you noticed.  This is much different from hfi1 as it doesn't directly access hardware and is hence smaller.
> We carefully looked at overlap with features in ib_core and the patch set contains a couple minor API additions to ib_core to simplify some operations
> which others may find useful.

Right, so if there is common functionality between hfi1 and rv then it 
might belong in the core. Especially considering if it's something 
that's common between a ULP and a HW driver.

>> I also don't know why you picked the name rv, this looks like it has little to do with the usual MPI rendezvous protocol.
> The focus of the design was to support the bulk transfer part of the MPI rendezvous protocol, hence the name rv.
> We'd welcome other name suggestions, wanted to keep the name simple and brief.

Like I said previously you can place the blame for the name on me. Kaike 
and Todd just carried it forward. I think Sean had an idea in one of the 
other replies. Let's hear some other suggestions too.

>> No pre-adding reserved stuff
>> Lots of alignment holes, don't do that either.
> We'd like advise on a challenging situation.  Some customers desire NICs to support nVidia GPUs in some environments.
> Unfortunately the nVidia GPU drivers are not upstream, and have not been for years.  So we are forced to have both out of tree
> and upstream versions of the code.  We need the same applications to be able to work over both, so we would like the
> GPU enabled versions of the code to have the same ABI as the upstream code as this greatly simplifies things.
> We have removed all GPU specific code from the upstream submission, but used both the "alignment holes" and the "reserved"
> mechanisms to hold places for GPU specific fields which can't be upstreamed.

This problem extends to other drivers as well. I'm also interested in 
advice on the situation. I don't particularly like this either, but we 
need a way to accomplish the goal. We owe it to users to be flexible. 
Please offer suggestions.

[1] https://www.youtube.com/watch?v=iOvt_Iqz0uU

-Denny

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-20 16:39                         ` Dennis Dalessandro
@ 2021-03-21  8:56                           ` Leon Romanovsky
  2021-03-21 16:24                             ` Dennis Dalessandro
  0 siblings, 1 reply; 52+ messages in thread
From: Leon Romanovsky @ 2021-03-21  8:56 UTC (permalink / raw)
  To: Dennis Dalessandro
  Cc: Rimmer, Todd, Wan, Kaike, Jason Gunthorpe, dledford, linux-rdma

On Sat, Mar 20, 2021 at 12:39:46PM -0400, Dennis Dalessandro wrote:
> On 3/19/2021 6:57 PM, Rimmer, Todd wrote:
> > > > [Wan, Kaike] Incorrect. The rv module works with hfi1.

<...>

> > We have removed all GPU specific code from the upstream submission, but used both the "alignment holes" and the "reserved"
> > mechanisms to hold places for GPU specific fields which can't be upstreamed.
> 
> This problem extends to other drivers as well. I'm also interested in advice
> on the situation. I don't particularly like this either, but we need a way
> to accomplish the goal. We owe it to users to be flexible. Please offer
> suggestions.

Sorry to interrupt, but it seems that solution was said here [1].
It wasn't said directly, but between the lines it means that you need
two things:
1. Upstream everything.
2. Find another vendor that jumps on this RV bandwagon.

[1] https://lore.kernel.org/linux-rdma/20210319230644.GI2356281@nvidia.com

Thanks

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 20:34       ` Hefty, Sean
@ 2021-03-21 12:08         ` Jason Gunthorpe
  0 siblings, 0 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-21 12:08 UTC (permalink / raw)
  To: Hefty, Sean; +Cc: Wan, Kaike, dledford, linux-rdma, Rimmer, Todd

On Fri, Mar 19, 2021 at 08:34:17PM +0000, Hefty, Sean wrote:
> > > > I also don't know why you picked the name rv, this looks like it has little to do
> > > > with the usual MPI rendezvous protocol. This is all about bulk transfers. It is
> > > > actually a lot like RDS. Maybe you should be using RDS?
> > 
> > > [Wan, Kaike] While there are similarities in concepts, details are
> > > different.
> > 
> > You should list these differences.
> > 
> > > Quite frankly this could be viewed as an application accelerator
> > > much like RDS served that purpose for Oracle, which continues to be
> > > its main use case.
> > 
> > Obviously, except it seems to be doing the same basic acceleration
> > technique as RDS.
> 
> A better name for this might be "scalable RDMA service", with RDMA
> meaning the transport operation.  My understanding is this is
> intended to be usable over any IB/RoCE device.

It looks like it is really all about bulk zero-copy transfers with
kernel involvment.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-21  8:56                           ` Leon Romanovsky
@ 2021-03-21 16:24                             ` Dennis Dalessandro
  2021-03-21 16:45                               ` Jason Gunthorpe
  2021-03-23 15:33                               ` Christoph Hellwig
  0 siblings, 2 replies; 52+ messages in thread
From: Dennis Dalessandro @ 2021-03-21 16:24 UTC (permalink / raw)
  To: Leon Romanovsky
  Cc: Rimmer, Todd, Wan, Kaike, Jason Gunthorpe, dledford, linux-rdma

On 3/21/2021 4:56 AM, Leon Romanovsky wrote:
> On Sat, Mar 20, 2021 at 12:39:46PM -0400, Dennis Dalessandro wrote:
>> On 3/19/2021 6:57 PM, Rimmer, Todd wrote:
>>>>> [Wan, Kaike] Incorrect. The rv module works with hfi1.
> 
> <...>
> 
>>> We have removed all GPU specific code from the upstream submission, but used both the "alignment holes" and the "reserved"
>>> mechanisms to hold places for GPU specific fields which can't be upstreamed.
>>
>> This problem extends to other drivers as well. I'm also interested in advice
>> on the situation. I don't particularly like this either, but we need a way
>> to accomplish the goal. We owe it to users to be flexible. Please offer
>> suggestions.
> 
> Sorry to interrupt, but it seems that solution was said here [1].
> It wasn't said directly, but between the lines it means that you need
> two things:
> 1. Upstream everything.

Completely agree. However the GPU code from nvidia is not upstream. I 
don't see that issue getting resolved in this code review. Let's move on.

> 2. Find another vendor that jumps on this RV bandwagon.

That's not a valid argument. Clearly this works over multiple vendors HW.

We should be trying to get things upstream, not putting up walls when 
people want to submit new drivers. Calling code "garbage" [1] is not 
productive.

 > [1] 
https://lore.kernel.org/linux-rdma/20210319230644.GI2356281@nvidia.com

-Denny

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-21 16:24                             ` Dennis Dalessandro
@ 2021-03-21 16:45                               ` Jason Gunthorpe
  2021-03-21 17:21                                 ` Dennis Dalessandro
  2021-03-23 15:35                                 ` Christoph Hellwig
  2021-03-23 15:33                               ` Christoph Hellwig
  1 sibling, 2 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-21 16:45 UTC (permalink / raw)
  To: Dennis Dalessandro
  Cc: Leon Romanovsky, Rimmer, Todd, Wan, Kaike, dledford, linux-rdma

On Sun, Mar 21, 2021 at 12:24:39PM -0400, Dennis Dalessandro wrote:
> On 3/21/2021 4:56 AM, Leon Romanovsky wrote:
> > On Sat, Mar 20, 2021 at 12:39:46PM -0400, Dennis Dalessandro wrote:
> > > On 3/19/2021 6:57 PM, Rimmer, Todd wrote:
> > > > > > [Wan, Kaike] Incorrect. The rv module works with hfi1.
> > 
> > <...>
> > 
> > > > We have removed all GPU specific code from the upstream submission, but used both the "alignment holes" and the "reserved"
> > > > mechanisms to hold places for GPU specific fields which can't be upstreamed.
> > > 
> > > This problem extends to other drivers as well. I'm also interested in advice
> > > on the situation. I don't particularly like this either, but we need a way
> > > to accomplish the goal. We owe it to users to be flexible. Please offer
> > > suggestions.
> > 
> > Sorry to interrupt, but it seems that solution was said here [1].
> > It wasn't said directly, but between the lines it means that you need
> > two things:
> > 1. Upstream everything.
> 
> Completely agree. However the GPU code from nvidia is not upstream. I don't
> see that issue getting resolved in this code review. Let's move on.
> 
> > 2. Find another vendor that jumps on this RV bandwagon.
> 
> That's not a valid argument. Clearly this works over multiple vendors HW.

At a certain point we have to decide if this is a generic code of some
kind or a driver-specific thing like HFI has.

There are also obvious technial problems designing it as a ULP, so it
is a very important question to answer. If it will only ever be used
by one Intel ethernet chip then maybe it isn't really generic code.

On the other hand it really looks like it overlaps in various ways
with both RDS and the qib/hfi1 cdev, so why isn't there any effort to
have some commonality??

> We should be trying to get things upstream, not putting up walls when people
> want to submit new drivers. Calling code "garbage" [1] is not productive.

Putting a bunch of misaligned structures and random reserved fields
*is* garbage by the upstream standard and if I send that to Linus I'll
get yelled at.

And you certainly can't say "we are already shipping this ABI so we
won't change it" either.

You can't square this circle by compromising the upstream world in any
way, it is simply not accepted by the overall community.

All of you should know this, I shouldn't have to lecture on this!

Also no, we should not be "trying to get things upstream" as some goal
in itself. Upstream is not a trashcan to dump stuff into, someone has
to maintain all of this long after it stops being interesting, so
there better be good reasons to put it here in the first place.

If it isn't obvious, I'll repeat again: I'm highly annoyed that Intel
is sending something like this RV, in the state it is in, to support
their own out of tree driver, that they themselves have been dragging
their feet on responding to review comments so it can be upstream for
*years*.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-21 16:45                               ` Jason Gunthorpe
@ 2021-03-21 17:21                                 ` Dennis Dalessandro
  2021-03-21 18:08                                   ` Jason Gunthorpe
                                                     ` (2 more replies)
  2021-03-23 15:35                                 ` Christoph Hellwig
  1 sibling, 3 replies; 52+ messages in thread
From: Dennis Dalessandro @ 2021-03-21 17:21 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Leon Romanovsky, Rimmer, Todd, Wan, Kaike, dledford, linux-rdma

On 3/21/2021 12:45 PM, Jason Gunthorpe wrote:
> On Sun, Mar 21, 2021 at 12:24:39PM -0400, Dennis Dalessandro wrote:
>> On 3/21/2021 4:56 AM, Leon Romanovsky wrote:
>>> On Sat, Mar 20, 2021 at 12:39:46PM -0400, Dennis Dalessandro wrote:
>>>> On 3/19/2021 6:57 PM, Rimmer, Todd wrote:
>>>>>>> [Wan, Kaike] Incorrect. The rv module works with hfi1.
>>>
>>> <...>
>>>
>>>>> We have removed all GPU specific code from the upstream submission, but used both the "alignment holes" and the "reserved"
>>>>> mechanisms to hold places for GPU specific fields which can't be upstreamed.
>>>>
>>>> This problem extends to other drivers as well. I'm also interested in advice
>>>> on the situation. I don't particularly like this either, but we need a way
>>>> to accomplish the goal. We owe it to users to be flexible. Please offer
>>>> suggestions.
>>>
>>> Sorry to interrupt, but it seems that solution was said here [1].
>>> It wasn't said directly, but between the lines it means that you need
>>> two things:
>>> 1. Upstream everything.
>>
>> Completely agree. However the GPU code from nvidia is not upstream. I don't
>> see that issue getting resolved in this code review. Let's move on.
>>
>>> 2. Find another vendor that jumps on this RV bandwagon.
>>
>> That's not a valid argument. Clearly this works over multiple vendors HW.
> 
> At a certain point we have to decide if this is a generic code of some
> kind or a driver-specific thing like HFI has.
> 
> There are also obvious technial problems designing it as a ULP, so it
> is a very important question to answer. If it will only ever be used
> by one Intel ethernet chip then maybe it isn't really generic code.

Todd/Kaike, is there something in here that is specific to the Intel 
ethernet chip?

> On the other hand it really looks like it overlaps in various ways
> with both RDS and the qib/hfi1 cdev, so why isn't there any effort to
> have some commonality??

Maybe that's something that should be explored. Isn't this along the 
lines of stuff we talked about with the verbs 2.0 stuff, or whatever we 
ended up calling it.

>> We should be trying to get things upstream, not putting up walls when people
>> want to submit new drivers. Calling code "garbage" [1] is not productive.
> 
> Putting a bunch of misaligned structures and random reserved fields
> *is* garbage by the upstream standard and if I send that to Linus I'll
> get yelled at.

Not saying you should send this to Linus. I'm saying we should figure 
out a way to make it better and insulting people and their hard work 
isn't helping. This is the kind of culture we are trying to get away 
from in the kernel world.

> And you certainly can't say "we are already shipping this ABI so we
> won't change it" either.
> 
> You can't square this circle by compromising the upstream world in any
> way, it is simply not accepted by the overall community.
> 
> All of you should know this, I shouldn't have to lecture on this!

No one is suggesting to compromise the upstream world. There is a bigger 
picture here. The answer for this driver may just be take out the 
reserved stuff. That's pretty simple. The bigger question is how do we 
deal with non-upstream code. It can't be a problem unique to the RDMA 
subsystem. How do others deal with it?

> Also no, we should not be "trying to get things upstream" as some goal
> in itself. Upstream is not a trashcan to dump stuff into, someone has
> to maintain all of this long after it stops being interesting, so
> there better be good reasons to put it here in the first place.

That is completely not what I meant at all. I mean we should be trying 
to get rid of the proprietary, and out of tree stuff. It doesn't at all 
imply to fling crap against the wall and hope it sticks. We should be 
encouraging vendors to submit their code and work with them to get it in 
shape. We clearly have a problem with vendor proprietary code not being 
open. Let's not encourage that behavior. Vendors should say I want to 
submit my code to the Linux kernel. Not eh, it's too much of a hassle 
and kernel people are jerks, so we'll just post it on our website.

> If it isn't obvious, I'll repeat again: I'm highly annoyed that Intel
> is sending something like this RV, in the state it is in, to support
> their own out of tree driver, that they themselves have been dragging
> their feet on responding to review comments so it can be upstream for
> *years*.

To be fair it is sent as RFC. So to me that says they know it's a ways 
off from being ready to be included and are starting the process early.

-Denny


^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-21 17:21                                 ` Dennis Dalessandro
@ 2021-03-21 18:08                                   ` Jason Gunthorpe
  2021-03-22 15:17                                     ` Rimmer, Todd
  2021-03-22 17:31                                     ` Hefty, Sean
  2021-03-21 19:19                                   ` Wan, Kaike
  2021-03-23 15:36                                   ` Christoph Hellwig
  2 siblings, 2 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-21 18:08 UTC (permalink / raw)
  To: Dennis Dalessandro
  Cc: Leon Romanovsky, Rimmer, Todd, Wan, Kaike, dledford, linux-rdma

On Sun, Mar 21, 2021 at 01:21:14PM -0400, Dennis Dalessandro wrote:

> Maybe that's something that should be explored. Isn't this along the lines
> of stuff we talked about with the verbs 2.0 stuff, or whatever we ended up
> calling it.

I think verbs 2.0 turned into the ioctl uapi stuff, I don't remember anymore.

> > > We should be trying to get things upstream, not putting up walls when people
> > > want to submit new drivers. Calling code "garbage" [1] is not productive.
> > 
> > Putting a bunch of misaligned structures and random reserved fields
> > *is* garbage by the upstream standard and if I send that to Linus I'll
> > get yelled at.
> 
> Not saying you should send this to Linus. I'm saying we should figure out a
> way to make it better and insulting people and their hard work isn't
> helping.

No - you've missed what happened here. Todd responded very fast and
explained - Intel *knowingly* sent code that was sub-standard as some
calculated attempt to make Intel's life maintaining their out of tree
drivers easier.

This absoultely needs strong language as I can't catch everything and
people need to understand there are real consequences to violating the
community trust in this way!

> No one is suggesting to compromise the upstream world. 

I'm not sure what you mean - what could upstream do to in any way
change the situation other than compromising on what will be merged?

> There is a bigger picture here. The answer for this driver may just
> be take out the reserved stuff. That's pretty simple. The bigger
> question is how do we deal with non-upstream code. It can't be a
> problem unique to the RDMA subsystem. How do others deal with it?

The kernel community consensus is that upstream code is on its own.

We don't change the kernel to accomodate out-of-tree code. If the kernel
changes and out of tree breaks nobody cares.

Over a long time it has been proven that this methodology is a good
way to effect business change to align with the community consensus
development model - eventually the costs of being out of tree have bad
ROI and companies align.

> That is completely not what I meant at all. I mean we should be
> trying to get rid of the proprietary, and out of tree stuff. It
> doesn't at all imply to fling crap against the wall and hope it
> sticks. We should be encouraging vendors to submit their code and
> work with them to get it in shape.

Well, I am working on something like 4-5 Intel series right now, and
it sometimes does feel like flinging. Have you seen Greg KH's remarks
that he won't even look at Intel patches until they have internal
expert sign off?

> not encourage that behavior. Vendors should say I want to submit my code to
> the Linux kernel. Not eh, it's too much of a hassle and kernel people are
> jerks, so we'll just post it on our website.

There is a very, very fine line between "working with the community"
and "the community will provide free expert engineering work to our
staff".

> To be fair it is sent as RFC. So to me that says they know it's a ways off
> from being ready to be included and are starting the process early.

I'm not sure why it is RFC. It sounds like it is much more than this
if someone has already made a version with links to the NVIDIA GPU
driver and has reached a point where they care about ABI stablility?

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-21 17:21                                 ` Dennis Dalessandro
  2021-03-21 18:08                                   ` Jason Gunthorpe
@ 2021-03-21 19:19                                   ` Wan, Kaike
  2021-03-23 15:36                                   ` Christoph Hellwig
  2 siblings, 0 replies; 52+ messages in thread
From: Wan, Kaike @ 2021-03-21 19:19 UTC (permalink / raw)
  To: Dennis Dalessandro, Jason Gunthorpe
  Cc: Leon Romanovsky, Rimmer, Todd, dledford, linux-rdma

> From: Dennis Dalessandro <dennis.dalessandro@cornelisnetworks.com>
> Sent: Sunday, March 21, 2021 1:21 PM
> To: Jason Gunthorpe <jgg@nvidia.com>
> Cc: Leon Romanovsky <leon@kernel.org>; Rimmer, Todd
> <todd.rimmer@intel.com>; Wan, Kaike <kaike.wan@intel.com>;
> dledford@redhat.com; linux-rdma@vger.kernel.org
> Subject: Re: [PATCH RFC 0/9] A rendezvous module
> 
> On 3/21/2021 12:45 PM, Jason Gunthorpe wrote:
> > On Sun, Mar 21, 2021 at 12:24:39PM -0400, Dennis Dalessandro wrote:
> >> On 3/21/2021 4:56 AM, Leon Romanovsky wrote:
> >>> On Sat, Mar 20, 2021 at 12:39:46PM -0400, Dennis Dalessandro wrote:
> >>>> On 3/19/2021 6:57 PM, Rimmer, Todd wrote:
> >>>>>>> [Wan, Kaike] Incorrect. The rv module works with hfi1.
> >>>
> >>> <...>
> >>>
> >>
> >>> 2. Find another vendor that jumps on this RV bandwagon.
> >>
> >> That's not a valid argument. Clearly this works over multiple vendors HW.
> >
> > At a certain point we have to decide if this is a generic code of some
> > kind or a driver-specific thing like HFI has.
> >
> > There are also obvious technial problems designing it as a ULP, so it
> > is a very important question to answer. If it will only ever be used
> > by one Intel ethernet chip then maybe it isn't really generic code.
> 
> Todd/Kaike, is there something in here that is specific to the Intel ethernet
> chip?
[Wan, Kaike] No. The code is generic to Verbs/ROCE devices.


^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-21 18:08                                   ` Jason Gunthorpe
@ 2021-03-22 15:17                                     ` Rimmer, Todd
  2021-03-22 16:47                                       ` Jason Gunthorpe
  2021-03-22 17:31                                     ` Hefty, Sean
  1 sibling, 1 reply; 52+ messages in thread
From: Rimmer, Todd @ 2021-03-22 15:17 UTC (permalink / raw)
  To: Jason Gunthorpe, Dennis Dalessandro
  Cc: Leon Romanovsky, Wan, Kaike, dledford, linux-rdma, Rimmer, Todd

> Over a long time it has been proven that this methodology is a good way to effect business change to align with the community consensus development model - eventually the costs of being out of tree have bad ROI and companies align.

Agree.  The key question is when will nVidia upstream it's drivers so companies don't have to endure the resulting "bad ROI" of being forced to have unique out of tree solutions.

> > Putting a bunch of misaligned structures and random reserved fields
> > *is* garbage by the upstream standard and if I send that to Linus 
> > I'll get yelled at.
Let's not overexaggerate this.  The fields we're organized in a logical manner for end user consumption and understanding.  A couple resv fields were used for alignment.  I alluded to the fact we may have some ideas on how those fields or gaps could be used for GPU support in the future and this might help deal with the lack of upstream nVidia code and reduce the ROI challenges for other vendors which that causes.  The discussion seemed centered around the attach parameters, attach is executed once per process at job start.

Todd


^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-22 15:17                                     ` Rimmer, Todd
@ 2021-03-22 16:47                                       ` Jason Gunthorpe
  0 siblings, 0 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-22 16:47 UTC (permalink / raw)
  To: Rimmer, Todd
  Cc: Dennis Dalessandro, Leon Romanovsky, Wan, Kaike, dledford, linux-rdma

On Mon, Mar 22, 2021 at 03:17:05PM +0000, Rimmer, Todd wrote:

> > Over a long time it has been proven that this methodology is a
> > good way to effect business change to align with the community
> > consensus development model - eventually the costs of being out of
> > tree have bad ROI and companies align.
> 
> Agree.  The key question is when will nVidia upstream it's drivers
> so companies don't have to endure the resulting "bad ROI" of being
> forced to have unique out of tree solutions.

If you are working with NVIDIA GPU and having inefficiencies then you
need to take it through your buisness relationship, not here.

> > > Putting a bunch of misaligned structures and random reserved
> > > fields *is* garbage by the upstream standard and if I send that
> > > to Linus I'll get yelled at.

> Let's not overexaggerate this.  The fields we're organized in a
> logical manner for end user consumption and understanding.  A couple
> resv fields were used for alignment.  

No, a couple of resv fields were randomly added and *a lot* of other
mis-alignements were ignored. That is not our standard for ABI design.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-21 18:08                                   ` Jason Gunthorpe
  2021-03-22 15:17                                     ` Rimmer, Todd
@ 2021-03-22 17:31                                     ` Hefty, Sean
  2021-03-23 22:56                                       ` Jason Gunthorpe
  1 sibling, 1 reply; 52+ messages in thread
From: Hefty, Sean @ 2021-03-22 17:31 UTC (permalink / raw)
  To: Jason Gunthorpe, Dennis Dalessandro
  Cc: Leon Romanovsky, Rimmer, Todd, Wan, Kaike, dledford, linux-rdma

> > To be fair it is sent as RFC. So to me that says they know it's a ways off
> > from being ready to be included and are starting the process early.
> 
> I'm not sure why it is RFC. It sounds like it is much more than this
> if someone has already made a version with links to the NVIDIA GPU
> driver and has reached a point where they care about ABI stablility?

I can take some blame here.  A couple of us were asked to look at this module.  Because the functionality is intended to be device agnostic, we assumed there would be more scrutiny, and we weren't sure how acceptable some aspects would be (e.g. mr cache, ib cm data format).  Rather than debate this internally for months, rework the code, and still miss, we asked Kaike to post an RFC to get community feedback.  For *upstreaming* purposes it was intended as an RFC to gather feedback on the overall approach.  That should have been made clearer.

The direct feedback that Kaike is examining is the difference between this approach and RDS.

- Sean

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-19 22:57                       ` Rimmer, Todd
  2021-03-19 23:06                         ` Jason Gunthorpe
  2021-03-20 16:39                         ` Dennis Dalessandro
@ 2021-03-23 15:30                         ` Christoph Hellwig
  2021-03-23 15:46                           ` Jason Gunthorpe
  2 siblings, 1 reply; 52+ messages in thread
From: Christoph Hellwig @ 2021-03-23 15:30 UTC (permalink / raw)
  To: Rimmer, Todd
  Cc: Dennis Dalessandro, Wan, Kaike, Jason Gunthorpe, dledford, linux-rdma

On Fri, Mar 19, 2021 at 10:57:20PM +0000, Rimmer, Todd wrote:
> We'd like advise on a challenging situation.  Some customers desire NICs to support nVidia GPUs in some environments.
> Unfortunately the nVidia GPU drivers are not upstream, and have not been for years.  So we are forced to have both out of tree
> and upstream versions of the code.  We need the same applications to be able to work over both, so we would like the
> GPU enabled versions of the code to have the same ABI as the upstream code as this greatly simplifies things.
> We have removed all GPU specific code from the upstream submission, but used both the "alignment holes" and the "reserved"
> mechanisms to hold places for GPU specific fields which can't be upstreamed.

NVIDIA GPUs are supported by drivers/gpu/drm/nouveau/, and your are
encourage to support them just like all the other in-tree GPU drivers.
Not sure what support a network protocol would need for a specific GPU.
You're probably trying to do something amazingly stupid here instead of
relying on proper kernel subsystem use.

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-21 16:24                             ` Dennis Dalessandro
  2021-03-21 16:45                               ` Jason Gunthorpe
@ 2021-03-23 15:33                               ` Christoph Hellwig
  1 sibling, 0 replies; 52+ messages in thread
From: Christoph Hellwig @ 2021-03-23 15:33 UTC (permalink / raw)
  To: Dennis Dalessandro
  Cc: Leon Romanovsky, Rimmer, Todd, Wan, Kaike, Jason Gunthorpe,
	dledford, linux-rdma

On Sun, Mar 21, 2021 at 12:24:39PM -0400, Dennis Dalessandro wrote:
> Completely agree. However the GPU code from nvidia is not upstream. I don't
> see that issue getting resolved in this code review. Let's move on.

In which case you need to stop wasting everyones time with you piece of
crap code base, because it is completely and utterly irrelevant.


^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-21 16:45                               ` Jason Gunthorpe
  2021-03-21 17:21                                 ` Dennis Dalessandro
@ 2021-03-23 15:35                                 ` Christoph Hellwig
  1 sibling, 0 replies; 52+ messages in thread
From: Christoph Hellwig @ 2021-03-23 15:35 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Dennis Dalessandro, Leon Romanovsky, Rimmer, Todd, Wan, Kaike,
	dledford, linux-rdma

On Sun, Mar 21, 2021 at 01:45:04PM -0300, Jason Gunthorpe wrote:
> > We should be trying to get things upstream, not putting up walls when people
> > want to submit new drivers. Calling code "garbage" [1] is not productive.
> 
> Putting a bunch of misaligned structures and random reserved fields
> *is* garbage by the upstream standard and if I send that to Linus I'll
> get yelled at.

And when this is for out of tree crap it is per the very definition of
the word garbage.  If I was in your position I would not waste any time
on such an utterly disrespectful submission that violates all the
community rules.

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-21 17:21                                 ` Dennis Dalessandro
  2021-03-21 18:08                                   ` Jason Gunthorpe
  2021-03-21 19:19                                   ` Wan, Kaike
@ 2021-03-23 15:36                                   ` Christoph Hellwig
  2 siblings, 0 replies; 52+ messages in thread
From: Christoph Hellwig @ 2021-03-23 15:36 UTC (permalink / raw)
  To: Dennis Dalessandro
  Cc: Jason Gunthorpe, Leon Romanovsky, Rimmer, Todd, Wan, Kaike,
	dledford, linux-rdma

On Sun, Mar 21, 2021 at 01:21:14PM -0400, Dennis Dalessandro wrote:
> No one is suggesting to compromise the upstream world. There is a bigger
> picture here. The answer for this driver may just be take out the reserved
> stuff. That's pretty simple. The bigger question is how do we deal with
> non-upstream code. It can't be a problem unique to the RDMA subsystem. How
> do others deal with it?

By ignoring it, or if it gets too annoying by making life for it so
utterly painful that people give up on it.

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-23 15:30                         ` Christoph Hellwig
@ 2021-03-23 15:46                           ` Jason Gunthorpe
  2021-03-23 16:07                             ` Christoph Hellwig
  0 siblings, 1 reply; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-23 15:46 UTC (permalink / raw)
  To: Christoph Hellwig
  Cc: Rimmer, Todd, Dennis Dalessandro, Wan, Kaike, dledford, linux-rdma

On Tue, Mar 23, 2021 at 03:30:41PM +0000, Christoph Hellwig wrote:

> On Fri, Mar 19, 2021 at 10:57:20PM +0000, Rimmer, Todd wrote:

> > We'd like advise on a challenging situation.  Some customers
> > desire NICs to support nVidia GPUs in some environments.
> > Unfortunately the nVidia GPU drivers are not upstream, and have
> > not been for years.  So we are forced to have both out of tree and
> > upstream versions of the code.  We need the same applications to
> > be able to work over both, so we would like the GPU enabled
> > versions of the code to have the same ABI as the upstream code as
> > this greatly simplifies things.  We have removed all GPU specific
> > code from the upstream submission, but used both the "alignment
> > holes" and the "reserved" mechanisms to hold places for GPU
> > specific fields which can't be upstreamed.
> 
> NVIDIA GPUs are supported by drivers/gpu/drm/nouveau/, and your are
> encourage to support them just like all the other in-tree GPU
> drivers.  Not sure what support a network protocol would need for a
> specific GPU.  You're probably trying to do something amazingly
> stupid here instead of relying on proper kernel subsystem use.

The kernel building block for what they are trying to do with the GPU
is the recently merged DMABUF MR support in the RDMA subsystem.

I'd like to think that since Daniel's team at Intel got the DMABUF
stuff merged to support the applications Todd's Intel team is building
that this RV stuff is already fully ready for dmabuf... (hint hint)

What Todd is alluding to here is the hacky DMABUF alternative that is
in the NVIDIA GPU driver - which HPC networking companies must support
if they want to interwork with the NVIDIA GPU.

Every RDMA vendor playing in the HPC space has some out-of-tree driver
to enable this. :(

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-23 15:46                           ` Jason Gunthorpe
@ 2021-03-23 16:07                             ` Christoph Hellwig
  2021-03-23 17:25                               ` Rimmer, Todd
  0 siblings, 1 reply; 52+ messages in thread
From: Christoph Hellwig @ 2021-03-23 16:07 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Christoph Hellwig, Rimmer, Todd, Dennis Dalessandro, Wan, Kaike,
	dledford, linux-rdma

On Tue, Mar 23, 2021 at 12:46:26PM -0300, Jason Gunthorpe wrote:
> What Todd is alluding to here is the hacky DMABUF alternative that is
> in the NVIDIA GPU driver - which HPC networking companies must support
> if they want to interwork with the NVIDIA GPU.
> 
> Every RDMA vendor playing in the HPC space has some out-of-tree driver
> to enable this. :(

I can only recommende everone to buy from a less fucked up GPU or
accelerator vendor.  We're certainly going to do everything we can to
discourage making life easier for people who insist on doing this
stupidest possible thing.

^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-23 16:07                             ` Christoph Hellwig
@ 2021-03-23 17:25                               ` Rimmer, Todd
  2021-03-23 17:44                                 ` Jason Gunthorpe
  0 siblings, 1 reply; 52+ messages in thread
From: Rimmer, Todd @ 2021-03-23 17:25 UTC (permalink / raw)
  To: Christoph Hellwig, Jason Gunthorpe
  Cc: Dennis Dalessandro, Wan, Kaike, dledford, linux-rdma, Rimmer, Todd

> I can only recommende everone to buy from a less f***** up GPU or
> accelerator vendor.  
I would certainly love that.  This is not just a recent problem, it's been going on for at least 3-5 years with no end in sight.  And the nvidia driver itself is closed-source in the kernel :-(  Making tuning and debug even harder and continuing to add costs to NIC vendors other than nVidia themselves to support this.

Back to the topic at hand, yes, there are a few misalignments in the ABI.  Most of the structures are carefully aligned.  Below I summarize the major structures and their alignment characteristics. In a few places we chose readability for the application programmer by ordering fields in a logical order, such as for statistics.   

In one place a superfluous resv field was used (rv_query_params_out)  and when I alluded that might be able to be taken advantage in the future to enable a common ABI for GPUs, we went down this deep rat hole.

In studying all the related fields, in most cases if we shuffled everything for maximum packing, the structures would still end up being about the same size and this would all be for non-performance path ABIs.

Here is a summary:
13 structures all perfectly aligned with no gaps, plus 7 structures below.

rv_query_params_out - has one 4 byte reserved field to guarantee alignment for a u64 which follows.

rv_attach_params_in - organized logically as early fields influence how later fields are interpreted.  Fair number of fields, two 1 byte gaps and one 7 byte gap.  Shuffling this might save about 4-8 bytes tops

rv_cache_stats_params_out - ordered logically by statistics meanings.  Two 4 byte gaps could be solved by having a less logical order.  Of course, applications reporting these statistics will tend to do output in the original order, so packing this results in a harder to use ABI and more difficult code review for application writers wanting to make sure they report all stats but do so in a logical order.

rv_conn_get_stats_params_out - one 2 byte gap (so the 1 bytes field mimicking the input request can be 1st), three 4 byte gaps.  Same explanation as rv_cache_stats_params_out

rv_conn_create_params_in - one 4 byte gap, easy enough to swap

rv_post_write_params_out - one 3 byte gap.  Presented in logical order, shuffling would still yield the same size as compiler will round up size.

rv_event - carefully packed and aligned.  Had to make this work on a wide range of compilers with a 1 byte common field defining which part of union was relevant.  Could put the same field in all unions to get rid of packed attribute if that is preferred.  We found other similar examples like this in an older 4.18 kernel, cited one below.

It should be noted, there are existing examples with small gaps or reserved fields in the existing kernel and RDMA stack.  A few examples in ib_user_verbs.h include:

ib_uverbs_send_wr - 4 byte gap after ex for the rdma field in union.

ib_uverbs_flow_attr - 2 byte reserved field declared

ib_flow_spec - 2 byte gap after size field

rdma_netdev - 7 byte gap after port_num

./hw/mthca/mthca_eq.c - very similar use of packed for mthca_eqe to the one complained about in rv_event

Todd

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-23 17:25                               ` Rimmer, Todd
@ 2021-03-23 17:44                                 ` Jason Gunthorpe
  0 siblings, 0 replies; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-23 17:44 UTC (permalink / raw)
  To: Rimmer, Todd
  Cc: Christoph Hellwig, Dennis Dalessandro, Wan, Kaike, dledford, linux-rdma

On Tue, Mar 23, 2021 at 05:25:43PM +0000, Rimmer, Todd wrote:

> In studying all the related fields, in most cases if we shuffled
> everything for maximum packing, the structures would still end up
> being about the same size and this would all be for non-performance
> path ABIs.

It is not about size, it is about uABI compatability. Do not place
packing holes in uABI structures as it is *very complicated* to
guarentee those work across all the combinations of user/kernel we
have to support.

Every uABI structure should have explicit padding, it should be
organized to minimize padding, and "easy on the programmer" is not a
top concern.

You must ensure that all compilers generate the same structure byte
layout. On x86 this means a classic i386 compiler, a AMD64 x32 ABI,
and the normal AMD64 compiler.

> It should be noted, there are existing examples with small gaps or
> reserved fields in the existing kernel and RDMA stack.  A few
> examples in ib_user_verbs.h include:

Past mistakes do not excuse future errors. Most of those are not uABI
structs.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* Re: [PATCH RFC 0/9] A rendezvous module
  2021-03-22 17:31                                     ` Hefty, Sean
@ 2021-03-23 22:56                                       ` Jason Gunthorpe
  2021-03-23 23:29                                         ` Rimmer, Todd
  0 siblings, 1 reply; 52+ messages in thread
From: Jason Gunthorpe @ 2021-03-23 22:56 UTC (permalink / raw)
  To: Hefty, Sean
  Cc: Dennis Dalessandro, Leon Romanovsky, Rimmer, Todd, Wan, Kaike,
	dledford, linux-rdma

On Mon, Mar 22, 2021 at 05:31:07PM +0000, Hefty, Sean wrote:
> > > To be fair it is sent as RFC. So to me that says they know it's a ways off
> > > from being ready to be included and are starting the process early.
> > 
> > I'm not sure why it is RFC. It sounds like it is much more than this
> > if someone has already made a version with links to the NVIDIA GPU
> > driver and has reached a point where they care about ABI stablility?
> 
> I can take some blame here.  A couple of us were asked to look at
> this module.  Because the functionality is intended to be device
> agnostic, we assumed there would be more scrutiny, and we weren't
> sure how acceptable some aspects would be (e.g. mr cache, ib cm data
> format).  Rather than debate this internally for months, rework the
> code, and still miss, we asked Kaike to post an RFC to get community
> feedback.  For *upstreaming* purposes it was intended as an RFC to
> gather feedback on the overall approach.  That should have been made
> clearer.

Well, it is hard to even have that kind of conversation when all the
details are wrong. The way this interfaces with uverbs is *just
wrong*, it completely ignores the locking model for how disassociation
works.

Something like this, that is trying to closely integrate with uverbs,
really cannot exist without major surgery to uverbs. If you want to do
something along these lines the uverbs parts cannot be in a ULP.

The mr cache could be moved into some kind of new uverb, that could be
interesting if it is carefully designed.

The actual transport code.. That is going to be really hard. RDS
doesn't integrate with uverbs for a reason, the kernel side owns the
QPs and PDs.

How you create a QP owned by the kernel but linked to a PD owned by
uverbs is going to need very delicate and careful work to be somehow
compatible with our disassociation model.

Are you *sure* this needs to be in the kernel? You can't take a
context switch to a user process and use the shared verbs FD stuff
to create the de-duplicated QPs instead? It is a much simpler design.

Have you thought about an actual *shared verbs QP* ? We have a lot of
other shared objects right now, it is not such a big step. It does
require inter-process locking though - and that is non-trivial.

A shared QP with a kernel owned send/recv to avoid the locking issue
could also be a very interesting solution.

Jason

^ permalink raw reply	[flat|nested] 52+ messages in thread

* RE: [PATCH RFC 0/9] A rendezvous module
  2021-03-23 22:56                                       ` Jason Gunthorpe
@ 2021-03-23 23:29                                         ` Rimmer, Todd
  0 siblings, 0 replies; 52+ messages in thread
From: Rimmer, Todd @ 2021-03-23 23:29 UTC (permalink / raw)
  To: Jason Gunthorpe, Hefty, Sean
  Cc: Dennis Dalessandro, Leon Romanovsky, Wan, Kaike, dledford,
	linux-rdma, Rimmer, Todd

> How you create a QP owned by the kernel but linked to a PD owned by uverbs is going to need very delicate and careful work to be somehow compatible with our disassociation model.

The primary usage mode is as follows:
The RC QP, PD and MR are all in the kernel.  The buffer virtual address and len is supplied by the user process and then used to lookup a MR in the cache, upon miss, a kernel MR is created against the kernel PD.  There are separate MR caches per user process.

The IOs are initiated by the user, matched to a MR in the cache, then a RDMA Write w/Immed is posted on the kernel RC QP.

In concept, this is not unlike other kernel ULPs which perform direct data placement into user memory but use kernel QPs for connections to remote resources, such as various RDMA storage and filesystem ULPs.
The separation of the MR registration call from the IO allows the registration cost of a miss to be partially hidden behind the end to end RTS/CTS exchange which is occurring in user space.



There is a secondary usage mode where the MRs are cached, but created against a user PD and later used by the user process against QPs in the user.  We found that usage mode offered some slight latency advantages over the primary mode for tiny jobs, but suffered significant scalability issues.  Those latency advantages mainly manifested in microbenchmarks, but did help a few apps.


If it would simplify things, we could focus the discussion on the primary usage mode.  Conceptually, the secondary usage mode may be a good candidate for an extension to uverbs (some form of register MR w/caching API where register MR checks a cache and deregister MR merely decrements a reference count in the cache).

Todd


^ permalink raw reply	[flat|nested] 52+ messages in thread

end of thread, other threads:[~2021-03-23 23:30 UTC | newest]

Thread overview: 52+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-03-19 12:56 [PATCH RFC 0/9] A rendezvous module kaike.wan
2021-03-19 12:56 ` [PATCH RFC 1/9] RDMA/rv: Public interferce for the RDMA Rendezvous module kaike.wan
2021-03-19 16:00   ` Jason Gunthorpe
2021-03-19 18:42   ` kernel test robot
2021-03-19 12:56 ` [PATCH RFC 2/9] RDMA/rv: Add the internal header files kaike.wan
2021-03-19 16:02   ` Jason Gunthorpe
2021-03-19 12:56 ` [PATCH RFC 3/9] RDMA/rv: Add the rv module kaike.wan
2021-03-19 12:56 ` [PATCH RFC 4/9] RDMA/rv: Add functions for memory region cache kaike.wan
2021-03-19 12:56 ` [PATCH RFC 5/9] RDMA/rv: Add function to register/deregister memory region kaike.wan
2021-03-19 12:56 ` [PATCH RFC 6/9] RDMA/rv: Add connection management functions kaike.wan
2021-03-19 12:56 ` [PATCH RFC 7/9] RDMA/rv: Add functions for RDMA transactions kaike.wan
2021-03-19 12:56 ` [PATCH RFC 8/9] RDMA/rv: Add functions for file operations kaike.wan
2021-03-19 12:56 ` [PATCH RFC 9/9] RDMA/rv: Integrate the file operations into the rv module kaike.wan
2021-03-19 13:53 ` [PATCH RFC 0/9] A rendezvous module Jason Gunthorpe
2021-03-19 14:49   ` Wan, Kaike
2021-03-19 15:48     ` Jason Gunthorpe
2021-03-19 19:22       ` Dennis Dalessandro
2021-03-19 19:44         ` Jason Gunthorpe
2021-03-19 20:12           ` Rimmer, Todd
2021-03-19 20:26             ` Jason Gunthorpe
2021-03-19 20:46               ` Rimmer, Todd
2021-03-19 20:54                 ` Jason Gunthorpe
2021-03-19 20:59                   ` Wan, Kaike
2021-03-19 21:28                     ` Dennis Dalessandro
2021-03-19 21:58                       ` Wan, Kaike
2021-03-19 22:35                         ` Jason Gunthorpe
2021-03-19 22:57                       ` Rimmer, Todd
2021-03-19 23:06                         ` Jason Gunthorpe
2021-03-20 16:39                         ` Dennis Dalessandro
2021-03-21  8:56                           ` Leon Romanovsky
2021-03-21 16:24                             ` Dennis Dalessandro
2021-03-21 16:45                               ` Jason Gunthorpe
2021-03-21 17:21                                 ` Dennis Dalessandro
2021-03-21 18:08                                   ` Jason Gunthorpe
2021-03-22 15:17                                     ` Rimmer, Todd
2021-03-22 16:47                                       ` Jason Gunthorpe
2021-03-22 17:31                                     ` Hefty, Sean
2021-03-23 22:56                                       ` Jason Gunthorpe
2021-03-23 23:29                                         ` Rimmer, Todd
2021-03-21 19:19                                   ` Wan, Kaike
2021-03-23 15:36                                   ` Christoph Hellwig
2021-03-23 15:35                                 ` Christoph Hellwig
2021-03-23 15:33                               ` Christoph Hellwig
2021-03-23 15:30                         ` Christoph Hellwig
2021-03-23 15:46                           ` Jason Gunthorpe
2021-03-23 16:07                             ` Christoph Hellwig
2021-03-23 17:25                               ` Rimmer, Todd
2021-03-23 17:44                                 ` Jason Gunthorpe
2021-03-19 20:18           ` Dennis Dalessandro
2021-03-19 20:30             ` Jason Gunthorpe
2021-03-19 20:34       ` Hefty, Sean
2021-03-21 12:08         ` Jason Gunthorpe

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.