DPDK patches and discussions
 help / color / mirror / Atom feed
* [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
@ 2021-10-19 18:14 jerinj
  2021-10-19 18:14 ` [dpdk-dev] [RFC PATCH 1/1] dwa: introduce dataplane workload accelerator subsystem jerinj
                   ` (2 more replies)
  0 siblings, 3 replies; 20+ messages in thread
From: jerinj @ 2021-10-19 18:14 UTC (permalink / raw)
  To: dev
  Cc: thomas, ferruh.yigit, ajit.khaparde, aboyer, andrew.rybchenko,
	beilei.xing, bruce.richardson, chas3, chenbo.xia, ciara.loftus,
	dsinghrawat, ed.czeck, evgenys, grive, g.singh, zhouguoyang,
	haiyue.wang, hkalra, heinrich.kuhn, hemant.agrawal, hyonkim,
	igorch, irusskikh, jgrajcia, jasvinder.singh, jianwang, jiawenwu,
	jingjing.wu, johndale, john.miller, linville, keith.wiles,
	kirankumark, oulijun, lironh, longli, mw, spinler, matan,
	matt.peters, maxime.coquelin, mk, humin29, pnalla, ndabilpuram,
	qiming.yang, qi.z.zhang, radhac, rahul.lakkireddy, rmody,
	rosen.xu, sachin.saxena, skoteshwar, shshaikh, shaibran,
	shepard.siegel, asomalap, somnath.kotur, sthemmin,
	steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, mattias.ronnblom, ruifeng.wang, drc,
	konstantin.ananyev, olivier.matz, jay.jayatheerthan, asekhar,
	pbhagavatula, eagostini, Jerin Jacob

From: Jerin Jacob <jerinj@marvell.com>


Dataplane Workload Accelerator library
======================================

Definition of Dataplane Workload Accelerator
--------------------------------------------
Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
Network controllers and programmable data acceleration engines for
packet processing, cryptography, regex engines, baseband processing, etc. 
This allows DWA to offload  compute/packet processing/baseband/
cryptography-related workload from the host CPU to save the cost and power. 
Also to enable scaling the workload by adding DWAs to the Host CPU as needed.

Unlike other devices in DPDK, the DWA device is not fixed-function
due to the fact that it has CPUs and programmable HW accelerators.
This enables DWA personality/workload to be completely programmable.
Typical examples of DWA offloads are Flow/Session management,
Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.


Motivation for the new library
------------------------------
Even though, a lot of semiconductor vendors offers a different form of DWA,
such as DPU(often called Smart-NIC), GPU, IPU, XPU, etc., 
Due to the lack of standard APIs to "Define the workload" and
"Communication between HOST and DWA", it is difficult for DPDK
consumers to use them in a portable way across different DWA vendors
and enable it in cloud environments.


Contents of RFC 
------------------
This RFC attempts to define standard APIs for: 

1) Definition of Profiles corresponding to well defined workloads, which includes
   a set of TLV(Messages) as a request  and response scheme to define 
   the contract between host and DWA to offload a workload.
   (See lib/dwa/rte_dwa_profile_* header files)
2) Discovery of a DWAs capabilities (e.g. which specific workloads it can support) 
   in a vendor independent fashion. (See rte_dwa_dev_disc_profiles())
3) Attaching a set of profiles to a DWA device(See rte_dwa_dev_attach())
4) A communication framework between Host and DWA(See rte_dwa_ctrl_op() for
   control plane and rte_dwa_port_host_* for user plane)
5) Virtualization of DWA hardware and firmware (Use standard DPDK device/bus model) 
6) Enablement of administrative functions such as FW updates,
   resource partitioning in a DWA like items in global in
   nature that is applicable for all DWA device under the DWA.
   (See rte_dwa_profile_admin.h) 

Also, this RFC define the L3FWD profile to offload L3FWD workload to DWA.
This RFC defines an ethernet-style host port for Host to DWA communication.
Different host port types may be required to cover the large spectrum of DWA types as
transports like PCIe DMA, Shared Memory, or Ethernet are fundamentally different,
and optimal performance need host port specific APIs.

The framework does not force an abstract of different transport interfaces as
single API, instead, decouples TLV from the transport interface and focuses on 
defining the TLVs and leaving vendors to specify the host ports
specific to their DWA architecture.

  
Roadmap
-------
1) Address the comments for this RFC and enable the common code
2) SW drivers/infrastructure for `DWA` and `DWA device`
as two separate DPDK processes over `memif` DPDK ethdev driver for 
L3FWD offload. This is to enable the framework without any special HW.
3) Example DWA device application for L3FWD profile.
4) Marvell DWA Device drivers.
5) Based on community interest new profile can be added in the future.


DWA library framework
---------------------

DWA components:

                                                  +--> rte_dwa_port_host_*()
                                                  |  (User Plane traffic as TLV)
                                                  |
                 +----------------------+         |   +--------------------+
                 |                      |         |   | DPDK DWA Device[0] |
                 |  +----------------+  |  Host Port  | +----------------+ |
                 |  |                |  |<========+==>| |                | |
                 |  |   Profile 0    |  |             | |   Profile X    | |
                 |  |                |  |             | |                | |
  <=============>|  +----------------+  | Control Port| +----------------+ |
    DWA Port0    |  +----------------+  |<========+==>|                    |
                 |  |                |  |         |   +--------------------+
                 |  |   Profile 1    |  |         |
                 |  |                |  |         +--> rte_dwa_ctrl_op()
                 |  +----------------+  |         (Control Plane traffic as TLV)
  <=============>|      Dataplane       |
    DWA Port1    |      Workload        |
                 |      Accelerator     |             +---------- ---------+
                 |      (HW/FW/SW)      |             | DPDK DWA Device[N] |
                 |                      |  Host Port  | +----------------+ |
  <=============>|  +----------------+  |<===========>| |                | |
    DWA PortN    |  |                |  |             | |   Profile Y    | |
                 |  |    Profile N   |  |             | |           ^    | |
                 |  |                |  | Control Port| +-----------|----+ |
                 |  +-------|--------+  |<===========>|             |      |
                 |          |           |             +-------------|------+
                 +----------|-----------+                           |    
                            |                                       |    
                            +---------------------------------------+
                                                       ^
                                                       |
                                                       +--rte_dwa_dev_attach()


Dataplane Workload Accelerator: It is an abstract model. The model is
capable of offloading the dataplane workload from application via
DPDK API over host and control ports of a DWA device.
Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
Network controllers, and programmable data acceleration engines for
packet processing, cryptography, regex engines, base-band processing, etc. 
This allows DWA to offload compute/packet processing/base-band/cryptography-related
workload from the host CPU to save cost and power. Also, 
enable scaling the workload by adding DWAs to the host CPU as needed.

DWA device: A DWA can be sliced to N number of DPDK DWA device(s)
based on the resources available in DWA.
The DPDK API interface operates on the DPDK DWA device.
It is a representation of a set of resources in DWA.

TLV: TLV (tag-length-value) encoded data stream contain tag as
message ID, followed by message length, and finally the message payload.
The 32bit message ID consists of two parts, 16bit Tag and 16bit Subtag.
The tag represents ID of the group of the similar message,
whereas, subtag represents a message tag ID under the group.

Control Port: Used for transferring the control plane TLVs. Every DPDK
DWA device must have a control port. Only one outstanding TLV can be
processed via this port by a single DWA device. This makes the control
port suitable for the control plane.

Host Port: Used for transferring the user plane TLVs. 
Ethernet, PCIe DMA, Shared Memory, etc.are the example of 
different transport mechanisms abstracted under the host port.
The primary purpose of host port to decouple the user plane TLVs with
underneath transport mechanism differences.
Unlike control port, more than one outstanding TLVs can be processed by 
a single DWA device via this port.
This makes, the host port transfer to be in asynchronous nature,
to support large volumes and less latency user plane traffic.

DWA Port: Used for transferring data between the external source and DWA.
Ethernet, eCPRI are examples of DWA ports. Unlike host ports,
the host CPU is not involved in transferring the data to/from DWA ports.
These ports typically connected to the Network controller inside the 
DWA to transfer the traffic from the external source.

TLV direction: `Host to DWA` and `DWA to Host` are the directions
of TLV messages. The former one is specified as H2D, and the later one is 
specified as D2H. The H2D control TLVs, used for requesting DWA to perform 
specific action and D2H control TLVs are used to respond to the requested
actions. The H2D user plane messages are used for transferring data from the
host to the DWA. The D2H user plane messages are used for transferring 
data from the DWA to the host.

DWA device states: Following are the different states of a DWA device.
- READY: DWA Device is ready to attach the profile.
See rte_dwa_dev_disc_profiles() API to discover the profile.
- ATTACHED: DWA Device attached to one or more profiles.
See rte_dwa_dev_attach() API to attach the profile(s).
- STOPPED: Profile is in the stop state.
TLV type `TYPE_ATTACHED`and `TYPE_STOPPED` messages are valid in this state.
After rte_dwa_dev_attach() or explicitly invoking the rte_dwa_stop() API
brings device to this state.
- RUNNING: Invoking rte_dwa_start() brings the device to this state.
TLV type `TYPE_STARTED` and `TYPE_USER_PLANE` are valid in this state.
- DETACHED: Invoking rte_dwa_dev_detach() brings the device to this state.
The device and profile must be in the STOPPED state prior to
invoking the rte_dwa_dev_detach().
- CLOSED: Closed a stopped/detached DWA device.The device cannot be restarted!.
Invoking rte_dwa_dev_close() brings the device to this state.

TLV types: Following are the different TLV types
- TYPE_ATTACHED: Valid when the device is in `ATTACHED`, `STOPPED` and `RUNNING` state.
- TYPE_STOPPED: Valid when the device is in `STOPPED` state.
- TYPE_STARTED: Valid when the device is in `RUNNING` state.
- TYPE_USER_PLANE: Valid when the device is in `RUNNING` state and
used to transfer only user plane traffic.

Profile: Specifies a workload that dataplane workload accelerator 
process on behalf of a DPDK application through a DPDK DWA device.
A profile is expressed as a set of TLV messages for control plane and user plane
functions. Each TLV message must have Tag, SubTag, Direction, Type, Payload attributes.

Programming model: Typical application programming sequence is as follows,
1) In the EAL initialization phase, the DWA devices shall be probed,
   the application can query the number of available DWA devices with
   rte_dwa_dev_count() API.
2) Application discovers the available profile(s) in a DWA device using
   rte_dwa_dev_disc_profiles() API.
3) Application attaches one or more profile(s) to a DWA device using
   rte_dwa_dev_attach().
4) Once the profile is attached, The device shall be in the STOPPED state.
   Configure the profile(s) with `TYPE_ATTACHED`and `TYPE_STOPPED` 
   type TLVs using rte_dwa_ctrl_op() API.
5) Once the profile is configured, move the profile to the `RUNNING` state
   by invoking rte_dwa_start() API.
6) Once the profile is in running state and if it has user plane TLV,
   transfer those TLVs using rte_dwa_port_host_() API based on the available 
   host port for the given profile attached.
7) Application can change the dynamic configuration aspects in
   `RUNNING` state using rte_dwa_ctrl_op() API by issuing `TYPE_STARTED` type
   of TLV messages.
8) Finally, use rte_dwa_stop(), rte_dwa_dev_detach(), rte_dwa_dev_close()
   sequence for tear-down.


L3FWD profile
-------------

                             +-------------->--[1]--------------+
                             |                                  |
                 +-----------|----------+                       |
                 |           |          |                       |
                 |  +--------|-------+  |                       |
                 |  |                |  |                       |
                 |  | L3FWD Profile  |  |                       |
      \          |  |                |  |                       |
  <====\========>|  +----------------+  |                       |
    DWA \Port0   |     Lookup Table     |             +---------|----------+
         \       |  +----------------+  |             | DPDK DWA|Device[0] |
          \      |  | IP    | Dport  |  |  Host Port  | +-------|--------+ |
           \     |  +----------------+  |<===========>| |       |        | |
            +~[3]~~~|~~~~~~~|~~~~~~~~|~~~~~~~~~~~~~~~~~>|->L3FWD Profile | |
  <=============>|  +----------------+  |             | |                | |
    DWA Port1    |  |       |        |  | Control Port| +-|---------|----+ |
                 |  +----------------+  |<===========>|   |         |      |
    ~~~>~~[5]~~~~|~~|~~~+   |        |  |             +---|---------|------+
                 |  +---+------------+  |                 |         |
    ~~~<~~~~~~~~~|~~|~~~+   |        |<-|------[2]--------+         |
                 |  +----------------+<-|------[4]------------------+
                 |    Dataplane         |
  <=============>|    Workload          |
    DWA PortN    |    Accelerator       |
                 |    (HW/FW/SW)        |
                 +----------------------+


L3FWD profile offloads Layer-3 forwarding between the DWA Ethernet ports.

The above diagram depicts the profile and application programming sequence.
1) DWA device attaches the L3FWD profile using rte_dwa_dev_attach().
2) Configure the L3FWD profile:
a) The application requests L3FWD profile capabilities of the DWA
   by using RTE_DWA_STAG_PROFILE_L3FWD_H2D_INFO, On response,
   the RTE_DWA_STAG_PROFILE_L3FWD_D2H_INFO returns the lookup modes
   supported, max rules supported, and available host ports for this profile.	
b) The application configures a set of DWA ports to use a 
   lookup mode(EM, LPM, or FIB) via RTE_DWA_STAG_PROFILE_L3FWD_H2D_CONFIG.
c) The application configures a valid host port to receive exception packets.
3) The exception that is not matching forwarding table entry comes as
   RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS TLV to host. DWA stores the exception 
   packet send back destination ports after completing step (4).
4) Parse the exception packet and add rules to the FWD table using
   RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_ADD. If the application knows the rules beforehand,
   it can add the rules in step 2.
5) When DWA ports receive the matching flows in the lookup table, DWA forwards
   to DWA Ethernet ports without host CPU intervention.


Example application usage with L3FWD profile
--------------------------------------------
This example application is to demonstrate the programming model of DWA library.
This example omits the error checks to simply the application.

void 
dwa_profile_l3fwd_add_rule(rte_dwa_obj_t obj obj, struct rte_mbuf *mbuf)
{
	struct rte_dwa_profile_l3fwd_h2d_lookup_add *lookup;
	struct rte_dwa_tlv *h2d, *d2h;
	struct rte_ether_hdr *eth_hdr;
	struct rte_ipv4_hdr *ipv4_hdr;
	uint32_t id;
	size_t len;

	id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_LOOKUP_ADD);
	len = sizeof(struct rte_dwa_profile_l3fwd_h2d_config);
	h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);

	lookup = h2d->msg;
        /* Simply hardcode to IPv4 instead of looking for Packet type to simplify example */
	lookup->rule_type = RTE_DWA_PROFILE_L3FWD_RULE_TYPE_IPV4;
	lookup->v4_rule.prefix.depth = 24;

	eth_hdr = rte_pktmbuf_mtod(mbuf, struct rte_ether_hdr *);
	ipv4_hdr = (struct rte_ipv4_hdr *)(eth_hdr + 1);
	lookup->v4_rule.prefix.ip_dst = rte_be_to_cpu_32(ipv4_hdr->dst_addr);
	lookup->eth_port_dst = mbuf->port;

	rte_dwa_tlv_fill(h2d, id, len, h2d);
	d2h = rte_dwa_ctrl_op(obj, h2h);
	free(h2d);
	free(d2h);
}

void
dwa_profile_l3fwd_port_host_ethernet_worker(rte_dwa_obj_t obj, struct app_ctx *ctx)
{
	struct rte_dwa_profile_l3fwd_d2h_exception_pkts *msg;
	struct rte_dwa_tlv *tlv;
	uint16_t i, rc, nb_tlvs;
	struct rte_mbuf *mbuf;

	while (!ctx->done) {
		rc = rte_dwa_port_host_ethernet_rx(obj, 0, &tlv, 1);
		if (!rc)
			continue;

		/* Since L3FWD profile has only one User Plane TLV, Message must be 
	         * RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS message
        	 */
		msg = (struct rte_dwa_profile_l3fwd_d2h_exception_pkts *)tlv->msg;
		for (i = 0; i < msg->nb_pkts; i++) {
				mbuf = msg->pkts[i];
				/* Got a exception pkt from DWA, handle it by adding as new rule in
                                 * lookup table in DWA
				 */ 				
				dwa_profile_l3fwd_add_rule(obj, mbuf);
				/* Free the mbuf to pool */
				rte_pktmbuf_free(mbuf);
		}
		
		/* Done with TLV mbuf container, free it back */
		rte_mempool_ops_enqueue_bulk(ctx->tlv_pool, tlv, 1);
}

bool
dwa_port_host_ethernet_config(rte_dwa_obj_t obj, struct app_ctx *ctx)
{
	struct rte_dwa_tlv info_h2d, *info_d2h, *h2d = NULL, *d2h;
	struct rte_dwa_port_host_ethernet_d2h_info *info;
	int tlv_pool_element_sz;
	bool rc = false;
	size_t len;

	/* Get the Ethernet host port info */
	id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_INFO);
	rte_dwa_tlv_fill(&info_h2d, id, 0, NULL);
	info_d2h = rte_dwa_ctrl_op(obj, &info_h2d)

	info = rte_dwa_tlv_d2h_to_msg(info_d2h);
	if (info == NULL)
		goto fail;
	/* Need min one Rx queue to Receive exception traffic */ 
	if (info->nb_rx_queues == 0)
		goto fail;
	/* Done with message from DWA. Free back to implementation */
	free(obj, info_d2h);

	/* Allocate exception packet pool */
	ctx->pkt_pool = rte_pktmbuf_pool_create("exception pool", /* Name */
                                ctx->pkt_pool_depth, /* Number of elements*/
                                512, /* Cache size*/
                                0,
                                RTE_MBUF_DEFAULT_BUF_SIZE,
                                ctx->socket_id));


	tlv_pool_element_sz = DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ * sizeof(rte_mbuf *);
	tlv_pool_element_sz  += sizeof(rte_dwa_profile_l3fwd_d2h_exception_pkts);

	/* Allocate TLV pool for RTE_DWA_STLV_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS_PACKETS tag */
	ctx->tlv_pool = rte_mempool_create("TLV pool", /* mempool name */
                                ctx->tlv_pool_depth, /* Number of elements*/
                                tlv_pool_element_sz, /* Element size*/
                                512, /* cache size*/
                                0, NULL, NULL, NULL /* Obj constructor */, NULL,
                                ctx->socket_id, 0 /* flags *);


	/* Configure Ethernet host port */
	id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_CONFIG);
	len = sizeof(struct rte_dwa_port_host_ethernet_config);
	h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);

	cfg = h2d->msg;
	/* Update the Ethernet configuration parameters */
	cfg->nb_rx_queues = 1;
	cfg->nb_tx_queues = 0;
	cfg->max_burst = DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ;
	cfg->pkt_pool = ctx->pkt_pool;
	cfg->tlv_pool = ctx->tlv_pool;
	rte_dwa_tlv_fill(h2d, id, len, h2d);
	d2h = rte_dwa_ctrl_op(obj, h2d);
	if (d2h == NULL))
		goto fail;

	free(h2d);

	/* Configure Rx queue 0 receive expectation traffic */
	id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_QUEUE_CONFIG);
	len = sizeof(struct rte_dwa_port_host_ethernet_queue_config);
	h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);

	cfg = h2d->msg;
	cfg->id = 0; /* 0th Queue */
	cfg->enable= 1;
	cfg->is_tx = 0; /* Rx queue */
	cfg->depth = ctx->rx_queue_depth;
	rte_dwa_tlv_fill(h2d, id, len, h2d);
	d2h = rte_dwa_ctrl_op(obj, h2d);
	if (d2h == NULL))
		goto fail;

	free(h2d);

	return true;
fail:
	if (h2d)
		free(h2d);
	return rc;
}

bool
dwa_profile_l3fwd_config(rte_dwa_obj_t obj, struct app_ctx *ctx)
{
	struct rte_dwa_tlv info_h2d, *info_d2h = NULL, *h2d, *d2h = NULL;
	struct rte_dwa_port_dwa_ethernet_d2h_info *info;
	struct rte_dwa_profile_l3fwd_h2d_config *cfg;
	bool rc = false;
 	uint32_t id;
	size_t len;

	/* Get DWA Ethernet port info */ 
	id = RTE_DWA_TLV_MK_ID(PORT_DWA_ETHERNET, H2D_INFO);
	rte_dwa_tlv_fill(&info_h2d, id, 0, NULL);
	info_d2h = rte_dwa_ctrl_op(obj, &info_h2d);

	info = rte_dwa_tlv_d2h_to_msg(info_d2h);
	if (info == NULL)
		goto fail;
	
	/* Not found any DWA ethernet ports */
	if (info->nb_ports == 0)
		goto fail;

	/* Configure L3FWD profile */
	id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_CONFIG);
	len = sizeof(struct rte_dwa_profile_l3fwd_h2d_config) + (sizeof(uint16_t) * info->nb_ports);
	h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);

	cfg = h2d->msg;
	/* Update the L3FWD configuration parameters */
	cfg->mode = ctx->mode;
	/* Attach all DWA Ethernet ports onto L3FWD profile */
	cfg->nb_eth_ports = info->nb_ports;
	memcpy(cfg->eth_ports, info->avail_ports, sizeof(uint16_t) * info->nb_ports);

	rte_dwa_tlv_fill(h2d, id, len, h2d);
	d2h = rte_dwa_ctrl_op(obj, h2d);
	free(h2d);

	/* All good */
	rc = true;
fail:
	if (info_d2h)
		free(obj, info_d2h);
	if (d2h)
		free(obj, d2h);

	return rc;
}

bool
dwa_profile_l3fwd_has_capa(rte_dwa_obj_t obj, struct app_ctx *ctx)
{
	struct rte_dwa_profile_l3fwd_d2h_info *info;
	struct rte_dwa_tlv h2d, *d2h;
	bool found = false;
 	uint32_t id;

	/* Get L3FWD profile info */
	id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_INFO);
	rte_dwa_tlv_fill(&h2d, id, 0, NULL);
	d2h = rte_dwa_ctrl_op(obj, &h2d);

	info = rte_dwa_tlv_d2h_to_msg(d2h);
	/* Request failed */
	if (info == NULL)
		goto fail;
	/* Required lookup modes is not supported */
	if (!(info->modes_supported & ctx->mode))
		goto fail;

	/* Check profile supports HOST_ETHERNET port as this application
         * supports only host port as Ethernet
         */
	for (i = 0; i < info->nb_host_ports; i++) {
		if (info->host_ports[i] == RTE_DWA_TAG_PORT_HOST_ETHERNET); {
			found = true;
		}	
	}

	/* Done with response, Free the d2h memory allocated by implementation */
	free(obj, d2h);
fail:
	return found;
}


bool
dwa_has_profile(enum rte_dwa_tag_profile pf)
{
	enum rte_dwa_tlv_profile *pfs = NULL;
	bool found = false;
	int nb_pfs;

	/* Get the number of profiles on the DWA device */
	nb_pfs = rte_dwa_dev_disc_profiles(0, NULL);
	pfs = malloc(sizeof(enum rte_dwa_tag_profile)  * nb_pfs);
	/* Fetch all the profiles */
	nb_pfs = rte_dwa_dev_disc_profiles(0, pfs);

	/* Check the list has requested profile */
	for (i = 0; i < nb_pfs; i++) {
		if (pfs[i] == pf);
			found = true;
	}
	free(pfs);


	return found;
}


#include <rte_dwa.h>

#define DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ		32

struct app_ctx {
	bool done;
	struct rte_mempool *pkt_pool;
	struct rte_mempool *tlv_pool;
	enum rte_dwa_profile_l3fwd_lookup_mode mode;
	int socket_id;
	int pkt_pool_depth;	
	int tlv_pool_depth;
	int rx_queue_depth;
} __rte_cache_aligned;

int
main(int argc, char **argv)
{
	rte_dwa_obj_t obj = NULL;
	struct app_ctx ctx;
	int rc;
		
	/* Initialize EAL */      
	rc= rte_eal_init(argc, argv);
        if (rc < 0)
              rte_exit(EXIT_FAILURE, "Invalid EAL arguments\n");
        argc -= ret;
        argv += ret;


	memset(&ctx, 0, sizeof(ctx));
	/* Set application default values */
	ctx->mode = RTE_DWA_PROFILE_L3FWD_MODE_LPM;
	ctx->socket_id = SOCKET_ID_ANY;
	ctx->pkt_pool_depth = 10000;
	ctx->tlv_pool_depth = 10000;
	ctx->rx_queue_depth = 10000;

	/* Step 1: Check any DWA devices present  */
	rc = rte_dwa_dev_count();
	if (rc <= 0)
		rte_exit(EXIT_FAILURE, "Failed to find DWA devices\n");

	/* Step 2: Check DWA device has L3FWD profile or not */
	if (!dwa_has_profile(RTE_DWA_TAG_PROFILE_L3FWD))
		rte_exit(EXIT_FAILURE, "L3FWD profile not found\n");

	/*
 	 * Step 3: Now that, workload accelerator has L3FWD profile,
 	 * offload L3FWD workload to accelerator by attaching the profile
	 * to accelerator.
 	 */ 
	enum rte_dwa_tlv_profile profile[] = {RTE_DWA_TAG_PROFILE_L3FWD};
	obj = rte_dwa_dev_attach(0, "my_custom_accelerator_device", profile, 1).;

	/* Step 4: Check Attached L3FWD profile has required capability to proceed */
	if (!dwa_profile_l3fwd_has_capa(obj, &ctx))
		rte_exit(EXIT_FAILURE, "L3FWD profile does not have enough capability \n");

	/* Step 5: Configure l3fwd profile */ 
	if (!dwa_profile_l3fwd_config(obj, &ctx))
		rte_exit(EXIT_FAILURE, "L3FWD profile configure failed \n");

	/* Step 6: Configure ethernet host port to receive exception packets */
	if (!dwa_port_host_ethernet_config(obj, &ctx))
		rte_exit(EXIT_FAILURE, "L3FWD profile configure failed \n");

	/* Step 7 : Move DWA profiles to start state */
	rte_dwa_start(obj);

	/* Step 8: Handle expectation packets and add lookup rules for it */
	dwa_profile_l3fwd_port_host_ethernet_worker(obj, &ctx);

	/* Step 9: Clean up */
	rte_dwa_stop(obj);
	rte_dwa_dev_detach(0, obj);
	rte_dwa_dev_close(0);
	
	return 0;
}


Jerin Jacob (1):
  dwa: introduce dataplane workload accelerator subsystem

 doc/api/doxy-api-index.md            |  13 +
 doc/api/doxy-api.conf.in             |   1 +
 lib/dwa/dwa.c                        |   7 +
 lib/dwa/meson.build                  |  17 ++
 lib/dwa/rte_dwa.h                    | 184 +++++++++++++
 lib/dwa/rte_dwa_core.h               | 264 +++++++++++++++++++
 lib/dwa/rte_dwa_dev.h                | 154 +++++++++++
 lib/dwa/rte_dwa_port_dwa_ethernet.h  |  68 +++++
 lib/dwa/rte_dwa_port_host_ethernet.h | 178 +++++++++++++
 lib/dwa/rte_dwa_profile_admin.h      |  85 ++++++
 lib/dwa/rte_dwa_profile_l3fwd.h      | 378 +++++++++++++++++++++++++++
 lib/dwa/version.map                  |   3 +
 lib/meson.build                      |   1 +
 13 files changed, 1353 insertions(+)
 create mode 100644 lib/dwa/dwa.c
 create mode 100644 lib/dwa/meson.build
 create mode 100644 lib/dwa/rte_dwa.h
 create mode 100644 lib/dwa/rte_dwa_core.h
 create mode 100644 lib/dwa/rte_dwa_dev.h
 create mode 100644 lib/dwa/rte_dwa_port_dwa_ethernet.h
 create mode 100644 lib/dwa/rte_dwa_port_host_ethernet.h
 create mode 100644 lib/dwa/rte_dwa_profile_admin.h
 create mode 100644 lib/dwa/rte_dwa_profile_l3fwd.h
 create mode 100644 lib/dwa/version.map

-- 
2.33.1


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

* [dpdk-dev] [RFC PATCH 1/1] dwa: introduce dataplane workload accelerator subsystem
  2021-10-19 18:14 [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library jerinj
@ 2021-10-19 18:14 ` jerinj
  2021-10-19 19:08 ` [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library Thomas Monjalon
  2021-10-25  7:35 ` Mattias Rönnblom
  2 siblings, 0 replies; 20+ messages in thread
From: jerinj @ 2021-10-19 18:14 UTC (permalink / raw)
  To: dev, Ray Kinsella
  Cc: thomas, ferruh.yigit, ajit.khaparde, aboyer, andrew.rybchenko,
	beilei.xing, bruce.richardson, chas3, chenbo.xia, ciara.loftus,
	dsinghrawat, ed.czeck, evgenys, grive, g.singh, zhouguoyang,
	haiyue.wang, hkalra, heinrich.kuhn, hemant.agrawal, hyonkim,
	igorch, irusskikh, jgrajcia, jasvinder.singh, jianwang, jiawenwu,
	jingjing.wu, johndale, john.miller, linville, keith.wiles,
	kirankumark, oulijun, lironh, longli, mw, spinler, matan,
	matt.peters, maxime.coquelin, mk, humin29, pnalla, ndabilpuram,
	qiming.yang, qi.z.zhang, radhac, rahul.lakkireddy, rmody,
	rosen.xu, sachin.saxena, skoteshwar, shshaikh, shaibran,
	shepard.siegel, asomalap, somnath.kotur, sthemmin,
	steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, mattias.ronnblom, ruifeng.wang, drc,
	konstantin.ananyev, olivier.matz, jay.jayatheerthan, asekhar,
	pbhagavatula, eagostini, Jerin Jacob

From: Jerin Jacob <jerinj@marvell.com>

Signed-off-by: Jerin Jacob <jerinj@marvell.com>
---
 doc/api/doxy-api-index.md            |  13 +
 doc/api/doxy-api.conf.in             |   1 +
 lib/dwa/dwa.c                        |   7 +
 lib/dwa/meson.build                  |  17 ++
 lib/dwa/rte_dwa.h                    | 184 +++++++++++++
 lib/dwa/rte_dwa_core.h               | 264 +++++++++++++++++++
 lib/dwa/rte_dwa_dev.h                | 154 +++++++++++
 lib/dwa/rte_dwa_port_dwa_ethernet.h  |  68 +++++
 lib/dwa/rte_dwa_port_host_ethernet.h | 178 +++++++++++++
 lib/dwa/rte_dwa_profile_admin.h      |  85 ++++++
 lib/dwa/rte_dwa_profile_l3fwd.h      | 378 +++++++++++++++++++++++++++
 lib/dwa/version.map                  |   3 +
 lib/meson.build                      |   1 +
 13 files changed, 1353 insertions(+)
 create mode 100644 lib/dwa/dwa.c
 create mode 100644 lib/dwa/meson.build
 create mode 100644 lib/dwa/rte_dwa.h
 create mode 100644 lib/dwa/rte_dwa_core.h
 create mode 100644 lib/dwa/rte_dwa_dev.h
 create mode 100644 lib/dwa/rte_dwa_port_dwa_ethernet.h
 create mode 100644 lib/dwa/rte_dwa_port_host_ethernet.h
 create mode 100644 lib/dwa/rte_dwa_profile_admin.h
 create mode 100644 lib/dwa/rte_dwa_profile_l3fwd.h
 create mode 100644 lib/dwa/version.map

diff --git a/doc/api/doxy-api-index.md b/doc/api/doxy-api-index.md
index 2939050431..d55a506ff3 100644
--- a/doc/api/doxy-api-index.md
+++ b/doc/api/doxy-api-index.md
@@ -199,6 +199,19 @@ The public API headers are grouped by topics:
     [eth_node]         (@ref rte_node_eth_api.h),
     [ip4_node]         (@ref rte_node_ip4_api.h)
 
+- **dataplane workload accelerator**:
+  * infrastructure:
+    [dwa]              (@ref rte_dwa.h),
+    [core]             (@ref rte_dwa_core.h),
+    [device]           (@ref rte_dwa_dev.h)
+  * dwa ports:
+    [ethernet]         (@ref rte_dwa_port_dwa_ethernet.h)
+  * host ports:
+    [ethernet]         (@ref rte_dwa_port_host_ethernet.h)
+  * profile:
+    [admin]            (@ref rte_dwa_profile_admin.h),
+    [l3fwd]            (@ref rte_dwa_profile_l3fwd.h)
+
 - **basic**:
   [bitops]             (@ref rte_bitops.h),
   [approx fraction]    (@ref rte_approx.h),
diff --git a/doc/api/doxy-api.conf.in b/doc/api/doxy-api.conf.in
index 109ec1f682..4fa0916d06 100644
--- a/doc/api/doxy-api.conf.in
+++ b/doc/api/doxy-api.conf.in
@@ -36,6 +36,7 @@ INPUT                   = @TOPDIR@/doc/api/doxy-api-index.md \
                           @TOPDIR@/lib/cryptodev \
                           @TOPDIR@/lib/distributor \
                           @TOPDIR@/lib/dmadev \
+                          @TOPDIR@/lib/dwa \
                           @TOPDIR@/lib/efd \
                           @TOPDIR@/lib/ethdev \
                           @TOPDIR@/lib/eventdev \
diff --git a/lib/dwa/dwa.c b/lib/dwa/dwa.c
new file mode 100644
index 0000000000..5331d613e8
--- /dev/null
+++ b/lib/dwa/dwa.c
@@ -0,0 +1,7 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(C) 2021 Marvell.
+ */
+
+
+#include <rte_dwa.h>
+
diff --git a/lib/dwa/meson.build b/lib/dwa/meson.build
new file mode 100644
index 0000000000..88cc0f410f
--- /dev/null
+++ b/lib/dwa/meson.build
@@ -0,0 +1,17 @@
+# SPDX-License-Identifier: BSD-3-Clause
+# Copyright(C) 2020 Marvell International Ltd.
+
+sources = files(
+        'dwa.c',
+)
+headers = files(
+        'rte_dwa.h',
+        'rte_dwa_core.h',
+        'rte_dwa_dev.h',
+        'rte_dwa_port_dwa_ethernet.h',
+        'rte_dwa_port_host_ethernet.h',
+        'rte_dwa_profile_admin.h',
+        'rte_dwa_profile_l3fwd.h',
+)
+
+deps += ['eal']
diff --git a/lib/dwa/rte_dwa.h b/lib/dwa/rte_dwa.h
new file mode 100644
index 0000000000..608be80e6c
--- /dev/null
+++ b/lib/dwa/rte_dwa.h
@@ -0,0 +1,184 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(C) 2021 Marvell.
+ */
+
+#ifndef RTE_DWA_H
+#define RTE_DWA_H
+
+/**
+ * @file
+ *
+ * RTE DWA API
+ *
+ * DWA components:
+ *
+ * \code{.c}
+ *
+ *
+ *                                                 +--> rte_dwa_port_host_*()
+ *                                                |  (User Plane traffic as TLV)
+ *                                                |
+ *               +----------------------+         |   +--------------------+
+ *               |                      |         |   | DPDK DWA Device[0] |
+ *               |  +----------------+  |  Host Port  | +----------------+ |
+ *               |  |                |  |<========+==>| |                | |
+ *               |  |   Profile 0    |  |             | |   Profile X    | |
+ *               |  |                |  |             | |                | |
+ *<=============>|  +----------------+  | Control Port| +----------------+ |
+ *  DWA Port0    |  +----------------+  |<========+==>|                    |
+ *               |  |                |  |         |   +--------------------+
+ *               |  |   Profile 1    |  |         |
+ *               |  |                |  |         +--> rte_dwa_ctrl_op()
+ *               |  +----------------+  |         (Control Plane traffic as TLV)
+ *<=============>|      Dataplane       |
+ *  DWA Port1    |      Workload        |
+ *               |      Accelerator     |             +---------- ---------+
+ *               |      (HW/FW/SW)      |             | DPDK DWA Device[N] |
+ *               |                      |  Host Port  | +----------------+ |
+ *<=============>|  +----------------+  |<===========>| |                | |
+ *  DWA PortN    |  |                |  |             | |   Profile Y    | |
+ *               |  |    Profile N   |  |             | |           ^    | |
+ *               |  |                |  | Control Port| +-----------|----+ |
+ *               |  +-------|--------+  |<===========>|             |      |
+ *               |          |           |             +-------------|------+
+ *               +----------|-----------+                           |
+ *                          |                                       |
+ *                          +---------------------------------------+
+ *                                                     ^
+ *                                                     |
+ *                                                     +--rte_dwa_dev_attach()
+ *
+ * \endcode
+ *
+ * **Dataplane Workload Accelerator**: It is an abstract model. The model is
+ * capable of offloading the dataplane workload from application via
+ * DPDK API over host and control ports of a DWA device.
+ * Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
+ * Network controllers, and programmable data acceleration engines for
+ * packet processing, cryptography, regex engines, base-band processing, etc.
+ * This allows DWA to offload compute/packet processing/base-band/cryptography
+ * related workload from the host CPU to save cost and power. Also,
+ * enable scaling the workload by adding DWAs to the host CPU as needed.
+ *
+ * **DWA device**: A DWA can be sliced to N number of DPDK DWA device(s)
+ * based on the resources available in DWA.
+ * The DPDK API interface operates on the DPDK DWA device.
+ * It is a representation of a set of resources in DWA.
+ *
+ * **TLV**: TLV (tag-length-value) encoded data stream contain tag as
+ * message ID, followed by message length, and finally the message payload.
+ * The 32bit message ID consists of two parts, 16bit Tag and 16bit Subtag.
+ * The tag represents ID of the group of the similar message,
+ * whereas, subtag represents a message tag ID under the group.
+
+ * **Control Port**: Used for transferring the control plane TLVs.
+ * Every DPDK  DWA device must have a control port.
+ * Only one outstanding TLV can be processed via this port by
+ * a single DWA device.
+ * This makes the control port suitable for the control plane.
+
+ * **Host Port**: Used for transferring the user plane TLVs.
+ * Ethernet, PCIe DMA, Shared Memory, etc.are the example of
+ * different transport mechanisms abstracted under the host port.
+ * The primary purpose of host port to decouple the user plane TLVs with
+ * underneath transport mechanism differences.
+ * Unlike control port, more than one outstanding TLVs can be processed by
+ * a single DWA device via this port.
+ * This makes, the host port transfer to be in asynchronous nature,
+ * to support large volumes and less latency user plane traffic.
+
+ * **DWA Port**: Used for transferring data between the external source and DWA.
+ * Ethernet, eCPRI are examples of DWA ports. Unlike host ports,
+ * the host CPU is not involved in transferring the data to/from DWA ports.
+ * These ports typically connected to the Network controller inside the
+ * DWA to transfer the traffic from the external source.
+ *
+ * **TLV direction**: `Host to DWA` and `DWA to Host` are the directions
+ * of TLV messages. The former one is specified as `H2D`, and the later one is
+ * specified as `D2H`. The `H2D` control TLVs, used for requesting DWA to
+ * perform specific action and `D2H` control TLVs are used to respond to the
+ * requested actions. The `H2D` user plane messages are used for transferring
+ * data from the host to the DWA. The `D2H` user plane messages are used for
+ * transferring data from the DWA to the host.
+ *
+ * **DWA device states**: Following are the different states of a DWA device.
+ * - `READY`: DWA Device is ready to attach the profile.
+ * See rte_dwa_dev_disc_profiles() API to discover the profile.
+ * - `ATTACHED`: DWA Device attached to one or more profiles.
+ * See rte_dwa_dev_attach() API to attach the profile(s).
+ * - `STOPPED`: Profile is in the stop state.
+ * TLV type `TYPE_ATTACHED` and `TYPE_STOPPED` messages are valid in this state.
+ * After rte_dwa_dev_attach() or explicitly invoking the rte_dwa_stop() API
+ * brings device to this state.
+ * - `RUNNING`: Invoking rte_dwa_start() brings the device to this state.
+ * TLV type `TYPE_STARTED` and `TYPE_USER_PLANE` are valid in this state.
+ * - `DETACHED`: Invoking rte_dwa_dev_detach() brings the device to this state
+ * The device and profile must be in the `STOPPED` state prior to invoking the
+ * rte_dwa_dev_reattach().
+ * - `CLOSED`: Closed a stopped/detached DWA device.
+ *  The device cannot be restarted. Invoking rte_dwa_dev_close() brings the
+ *  device to this state.
+ *
+ * **TLV types**: Following are the different TLV types
+ * - `TYPE_ATTACHED`: Valid when the device is in `ATTACHED`, `STOPPED` and
+ *   `RUNNING` state.
+ * - `TYPE_STOPPED`: Valid when the device is in `STOPPED` state.
+ * - `TYPE_STARTED`: Valid when the device is in `RUNNING` state.
+ * - `TYPE_USER_PLANE`: Valid when the device is in `RUNNING` state and used to
+ *   transfer only user plane traffic.
+ *
+ * **Profile**: Specifies a workload that dataplane workload accelerator
+ * process on behalf of a DPDK application through a DPDK DWA device.
+ * A profile is expressed as a set of TLV messages for control plane and
+ * user plane functions.
+ * Each TLV message must have Tag, SubTag, Direction, Type, Payload attributes.
+ *
+ * **Programming model**: Typical application programming sequence is as follows
+ * -# In the EAL initialization phase, the DWA devices shall be probed,
+ * the application can query the number of available DWA devices
+ * using rte_dwa_dev_count() API.
+ * -# Application discovers the available profile(s) in a DWA device using
+ * rte_dwa_dev_disc_profiles() API.
+ * -# Application attaches one or more profile(s) to a DWA device using
+ * rte_dwa_dev_attach().
+ * -# Once the profile is attached, The device shall be in the `STOPPED` state.
+ * Configure the profile(s) with `TYPE_ATTACHED` and `TYPE_STOPPED` type TLVs
+ * using rte_dwa_ctrl_op() API.
+ * -# Once the profile is configured, move the profile to the `RUNNING` state
+ * by invoking rte_dwa_start() API.
+ * -# Once the profile is in running state and if it has user plane TLV,
+ * transfer those TLVs using rte_dwa_port_host_() API based on the available
+ * host port for the given profile attached.
+ * -# Application can change the dynamic configuration aspects in
+ * `RUNNING` state using rte_dwa_ctrl_op() API by issuing
+ * `TYPE_STARTED` type of TLV messages.
+ * -# Finally, use rte_dwa_stop(), rte_dwa_dev_detach(), rte_dwa_dev_close()
+ * sequence for tear-down.
+ *
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+/* Core */
+#include <rte_dwa_core.h>
+
+/* DWA Device */
+#include <rte_dwa_dev.h>
+
+/* DWA Ports */
+#include <rte_dwa_port_dwa_ethernet.h>
+
+/* Host ports */
+#include <rte_dwa_port_host_ethernet.h>
+
+/* Profiles */
+#include <rte_dwa_profile_admin.h>
+#include <rte_dwa_profile_l3fwd.h>
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* RTE_DWA_H */
diff --git a/lib/dwa/rte_dwa_core.h b/lib/dwa/rte_dwa_core.h
new file mode 100644
index 0000000000..3637a843dc
--- /dev/null
+++ b/lib/dwa/rte_dwa_core.h
@@ -0,0 +1,264 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(C) 2021 Marvell.
+ */
+
+#ifndef RTE_DWA_CORE_H
+#define RTE_DWA_CORE_H
+
+/**
+ * @file
+ *
+ * @warning
+ * @b EXPERIMENTAL:
+ * All functions in this file may be changed or removed without prior notice.
+ *
+ * RTE DWA core API
+ *
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <stdint.h>
+
+#include <rte_common.h>
+#include <rte_memcpy.h>
+
+/* Types */
+typedef void *rte_dwa_obj_t; /**< DWA object type. */
+
+/* Tag partitions */
+/**
+ * Enumerates different tag sections.
+ */
+enum rte_dwa_tag {
+	RTE_DWA_TAG_COMMON, /**< Tags common for all profile and ports. */
+	RTE_DWA_TAG_PORT_DWA_BASE = 0x400, /**< Base tag value for DWA port. */
+	RTE_DWA_TAG_PORT_HOST_BASE = 0x800, /**< Base tag value for host port */
+	RTE_DWA_TAG_VENDOR_EXTENSION = 0x1000,
+	/**< Base tag value for vendor extension TLVs. */
+	RTE_DWA_TAG_PROFILE_BASE = 0x3000, /**< Base tag value for profile. */
+	RTE_DWA_TAG_MAX = UINT16_MAX, /* Max available tags space. */
+};
+
+/**
+ * Enumerates DWA port types.
+ */
+enum rte_dwa_tag_port_dwa {
+	RTE_DWA_TAG_PORT_DWA_ETHERNET = RTE_DWA_TAG_PORT_DWA_BASE,
+	/**< Tag value for DWA ethernet port. */
+};
+
+/**
+ * Enumerates host port types.
+ */
+enum rte_dwa_tag_port_host {
+	RTE_DWA_TAG_PORT_HOST_ETHERNET = RTE_DWA_TAG_PORT_HOST_BASE,
+	/**< Tag value for host ethernet port. */
+};
+
+/**
+ * Enumerates profiles types.
+ */
+enum rte_dwa_tag_profile {
+	RTE_DWA_TAG_PROFILE_ADMIN = RTE_DWA_TAG_PROFILE_BASE,
+	/**< Tag value for admin profile. */
+	RTE_DWA_TAG_PROFILE_L3FWD,
+	/**< Tag value for l3fwd profile. */
+};
+
+/* Common sub tags */
+#define RTE_DWA_ERROR_STR_LEN_MAX 128 /**< Max error string length.*/
+
+/**
+ * Payload of RTE_DWA_STAG_COMMON_D2H_ERR message.
+ */
+struct rte_dwa_common_d2h_err {
+	int32_t dwa_errno; /**< Error number of the failure */
+	uint8_t has_reason_str; /**< If set, valid string in reason */
+	char reason[RTE_DWA_ERROR_STR_LEN_MAX]; /**< Failure reason as string */
+} __rte_packed;
+
+/**
+ * Enumerates the stag list for RTE_DWA_TAG_COMMON tag.
+ */
+enum rte_dwa_stag_common {
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_COMMON
+	 * Stag      | RTE_DWA_STAG_COMMON_D2H_SUCCESS
+	 * Direction | D2H
+	 * Type      | TYPE_ATTACHED
+	 * Payload   | NA
+	 * Pair TLV  | NA
+	 *
+	 * D2H response for successful TLV action.
+	 */
+	RTE_DWA_STAG_COMMON_D2H_SUCCESS,
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_COMMON
+	 * Stag      | RTE_DWA_STAG_COMMON_D2H_ERR
+	 * Direction | D2H
+	 * Type      | TYPE_ATTACHED
+	 * Payload   | struct rte_dwa_common_d2h_err
+	 * Pair TLV  | NA
+	 *
+	 * D2H response for unsuccessful TLV action.
+	 */
+	RTE_DWA_STAG_COMMON_D2H_ERR,
+	RTE_DWA_STAG_COMMON_MAX = UINT16_MAX, /**< Max stags for common tag.*/
+};
+
+/* TLV */
+/** Macro to get TLV ID from tag and stag. */
+#define RTE_DWA_TLV_ID(tag, stag) ((uint32_t)((uint32_t)(tag) << 16 | (stag)))
+/** Macro to form TLV ID from tag and stag. */
+#define RTE_DWA_TLV_MK_ID(tag, stag) \
+	RTE_DWA_TLV_ID(RTE_DWA_TAG_ ## tag, RTE_DWA_STAG_ ## tag ##_ ## stag)
+
+/** TLV representation of the DWA message */
+struct rte_dwa_tlv {
+	RTE_STD_C11
+	union {
+		uint32_t id; /**< ID as tag and stag tuple. */
+		RTE_STD_C11
+		union {
+			uint16_t tag; /**< Tag. */
+			uint16_t stag;/**< Sub Tag. */
+		};
+	};
+	uint32_t len; /**< Length of payload. */
+	char  msg[]; /**< Zero length array points to payload of TLV. */
+} __rte_packed;
+
+/** DWA TLV header size */
+#define RTE_DWA_TLV_HDR_SZ offsetof(struct rte_dwa_tlv, msg)
+
+/**
+ * Fill DWA TLV.
+ *
+ * Fill the DWA TLV attributes from arguments.
+ *
+ * @param [out] tlv
+ *   TLV pointer to fill.
+ * @param id
+ *   TLV ID value. @see RTE_DWA_TLV_MK_ID RTE_DWA_TLV_ID
+ * @param len
+ *   TLV payload length.
+ * @param msg
+ *   TLV payload message.
+ */
+static inline void
+rte_dwa_tlv_fill(struct rte_dwa_tlv *tlv, uint32_t id, uint32_t len, void *msg)
+{
+	tlv->id = id;
+	tlv->len = len;
+	if (len)
+		rte_memcpy(tlv->msg, msg, len);
+}
+
+/**
+ * Get payload of a D2H TLV message.
+ *
+ * Get payload message pointer from the D2H TLV message.
+ *
+ * @param tlv
+ *   TLV pointer.
+ *
+ * @return
+ *   TLV payload on success, NULL otherwise.
+ */
+static inline void*
+rte_dwa_tlv_d2h_to_msg(struct rte_dwa_tlv *tlv)
+{
+	if (tlv == NULL || tlv->id == RTE_DWA_TLV_MK_ID(COMMON, D2H_ERR))
+		return NULL;
+	else
+		return tlv->msg;
+}
+
+/**
+ * Get TLV name from ID.
+ *
+ * Get TLV name as string from ID.
+ *
+ * @param id
+ *   TLV ID.
+ *
+ * @return
+ *   TLV name string on success, NULL otherwise.
+ */
+char *rte_dwa_tlv_id_to_str(uint32_t id);
+
+/**
+ * Get TLV payload length.
+ *
+ * Get TLV payload length from the given TLV ID.
+ *
+ * @param id
+ *   TLV ID.
+ *
+ * @return
+ *   >=0 On success, <0 on invalid ID.
+ */
+int32_t rte_dwa_tlv_len(uint32_t id);
+
+/* Control plane operation */
+
+/**
+ * Execute a control plane operation on DWA.
+ *
+ * @param obj
+ *   DWA object.
+ *
+ * @param h2d
+ *   H2D direction TLV to execute a control plane operation on DWA.
+ *
+ * @return
+ *   D2H TLV response on success, NULL otherwise.
+ *   Implementation allocates the memory for the response using malloc(),
+ *   Application must free this memory using free() API after use.
+ */
+struct rte_dwa_tlv *rte_dwa_ctrl_op(rte_dwa_obj_t obj, struct rte_dwa_tlv *h2d);
+
+/* State management */
+
+/**
+ * Move DWA device to `RUNNING` state.
+ *
+ * Move DWA device and its associated profiles to `RUNNING` state.
+ * `TYPE_ATTACHED`, `TYPE_STARTED` and `TYPE_USER_PLANE` type messages are
+ * valid in this state.
+ *
+ * @param obj
+ *   DWA object.
+ *
+ * @return
+ *   0 on success, error otherwise.
+ */
+
+int rte_dwa_start(rte_dwa_obj_t obj);
+
+/**
+ * Move DWA device to `STOPPED` state.
+ *
+ * Move DWA device and its associated profiles to `STOPPED` state.
+ * `TYPE_ATTACHED`, `TYPE_STOPPED` type messages are valid in this state.
+ *
+ * @param obj
+ *   DWA object.
+ *
+ * @return
+ *   0 on success, error otherwise.
+ */
+int rte_dwa_stop(rte_dwa_obj_t obj);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* RTE_DWA_CORE_H */
diff --git a/lib/dwa/rte_dwa_dev.h b/lib/dwa/rte_dwa_dev.h
new file mode 100644
index 0000000000..7dff9be5b9
--- /dev/null
+++ b/lib/dwa/rte_dwa_dev.h
@@ -0,0 +1,154 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(C) 2021 Marvell.
+ */
+
+#ifndef RTE_DWA_DEV_H
+#define RTE_DWA_DEV_H
+
+/**
+ * @file
+ *
+ * @warning
+ * @b EXPERIMENTAL:
+ * All functions in this file may be changed or removed without prior notice.
+ *
+ * RTE DWA Device API
+ *
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <stdbool.h>
+
+#include <rte_common.h>
+
+/* Device utils */
+
+/**
+ * Check if dev_id of DWA device is valid.
+ *
+ * @param dev_id
+ *   DWA device id.
+ *
+ * @return
+ *   true if device is valid, false otherwise.
+ */
+bool
+rte_dwa_dev_is_valid(uint16_t dev_id);
+
+/**
+ * Get the total number of DWA devices that have been successfully
+ * initialised.
+ *
+ * @return
+ *   The total number of usable DWA devices.
+ */
+uint16_t
+rte_dwa_dev_count(void);
+
+/* Discovery */
+
+/**
+ * Get the list of profiles available in a DWA device.
+ *
+ * @param dev_id
+ *   DWA device id.
+ *
+ * @param [out] pfs
+ *   An rte_dwa_tag_profile array to be filled with available profiles.
+ *   If set to NULL, this function returns the number of available profiles.
+ *
+ * @return
+ *   Number of profiles available.
+ */
+int
+rte_dwa_dev_disc_profiles(uint16_t dev_id, enum rte_dwa_tag_profile *pfs);
+
+/* Attach */
+
+/**
+ * Attach a list of profiles on a DWA device.
+ *
+ * Upon successful attach operation, devices moves to `ATTACHED` state.
+ * `TYPE_ATTACHED` and `TYPE_STOPPED` types of TLV message are valid
+ * in this state.
+ *
+ * @param dev_id
+ *   DWA device id.
+ *
+ * @param name
+ *   Unique name for getting DWA object on secondary process.
+ *   @see rte_dwa_dev_lookup()
+ * @param pfs
+ *   An array profiles as enum rte_dwa_tag_profile to be attached.
+ * @param nb_pfs
+ *   Number of profiles to be attached.
+ *
+ * @return
+ *   DWA object.
+ */
+rte_dwa_obj_t
+rte_dwa_dev_attach(uint16_t dev_id, const char *name,
+		   enum rte_dwa_tag_profile pfs[], uint16_t nb_pfs);
+
+/* Lookup */
+
+/**
+ * Search the DWA object from its name.
+ *
+ * @param dev_id
+ *   DWA device id.
+ *
+ * @param name
+ *   The name provided in rte_dwa_dev_attach().
+ *
+ * @return
+ *   DWA object or NULL if not found.
+ */
+rte_dwa_obj_t
+rte_dwa_dev_lookup(uint16_t dev_id, const char *name);
+
+/* Detach */
+
+/**
+ * Detach all profiles from the given DWA device.
+ *
+ * Detach all profile and move the DWA device to `DETACHED` state.
+ * The device and profile must be in the `STOPPED` state prior to invoking the
+ * this API. @see rte_dwa_stop()
+ *
+ * @param dev_id
+ *   DWA device id.
+
+ * @param obj
+ *   DWA object.
+ *
+ * @return
+ *   0 on success, error otherwise.
+ */
+int
+rte_dwa_dev_detach(uint16_t dev_id, rte_dwa_obj_t obj);
+
+/* Close */
+
+/**
+ * Close a DWA device. The device cannot be restarted!
+ * The device and profile must be in the `DETACHED` state prior to invoking
+ * this API. @see rte_dwa_dev_detach()
+ *
+ * @param dev_id
+ *   DWA device id.
+ *
+ * @return
+ *   0 on success, error otherwise.
+ */
+int
+rte_dwa_dev_close(uint16_t dev_id);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* RTE_DWA_DEV_H */
diff --git a/lib/dwa/rte_dwa_port_dwa_ethernet.h b/lib/dwa/rte_dwa_port_dwa_ethernet.h
new file mode 100644
index 0000000000..5ada6d0335
--- /dev/null
+++ b/lib/dwa/rte_dwa_port_dwa_ethernet.h
@@ -0,0 +1,68 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(C) 2021 Marvell.
+ */
+
+#ifndef RTE_DWA_PORT_DWA_ETHERNET_H
+#define RTE_DWA_PORT_DWA_ETHERNET_H
+
+/**
+ * @file
+ *
+ * @warning
+ * @b EXPERIMENTAL:
+ * All functions in this file may be changed or removed without prior notice.
+ *
+ * RTE API related to DWA ethernet based port.
+ *
+ */
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+/**
+ * Payload of RTE_DWA_STAG_PORT_DWA_ETHERNET_D2H_INFO message.
+ */
+struct rte_dwa_port_dwa_ethernet_d2h_info {
+	uint16_t nb_ports; /**< Number of available ports. */
+	uint16_t avail_ports[]; /**< Array of available port of size nb_ports */
+} __rte_packed;
+
+/**
+ * Enumerates the stag list for RTE_DWA_TAG_PORT_DWA_ETHERNET tag.
+ */
+enum rte_dwa_stag_port_dwa_ethernet {
+	/**
+	 * Attribute |  Value
+	 * ----------|---------
+	 * Tag       | RTE_DWA_TAG_PORT_DWA_ETHERNET
+	 * Stag      | RTE_DWA_STAG_PORT_DWA_ETHERNET_H2D_INFO
+	 * Direction | H2D
+	 * Type      | TYPE_ATTACHED
+	 * Payload   | NA
+	 * Pair TLV  | RTE_DWA_STAG_PORT_DWA_ETHERNET_D2H_INFO
+	 *
+	 * Request DWA ethernet port information.
+	 */
+	RTE_DWA_STAG_PORT_DWA_ETHERNET_H2D_INFO,
+	/**
+	 * Attribute |  Value
+	 * ----------|---------
+	 * Tag       | RTE_DWA_TAG_PORT_DWA_ETHERNET
+	 * Stag      | RTE_DWA_STAG_PORT_DWA_ETHERNET_D2H_INFO
+	 * Direction | D2H
+	 * Type      | TYPE_ATTACHED
+	 * Payload   | struct rte_dwa_port_dwa_ethernet_d2h_info
+	 * Pair TLV  | RTE_DWA_STAG_PORT_DWA_ETHERNET_H2D_INFO
+	 *
+	 * Response for DWA ethernet port information.
+	 */
+	RTE_DWA_STAG_PORT_DWA_ETHERNET_D2H_INFO,
+	RTE_DWA_STAG_PORT_DWA_ETHERNET_MAX = UINT16_MAX,
+	/**< Max stags for RTE_DWA_TAG_PORT_DWA_ETHERNET tag*/
+};
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* RTE_DWA_PORT_DWA_ETHERNET_H */
diff --git a/lib/dwa/rte_dwa_port_host_ethernet.h b/lib/dwa/rte_dwa_port_host_ethernet.h
new file mode 100644
index 0000000000..afcd5a9fd5
--- /dev/null
+++ b/lib/dwa/rte_dwa_port_host_ethernet.h
@@ -0,0 +1,178 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(C) 2021 Marvell.
+ */
+
+#ifndef RTE_DWA_PORT_HOST_ETHERNET_H
+#define RTE_DWA_PORT_HOST_ETHERNET_H
+
+/**
+ * @file
+ *
+ * @warning
+ * @b EXPERIMENTAL:
+ * All functions in this file may be changed or removed without prior notice.
+ *
+ * RTE API related to host ethernet based port.
+ *
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+/**
+ * Payload of RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO message.
+ */
+struct rte_dwa_port_host_ethernet_d2h_info {
+	uint16_t nb_rx_queues; /**< Number of Rx queues available */
+	uint16_t nb_tx_queues; /**< Number of Tx queues available */
+} __rte_packed;
+
+/**
+ * Payload of RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_CONFIG message.
+ */
+struct rte_dwa_port_host_ethernet_config {
+	uint16_t nb_rx_queues; /**< Number of Rx queues to configure */
+	uint16_t nb_tx_queues; /**< Number of Tx queues to configure */
+	uint16_t max_burst; /**< Max burst size */
+	RTE_STD_C11
+	union {
+		struct rte_mempool *pkt_pool;
+		/**< Packet pool to allocate packets */
+		uint64_t pkt_pool_u64;
+		/**< uint64_t representation of packet pool */
+	};
+	RTE_STD_C11
+	union {
+		struct rte_mempool *tlv_pool;
+		/**< TLV pool to allocate TLVs */
+		uint64_t tlv_pool_u64;
+		/**< uint64_t representation of TLV pool */
+	};
+} __rte_packed;
+
+/**
+ * Payload of RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_QUEUE_CONFIG message.
+ */
+struct rte_dwa_port_host_ethernet_queue_config {
+	uint16_t id;
+	uint8_t enable;
+	uint8_t is_tx;
+	uint16_t depth;
+} __rte_packed;
+
+/**
+ * Enumerates the stag list for RTE_DWA_TAG_PORT_HOST_ETHERNET tag.
+ */
+enum rte_dwa_port_host_ethernet {
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_PORT_HOST_ETHERNET
+	 * Stag      | RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO
+	 * Direction | H2D
+	 * Type      | TYPE_ATTACHED
+	 * Payload   | NA
+	 * Pair TLV  | RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO
+	 *
+	 * Request DWA host ethernet port information.
+	 */
+	RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO,
+	/**
+	 * Attribute |  Value
+	 * ----------|---------
+	 * Tag       | RTE_DWA_TAG_PORT_HOST_ETHERNET
+	 * Stag      | RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO
+	 * Direction | H2D
+	 * Type      | TYPE_ATTACHED
+	 * Payload   | struct rte_dwa_port_host_ethernet_d2h_info
+	 * Pair TLV  | RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO
+	 *
+	 * Response for DWA host ethernet port information.
+	 */
+	RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO,
+	/**
+	 * Attribute |  Value
+	 * ----------|---------
+	 * Tag       | RTE_DWA_TAG_PORT_HOST_ETHERNET
+	 * Stag      | RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_CONFIG
+	 * Direction | H2D
+	 * Type      | TYPE_STOPPED
+	 * Payload   | struct rte_dwa_port_host_ethernet_config
+	 * Pair TLV  | RTE_DWA_STAG_COMMON_D2H_SUCCESS
+	 * ^         | RTE_DWA_STAG_COMMON_D2H_ERR
+	 *
+	 * Request DWA host ethernet port configuration.
+	 */
+	RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_CONFIG,
+	/**
+	 * Attribute |  Value
+	 * ----------|---------
+	 * Tag       | RTE_DWA_TAG_PORT_HOST_ETHERNET
+	 * Stag      | RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_QUEUE_CONFIG
+	 * Direction | H2D
+	 * Type      | TYPE_STOPPED
+	 * Payload   | struct rte_dwa_port_host_ethernet_queue_config
+	 * Pair TLV  | RTE_DWA_STAG_COMMON_D2H_SUCCESS
+	 * ^         | RTE_DWA_STAG_COMMON_D2H_ERR
+	 *
+	 * Request DWA host ethernet port queue configuration.
+	 *
+	 * @note RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_CONFIG must be called
+	 * before invoking this message.
+	 */
+	RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_QUEUE_CONFIG,
+	RTE_DWA_STAG_PORT_HOST_ETHERNET_MAX = UINT16_MAX,
+	/**< Max stags for RTE_DWA_TAG_PORT_HOST_ETHERNET tag*/
+};
+
+/**
+ * Transmit a burst of TLVs of type `TYPE_USER_PLANE` on the Tx queue
+ * designated by its *queue_id* of DWA object *obj*.
+ *
+ * @param obj
+ *   DWA object.
+ * @param queue_id
+ *   The identifier of Tx queue id. The queue id should in the range of
+ *   [0 to rte_dwa_port_host_ethernet_config::nb_tx_queues].
+ * @param[out] tlvs
+ *   Points to an array of *nb_tlvs* tlvs of type *rte_dwa_tlv* structure
+ *   to be transmitted.
+ * @param nb_tlvs
+ *   The maximum number of TLVs to transmit.
+ *
+ * @return
+ * The number of TLVs actually transmitted on the Tx queue. The return
+ * value can be less than the value of the *nb_tlvs* parameter when the
+ * Tx queue is full.
+ */
+uint16_t rte_dwa_port_host_ethernet_tx(rte_dwa_obj_t obj, uint16_t queue_id,
+			      struct rte_dwa_tlv **tlvs, uint16_t nb_tlvs);
+/**
+ * Receive a burst of TLVs of type `TYPE_USER_PLANE` from the Rx queue
+ * designated by its *queue_id* of DWA object *obj*.
+ *
+ * @param obj
+ *   DWA object.
+ * @param queue_id
+ *   The identifier of Rx queue id. The queue id should in the range of
+ *   [0 to rte_dwa_port_host_ethernet_config::nb_rx_queues].
+ * @param[out] tlvs
+ *   Points to an array of *nb_tlvs* tlvs of type *rte_dwa_tlv* structure
+ *   to be received.
+ * @param nb_tlvs
+ *   The maximum number of TLVs to received.
+ *
+ * @return
+ * The number of TLVs actually received on the Rx queue. The return
+ * value can be less than the value of the *nb_tlvs* parameter when the
+ * Rx queue is not full.
+ */
+uint16_t rte_dwa_port_host_ethernet_rx(rte_dwa_obj_t obj, uint16_t queue_id,
+			      struct rte_dwa_tlv **tlvs, uint16_t nb_tlvs);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* RTE_DWA_PORT_HOST_ETHERNET_H */
diff --git a/lib/dwa/rte_dwa_profile_admin.h b/lib/dwa/rte_dwa_profile_admin.h
new file mode 100644
index 0000000000..9443a049fb
--- /dev/null
+++ b/lib/dwa/rte_dwa_profile_admin.h
@@ -0,0 +1,85 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(C) 2021 Marvell.
+ */
+
+#ifndef RTE_DWA_PROFIE_ADMIN_H
+#define RTE_DWA_PROFIE_ADMIN_H
+
+/**
+ * @file
+ *
+ * @warning
+ * @b EXPERIMENTAL:
+ * All functions in this file may be changed or removed without prior notice.
+ *
+ * RTE API related to admin profile which includes administrative functions
+ * such as FW updates, resource partitioning in a DWA and items in global in
+ * nature that is applicable for all DWA device under the DWA.
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <rte_uuid.h>
+
+/**
+ * Payload of RTE_DWA_STAG_PROFILE_ADMIN_H2D_ATTACH message.
+ */
+struct rte_dwa_profile_admin_h2d_attach {
+	rte_uuid_t uuid; /**< uuid to validate the integrity of actor */
+} __rte_packed;
+
+/**
+ * Payload of RTE_DWA_STAG_PROFILE_ADMIN_H2D_FW_UPDATE message.
+ */
+struct rte_dwa_profile_admin_h2d_fw_update {
+	char fw[PATH_MAX]; /**< Firmware filename to update */
+} __rte_packed;
+
+/**
+ * Enumerates the stag list for RTE_DWA_TAG_PROFILE_ADMIN tag.
+ *
+ * @note A successful RTE_DWA_STAG_PROFILE_ADMIN_H2D_ATTACH operation
+ * must be prerequisite for all the admin operation.
+ */
+enum rte_dwa_profile_admin {
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_PROFILE_ADMIN
+	 * Stag      | RTE_DWA_STAG_PROFILE_ADMIN_H2D_ATTACH
+	 * Direction | H2D
+	 * Type      | TYPE_ATTACHED
+	 * Payload   | struct rte_dwa_profile_admin_h2d_attach
+	 * Pair TLV  | RTE_DWA_STAG_COMMON_D2H_SUCCESS
+	 * ^         | RTE_DWA_STAG_COMMON_D2H_ERR
+	 *
+	 * Request to attach DWA for administrative operation.
+	 * A universally unique identifier (UUID) used to validate
+	 * the validate the actor.
+	 */
+	RTE_DWA_STAG_PROFILE_ADMIN_H2D_ATTACH,
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_PROFILE_ADMIN
+	 * Stag      | RTE_DWA_STAG_PROFILE_ADMIN_H2D_FW_UPDATE
+	 * Direction | H2D
+	 * Type      | TYPE_ATTACHED
+	 * Payload   | struct rte_dwa_profile_admin_h2d_fw_update
+	 * Pair TLV  | RTE_DWA_STAG_COMMON_D2H_SUCCESS
+	 * ^         | RTE_DWA_STAG_COMMON_D2H_ERR
+	 *
+	 * Request DWA host ethernet port information.
+	 */
+	RTE_DWA_STAG_PROFILE_ADMIN_H2D_FW_UPDATE,
+	RTE_DWA_STAG_PROFILE_ADMIN_MAX = UINT16_MAX,
+	/**< Max stags for RTE_DWA_TAG_PROFILE_ADMIN tag*/
+};
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* RTE_DWA_PROFILE_ADMIN_H */
diff --git a/lib/dwa/rte_dwa_profile_l3fwd.h b/lib/dwa/rte_dwa_profile_l3fwd.h
new file mode 100644
index 0000000000..29b4e493e5
--- /dev/null
+++ b/lib/dwa/rte_dwa_profile_l3fwd.h
@@ -0,0 +1,378 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(C) 2021 Marvell.
+ */
+
+#ifndef RTE_DWA_PROFILE_L3FWD_H
+#define RTE_DWA_PROFILE_L3FWD_H
+
+/**
+ * @file
+ *
+ * L3FWD Profile
+ *
+ * \code{.c}
+ *                           +-------------->--[1]--------------+
+ *                           |                                  |
+ *               +-----------|----------+                       |
+ *               |           |          |                       |
+ *               |  +--------|-------+  |                       |
+ *               |  |                |  |                       |
+ *               |  | L3FWD Profile  |  |                       |
+ *    \          |  |                |  |                       |
+ *<====\========>|  +----------------+  |                       |
+ *  DWA \Port0   |     Lookup Table     |             +---------|----------+
+ *       \       |  +----------------+  |             | DPDK DWA|Device[0] |
+ *        \      |  | IP    | Dport  |  |  Host Port  | +-------|--------+ |
+ *         \     |  +----------------+  |<===========>| |       |        | |
+ *          +~[3]~~~|~~~~~~~|~~~~~~~~|~~~~~~~~~~~~~~~~~>|->L3FWD Profile | |
+ *<=============>|  +----------------+  |             | |                | |
+ *  DWA Port1    |  |       |        |  | Control Port| +-|---------|----+ |
+ *               |  +----------------+  |<===========>|   |         |      |
+ *  ~~~>~~[5]~~~~|~~|~~~+   |        |  |             +---|---------|------+
+ *               |  +---+------------+  |                 |         |
+ *  ~~~<~~~~~~~~~|~~|~~~+   |        |<-|------[2]--------+         |
+ *               |  +----------------+<-|------[4]------------------+
+ *               |    Dataplane         |
+ *<=============>|    Workload          |
+ *  DWA PortN    |    Accelerator       |
+ *               |    (HW/FW/SW)        |
+ *               +----------------------+
+ * \endcode
+ *
+ * L3FWD profile offloads Layer-3 forwarding between the DWA Ethernet ports.
+ *
+ * The above diagram depicts the profile and application programming sequence.
+ *
+ * -# DWA device attaches the L3FWD profile using rte_dwa_dev_attach().
+ * -# Configure the L3FWD profile:
+ *    - The application requests L3FWD profile capabilities of the DWA
+ *      by using RTE_DWA_STAG_PROFILE_L3FWD_H2D_INFO, On response, the
+ *      RTE_DWA_STAG_PROFILE_L3FWD_D2H_INFO returns the lookup modes supported,
+ *      max rules supported, and available host ports for this profile.
+ *    - The application configures a set of DWA ports to use a
+ *      lookup mode(EM, LPM, or FIB) via RTE_DWA_STAG_PROFILE_L3FWD_H2D_CONFIG.
+ *    - The application configures a valid host port to receive exception
+ *      packets.
+ * -# The exception that is not matching forwarding table entry comes as
+ * RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS TLV to host. DWA stores the
+ * exception packet send back destination ports after completing step (4).
+ * -# Parse the exception packet and add rules to the FWD table using
+ * RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_ADD. If the application knows the rules
+ * beforehand, it can add the rules in step 2.
+ * -# When DWA ports receive the matching flows in the lookup table, DWA
+ *  forwards to DWA Ethernet ports without host CPU intervention.
+ *
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <rte_common.h>
+
+/**
+ * Enumerates L3FWD profile lookup modes.
+ */
+enum rte_dwa_profile_l3fwd_lookup_mode {
+	RTE_DWA_PROFILE_L3FWD_MODE_EM = 1U << 0, /**< Exact match mode. */
+	RTE_DWA_PROFILE_L3FWD_MODE_LPM = 1U << 1,
+	/**< Longest prefix match mode. */
+	RTE_DWA_PROFILE_L3FWD_MODE_FIB = 1U << 2,
+	/**< Forwarding information base mode. */
+	RTE_DWA_PROFILE_L3FWD_MODE_MAX = 1U << 15, /**< Max modes. */
+};
+
+/**
+ * Payload of RTE_DWA_STAG_PROFILE_L3FWD_D2H_INFO message.
+ */
+struct rte_dwa_profile_l3fwd_d2h_info {
+	uint32_t max_lookup_rules;
+	/**< Maximum Supported lookup rules. */
+	uint16_t modes_supported;
+	/**< Each bit enabled represents a mode supported in
+	 * enum rte_dwa_profile_l3fwd_lookup_mode
+	 */
+	uint16_t nb_host_ports;
+	/**< Number of host ports in the host_ports. */
+	uint16_t host_ports[];
+	/**< Array of available host port of type enum rte_dwa_tag_port_host
+	 * of size nb_host_ports.
+	 */
+} __rte_packed;
+
+/**
+ * Payload of RTE_DWA_STAG_PROFILE_L3FWD_H2D_CONFIG message.
+ */
+struct rte_dwa_profile_l3fwd_h2d_config {
+	uint16_t mode;
+	/**< L3FWD profile mode.
+	 * @see enum rte_dwa_profile_l3fwd_modes
+	 */
+	uint16_t nb_eth_ports;
+	/**< Number of DWA ethernet ports in the eth_ports list. */
+	uint16_t eth_ports[];
+	/**< List of DWA ethernet ports to apply the profile on.*/
+} __rte_packed;
+
+/** L3FWD profile IPv4 rule attributes. */
+struct rte_dwa_profile_l3fwd_v4_5tpl {
+	uint32_t ip_dst;
+	/**< IPv4 destination address. */
+	uint32_t ip_src;
+	/**< IPv4 source address. */
+	uint16_t port_dst;
+	/**< Destination port. */
+	uint16_t port_src;
+	/**< Source port. */
+	uint8_t proto;
+	/**< Protocol. */
+} __rte_packed;
+
+/** L3FWD profile IPv4 rule prefix. */
+struct rte_dwa_profile_l3fwd_v4_prefix {
+	uint32_t ip_dst;
+	/**< IPv4 destination address. */
+	uint8_t depth;
+	/**< LPM depth. */
+} __rte_packed;
+
+/** L3FWD profile IPv4 rule. */
+struct rte_dwa_profile_l3fwd_v4_rule {
+	RTE_STD_C11
+	union {
+		struct rte_dwa_profile_l3fwd_v4_5tpl match;
+		/**< Match data. */
+		struct rte_dwa_profile_l3fwd_v4_prefix prefix;
+		/**< Prefix data. */
+	};
+} __rte_packed;
+
+/* IPv6 rule attributes */
+
+/** L3FWD profile IPV6 address length */
+#define RTE_DWA_PROFILE_L3FWD_IPV6_ADDR_LEN 16
+
+/** L3FWD profile IPv6 rule attributes. */
+struct rte_dwa_profile_l3fwd_v6_5tpl {
+	uint8_t ip_dst[RTE_DWA_PROFILE_L3FWD_IPV6_ADDR_LEN];
+	/**< IPv6 destination address. */
+	uint8_t ip_src[RTE_DWA_PROFILE_L3FWD_IPV6_ADDR_LEN];
+	/**< IPv6 source address. */
+	uint16_t port_dst;
+	/**< Destination port. */
+	uint16_t port_src;
+	/**< Source port. */
+	uint8_t proto;
+	/**< Protocol. */
+} __rte_packed;
+
+/** L3FWD profile IPv6 rule prefix. */
+struct rte_dwa_profile_l3fwd_v6_prefix {
+	uint8_t ip_dst[RTE_DWA_PROFILE_L3FWD_IPV6_ADDR_LEN];
+	/**< IPv6 destination address. */
+	uint8_t depth;
+	/**< LPM depth. */
+} __rte_packed;
+
+/** L3FWD profile IPv6 rule. */
+struct rte_dwa_profile_l3fwd_v6_rule {
+	RTE_STD_C11
+	union {
+		struct rte_dwa_profile_l3fwd_v6_5tpl match;
+		/**< Match data. */
+		struct rte_dwa_profile_l3fwd_v6_prefix prefix;
+		/**< Prefix data. */
+	};
+} __rte_packed;
+
+/** L3FWD profile rule type. */
+enum rte_dwa_profile_l3fwd_rule_type {
+	RTE_DWA_PROFILE_L3FWD_RULE_TYPE_IPV4 = 1U << 0,
+	/**< Rule type IPv4. */
+	RTE_DWA_PROFILE_L3FWD_RULE_TYPE_IPV6 = 1U << 1,
+	/**< Rule type IPv6. */
+};
+
+/**
+ * Payload of RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_ADD message.
+ */
+struct rte_dwa_profile_l3fwd_h2d_lookup_add {
+	enum rte_dwa_profile_l3fwd_rule_type rule_type;
+	/**< Rule type that is being added. */
+	struct rte_dwa_profile_l3fwd_v4_rule v4_rule;
+	/**< IPv4 rule. */
+	struct rte_dwa_profile_l3fwd_v6_rule v6_rule;
+	/**< IPv6 rule. */
+	uint16_t eth_port_dst;
+	/**< Destination lookup port. */
+} __rte_packed;
+
+/**
+ * Payload of RTE_DWA_STAG_PROFILE_L3FWD_D2H_LOOKUP_ADD message.
+ */
+struct rte_dwa_profile_l3fwd_d2h_lookup_add {
+	uint64_t handle; /**< Lookup rule handle. */
+} __rte_packed;
+
+/**
+ * Payload of RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_UPDATE message.
+ */
+struct rte_dwa_profile_l3fwd_h2d_lookup_update {
+	uint64_t handle;
+	/**< Rule handle to update a rule.
+	 * @see rte_dwa_profile_l3fwd_d2h_lookup_add
+	 */
+	uint16_t eth_port_dst;
+	/**< Destination lookup port to update. */
+} __rte_packed;
+
+/**
+ * Payload of RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_DEL message.
+ */
+struct rte_dwa_profile_l3fwd_h2d_lookup_delete {
+	uint64_t handle;
+	/**< Rule handle to delete a rule.
+	 * @see rte_dwa_profile_l3fwd_d2h_lookup_add
+	 */
+} __rte_packed;
+
+/**
+ * Payload of RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS message.
+ */
+struct rte_dwa_profile_l3fwd_d2h_exception_pkts {
+	uint16_t nb_pkts;
+	/**< Number of packets in the variable size array.*/
+	uint16_t rsvd16;
+	/**< Reserved field to make pkts[0] to be 64bit aligned.*/
+	uint32_t rsvd32;
+	/**< Reserved field to make pkts[0] to be 64bit aligned.*/
+	struct rte_mbuf *pkts[0];
+	/**< Array of rte_mbufs of size nb_pkts. */
+} __rte_packed;
+
+/**
+ * Enumerates the stag list for RTE_DWA_TAG_PROFILE_L3FWD tag.
+ *
+ */
+enum rte_dwa_profile_l3fwd {
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_PROFILE_L3FWD
+	 * Stag      | RTE_DWA_STAG_PROFILE_L3FWD_H2D_INFO
+	 * Direction | H2D
+	 * Type      | TYPE_ATTACHED
+	 * Payload   | NA
+	 * Pair TLV  | RTE_DWA_STAG_PROFILE_L3FWD_D2H_INFO
+	 *
+	 * Request to L3FWD profile information.
+	 */
+	RTE_DWA_STAG_PROFILE_L3FWD_H2D_INFO,
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_PROFILE_L3FWD
+	 * Stag      | RTE_DWA_STAG_PROFILE_L3FWD_D2H_INFO
+	 * Direction | D2H
+	 * Type      | TYPE_ATTACHED
+	 * Payload   | struct rte_dwa_profile_l3fwd_d2h_info
+	 * Pair TLV  | RTE_DWA_STAG_PROFILE_L3FWD_D2H_INFO
+	 *
+	 * Response for L3FWD profile information.
+	 */
+	RTE_DWA_STAG_PROFILE_L3FWD_D2H_INFO,
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_PROFILE_L3FWD
+	 * Stag      | RTE_DWA_STAG_PROFILE_L3FWD_H2D_CONFIG
+	 * Direction | H2D
+	 * Type      | TYPE_STOPPED
+	 * Payload   | struct rte_dwa_profile_l3fwd_h2d_config
+	 * Pair TLV  | RTE_DWA_STAG_COMMON_D2H_SUCCESS
+	 * ^         | RTE_DWA_STAG_COMMON_D2H_ERR
+	 *
+	 * Request to configure L3FWD profile.
+	 */
+	RTE_DWA_STAG_PROFILE_L3FWD_H2D_CONFIG,
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_PROFILE_L3FWD
+	 * Stag      | RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_ADD
+	 * Direction | H2D
+	 * Type      | TYPE_STOPPED
+	 * ^         | TYPE_STARTED
+	 * Payload   | struct rte_dwa_profile_l3fwd_h2d_lookup_add
+	 * Pair TLV  | RTE_DWA_STAG_PROFILE_L3FWD_D2H_LOOKUP_ADD
+	 *
+	 * Request to add rule in L3FWD profile.
+	 */
+	RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_ADD,
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_PROFILE_L3FWD
+	 * Stag      | RTE_DWA_STAG_PROFILE_L3FWD_D2H_LOOKUP_ADD
+	 * Direction | D2H
+	 * Type      | TYPE_STOPPED
+	 * ^         | TYPE_STARTED
+	 * Payload   | struct rte_dwa_profile_l3fwd_d2h_lookup_add
+	 * Pair TLV  | RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_ADD
+	 *
+	 * Response for L3FWD profile rule add.
+	 * It contains the handle for further operation on this rule.
+	 */
+	RTE_DWA_STAG_PROFILE_L3FWD_D2H_LOOKUP_ADD,
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_PROFILE_L3FWD
+	 * Stag      | RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_UPDATE
+	 * Direction | H2D
+	 * Type      | TYPE_STOPPED
+	 * ^         | TYPE_STARTED
+	 * Payload   | struct rte_dwa_profile_l3fwd_h2d_lookup_update
+	 * Pair TLV  | RTE_DWA_STAG_COMMON_D2H_SUCCESS
+	 * ^         | RTE_DWA_STAG_COMMON_D2H_ERR
+	 *
+	 * Request to update the rule in L3FWD profile.
+	 */
+	RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_UPDATE,
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_PROFILE_L3FWD
+	 * Stag      | RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_DEL
+	 * Direction | H2D
+	 * Type      | TYPE_STOPPED
+	 * ^         | TYPE_STARTED
+	 * Payload   | struct rte_dwa_profile_l3fwd_h2d_lookup_delete
+	 * Pair TLV  | RTE_DWA_STAG_COMMON_D2H_SUCCESS
+	 * ^         | RTE_DWA_STAG_COMMON_D2H_ERR
+	 *
+	 * Request to delete the rule in L3FWD profile.
+	 */
+	RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_DEL,
+	/**
+	 * Attribute |  Value
+	 * ----------|--------
+	 * Tag       | RTE_DWA_TAG_PROFILE_L3FWD
+	 * Stag      | RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS
+	 * Direction | D2H
+	 * Type      | TYPE_USER_PLANE
+	 * Payload   | struct rte_dwa_profile_l3fwd_d2h_exception_pkts
+	 * Pair TLV  | NA
+	 *
+	 * Response from DWA of exception packets.
+	 */
+	RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXECPTION_PACKETS,
+	RTE_DWA_STAG_PROFILE_L3FWD_MAX = UINT16_MAX,
+	/**< Max stags for RTE_DWA_TAG_PROFILE_L3FWD tag*/
+};
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* RTE_DWA_PROFILE_L3FWD_H */
diff --git a/lib/dwa/version.map b/lib/dwa/version.map
new file mode 100644
index 0000000000..33c1b976f1
--- /dev/null
+++ b/lib/dwa/version.map
@@ -0,0 +1,3 @@
+EXPERIMENTAL {
+	local: *;
+};
diff --git a/lib/meson.build b/lib/meson.build
index 3b8b099820..f211557560 100644
--- a/lib/meson.build
+++ b/lib/meson.build
@@ -61,6 +61,7 @@ libraries = [
         'bpf',
         'graph',
         'node',
+        'dwa',
 ]
 
 if is_windows
-- 
2.33.1


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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-19 18:14 [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library jerinj
  2021-10-19 18:14 ` [dpdk-dev] [RFC PATCH 1/1] dwa: introduce dataplane workload accelerator subsystem jerinj
@ 2021-10-19 19:08 ` Thomas Monjalon
  2021-10-19 19:36   ` Jerin Jacob
  2021-10-25  7:35 ` Mattias Rönnblom
  2 siblings, 1 reply; 20+ messages in thread
From: Thomas Monjalon @ 2021-10-19 19:08 UTC (permalink / raw)
  To: jerinj
  Cc: dev, ferruh.yigit, ajit.khaparde, aboyer, andrew.rybchenko,
	beilei.xing, bruce.richardson, chas3, chenbo.xia, ciara.loftus,
	dsinghrawat, ed.czeck, evgenys, grive, g.singh, zhouguoyang,
	haiyue.wang, hkalra, heinrich.kuhn, hemant.agrawal, hyonkim,
	igorch, irusskikh, jgrajcia, jasvinder.singh, jianwang, jiawenwu,
	jingjing.wu, johndale, john.miller, linville, keith.wiles,
	kirankumark, oulijun, lironh, longli, mw, spinler, matan,
	matt.peters, maxime.coquelin, mk, humin29, pnalla, ndabilpuram,
	qiming.yang, qi.z.zhang, radhac, rahul.lakkireddy, rmody,
	rosen.xu, sachin.saxena, skoteshwar, shshaikh, shaibran,
	shepard.siegel, asomalap, somnath.kotur, sthemmin,
	steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, mattias.ronnblom, ruifeng.wang, drc,
	konstantin.ananyev, olivier.matz, jay.jayatheerthan, asekhar,
	pbhagavatula, eagostini, david.marchand, tom

19/10/2021 20:14, jerinj@marvell.com:
> Definition of Dataplane Workload Accelerator
> --------------------------------------------
> Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> Network controllers and programmable data acceleration engines for
> packet processing, cryptography, regex engines, baseband processing, etc. 
> This allows DWA to offload  compute/packet processing/baseband/
> cryptography-related workload from the host CPU to save the cost and power. 
> Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
> 
> Unlike other devices in DPDK, the DWA device is not fixed-function
> due to the fact that it has CPUs and programmable HW accelerators.
> This enables DWA personality/workload to be completely programmable.
> Typical examples of DWA offloads are Flow/Session management,
> Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.

If I understand well, the idea is to abstract the offload
of some stack layers in the hardware.
I am not sure we should give an API for such stack layers in DPDK.
It looks to be the role of the dataplane application to finely manage
how to use the hardware for a specific dataplane.
I believe the API for such layer would be either too big, or too limited,
or not optimized for specific needs.
If we really want to automate or abstract the HW/SW co-design,
I think we should better look at compiler work like P4 or PANDA.



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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-19 19:08 ` [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library Thomas Monjalon
@ 2021-10-19 19:36   ` Jerin Jacob
  2021-10-19 20:42     ` Stephen Hemminger
                       ` (2 more replies)
  0 siblings, 3 replies; 20+ messages in thread
From: Jerin Jacob @ 2021-10-19 19:36 UTC (permalink / raw)
  To: Thomas Monjalon
  Cc: Jerin Jacob, dpdk-dev, Ferruh Yigit, Ajit Khaparde, Andrew Boyer,
	Andrew Rybchenko, Beilei Xing, Richardson, Bruce, Chas Williams,
	Xia, Chenbo, Ciara Loftus, Devendra Singh Rawat, Ed Czeck,
	Evgeny Schemeilin, Gaetan Rivet, Gagandeep Singh, Guoyang Zhou,
	Haiyue Wang, Harman Kalra, heinrich.kuhn, Hemant Agrawal,
	Hyong Youb Kim, Igor Chauskin, Igor Russkikh, Jakub Grajciar,
	Jasvinder Singh, Jian Wang, Jiawen Wu, Jingjing Wu, John Daley,
	John Miller, John W. Linville, Wiles, Keith, Kiran Kumar K,
	Lijun Ou, Liron Himi, Long Li, Marcin Wojtas, Martin Spinler,
	Matan Azrad, Matt Peters, Maxime Coquelin, Michal Krawczyk,
	Min Hu (Connor, Pradeep Kumar Nalla, Nithin Dabilpuram,
	Qiming Yang, Qi Zhang, Radha Mohan Chintakuntla,
	Rahul Lakkireddy, Rasesh Mody, Rosen Xu, Sachin Saxena,
	Satha Koteswara Rao Kottidi, Shahed Shaikh, Shai Brandes,
	Shepard Siegel, Somalapuram Amaranath, Somnath Kotur,
	Stephen Hemminger, Steven Webster, Sunil Kumar Kori,
	Tetsuya Mukawa, Veerasenareddy Burru, Viacheslav Ovsiienko,
	Xiao Wang, Xiaoyun Wang, Yisen Zhuang, Yong Wang, Ziyang Xuan,
	Prasun Kapoor, nadavh, Satananda Burla, Narayana Prasad,
	Akhil Goyal, Ray Kinsella, Dmitry Kozlyuk, Anatoly Burakov,
	Cristian Dumitrescu, Honnappa Nagarahalli, Mattias Rönnblom,
	Ruifeng Wang (Arm Technology China),
	David Christensen, Ananyev, Konstantin, Olivier Matz,
	Jayatheerthan, Jay, Ashwin Sekhar Thalakalath Kottilveetil,
	Pavan Nikhilesh, Elena Agostini, David Marchand, tom

On Wed, Oct 20, 2021 at 12:38 AM Thomas Monjalon <thomas@monjalon.net> wrote:
>
> 19/10/2021 20:14, jerinj@marvell.com:
> > Definition of Dataplane Workload Accelerator
> > --------------------------------------------
> > Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> > Network controllers and programmable data acceleration engines for
> > packet processing, cryptography, regex engines, baseband processing, etc.
> > This allows DWA to offload  compute/packet processing/baseband/
> > cryptography-related workload from the host CPU to save the cost and power.
> > Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
> >
> > Unlike other devices in DPDK, the DWA device is not fixed-function
> > due to the fact that it has CPUs and programmable HW accelerators.
> > This enables DWA personality/workload to be completely programmable.
> > Typical examples of DWA offloads are Flow/Session management,
> > Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.
>
> If I understand well, the idea is to abstract the offload
> of some stack layers in the hardware.

Yes. It may not just HW, For expressing the complicated workloads
may need CPU and/or other HW accelerators.

> I am not sure we should give an API for such stack layers in DPDK.

Why not?

> It looks to be the role of the dataplane application to finely manage
> how to use the hardware for a specific dataplane.

It is possible with this scheme.

> I believe the API for such layer would be either too big, or too limited,
> or not optimized for specific needs.

It will be optimized for specific needs as applications ask for what to do?
not how to do?

> If we really want to automate or abstract the HW/SW co-design,
> I think we should better look at compiler work like P4 or PANDA.

The compiler stuff is very static in nature. It can address the packet
transformation
workloads. Not the ones like IPsec or baseband offload.
Another way to look at it, GPU RFC started just because you are not able
to express all the workload in P4.

>
>

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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-19 19:36   ` Jerin Jacob
@ 2021-10-19 20:42     ` Stephen Hemminger
  2021-10-20  5:25       ` Jerin Jacob
  2021-10-19 20:42     ` Tom Herbert
  2021-10-22 12:00     ` Elena Agostini
  2 siblings, 1 reply; 20+ messages in thread
From: Stephen Hemminger @ 2021-10-19 20:42 UTC (permalink / raw)
  To: Jerin Jacob
  Cc: Thomas Monjalon, Jerin Jacob, dpdk-dev, Ferruh Yigit,
	Ajit Khaparde, Andrew Boyer, Andrew Rybchenko, Beilei Xing,
	Richardson, Bruce, Chas Williams, Xia, Chenbo, Ciara Loftus,
	Devendra Singh Rawat, Ed Czeck, Evgeny Schemeilin, Gaetan Rivet,
	Gagandeep Singh, Guoyang Zhou, Haiyue Wang, Harman Kalra,
	heinrich.kuhn, Hemant Agrawal, Hyong Youb Kim, Igor Chauskin,
	Igor Russkikh, Jakub Grajciar, Jasvinder Singh, Jian Wang,
	Jiawen Wu, Jingjing Wu, John Daley, John Miller,
	John W. Linville, Wiles, Keith, Kiran Kumar K, Lijun Ou,
	Liron Himi, Long Li, Marcin Wojtas, Martin Spinler, Matan Azrad,
	Matt Peters, Maxime Coquelin, Michal Krawczyk, Min Hu (Connor,
	Pradeep Kumar Nalla, Nithin Dabilpuram, Qiming Yang, Qi Zhang,
	Radha Mohan Chintakuntla, Rahul Lakkireddy, Rasesh Mody,
	Rosen Xu, Sachin Saxena, Satha Koteswara Rao Kottidi,
	Shahed Shaikh, Shai Brandes, Shepard Siegel,
	Somalapuram Amaranath, Somnath Kotur, Stephen Hemminger,
	Steven Webster, Sunil Kumar Kori, Tetsuya Mukawa,
	Veerasenareddy Burru, Viacheslav Ovsiienko, Xiao Wang,
	Xiaoyun Wang, Yisen Zhuang, Yong Wang, Ziyang Xuan,
	Prasun Kapoor, nadavh, Satananda Burla, Narayana Prasad,
	Akhil Goyal, Ray Kinsella, Dmitry Kozlyuk, Anatoly Burakov,
	Cristian Dumitrescu, Honnappa Nagarahalli, Mattias Rönnblom,
	Ruifeng Wang (Arm Technology China),
	David Christensen, Ananyev, Konstantin, Olivier Matz,
	Jayatheerthan, Jay, Ashwin Sekhar Thalakalath Kottilveetil,
	Pavan Nikhilesh, Elena Agostini, David Marchand, tom

On Wed, 20 Oct 2021 01:06:10 +0530
Jerin Jacob <jerinjacobk@gmail.com> wrote:

> On Wed, Oct 20, 2021 at 12:38 AM Thomas Monjalon <thomas@monjalon.net> wrote:
> >
> > 19/10/2021 20:14, jerinj@marvell.com:  
> > > Definition of Dataplane Workload Accelerator
> > > --------------------------------------------
> > > Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> > > Network controllers and programmable data acceleration engines for
> > > packet processing, cryptography, regex engines, baseband processing, etc.
> > > This allows DWA to offload  compute/packet processing/baseband/
> > > cryptography-related workload from the host CPU to save the cost and power.
> > > Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
> > >
> > > Unlike other devices in DPDK, the DWA device is not fixed-function
> > > due to the fact that it has CPUs and programmable HW accelerators.
> > > This enables DWA personality/workload to be completely programmable.
> > > Typical examples of DWA offloads are Flow/Session management,
> > > Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.  
> >
> > If I understand well, the idea is to abstract the offload
> > of some stack layers in the hardware.  
> 
> Yes. It may not just HW, For expressing the complicated workloads
> may need CPU and/or other HW accelerators.
> 
> > I am not sure we should give an API for such stack layers in DPDK.  
> 
> Why not
This is the problem of should DPDK reinvent higher protocol layers?
Given the myriad of other projects using DPDK that already provide these
protocols; the de facto decision has been to not add higher layers.

The boundary between DPDK and OVS, VPP, yastack, ... has stayed roughly
the same for several years. If DPDK starts to overlap these other projects
it makes life harder.

If the HW support crosses over between projects it gets very complex.


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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-19 19:36   ` Jerin Jacob
  2021-10-19 20:42     ` Stephen Hemminger
@ 2021-10-19 20:42     ` Tom Herbert
  2021-10-20  5:38       ` Jerin Jacob
  2021-10-22 12:00     ` Elena Agostini
  2 siblings, 1 reply; 20+ messages in thread
From: Tom Herbert @ 2021-10-19 20:42 UTC (permalink / raw)
  To: Jerin Jacob
  Cc: Thomas Monjalon, Jerin Jacob, dpdk-dev, Ferruh Yigit,
	Ajit Khaparde, Andrew Boyer, Andrew Rybchenko, Beilei Xing,
	Richardson, Bruce, Chas Williams, Xia, Chenbo, Ciara Loftus,
	Devendra Singh Rawat, Ed Czeck, Evgeny Schemeilin, Gaetan Rivet,
	Gagandeep Singh, Guoyang Zhou, Haiyue Wang, Harman Kalra,
	heinrich.kuhn, Hemant Agrawal, Hyong Youb Kim, Igor Chauskin,
	Igor Russkikh, Jakub Grajciar, Jasvinder Singh, Jian Wang,
	Jiawen Wu, Jingjing Wu, John Daley, John Miller,
	John W. Linville, Wiles, Keith, Kiran Kumar K, Lijun Ou,
	Liron Himi, Long Li, Marcin Wojtas, Martin Spinler, Matan Azrad,
	Matt Peters, Maxime Coquelin, Michal Krawczyk, Min Hu (Connor,
	Pradeep Kumar Nalla, Nithin Dabilpuram, Qiming Yang, Qi Zhang,
	Radha Mohan Chintakuntla, Rahul Lakkireddy, Rasesh Mody,
	Rosen Xu, Sachin Saxena, Satha Koteswara Rao Kottidi,
	Shahed Shaikh, Shai Brandes, Shepard Siegel,
	Somalapuram Amaranath, Somnath Kotur, Stephen Hemminger,
	Steven Webster, Sunil Kumar Kori, Tetsuya Mukawa,
	Veerasenareddy Burru, Viacheslav Ovsiienko, Xiao Wang,
	Xiaoyun Wang, Yisen Zhuang, Yong Wang, Ziyang Xuan,
	Prasun Kapoor, nadavh, Satananda Burla, Narayana Prasad,
	Akhil Goyal, Ray Kinsella, Dmitry Kozlyuk, Anatoly Burakov,
	Cristian Dumitrescu, Honnappa Nagarahalli, Mattias Rönnblom,
	Ruifeng Wang (Arm Technology China),
	David Christensen, Ananyev, Konstantin, Olivier Matz,
	Jayatheerthan, Jay, Ashwin Sekhar Thalakalath Kottilveetil,
	Pavan Nikhilesh, Elena Agostini, David Marchand, felipe, chethan,
	siva, Tom Herbert

On Tue, Oct 19, 2021 at 12:36 PM Jerin Jacob <jerinjacobk@gmail.com> wrote:
>
> On Wed, Oct 20, 2021 at 12:38 AM Thomas Monjalon <thomas@monjalon.net> wrote:
> >
> > 19/10/2021 20:14, jerinj@marvell.com:
> > > Definition of Dataplane Workload Accelerator
> > > --------------------------------------------
> > > Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> > > Network controllers and programmable data acceleration engines for
> > > packet processing, cryptography, regex engines, baseband processing, etc.
> > > This allows DWA to offload  compute/packet processing/baseband/
> > > cryptography-related workload from the host CPU to save the cost and power.
> > > Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
> > >
> > > Unlike other devices in DPDK, the DWA device is not fixed-function
> > > due to the fact that it has CPUs and programmable HW accelerators.
> > > This enables DWA personality/workload to be completely programmable.
> > > Typical examples of DWA offloads are Flow/Session management,
> > > Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.
> >
> > If I understand well, the idea is to abstract the offload
> > of some stack layers in the hardware.
>
> Yes. It may not just HW, For expressing the complicated workloads
> may need CPU and/or other HW accelerators.
>
> > I am not sure we should give an API for such stack layers in DPDK.
>
> Why not?
>
> > It looks to be the role of the dataplane application to finely manage
> > how to use the hardware for a specific dataplane.
>
> It is possible with this scheme.
>
> > I believe the API for such layer would be either too big, or too limited,
> > or not optimized for specific needs.
>
> It will be optimized for specific needs as applications ask for what to do?
> not how to do?
>
> > If we really want to automate or abstract the HW/SW co-design,
> > I think we should better look at compiler work like P4 or PANDA.
>
> The compiler stuff is very static in nature. It can address the packet
> transformation
> workloads. Not the ones like IPsec or baseband offload.
> Another way to look at it, GPU RFC started just because you are not able
> to express all the workload in P4.

Hi,

Indeed, you may want to look at PANDA
(https://github.com/panda-net/panda) for this purpose especially with
regard to HW/SW co-design. Fundamentally, it is C/C++ so it's "Turing
Complete" as far as being able to express arbitrary workloads. The
program structure abstracts out any underlying details of the runtime
environment (hence it is "right once, run anywhere"). It is the
auspices of a compiler to convert the user's expression of intent into
optimized code for the backend; the backends can be software, such as
DPDK (which PANDA will soon support), or even hardware. In any case,
the code emitted will be optimized per the environment, taking
advantage of hardware acceleration for instance, which leads to the
PANDA mantra "write once, run anywhere, run well". This does require
APIs to control hardware acceleration, but our goal is to hide that
complexity from the user without the loss of benefits of emerging
hardware features. Also with emerging compiler techniques like LLVM's
MLIR and dynamically defined instructions, the historically "static"
nature of compilers can be undone.

Tom


>
> >
> >

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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-19 20:42     ` Stephen Hemminger
@ 2021-10-20  5:25       ` Jerin Jacob
  0 siblings, 0 replies; 20+ messages in thread
From: Jerin Jacob @ 2021-10-20  5:25 UTC (permalink / raw)
  To: Stephen Hemminger
  Cc: Thomas Monjalon, Jerin Jacob, dpdk-dev, Ferruh Yigit,
	Ajit Khaparde, Andrew Boyer, Andrew Rybchenko, Beilei Xing,
	Richardson, Bruce, Chas Williams, Xia, Chenbo, Ciara Loftus,
	Devendra Singh Rawat, Ed Czeck, Evgeny Schemeilin, Gaetan Rivet,
	Gagandeep Singh, Guoyang Zhou, Haiyue Wang, Harman Kalra,
	heinrich.kuhn, Hemant Agrawal, Hyong Youb Kim, Igor Chauskin,
	Igor Russkikh, Jakub Grajciar, Jasvinder Singh, Jian Wang,
	Jiawen Wu, Jingjing Wu, John Daley, John Miller,
	John W. Linville, Wiles, Keith, Kiran Kumar K, Lijun Ou,
	Liron Himi, Long Li, Marcin Wojtas, Martin Spinler, Matan Azrad,
	Matt Peters, Maxime Coquelin, Michal Krawczyk, Min Hu (Connor,
	Pradeep Kumar Nalla, Nithin Dabilpuram, Qiming Yang, Qi Zhang,
	Radha Mohan Chintakuntla, Rahul Lakkireddy, Rasesh Mody,
	Rosen Xu, Sachin Saxena, Satha Koteswara Rao Kottidi,
	Shahed Shaikh, Shai Brandes, Shepard Siegel,
	Somalapuram Amaranath, Somnath Kotur, Stephen Hemminger,
	Steven Webster, Sunil Kumar Kori, Tetsuya Mukawa,
	Veerasenareddy Burru, Viacheslav Ovsiienko, Xiao Wang,
	Xiaoyun Wang, Yisen Zhuang, Yong Wang, Ziyang Xuan,
	Prasun Kapoor, nadavh, Satananda Burla, Narayana Prasad,
	Akhil Goyal, Ray Kinsella, Dmitry Kozlyuk, Anatoly Burakov,
	Cristian Dumitrescu, Honnappa Nagarahalli, Mattias Rönnblom,
	Ruifeng Wang (Arm Technology China),
	David Christensen, Ananyev, Konstantin, Olivier Matz,
	Jayatheerthan, Jay, Ashwin Sekhar Thalakalath Kottilveetil,
	Pavan Nikhilesh, Elena Agostini, David Marchand, Tom Herbert

On Wed, Oct 20, 2021 at 2:12 AM Stephen Hemminger
<stephen@networkplumber.org> wrote:
>
> On Wed, 20 Oct 2021 01:06:10 +0530
> Jerin Jacob <jerinjacobk@gmail.com> wrote:
>
> > On Wed, Oct 20, 2021 at 12:38 AM Thomas Monjalon <thomas@monjalon.net> wrote:
> > >
> > > 19/10/2021 20:14, jerinj@marvell.com:
> > > > Definition of Dataplane Workload Accelerator
> > > > --------------------------------------------
> > > > Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> > > > Network controllers and programmable data acceleration engines for
> > > > packet processing, cryptography, regex engines, baseband processing, etc.
> > > > This allows DWA to offload  compute/packet processing/baseband/
> > > > cryptography-related workload from the host CPU to save the cost and power.
> > > > Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
> > > >
> > > > Unlike other devices in DPDK, the DWA device is not fixed-function
> > > > due to the fact that it has CPUs and programmable HW accelerators.
> > > > This enables DWA personality/workload to be completely programmable.
> > > > Typical examples of DWA offloads are Flow/Session management,
> > > > Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.
> > >
> > > If I understand well, the idea is to abstract the offload
> > > of some stack layers in the hardware.
> >
> > Yes. It may not just HW, For expressing the complicated workloads
> > may need CPU and/or other HW accelerators.
> >
> > > I am not sure we should give an API for such stack layers in DPDK.
> >
> > Why not
> This is the problem of should DPDK reinvent higher protocol layers?
> Given the myriad of other projects using DPDK that already provide these
> protocols; the de facto decision has been to not add higher layers.
>
> The boundary between DPDK and OVS, VPP, yastack, ... has stayed roughly
> the same for several years. If DPDK starts to overlap these other projects
> it makes life harder.

The intention is to NOT replace these projects, instead, the intention
is to use these projects in the accelerator.
There is a subtle difference between defining the workload vs
implementing the workload.
This RFC attempts the former.

I am trying to address the use case where the end-user needs to
offload a workload
to the accelerator, which contains a set of CPUs, Network controllers
and programmable data acceleration
engines for packet processing, cryptography, regex engines, baseband
processing, etc.

Offload is expressed in very high-level TLV messages. Accelerator
picks these messages,
and using the above technologies to implement it in acceleration HW.
This enables offloading the workloads from the Host CPU.

There are discrete attempts to solve this problem in isolation like
https://github.com/att/sessionOffload.
If you see the above workload the application cares for very
high-level messages for
AddSession, DeleteSession, GetSession, SendStats etc and leave the
accelerator for _implementing_ those workloads.

In order to enable the above use case, the following items need to be
sorted out:
1) Communication between Accelerator and host CPU application
2) Implement the workload in Accelerator

Since this is anyway discussed in today's TB meeting. I am sharing my
thoughts here.

IMO, There are two ways to do it:

Way 1:
1) Introduce libraries specific to accelerator communication like
gpudev, xpudev, dpudev etc
2) Implement the workload in accelerator specific fashion in the application.

Pros:

Cons:
- Need for introducing multiple libraries for specific accelerator
communication in DPDK
- Specific application needs to be written for each accelerator that
needs to be offloaded.


Way 2:
1) Introduce the generic library for accelerator communication.
2) Application express the workload in a non-accelator specific fashion and let
the accelerator implementation(Dont have the requirement to add it in
DPDK code base) implement
the functional model.


Pros:
-  The same DPDK application can use to offload various accelerator technologies
-  More contributors can use the library as it is no specific one type
of accelerator device

Cons:


















>
> If the HW support crosses over between projects it gets very complex.
>

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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-19 20:42     ` Tom Herbert
@ 2021-10-20  5:38       ` Jerin Jacob
  0 siblings, 0 replies; 20+ messages in thread
From: Jerin Jacob @ 2021-10-20  5:38 UTC (permalink / raw)
  To: Tom Herbert
  Cc: Thomas Monjalon, Jerin Jacob, dpdk-dev, Ferruh Yigit,
	Ajit Khaparde, Andrew Boyer, Andrew Rybchenko, Beilei Xing,
	Richardson, Bruce, Chas Williams, Xia, Chenbo, Ciara Loftus,
	Devendra Singh Rawat, Ed Czeck, Evgeny Schemeilin, Gaetan Rivet,
	Gagandeep Singh, Guoyang Zhou, Haiyue Wang, Harman Kalra,
	heinrich.kuhn, Hemant Agrawal, Hyong Youb Kim, Igor Chauskin,
	Igor Russkikh, Jakub Grajciar, Jasvinder Singh, Jian Wang,
	Jiawen Wu, Jingjing Wu, John Daley, John Miller,
	John W. Linville, Wiles, Keith, Kiran Kumar K, Lijun Ou,
	Liron Himi, Long Li, Marcin Wojtas, Martin Spinler, Matan Azrad,
	Matt Peters, Maxime Coquelin, Michal Krawczyk, Min Hu (Connor,
	Pradeep Kumar Nalla, Nithin Dabilpuram, Qiming Yang, Qi Zhang,
	Radha Mohan Chintakuntla, Rahul Lakkireddy, Rasesh Mody,
	Rosen Xu, Sachin Saxena, Satha Koteswara Rao Kottidi,
	Shahed Shaikh, Shai Brandes, Shepard Siegel,
	Somalapuram Amaranath, Somnath Kotur, Stephen Hemminger,
	Steven Webster, Sunil Kumar Kori, Tetsuya Mukawa,
	Veerasenareddy Burru, Viacheslav Ovsiienko, Xiao Wang,
	Xiaoyun Wang, Yisen Zhuang, Yong Wang, Ziyang Xuan,
	Prasun Kapoor, nadavh, Satananda Burla, Narayana Prasad,
	Akhil Goyal, Ray Kinsella, Dmitry Kozlyuk, Anatoly Burakov,
	Cristian Dumitrescu, Honnappa Nagarahalli, Mattias Rönnblom,
	Ruifeng Wang (Arm Technology China),
	David Christensen, Ananyev, Konstantin, Olivier Matz,
	Jayatheerthan, Jay, Ashwin Sekhar Thalakalath Kottilveetil,
	Pavan Nikhilesh, Elena Agostini, David Marchand, felipe, chethan,
	siva, Tom Herbert

On Wed, Oct 20, 2021 at 2:12 AM Tom Herbert <tom@herbertland.com> wrote:
>
> On Tue, Oct 19, 2021 at 12:36 PM Jerin Jacob <jerinjacobk@gmail.com> wrote:
> >
> > On Wed, Oct 20, 2021 at 12:38 AM Thomas Monjalon <thomas@monjalon.net> wrote:
> > >
> > > 19/10/2021 20:14, jerinj@marvell.com:
> > > > Definition of Dataplane Workload Accelerator
> > > > --------------------------------------------
> > > > Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> > > > Network controllers and programmable data acceleration engines for
> > > > packet processing, cryptography, regex engines, baseband processing, etc.
> > > > This allows DWA to offload  compute/packet processing/baseband/
> > > > cryptography-related workload from the host CPU to save the cost and power.
> > > > Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
> > > >
> > > > Unlike other devices in DPDK, the DWA device is not fixed-function
> > > > due to the fact that it has CPUs and programmable HW accelerators.
> > > > This enables DWA personality/workload to be completely programmable.
> > > > Typical examples of DWA offloads are Flow/Session management,
> > > > Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.
> > >
> > > If I understand well, the idea is to abstract the offload
> > > of some stack layers in the hardware.
> >
> > Yes. It may not just HW, For expressing the complicated workloads
> > may need CPU and/or other HW accelerators.
> >
> > > I am not sure we should give an API for such stack layers in DPDK.
> >
> > Why not?
> >
> > > It looks to be the role of the dataplane application to finely manage
> > > how to use the hardware for a specific dataplane.
> >
> > It is possible with this scheme.
> >
> > > I believe the API for such layer would be either too big, or too limited,
> > > or not optimized for specific needs.
> >
> > It will be optimized for specific needs as applications ask for what to do?
> > not how to do?
> >
> > > If we really want to automate or abstract the HW/SW co-design,
> > > I think we should better look at compiler work like P4 or PANDA.
> >
> > The compiler stuff is very static in nature. It can address the packet
> > transformation
> > workloads. Not the ones like IPsec or baseband offload.
> > Another way to look at it, GPU RFC started just because you are not able
> > to express all the workload in P4.
>
> Hi,
>
> Indeed, you may want to look at PANDA
> (https://github.com/panda-net/panda) for this purpose especially with
> regard to HW/SW co-design. Fundamentally, it is C/C++ so it's "Turing
> Complete" as far as being able to express arbitrary workloads. The
> program structure abstracts out any underlying details of the runtime
> environment (hence it is "right once, run anywhere"). It is the
> auspices of a compiler to convert the user's expression of intent into
> optimized code for the backend; the backends can be software, such as
> DPDK (which PANDA will soon support), or even hardware. In any case,
> the code emitted will be optimized per the environment, taking
> advantage of hardware acceleration for instance, which leads to the
> PANDA mantra "write once, run anywhere, run well". This does require
> APIs to control hardware acceleration, but our goal is to hide that
> complexity from the user without the loss of benefits of emerging
> hardware features. Also with emerging compiler techniques like LLVM's
> MLIR and dynamically defined instructions, the historically "static"
> nature of compilers can be undone.

Thanks for the insight.
These technologies we are using in Accelerator device and not host CPU
interface. host-side is all about memory management, communication, and
expressing the workload. And workload like vDPU application to
offload ORAN 7.2 split for 5G RU device is much beyond packet
processing.

>
> Tom
>
>
> >
> > >
> > >

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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-19 19:36   ` Jerin Jacob
  2021-10-19 20:42     ` Stephen Hemminger
  2021-10-19 20:42     ` Tom Herbert
@ 2021-10-22 12:00     ` Elena Agostini
  2021-10-22 13:39       ` Jerin Jacob
  2 siblings, 1 reply; 20+ messages in thread
From: Elena Agostini @ 2021-10-22 12:00 UTC (permalink / raw)
  To: Jerin Jacob, NBU-Contact-Thomas Monjalon
  Cc: Jerin Jacob, dpdk-dev, Ferruh Yigit, Ajit Khaparde, Andrew Boyer,
	Andrew Rybchenko, Beilei Xing, Richardson, Bruce, Chas Williams,
	Xia, Chenbo, Ciara Loftus, Devendra Singh Rawat, Ed Czeck,
	Evgeny Schemeilin, Gaetan Rivet, Gagandeep Singh, Guoyang Zhou,
	Haiyue Wang, Harman Kalra, heinrich.kuhn, Hemant Agrawal,
	Hyong Youb Kim, Igor Chauskin, Igor Russkikh, Jakub Grajciar,
	Jasvinder Singh, Jian Wang, Jiawen Wu, Jingjing Wu, John Daley,
	John Miller, John W. Linville, Wiles, Keith, Kiran Kumar K,
	Lijun Ou, Liron Himi, NBU-Contact-longli, Marcin Wojtas,
	Martin Spinler, Matan Azrad, Matt Peters, Maxime Coquelin,
	Michal Krawczyk, Min Hu (Connor, Pradeep Kumar Nalla,
	Nithin Dabilpuram, Qiming Yang, Qi Zhang,
	Radha Mohan Chintakuntla, Rahul Lakkireddy, Rasesh Mody,
	Rosen Xu, Sachin Saxena, Satha Koteswara Rao Kottidi,
	Shahed Shaikh, Shai Brandes, Shepard Siegel,
	Somalapuram Amaranath, Somnath Kotur, Stephen Hemminger,
	Steven Webster, Sunil Kumar Kori, Tetsuya Mukawa,
	Veerasenareddy Burru, Slava Ovsiienko, Xiao Wang, Xiaoyun Wang,
	Yisen Zhuang, Yong Wang, Ziyang Xuan, Prasun Kapoor, nadavh,
	Satananda Burla, Narayana Prasad, Akhil Goyal, Ray Kinsella,
	Dmitry Kozlyuk, Anatoly Burakov, Cristian Dumitrescu,
	Honnappa Nagarahalli, Mattias Rönnblom,
	Ruifeng Wang (Arm Technology China),
	David Christensen, Ananyev,  Konstantin, Olivier Matz,
	Jayatheerthan, Jay, Ashwin Sekhar Thalakalath Kottilveetil,
	Pavan Nikhilesh, David Marchand, tom

On Tue, Oct 19, 2021 at 21:36 Jerin Jacob <jerinjacobk@gmail.com> wrote:

> On Wed, Oct 20, 2021 at 12:38 AM Thomas Monjalon <thomas@monjalon.net> wrote:
> >
> > 19/10/2021 20:14, jerinj@marvell.com:
> > > Definition of Dataplane Workload Accelerator
> > > --------------------------------------------
> > > Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> > > Network controllers and programmable data acceleration engines for
> > > packet processing, cryptography, regex engines, baseband processing, etc.
> > > This allows DWA to offload  compute/packet processing/baseband/
> > > cryptography-related workload from the host CPU to save the cost and power.
> > > Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
> > >
> > > Unlike other devices in DPDK, the DWA device is not fixed-function
> > > due to the fact that it has CPUs and programmable HW accelerators.
> > > This enables DWA personality/workload to be completely programmable.
> > > Typical examples of DWA offloads are Flow/Session management,
> > > Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.
> >
> > If I understand well, the idea is to abstract the offload
> > of some stack layers in the hardware.
>
> Yes. It may not just HW, For expressing the complicated workloads
> may need CPU and/or other HW accelerators.
>
> > I am not sure we should give an API for such stack layers in DPDK.
>
> Why not?
>
> > It looks to be the role of the dataplane application to finely manage
> > how to use the hardware for a specific dataplane.
>
> It is possible with this scheme.
>
> > I believe the API for such layer would be either too big, or too limited,
> > or not optimized for specific needs.
>
> It will be optimized for specific needs as applications ask for what to do?
> not how to do?
>
> > If we really want to automate or abstract the HW/SW co-design,
> > I think we should better look at compiler work like P4 or PANDA.
>
> The compiler stuff is very static in nature. It can address the packet
> transformation
> workloads. Not the ones like IPsec or baseband offload.
> Another way to look at it, GPU RFC started just because you are not able
> to express all the workload in P4.
>

That’s not the purpose of the GPU RFC.
gpudev library goal is to enhance the dialog between GPU, CPU and NIC offering the possibility to:

- Have DPDK aware of non-CPU memory like device memory (e.g. similarly to what happened with MPI)
- Hide some memory management GPU library specific implementation details
- Reduce the gap between network activity and device activity (e.g. receive/send packets directly using the device memory)
- Reduce the gap between CPU activity and application-defined GPU workload
- Open to the capability to interact with the GPU device, not managing it

gpudev library can be easily embedded in any GPU specific application with a relatively small effort.
The application can allocate, communicate and manage the memory with the device transparently through DPDK.
What you are providing here is different and out of the scope of the gpudev library: control and manage the workload submission of possibly any
accelerator device, hiding a lot of implementation details within DPDK.
A wrapper for accelerator devices specific libraries and I think that it’s too far to be realistic.
As a GPU user, I don’t want to delegate my tasks to DWA because it can’t be fully optimized, updated to the latest GPU specific feature, etc..

Additionally, a generic DWA won't work for a GPU:
- Memory copies of DWA to CPU / CPU to DWA is latency expensive. Packets can directly be received in device memory
- When launching multiple processing blocks, efficiency may be compromised

I don’t actually see a real comparison between gpudev and DWA.
If in the future we’ll expose some GPU workload through the gpudev library, it will be for some network specific and well-defined problems.

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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-22 12:00     ` Elena Agostini
@ 2021-10-22 13:39       ` Jerin Jacob
  0 siblings, 0 replies; 20+ messages in thread
From: Jerin Jacob @ 2021-10-22 13:39 UTC (permalink / raw)
  To: Elena Agostini
  Cc: NBU-Contact-Thomas Monjalon, Jerin Jacob, dpdk-dev, Ferruh Yigit,
	Ajit Khaparde, Andrew Boyer, Andrew Rybchenko, Beilei Xing,
	Richardson, Bruce, Chas Williams, Xia, Chenbo, Ciara Loftus,
	Devendra Singh Rawat, Ed Czeck, Evgeny Schemeilin, Gaetan Rivet,
	Gagandeep Singh, Guoyang Zhou, Haiyue Wang, Harman Kalra,
	heinrich.kuhn, Hemant Agrawal, Hyong Youb Kim, Igor Chauskin,
	Igor Russkikh, Jakub Grajciar, Jasvinder Singh, Jian Wang,
	Jiawen Wu, Jingjing Wu, John Daley, John Miller,
	John W. Linville, Wiles, Keith, Kiran Kumar K, Lijun Ou,
	Liron Himi, NBU-Contact-longli, Marcin Wojtas, Martin Spinler,
	Matan Azrad, Matt Peters, Maxime Coquelin, Michal Krawczyk,
	Min Hu (Connor, Pradeep Kumar Nalla, Nithin Dabilpuram,
	Qiming Yang, Qi Zhang, Radha Mohan Chintakuntla,
	Rahul Lakkireddy, Rasesh Mody, Rosen Xu, Sachin Saxena,
	Satha Koteswara Rao Kottidi, Shahed Shaikh, Shai Brandes,
	Shepard Siegel, Somalapuram Amaranath, Somnath Kotur,
	Stephen Hemminger, Steven Webster, Sunil Kumar Kori,
	Tetsuya Mukawa, Veerasenareddy Burru, Slava Ovsiienko, Xiao Wang,
	Xiaoyun Wang, Yisen Zhuang, Yong Wang, Ziyang Xuan,
	Prasun Kapoor, nadavh, Satananda Burla, Narayana Prasad,
	Akhil Goyal, Ray Kinsella, Dmitry Kozlyuk, Anatoly Burakov,
	Cristian Dumitrescu, Honnappa Nagarahalli, Mattias Rönnblom,
	Ruifeng Wang (Arm Technology China),
	David Christensen, Ananyev, Konstantin, Olivier Matz,
	Jayatheerthan, Jay, Ashwin Sekhar Thalakalath Kottilveetil,
	Pavan Nikhilesh, David Marchand, tom

On Fri, Oct 22, 2021 at 5:30 PM Elena Agostini <eagostini@nvidia.com> wrote:
>
> On Tue, Oct 19, 2021 at 21:36 Jerin Jacob <jerinjacobk@gmail.com> wrote:
>
>
>
> > On Wed, Oct 20, 2021 at 12:38 AM Thomas Monjalon <thomas@monjalon.net> wrote:
>
> > >
>
> > > 19/10/2021 20:14, jerinj@marvell.com:
>
> > > > Definition of Dataplane Workload Accelerator
>
> > > > --------------------------------------------
>
> > > > Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
>
> > > > Network controllers and programmable data acceleration engines for
>
> > > > packet processing, cryptography, regex engines, baseband processing, etc.
>
> > > > This allows DWA to offload  compute/packet processing/baseband/
>
> > > > cryptography-related workload from the host CPU to save the cost and power.
>
> > > > Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
>
> > > >
>
> > > > Unlike other devices in DPDK, the DWA device is not fixed-function
>
> > > > due to the fact that it has CPUs and programmable HW accelerators.
>
> > > > This enables DWA personality/workload to be completely programmable.
>
> > > > Typical examples of DWA offloads are Flow/Session management,
>
> > > > Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.
>
> > >
>
> > > If I understand well, the idea is to abstract the offload
>
> > > of some stack layers in the hardware.
>
> >
>
> > Yes. It may not just HW, For expressing the complicated workloads
>
> > may need CPU and/or other HW accelerators.
>
> >
>
> > > I am not sure we should give an API for such stack layers in DPDK.
>
> >
>
> > Why not?
>
> >
>
> > > It looks to be the role of the dataplane application to finely manage
>
> > > how to use the hardware for a specific dataplane.
>
> >
>
> > It is possible with this scheme.
>
> >
>
> > > I believe the API for such layer would be either too big, or too limited,
>
> > > or not optimized for specific needs.
>
> >
>
> > It will be optimized for specific needs as applications ask for what to do?
>
> > not how to do?
>
> >
>
> > > If we really want to automate or abstract the HW/SW co-design,
>
> > > I think we should better look at compiler work like P4 or PANDA.
>
> >
>
> > The compiler stuff is very static in nature. It can address the packet
>
> > transformation
>
> > workloads. Not the ones like IPsec or baseband offload.
>
> > Another way to look at it, GPU RFC started just because you are not able
>
> > to express all the workload in P4.
>
> >
>
>
>
> That’s not the purpose of the GPU RFC.
>
> gpudev library goal is to enhance the dialog between GPU, CPU and NIC offering the possibility to:
>
>
>
> - Have DPDK aware of non-CPU memory like device memory (e.g. similarly to what happened with MPI)
>
> - Hide some memory management GPU library specific implementation details
>
> - Reduce the gap between network activity and device activity (e.g. receive/send packets directly using the device memory)
>
> - Reduce the gap between CPU activity and application-defined GPU workload
>
> - Open to the capability to interact with the GPU device, not managing it

Agree. I am not advocating P4 as the replacement for gpulib or DWA. If
someone thinks possible. It would be
great to how to express that for complex workload like TLS offload or
ORAN 7.2 split highphy baseband offload etc.

Could you give more details on "Open to the capability to interact
with the GPU device, not managing it"
What do you mean by managing it and what this RFC doing to manage it?


>
>
>
> gpudev library can be easily embedded in any GPU specific application with a relatively small effort.
>
> The application can allocate, communicate and manage the memory with the device transparently through DPDK.

See below


>
> What you are providing here is different and out of the scope of the gpudev library: control and manage the workload submission of possibly any
>
> accelerator device, hiding a lot of implementation details within DPDK.

No. it has both control and user plane. which also allows an
implementation to allocate, communicate and manage the memory with the
device transparently through DPDK
using user action. TLV messages can be at level. We can define the
profile from a low and higher level based on
what feature we need to offload. Or chain the multiple small profiles
to create complex workloads.


>
> A wrapper for accelerator devices specific libraries and I think that it’s too far to be realistic.
>
> As a GPU user, I don’t want to delegate my tasks to DWA because it can’t be fully optimized, updated to the latest GPU specific feature, etc..

DWA is the GPU.Task are expressed in generic representation, so it can
be optimized for GPU/DPU/IPU based on accelerator speciifics.


>
>
>
> Additionally, a generic DWA won't work for a GPU:
>
> - Memory copies of DWA to CPU / CPU to DWA is latency expensive. Packets can directly be received in device memory

No copy involved. The host port is just an abstract model. You can use
just shared memory as underneath.
Also, If you see the RFC, We can add new host ports that are specific
to the category for transport(Ethernet, PCIe, Shared memory)

>
> - When launching multiple processing blocks, efficiency may be compromised

How you are avoiding that with gpulib, the same logic can be moved to
driver implementation. Right?

>
>
>
> I don’t actually see a real comparison between gpudev and DWA.
>
> If in the future we’ll expose some GPU workload through the gpudev library, it will be for some network specific and well-defined problems.

How do you want to represent the "network specific" and "well-defined"
problem from application PoV.

The problem, I am trying to address, if every vendor express the
workload in accelerator specific fashion then we need
N library and N application code to solve a single problem,

I have provided an example for L3FWD, it will be good to know, how it
can not map to GPU.
Such level of depth discussion will give more ideas instead of an
abstract level. Or you can take up
a workload that can be NOT expressed with DWA RFC. That helps to
understand the gap.

I think,  TB board/DPDK community needs to decide the direction
following questions

1)  Agree/Disagree on the need for workload offload accelerators in DPDK.

2)  Do we need to expose accelerator-specific workload libraries (ie
separate libraries for GPU, DPU etc) let the _DPDK_
application deal with using acceleration-specific API for the
workload. If the majority thinks yes, In such case,
we can have dpudev library in addition to gpudev, basically, it will
be removing the profile concept from this RFC.

3)  Allow accelerator-specific libraries and DWA kind of model and
application to pick the model they want.

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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-19 18:14 [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library jerinj
  2021-10-19 18:14 ` [dpdk-dev] [RFC PATCH 1/1] dwa: introduce dataplane workload accelerator subsystem jerinj
  2021-10-19 19:08 ` [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library Thomas Monjalon
@ 2021-10-25  7:35 ` Mattias Rönnblom
  2021-10-25  9:03   ` Jerin Jacob
  2 siblings, 1 reply; 20+ messages in thread
From: Mattias Rönnblom @ 2021-10-25  7:35 UTC (permalink / raw)
  To: jerinj, dev
  Cc: thomas, ferruh.yigit, ajit.khaparde, aboyer, andrew.rybchenko,
	beilei.xing, bruce.richardson, chas3, chenbo.xia, ciara.loftus,
	dsinghrawat, ed.czeck, evgenys, grive, g.singh, zhouguoyang,
	haiyue.wang, hkalra, heinrich.kuhn, hemant.agrawal, hyonkim,
	igorch, irusskikh, jgrajcia, jasvinder.singh, jianwang, jiawenwu,
	jingjing.wu, johndale, john.miller, linville, keith.wiles,
	kirankumark, oulijun, lironh, longli, mw, spinler, matan,
	matt.peters, maxime.coquelin, mk, humin29, pnalla, ndabilpuram,
	qiming.yang, qi.z.zhang, radhac, rahul.lakkireddy, rmody,
	rosen.xu, sachin.saxena, skoteshwar, shshaikh, shaibran,
	shepard.siegel, asomalap, somnath.kotur, sthemmin,
	steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, ruifeng.wang, drc, konstantin.ananyev,
	olivier.matz, jay.jayatheerthan, asekhar, pbhagavatula,
	Elana Agostini

On 2021-10-19 20:14, jerinj@marvell.com wrote:
> From: Jerin Jacob <jerinj@marvell.com>
>
>
> Dataplane Workload Accelerator library
> ======================================
>
> Definition of Dataplane Workload Accelerator
> --------------------------------------------
> Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> Network controllers and programmable data acceleration engines for
> packet processing, cryptography, regex engines, baseband processing, etc.
> This allows DWA to offload  compute/packet processing/baseband/
> cryptography-related workload from the host CPU to save the cost and power.
> Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
>
> Unlike other devices in DPDK, the DWA device is not fixed-function
> due to the fact that it has CPUs and programmable HW accelerators.


There are already several instances of DPDK devices with pure-software 
implementation. In this regard, a DPU/SmartNIC represents nothing new. 
What's new, it seems to me, is a much-increased need to 
configure/arrange the processing in complex manners, to avoid bouncing 
everything to the host CPU. Something like P4 or rte_flow-based hooks or 
some other kind of extension. The eventdev adapters solve the same 
problem (where on some systems packets go through the host CPU on their 
way to the event device, and others do not) - although on a *much* 
smaller scale.


"Not-fixed function" seems to call for more hot plug support in the 
device APIs. Such functionality could then be reused by anything that 
can be reconfigured dynamically (FPGAs, firmware-programmed 
accelerators, etc.), but which may not be able to serve as a RPC 
endpoint, like a SmartNIC.


DWA could be some kind of DPDK-internal framework for managing certain 
type of DPUs, but should it be exposed to the user application?


> This enables DWA personality/workload to be completely programmable.
> Typical examples of DWA offloads are Flow/Session management,
> Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.
> Motivation for the new library
> ------------------------------
> Even though, a lot of semiconductor vendors offers a different form of DWA,
> such as DPU(often called Smart-NIC), GPU, IPU, XPU, etc.,
> Due to the lack of standard APIs to "Define the workload" and
> "Communication between HOST and DWA", it is difficult for DPDK
> consumers to use them in a portable way across different DWA vendors
> and enable it in cloud environments.
>
>
> Contents of RFC
> ------------------
> This RFC attempts to define standard APIs for:
>
> 1) Definition of Profiles corresponding to well defined workloads, which includes
>     a set of TLV(Messages) as a request  and response scheme to define
>     the contract between host and DWA to offload a workload.
>     (See lib/dwa/rte_dwa_profile_* header files)
> 2) Discovery of a DWAs capabilities (e.g. which specific workloads it can support)
>     in a vendor independent fashion. (See rte_dwa_dev_disc_profiles())
> 3) Attaching a set of profiles to a DWA device(See rte_dwa_dev_attach())
> 4) A communication framework between Host and DWA(See rte_dwa_ctrl_op() for
>     control plane and rte_dwa_port_host_* for user plane)
> 5) Virtualization of DWA hardware and firmware (Use standard DPDK device/bus model)
> 6) Enablement of administrative functions such as FW updates,
>     resource partitioning in a DWA like items in global in
>     nature that is applicable for all DWA device under the DWA.
>     (See rte_dwa_profile_admin.h)
>
> Also, this RFC define the L3FWD profile to offload L3FWD workload to DWA.
> This RFC defines an ethernet-style host port for Host to DWA communication.
> Different host port types may be required to cover the large spectrum of DWA types as
> transports like PCIe DMA, Shared Memory, or Ethernet are fundamentally different,
> and optimal performance need host port specific APIs.
>
> The framework does not force an abstract of different transport interfaces as
> single API, instead, decouples TLV from the transport interface and focuses on
> defining the TLVs and leaving vendors to specify the host ports
> specific to their DWA architecture.
>
>    
> Roadmap
> -------
> 1) Address the comments for this RFC and enable the common code
> 2) SW drivers/infrastructure for `DWA` and `DWA device`
> as two separate DPDK processes over `memif` DPDK ethdev driver for
> L3FWD offload. This is to enable the framework without any special HW.
> 3) Example DWA device application for L3FWD profile.
> 4) Marvell DWA Device drivers.
> 5) Based on community interest new profile can be added in the future.
>
>
> DWA library framework
> ---------------------
>
> DWA components:
>
>                                                    +--> rte_dwa_port_host_*()
>                                                    |  (User Plane traffic as TLV)
>                                                    |
>                   +----------------------+         |   +--------------------+
>                   |                      |         |   | DPDK DWA Device[0] |
>                   |  +----------------+  |  Host Port  | +----------------+ |
>                   |  |                |  |<========+==>| |                | |
>                   |  |   Profile 0    |  |             | |   Profile X    | |
>                   |  |                |  |             | |                | |
>    <=============>|  +----------------+  | Control Port| +----------------+ |
>      DWA Port0    |  +----------------+  |<========+==>|                    |
>                   |  |                |  |         |   +--------------------+
>                   |  |   Profile 1    |  |         |
>                   |  |                |  |         +--> rte_dwa_ctrl_op()
>                   |  +----------------+  |         (Control Plane traffic as TLV)
>    <=============>|      Dataplane       |
>      DWA Port1    |      Workload        |
>                   |      Accelerator     |             +---------- ---------+
>                   |      (HW/FW/SW)      |             | DPDK DWA Device[N] |
>                   |                      |  Host Port  | +----------------+ |
>    <=============>|  +----------------+  |<===========>| |                | |
>      DWA PortN    |  |                |  |             | |   Profile Y    | |
>                   |  |    Profile N   |  |             | |           ^    | |
>                   |  |                |  | Control Port| +-----------|----+ |
>                   |  +-------|--------+  |<===========>|             |      |
>                   |          |           |             +-------------|------+
>                   +----------|-----------+                           |
>                              |                                       |
>                              +---------------------------------------+
>                                                         ^
>                                                         |
>                                                         +--rte_dwa_dev_attach()
>
>
> Dataplane Workload Accelerator: It is an abstract model. The model is
> capable of offloading the dataplane workload from application via
> DPDK API over host and control ports of a DWA device.
> Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> Network controllers, and programmable data acceleration engines for
> packet processing, cryptography, regex engines, base-band processing, etc.
> This allows DWA to offload compute/packet processing/base-band/cryptography-related
> workload from the host CPU to save cost and power. Also,
> enable scaling the workload by adding DWAs to the host CPU as needed.
>
> DWA device: A DWA can be sliced to N number of DPDK DWA device(s)
> based on the resources available in DWA.
> The DPDK API interface operates on the DPDK DWA device.
> It is a representation of a set of resources in DWA.
>
> TLV: TLV (tag-length-value) encoded data stream contain tag as
> message ID, followed by message length, and finally the message payload.
> The 32bit message ID consists of two parts, 16bit Tag and 16bit Subtag.
> The tag represents ID of the group of the similar message,
> whereas, subtag represents a message tag ID under the group.
>
> Control Port: Used for transferring the control plane TLVs. Every DPDK
> DWA device must have a control port. Only one outstanding TLV can be
> processed via this port by a single DWA device. This makes the control
> port suitable for the control plane.
>
> Host Port: Used for transferring the user plane TLVs.
> Ethernet, PCIe DMA, Shared Memory, etc.are the example of
> different transport mechanisms abstracted under the host port.
> The primary purpose of host port to decouple the user plane TLVs with
> underneath transport mechanism differences.
> Unlike control port, more than one outstanding TLVs can be processed by
> a single DWA device via this port.
> This makes, the host port transfer to be in asynchronous nature,
> to support large volumes and less latency user plane traffic.
>
> DWA Port: Used for transferring data between the external source and DWA.
> Ethernet, eCPRI are examples of DWA ports. Unlike host ports,
> the host CPU is not involved in transferring the data to/from DWA ports.
> These ports typically connected to the Network controller inside the
> DWA to transfer the traffic from the external source.
>
> TLV direction: `Host to DWA` and `DWA to Host` are the directions
> of TLV messages. The former one is specified as H2D, and the later one is
> specified as D2H. The H2D control TLVs, used for requesting DWA to perform
> specific action and D2H control TLVs are used to respond to the requested
> actions. The H2D user plane messages are used for transferring data from the
> host to the DWA. The D2H user plane messages are used for transferring
> data from the DWA to the host.
>
> DWA device states: Following are the different states of a DWA device.
> - READY: DWA Device is ready to attach the profile.
> See rte_dwa_dev_disc_profiles() API to discover the profile.
> - ATTACHED: DWA Device attached to one or more profiles.
> See rte_dwa_dev_attach() API to attach the profile(s).
> - STOPPED: Profile is in the stop state.
> TLV type `TYPE_ATTACHED`and `TYPE_STOPPED` messages are valid in this state.
> After rte_dwa_dev_attach() or explicitly invoking the rte_dwa_stop() API
> brings device to this state.
> - RUNNING: Invoking rte_dwa_start() brings the device to this state.
> TLV type `TYPE_STARTED` and `TYPE_USER_PLANE` are valid in this state.
> - DETACHED: Invoking rte_dwa_dev_detach() brings the device to this state.
> The device and profile must be in the STOPPED state prior to
> invoking the rte_dwa_dev_detach().
> - CLOSED: Closed a stopped/detached DWA device.The device cannot be restarted!.
> Invoking rte_dwa_dev_close() brings the device to this state.
>
> TLV types: Following are the different TLV types
> - TYPE_ATTACHED: Valid when the device is in `ATTACHED`, `STOPPED` and `RUNNING` state.
> - TYPE_STOPPED: Valid when the device is in `STOPPED` state.
> - TYPE_STARTED: Valid when the device is in `RUNNING` state.
> - TYPE_USER_PLANE: Valid when the device is in `RUNNING` state and
> used to transfer only user plane traffic.
>
> Profile: Specifies a workload that dataplane workload accelerator
> process on behalf of a DPDK application through a DPDK DWA device.
> A profile is expressed as a set of TLV messages for control plane and user plane
> functions. Each TLV message must have Tag, SubTag, Direction, Type, Payload attributes.
>
> Programming model: Typical application programming sequence is as follows,
> 1) In the EAL initialization phase, the DWA devices shall be probed,
>     the application can query the number of available DWA devices with
>     rte_dwa_dev_count() API.
> 2) Application discovers the available profile(s) in a DWA device using
>     rte_dwa_dev_disc_profiles() API.
> 3) Application attaches one or more profile(s) to a DWA device using
>     rte_dwa_dev_attach().
> 4) Once the profile is attached, The device shall be in the STOPPED state.
>     Configure the profile(s) with `TYPE_ATTACHED`and `TYPE_STOPPED`
>     type TLVs using rte_dwa_ctrl_op() API.
> 5) Once the profile is configured, move the profile to the `RUNNING` state
>     by invoking rte_dwa_start() API.
> 6) Once the profile is in running state and if it has user plane TLV,
>     transfer those TLVs using rte_dwa_port_host_() API based on the available
>     host port for the given profile attached.
> 7) Application can change the dynamic configuration aspects in
>     `RUNNING` state using rte_dwa_ctrl_op() API by issuing `TYPE_STARTED` type
>     of TLV messages.
> 8) Finally, use rte_dwa_stop(), rte_dwa_dev_detach(), rte_dwa_dev_close()
>     sequence for tear-down.
>
>
> L3FWD profile
> -------------
>
>                               +-------------->--[1]--------------+
>                               |                                  |
>                   +-----------|----------+                       |
>                   |           |          |                       |
>                   |  +--------|-------+  |                       |
>                   |  |                |  |                       |
>                   |  | L3FWD Profile  |  |                       |
>        \          |  |                |  |                       |
>    <====\========>|  +----------------+  |                       |
>      DWA \Port0   |     Lookup Table     |             +---------|----------+
>           \       |  +----------------+  |             | DPDK DWA|Device[0] |
>            \      |  | IP    | Dport  |  |  Host Port  | +-------|--------+ |
>             \     |  +----------------+  |<===========>| |       |        | |
>              +~[3]~~~|~~~~~~~|~~~~~~~~|~~~~~~~~~~~~~~~~~>|->L3FWD Profile | |
>    <=============>|  +----------------+  |             | |                | |
>      DWA Port1    |  |       |        |  | Control Port| +-|---------|----+ |
>                   |  +----------------+  |<===========>|   |         |      |
>      ~~~>~~[5]~~~~|~~|~~~+   |        |  |             +---|---------|------+
>                   |  +---+------------+  |                 |         |
>      ~~~<~~~~~~~~~|~~|~~~+   |        |<-|------[2]--------+         |
>                   |  +----------------+<-|------[4]------------------+
>                   |    Dataplane         |
>    <=============>|    Workload          |
>      DWA PortN    |    Accelerator       |
>                   |    (HW/FW/SW)        |
>                   +----------------------+
>
>
> L3FWD profile offloads Layer-3 forwarding between the DWA Ethernet ports.
>
> The above diagram depicts the profile and application programming sequence.
> 1) DWA device attaches the L3FWD profile using rte_dwa_dev_attach().
> 2) Configure the L3FWD profile:
> a) The application requests L3FWD profile capabilities of the DWA
>     by using RTE_DWA_STAG_PROFILE_L3FWD_H2D_INFO, On response,
>     the RTE_DWA_STAG_PROFILE_L3FWD_D2H_INFO returns the lookup modes
>     supported, max rules supported, and available host ports for this profile.	
> b) The application configures a set of DWA ports to use a
>     lookup mode(EM, LPM, or FIB) via RTE_DWA_STAG_PROFILE_L3FWD_H2D_CONFIG.
> c) The application configures a valid host port to receive exception packets.
> 3) The exception that is not matching forwarding table entry comes as
>     RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS TLV to host. DWA stores the exception
>     packet send back destination ports after completing step (4).
> 4) Parse the exception packet and add rules to the FWD table using
>     RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_ADD. If the application knows the rules beforehand,
>     it can add the rules in step 2.
> 5) When DWA ports receive the matching flows in the lookup table, DWA forwards
>     to DWA Ethernet ports without host CPU intervention.
>
>
> Example application usage with L3FWD profile
> --------------------------------------------
> This example application is to demonstrate the programming model of DWA library.
> This example omits the error checks to simply the application.
>
> void
> dwa_profile_l3fwd_add_rule(rte_dwa_obj_t obj obj, struct rte_mbuf *mbuf)
> {
> 	struct rte_dwa_profile_l3fwd_h2d_lookup_add *lookup;
> 	struct rte_dwa_tlv *h2d, *d2h;
> 	struct rte_ether_hdr *eth_hdr;
> 	struct rte_ipv4_hdr *ipv4_hdr;
> 	uint32_t id;
> 	size_t len;
>
> 	id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_LOOKUP_ADD);
> 	len = sizeof(struct rte_dwa_profile_l3fwd_h2d_config);
> 	h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
>
> 	lookup = h2d->msg;
>          /* Simply hardcode to IPv4 instead of looking for Packet type to simplify example */
> 	lookup->rule_type = RTE_DWA_PROFILE_L3FWD_RULE_TYPE_IPV4;
> 	lookup->v4_rule.prefix.depth = 24;
>
> 	eth_hdr = rte_pktmbuf_mtod(mbuf, struct rte_ether_hdr *);
> 	ipv4_hdr = (struct rte_ipv4_hdr *)(eth_hdr + 1);
> 	lookup->v4_rule.prefix.ip_dst = rte_be_to_cpu_32(ipv4_hdr->dst_addr);
> 	lookup->eth_port_dst = mbuf->port;
>
> 	rte_dwa_tlv_fill(h2d, id, len, h2d);
> 	d2h = rte_dwa_ctrl_op(obj, h2h);
> 	free(h2d);
> 	free(d2h);
> }
>
> void
> dwa_profile_l3fwd_port_host_ethernet_worker(rte_dwa_obj_t obj, struct app_ctx *ctx)
> {
> 	struct rte_dwa_profile_l3fwd_d2h_exception_pkts *msg;
> 	struct rte_dwa_tlv *tlv;
> 	uint16_t i, rc, nb_tlvs;
> 	struct rte_mbuf *mbuf;
>
> 	while (!ctx->done) {
> 		rc = rte_dwa_port_host_ethernet_rx(obj, 0, &tlv, 1);
> 		if (!rc)
> 			continue;
>
> 		/* Since L3FWD profile has only one User Plane TLV, Message must be
> 	         * RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS message
>          	 */
> 		msg = (struct rte_dwa_profile_l3fwd_d2h_exception_pkts *)tlv->msg;
> 		for (i = 0; i < msg->nb_pkts; i++) {
> 				mbuf = msg->pkts[i];
> 				/* Got a exception pkt from DWA, handle it by adding as new rule in
>                                   * lookup table in DWA
> 				 */ 				
> 				dwa_profile_l3fwd_add_rule(obj, mbuf);
> 				/* Free the mbuf to pool */
> 				rte_pktmbuf_free(mbuf);
> 		}
> 		
> 		/* Done with TLV mbuf container, free it back */
> 		rte_mempool_ops_enqueue_bulk(ctx->tlv_pool, tlv, 1);
> }
>
> bool
> dwa_port_host_ethernet_config(rte_dwa_obj_t obj, struct app_ctx *ctx)
> {
> 	struct rte_dwa_tlv info_h2d, *info_d2h, *h2d = NULL, *d2h;
> 	struct rte_dwa_port_host_ethernet_d2h_info *info;
> 	int tlv_pool_element_sz;
> 	bool rc = false;
> 	size_t len;
>
> 	/* Get the Ethernet host port info */
> 	id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_INFO);
> 	rte_dwa_tlv_fill(&info_h2d, id, 0, NULL);
> 	info_d2h = rte_dwa_ctrl_op(obj, &info_h2d)
>
> 	info = rte_dwa_tlv_d2h_to_msg(info_d2h);
> 	if (info == NULL)
> 		goto fail;
> 	/* Need min one Rx queue to Receive exception traffic */
> 	if (info->nb_rx_queues == 0)
> 		goto fail;
> 	/* Done with message from DWA. Free back to implementation */
> 	free(obj, info_d2h);
>
> 	/* Allocate exception packet pool */
> 	ctx->pkt_pool = rte_pktmbuf_pool_create("exception pool", /* Name */
>                                  ctx->pkt_pool_depth, /* Number of elements*/
>                                  512, /* Cache size*/
>                                  0,
>                                  RTE_MBUF_DEFAULT_BUF_SIZE,
>                                  ctx->socket_id));
>
>
> 	tlv_pool_element_sz = DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ * sizeof(rte_mbuf *);
> 	tlv_pool_element_sz  += sizeof(rte_dwa_profile_l3fwd_d2h_exception_pkts);
>
> 	/* Allocate TLV pool for RTE_DWA_STLV_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS_PACKETS tag */
> 	ctx->tlv_pool = rte_mempool_create("TLV pool", /* mempool name */
>                                  ctx->tlv_pool_depth, /* Number of elements*/
>                                  tlv_pool_element_sz, /* Element size*/
>                                  512, /* cache size*/
>                                  0, NULL, NULL, NULL /* Obj constructor */, NULL,
>                                  ctx->socket_id, 0 /* flags *);
>
>
> 	/* Configure Ethernet host port */
> 	id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_CONFIG);
> 	len = sizeof(struct rte_dwa_port_host_ethernet_config);
> 	h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
>
> 	cfg = h2d->msg;
> 	/* Update the Ethernet configuration parameters */
> 	cfg->nb_rx_queues = 1;
> 	cfg->nb_tx_queues = 0;
> 	cfg->max_burst = DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ;
> 	cfg->pkt_pool = ctx->pkt_pool;
> 	cfg->tlv_pool = ctx->tlv_pool;
> 	rte_dwa_tlv_fill(h2d, id, len, h2d);
> 	d2h = rte_dwa_ctrl_op(obj, h2d);
> 	if (d2h == NULL))
> 		goto fail;
>
> 	free(h2d);
>
> 	/* Configure Rx queue 0 receive expectation traffic */
> 	id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_QUEUE_CONFIG);
> 	len = sizeof(struct rte_dwa_port_host_ethernet_queue_config);
> 	h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
>
> 	cfg = h2d->msg;
> 	cfg->id = 0; /* 0th Queue */
> 	cfg->enable= 1;
> 	cfg->is_tx = 0; /* Rx queue */
> 	cfg->depth = ctx->rx_queue_depth;
> 	rte_dwa_tlv_fill(h2d, id, len, h2d);
> 	d2h = rte_dwa_ctrl_op(obj, h2d);
> 	if (d2h == NULL))
> 		goto fail;
>
> 	free(h2d);
>
> 	return true;
> fail:
> 	if (h2d)
> 		free(h2d);
> 	return rc;
> }
>
> bool
> dwa_profile_l3fwd_config(rte_dwa_obj_t obj, struct app_ctx *ctx)
> {
> 	struct rte_dwa_tlv info_h2d, *info_d2h = NULL, *h2d, *d2h = NULL;
> 	struct rte_dwa_port_dwa_ethernet_d2h_info *info;
> 	struct rte_dwa_profile_l3fwd_h2d_config *cfg;
> 	bool rc = false;
>   	uint32_t id;
> 	size_t len;
>
> 	/* Get DWA Ethernet port info */
> 	id = RTE_DWA_TLV_MK_ID(PORT_DWA_ETHERNET, H2D_INFO);
> 	rte_dwa_tlv_fill(&info_h2d, id, 0, NULL);
> 	info_d2h = rte_dwa_ctrl_op(obj, &info_h2d);
>
> 	info = rte_dwa_tlv_d2h_to_msg(info_d2h);
> 	if (info == NULL)
> 		goto fail;
> 	
> 	/* Not found any DWA ethernet ports */
> 	if (info->nb_ports == 0)
> 		goto fail;
>
> 	/* Configure L3FWD profile */
> 	id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_CONFIG);
> 	len = sizeof(struct rte_dwa_profile_l3fwd_h2d_config) + (sizeof(uint16_t) * info->nb_ports);
> 	h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
>
> 	cfg = h2d->msg;
> 	/* Update the L3FWD configuration parameters */
> 	cfg->mode = ctx->mode;
> 	/* Attach all DWA Ethernet ports onto L3FWD profile */
> 	cfg->nb_eth_ports = info->nb_ports;
> 	memcpy(cfg->eth_ports, info->avail_ports, sizeof(uint16_t) * info->nb_ports);
>
> 	rte_dwa_tlv_fill(h2d, id, len, h2d);
> 	d2h = rte_dwa_ctrl_op(obj, h2d);
> 	free(h2d);
>
> 	/* All good */
> 	rc = true;
> fail:
> 	if (info_d2h)
> 		free(obj, info_d2h);
> 	if (d2h)
> 		free(obj, d2h);
>
> 	return rc;
> }
>
> bool
> dwa_profile_l3fwd_has_capa(rte_dwa_obj_t obj, struct app_ctx *ctx)
> {
> 	struct rte_dwa_profile_l3fwd_d2h_info *info;
> 	struct rte_dwa_tlv h2d, *d2h;
> 	bool found = false;
>   	uint32_t id;
>
> 	/* Get L3FWD profile info */
> 	id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_INFO);
> 	rte_dwa_tlv_fill(&h2d, id, 0, NULL);
> 	d2h = rte_dwa_ctrl_op(obj, &h2d);
>
> 	info = rte_dwa_tlv_d2h_to_msg(d2h);
> 	/* Request failed */
> 	if (info == NULL)
> 		goto fail;
> 	/* Required lookup modes is not supported */
> 	if (!(info->modes_supported & ctx->mode))
> 		goto fail;
>
> 	/* Check profile supports HOST_ETHERNET port as this application
>           * supports only host port as Ethernet
>           */
> 	for (i = 0; i < info->nb_host_ports; i++) {
> 		if (info->host_ports[i] == RTE_DWA_TAG_PORT_HOST_ETHERNET); {
> 			found = true;
> 		}	
> 	}
>
> 	/* Done with response, Free the d2h memory allocated by implementation */
> 	free(obj, d2h);
> fail:
> 	return found;
> }
>
>
> bool
> dwa_has_profile(enum rte_dwa_tag_profile pf)
> {
> 	enum rte_dwa_tlv_profile *pfs = NULL;
> 	bool found = false;
> 	int nb_pfs;
>
> 	/* Get the number of profiles on the DWA device */
> 	nb_pfs = rte_dwa_dev_disc_profiles(0, NULL);
> 	pfs = malloc(sizeof(enum rte_dwa_tag_profile)  * nb_pfs);
> 	/* Fetch all the profiles */
> 	nb_pfs = rte_dwa_dev_disc_profiles(0, pfs);
>
> 	/* Check the list has requested profile */
> 	for (i = 0; i < nb_pfs; i++) {
> 		if (pfs[i] == pf);
> 			found = true;
> 	}
> 	free(pfs);
>
>
> 	return found;
> }
>
>
> #include <rte_dwa.h>
>
> #define DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ		32
>
> struct app_ctx {
> 	bool done;
> 	struct rte_mempool *pkt_pool;
> 	struct rte_mempool *tlv_pool;
> 	enum rte_dwa_profile_l3fwd_lookup_mode mode;
> 	int socket_id;
> 	int pkt_pool_depth;	
> 	int tlv_pool_depth;
> 	int rx_queue_depth;
> } __rte_cache_aligned;
>
> int
> main(int argc, char **argv)
> {
> 	rte_dwa_obj_t obj = NULL;
> 	struct app_ctx ctx;
> 	int rc;
> 		
> 	/* Initialize EAL */
> 	rc= rte_eal_init(argc, argv);
>          if (rc < 0)
>                rte_exit(EXIT_FAILURE, "Invalid EAL arguments\n");
>          argc -= ret;
>          argv += ret;
>
>
> 	memset(&ctx, 0, sizeof(ctx));
> 	/* Set application default values */
> 	ctx->mode = RTE_DWA_PROFILE_L3FWD_MODE_LPM;
> 	ctx->socket_id = SOCKET_ID_ANY;
> 	ctx->pkt_pool_depth = 10000;
> 	ctx->tlv_pool_depth = 10000;
> 	ctx->rx_queue_depth = 10000;
>
> 	/* Step 1: Check any DWA devices present  */
> 	rc = rte_dwa_dev_count();
> 	if (rc <= 0)
> 		rte_exit(EXIT_FAILURE, "Failed to find DWA devices\n");
>
> 	/* Step 2: Check DWA device has L3FWD profile or not */
> 	if (!dwa_has_profile(RTE_DWA_TAG_PROFILE_L3FWD))
> 		rte_exit(EXIT_FAILURE, "L3FWD profile not found\n");
>
> 	/*
>   	 * Step 3: Now that, workload accelerator has L3FWD profile,
>   	 * offload L3FWD workload to accelerator by attaching the profile
> 	 * to accelerator.
>   	 */
> 	enum rte_dwa_tlv_profile profile[] = {RTE_DWA_TAG_PROFILE_L3FWD};
> 	obj = rte_dwa_dev_attach(0, "my_custom_accelerator_device", profile, 1).;
>
> 	/* Step 4: Check Attached L3FWD profile has required capability to proceed */
> 	if (!dwa_profile_l3fwd_has_capa(obj, &ctx))
> 		rte_exit(EXIT_FAILURE, "L3FWD profile does not have enough capability \n");
>
> 	/* Step 5: Configure l3fwd profile */
> 	if (!dwa_profile_l3fwd_config(obj, &ctx))
> 		rte_exit(EXIT_FAILURE, "L3FWD profile configure failed \n");
>
> 	/* Step 6: Configure ethernet host port to receive exception packets */
> 	if (!dwa_port_host_ethernet_config(obj, &ctx))
> 		rte_exit(EXIT_FAILURE, "L3FWD profile configure failed \n");
>
> 	/* Step 7 : Move DWA profiles to start state */
> 	rte_dwa_start(obj);
>
> 	/* Step 8: Handle expectation packets and add lookup rules for it */
> 	dwa_profile_l3fwd_port_host_ethernet_worker(obj, &ctx);
>
> 	/* Step 9: Clean up */
> 	rte_dwa_stop(obj);
> 	rte_dwa_dev_detach(0, obj);
> 	rte_dwa_dev_close(0);
> 	
> 	return 0;
> }
>
>
> Jerin Jacob (1):
>    dwa: introduce dataplane workload accelerator subsystem
>
>   doc/api/doxy-api-index.md            |  13 +
>   doc/api/doxy-api.conf.in             |   1 +
>   lib/dwa/dwa.c                        |   7 +
>   lib/dwa/meson.build                  |  17 ++
>   lib/dwa/rte_dwa.h                    | 184 +++++++++++++
>   lib/dwa/rte_dwa_core.h               | 264 +++++++++++++++++++
>   lib/dwa/rte_dwa_dev.h                | 154 +++++++++++
>   lib/dwa/rte_dwa_port_dwa_ethernet.h  |  68 +++++
>   lib/dwa/rte_dwa_port_host_ethernet.h | 178 +++++++++++++
>   lib/dwa/rte_dwa_profile_admin.h      |  85 ++++++
>   lib/dwa/rte_dwa_profile_l3fwd.h      | 378 +++++++++++++++++++++++++++
>   lib/dwa/version.map                  |   3 +
>   lib/meson.build                      |   1 +
>   13 files changed, 1353 insertions(+)
>   create mode 100644 lib/dwa/dwa.c
>   create mode 100644 lib/dwa/meson.build
>   create mode 100644 lib/dwa/rte_dwa.h
>   create mode 100644 lib/dwa/rte_dwa_core.h
>   create mode 100644 lib/dwa/rte_dwa_dev.h
>   create mode 100644 lib/dwa/rte_dwa_port_dwa_ethernet.h
>   create mode 100644 lib/dwa/rte_dwa_port_host_ethernet.h
>   create mode 100644 lib/dwa/rte_dwa_profile_admin.h
>   create mode 100644 lib/dwa/rte_dwa_profile_l3fwd.h
>   create mode 100644 lib/dwa/version.map
>


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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-25  7:35 ` Mattias Rönnblom
@ 2021-10-25  9:03   ` Jerin Jacob
  2021-10-29 11:57     ` Mattias Rönnblom
  0 siblings, 1 reply; 20+ messages in thread
From: Jerin Jacob @ 2021-10-25  9:03 UTC (permalink / raw)
  To: Mattias Rönnblom
  Cc: jerinj, dev, thomas, ferruh.yigit, ajit.khaparde, aboyer,
	andrew.rybchenko, beilei.xing, bruce.richardson, chas3,
	chenbo.xia, ciara.loftus, dsinghrawat, ed.czeck, evgenys, grive,
	g.singh, zhouguoyang, haiyue.wang, hkalra, heinrich.kuhn,
	hemant.agrawal, hyonkim, igorch, irusskikh, jgrajcia,
	jasvinder.singh, jianwang, jiawenwu, jingjing.wu, johndale,
	john.miller, linville, keith.wiles, kirankumark, oulijun, lironh,
	longli, mw, spinler, matan, matt.peters, maxime.coquelin, mk,
	humin29, pnalla, ndabilpuram, qiming.yang, qi.z.zhang, radhac,
	rahul.lakkireddy, rmody, rosen.xu, sachin.saxena, skoteshwar,
	shshaikh, shaibran, shepard.siegel, asomalap, somnath.kotur,
	sthemmin, steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, ruifeng.wang, drc, konstantin.ananyev,
	olivier.matz, jay.jayatheerthan, asekhar, pbhagavatula,
	Elana Agostini

On Mon, Oct 25, 2021 at 1:05 PM Mattias Rönnblom
<mattias.ronnblom@ericsson.com> wrote:
>
> On 2021-10-19 20:14, jerinj@marvell.com wrote:
> > From: Jerin Jacob <jerinj@marvell.com>
> >
> >
> > Dataplane Workload Accelerator library
> > ======================================
> >
> > Definition of Dataplane Workload Accelerator
> > --------------------------------------------
> > Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> > Network controllers and programmable data acceleration engines for
> > packet processing, cryptography, regex engines, baseband processing, etc.
> > This allows DWA to offload  compute/packet processing/baseband/
> > cryptography-related workload from the host CPU to save the cost and power.
> > Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
> >
> > Unlike other devices in DPDK, the DWA device is not fixed-function
> > due to the fact that it has CPUs and programmable HW accelerators.
>
>
> There are already several instances of DPDK devices with pure-software
> implementation. In this regard, a DPU/SmartNIC represents nothing new.
> What's new, it seems to me, is a much-increased need to
> configure/arrange the processing in complex manners, to avoid bouncing
> everything to the host CPU.

Yes and No. It will be based on the profile. The TLV type TYPE_USER_PLANE will
have user plane traffic from/to host. For example, offloading ORAN split 7.2
baseband profile. Transport blocks sent to/from host as TYPE_USER_PLANE.

> Something like P4 or rte_flow-based hooks or
> some other kind of extension. The eventdev adapters solve the same
> problem (where on some systems packets go through the host CPU on their
> way to the event device, and others do not) - although on a *much*
> smaller scale.

Yes. Eventdev Adapters only for event device plumbing.


>
>
> "Not-fixed function" seems to call for more hot plug support in the
> device APIs. Such functionality could then be reused by anything that
> can be reconfigured dynamically (FPGAs, firmware-programmed
> accelerators, etc.),

Yes.

> but which may not be able to serve as a RPC
> endpoint, like a SmartNIC.

It can. That's the reason for choosing TLVs. So that
any higher level language can use TLVs like https://github.com/ustropo/uttlv
to communicate with the accelerator.  TLVs follow the request and
response scheme like RPC. So it can warp it under application if needed.

>
>
> DWA could be some kind of DPDK-internal framework for managing certain
> type of DPUs, but should it be exposed to the user application?


Could you clarify a bit more.
The offload is represented as a set of TLVs in generic fashion. There
is no DPU specific bit in offload representation. See
rte_dwa_profiile_l3fwd.h header file.

TB hosted a meeting for this at Date: Wednesday, October 27th Time:
3pm UTC, https://meet.jit.si/DPDK
Feel free to join.


>
>
> > This enables DWA personality/workload to be completely programmable.
> > Typical examples of DWA offloads are Flow/Session management,
> > Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.
> > Motivation for the new library
> > ------------------------------
> > Even though, a lot of semiconductor vendors offers a different form of DWA,
> > such as DPU(often called Smart-NIC), GPU, IPU, XPU, etc.,
> > Due to the lack of standard APIs to "Define the workload" and
> > "Communication between HOST and DWA", it is difficult for DPDK
> > consumers to use them in a portable way across different DWA vendors
> > and enable it in cloud environments.
> >
> >
> > Contents of RFC
> > ------------------
> > This RFC attempts to define standard APIs for:
> >
> > 1) Definition of Profiles corresponding to well defined workloads, which includes
> >     a set of TLV(Messages) as a request  and response scheme to define
> >     the contract between host and DWA to offload a workload.
> >     (See lib/dwa/rte_dwa_profile_* header files)
> > 2) Discovery of a DWAs capabilities (e.g. which specific workloads it can support)
> >     in a vendor independent fashion. (See rte_dwa_dev_disc_profiles())
> > 3) Attaching a set of profiles to a DWA device(See rte_dwa_dev_attach())
> > 4) A communication framework between Host and DWA(See rte_dwa_ctrl_op() for
> >     control plane and rte_dwa_port_host_* for user plane)
> > 5) Virtualization of DWA hardware and firmware (Use standard DPDK device/bus model)
> > 6) Enablement of administrative functions such as FW updates,
> >     resource partitioning in a DWA like items in global in
> >     nature that is applicable for all DWA device under the DWA.
> >     (See rte_dwa_profile_admin.h)
> >
> > Also, this RFC define the L3FWD profile to offload L3FWD workload to DWA.
> > This RFC defines an ethernet-style host port for Host to DWA communication.
> > Different host port types may be required to cover the large spectrum of DWA types as
> > transports like PCIe DMA, Shared Memory, or Ethernet are fundamentally different,
> > and optimal performance need host port specific APIs.
> >
> > The framework does not force an abstract of different transport interfaces as
> > single API, instead, decouples TLV from the transport interface and focuses on
> > defining the TLVs and leaving vendors to specify the host ports
> > specific to their DWA architecture.
> >
> >
> > Roadmap
> > -------
> > 1) Address the comments for this RFC and enable the common code
> > 2) SW drivers/infrastructure for `DWA` and `DWA device`
> > as two separate DPDK processes over `memif` DPDK ethdev driver for
> > L3FWD offload. This is to enable the framework without any special HW.
> > 3) Example DWA device application for L3FWD profile.
> > 4) Marvell DWA Device drivers.
> > 5) Based on community interest new profile can be added in the future.
> >
> >
> > DWA library framework
> > ---------------------
> >
> > DWA components:
> >
> >                                                    +--> rte_dwa_port_host_*()
> >                                                    |  (User Plane traffic as TLV)
> >                                                    |
> >                   +----------------------+         |   +--------------------+
> >                   |                      |         |   | DPDK DWA Device[0] |
> >                   |  +----------------+  |  Host Port  | +----------------+ |
> >                   |  |                |  |<========+==>| |                | |
> >                   |  |   Profile 0    |  |             | |   Profile X    | |
> >                   |  |                |  |             | |                | |
> >    <=============>|  +----------------+  | Control Port| +----------------+ |
> >      DWA Port0    |  +----------------+  |<========+==>|                    |
> >                   |  |                |  |         |   +--------------------+
> >                   |  |   Profile 1    |  |         |
> >                   |  |                |  |         +--> rte_dwa_ctrl_op()
> >                   |  +----------------+  |         (Control Plane traffic as TLV)
> >    <=============>|      Dataplane       |
> >      DWA Port1    |      Workload        |
> >                   |      Accelerator     |             +---------- ---------+
> >                   |      (HW/FW/SW)      |             | DPDK DWA Device[N] |
> >                   |                      |  Host Port  | +----------------+ |
> >    <=============>|  +----------------+  |<===========>| |                | |
> >      DWA PortN    |  |                |  |             | |   Profile Y    | |
> >                   |  |    Profile N   |  |             | |           ^    | |
> >                   |  |                |  | Control Port| +-----------|----+ |
> >                   |  +-------|--------+  |<===========>|             |      |
> >                   |          |           |             +-------------|------+
> >                   +----------|-----------+                           |
> >                              |                                       |
> >                              +---------------------------------------+
> >                                                         ^
> >                                                         |
> >                                                         +--rte_dwa_dev_attach()
> >
> >
> > Dataplane Workload Accelerator: It is an abstract model. The model is
> > capable of offloading the dataplane workload from application via
> > DPDK API over host and control ports of a DWA device.
> > Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> > Network controllers, and programmable data acceleration engines for
> > packet processing, cryptography, regex engines, base-band processing, etc.
> > This allows DWA to offload compute/packet processing/base-band/cryptography-related
> > workload from the host CPU to save cost and power. Also,
> > enable scaling the workload by adding DWAs to the host CPU as needed.
> >
> > DWA device: A DWA can be sliced to N number of DPDK DWA device(s)
> > based on the resources available in DWA.
> > The DPDK API interface operates on the DPDK DWA device.
> > It is a representation of a set of resources in DWA.
> >
> > TLV: TLV (tag-length-value) encoded data stream contain tag as
> > message ID, followed by message length, and finally the message payload.
> > The 32bit message ID consists of two parts, 16bit Tag and 16bit Subtag.
> > The tag represents ID of the group of the similar message,
> > whereas, subtag represents a message tag ID under the group.
> >
> > Control Port: Used for transferring the control plane TLVs. Every DPDK
> > DWA device must have a control port. Only one outstanding TLV can be
> > processed via this port by a single DWA device. This makes the control
> > port suitable for the control plane.
> >
> > Host Port: Used for transferring the user plane TLVs.
> > Ethernet, PCIe DMA, Shared Memory, etc.are the example of
> > different transport mechanisms abstracted under the host port.
> > The primary purpose of host port to decouple the user plane TLVs with
> > underneath transport mechanism differences.
> > Unlike control port, more than one outstanding TLVs can be processed by
> > a single DWA device via this port.
> > This makes, the host port transfer to be in asynchronous nature,
> > to support large volumes and less latency user plane traffic.
> >
> > DWA Port: Used for transferring data between the external source and DWA.
> > Ethernet, eCPRI are examples of DWA ports. Unlike host ports,
> > the host CPU is not involved in transferring the data to/from DWA ports.
> > These ports typically connected to the Network controller inside the
> > DWA to transfer the traffic from the external source.
> >
> > TLV direction: `Host to DWA` and `DWA to Host` are the directions
> > of TLV messages. The former one is specified as H2D, and the later one is
> > specified as D2H. The H2D control TLVs, used for requesting DWA to perform
> > specific action and D2H control TLVs are used to respond to the requested
> > actions. The H2D user plane messages are used for transferring data from the
> > host to the DWA. The D2H user plane messages are used for transferring
> > data from the DWA to the host.
> >
> > DWA device states: Following are the different states of a DWA device.
> > - READY: DWA Device is ready to attach the profile.
> > See rte_dwa_dev_disc_profiles() API to discover the profile.
> > - ATTACHED: DWA Device attached to one or more profiles.
> > See rte_dwa_dev_attach() API to attach the profile(s).
> > - STOPPED: Profile is in the stop state.
> > TLV type `TYPE_ATTACHED`and `TYPE_STOPPED` messages are valid in this state.
> > After rte_dwa_dev_attach() or explicitly invoking the rte_dwa_stop() API
> > brings device to this state.
> > - RUNNING: Invoking rte_dwa_start() brings the device to this state.
> > TLV type `TYPE_STARTED` and `TYPE_USER_PLANE` are valid in this state.
> > - DETACHED: Invoking rte_dwa_dev_detach() brings the device to this state.
> > The device and profile must be in the STOPPED state prior to
> > invoking the rte_dwa_dev_detach().
> > - CLOSED: Closed a stopped/detached DWA device.The device cannot be restarted!.
> > Invoking rte_dwa_dev_close() brings the device to this state.
> >
> > TLV types: Following are the different TLV types
> > - TYPE_ATTACHED: Valid when the device is in `ATTACHED`, `STOPPED` and `RUNNING` state.
> > - TYPE_STOPPED: Valid when the device is in `STOPPED` state.
> > - TYPE_STARTED: Valid when the device is in `RUNNING` state.
> > - TYPE_USER_PLANE: Valid when the device is in `RUNNING` state and
> > used to transfer only user plane traffic.
> >
> > Profile: Specifies a workload that dataplane workload accelerator
> > process on behalf of a DPDK application through a DPDK DWA device.
> > A profile is expressed as a set of TLV messages for control plane and user plane
> > functions. Each TLV message must have Tag, SubTag, Direction, Type, Payload attributes.
> >
> > Programming model: Typical application programming sequence is as follows,
> > 1) In the EAL initialization phase, the DWA devices shall be probed,
> >     the application can query the number of available DWA devices with
> >     rte_dwa_dev_count() API.
> > 2) Application discovers the available profile(s) in a DWA device using
> >     rte_dwa_dev_disc_profiles() API.
> > 3) Application attaches one or more profile(s) to a DWA device using
> >     rte_dwa_dev_attach().
> > 4) Once the profile is attached, The device shall be in the STOPPED state.
> >     Configure the profile(s) with `TYPE_ATTACHED`and `TYPE_STOPPED`
> >     type TLVs using rte_dwa_ctrl_op() API.
> > 5) Once the profile is configured, move the profile to the `RUNNING` state
> >     by invoking rte_dwa_start() API.
> > 6) Once the profile is in running state and if it has user plane TLV,
> >     transfer those TLVs using rte_dwa_port_host_() API based on the available
> >     host port for the given profile attached.
> > 7) Application can change the dynamic configuration aspects in
> >     `RUNNING` state using rte_dwa_ctrl_op() API by issuing `TYPE_STARTED` type
> >     of TLV messages.
> > 8) Finally, use rte_dwa_stop(), rte_dwa_dev_detach(), rte_dwa_dev_close()
> >     sequence for tear-down.
> >
> >
> > L3FWD profile
> > -------------
> >
> >                               +-------------->--[1]--------------+
> >                               |                                  |
> >                   +-----------|----------+                       |
> >                   |           |          |                       |
> >                   |  +--------|-------+  |                       |
> >                   |  |                |  |                       |
> >                   |  | L3FWD Profile  |  |                       |
> >        \          |  |                |  |                       |
> >    <====\========>|  +----------------+  |                       |
> >      DWA \Port0   |     Lookup Table     |             +---------|----------+
> >           \       |  +----------------+  |             | DPDK DWA|Device[0] |
> >            \      |  | IP    | Dport  |  |  Host Port  | +-------|--------+ |
> >             \     |  +----------------+  |<===========>| |       |        | |
> >              +~[3]~~~|~~~~~~~|~~~~~~~~|~~~~~~~~~~~~~~~~~>|->L3FWD Profile | |
> >    <=============>|  +----------------+  |             | |                | |
> >      DWA Port1    |  |       |        |  | Control Port| +-|---------|----+ |
> >                   |  +----------------+  |<===========>|   |         |      |
> >      ~~~>~~[5]~~~~|~~|~~~+   |        |  |             +---|---------|------+
> >                   |  +---+------------+  |                 |         |
> >      ~~~<~~~~~~~~~|~~|~~~+   |        |<-|------[2]--------+         |
> >                   |  +----------------+<-|------[4]------------------+
> >                   |    Dataplane         |
> >    <=============>|    Workload          |
> >      DWA PortN    |    Accelerator       |
> >                   |    (HW/FW/SW)        |
> >                   +----------------------+
> >
> >
> > L3FWD profile offloads Layer-3 forwarding between the DWA Ethernet ports.
> >
> > The above diagram depicts the profile and application programming sequence.
> > 1) DWA device attaches the L3FWD profile using rte_dwa_dev_attach().
> > 2) Configure the L3FWD profile:
> > a) The application requests L3FWD profile capabilities of the DWA
> >     by using RTE_DWA_STAG_PROFILE_L3FWD_H2D_INFO, On response,
> >     the RTE_DWA_STAG_PROFILE_L3FWD_D2H_INFO returns the lookup modes
> >     supported, max rules supported, and available host ports for this profile.
> > b) The application configures a set of DWA ports to use a
> >     lookup mode(EM, LPM, or FIB) via RTE_DWA_STAG_PROFILE_L3FWD_H2D_CONFIG.
> > c) The application configures a valid host port to receive exception packets.
> > 3) The exception that is not matching forwarding table entry comes as
> >     RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS TLV to host. DWA stores the exception
> >     packet send back destination ports after completing step (4).
> > 4) Parse the exception packet and add rules to the FWD table using
> >     RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_ADD. If the application knows the rules beforehand,
> >     it can add the rules in step 2.
> > 5) When DWA ports receive the matching flows in the lookup table, DWA forwards
> >     to DWA Ethernet ports without host CPU intervention.
> >
> >
> > Example application usage with L3FWD profile
> > --------------------------------------------
> > This example application is to demonstrate the programming model of DWA library.
> > This example omits the error checks to simply the application.
> >
> > void
> > dwa_profile_l3fwd_add_rule(rte_dwa_obj_t obj obj, struct rte_mbuf *mbuf)
> > {
> >       struct rte_dwa_profile_l3fwd_h2d_lookup_add *lookup;
> >       struct rte_dwa_tlv *h2d, *d2h;
> >       struct rte_ether_hdr *eth_hdr;
> >       struct rte_ipv4_hdr *ipv4_hdr;
> >       uint32_t id;
> >       size_t len;
> >
> >       id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_LOOKUP_ADD);
> >       len = sizeof(struct rte_dwa_profile_l3fwd_h2d_config);
> >       h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
> >
> >       lookup = h2d->msg;
> >          /* Simply hardcode to IPv4 instead of looking for Packet type to simplify example */
> >       lookup->rule_type = RTE_DWA_PROFILE_L3FWD_RULE_TYPE_IPV4;
> >       lookup->v4_rule.prefix.depth = 24;
> >
> >       eth_hdr = rte_pktmbuf_mtod(mbuf, struct rte_ether_hdr *);
> >       ipv4_hdr = (struct rte_ipv4_hdr *)(eth_hdr + 1);
> >       lookup->v4_rule.prefix.ip_dst = rte_be_to_cpu_32(ipv4_hdr->dst_addr);
> >       lookup->eth_port_dst = mbuf->port;
> >
> >       rte_dwa_tlv_fill(h2d, id, len, h2d);
> >       d2h = rte_dwa_ctrl_op(obj, h2h);
> >       free(h2d);
> >       free(d2h);
> > }
> >
> > void
> > dwa_profile_l3fwd_port_host_ethernet_worker(rte_dwa_obj_t obj, struct app_ctx *ctx)
> > {
> >       struct rte_dwa_profile_l3fwd_d2h_exception_pkts *msg;
> >       struct rte_dwa_tlv *tlv;
> >       uint16_t i, rc, nb_tlvs;
> >       struct rte_mbuf *mbuf;
> >
> >       while (!ctx->done) {
> >               rc = rte_dwa_port_host_ethernet_rx(obj, 0, &tlv, 1);
> >               if (!rc)
> >                       continue;
> >
> >               /* Since L3FWD profile has only one User Plane TLV, Message must be
> >                * RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS message
> >                */
> >               msg = (struct rte_dwa_profile_l3fwd_d2h_exception_pkts *)tlv->msg;
> >               for (i = 0; i < msg->nb_pkts; i++) {
> >                               mbuf = msg->pkts[i];
> >                               /* Got a exception pkt from DWA, handle it by adding as new rule in
> >                                   * lookup table in DWA
> >                                */
> >                               dwa_profile_l3fwd_add_rule(obj, mbuf);
> >                               /* Free the mbuf to pool */
> >                               rte_pktmbuf_free(mbuf);
> >               }
> >
> >               /* Done with TLV mbuf container, free it back */
> >               rte_mempool_ops_enqueue_bulk(ctx->tlv_pool, tlv, 1);
> > }
> >
> > bool
> > dwa_port_host_ethernet_config(rte_dwa_obj_t obj, struct app_ctx *ctx)
> > {
> >       struct rte_dwa_tlv info_h2d, *info_d2h, *h2d = NULL, *d2h;
> >       struct rte_dwa_port_host_ethernet_d2h_info *info;
> >       int tlv_pool_element_sz;
> >       bool rc = false;
> >       size_t len;
> >
> >       /* Get the Ethernet host port info */
> >       id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_INFO);
> >       rte_dwa_tlv_fill(&info_h2d, id, 0, NULL);
> >       info_d2h = rte_dwa_ctrl_op(obj, &info_h2d)
> >
> >       info = rte_dwa_tlv_d2h_to_msg(info_d2h);
> >       if (info == NULL)
> >               goto fail;
> >       /* Need min one Rx queue to Receive exception traffic */
> >       if (info->nb_rx_queues == 0)
> >               goto fail;
> >       /* Done with message from DWA. Free back to implementation */
> >       free(obj, info_d2h);
> >
> >       /* Allocate exception packet pool */
> >       ctx->pkt_pool = rte_pktmbuf_pool_create("exception pool", /* Name */
> >                                  ctx->pkt_pool_depth, /* Number of elements*/
> >                                  512, /* Cache size*/
> >                                  0,
> >                                  RTE_MBUF_DEFAULT_BUF_SIZE,
> >                                  ctx->socket_id));
> >
> >
> >       tlv_pool_element_sz = DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ * sizeof(rte_mbuf *);
> >       tlv_pool_element_sz  += sizeof(rte_dwa_profile_l3fwd_d2h_exception_pkts);
> >
> >       /* Allocate TLV pool for RTE_DWA_STLV_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS_PACKETS tag */
> >       ctx->tlv_pool = rte_mempool_create("TLV pool", /* mempool name */
> >                                  ctx->tlv_pool_depth, /* Number of elements*/
> >                                  tlv_pool_element_sz, /* Element size*/
> >                                  512, /* cache size*/
> >                                  0, NULL, NULL, NULL /* Obj constructor */, NULL,
> >                                  ctx->socket_id, 0 /* flags *);
> >
> >
> >       /* Configure Ethernet host port */
> >       id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_CONFIG);
> >       len = sizeof(struct rte_dwa_port_host_ethernet_config);
> >       h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
> >
> >       cfg = h2d->msg;
> >       /* Update the Ethernet configuration parameters */
> >       cfg->nb_rx_queues = 1;
> >       cfg->nb_tx_queues = 0;
> >       cfg->max_burst = DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ;
> >       cfg->pkt_pool = ctx->pkt_pool;
> >       cfg->tlv_pool = ctx->tlv_pool;
> >       rte_dwa_tlv_fill(h2d, id, len, h2d);
> >       d2h = rte_dwa_ctrl_op(obj, h2d);
> >       if (d2h == NULL))
> >               goto fail;
> >
> >       free(h2d);
> >
> >       /* Configure Rx queue 0 receive expectation traffic */
> >       id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_QUEUE_CONFIG);
> >       len = sizeof(struct rte_dwa_port_host_ethernet_queue_config);
> >       h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
> >
> >       cfg = h2d->msg;
> >       cfg->id = 0; /* 0th Queue */
> >       cfg->enable= 1;
> >       cfg->is_tx = 0; /* Rx queue */
> >       cfg->depth = ctx->rx_queue_depth;
> >       rte_dwa_tlv_fill(h2d, id, len, h2d);
> >       d2h = rte_dwa_ctrl_op(obj, h2d);
> >       if (d2h == NULL))
> >               goto fail;
> >
> >       free(h2d);
> >
> >       return true;
> > fail:
> >       if (h2d)
> >               free(h2d);
> >       return rc;
> > }
> >
> > bool
> > dwa_profile_l3fwd_config(rte_dwa_obj_t obj, struct app_ctx *ctx)
> > {
> >       struct rte_dwa_tlv info_h2d, *info_d2h = NULL, *h2d, *d2h = NULL;
> >       struct rte_dwa_port_dwa_ethernet_d2h_info *info;
> >       struct rte_dwa_profile_l3fwd_h2d_config *cfg;
> >       bool rc = false;
> >       uint32_t id;
> >       size_t len;
> >
> >       /* Get DWA Ethernet port info */
> >       id = RTE_DWA_TLV_MK_ID(PORT_DWA_ETHERNET, H2D_INFO);
> >       rte_dwa_tlv_fill(&info_h2d, id, 0, NULL);
> >       info_d2h = rte_dwa_ctrl_op(obj, &info_h2d);
> >
> >       info = rte_dwa_tlv_d2h_to_msg(info_d2h);
> >       if (info == NULL)
> >               goto fail;
> >
> >       /* Not found any DWA ethernet ports */
> >       if (info->nb_ports == 0)
> >               goto fail;
> >
> >       /* Configure L3FWD profile */
> >       id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_CONFIG);
> >       len = sizeof(struct rte_dwa_profile_l3fwd_h2d_config) + (sizeof(uint16_t) * info->nb_ports);
> >       h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
> >
> >       cfg = h2d->msg;
> >       /* Update the L3FWD configuration parameters */
> >       cfg->mode = ctx->mode;
> >       /* Attach all DWA Ethernet ports onto L3FWD profile */
> >       cfg->nb_eth_ports = info->nb_ports;
> >       memcpy(cfg->eth_ports, info->avail_ports, sizeof(uint16_t) * info->nb_ports);
> >
> >       rte_dwa_tlv_fill(h2d, id, len, h2d);
> >       d2h = rte_dwa_ctrl_op(obj, h2d);
> >       free(h2d);
> >
> >       /* All good */
> >       rc = true;
> > fail:
> >       if (info_d2h)
> >               free(obj, info_d2h);
> >       if (d2h)
> >               free(obj, d2h);
> >
> >       return rc;
> > }
> >
> > bool
> > dwa_profile_l3fwd_has_capa(rte_dwa_obj_t obj, struct app_ctx *ctx)
> > {
> >       struct rte_dwa_profile_l3fwd_d2h_info *info;
> >       struct rte_dwa_tlv h2d, *d2h;
> >       bool found = false;
> >       uint32_t id;
> >
> >       /* Get L3FWD profile info */
> >       id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_INFO);
> >       rte_dwa_tlv_fill(&h2d, id, 0, NULL);
> >       d2h = rte_dwa_ctrl_op(obj, &h2d);
> >
> >       info = rte_dwa_tlv_d2h_to_msg(d2h);
> >       /* Request failed */
> >       if (info == NULL)
> >               goto fail;
> >       /* Required lookup modes is not supported */
> >       if (!(info->modes_supported & ctx->mode))
> >               goto fail;
> >
> >       /* Check profile supports HOST_ETHERNET port as this application
> >           * supports only host port as Ethernet
> >           */
> >       for (i = 0; i < info->nb_host_ports; i++) {
> >               if (info->host_ports[i] == RTE_DWA_TAG_PORT_HOST_ETHERNET); {
> >                       found = true;
> >               }
> >       }
> >
> >       /* Done with response, Free the d2h memory allocated by implementation */
> >       free(obj, d2h);
> > fail:
> >       return found;
> > }
> >
> >
> > bool
> > dwa_has_profile(enum rte_dwa_tag_profile pf)
> > {
> >       enum rte_dwa_tlv_profile *pfs = NULL;
> >       bool found = false;
> >       int nb_pfs;
> >
> >       /* Get the number of profiles on the DWA device */
> >       nb_pfs = rte_dwa_dev_disc_profiles(0, NULL);
> >       pfs = malloc(sizeof(enum rte_dwa_tag_profile)  * nb_pfs);
> >       /* Fetch all the profiles */
> >       nb_pfs = rte_dwa_dev_disc_profiles(0, pfs);
> >
> >       /* Check the list has requested profile */
> >       for (i = 0; i < nb_pfs; i++) {
> >               if (pfs[i] == pf);
> >                       found = true;
> >       }
> >       free(pfs);
> >
> >
> >       return found;
> > }
> >
> >
> > #include <rte_dwa.h>
> >
> > #define DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ                32
> >
> > struct app_ctx {
> >       bool done;
> >       struct rte_mempool *pkt_pool;
> >       struct rte_mempool *tlv_pool;
> >       enum rte_dwa_profile_l3fwd_lookup_mode mode;
> >       int socket_id;
> >       int pkt_pool_depth;
> >       int tlv_pool_depth;
> >       int rx_queue_depth;
> > } __rte_cache_aligned;
> >
> > int
> > main(int argc, char **argv)
> > {
> >       rte_dwa_obj_t obj = NULL;
> >       struct app_ctx ctx;
> >       int rc;
> >
> >       /* Initialize EAL */
> >       rc= rte_eal_init(argc, argv);
> >          if (rc < 0)
> >                rte_exit(EXIT_FAILURE, "Invalid EAL arguments\n");
> >          argc -= ret;
> >          argv += ret;
> >
> >
> >       memset(&ctx, 0, sizeof(ctx));
> >       /* Set application default values */
> >       ctx->mode = RTE_DWA_PROFILE_L3FWD_MODE_LPM;
> >       ctx->socket_id = SOCKET_ID_ANY;
> >       ctx->pkt_pool_depth = 10000;
> >       ctx->tlv_pool_depth = 10000;
> >       ctx->rx_queue_depth = 10000;
> >
> >       /* Step 1: Check any DWA devices present  */
> >       rc = rte_dwa_dev_count();
> >       if (rc <= 0)
> >               rte_exit(EXIT_FAILURE, "Failed to find DWA devices\n");
> >
> >       /* Step 2: Check DWA device has L3FWD profile or not */
> >       if (!dwa_has_profile(RTE_DWA_TAG_PROFILE_L3FWD))
> >               rte_exit(EXIT_FAILURE, "L3FWD profile not found\n");
> >
> >       /*
> >        * Step 3: Now that, workload accelerator has L3FWD profile,
> >        * offload L3FWD workload to accelerator by attaching the profile
> >        * to accelerator.
> >        */
> >       enum rte_dwa_tlv_profile profile[] = {RTE_DWA_TAG_PROFILE_L3FWD};
> >       obj = rte_dwa_dev_attach(0, "my_custom_accelerator_device", profile, 1).;
> >
> >       /* Step 4: Check Attached L3FWD profile has required capability to proceed */
> >       if (!dwa_profile_l3fwd_has_capa(obj, &ctx))
> >               rte_exit(EXIT_FAILURE, "L3FWD profile does not have enough capability \n");
> >
> >       /* Step 5: Configure l3fwd profile */
> >       if (!dwa_profile_l3fwd_config(obj, &ctx))
> >               rte_exit(EXIT_FAILURE, "L3FWD profile configure failed \n");
> >
> >       /* Step 6: Configure ethernet host port to receive exception packets */
> >       if (!dwa_port_host_ethernet_config(obj, &ctx))
> >               rte_exit(EXIT_FAILURE, "L3FWD profile configure failed \n");
> >
> >       /* Step 7 : Move DWA profiles to start state */
> >       rte_dwa_start(obj);
> >
> >       /* Step 8: Handle expectation packets and add lookup rules for it */
> >       dwa_profile_l3fwd_port_host_ethernet_worker(obj, &ctx);
> >
> >       /* Step 9: Clean up */
> >       rte_dwa_stop(obj);
> >       rte_dwa_dev_detach(0, obj);
> >       rte_dwa_dev_close(0);
> >
> >       return 0;
> > }
> >
> >
> > Jerin Jacob (1):
> >    dwa: introduce dataplane workload accelerator subsystem
> >
> >   doc/api/doxy-api-index.md            |  13 +
> >   doc/api/doxy-api.conf.in             |   1 +
> >   lib/dwa/dwa.c                        |   7 +
> >   lib/dwa/meson.build                  |  17 ++
> >   lib/dwa/rte_dwa.h                    | 184 +++++++++++++
> >   lib/dwa/rte_dwa_core.h               | 264 +++++++++++++++++++
> >   lib/dwa/rte_dwa_dev.h                | 154 +++++++++++
> >   lib/dwa/rte_dwa_port_dwa_ethernet.h  |  68 +++++
> >   lib/dwa/rte_dwa_port_host_ethernet.h | 178 +++++++++++++
> >   lib/dwa/rte_dwa_profile_admin.h      |  85 ++++++
> >   lib/dwa/rte_dwa_profile_l3fwd.h      | 378 +++++++++++++++++++++++++++
> >   lib/dwa/version.map                  |   3 +
> >   lib/meson.build                      |   1 +
> >   13 files changed, 1353 insertions(+)
> >   create mode 100644 lib/dwa/dwa.c
> >   create mode 100644 lib/dwa/meson.build
> >   create mode 100644 lib/dwa/rte_dwa.h
> >   create mode 100644 lib/dwa/rte_dwa_core.h
> >   create mode 100644 lib/dwa/rte_dwa_dev.h
> >   create mode 100644 lib/dwa/rte_dwa_port_dwa_ethernet.h
> >   create mode 100644 lib/dwa/rte_dwa_port_host_ethernet.h
> >   create mode 100644 lib/dwa/rte_dwa_profile_admin.h
> >   create mode 100644 lib/dwa/rte_dwa_profile_l3fwd.h
> >   create mode 100644 lib/dwa/version.map
> >
>

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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-25  9:03   ` Jerin Jacob
@ 2021-10-29 11:57     ` Mattias Rönnblom
  2021-10-29 15:51       ` Jerin Jacob
  0 siblings, 1 reply; 20+ messages in thread
From: Mattias Rönnblom @ 2021-10-29 11:57 UTC (permalink / raw)
  To: Jerin Jacob
  Cc: jerinj, dev, thomas, ferruh.yigit, ajit.khaparde, aboyer,
	andrew.rybchenko, beilei.xing, bruce.richardson, chas3,
	chenbo.xia, ciara.loftus, dsinghrawat, ed.czeck, evgenys, grive,
	g.singh, zhouguoyang, haiyue.wang, hkalra, heinrich.kuhn,
	hemant.agrawal, hyonkim, igorch, irusskikh, jgrajcia,
	jasvinder.singh, jianwang, jiawenwu, jingjing.wu, johndale,
	john.miller, linville, keith.wiles, kirankumark, oulijun, lironh,
	longli, mw, spinler, matan, matt.peters, maxime.coquelin, mk,
	humin29, pnalla, ndabilpuram, qiming.yang, qi.z.zhang, radhac,
	rahul.lakkireddy, rmody, rosen.xu, sachin.saxena, skoteshwar,
	shshaikh, shaibran, shepard.siegel, asomalap, somnath.kotur,
	sthemmin, steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, ruifeng.wang, drc, konstantin.ananyev,
	olivier.matz, jay.jayatheerthan, asekhar, pbhagavatula,
	Elana Agostini

On 2021-10-25 11:03, Jerin Jacob wrote:
> On Mon, Oct 25, 2021 at 1:05 PM Mattias Rönnblom
> <mattias.ronnblom@ericsson.com> wrote:
>> On 2021-10-19 20:14, jerinj@marvell.com wrote:
>>> From: Jerin Jacob <jerinj@marvell.com>
>>>
>>>
>>> Dataplane Workload Accelerator library
>>> ======================================
>>>
>>> Definition of Dataplane Workload Accelerator
>>> --------------------------------------------
>>> Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
>>> Network controllers and programmable data acceleration engines for
>>> packet processing, cryptography, regex engines, baseband processing, etc.
>>> This allows DWA to offload  compute/packet processing/baseband/
>>> cryptography-related workload from the host CPU to save the cost and power.
>>> Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
>>>
>>> Unlike other devices in DPDK, the DWA device is not fixed-function
>>> due to the fact that it has CPUs and programmable HW accelerators.
>>
>> There are already several instances of DPDK devices with pure-software
>> implementation. In this regard, a DPU/SmartNIC represents nothing new.
>> What's new, it seems to me, is a much-increased need to
>> configure/arrange the processing in complex manners, to avoid bouncing
>> everything to the host CPU.
> Yes and No. It will be based on the profile. The TLV type TYPE_USER_PLANE will
> have user plane traffic from/to host. For example, offloading ORAN split 7.2
> baseband profile. Transport blocks sent to/from host as TYPE_USER_PLANE.
>
>> Something like P4 or rte_flow-based hooks or
>> some other kind of extension. The eventdev adapters solve the same
>> problem (where on some systems packets go through the host CPU on their
>> way to the event device, and others do not) - although on a *much*
>> smaller scale.
> Yes. Eventdev Adapters only for event device plumbing.
>
>
>>
>> "Not-fixed function" seems to call for more hot plug support in the
>> device APIs. Such functionality could then be reused by anything that
>> can be reconfigured dynamically (FPGAs, firmware-programmed
>> accelerators, etc.),
> Yes.
>
>> but which may not be able to serve as a RPC
>> endpoint, like a SmartNIC.
> It can. That's the reason for choosing TLVs. So that
> any higher level language can use TLVs like https://protect2.fireeye.com/v1/url?k=96886daf-c91357b6-96882d34-8682aaa22bc0-c994a5dcbda5d9e8&q=1&e=e89c0aca-a3b3-4f72-b616-ba4550b856b6&u=https%3A%2F%2Fgithub.com%2Fustropo%2Futtlv
> to communicate with the accelerator.  TLVs follow the request and
> response scheme like RPC. So it can warp it under application if needed.
>
>>
>> DWA could be some kind of DPDK-internal framework for managing certain
>> type of DPUs, but should it be exposed to the user application?
>
> Could you clarify a bit more.
> The offload is represented as a set of TLVs in generic fashion. There
> is no DPU specific bit in offload representation. See
> rte_dwa_profiile_l3fwd.h header file.


It seems a bit cumbersome to work with TLVs on the user application 
side. Would it be an alternative to have the profile API as a set of C 
APIs instead of TLV-based messaging interface? The underlying 
implementation could still be - in many or all cases - be TLVs sent over 
some appropriate transport.


Such a C API could still be asynchronous, and still be a profile API 
(rather than a set of new DPDK device types).


What I tried to ask during the meeting but where I didn't get an answer 
(or at least one that I could understand) was how the profiles was to be 
specified and/or documented. Maybe the above is what you had in mind 
already.


> TB hosted a meeting for this at Date: Wednesday, October 27th Time:
> 3pm UTC, https://meet.jit.si/DPDK
> Feel free to join.
>
>
>>
>>> This enables DWA personality/workload to be completely programmable.
>>> Typical examples of DWA offloads are Flow/Session management,
>>> Virtual switch, TLS offload, IPsec offload, l3fwd offload, etc.
>>> Motivation for the new library
>>> ------------------------------
>>> Even though, a lot of semiconductor vendors offers a different form of DWA,
>>> such as DPU(often called Smart-NIC), GPU, IPU, XPU, etc.,
>>> Due to the lack of standard APIs to "Define the workload" and
>>> "Communication between HOST and DWA", it is difficult for DPDK
>>> consumers to use them in a portable way across different DWA vendors
>>> and enable it in cloud environments.
>>>
>>>
>>> Contents of RFC
>>> ------------------
>>> This RFC attempts to define standard APIs for:
>>>
>>> 1) Definition of Profiles corresponding to well defined workloads, which includes
>>>      a set of TLV(Messages) as a request  and response scheme to define
>>>      the contract between host and DWA to offload a workload.
>>>      (See lib/dwa/rte_dwa_profile_* header files)
>>> 2) Discovery of a DWAs capabilities (e.g. which specific workloads it can support)
>>>      in a vendor independent fashion. (See rte_dwa_dev_disc_profiles())
>>> 3) Attaching a set of profiles to a DWA device(See rte_dwa_dev_attach())
>>> 4) A communication framework between Host and DWA(See rte_dwa_ctrl_op() for
>>>      control plane and rte_dwa_port_host_* for user plane)
>>> 5) Virtualization of DWA hardware and firmware (Use standard DPDK device/bus model)
>>> 6) Enablement of administrative functions such as FW updates,
>>>      resource partitioning in a DWA like items in global in
>>>      nature that is applicable for all DWA device under the DWA.
>>>      (See rte_dwa_profile_admin.h)
>>>
>>> Also, this RFC define the L3FWD profile to offload L3FWD workload to DWA.
>>> This RFC defines an ethernet-style host port for Host to DWA communication.
>>> Different host port types may be required to cover the large spectrum of DWA types as
>>> transports like PCIe DMA, Shared Memory, or Ethernet are fundamentally different,
>>> and optimal performance need host port specific APIs.
>>>
>>> The framework does not force an abstract of different transport interfaces as
>>> single API, instead, decouples TLV from the transport interface and focuses on
>>> defining the TLVs and leaving vendors to specify the host ports
>>> specific to their DWA architecture.
>>>
>>>
>>> Roadmap
>>> -------
>>> 1) Address the comments for this RFC and enable the common code
>>> 2) SW drivers/infrastructure for `DWA` and `DWA device`
>>> as two separate DPDK processes over `memif` DPDK ethdev driver for
>>> L3FWD offload. This is to enable the framework without any special HW.
>>> 3) Example DWA device application for L3FWD profile.
>>> 4) Marvell DWA Device drivers.
>>> 5) Based on community interest new profile can be added in the future.
>>>
>>>
>>> DWA library framework
>>> ---------------------
>>>
>>> DWA components:
>>>
>>>                                                     +--> rte_dwa_port_host_*()
>>>                                                     |  (User Plane traffic as TLV)
>>>                                                     |
>>>                    +----------------------+         |   +--------------------+
>>>                    |                      |         |   | DPDK DWA Device[0] |
>>>                    |  +----------------+  |  Host Port  | +----------------+ |
>>>                    |  |                |  |<========+==>| |                | |
>>>                    |  |   Profile 0    |  |             | |   Profile X    | |
>>>                    |  |                |  |             | |                | |
>>>     <=============>|  +----------------+  | Control Port| +----------------+ |
>>>       DWA Port0    |  +----------------+  |<========+==>|                    |
>>>                    |  |                |  |         |   +--------------------+
>>>                    |  |   Profile 1    |  |         |
>>>                    |  |                |  |         +--> rte_dwa_ctrl_op()
>>>                    |  +----------------+  |         (Control Plane traffic as TLV)
>>>     <=============>|      Dataplane       |
>>>       DWA Port1    |      Workload        |
>>>                    |      Accelerator     |             +---------- ---------+
>>>                    |      (HW/FW/SW)      |             | DPDK DWA Device[N] |
>>>                    |                      |  Host Port  | +----------------+ |
>>>     <=============>|  +----------------+  |<===========>| |                | |
>>>       DWA PortN    |  |                |  |             | |   Profile Y    | |
>>>                    |  |    Profile N   |  |             | |           ^    | |
>>>                    |  |                |  | Control Port| +-----------|----+ |
>>>                    |  +-------|--------+  |<===========>|             |      |
>>>                    |          |           |             +-------------|------+
>>>                    +----------|-----------+                           |
>>>                               |                                       |
>>>                               +---------------------------------------+
>>>                                                          ^
>>>                                                          |
>>>                                                          +--rte_dwa_dev_attach()
>>>
>>>
>>> Dataplane Workload Accelerator: It is an abstract model. The model is
>>> capable of offloading the dataplane workload from application via
>>> DPDK API over host and control ports of a DWA device.
>>> Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
>>> Network controllers, and programmable data acceleration engines for
>>> packet processing, cryptography, regex engines, base-band processing, etc.
>>> This allows DWA to offload compute/packet processing/base-band/cryptography-related
>>> workload from the host CPU to save cost and power. Also,
>>> enable scaling the workload by adding DWAs to the host CPU as needed.
>>>
>>> DWA device: A DWA can be sliced to N number of DPDK DWA device(s)
>>> based on the resources available in DWA.
>>> The DPDK API interface operates on the DPDK DWA device.
>>> It is a representation of a set of resources in DWA.
>>>
>>> TLV: TLV (tag-length-value) encoded data stream contain tag as
>>> message ID, followed by message length, and finally the message payload.
>>> The 32bit message ID consists of two parts, 16bit Tag and 16bit Subtag.
>>> The tag represents ID of the group of the similar message,
>>> whereas, subtag represents a message tag ID under the group.
>>>
>>> Control Port: Used for transferring the control plane TLVs. Every DPDK
>>> DWA device must have a control port. Only one outstanding TLV can be
>>> processed via this port by a single DWA device. This makes the control
>>> port suitable for the control plane.
>>>
>>> Host Port: Used for transferring the user plane TLVs.
>>> Ethernet, PCIe DMA, Shared Memory, etc.are the example of
>>> different transport mechanisms abstracted under the host port.
>>> The primary purpose of host port to decouple the user plane TLVs with
>>> underneath transport mechanism differences.
>>> Unlike control port, more than one outstanding TLVs can be processed by
>>> a single DWA device via this port.
>>> This makes, the host port transfer to be in asynchronous nature,
>>> to support large volumes and less latency user plane traffic.
>>>
>>> DWA Port: Used for transferring data between the external source and DWA.
>>> Ethernet, eCPRI are examples of DWA ports. Unlike host ports,
>>> the host CPU is not involved in transferring the data to/from DWA ports.
>>> These ports typically connected to the Network controller inside the
>>> DWA to transfer the traffic from the external source.
>>>
>>> TLV direction: `Host to DWA` and `DWA to Host` are the directions
>>> of TLV messages. The former one is specified as H2D, and the later one is
>>> specified as D2H. The H2D control TLVs, used for requesting DWA to perform
>>> specific action and D2H control TLVs are used to respond to the requested
>>> actions. The H2D user plane messages are used for transferring data from the
>>> host to the DWA. The D2H user plane messages are used for transferring
>>> data from the DWA to the host.
>>>
>>> DWA device states: Following are the different states of a DWA device.
>>> - READY: DWA Device is ready to attach the profile.
>>> See rte_dwa_dev_disc_profiles() API to discover the profile.
>>> - ATTACHED: DWA Device attached to one or more profiles.
>>> See rte_dwa_dev_attach() API to attach the profile(s).
>>> - STOPPED: Profile is in the stop state.
>>> TLV type `TYPE_ATTACHED`and `TYPE_STOPPED` messages are valid in this state.
>>> After rte_dwa_dev_attach() or explicitly invoking the rte_dwa_stop() API
>>> brings device to this state.
>>> - RUNNING: Invoking rte_dwa_start() brings the device to this state.
>>> TLV type `TYPE_STARTED` and `TYPE_USER_PLANE` are valid in this state.
>>> - DETACHED: Invoking rte_dwa_dev_detach() brings the device to this state.
>>> The device and profile must be in the STOPPED state prior to
>>> invoking the rte_dwa_dev_detach().
>>> - CLOSED: Closed a stopped/detached DWA device.The device cannot be restarted!.
>>> Invoking rte_dwa_dev_close() brings the device to this state.
>>>
>>> TLV types: Following are the different TLV types
>>> - TYPE_ATTACHED: Valid when the device is in `ATTACHED`, `STOPPED` and `RUNNING` state.
>>> - TYPE_STOPPED: Valid when the device is in `STOPPED` state.
>>> - TYPE_STARTED: Valid when the device is in `RUNNING` state.
>>> - TYPE_USER_PLANE: Valid when the device is in `RUNNING` state and
>>> used to transfer only user plane traffic.
>>>
>>> Profile: Specifies a workload that dataplane workload accelerator
>>> process on behalf of a DPDK application through a DPDK DWA device.
>>> A profile is expressed as a set of TLV messages for control plane and user plane
>>> functions. Each TLV message must have Tag, SubTag, Direction, Type, Payload attributes.
>>>
>>> Programming model: Typical application programming sequence is as follows,
>>> 1) In the EAL initialization phase, the DWA devices shall be probed,
>>>      the application can query the number of available DWA devices with
>>>      rte_dwa_dev_count() API.
>>> 2) Application discovers the available profile(s) in a DWA device using
>>>      rte_dwa_dev_disc_profiles() API.
>>> 3) Application attaches one or more profile(s) to a DWA device using
>>>      rte_dwa_dev_attach().
>>> 4) Once the profile is attached, The device shall be in the STOPPED state.
>>>      Configure the profile(s) with `TYPE_ATTACHED`and `TYPE_STOPPED`
>>>      type TLVs using rte_dwa_ctrl_op() API.
>>> 5) Once the profile is configured, move the profile to the `RUNNING` state
>>>      by invoking rte_dwa_start() API.
>>> 6) Once the profile is in running state and if it has user plane TLV,
>>>      transfer those TLVs using rte_dwa_port_host_() API based on the available
>>>      host port for the given profile attached.
>>> 7) Application can change the dynamic configuration aspects in
>>>      `RUNNING` state using rte_dwa_ctrl_op() API by issuing `TYPE_STARTED` type
>>>      of TLV messages.
>>> 8) Finally, use rte_dwa_stop(), rte_dwa_dev_detach(), rte_dwa_dev_close()
>>>      sequence for tear-down.
>>>
>>>
>>> L3FWD profile
>>> -------------
>>>
>>>                                +-------------->--[1]--------------+
>>>                                |                                  |
>>>                    +-----------|----------+                       |
>>>                    |           |          |                       |
>>>                    |  +--------|-------+  |                       |
>>>                    |  |                |  |                       |
>>>                    |  | L3FWD Profile  |  |                       |
>>>         \          |  |                |  |                       |
>>>     <====\========>|  +----------------+  |                       |
>>>       DWA \Port0   |     Lookup Table     |             +---------|----------+
>>>            \       |  +----------------+  |             | DPDK DWA|Device[0] |
>>>             \      |  | IP    | Dport  |  |  Host Port  | +-------|--------+ |
>>>              \     |  +----------------+  |<===========>| |       |        | |
>>>               +~[3]~~~|~~~~~~~|~~~~~~~~|~~~~~~~~~~~~~~~~~>|->L3FWD Profile | |
>>>     <=============>|  +----------------+  |             | |                | |
>>>       DWA Port1    |  |       |        |  | Control Port| +-|---------|----+ |
>>>                    |  +----------------+  |<===========>|   |         |      |
>>>       ~~~>~~[5]~~~~|~~|~~~+   |        |  |             +---|---------|------+
>>>                    |  +---+------------+  |                 |         |
>>>       ~~~<~~~~~~~~~|~~|~~~+   |        |<-|------[2]--------+         |
>>>                    |  +----------------+<-|------[4]------------------+
>>>                    |    Dataplane         |
>>>     <=============>|    Workload          |
>>>       DWA PortN    |    Accelerator       |
>>>                    |    (HW/FW/SW)        |
>>>                    +----------------------+
>>>
>>>
>>> L3FWD profile offloads Layer-3 forwarding between the DWA Ethernet ports.
>>>
>>> The above diagram depicts the profile and application programming sequence.
>>> 1) DWA device attaches the L3FWD profile using rte_dwa_dev_attach().
>>> 2) Configure the L3FWD profile:
>>> a) The application requests L3FWD profile capabilities of the DWA
>>>      by using RTE_DWA_STAG_PROFILE_L3FWD_H2D_INFO, On response,
>>>      the RTE_DWA_STAG_PROFILE_L3FWD_D2H_INFO returns the lookup modes
>>>      supported, max rules supported, and available host ports for this profile.
>>> b) The application configures a set of DWA ports to use a
>>>      lookup mode(EM, LPM, or FIB) via RTE_DWA_STAG_PROFILE_L3FWD_H2D_CONFIG.
>>> c) The application configures a valid host port to receive exception packets.
>>> 3) The exception that is not matching forwarding table entry comes as
>>>      RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS TLV to host. DWA stores the exception
>>>      packet send back destination ports after completing step (4).
>>> 4) Parse the exception packet and add rules to the FWD table using
>>>      RTE_DWA_STAG_PROFILE_L3FWD_H2D_LOOKUP_ADD. If the application knows the rules beforehand,
>>>      it can add the rules in step 2.
>>> 5) When DWA ports receive the matching flows in the lookup table, DWA forwards
>>>      to DWA Ethernet ports without host CPU intervention.
>>>
>>>
>>> Example application usage with L3FWD profile
>>> --------------------------------------------
>>> This example application is to demonstrate the programming model of DWA library.
>>> This example omits the error checks to simply the application.
>>>
>>> void
>>> dwa_profile_l3fwd_add_rule(rte_dwa_obj_t obj obj, struct rte_mbuf *mbuf)
>>> {
>>>        struct rte_dwa_profile_l3fwd_h2d_lookup_add *lookup;
>>>        struct rte_dwa_tlv *h2d, *d2h;
>>>        struct rte_ether_hdr *eth_hdr;
>>>        struct rte_ipv4_hdr *ipv4_hdr;
>>>        uint32_t id;
>>>        size_t len;
>>>
>>>        id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_LOOKUP_ADD);
>>>        len = sizeof(struct rte_dwa_profile_l3fwd_h2d_config);
>>>        h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
>>>
>>>        lookup = h2d->msg;
>>>           /* Simply hardcode to IPv4 instead of looking for Packet type to simplify example */
>>>        lookup->rule_type = RTE_DWA_PROFILE_L3FWD_RULE_TYPE_IPV4;
>>>        lookup->v4_rule.prefix.depth = 24;
>>>
>>>        eth_hdr = rte_pktmbuf_mtod(mbuf, struct rte_ether_hdr *);
>>>        ipv4_hdr = (struct rte_ipv4_hdr *)(eth_hdr + 1);
>>>        lookup->v4_rule.prefix.ip_dst = rte_be_to_cpu_32(ipv4_hdr->dst_addr);
>>>        lookup->eth_port_dst = mbuf->port;
>>>
>>>        rte_dwa_tlv_fill(h2d, id, len, h2d);
>>>        d2h = rte_dwa_ctrl_op(obj, h2h);
>>>        free(h2d);
>>>        free(d2h);
>>> }
>>>
>>> void
>>> dwa_profile_l3fwd_port_host_ethernet_worker(rte_dwa_obj_t obj, struct app_ctx *ctx)
>>> {
>>>        struct rte_dwa_profile_l3fwd_d2h_exception_pkts *msg;
>>>        struct rte_dwa_tlv *tlv;
>>>        uint16_t i, rc, nb_tlvs;
>>>        struct rte_mbuf *mbuf;
>>>
>>>        while (!ctx->done) {
>>>                rc = rte_dwa_port_host_ethernet_rx(obj, 0, &tlv, 1);
>>>                if (!rc)
>>>                        continue;
>>>
>>>                /* Since L3FWD profile has only one User Plane TLV, Message must be
>>>                 * RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS message
>>>                 */
>>>                msg = (struct rte_dwa_profile_l3fwd_d2h_exception_pkts *)tlv->msg;
>>>                for (i = 0; i < msg->nb_pkts; i++) {
>>>                                mbuf = msg->pkts[i];
>>>                                /* Got a exception pkt from DWA, handle it by adding as new rule in
>>>                                    * lookup table in DWA
>>>                                 */
>>>                                dwa_profile_l3fwd_add_rule(obj, mbuf);
>>>                                /* Free the mbuf to pool */
>>>                                rte_pktmbuf_free(mbuf);
>>>                }
>>>
>>>                /* Done with TLV mbuf container, free it back */
>>>                rte_mempool_ops_enqueue_bulk(ctx->tlv_pool, tlv, 1);
>>> }
>>>
>>> bool
>>> dwa_port_host_ethernet_config(rte_dwa_obj_t obj, struct app_ctx *ctx)
>>> {
>>>        struct rte_dwa_tlv info_h2d, *info_d2h, *h2d = NULL, *d2h;
>>>        struct rte_dwa_port_host_ethernet_d2h_info *info;
>>>        int tlv_pool_element_sz;
>>>        bool rc = false;
>>>        size_t len;
>>>
>>>        /* Get the Ethernet host port info */
>>>        id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_INFO);
>>>        rte_dwa_tlv_fill(&info_h2d, id, 0, NULL);
>>>        info_d2h = rte_dwa_ctrl_op(obj, &info_h2d)
>>>
>>>        info = rte_dwa_tlv_d2h_to_msg(info_d2h);
>>>        if (info == NULL)
>>>                goto fail;
>>>        /* Need min one Rx queue to Receive exception traffic */
>>>        if (info->nb_rx_queues == 0)
>>>                goto fail;
>>>        /* Done with message from DWA. Free back to implementation */
>>>        free(obj, info_d2h);
>>>
>>>        /* Allocate exception packet pool */
>>>        ctx->pkt_pool = rte_pktmbuf_pool_create("exception pool", /* Name */
>>>                                   ctx->pkt_pool_depth, /* Number of elements*/
>>>                                   512, /* Cache size*/
>>>                                   0,
>>>                                   RTE_MBUF_DEFAULT_BUF_SIZE,
>>>                                   ctx->socket_id));
>>>
>>>
>>>        tlv_pool_element_sz = DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ * sizeof(rte_mbuf *);
>>>        tlv_pool_element_sz  += sizeof(rte_dwa_profile_l3fwd_d2h_exception_pkts);
>>>
>>>        /* Allocate TLV pool for RTE_DWA_STLV_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS_PACKETS tag */
>>>        ctx->tlv_pool = rte_mempool_create("TLV pool", /* mempool name */
>>>                                   ctx->tlv_pool_depth, /* Number of elements*/
>>>                                   tlv_pool_element_sz, /* Element size*/
>>>                                   512, /* cache size*/
>>>                                   0, NULL, NULL, NULL /* Obj constructor */, NULL,
>>>                                   ctx->socket_id, 0 /* flags *);
>>>
>>>
>>>        /* Configure Ethernet host port */
>>>        id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_CONFIG);
>>>        len = sizeof(struct rte_dwa_port_host_ethernet_config);
>>>        h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
>>>
>>>        cfg = h2d->msg;
>>>        /* Update the Ethernet configuration parameters */
>>>        cfg->nb_rx_queues = 1;
>>>        cfg->nb_tx_queues = 0;
>>>        cfg->max_burst = DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ;
>>>        cfg->pkt_pool = ctx->pkt_pool;
>>>        cfg->tlv_pool = ctx->tlv_pool;
>>>        rte_dwa_tlv_fill(h2d, id, len, h2d);
>>>        d2h = rte_dwa_ctrl_op(obj, h2d);
>>>        if (d2h == NULL))
>>>                goto fail;
>>>
>>>        free(h2d);
>>>
>>>        /* Configure Rx queue 0 receive expectation traffic */
>>>        id = RTE_DWA_TLV_MK_ID(PORT_HOST_ETHERNET, H2D_QUEUE_CONFIG);
>>>        len = sizeof(struct rte_dwa_port_host_ethernet_queue_config);
>>>        h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
>>>
>>>        cfg = h2d->msg;
>>>        cfg->id = 0; /* 0th Queue */
>>>        cfg->enable= 1;
>>>        cfg->is_tx = 0; /* Rx queue */
>>>        cfg->depth = ctx->rx_queue_depth;
>>>        rte_dwa_tlv_fill(h2d, id, len, h2d);
>>>        d2h = rte_dwa_ctrl_op(obj, h2d);
>>>        if (d2h == NULL))
>>>                goto fail;
>>>
>>>        free(h2d);
>>>
>>>        return true;
>>> fail:
>>>        if (h2d)
>>>                free(h2d);
>>>        return rc;
>>> }
>>>
>>> bool
>>> dwa_profile_l3fwd_config(rte_dwa_obj_t obj, struct app_ctx *ctx)
>>> {
>>>        struct rte_dwa_tlv info_h2d, *info_d2h = NULL, *h2d, *d2h = NULL;
>>>        struct rte_dwa_port_dwa_ethernet_d2h_info *info;
>>>        struct rte_dwa_profile_l3fwd_h2d_config *cfg;
>>>        bool rc = false;
>>>        uint32_t id;
>>>        size_t len;
>>>
>>>        /* Get DWA Ethernet port info */
>>>        id = RTE_DWA_TLV_MK_ID(PORT_DWA_ETHERNET, H2D_INFO);
>>>        rte_dwa_tlv_fill(&info_h2d, id, 0, NULL);
>>>        info_d2h = rte_dwa_ctrl_op(obj, &info_h2d);
>>>
>>>        info = rte_dwa_tlv_d2h_to_msg(info_d2h);
>>>        if (info == NULL)
>>>                goto fail;
>>>
>>>        /* Not found any DWA ethernet ports */
>>>        if (info->nb_ports == 0)
>>>                goto fail;
>>>
>>>        /* Configure L3FWD profile */
>>>        id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_CONFIG);
>>>        len = sizeof(struct rte_dwa_profile_l3fwd_h2d_config) + (sizeof(uint16_t) * info->nb_ports);
>>>        h2d = malloc(RTE_DWA_TLV_HDR_SZ + len);
>>>
>>>        cfg = h2d->msg;
>>>        /* Update the L3FWD configuration parameters */
>>>        cfg->mode = ctx->mode;
>>>        /* Attach all DWA Ethernet ports onto L3FWD profile */
>>>        cfg->nb_eth_ports = info->nb_ports;
>>>        memcpy(cfg->eth_ports, info->avail_ports, sizeof(uint16_t) * info->nb_ports);
>>>
>>>        rte_dwa_tlv_fill(h2d, id, len, h2d);
>>>        d2h = rte_dwa_ctrl_op(obj, h2d);
>>>        free(h2d);
>>>
>>>        /* All good */
>>>        rc = true;
>>> fail:
>>>        if (info_d2h)
>>>                free(obj, info_d2h);
>>>        if (d2h)
>>>                free(obj, d2h);
>>>
>>>        return rc;
>>> }
>>>
>>> bool
>>> dwa_profile_l3fwd_has_capa(rte_dwa_obj_t obj, struct app_ctx *ctx)
>>> {
>>>        struct rte_dwa_profile_l3fwd_d2h_info *info;
>>>        struct rte_dwa_tlv h2d, *d2h;
>>>        bool found = false;
>>>        uint32_t id;
>>>
>>>        /* Get L3FWD profile info */
>>>        id = RTE_DWA_TLV_MK_ID(PROFILE_L3FWD, H2D_INFO);
>>>        rte_dwa_tlv_fill(&h2d, id, 0, NULL);
>>>        d2h = rte_dwa_ctrl_op(obj, &h2d);
>>>
>>>        info = rte_dwa_tlv_d2h_to_msg(d2h);
>>>        /* Request failed */
>>>        if (info == NULL)
>>>                goto fail;
>>>        /* Required lookup modes is not supported */
>>>        if (!(info->modes_supported & ctx->mode))
>>>                goto fail;
>>>
>>>        /* Check profile supports HOST_ETHERNET port as this application
>>>            * supports only host port as Ethernet
>>>            */
>>>        for (i = 0; i < info->nb_host_ports; i++) {
>>>                if (info->host_ports[i] == RTE_DWA_TAG_PORT_HOST_ETHERNET); {
>>>                        found = true;
>>>                }
>>>        }
>>>
>>>        /* Done with response, Free the d2h memory allocated by implementation */
>>>        free(obj, d2h);
>>> fail:
>>>        return found;
>>> }
>>>
>>>
>>> bool
>>> dwa_has_profile(enum rte_dwa_tag_profile pf)
>>> {
>>>        enum rte_dwa_tlv_profile *pfs = NULL;
>>>        bool found = false;
>>>        int nb_pfs;
>>>
>>>        /* Get the number of profiles on the DWA device */
>>>        nb_pfs = rte_dwa_dev_disc_profiles(0, NULL);
>>>        pfs = malloc(sizeof(enum rte_dwa_tag_profile)  * nb_pfs);
>>>        /* Fetch all the profiles */
>>>        nb_pfs = rte_dwa_dev_disc_profiles(0, pfs);
>>>
>>>        /* Check the list has requested profile */
>>>        for (i = 0; i < nb_pfs; i++) {
>>>                if (pfs[i] == pf);
>>>                        found = true;
>>>        }
>>>        free(pfs);
>>>
>>>
>>>        return found;
>>> }
>>>
>>>
>>> #include <rte_dwa.h>
>>>
>>> #define DWA_EXCEPTION_PACKETS_PKT_BURST_MAX_SZ                32
>>>
>>> struct app_ctx {
>>>        bool done;
>>>        struct rte_mempool *pkt_pool;
>>>        struct rte_mempool *tlv_pool;
>>>        enum rte_dwa_profile_l3fwd_lookup_mode mode;
>>>        int socket_id;
>>>        int pkt_pool_depth;
>>>        int tlv_pool_depth;
>>>        int rx_queue_depth;
>>> } __rte_cache_aligned;
>>>
>>> int
>>> main(int argc, char **argv)
>>> {
>>>        rte_dwa_obj_t obj = NULL;
>>>        struct app_ctx ctx;
>>>        int rc;
>>>
>>>        /* Initialize EAL */
>>>        rc= rte_eal_init(argc, argv);
>>>           if (rc < 0)
>>>                 rte_exit(EXIT_FAILURE, "Invalid EAL arguments\n");
>>>           argc -= ret;
>>>           argv += ret;
>>>
>>>
>>>        memset(&ctx, 0, sizeof(ctx));
>>>        /* Set application default values */
>>>        ctx->mode = RTE_DWA_PROFILE_L3FWD_MODE_LPM;
>>>        ctx->socket_id = SOCKET_ID_ANY;
>>>        ctx->pkt_pool_depth = 10000;
>>>        ctx->tlv_pool_depth = 10000;
>>>        ctx->rx_queue_depth = 10000;
>>>
>>>        /* Step 1: Check any DWA devices present  */
>>>        rc = rte_dwa_dev_count();
>>>        if (rc <= 0)
>>>                rte_exit(EXIT_FAILURE, "Failed to find DWA devices\n");
>>>
>>>        /* Step 2: Check DWA device has L3FWD profile or not */
>>>        if (!dwa_has_profile(RTE_DWA_TAG_PROFILE_L3FWD))
>>>                rte_exit(EXIT_FAILURE, "L3FWD profile not found\n");
>>>
>>>        /*
>>>         * Step 3: Now that, workload accelerator has L3FWD profile,
>>>         * offload L3FWD workload to accelerator by attaching the profile
>>>         * to accelerator.
>>>         */
>>>        enum rte_dwa_tlv_profile profile[] = {RTE_DWA_TAG_PROFILE_L3FWD};
>>>        obj = rte_dwa_dev_attach(0, "my_custom_accelerator_device", profile, 1).;
>>>
>>>        /* Step 4: Check Attached L3FWD profile has required capability to proceed */
>>>        if (!dwa_profile_l3fwd_has_capa(obj, &ctx))
>>>                rte_exit(EXIT_FAILURE, "L3FWD profile does not have enough capability \n");
>>>
>>>        /* Step 5: Configure l3fwd profile */
>>>        if (!dwa_profile_l3fwd_config(obj, &ctx))
>>>                rte_exit(EXIT_FAILURE, "L3FWD profile configure failed \n");
>>>
>>>        /* Step 6: Configure ethernet host port to receive exception packets */
>>>        if (!dwa_port_host_ethernet_config(obj, &ctx))
>>>                rte_exit(EXIT_FAILURE, "L3FWD profile configure failed \n");
>>>
>>>        /* Step 7 : Move DWA profiles to start state */
>>>        rte_dwa_start(obj);
>>>
>>>        /* Step 8: Handle expectation packets and add lookup rules for it */
>>>        dwa_profile_l3fwd_port_host_ethernet_worker(obj, &ctx);
>>>
>>>        /* Step 9: Clean up */
>>>        rte_dwa_stop(obj);
>>>        rte_dwa_dev_detach(0, obj);
>>>        rte_dwa_dev_close(0);
>>>
>>>        return 0;
>>> }
>>>
>>>
>>> Jerin Jacob (1):
>>>     dwa: introduce dataplane workload accelerator subsystem
>>>
>>>    doc/api/doxy-api-index.md            |  13 +
>>>    doc/api/doxy-api.conf.in             |   1 +
>>>    lib/dwa/dwa.c                        |   7 +
>>>    lib/dwa/meson.build                  |  17 ++
>>>    lib/dwa/rte_dwa.h                    | 184 +++++++++++++
>>>    lib/dwa/rte_dwa_core.h               | 264 +++++++++++++++++++
>>>    lib/dwa/rte_dwa_dev.h                | 154 +++++++++++
>>>    lib/dwa/rte_dwa_port_dwa_ethernet.h  |  68 +++++
>>>    lib/dwa/rte_dwa_port_host_ethernet.h | 178 +++++++++++++
>>>    lib/dwa/rte_dwa_profile_admin.h      |  85 ++++++
>>>    lib/dwa/rte_dwa_profile_l3fwd.h      | 378 +++++++++++++++++++++++++++
>>>    lib/dwa/version.map                  |   3 +
>>>    lib/meson.build                      |   1 +
>>>    13 files changed, 1353 insertions(+)
>>>    create mode 100644 lib/dwa/dwa.c
>>>    create mode 100644 lib/dwa/meson.build
>>>    create mode 100644 lib/dwa/rte_dwa.h
>>>    create mode 100644 lib/dwa/rte_dwa_core.h
>>>    create mode 100644 lib/dwa/rte_dwa_dev.h
>>>    create mode 100644 lib/dwa/rte_dwa_port_dwa_ethernet.h
>>>    create mode 100644 lib/dwa/rte_dwa_port_host_ethernet.h
>>>    create mode 100644 lib/dwa/rte_dwa_profile_admin.h
>>>    create mode 100644 lib/dwa/rte_dwa_profile_l3fwd.h
>>>    create mode 100644 lib/dwa/version.map
>>>


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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-29 11:57     ` Mattias Rönnblom
@ 2021-10-29 15:51       ` Jerin Jacob
  2021-10-31  9:18         ` Mattias Rönnblom
  0 siblings, 1 reply; 20+ messages in thread
From: Jerin Jacob @ 2021-10-29 15:51 UTC (permalink / raw)
  To: Mattias Rönnblom
  Cc: jerinj, dev, thomas, ferruh.yigit, ajit.khaparde, aboyer,
	andrew.rybchenko, beilei.xing, bruce.richardson, chas3,
	chenbo.xia, ciara.loftus, dsinghrawat, ed.czeck, evgenys, grive,
	g.singh, zhouguoyang, haiyue.wang, hkalra, heinrich.kuhn,
	hemant.agrawal, hyonkim, igorch, irusskikh, jgrajcia,
	jasvinder.singh, jianwang, jiawenwu, jingjing.wu, johndale,
	john.miller, linville, keith.wiles, kirankumark, oulijun, lironh,
	longli, mw, spinler, matan, matt.peters, maxime.coquelin, mk,
	humin29, pnalla, ndabilpuram, qiming.yang, qi.z.zhang, radhac,
	rahul.lakkireddy, rmody, rosen.xu, sachin.saxena, skoteshwar,
	shshaikh, shaibran, shepard.siegel, asomalap, somnath.kotur,
	sthemmin, steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, ruifeng.wang, drc, konstantin.ananyev,
	olivier.matz, jay.jayatheerthan, asekhar, pbhagavatula,
	Elana Agostini

On Fri, Oct 29, 2021 at 5:27 PM Mattias Rönnblom
<mattias.ronnblom@ericsson.com> wrote:
>
> On 2021-10-25 11:03, Jerin Jacob wrote:
> > On Mon, Oct 25, 2021 at 1:05 PM Mattias Rönnblom
> > <mattias.ronnblom@ericsson.com> wrote:
> >> On 2021-10-19 20:14, jerinj@marvell.com wrote:
> >>> From: Jerin Jacob <jerinj@marvell.com>
> >>>
> >>>
> >>> Dataplane Workload Accelerator library
> >>> ======================================
> >>>
> >>> Definition of Dataplane Workload Accelerator
> >>> --------------------------------------------
> >>> Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> >>> Network controllers and programmable data acceleration engines for
> >>> packet processing, cryptography, regex engines, baseband processing, etc.
> >>> This allows DWA to offload  compute/packet processing/baseband/
> >>> cryptography-related workload from the host CPU to save the cost and power.
> >>> Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
> >>>
> >>> Unlike other devices in DPDK, the DWA device is not fixed-function
> >>> due to the fact that it has CPUs and programmable HW accelerators.
> >>
> >> There are already several instances of DPDK devices with pure-software
> >> implementation. In this regard, a DPU/SmartNIC represents nothing new.
> >> What's new, it seems to me, is a much-increased need to
> >> configure/arrange the processing in complex manners, to avoid bouncing
> >> everything to the host CPU.
> > Yes and No. It will be based on the profile. The TLV type TYPE_USER_PLANE will
> > have user plane traffic from/to host. For example, offloading ORAN split 7.2
> > baseband profile. Transport blocks sent to/from host as TYPE_USER_PLANE.
> >
> >> Something like P4 or rte_flow-based hooks or
> >> some other kind of extension. The eventdev adapters solve the same
> >> problem (where on some systems packets go through the host CPU on their
> >> way to the event device, and others do not) - although on a *much*
> >> smaller scale.
> > Yes. Eventdev Adapters only for event device plumbing.
> >
> >
> >>
> >> "Not-fixed function" seems to call for more hot plug support in the
> >> device APIs. Such functionality could then be reused by anything that
> >> can be reconfigured dynamically (FPGAs, firmware-programmed
> >> accelerators, etc.),
> > Yes.
> >
> >> but which may not be able to serve as a RPC
> >> endpoint, like a SmartNIC.
> > It can. That's the reason for choosing TLVs. So that
> > any higher level language can use TLVs like https://protect2.fireeye.com/v1/url?k=96886daf-c91357b6-96882d34-8682aaa22bc0-c994a5dcbda5d9e8&q=1&e=e89c0aca-a3b3-4f72-b616-ba4550b856b6&u=https%3A%2F%2Fgithub.com%2Fustropo%2Futtlv
> > to communicate with the accelerator.  TLVs follow the request and
> > response scheme like RPC. So it can warp it under application if needed.
> >
> >>
> >> DWA could be some kind of DPDK-internal framework for managing certain
> >> type of DPUs, but should it be exposed to the user application?
> >
> > Could you clarify a bit more.
> > The offload is represented as a set of TLVs in generic fashion. There
> > is no DPU specific bit in offload representation. See
> > rte_dwa_profiile_l3fwd.h header file.
>
>
> It seems a bit cumbersome to work with TLVs on the user application
> side. Would it be an alternative to have the profile API as a set of C
> APIs instead of TLV-based messaging interface? The underlying
> implementation could still be - in many or all cases - be TLVs sent over
> some appropriate transport.

The reason to pick TLVs is as follows

1) Very easy to enable ABI compatibility. (Learned from rte_flow)
2) If it needs to be transported over network etc it needs to be
packed so that way
it is easy for implementation to do that with TLV also it gives better
performance in such
cases by avoiding reformatting or possibly avoiding memcpy etc.
3) It is easy to plugin with another high-level programing language as
just one API
4) Easy to decouple DWA core library functionalities from profile.
5) Easy to enable asynchronous scheme using request and response TLVs.
6) Most importantly, We could introduce type notion with TLV
(connected with the type of message  See TYPE_ATTACHED, TYPE_STOPPED,
TYPE_USER_PLANE etc ),
That way, we can have a uniform outlook of profiles instead of each profile
coming with a setup of its own APIs and __rules__ on the state machine.
I think, for a framework to leverage communication mechanisms and other
aspects between profiles, it's important to have some synergy between profiles.


Yes. I agree that a bit more logic is required on the application side
to use TLV,
But I think we can have a wrapper function getting req and response structures.

>
> Such a C API could still be asynchronous, and still be a profile API
> (rather than a set of new DPDK device types).
>
>
> What I tried to ask during the meeting but where I didn't get an answer
> (or at least one that I could understand) was how the profiles was to be
> specified and/or documented. Maybe the above is what you had in mind
> already.

Yes. Documentation is easy, please check the RFC header file for Doxygen
meta to express all the attributes of a TLV.


+enum rte_dwa_port_host_ethernet {
+ /**
+ * Attribute |  Value
+ * ----------|--------
+ * Tag       | RTE_DWA_TAG_PORT_HOST_ETHERNET
+ * Stag      | RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO
+ * Direction | H2D
+ * Type      | TYPE_ATTACHED
+ * Payload   | NA
+ * Pair TLV  | RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO
+ *
+ * Request DWA host ethernet port information.
+ */
+ RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO,
+ /**
+ * Attribute |  Value
+ * ----------|---------
+ * Tag       | RTE_DWA_TAG_PORT_HOST_ETHERNET
+ * Stag      | RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO
+ * Direction | H2D
+ * Type      | TYPE_ATTACHED
+ * Payload   | struct rte_dwa_port_host_ethernet_d2h_info
+ * Pair TLV  | RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO
+ *
+ * Response for DWA host ethernet port information.
+ */
+ RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO,

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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-29 15:51       ` Jerin Jacob
@ 2021-10-31  9:18         ` Mattias Rönnblom
  2021-10-31 14:01           ` Jerin Jacob
  0 siblings, 1 reply; 20+ messages in thread
From: Mattias Rönnblom @ 2021-10-31  9:18 UTC (permalink / raw)
  To: Jerin Jacob
  Cc: jerinj, dev, thomas, ferruh.yigit, ajit.khaparde, aboyer,
	andrew.rybchenko, beilei.xing, bruce.richardson, chas3,
	chenbo.xia, ciara.loftus, dsinghrawat, ed.czeck, evgenys, grive,
	g.singh, zhouguoyang, haiyue.wang, hkalra, heinrich.kuhn,
	hemant.agrawal, hyonkim, igorch, irusskikh, jgrajcia,
	jasvinder.singh, jianwang, jiawenwu, jingjing.wu, johndale,
	john.miller, linville, keith.wiles, kirankumark, oulijun, lironh,
	longli, mw, spinler, matan, matt.peters, maxime.coquelin, mk,
	humin29, pnalla, ndabilpuram, qiming.yang, qi.z.zhang, radhac,
	rahul.lakkireddy, rmody, rosen.xu, sachin.saxena, skoteshwar,
	shshaikh, shaibran, shepard.siegel, asomalap, somnath.kotur,
	sthemmin, steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, ruifeng.wang, drc, konstantin.ananyev,
	olivier.matz, jay.jayatheerthan, asekhar, pbhagavatula,
	Elana Agostini

On 2021-10-29 17:51, Jerin Jacob wrote:
> On Fri, Oct 29, 2021 at 5:27 PM Mattias Rönnblom
> <mattias.ronnblom@ericsson.com> wrote:
>> On 2021-10-25 11:03, Jerin Jacob wrote:
>>> On Mon, Oct 25, 2021 at 1:05 PM Mattias Rönnblom
>>> <mattias.ronnblom@ericsson.com> wrote:
>>>> On 2021-10-19 20:14, jerinj@marvell.com wrote:
>>>>> From: Jerin Jacob <jerinj@marvell.com>
>>>>>
>>>>>
>>>>> Dataplane Workload Accelerator library
>>>>> ======================================
>>>>>
>>>>> Definition of Dataplane Workload Accelerator
>>>>> --------------------------------------------
>>>>> Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
>>>>> Network controllers and programmable data acceleration engines for
>>>>> packet processing, cryptography, regex engines, baseband processing, etc.
>>>>> This allows DWA to offload  compute/packet processing/baseband/
>>>>> cryptography-related workload from the host CPU to save the cost and power.
>>>>> Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
>>>>>
>>>>> Unlike other devices in DPDK, the DWA device is not fixed-function
>>>>> due to the fact that it has CPUs and programmable HW accelerators.
>>>> There are already several instances of DPDK devices with pure-software
>>>> implementation. In this regard, a DPU/SmartNIC represents nothing new.
>>>> What's new, it seems to me, is a much-increased need to
>>>> configure/arrange the processing in complex manners, to avoid bouncing
>>>> everything to the host CPU.
>>> Yes and No. It will be based on the profile. The TLV type TYPE_USER_PLANE will
>>> have user plane traffic from/to host. For example, offloading ORAN split 7.2
>>> baseband profile. Transport blocks sent to/from host as TYPE_USER_PLANE.
>>>
>>>> Something like P4 or rte_flow-based hooks or
>>>> some other kind of extension. The eventdev adapters solve the same
>>>> problem (where on some systems packets go through the host CPU on their
>>>> way to the event device, and others do not) - although on a *much*
>>>> smaller scale.
>>> Yes. Eventdev Adapters only for event device plumbing.
>>>
>>>
>>>> "Not-fixed function" seems to call for more hot plug support in the
>>>> device APIs. Such functionality could then be reused by anything that
>>>> can be reconfigured dynamically (FPGAs, firmware-programmed
>>>> accelerators, etc.),
>>> Yes.
>>>
>>>> but which may not be able to serve as a RPC
>>>> endpoint, like a SmartNIC.
>>> It can. That's the reason for choosing TLVs. So that
>>> any higher level language can use TLVs like https://protect2.fireeye.com/v1/url?k=96886daf-c91357b6-96882d34-8682aaa22bc0-c994a5dcbda5d9e8&q=1&e=e89c0aca-a3b3-4f72-b616-ba4550b856b6&u=https%3A%2F%2Fgithub.com%2Fustropo%2Futtlv
>>> to communicate with the accelerator.  TLVs follow the request and
>>> response scheme like RPC. So it can warp it under application if needed.
>>>
>>>> DWA could be some kind of DPDK-internal framework for managing certain
>>>> type of DPUs, but should it be exposed to the user application?
>>> Could you clarify a bit more.
>>> The offload is represented as a set of TLVs in generic fashion. There
>>> is no DPU specific bit in offload representation. See
>>> rte_dwa_profiile_l3fwd.h header file.
>>
>> It seems a bit cumbersome to work with TLVs on the user application
>> side. Would it be an alternative to have the profile API as a set of C
>> APIs instead of TLV-based messaging interface? The underlying
>> implementation could still be - in many or all cases - be TLVs sent over
>> some appropriate transport.
> The reason to pick TLVs is as follows
>
> 1) Very easy to enable ABI compatibility. (Learned from rte_flow)


Do you include the TLV-defined profile interface in "ABI"? Or do you 
with ABI only mean the C ABI to send/receive TLVs? To me, the former 
makes the most sense, since changing the profile will break binary 
compatibility with then-existing applications.


> 2) If it needs to be transported over network etc it needs to be
> packed so that way
> it is easy for implementation to do that with TLV also it gives better
> performance in such
> cases by avoiding reformatting or possibly avoiding memcpy etc.

My question was not "why TLVs", but the more specific "why are TLVs 
exposed to the user application." I find it likely the user applications 
are going to wrap the TLV serialization and de-serialization into their 
own functions.


> 3) It is easy to plugin with another high-level programing language as
> just one API


Make sense. One note though: the transport is just one API, but then 
each profile makes up an API as well, although it's not C, but TLV-based.


> 4) Easy to decouple DWA core library functionalities from profile.
> 5) Easy to enable asynchronous scheme using request and response TLVs.
> 6) Most importantly, We could introduce type notion with TLV
> (connected with the type of message  See TYPE_ATTACHED, TYPE_STOPPED,
> TYPE_USER_PLANE etc ),
> That way, we can have a uniform outlook of profiles instead of each profile
> coming with a setup of its own APIs and __rules__ on the state machine.
> I think, for a framework to leverage communication mechanisms and other
> aspects between profiles, it's important to have some synergy between profiles.
>
>
> Yes. I agree that a bit more logic is required on the application side
> to use TLV,
> But I think we can have a wrapper function getting req and response structures.


Do you think ethdev, eventdev, cryptodev and the other DPDK APIs had 
been better off as TLV-based messaging interfaces as well? From a user 
point of view, I'm not sure I see what's so special about talking to a 
SmartNIC compared to functions implemented in a GPU, FPGA, an 
fix-function ASIC, a large array of garden gnomes or some other manner. 
More functionality and more need for asynchronicity (if that's a word) 
maybe.


>> Such a C API could still be asynchronous, and still be a profile API
>> (rather than a set of new DPDK device types).
>>
>>
>> What I tried to ask during the meeting but where I didn't get an answer
>> (or at least one that I could understand) was how the profiles was to be
>> specified and/or documented. Maybe the above is what you had in mind
>> already.
> Yes. Documentation is easy, please check the RFC header file for Doxygen
> meta to express all the attributes of a TLV.
>
>
> +enum rte_dwa_port_host_ethernet {
> + /**
> + * Attribute |  Value
> + * ----------|--------
> + * Tag       | RTE_DWA_TAG_PORT_HOST_ETHERNET
> + * Stag      | RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO
> + * Direction | H2D
> + * Type      | TYPE_ATTACHED
> + * Payload   | NA
> + * Pair TLV  | RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO
> + *
> + * Request DWA host ethernet port information.
> + */
> + RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO,
> + /**
> + * Attribute |  Value
> + * ----------|---------
> + * Tag       | RTE_DWA_TAG_PORT_HOST_ETHERNET
> + * Stag      | RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO
> + * Direction | H2D
> + * Type      | TYPE_ATTACHED
> + * Payload   | struct rte_dwa_port_host_ethernet_d2h_info
> + * Pair TLV  | RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO
> + *
> + * Response for DWA host ethernet port information.
> + */
> + RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO,


Thanks for the pointer.


It would make sense to have a machine-readable schema, so you can 
generate the (in my view) inevitable wrapper code. Much like what gRPC 
is to protobuf, or Sun RPC to XDR.


Why not use protobuf and its IDL to specify the interface?



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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-31  9:18         ` Mattias Rönnblom
@ 2021-10-31 14:01           ` Jerin Jacob
  2021-10-31 19:34             ` Thomas Monjalon
  0 siblings, 1 reply; 20+ messages in thread
From: Jerin Jacob @ 2021-10-31 14:01 UTC (permalink / raw)
  To: Mattias Rönnblom
  Cc: jerinj, dev, thomas, ferruh.yigit, ajit.khaparde, aboyer,
	andrew.rybchenko, beilei.xing, bruce.richardson, chas3,
	chenbo.xia, ciara.loftus, dsinghrawat, ed.czeck, evgenys, grive,
	g.singh, zhouguoyang, haiyue.wang, hkalra, heinrich.kuhn,
	hemant.agrawal, hyonkim, igorch, irusskikh, jgrajcia,
	jasvinder.singh, jianwang, jiawenwu, jingjing.wu, johndale,
	john.miller, linville, keith.wiles, kirankumark, oulijun, lironh,
	longli, mw, spinler, matan, matt.peters, maxime.coquelin, mk,
	humin29, pnalla, ndabilpuram, qiming.yang, qi.z.zhang, radhac,
	rahul.lakkireddy, rmody, rosen.xu, sachin.saxena, skoteshwar,
	shshaikh, shaibran, shepard.siegel, asomalap, somnath.kotur,
	sthemmin, steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, ruifeng.wang, drc, konstantin.ananyev,
	olivier.matz, jay.jayatheerthan, asekhar, pbhagavatula,
	Elana Agostini

On Sun, Oct 31, 2021 at 2:48 PM Mattias Rönnblom
<mattias.ronnblom@ericsson.com> wrote:
>
> On 2021-10-29 17:51, Jerin Jacob wrote:
> > On Fri, Oct 29, 2021 at 5:27 PM Mattias Rönnblom
> > <mattias.ronnblom@ericsson.com> wrote:
> >> On 2021-10-25 11:03, Jerin Jacob wrote:
> >>> On Mon, Oct 25, 2021 at 1:05 PM Mattias Rönnblom
> >>> <mattias.ronnblom@ericsson.com> wrote:
> >>>> On 2021-10-19 20:14, jerinj@marvell.com wrote:
> >>>>> From: Jerin Jacob <jerinj@marvell.com>
> >>>>>
> >>>>>
> >>>>> Dataplane Workload Accelerator library
> >>>>> ======================================
> >>>>>
> >>>>> Definition of Dataplane Workload Accelerator
> >>>>> --------------------------------------------
> >>>>> Dataplane Workload Accelerator(DWA) typically contains a set of CPUs,
> >>>>> Network controllers and programmable data acceleration engines for
> >>>>> packet processing, cryptography, regex engines, baseband processing, etc.
> >>>>> This allows DWA to offload  compute/packet processing/baseband/
> >>>>> cryptography-related workload from the host CPU to save the cost and power.
> >>>>> Also to enable scaling the workload by adding DWAs to the Host CPU as needed.
> >>>>>
> >>>>> Unlike other devices in DPDK, the DWA device is not fixed-function
> >>>>> due to the fact that it has CPUs and programmable HW accelerators.
> >>>> There are already several instances of DPDK devices with pure-software
> >>>> implementation. In this regard, a DPU/SmartNIC represents nothing new.
> >>>> What's new, it seems to me, is a much-increased need to
> >>>> configure/arrange the processing in complex manners, to avoid bouncing
> >>>> everything to the host CPU.
> >>> Yes and No. It will be based on the profile. The TLV type TYPE_USER_PLANE will
> >>> have user plane traffic from/to host. For example, offloading ORAN split 7.2
> >>> baseband profile. Transport blocks sent to/from host as TYPE_USER_PLANE.
> >>>
> >>>> Something like P4 or rte_flow-based hooks or
> >>>> some other kind of extension. The eventdev adapters solve the same
> >>>> problem (where on some systems packets go through the host CPU on their
> >>>> way to the event device, and others do not) - although on a *much*
> >>>> smaller scale.
> >>> Yes. Eventdev Adapters only for event device plumbing.
> >>>
> >>>
> >>>> "Not-fixed function" seems to call for more hot plug support in the
> >>>> device APIs. Such functionality could then be reused by anything that
> >>>> can be reconfigured dynamically (FPGAs, firmware-programmed
> >>>> accelerators, etc.),
> >>> Yes.
> >>>
> >>>> but which may not be able to serve as a RPC
> >>>> endpoint, like a SmartNIC.
> >>> It can. That's the reason for choosing TLVs. So that
> >>> any higher level language can use TLVs like https://protect2.fireeye.com/v1/url?k=96886daf-c91357b6-96882d34-8682aaa22bc0-c994a5dcbda5d9e8&q=1&e=e89c0aca-a3b3-4f72-b616-ba4550b856b6&u=https%3A%2F%2Fgithub.com%2Fustropo%2Futtlv
> >>> to communicate with the accelerator.  TLVs follow the request and
> >>> response scheme like RPC. So it can warp it under application if needed.
> >>>
> >>>> DWA could be some kind of DPDK-internal framework for managing certain
> >>>> type of DPUs, but should it be exposed to the user application?
> >>> Could you clarify a bit more.
> >>> The offload is represented as a set of TLVs in generic fashion. There
> >>> is no DPU specific bit in offload representation. See
> >>> rte_dwa_profiile_l3fwd.h header file.
> >>
> >> It seems a bit cumbersome to work with TLVs on the user application
> >> side. Would it be an alternative to have the profile API as a set of C
> >> APIs instead of TLV-based messaging interface? The underlying
> >> implementation could still be - in many or all cases - be TLVs sent over
> >> some appropriate transport.
> > The reason to pick TLVs is as follows
> >
> > 1) Very easy to enable ABI compatibility. (Learned from rte_flow)
>
>
> Do you include the TLV-defined profile interface in "ABI"? Or do you
> with ABI only mean the C ABI to send/receive TLVs? To me, the former
> makes the most sense, since changing the profile will break binary
> compatibility with then-existing applications.

The TLV payload will be as part of ABI just like rte_flow.
If there is  ABI breakage on any TLV we can add a new Tag and it is associated
payload to enable backward compatibility. i.e old TLV will work
without any change

>
>
> > 2) If it needs to be transported over network etc it needs to be
> > packed so that way
> > it is easy for implementation to do that with TLV also it gives better
> > performance in such
> > cases by avoiding reformatting or possibly avoiding memcpy etc.
>
> My question was not "why TLVs", but the more specific "why are TLVs
> exposed to the user application." I find it likely the user applications
> are going to wrap the TLV serialization and de-serialization into their
> own functions.

We can stack up the TLVs, unlike traditional function calls.
Those things really need if the device supports N profiles so multiple TLVs
can be used in a single shot in fastpath.


>
>
> > 3) It is easy to plugin with another high-level programing language as
> > just one API
>
>
> Make sense. One note though: the transport is just one API, but then
> each profile makes up an API as well, although it's not C, but TLV-based.

Yes,

>
>
> > 4) Easy to decouple DWA core library functionalities from profile.
> > 5) Easy to enable asynchronous scheme using request and response TLVs.
> > 6) Most importantly, We could introduce type notion with TLV
> > (connected with the type of message  See TYPE_ATTACHED, TYPE_STOPPED,
> > TYPE_USER_PLANE etc ),
> > That way, we can have a uniform outlook of profiles instead of each profile
> > coming with a setup of its own APIs and __rules__ on the state machine.
> > I think, for a framework to leverage communication mechanisms and other
> > aspects between profiles, it's important to have some synergy between profiles.
> >
> >
> > Yes. I agree that a bit more logic is required on the application side
> > to use TLV,
> > But I think we can have a wrapper function getting req and response structures.
>
>
> Do you think ethdev, eventdev, cryptodev and the other DPDK APIs had
> been better off as TLV-based messaging interfaces as well? From a user
> point of view, I'm not sure I see what's so special about talking to a
> SmartNIC compared to functions implemented in a GPU, FPGA, an
> fix-function ASIC, a large array of garden gnomes or some other manner.
> More functionality and more need for asynchronicity (if that's a word)
> maybe.

No. I am trying to avoid creating 1000s of API and it is driver hooks
for all profiles and enable symmetry between all the profiles by
attaching state, type attributes to TLV so that we can get a unified view.
Nothing specific to  SmartNIC/GPU/FPGA.
Also, TLVs are very common in interoperable solutions like
https://scf.io/en/documents/222_5G_FAPI_PHY_API_Specification.php


>
> >> Such a C API could still be asynchronous, and still be a profile API
> >> (rather than a set of new DPDK device types).
> >>
> >>
> >> What I tried to ask during the meeting but where I didn't get an answer
> >> (or at least one that I could understand) was how the profiles was to be
> >> specified and/or documented. Maybe the above is what you had in mind
> >> already.
> > Yes. Documentation is easy, please check the RFC header file for Doxygen
> > meta to express all the attributes of a TLV.
> >
> >
> > +enum rte_dwa_port_host_ethernet {
> > + /**
> > + * Attribute |  Value
> > + * ----------|--------
> > + * Tag       | RTE_DWA_TAG_PORT_HOST_ETHERNET
> > + * Stag      | RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO
> > + * Direction | H2D
> > + * Type      | TYPE_ATTACHED
> > + * Payload   | NA
> > + * Pair TLV  | RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO
> > + *
> > + * Request DWA host ethernet port information.
> > + */
> > + RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO,
> > + /**
> > + * Attribute |  Value
> > + * ----------|---------
> > + * Tag       | RTE_DWA_TAG_PORT_HOST_ETHERNET
> > + * Stag      | RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO
> > + * Direction | H2D
> > + * Type      | TYPE_ATTACHED
> > + * Payload   | struct rte_dwa_port_host_ethernet_d2h_info
> > + * Pair TLV  | RTE_DWA_STAG_PORT_HOST_ETHERNET_H2D_INFO
> > + *
> > + * Response for DWA host ethernet port information.
> > + */
> > + RTE_DWA_STAG_PORT_HOST_ETHERNET_D2H_INFO,
>
>
> Thanks for the pointer.
>
>
> It would make sense to have a machine-readable schema, so you can
> generate the (in my view) inevitable wrapper code. Much like what gRPC
> is to protobuf, or Sun RPC to XDR.

I thought of doing that, I thought it may not be good due to
1) Additional library dependencies
2) Performance overhead of such solutions.
Not all the transports are not supported in all the libraries
and allow drivers to enable any sort of transport.
3) Keep it simple
4) Better asynchronous support.
5) If someone needs gRPC kind of thing, it can be wrapped over TLV.

Since rte_flow already has the TLV concept it may not be new to DPDK.
I really liked rte_flow enablement of ABI combability and its ease of adding
new stuff. Try to follow similar stuff which is proven in DPDK.
Ie. New profile creation will very easy, it will be a matter of identifying
the TLVs and their type and payload, rather than everyone comes with
new APIs in every profile.


>
>
> Why not use protobuf and its IDL to specify the interface?
>
>

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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-31 14:01           ` Jerin Jacob
@ 2021-10-31 19:34             ` Thomas Monjalon
  2021-10-31 21:13               ` Jerin Jacob
  0 siblings, 1 reply; 20+ messages in thread
From: Thomas Monjalon @ 2021-10-31 19:34 UTC (permalink / raw)
  To: Mattias Rönnblom, Jerin Jacob
  Cc: jerinj, dev, ferruh.yigit, ajit.khaparde, aboyer,
	andrew.rybchenko, beilei.xing, bruce.richardson, chas3,
	chenbo.xia, ciara.loftus, dsinghrawat, ed.czeck, evgenys, grive,
	g.singh, zhouguoyang, haiyue.wang, hkalra, heinrich.kuhn,
	hemant.agrawal, hyonkim, igorch, irusskikh, jgrajcia,
	jasvinder.singh, jianwang, jiawenwu, jingjing.wu, johndale,
	john.miller, linville, keith.wiles, kirankumark, oulijun, lironh,
	longli, mw, spinler, matan, matt.peters, maxime.coquelin, mk,
	humin29, pnalla, ndabilpuram, qiming.yang, qi.z.zhang, radhac,
	rahul.lakkireddy, rmody, rosen.xu, sachin.saxena, skoteshwar,
	shshaikh, shaibran, shepard.siegel, asomalap, somnath.kotur,
	sthemmin, steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, ruifeng.wang, drc, konstantin.ananyev,
	olivier.matz, jay.jayatheerthan, asekhar, pbhagavatula,
	Elana Agostini

31/10/2021 15:01, Jerin Jacob:
> Since rte_flow already has the TLV concept it may not be new to DPDK.

Where is there TLV in rte_flow?

> I really liked rte_flow enablement of ABI combability and its ease of adding
> new stuff. Try to follow similar stuff which is proven in DPDK.
> Ie. New profile creation will very easy, it will be a matter of identifying
> the TLVs and their type and payload, rather than everyone comes with
> new APIs in every profile.
> 
> > Why not use protobuf and its IDL to specify the interface?

Yes I think it is important to discuss alternatives,
and at least get justifications of why TLV is chosen among others.



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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-31 19:34             ` Thomas Monjalon
@ 2021-10-31 21:13               ` Jerin Jacob
  2021-10-31 21:55                 ` Thomas Monjalon
  0 siblings, 1 reply; 20+ messages in thread
From: Jerin Jacob @ 2021-10-31 21:13 UTC (permalink / raw)
  To: Thomas Monjalon
  Cc: Mattias Rönnblom, jerinj, dev, ferruh.yigit, ajit.khaparde,
	aboyer, andrew.rybchenko, beilei.xing, bruce.richardson, chas3,
	chenbo.xia, ciara.loftus, dsinghrawat, ed.czeck, evgenys, grive,
	g.singh, zhouguoyang, haiyue.wang, hkalra, heinrich.kuhn,
	hemant.agrawal, hyonkim, igorch, irusskikh, jgrajcia,
	jasvinder.singh, jianwang, jiawenwu, jingjing.wu, johndale,
	john.miller, linville, keith.wiles, kirankumark, oulijun, lironh,
	longli, mw, spinler, matan, matt.peters, maxime.coquelin, mk,
	humin29, pnalla, ndabilpuram, qiming.yang, qi.z.zhang, radhac,
	rahul.lakkireddy, rmody, rosen.xu, sachin.saxena, skoteshwar,
	shshaikh, shaibran, shepard.siegel, asomalap, somnath.kotur,
	sthemmin, steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, ruifeng.wang, drc, konstantin.ananyev,
	olivier.matz, jay.jayatheerthan, asekhar, pbhagavatula,
	Elana Agostini

On Mon, Nov 1, 2021 at 1:04 AM Thomas Monjalon <thomas@monjalon.net> wrote:
>
> 31/10/2021 15:01, Jerin Jacob:
> > Since rte_flow already has the TLV concept it may not be new to DPDK.
>
> Where is there TLV in rte_flow?

struct rte_flow_item {
        enum rte_flow_item_type type; /**< Item type. */
        const void *spec; /**< Pointer to item specification structure. */

Type is the tag here and the spec is the value here. Length is the
size of the specification structure.
rte_flows spec does not support/need zero length variable at the end
of spec structure,
that reason for not embedding explicit length value as it is can be
derived from sizeof(specification structure).


>
> > I really liked rte_flow enablement of ABI combability and its ease of adding
> > new stuff. Try to follow similar stuff which is proven in DPDK.
> > Ie. New profile creation will very easy, it will be a matter of identifying
> > the TLVs and their type and payload, rather than everyone comes with
> > new APIs in every profile.
> >
> > > Why not use protobuf and its IDL to specify the interface?
>
> Yes I think it is important to discuss alternatives,
> and at least get justifications of why TLV is chosen among others.

Yes. Current list is

1) Very easy to enable ABI compatibility.
2) If it needs to be transported over network etc it needs to be
packed so that way it is easy for implementation to do that
with TLV also gives better performance in such
cases by avoiding reformatting or possibly avoiding memcpy etc.
3) It is easy to plugin with another high-level programing language as
just one API.
4) Easy to decouple DWA core library functionalities from profile.
5) Easy to enable asynchronous scheme using request and response TLVs.
6) Most importantly, We could introduce type notion with TLV
(connected with the type of message  See TYPE_ATTACHED, TYPE_STOPPED,
TYPE_USER_PLANE etc ),
That way, we can have a uniform outlook of profiles instead of each profile
coming with a setup of its own APIs and __rules__ on the state machine.
I think, for a framework to leverage communication mechanisms and other
aspects between profiles, it's important to have some synergy between profiles.
7) No Additional library dependencies like gRPC, protobuf
8) Provide driver to implement the optimized means of supporting different
transport such as Ethernet, Shared memory, PCIe DMA style HW etc.
9) Avoid creating endless APIs and their associated driver function
calls for each
profile APIs.


>
>

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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-31 21:13               ` Jerin Jacob
@ 2021-10-31 21:55                 ` Thomas Monjalon
  2021-10-31 22:19                   ` Jerin Jacob
  0 siblings, 1 reply; 20+ messages in thread
From: Thomas Monjalon @ 2021-10-31 21:55 UTC (permalink / raw)
  To: Jerin Jacob
  Cc: Mattias Rönnblom, jerinj, dev, ferruh.yigit, ajit.khaparde,
	aboyer, andrew.rybchenko, beilei.xing, bruce.richardson, chas3,
	chenbo.xia, ciara.loftus, dsinghrawat, ed.czeck, evgenys, grive,
	g.singh, zhouguoyang, haiyue.wang, hkalra, heinrich.kuhn,
	hemant.agrawal, hyonkim, igorch, irusskikh, jgrajcia,
	jasvinder.singh, jianwang, jiawenwu, jingjing.wu, johndale,
	john.miller, linville, keith.wiles, kirankumark, oulijun, lironh,
	longli, mw, spinler, matan, matt.peters, maxime.coquelin, mk,
	humin29, pnalla, ndabilpuram, qiming.yang, qi.z.zhang, radhac,
	rahul.lakkireddy, rmody, rosen.xu, sachin.saxena, skoteshwar,
	shshaikh, shaibran, shepard.siegel, asomalap, somnath.kotur,
	sthemmin, steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, ruifeng.wang, drc, konstantin.ananyev,
	olivier.matz, jay.jayatheerthan, asekhar, pbhagavatula,
	Elana Agostini

31/10/2021 22:13, Jerin Jacob:
> On Mon, Nov 1, 2021 at 1:04 AM Thomas Monjalon <thomas@monjalon.net> wrote:
> >
> > 31/10/2021 15:01, Jerin Jacob:
> > > Since rte_flow already has the TLV concept it may not be new to DPDK.
> >
> > Where is there TLV in rte_flow?
> 
> struct rte_flow_item {
>         enum rte_flow_item_type type; /**< Item type. */
>         const void *spec; /**< Pointer to item specification structure. */
> 
> Type is the tag here and the spec is the value here. Length is the
> size of the specification structure.
> rte_flows spec does not support/need zero length variable at the end
> of spec structure,
> that reason for not embedding explicit length value as it is can be
> derived from sizeof(specification structure).

Ah OK I see what you mean.
But rte_flow_item is quite limited,
it is not the kind of TLV with multiple levels of nesting.
Do you need nesting of objects in DWA?

> > > I really liked rte_flow enablement of ABI combability and its ease of adding
> > > new stuff. Try to follow similar stuff which is proven in DPDK.
> > > Ie. New profile creation will very easy, it will be a matter of identifying
> > > the TLVs and their type and payload, rather than everyone comes with
> > > new APIs in every profile.
> > >
> > > > Why not use protobuf and its IDL to specify the interface?
> >
> > Yes I think it is important to discuss alternatives,
> > and at least get justifications of why TLV is chosen among others.
> 
> Yes. Current list is
> 
> 1) Very easy to enable ABI compatibility.
> 2) If it needs to be transported over network etc it needs to be
> packed so that way it is easy for implementation to do that
> with TLV also gives better performance in such
> cases by avoiding reformatting or possibly avoiding memcpy etc.
> 3) It is easy to plugin with another high-level programing language as
> just one API.
> 4) Easy to decouple DWA core library functionalities from profile.
> 5) Easy to enable asynchronous scheme using request and response TLVs.
> 6) Most importantly, We could introduce type notion with TLV
> (connected with the type of message  See TYPE_ATTACHED, TYPE_STOPPED,
> TYPE_USER_PLANE etc ),
> That way, we can have a uniform outlook of profiles instead of each profile
> coming with a setup of its own APIs and __rules__ on the state machine.
> I think, for a framework to leverage communication mechanisms and other
> aspects between profiles, it's important to have some synergy between profiles.
> 7) No Additional library dependencies like gRPC, protobuf
> 8) Provide driver to implement the optimized means of supporting different
> transport such as Ethernet, Shared memory, PCIe DMA style HW etc.
> 9) Avoid creating endless APIs and their associated driver function
> calls for each
> profile APIs.




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

* Re: [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library
  2021-10-31 21:55                 ` Thomas Monjalon
@ 2021-10-31 22:19                   ` Jerin Jacob
  0 siblings, 0 replies; 20+ messages in thread
From: Jerin Jacob @ 2021-10-31 22:19 UTC (permalink / raw)
  To: Thomas Monjalon
  Cc: Mattias Rönnblom, jerinj, dev, ferruh.yigit, ajit.khaparde,
	aboyer, andrew.rybchenko, beilei.xing, bruce.richardson, chas3,
	chenbo.xia, ciara.loftus, dsinghrawat, ed.czeck, evgenys, grive,
	g.singh, zhouguoyang, haiyue.wang, hkalra, heinrich.kuhn,
	hemant.agrawal, hyonkim, igorch, irusskikh, jgrajcia,
	jasvinder.singh, jianwang, jiawenwu, jingjing.wu, johndale,
	john.miller, linville, keith.wiles, kirankumark, oulijun, lironh,
	longli, mw, spinler, matan, matt.peters, maxime.coquelin, mk,
	humin29, pnalla, ndabilpuram, qiming.yang, qi.z.zhang, radhac,
	rahul.lakkireddy, rmody, rosen.xu, sachin.saxena, skoteshwar,
	shshaikh, shaibran, shepard.siegel, asomalap, somnath.kotur,
	sthemmin, steven.webster, skori, mtetsuyah, vburru, viacheslavo,
	xiao.w.wang, cloud.wangxiaoyun, yisen.zhuang, yongwang,
	xuanziyang2, pkapoor, nadavh, sburla, pathreya, gakhil, mdr,
	dmitry.kozliuk, anatoly.burakov, cristian.dumitrescu,
	honnappa.nagarahalli, ruifeng.wang, drc, konstantin.ananyev,
	olivier.matz, jay.jayatheerthan, asekhar, pbhagavatula,
	Elana Agostini

On Mon, Nov 1, 2021 at 3:25 AM Thomas Monjalon <thomas@monjalon.net> wrote:
>
> 31/10/2021 22:13, Jerin Jacob:
> > On Mon, Nov 1, 2021 at 1:04 AM Thomas Monjalon <thomas@monjalon.net> wrote:
> > >
> > > 31/10/2021 15:01, Jerin Jacob:
> > > > Since rte_flow already has the TLV concept it may not be new to DPDK.
> > >
> > > Where is there TLV in rte_flow?
> >
> > struct rte_flow_item {
> >         enum rte_flow_item_type type; /**< Item type. */
> >         const void *spec; /**< Pointer to item specification structure. */
> >
> > Type is the tag here and the spec is the value here. Length is the
> > size of the specification structure.
> > rte_flows spec does not support/need zero length variable at the end
> > of spec structure,
> > that reason for not embedding explicit length value as it is can be
> > derived from sizeof(specification structure).
>
> Ah OK I see what you mean.
> But rte_flow_item is quite limited,
> it is not the kind of TLV with multiple levels of nesting.
> Do you need nesting of objects in DWA?

No. Currently, ethernet-based on host port has the following
prototype[1] and it has array
of TLV(not in continuous memory). For simplicity, we could remove
legth value from rte_dwa_tlv and just
keep like rte_flow and let the payload contain the length of the
message if the message has
a variable length. See rte_dwa_profile_l3fwd_d2h_exception_pkts::nb_pkts below.


[1]
+/**
+ * Receive a burst of TLVs of type `TYPE_USER_PLANE` from the Rx queue
+ * designated by its *queue_id* of DWA object *obj*.
+ *
+ * @param obj
+ *   DWA object.
+ * @param queue_id
+ *   The identifier of Rx queue id. The queue id should in the range of
+ *   [0 to rte_dwa_port_host_ethernet_config::nb_rx_queues].
+ * @param[out] tlvs
+ *   Points to an array of *nb_tlvs* tlvs of type *rte_dwa_tlv* structure
+ *   to be received.
+ * @param nb_tlvs
+ *   The maximum number of TLVs to received.
+ *
+ * @return
+ * The number of TLVs actually received on the Rx queue. The return
+ * value can be less than the value of the *nb_tlvs* parameter when the
+ * Rx queue is not full.
+ */
+uint16_t rte_dwa_port_host_ethernet_rx(rte_dwa_obj_t obj, uint16_t queue_id,
+ struct rte_dwa_tlv **tlvs, uint16_t nb_tlvs);


[2]
example TLV for TYPE_USER_PLANE traffic.


+ /**
+ * Attribute |  Value
+ * ----------|--------
+ * Tag       | RTE_DWA_TAG_PROFILE_L3FWD
+ * Stag      | RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS
+ * Direction | D2H
+ * Type      | TYPE_USER_PLANE
+ * Payload   | struct rte_dwa_profile_l3fwd_d2h_exception_pkts
+ * Pair TLV  | NA
+ *
+ * Response from DWA of exception packets.
+ */

+/**
+ * Payload of RTE_DWA_STAG_PROFILE_L3FWD_D2H_EXCEPTION_PACKETS message.
+ */
+struct rte_dwa_profile_l3fwd_d2h_exception_pkts {
+ uint16_t nb_pkts;
+ /**< Number of packets in the variable size array.*/
+ uint16_t rsvd16;
+ /**< Reserved field to make pkts[0] to be 64bit aligned.*/
+ uint32_t rsvd32;
+ /**< Reserved field to make pkts[0] to be 64bit aligned.*/
+ struct rte_mbuf *pkts[0];
+ /**< Array of rte_mbufs of size nb_pkts. */
+} __rte_packed;


>
> > > > I really liked rte_flow enablement of ABI combability and its ease of adding
> > > > new stuff. Try to follow similar stuff which is proven in DPDK.
> > > > Ie. New profile creation will very easy, it will be a matter of identifying
> > > > the TLVs and their type and payload, rather than everyone comes with
> > > > new APIs in every profile.
> > > >
> > > > > Why not use protobuf and its IDL to specify the interface?
> > >
> > > Yes I think it is important to discuss alternatives,
> > > and at least get justifications of why TLV is chosen among others.
> >
> > Yes. Current list is
> >
> > 1) Very easy to enable ABI compatibility.
> > 2) If it needs to be transported over network etc it needs to be
> > packed so that way it is easy for implementation to do that
> > with TLV also gives better performance in such
> > cases by avoiding reformatting or possibly avoiding memcpy etc.
> > 3) It is easy to plugin with another high-level programing language as
> > just one API.
> > 4) Easy to decouple DWA core library functionalities from profile.
> > 5) Easy to enable asynchronous scheme using request and response TLVs.
> > 6) Most importantly, We could introduce type notion with TLV
> > (connected with the type of message  See TYPE_ATTACHED, TYPE_STOPPED,
> > TYPE_USER_PLANE etc ),
> > That way, we can have a uniform outlook of profiles instead of each profile
> > coming with a setup of its own APIs and __rules__ on the state machine.
> > I think, for a framework to leverage communication mechanisms and other
> > aspects between profiles, it's important to have some synergy between profiles.
> > 7) No Additional library dependencies like gRPC, protobuf
> > 8) Provide driver to implement the optimized means of supporting different
> > transport such as Ethernet, Shared memory, PCIe DMA style HW etc.
> > 9) Avoid creating endless APIs and their associated driver function
> > calls for each
> > profile APIs.
>
>
>

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

end of thread, other threads:[~2021-10-31 22:19 UTC | newest]

Thread overview: 20+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-10-19 18:14 [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library jerinj
2021-10-19 18:14 ` [dpdk-dev] [RFC PATCH 1/1] dwa: introduce dataplane workload accelerator subsystem jerinj
2021-10-19 19:08 ` [dpdk-dev] [RFC PATCH 0/1] Dataplane Workload Accelerator library Thomas Monjalon
2021-10-19 19:36   ` Jerin Jacob
2021-10-19 20:42     ` Stephen Hemminger
2021-10-20  5:25       ` Jerin Jacob
2021-10-19 20:42     ` Tom Herbert
2021-10-20  5:38       ` Jerin Jacob
2021-10-22 12:00     ` Elena Agostini
2021-10-22 13:39       ` Jerin Jacob
2021-10-25  7:35 ` Mattias Rönnblom
2021-10-25  9:03   ` Jerin Jacob
2021-10-29 11:57     ` Mattias Rönnblom
2021-10-29 15:51       ` Jerin Jacob
2021-10-31  9:18         ` Mattias Rönnblom
2021-10-31 14:01           ` Jerin Jacob
2021-10-31 19:34             ` Thomas Monjalon
2021-10-31 21:13               ` Jerin Jacob
2021-10-31 21:55                 ` Thomas Monjalon
2021-10-31 22:19                   ` Jerin Jacob

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).