All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion
@ 2016-02-25 17:37 Pablo Neira Ayuso
  2016-02-25 17:37 ` [PATCH RFC 1/3] net: ixgbe: add struct igxbe_filter Pablo Neira Ayuso
                   ` (3 more replies)
  0 siblings, 4 replies; 19+ messages in thread
From: Pablo Neira Ayuso @ 2016-02-25 17:37 UTC (permalink / raw)
  To: netdev; +Cc: davem, jiri, horms, john.fastabend

Hi,

This patchset contains the core infrastructure for the generic
intermediate representation that I presented during NetDev 1.1's
nftables switchdev talk. This includes the basic infrastructure to
convert the tc cls_u32 based on John's parser.

The main goals of this patchset are:

* Provide an unified abstract syntax tree (ast) that can be passed to
  the backend driver for translation to internal representation. Then,
  based on the backend description, generate the internal
  representation. This should reduce the amount of code to maintain
  in the driver since every frontend, via parser, generates the ast
  that is consumed by the driver through one single ndo indirection.

* Avoid exposing low-level frontend details to the backend, such as
  structure layouts. If the frontend needs to be updated to support a
  new software feature, it is desiderable that such changes don't
  trigger large updates to every driver supporting offloads.

* Having a common parser for every frontend, instead of allowing each
  backend driver to re-invent the wheel with its own parser, this is
  just spreading out complexity all over the place.

A summary picture of the infrastructure looks like this:

                parser
        tc-u32 -------
                       \            jit
     tc-flower -------------- ast ------> Backend driver
                       /
           nft -------

So the idea is that every frontend implements a parser that builds the
ast, then this ast is passed via ndo to the driver. The parser is common
to everyone, is part of the common core infrastructure.

The tc-u32 parser is a bit complicated because of having the matching
spread out in different rules through links, but John already came up a
basic parser than should be placed in the frontend so everyone can
improve it to generate more expressive ast.

Note: I don't have access to ixgbe hardware, so I have validated this
patchset by splicing main parts of the ixgbe backend jit code in simple
debugging patches that I have here. Quite rudimentary but it has passed
some basic tests, may still have gotten anything broken. Anyway, the
main goal is to generate debate on this.

Comments welcome, thanks.

Pablo Neira Ayuso (3):
  net: ixgbe: add struct igxbe_filter
  net: intermediate representation for jit translation
  net: convert tc_u32 to use the intermediate representation

 drivers/net/ethernet/intel/ixgbe/ixgbe.h       |   4 -
 drivers/net/ethernet/intel/ixgbe/ixgbe_main.c  | 249 +++++++++---------
 drivers/net/ethernet/intel/ixgbe/ixgbe_model.h | 112 --------
 include/net/ir.h                               | 173 +++++++++++++
 include/net/pkt_cls.h                          |   3 +
 net/core/Makefile                              |   2 +-
 net/core/ir.c                                  | 219 ++++++++++++++++
 net/sched/cls_u32.c                            | 344 +++++++++++++++++++++++++
 8 files changed, 866 insertions(+), 240 deletions(-)
 delete mode 100644 drivers/net/ethernet/intel/ixgbe/ixgbe_model.h
 create mode 100644 include/net/ir.h
 create mode 100644 net/core/ir.c

-- 
2.1.4

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

* [PATCH RFC 1/3] net: ixgbe: add struct igxbe_filter
  2016-02-25 17:37 [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion Pablo Neira Ayuso
@ 2016-02-25 17:37 ` Pablo Neira Ayuso
  2016-02-25 17:37 ` [PATCH RFC 2/3] net: intermediate representation for jit translation Pablo Neira Ayuso
                   ` (2 subsequent siblings)
  3 siblings, 0 replies; 19+ messages in thread
From: Pablo Neira Ayuso @ 2016-02-25 17:37 UTC (permalink / raw)
  To: netdev; +Cc: davem, jiri, horms, john.fastabend

Just a container structure, this change is required by follow up patches
that rework the u32 classifier offload on top of the generic
intermediate representation.

Signed-off-by: Pablo Neira Ayuso <pablo@netfilter.org>
---
 drivers/net/ethernet/intel/ixgbe/ixgbe_main.c | 51 +++++++++++++++------------
 1 file changed, 28 insertions(+), 23 deletions(-)

diff --git a/drivers/net/ethernet/intel/ixgbe/ixgbe_main.c b/drivers/net/ethernet/intel/ixgbe/ixgbe_main.c
index cf4b729..252e9ff 100644
--- a/drivers/net/ethernet/intel/ixgbe/ixgbe_main.c
+++ b/drivers/net/ethernet/intel/ixgbe/ixgbe_main.c
@@ -8238,6 +8238,12 @@ static int ixgbe_configure_clsu32_del_hnode(struct ixgbe_adapter *adapter,
 	return 0;
 }
 
+struct ixgbe_filter {
+	struct ixgbe_fdir_filter *input;
+	union ixgbe_atr_input mask;
+	u8 queue;
+};
+
 static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
 				  __be16 protocol,
 				  struct tc_cls_u32_offload *cls)
@@ -8245,16 +8251,14 @@ static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
 	u32 loc = cls->knode.handle & 0xfffff;
 	struct ixgbe_hw *hw = &adapter->hw;
 	struct ixgbe_mat_field *field_ptr;
-	struct ixgbe_fdir_filter *input;
-	union ixgbe_atr_input mask;
+	struct ixgbe_filter f;
 #ifdef CONFIG_NET_CLS_ACT
 	const struct tc_action *a;
 #endif
 	int i, err = 0;
-	u8 queue;
 	u32 handle;
 
-	memset(&mask, 0, sizeof(union ixgbe_atr_input));
+	memset(&f.mask, 0, sizeof(union ixgbe_atr_input));
 	handle = cls->knode.handle;
 
 	/* At the moment cls_u32 jumps to transport layer and skips past
@@ -8320,8 +8324,8 @@ static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
 	if (!field_ptr)
 		return -EINVAL;
 
-	input = kzalloc(sizeof(*input), GFP_KERNEL);
-	if (!input)
+	f.input = kzalloc(sizeof(struct ixgbe_fdir_filter), GFP_KERNEL);
+	if (!f.input)
 		return -ENOMEM;
 
 	for (i = 0; i < cls->knode.sel->nkeys; i++) {
@@ -8334,8 +8338,8 @@ static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
 		for (j = 0; field_ptr[j].val; j++) {
 			if (field_ptr[j].off == off &&
 			    field_ptr[j].mask == m) {
-				field_ptr[j].val(input, &mask, val, m);
-				input->filter.formatted.flow_type |=
+				field_ptr[j].val(f.input, &f.mask, val, m);
+				f.input->filter.formatted.flow_type |=
 					field_ptr[j].type;
 				found_entry = true;
 				break;
@@ -8346,11 +8350,11 @@ static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
 			goto err_out;
 	}
 
-	mask.formatted.flow_type = IXGBE_ATR_L4TYPE_IPV6_MASK |
-				   IXGBE_ATR_L4TYPE_MASK;
+	f.mask.formatted.flow_type = IXGBE_ATR_L4TYPE_IPV6_MASK |
+				     IXGBE_ATR_L4TYPE_MASK;
 
-	if (input->filter.formatted.flow_type == IXGBE_ATR_FLOW_TYPE_IPV4)
-		mask.formatted.flow_type &= IXGBE_ATR_L4TYPE_IPV6_MASK;
+	if (f.input->filter.formatted.flow_type == IXGBE_ATR_FLOW_TYPE_IPV4)
+		f.mask.formatted.flow_type &= IXGBE_ATR_L4TYPE_IPV6_MASK;
 
 #ifdef CONFIG_NET_CLS_ACT
 	if (list_empty(&cls->knode.exts->actions))
@@ -8362,34 +8366,35 @@ static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
 	}
 #endif
 
-	input->action = IXGBE_FDIR_DROP_QUEUE;
-	queue = IXGBE_FDIR_DROP_QUEUE;
-	input->sw_idx = loc;
+	f.input->action = IXGBE_FDIR_DROP_QUEUE;
+	f.queue = IXGBE_FDIR_DROP_QUEUE;
+	f.input->sw_idx = loc;
 
 	spin_lock(&adapter->fdir_perfect_lock);
 
 	if (hlist_empty(&adapter->fdir_filter_list)) {
-		memcpy(&adapter->fdir_mask, &mask, sizeof(mask));
-		err = ixgbe_fdir_set_input_mask_82599(hw, &mask);
+		memcpy(&adapter->fdir_mask, &f.mask, sizeof(f.mask));
+		err = ixgbe_fdir_set_input_mask_82599(hw, &f.mask);
 		if (err)
 			goto err_out_w_lock;
-	} else if (memcmp(&adapter->fdir_mask, &mask, sizeof(mask))) {
+	} else if (memcmp(&adapter->fdir_mask, &f.mask, sizeof(f.mask))) {
 		err = -EINVAL;
 		goto err_out_w_lock;
 	}
 
-	ixgbe_atr_compute_perfect_hash_82599(&input->filter, &mask);
-	err = ixgbe_fdir_write_perfect_filter_82599(hw, &input->filter,
-						    input->sw_idx, queue);
+	ixgbe_atr_compute_perfect_hash_82599(&f.input->filter, &f.mask);
+	err = ixgbe_fdir_write_perfect_filter_82599(hw, &f.input->filter,
+						    f.input->sw_idx, f.queue);
 	if (!err)
-		ixgbe_update_ethtool_fdir_entry(adapter, input, input->sw_idx);
+		ixgbe_update_ethtool_fdir_entry(adapter, f.input,
+						f.input->sw_idx);
 	spin_unlock(&adapter->fdir_perfect_lock);
 
 	return err;
 err_out_w_lock:
 	spin_unlock(&adapter->fdir_perfect_lock);
 err_out:
-	kfree(input);
+	kfree(f.input);
 	return -EINVAL;
 }
 
-- 
2.1.4

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

* [PATCH RFC 2/3] net: intermediate representation for jit translation
  2016-02-25 17:37 [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion Pablo Neira Ayuso
  2016-02-25 17:37 ` [PATCH RFC 1/3] net: ixgbe: add struct igxbe_filter Pablo Neira Ayuso
@ 2016-02-25 17:37 ` Pablo Neira Ayuso
  2016-02-25 17:37 ` [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation Pablo Neira Ayuso
  2016-02-25 18:11 ` [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion John Fastabend
  3 siblings, 0 replies; 19+ messages in thread
From: Pablo Neira Ayuso @ 2016-02-25 17:37 UTC (permalink / raw)
  To: netdev; +Cc: davem, jiri, horms, john.fastabend

This patch adds the generic intermediate representation that consists of
abstract syntax tree (ast) that is expressed via a list of statements.
Currently there are two type of statements:

* Expression, that is used to match packet selector and meta
  information. Expressions are basically small trees that represent what
  you want to match.

* Actions, that you can use to indicate what action you want to perform
  on the packets, currently only verdict supported but it should be
  easy to extend it to support other actions.

The general idea is that every frontend needs to provide a parser that
builds the ast, then pass this to the backend. Since the representation
is the same for every frontend, the plan is to achieve a single unified
backend jit translation from the ast to the internal backend
representation.

>From the backend side, you can use this generic infrastructure to
describe the jit translation from the intermediate representation
expressed through an abstract syntax tree to target internal
representation.

The backend has to define the jit description structure, which looks
like the following:

 struct net_ir_jit_desc {
       enum net_ir_payload_bases               base;
       const struct net_ir_proto_desc          *proto_desc;
       int                                     (*verdict)(struct net_ir_jit_ctx *ctx,
                                                          enum net_ir_stmt_verdict verdict,
                                                          void *data);
 };

This structure contains a protocol description (defined by struct
net_ir_proto_desc) that is the initial node of the protocol graph that
describes the protocol translation. This initial node starts from lower
supported layer as base (eg. link-layer) then describing the upper
protocols up to the transport protocols through the following structure:

 struct net_ir_proto_desc {
       enum net_ir_payload_bases               base;
       u32                                     protonum;
       int                                     (*jit)(struct net_ir_jit_ctx *ctx,
                                                      const struct net_ir_expr *expr,
                                                      void *data);
       const struct net_ir_proto_desc          *protocols[];
 };

This patch also includes an initial action to describe the verdict on
the packet. Meta matching and other actions are not yet available, but
it should be easy to add them to this representation.

The net_ir_jit_ctx structure is opaque to the backend, currently you use
this context object to update protocol context via:

 int net_ir_jit_update_pctx(struct net_ir_jit_ctx *ctx, u32 base, u32 proto);

Taking the ast as input, the backend can generate the internal
backend representation invoking the net_ir_jit() function.

Signed-off-by: Pablo Neira Ayuso <pablo@netfilter.org>
---
 include/net/ir.h  | 173 ++++++++++++++++++++++++++++++++++++++++++
 net/core/Makefile |   2 +-
 net/core/ir.c     | 219 ++++++++++++++++++++++++++++++++++++++++++++++++++++++
 3 files changed, 393 insertions(+), 1 deletion(-)
 create mode 100644 include/net/ir.h
 create mode 100644 net/core/ir.c

diff --git a/include/net/ir.h b/include/net/ir.h
new file mode 100644
index 0000000..e3012a7
--- /dev/null
+++ b/include/net/ir.h
@@ -0,0 +1,173 @@
+#ifndef _NET_IR_H_
+#define _NET_IR_H_
+
+/**
+ * net_ir_expr_type - expression type
+ *
+ * NET_IR_EXPR_RELATIONAL: this relates two different expression, eg. matching
+ * NET_IR_EXPR_VALUE: a value
+ * NET_IR_EXPR_PAYLOAD: a payload expression [base, offset, length]
+ * NET_IR_EXPR_BINOP: a binary expression for bitwise operation
+ */
+enum net_ir_expr_type {
+	NET_IR_EXPR_UNSPEC	= 0,
+	NET_IR_EXPR_RELATIONAL,
+	NET_IR_EXPR_VALUE,
+	NET_IR_EXPR_PAYLOAD,
+	NET_IR_EXPR_BINOP,
+};
+
+/**
+ * net_ir_expr_ops - expression operations
+ *
+ * NET_IR_OP_EQ: expressions on both hand sides are equal
+ * NET_IR_OP_AND: expression on the lhs is and'ed to the one on the rhs
+ */
+enum net_ir_expr_ops {
+	NET_IR_OP_INVALID,
+	NET_IR_OP_EQ,
+	NET_IR_OP_AND,
+};
+
+/**
+ * net_ir_payload_baser_payload_basess - payload expression bases
+ *
+ * This indicates where the payload expression starts from.
+ *
+ * NET_IR_PAYLOAD_LL_HDR: link-layer header.
+ * NET_IR_PAYLOAD_NETWORK_HDR: network header.
+ * NET_IR_PAYLOAD_TRANSPORT_HDR: transport header.
+ */
+enum net_ir_payload_bases {
+	NET_IR_PAYLOAD_LL_HDR		= 0,
+	NET_IR_PAYLOAD_NETWORK_HDR,
+	NET_IR_PAYLOAD_TRANSPORT_HDR,
+	NET_IR_PAYLOAD_MAX_HDR
+};
+
+/**
+ *     struct net_ir_expr - expression
+ *
+ *     @type: expression type
+ *     @op: type of operation
+ *     @len: length of expression
+ */
+struct net_ir_expr {
+	enum net_ir_expr_type				type;
+	enum net_ir_expr_ops				op;
+	u32						len;
+	union {
+		struct {
+			u32				data;
+		} value;
+		struct {
+			enum net_ir_payload_bases	base;
+			u32				offset;
+		} payload;
+		struct {
+			struct net_ir_expr		*left;
+			struct net_ir_expr		*right;
+		} relational;
+		struct {
+			struct net_ir_expr		*left;
+			struct net_ir_expr		*right;
+		} binop;
+	};
+};
+
+struct net_ir_expr *net_ir_expr_alloc(enum net_ir_expr_type type);
+void net_ir_expr_free(struct net_ir_expr *expr);
+
+/**
+ * net_ir_stmt_type - statement types
+ *
+ * NET_IR_STMT_EXPR: expression statement, eg. matching payload.
+ * NET_IR_STMT_VERDICT: verdict statement (accept, drop)
+ */
+enum net_ir_stmt_type {
+	NET_IR_STMT_EXPR	= 0,
+	NET_IR_STMT_VERDICT,
+};
+
+/**
+ * net_stmt_verdict - verdict statement
+ *
+ * NET_IR_DROP: drop packet
+ * NET_IR_ACCEPT: accept packet
+ */
+enum net_ir_stmt_verdict {
+	NET_IR_VERDICT_DROP	= 0,
+	NET_IR_VERDICT_ACCEPT,
+};
+
+/**
+ * struct net_ir_stmt - statement
+ *
+ * @type: statement type
+ */
+struct net_ir_stmt {
+	struct list_head			list;
+
+	enum net_ir_stmt_type			type;
+	union {
+		struct net_ir_expr		*expr;
+		enum net_ir_stmt_verdict	verdict;
+	};
+};
+
+struct net_ir_stmt *net_ir_stmt_alloc(enum net_ir_stmt_type type);
+
+/* Opaque context object during jit transformation. */
+struct net_ir_jit_ctx;
+
+/**
+ * struct net_ir_proto_desc - protocol description
+ *
+ * @base: payload base
+ * @protonum: protocol number
+ * @jit: jit function
+ * @protocols: array of upper layer protocols on top of this
+ */
+struct net_ir_proto_desc {
+	enum net_ir_payload_bases		base;
+	u32					protonum;
+	int					(*jit)(struct net_ir_jit_ctx *ctx,
+						       const struct net_ir_expr *expr,
+						       void *data);
+	const struct net_ir_proto_desc		*protocols[];
+};
+
+/**
+ * struct net_ir_jit_desc - generic description
+ */
+struct net_ir_jit_desc {
+	enum net_ir_payload_bases		base;
+	const struct net_ir_proto_desc		*proto_desc;
+	int					(*verdict)(struct net_ir_jit_ctx *ctx,
+							   enum net_ir_stmt_verdict verdict,
+							   void *data);
+};
+
+struct net_ir_ast {
+	struct list_head stmt_list;
+};
+
+static inline void net_ir_ast_init(struct net_ir_ast *ast)
+{
+	INIT_LIST_HEAD(&ast->stmt_list);
+}
+
+static inline void net_ir_ast_add_stmt(struct net_ir_ast *ast,
+				       struct net_ir_stmt *stmt)
+{
+	list_add_tail(&stmt->list, &ast->stmt_list);
+}
+
+void net_ir_ast_free(struct net_ir_ast *ast);
+
+int net_ir_jit_update_pctx(struct net_ir_jit_ctx *ctx, u32 base, u32 proto);
+
+int net_ir_jit(const struct net_ir_ast *ast,
+	       const struct net_ir_jit_desc *jit_desc, void *data);
+
+#endif
diff --git a/net/core/Makefile b/net/core/Makefile
index 7a8fb8a..4070019 100644
--- a/net/core/Makefile
+++ b/net/core/Makefile
@@ -9,7 +9,7 @@ obj-$(CONFIG_SYSCTL) += sysctl_net_core.o
 
 obj-y		     += dev.o ethtool.o dev_addr_lists.o dst.o netevent.o \
 			neighbour.o rtnetlink.o utils.o link_watch.o filter.o \
-			sock_diag.o dev_ioctl.o tso.o sock_reuseport.o
+			sock_diag.o dev_ioctl.o tso.o sock_reuseport.o ir.o
 
 obj-$(CONFIG_XFRM) += flow.o
 obj-y += net-sysfs.o
diff --git a/net/core/ir.c b/net/core/ir.c
new file mode 100644
index 0000000..e3340c8
--- /dev/null
+++ b/net/core/ir.c
@@ -0,0 +1,219 @@
+/*
+ * Copyright (c) 2016 Pablo Neira Ayuso <pablo@netfilter.org>
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License version 2 as
+ * published by the Free Software Foundation.
+ */
+
+#include <linux/init.h>
+#include <linux/kernel.h>
+#include <linux/slab.h>
+#include <linux/list.h>
+#include <net/ir.h>
+
+struct net_ir_expr *net_ir_expr_alloc(enum net_ir_expr_type type)
+{
+	struct net_ir_expr *expr;
+
+	expr = kzalloc(sizeof(struct net_ir_expr), GFP_KERNEL);
+	if (expr == NULL)
+		return NULL;
+
+	expr->type = type;
+	expr->op   = NET_IR_OP_INVALID;
+
+	return expr;
+}
+EXPORT_SYMBOL_GPL(net_ir_expr_alloc);
+
+void net_ir_expr_free(struct net_ir_expr *expr)
+{
+	switch (expr->type) {
+	case NET_IR_EXPR_VALUE:
+	case NET_IR_EXPR_PAYLOAD:
+		kfree(expr);
+		break;
+	case NET_IR_EXPR_RELATIONAL:
+		net_ir_expr_free(expr->relational.left);
+		net_ir_expr_free(expr->relational.right);
+		kfree(expr);
+		break;
+	case NET_IR_EXPR_BINOP:
+		net_ir_expr_free(expr->binop.left);
+		net_ir_expr_free(expr->binop.right);
+		kfree(expr);
+		break;
+	default:
+		WARN_ONCE(1, "Destroying unknown expr %u\n", expr->type);
+	}
+}
+EXPORT_SYMBOL_GPL(net_ir_expr_free);
+
+struct net_ir_stmt *net_ir_stmt_alloc(enum net_ir_stmt_type type)
+{
+	struct net_ir_stmt *stmt;
+
+	stmt = kmalloc(sizeof(struct net_ir_stmt), GFP_KERNEL);
+	if (stmt == NULL)
+		return NULL;
+
+	stmt->type = type;
+	return stmt;
+}
+EXPORT_SYMBOL_GPL(net_ir_stmt_alloc);
+
+static void net_ir_stmt_free(struct net_ir_stmt *stmt)
+{
+	switch (stmt->type) {
+	case NET_IR_STMT_EXPR:
+		net_ir_expr_free(stmt->expr);
+		break;
+	case NET_IR_STMT_VERDICT:
+		break;
+	}
+	kfree(stmt);
+}
+
+void net_ir_ast_free(struct net_ir_ast *ast)
+{
+	struct net_ir_stmt *stmt, *next;
+
+	list_for_each_entry_safe(stmt, next, &ast->stmt_list, list) {
+		list_del(&stmt->list);
+		net_ir_stmt_free(stmt);
+	}
+}
+EXPORT_SYMBOL_GPL(net_ir_ast_free);
+
+struct net_ir_jit_ctx {
+	const struct net_ir_jit_desc	*jit_desc;
+	const struct net_ir_proto_desc	*pctx[NET_IR_PAYLOAD_MAX_HDR];
+	void				*data;
+};
+
+int net_ir_jit_update_pctx(struct net_ir_jit_ctx *ctx, u32 base, u32 proto)
+{
+	struct net_ir_proto_desc **protos =
+		(struct net_ir_proto_desc **)ctx->pctx[base - 1]->protocols;
+	int i;
+
+	/* We need to know about the underlying protocol to push a new one on
+	 * top of it.
+	 */
+	if (protos == NULL)
+		return -EINVAL;
+
+	for (i = 0; protos[i] != NULL; i++) {
+		if (protos[i]->protonum != proto)
+			continue;
+
+		ctx->pctx[base] = protos[i];
+		return 0;
+	}
+	return -ENOENT;
+}
+EXPORT_SYMBOL_GPL(net_ir_jit_update_pctx);
+
+static int net_ir_jit_relational(struct net_ir_jit_ctx *ctx,
+				 const struct net_ir_expr *expr)
+{
+	const struct net_ir_expr *left = expr->relational.left;
+	const struct net_ir_expr *right = expr->relational.right;
+	const struct net_ir_proto_desc *proto_desc;
+	int err;
+
+	if (right->type != NET_IR_EXPR_VALUE)
+		return -EINVAL;
+
+	if (left->type == NET_IR_EXPR_BINOP)
+		left = left->binop.left;
+
+	switch (left->type) {
+	case NET_IR_EXPR_PAYLOAD:
+		proto_desc = ctx->pctx[left->payload.base];
+		if (!proto_desc)
+			return -EOPNOTSUPP;
+
+		err = proto_desc->jit(ctx, expr, ctx->data);
+		break;
+	default:
+		return -EOPNOTSUPP;
+	}
+
+	return err;
+}
+
+static int net_ir_jit_expr(struct net_ir_jit_ctx *ctx,
+			   const struct net_ir_expr *expr)
+{
+	int err;
+
+	switch (expr->type) {
+	case NET_IR_EXPR_RELATIONAL:
+		err = net_ir_jit_relational(ctx, expr);
+		break;
+	default:
+		return -EOPNOTSUPP;
+	}
+
+	return err;
+}
+
+static int net_ir_jit_verdict(struct net_ir_jit_ctx *ctx,
+			      enum net_ir_stmt_verdict verdict)
+{
+	int err;
+
+	switch (verdict) {
+	case NET_IR_VERDICT_DROP:
+	case NET_IR_VERDICT_ACCEPT:
+		err = ctx->jit_desc->verdict(ctx, verdict, ctx->data);
+		break;
+	default:
+		return -EOPNOTSUPP;
+	}
+
+	return err;
+}
+
+/**
+ * net_ir_jit_ctx_init - initialize jit context
+ *
+ * @ctx: jit context object
+ * @proto_desc: protocol description
+ */
+static void net_ir_jit_ctx_init(struct net_ir_jit_ctx *ctx,
+				const struct net_ir_jit_desc *jit_desc,
+				void *data)
+{
+	memset(ctx, 0, sizeof(*ctx));
+	ctx->jit_desc = jit_desc;
+	ctx->pctx[jit_desc->base] = jit_desc->proto_desc;
+	ctx->data = data;
+}
+
+int net_ir_jit(const struct net_ir_ast *ast,
+	       const struct net_ir_jit_desc *jit_desc, void *target)
+{
+	struct net_ir_jit_ctx ctx;
+	struct net_ir_stmt *stmt;
+	int err = 0;
+
+	net_ir_jit_ctx_init(&ctx, jit_desc, target);
+
+	list_for_each_entry(stmt, &ast->stmt_list, list) {
+		switch (stmt->type) {
+		case NET_IR_STMT_EXPR:
+			err = net_ir_jit_expr(&ctx, stmt->expr);
+			break;
+		case NET_IR_STMT_VERDICT:
+			err = net_ir_jit_verdict(&ctx, stmt->verdict);
+			break;
+		default:
+			return -EOPNOTSUPP;
+		}
+	}
+	return err;
+}
+EXPORT_SYMBOL_GPL(net_ir_jit);
-- 
2.1.4

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

* [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation
  2016-02-25 17:37 [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion Pablo Neira Ayuso
  2016-02-25 17:37 ` [PATCH RFC 1/3] net: ixgbe: add struct igxbe_filter Pablo Neira Ayuso
  2016-02-25 17:37 ` [PATCH RFC 2/3] net: intermediate representation for jit translation Pablo Neira Ayuso
@ 2016-02-25 17:37 ` Pablo Neira Ayuso
  2016-02-25 20:37   ` John Fastabend
  2016-02-25 18:11 ` [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion John Fastabend
  3 siblings, 1 reply; 19+ messages in thread
From: Pablo Neira Ayuso @ 2016-02-25 17:37 UTC (permalink / raw)
  To: netdev; +Cc: davem, jiri, horms, john.fastabend

This patch moves the u32 parser from the ixgbe that John has made to the
core u32. This parser has been adapted to build the intermediate
representation.

To store the parsing information, this patch introduces a parse table
object, one per device, so we don't need to store the parsing states in
the adapter, which is the major dependency with previous patches.

Since u32 allows rules connected via links, the u32 parser tracks this
links and then generates the intermediate representation that is passed
to the ixgbe driver.

New drivers will only have to implement the jit translation code based
on the intermediate representation. With some extra work, I think it
should be possible to generalize the existing tc specific ndo action so
it can be used by other frontends.

I tried to stick to John's original u32 frontend parser as much as
possible, adapting it to build the intermediate representation.

After this change, we don't expose the tc action structure layout and
other similar frontend details to the backend anymore to the backend
anymore. I think this is good since changes in the frontend should not
need to be propagated to the 1..n drivers supporting u32 offloads. In
that sense, this helps to keep the frontend software representation
separated from low-level backend driver details.

After this patch, it should be possible to put the tc_cls_u32_knode
structure into diet since we only need the handle (as unique id) and the
ast tree.

I couldn't send any more incremental changes to update previous work
since the u32 parser and the internal representation were put together,
that why this patch is slightly large.

Signed-off-by: Pablo Neira Ayuso <pablo@netfilter.org>
---
 drivers/net/ethernet/intel/ixgbe/ixgbe.h       |   4 -
 drivers/net/ethernet/intel/ixgbe/ixgbe_main.c  | 216 ++++++++--------
 drivers/net/ethernet/intel/ixgbe/ixgbe_model.h | 112 --------
 include/net/pkt_cls.h                          |   3 +
 net/sched/cls_u32.c                            | 344 +++++++++++++++++++++++++
 5 files changed, 454 insertions(+), 225 deletions(-)
 delete mode 100644 drivers/net/ethernet/intel/ixgbe/ixgbe_model.h

diff --git a/drivers/net/ethernet/intel/ixgbe/ixgbe.h b/drivers/net/ethernet/intel/ixgbe/ixgbe.h
index 84fa28c..09c2d9b 100644
--- a/drivers/net/ethernet/intel/ixgbe/ixgbe.h
+++ b/drivers/net/ethernet/intel/ixgbe/ixgbe.h
@@ -796,10 +796,6 @@ struct ixgbe_adapter {
 	u8 default_up;
 	unsigned long fwd_bitmask; /* Bitmask indicating in use pools */
 
-#define IXGBE_MAX_LINK_HANDLE 10
-	struct ixgbe_mat_field *jump_tables[IXGBE_MAX_LINK_HANDLE];
-	unsigned long tables;
-
 /* maximum number of RETA entries among all devices supported by ixgbe
  * driver: currently it's x550 device in non-SRIOV mode
  */
diff --git a/drivers/net/ethernet/intel/ixgbe/ixgbe_main.c b/drivers/net/ethernet/intel/ixgbe/ixgbe_main.c
index 252e9ff..416be60 100644
--- a/drivers/net/ethernet/intel/ixgbe/ixgbe_main.c
+++ b/drivers/net/ethernet/intel/ixgbe/ixgbe_main.c
@@ -67,7 +67,6 @@
 #include "ixgbe_common.h"
 #include "ixgbe_dcb_82599.h"
 #include "ixgbe_sriov.h"
-#include "ixgbe_model.h"
 
 char ixgbe_driver_name[] = "ixgbe";
 static const char ixgbe_driver_string[] =
@@ -5548,9 +5547,6 @@ static int ixgbe_sw_init(struct ixgbe_adapter *adapter)
 #endif /* CONFIG_IXGBE_DCB */
 #endif /* IXGBE_FCOE */
 
-	/* initialize static ixgbe jump table entries */
-	adapter->jump_tables[0] = ixgbe_ipv4_fields;
-
 	adapter->mac_table = kzalloc(sizeof(struct ixgbe_mac_addr) *
 				     hw->mac.num_rar_entries,
 				     GFP_ATOMIC);
@@ -8221,20 +8217,12 @@ static int ixgbe_configure_clsu32_add_hnode(struct ixgbe_adapter *adapter,
 					    __be16 protocol,
 					    struct tc_cls_u32_offload *cls)
 {
-	/* This ixgbe devices do not support hash tables at the moment
-	 * so abort when given hash tables.
-	 */
-	if (cls->hnode.divisor > 0)
-		return -EINVAL;
-
-	set_bit(TC_U32_USERHTID(cls->hnode.handle), &adapter->tables);
 	return 0;
 }
 
 static int ixgbe_configure_clsu32_del_hnode(struct ixgbe_adapter *adapter,
 					    struct tc_cls_u32_offload *cls)
 {
-	clear_bit(TC_U32_USERHTID(cls->hnode.handle), &adapter->tables);
 	return 0;
 }
 
@@ -8244,111 +8232,134 @@ struct ixgbe_filter {
 	u8 queue;
 };
 
+static int ixgbe_tcp_jit(struct net_ir_jit_ctx *ctx,
+			 const struct net_ir_expr *expr,
+			 void *data)
+{
+	struct ixgbe_filter *f = (struct ixgbe_filter *)data;
+	struct net_ir_expr *right = expr->relational.right;
+	struct net_ir_expr *payload;
+	u32 mask = 0xffffffff;
+
+	if (expr->relational.left->type == NET_IR_EXPR_BINOP) {
+		payload = expr->relational.left->binop.left;
+		mask = expr->relational.left->binop.right->value.data;
+	} else {
+		payload = expr->relational.left;
+	}
+
+	switch (payload->payload.offset) {
+	case offsetof(struct tcphdr, source):
+		f->input->filter.formatted.src_port = right->value.data & 0xffff;
+		f->mask.formatted.src_port = mask & 0xffff;
+		break;
+	case offsetof(struct tcphdr, dest):
+		f->input->filter.formatted.dst_port = right->value.data & 0xffff;
+		f->mask.formatted.dst_port = mask & 0xffff;
+		break;
+	default:
+		return -EOPNOTSUPP;
+	}
+	return 0;
+}
+
+static struct net_ir_proto_desc ixgbe_tcp_desc = {
+	.base		= NET_IR_PAYLOAD_TRANSPORT_HDR,
+	.protonum	= IPPROTO_TCP,
+	.jit		= ixgbe_tcp_jit,
+};
+
+static int ixgbe_ipv4_jit(struct net_ir_jit_ctx *ctx,
+			  const struct net_ir_expr *expr,
+			  void *data)
+{
+	struct ixgbe_filter *f = (struct ixgbe_filter *)data;
+	struct net_ir_expr *right = expr->relational.right;
+	struct net_ir_expr *payload;
+	u32 mask = 0xffffffff;
+
+	if (expr->relational.left->type == NET_IR_EXPR_BINOP) {
+		payload = expr->relational.left->binop.left;
+		mask = expr->relational.left->binop.right->value.data;
+	} else {
+		payload = expr->relational.left;
+	}
+
+	switch (payload->payload.offset) {
+	case offsetof(struct iphdr, saddr):
+		f->input->filter.formatted.src_ip[0] = right->value.data;
+		f->mask.formatted.src_ip[0] = mask;
+		break;
+	case offsetof(struct iphdr, daddr):
+		f->input->filter.formatted.dst_ip[0] = right->value.data;
+		f->mask.formatted.dst_ip[0] = mask;
+		break;
+	case offsetof(struct iphdr, protocol):
+		net_ir_jit_update_pctx(ctx, NET_IR_PAYLOAD_TRANSPORT_HDR,
+				       right->value.data);
+		break;
+	default:
+		return -EOPNOTSUPP;
+	}
+	return 0;
+}
+
+static struct net_ir_proto_desc ixgbe_ipv4_desc = {
+	.base		= NET_IR_PAYLOAD_NETWORK_HDR,
+	.jit		= ixgbe_ipv4_jit,
+	.protocols	= {
+		&ixgbe_tcp_desc,
+		NULL
+	},
+};
+
+static int ixgbe_verdict(struct net_ir_jit_ctx *ctx,
+			 enum net_ir_stmt_verdict verdict, void *data)
+{
+	struct ixgbe_filter *f = (struct ixgbe_filter *)data;
+
+	switch (verdict) {
+	case NET_IR_VERDICT_DROP:
+		f->input->action = IXGBE_FDIR_DROP_QUEUE;
+		f->queue = IXGBE_FDIR_DROP_QUEUE;
+		break;
+	default:
+		return -EOPNOTSUPP;
+	}
+
+	return 0;
+}
+
+static struct net_ir_jit_desc ixgbe_desc = {
+	.base		= NET_IR_PAYLOAD_NETWORK_HDR,
+	.proto_desc	= &ixgbe_ipv4_desc,
+	.verdict	= ixgbe_verdict,
+};
+
 static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
 				  __be16 protocol,
 				  struct tc_cls_u32_offload *cls)
 {
 	u32 loc = cls->knode.handle & 0xfffff;
 	struct ixgbe_hw *hw = &adapter->hw;
-	struct ixgbe_mat_field *field_ptr;
 	struct ixgbe_filter f;
-#ifdef CONFIG_NET_CLS_ACT
-	const struct tc_action *a;
-#endif
-	int i, err = 0;
+	int err = 0;
 	u32 handle;
 
 	memset(&f.mask, 0, sizeof(union ixgbe_atr_input));
 	handle = cls->knode.handle;
 
-	/* At the moment cls_u32 jumps to transport layer and skips past
-	 * L2 headers. The canonical method to match L2 frames is to use
-	 * negative values. However this is error prone at best but really
-	 * just broken because there is no way to "know" what sort of hdr
-	 * is in front of the transport layer. Fix cls_u32 to support L2
-	 * headers when needed.
-	 */
-	if (protocol != htons(ETH_P_IP))
-		return -EINVAL;
-
-	if (cls->knode.link_handle ||
-	    cls->knode.link_handle >= IXGBE_MAX_LINK_HANDLE) {
-		struct ixgbe_nexthdr *nexthdr = ixgbe_ipv4_jumps;
-		u32 uhtid = TC_U32_USERHTID(cls->knode.link_handle);
-
-		if (!test_bit(uhtid, &adapter->tables))
-			return -EINVAL;
-
-		for (i = 0; nexthdr[i].jump; i++) {
-			if (nexthdr->o != cls->knode.sel->offoff ||
-			    nexthdr->s != cls->knode.sel->offshift ||
-			    nexthdr->m != cls->knode.sel->offmask ||
-			    /* do not support multiple key jumps its just mad */
-			    cls->knode.sel->nkeys > 1)
-				return -EINVAL;
-
-			if (nexthdr->off != cls->knode.sel->keys[0].off ||
-			    nexthdr->val != cls->knode.sel->keys[0].val ||
-			    nexthdr->mask != cls->knode.sel->keys[0].mask)
-				return -EINVAL;
-
-			if (uhtid >= IXGBE_MAX_LINK_HANDLE)
-				return -EINVAL;
-
-			adapter->jump_tables[uhtid] = nexthdr->jump;
-		}
-		return 0;
-	}
-
 	if (loc >= ((1024 << adapter->fdir_pballoc) - 2)) {
 		e_err(drv, "Location out of range\n");
 		return -EINVAL;
 	}
 
-	/* cls u32 is a graph starting at root node 0x800. The driver tracks
-	 * links and also the fields used to advance the parser across each
-	 * link (e.g. nexthdr/eat parameters from 'tc'). This way we can map
-	 * the u32 graph onto the hardware parse graph denoted in ixgbe_model.h
-	 * To add support for new nodes update ixgbe_model.h parse structures
-	 * this function _should_ be generic try not to hardcode values here.
-	 */
-	if (TC_U32_USERHTID(handle) == 0x800) {
-		field_ptr = adapter->jump_tables[0];
-	} else {
-		if (TC_U32_USERHTID(handle) >= ARRAY_SIZE(adapter->jump_tables))
-			return -EINVAL;
-
-		field_ptr = adapter->jump_tables[TC_U32_USERHTID(handle)];
-	}
-
-	if (!field_ptr)
-		return -EINVAL;
-
 	f.input = kzalloc(sizeof(struct ixgbe_fdir_filter), GFP_KERNEL);
 	if (!f.input)
 		return -ENOMEM;
 
-	for (i = 0; i < cls->knode.sel->nkeys; i++) {
-		int off = cls->knode.sel->keys[i].off;
-		__be32 val = cls->knode.sel->keys[i].val;
-		__be32 m = cls->knode.sel->keys[i].mask;
-		bool found_entry = false;
-		int j;
-
-		for (j = 0; field_ptr[j].val; j++) {
-			if (field_ptr[j].off == off &&
-			    field_ptr[j].mask == m) {
-				field_ptr[j].val(f.input, &f.mask, val, m);
-				f.input->filter.formatted.flow_type |=
-					field_ptr[j].type;
-				found_entry = true;
-				break;
-			}
-		}
-
-		if (!found_entry)
-			goto err_out;
-	}
+	if (net_ir_jit(&cls->knode.ast, &ixgbe_desc, &f) < 0)
+		return -EINVAL;
 
 	f.mask.formatted.flow_type = IXGBE_ATR_L4TYPE_IPV6_MASK |
 				     IXGBE_ATR_L4TYPE_MASK;
@@ -8356,18 +8367,6 @@ static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
 	if (f.input->filter.formatted.flow_type == IXGBE_ATR_FLOW_TYPE_IPV4)
 		f.mask.formatted.flow_type &= IXGBE_ATR_L4TYPE_IPV6_MASK;
 
-#ifdef CONFIG_NET_CLS_ACT
-	if (list_empty(&cls->knode.exts->actions))
-		goto err_out;
-
-	list_for_each_entry(a, &cls->knode.exts->actions, list) {
-		if (!is_tcf_gact_shot(a))
-			goto err_out;
-	}
-#endif
-
-	f.input->action = IXGBE_FDIR_DROP_QUEUE;
-	f.queue = IXGBE_FDIR_DROP_QUEUE;
 	f.input->sw_idx = loc;
 
 	spin_lock(&adapter->fdir_perfect_lock);
@@ -8393,7 +8392,6 @@ static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
 	return err;
 err_out_w_lock:
 	spin_unlock(&adapter->fdir_perfect_lock);
-err_out:
 	kfree(f.input);
 	return -EINVAL;
 }
diff --git a/drivers/net/ethernet/intel/ixgbe/ixgbe_model.h b/drivers/net/ethernet/intel/ixgbe/ixgbe_model.h
deleted file mode 100644
index ce48872..0000000
--- a/drivers/net/ethernet/intel/ixgbe/ixgbe_model.h
+++ /dev/null
@@ -1,112 +0,0 @@
-/*******************************************************************************
- *
- * Intel 10 Gigabit PCI Express Linux drive
- * Copyright(c) 2016 Intel Corporation.
- *
- * This program is free software; you can redistribute it and/or modify it
- * under the terms and conditions of the GNU General Public License,
- * version 2, as published by the Free Software Foundation.
- *
- * This program is distributed in the hope it will be useful, but WITHOUT
- * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
- * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
- * more details.
- *
- * You should have received a copy of the GNU General Public License along
- * with this program.  If not, see <http://www.gnu.org/licenses/>.
- *
- * The full GNU General Public License is included in this distribution in
- * the file called "COPYING".
- *
- * Contact Information:
- * e1000-devel Mailing List <e1000-devel@lists.sourceforge.net>
- * Intel Corporation, 5200 N.E. Elam Young Parkway, Hillsboro, OR 97124-6497
- *
- ******************************************************************************/
-
-#ifndef _IXGBE_MODEL_H_
-#define _IXGBE_MODEL_H_
-
-#include "ixgbe.h"
-#include "ixgbe_type.h"
-
-struct ixgbe_mat_field {
-	unsigned int off;
-	unsigned int mask;
-	int (*val)(struct ixgbe_fdir_filter *input,
-		   union ixgbe_atr_input *mask,
-		   u32 val, u32 m);
-	unsigned int type;
-};
-
-static inline int ixgbe_mat_prgm_sip(struct ixgbe_fdir_filter *input,
-				     union ixgbe_atr_input *mask,
-				     u32 val, u32 m)
-{
-	input->filter.formatted.src_ip[0] = val;
-	mask->formatted.src_ip[0] = m;
-	return 0;
-}
-
-static inline int ixgbe_mat_prgm_dip(struct ixgbe_fdir_filter *input,
-				     union ixgbe_atr_input *mask,
-				     u32 val, u32 m)
-{
-	input->filter.formatted.dst_ip[0] = val;
-	mask->formatted.dst_ip[0] = m;
-	return 0;
-}
-
-static struct ixgbe_mat_field ixgbe_ipv4_fields[] = {
-	{ .off = 12, .mask = -1, .val = ixgbe_mat_prgm_sip,
-	  .type = IXGBE_ATR_FLOW_TYPE_IPV4},
-	{ .off = 16, .mask = -1, .val = ixgbe_mat_prgm_dip,
-	  .type = IXGBE_ATR_FLOW_TYPE_IPV4},
-	{ .val = NULL } /* terminal node */
-};
-
-static inline int ixgbe_mat_prgm_sport(struct ixgbe_fdir_filter *input,
-				       union ixgbe_atr_input *mask,
-				       u32 val, u32 m)
-{
-	input->filter.formatted.src_port = val & 0xffff;
-	mask->formatted.src_port = m & 0xffff;
-	return 0;
-};
-
-static inline int ixgbe_mat_prgm_dport(struct ixgbe_fdir_filter *input,
-				       union ixgbe_atr_input *mask,
-				       u32 val, u32 m)
-{
-	input->filter.formatted.dst_port = val & 0xffff;
-	mask->formatted.dst_port = m & 0xffff;
-	return 0;
-};
-
-static struct ixgbe_mat_field ixgbe_tcp_fields[] = {
-	{.off = 0, .mask = 0xffff, .val = ixgbe_mat_prgm_sport,
-	 .type = IXGBE_ATR_FLOW_TYPE_TCPV4},
-	{.off = 2, .mask = 0xffff, .val = ixgbe_mat_prgm_dport,
-	 .type = IXGBE_ATR_FLOW_TYPE_TCPV4},
-	{ .val = NULL } /* terminal node */
-};
-
-struct ixgbe_nexthdr {
-	/* offset, shift, and mask of position to next header */
-	unsigned int o;
-	u32 s;
-	u32 m;
-	/* match criteria to make this jump*/
-	unsigned int off;
-	u32 val;
-	u32 mask;
-	/* location of jump to make */
-	struct ixgbe_mat_field *jump;
-};
-
-static struct ixgbe_nexthdr ixgbe_ipv4_jumps[] = {
-	{ .o = 0, .s = 6, .m = 0xf,
-	  .off = 8, .val = 0x600, .mask = 0xff00, .jump = ixgbe_tcp_fields},
-	{ .jump = NULL } /* terminal node */
-};
-#endif /* _IXGBE_MODEL_H_ */
diff --git a/include/net/pkt_cls.h b/include/net/pkt_cls.h
index 2121df5..c276ba2 100644
--- a/include/net/pkt_cls.h
+++ b/include/net/pkt_cls.h
@@ -358,6 +358,8 @@ tcf_match_indev(struct sk_buff *skb, int ifindex)
 }
 #endif /* CONFIG_NET_CLS_IND */
 
+#include <net/ir.h>
+
 struct tc_cls_u32_knode {
 	struct tcf_exts *exts;
 	struct tc_u32_sel *sel;
@@ -366,6 +368,7 @@ struct tc_cls_u32_knode {
 	u32 mask;
 	u32 link_handle;
 	u8 fshift;
+	struct net_ir_ast ast;
 };
 
 struct tc_cls_u32_hnode {
diff --git a/net/sched/cls_u32.c b/net/sched/cls_u32.c
index d54bc94..b79b4675 100644
--- a/net/sched/cls_u32.c
+++ b/net/sched/cls_u32.c
@@ -44,6 +44,8 @@
 #include <net/act_api.h>
 #include <net/pkt_cls.h>
 #include <linux/netdevice.h>
+#include <net/tc_act/tc_gact.h>
+#include <net/ir.h>
 
 struct tc_u_knode {
 	struct tc_u_knode __rcu	*next;
@@ -442,11 +444,208 @@ static void u32_remove_hw_knode(struct tcf_proto *tp, u32 handle)
 	}
 }
 
+static struct net_ir_stmt *
+u32_payload_stmt_alloc(enum net_ir_payload_bases base, u32 offset, u32 value,
+		       u32 mask)
+{
+	struct net_ir_expr *expr, *payload, *binop;
+	struct net_ir_stmt *stmt;
+
+	expr = net_ir_expr_alloc(NET_IR_EXPR_RELATIONAL);
+	if (!expr)
+		return NULL;
+
+	expr->op = NET_IR_OP_EQ;
+
+	payload = net_ir_expr_alloc(NET_IR_EXPR_PAYLOAD);
+	if (!payload) {
+		net_ir_expr_free(expr);
+		return NULL;
+	}
+
+	payload->payload.base = base;
+	payload->payload.offset = offset;
+
+	if (mask) {
+		binop = net_ir_expr_alloc(NET_IR_EXPR_BINOP);
+		if (!binop) {
+			net_ir_expr_free(expr);
+			net_ir_expr_free(payload);
+			return NULL;
+		}
+
+		binop->op = NET_IR_OP_AND;
+
+		binop->binop.left = payload;
+		binop->binop.right = net_ir_expr_alloc(NET_IR_EXPR_VALUE);
+		if (!binop->binop.right) {
+			net_ir_expr_free(expr);
+			net_ir_expr_free(binop);
+			return NULL;
+		}
+
+		binop->binop.right->value.data = mask;
+		expr->relational.left = binop;
+	} else {
+		expr->relational.left = payload;
+	}
+
+	expr->relational.right = net_ir_expr_alloc(NET_IR_EXPR_VALUE);
+	if (!expr->relational.right) {
+		net_ir_expr_free(expr);
+		return NULL;
+	}
+	expr->relational.right->value.data = value;
+
+	stmt = net_ir_stmt_alloc(NET_IR_STMT_EXPR);
+	if (!stmt) {
+		net_ir_expr_free(expr);
+		return NULL;
+	}
+
+	stmt->expr = expr;
+	return stmt;
+}
+
+struct u32_mat_field {
+	unsigned int off;
+	unsigned int mask;
+};
+
+struct u32_proto_def {
+	u32			protocol;
+	int			(*parse)(struct net_ir_ast *ast,
+					 struct u32_mat_field *field,
+					 u32 m, u32 val);
+	u32			field_num;
+	struct u32_mat_field	fields[];
+};
+
+static int u32_parse_ip(struct net_ir_ast *ast, struct u32_mat_field *field,
+			u32 mask, u32 val)
+{
+	struct net_ir_stmt *stmt;
+
+	stmt = u32_payload_stmt_alloc(NET_IR_PAYLOAD_NETWORK_HDR,
+				      field->off, val, mask);
+	if (!stmt)
+		return -ENOMEM;
+
+	net_ir_ast_add_stmt(ast, stmt);
+	return 0;
+}
+
+static struct u32_proto_def u32_ipv4_fields = {
+	.protocol 	= ETH_P_IP,
+	.parse		= u32_parse_ip,
+	.field_num	= 2,
+	.fields = {
+		{.off = 12,	.mask = -1, },
+		{.off = 16,	.mask = -1, },
+	},
+};
+
+static int u32_parse_tcp(struct net_ir_ast *ast, struct u32_mat_field *field,
+			 u32 mask, u32 val)
+{
+	struct net_ir_stmt *stmt;
+
+	/* Manually add ip protocol field to the abstract syntax tree. We can
+	 * get rid of this by storing context into the parser.
+	 */
+	stmt = u32_payload_stmt_alloc(NET_IR_PAYLOAD_NETWORK_HDR, 9,
+				      IPPROTO_TCP, 0);
+	if (!stmt)
+		return -ENOMEM;
+	net_ir_ast_add_stmt(ast, stmt);
+
+	stmt = u32_payload_stmt_alloc(NET_IR_PAYLOAD_TRANSPORT_HDR, field->off,
+				      val, mask);
+	if (!stmt)
+		return -ENOMEM;
+
+	net_ir_ast_add_stmt(ast, stmt);
+	return 0;
+}
+
+static struct u32_proto_def u32_tcp_fields = {
+	.protocol 	= IPPROTO_TCP,
+	.parse		= u32_parse_tcp,
+	.field_num	= 2,
+	.fields		= {
+		{ .off = 0,	.mask = 0xffff, },
+		{ .off = 2,	.mask = 0xffff, },
+	},
+};
+
+struct u32_nexthdr {
+	/* offset, shift, and mask of position to next header */
+	unsigned int o;
+	u32 s;
+	u32 m;
+	/* match criteria to make this jump*/
+	unsigned int off;
+	u32 val;
+	u32 mask;
+	/* location of jump to make */
+	struct u32_proto_def *jump;
+};
+
+static struct u32_nexthdr u32_ipv4_jumps[] = {
+	{ .o = 0, .s = 6, .m = 0xf,
+	  .off = 8, .val = 0x600, .mask = 0xff00, .jump = &u32_tcp_fields},
+	{ .jump = NULL } /* terminal node */
+};
+
+#define U32_PARSER_MAX_LINK_HANDLE 10
+
+static LIST_HEAD(u32_parser_tables);
+
+/* This stores the context for the u32 parser */
+struct u32_parser_table {
+	struct list_head	list;
+	struct net_device	*dev;	/* the owner of this table */
+	struct u32_proto_def	*jump_tables[U32_PARSER_MAX_LINK_HANDLE];
+	unsigned long		tables;
+};
+
+static struct u32_parser_table *u32_parser_table_alloc(struct net_device *dev)
+{
+	struct u32_parser_table *ptable;
+
+	ptable = kzalloc(sizeof(struct u32_parser_table), GFP_KERNEL);
+	if (!ptable)
+		return NULL;
+
+	ptable->dev = dev;
+	ptable->jump_tables[0] = &u32_ipv4_fields;
+
+	return ptable;
+}
+
+static struct u32_parser_table *u32_parser_tables_get(struct tcf_proto *tp)
+{
+	struct net_device *dev = tp->q->dev_queue->dev;
+	struct u32_parser_table *ptable;
+
+	list_for_each_entry(ptable, &u32_parser_tables, list) {
+		if (ptable->dev == dev)
+			return ptable;
+	}
+	ptable = u32_parser_table_alloc(dev);
+	if (!ptable)
+		return NULL;
+
+	list_add(&ptable->list, &u32_parser_tables);
+	return ptable;
+}
+
 static void u32_replace_hw_hnode(struct tcf_proto *tp, struct tc_u_hnode *h)
 {
 	struct net_device *dev = tp->q->dev_queue->dev;
 	struct tc_cls_u32_offload u32_offload = {0};
 	struct tc_to_netdev offload;
+	struct u32_parser_table *p;
 
 	offload.type = TC_SETUP_CLSU32;
 	offload.cls_u32 = &u32_offload;
@@ -457,6 +656,19 @@ static void u32_replace_hw_hnode(struct tcf_proto *tp, struct tc_u_hnode *h)
 		offload.cls_u32->hnode.handle = h->handle;
 		offload.cls_u32->hnode.prio = h->prio;
 
+		/* No support hash tables at the moment so abort when given
+		 * hash tables.
+		 */
+		if (h->divisor > 0)
+			return;
+
+		p = u32_parser_tables_get(tp);
+		if (!p)
+			return;
+
+		set_bit(TC_U32_USERHTID(offload.cls_u32->hnode.handle),
+			&p->tables);
+
 		dev->netdev_ops->ndo_setup_tc(dev, tp->q->handle,
 					      tp->protocol, &offload);
 	}
@@ -467,6 +679,7 @@ static void u32_clear_hw_hnode(struct tcf_proto *tp, struct tc_u_hnode *h)
 	struct net_device *dev = tp->q->dev_queue->dev;
 	struct tc_cls_u32_offload u32_offload = {0};
 	struct tc_to_netdev offload;
+	struct u32_parser_table *p;
 
 	offload.type = TC_SETUP_CLSU32;
 	offload.cls_u32 = &u32_offload;
@@ -477,11 +690,135 @@ static void u32_clear_hw_hnode(struct tcf_proto *tp, struct tc_u_hnode *h)
 		offload.cls_u32->hnode.handle = h->handle;
 		offload.cls_u32->hnode.prio = h->prio;
 
+		p = u32_parser_tables_get(tp);
+		if (!p)
+			return;
+
+		clear_bit(TC_U32_USERHTID(offload.cls_u32->hnode.handle),
+			  &p->tables);
+
 		dev->netdev_ops->ndo_setup_tc(dev, tp->q->handle,
 					      tp->protocol, &offload);
 	}
 }
 
+static int u32_parser(struct tcf_proto *tp, struct tc_to_netdev *tc)
+{
+	struct tc_cls_u32_offload *cls = tc->cls_u32;
+	struct net_ir_ast *ast = &cls->knode.ast;
+	struct u32_proto_def *proto_def;
+	__be16 protocol = tp->protocol;
+	u32 handle = cls->knode.handle;
+	struct u32_parser_table *p;
+#ifdef CONFIG_NET_CLS_ACT
+	const struct tc_action *a;
+#endif
+	struct net_ir_stmt *stmt;
+	int i;
+
+	/* At the moment cls_u32 jumps to transport layer and skips past
+	 * L2 headers. The canonical method to match L2 frames is to use
+	 * negative values. However this is error prone at best but really
+	 * just broken because there is no way to "know" what sort of hdr
+	 * is in front of the transport layer. Fix cls_u32 to support L2
+	 * headers when needed.
+	 */
+	if (protocol != htons(ETH_P_IP))
+		return -EINVAL;
+
+	p = u32_parser_tables_get(tp);
+	if (!p)
+		return -ENOMEM;
+
+	if (cls->knode.link_handle ||
+	    cls->knode.link_handle >= U32_PARSER_MAX_LINK_HANDLE) {
+		struct u32_nexthdr *nexthdr = u32_ipv4_jumps;
+		u32 uhtid = TC_U32_USERHTID(cls->knode.link_handle);
+
+		if (!test_bit(uhtid, &p->tables))
+			return -EINVAL;
+
+		for (i = 0; nexthdr[i].jump; i++) {
+			if (nexthdr->o != cls->knode.sel->offoff ||
+			    nexthdr->s != cls->knode.sel->offshift ||
+			    nexthdr->m != cls->knode.sel->offmask ||
+			/* do not support multiple key jumps its just mad */
+			    cls->knode.sel->nkeys > 1)
+				return -EINVAL;
+
+			if (nexthdr->off != cls->knode.sel->keys[0].off ||
+			    nexthdr->val != cls->knode.sel->keys[0].val ||
+			    nexthdr->mask != cls->knode.sel->keys[0].mask)
+				return -EINVAL;
+
+			if (uhtid >= U32_PARSER_MAX_LINK_HANDLE)
+				return -EINVAL;
+
+			p->jump_tables[uhtid] = nexthdr->jump;
+		}
+		return -EINVAL; /* don't push this into hardware yet */
+	}
+
+	/* cls u32 is a graph starting at root node 0x800. This parser tracks
+	 * links and also the fields used to advance the parser across each
+	 * link (e.g. nexthdr/eat parameters from 'tc'). This way we can map
+	 * the u32 graph onto the intermediate representation denoted in ir.h.
+	 * To add support for new nodes update parse structures this function
+	 * _should_ be generic try not to hardcode values here.
+	 */
+	if (TC_U32_USERHTID(handle) == 0x800) {
+		proto_def = p->jump_tables[0];
+	} else {
+		if (TC_U32_USERHTID(handle) >= ARRAY_SIZE(p->jump_tables))
+			return -EINVAL;
+
+		proto_def = p->jump_tables[TC_U32_USERHTID(handle)];
+	}
+
+	if (!proto_def)
+		return -EINVAL;
+
+	for (i = 0; i < cls->knode.sel->nkeys; i++) {
+		int off = cls->knode.sel->keys[i].off;
+		__be32 val = cls->knode.sel->keys[i].val;
+		__be32 m = cls->knode.sel->keys[i].mask;
+		bool found_entry = false;
+		int j;
+
+		for (j = 0; j < proto_def->field_num; j++) {
+			if (proto_def->fields[j].off == off &&
+			    proto_def->fields[j].mask == m) {
+				if (proto_def->parse(ast, &proto_def->fields[j],
+						     m, val) < 0)
+					return -EINVAL;
+
+				found_entry = true;
+				break;
+			}
+		}
+
+		if (!found_entry)
+			return -EINVAL;
+	}
+
+#ifdef CONFIG_NET_CLS_ACT
+	if (list_empty(&cls->knode.exts->actions))
+		return -EINVAL;
+
+	list_for_each_entry(a, &cls->knode.exts->actions, list) {
+		if (!is_tcf_gact_shot(a))
+			return -EINVAL;
+	}
+	stmt = net_ir_stmt_alloc(NET_IR_STMT_VERDICT);
+	if (!stmt)
+		return -EINVAL;
+
+	stmt->verdict = NET_IR_VERDICT_DROP;
+	net_ir_ast_add_stmt(ast, stmt);
+#endif
+	return 0;
+}
+
 static void u32_replace_hw_knode(struct tcf_proto *tp, struct tc_u_knode *n)
 {
 	struct net_device *dev = tp->q->dev_queue->dev;
@@ -507,8 +844,15 @@ static void u32_replace_hw_knode(struct tcf_proto *tp, struct tc_u_knode *n)
 		if (n->ht_down)
 			offload.cls_u32->knode.link_handle = n->ht_down->handle;
 
+		net_ir_ast_init(&offload.cls_u32->knode.ast);
+
+		if (u32_parser(tp, &offload) < 0)
+			return;
+
 		dev->netdev_ops->ndo_setup_tc(dev, tp->q->handle,
 					      tp->protocol, &offload);
+
+		net_ir_ast_free(&offload.cls_u32->knode.ast);
 	}
 }
 
-- 
2.1.4

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

* Re: [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion
  2016-02-25 17:37 [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion Pablo Neira Ayuso
                   ` (2 preceding siblings ...)
  2016-02-25 17:37 ` [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation Pablo Neira Ayuso
@ 2016-02-25 18:11 ` John Fastabend
  2016-02-26 11:47   ` Pablo Neira Ayuso
  3 siblings, 1 reply; 19+ messages in thread
From: John Fastabend @ 2016-02-25 18:11 UTC (permalink / raw)
  To: Pablo Neira Ayuso, netdev; +Cc: davem, jiri, horms

On 16-02-25 09:37 AM, Pablo Neira Ayuso wrote:
> Hi,
> 
> This patchset contains the core infrastructure for the generic
> intermediate representation that I presented during NetDev 1.1's
> nftables switchdev talk. This includes the basic infrastructure to
> convert the tc cls_u32 based on John's parser.
> 
> The main goals of this patchset are:
> 
> * Provide an unified abstract syntax tree (ast) that can be passed to
>   the backend driver for translation to internal representation. Then,
>   based on the backend description, generate the internal
>   representation. This should reduce the amount of code to maintain
>   in the driver since every frontend, via parser, generates the ast
>   that is consumed by the driver through one single ndo indirection.
> 

The trouble here is we lose a lot of the goodness of any particular
classifier this way. For example flower can be implemented with a
single jump table and this should translate into updates per second
being very high.

cls_u32 on the other hand is going to require more state and I believe
will be slightly slower in the updates per second benchmark because
it requires a set of jump tables.

My general argument here is _if_ we can build one universal ast why
don't we do this higher up the stack. I'm a bit confused on why we
think (a) all classifiers/nftables/etc can be pushed on to a single
IR and (b) all hw can consume a single IR if we actually believe this
lets whack the upper layers and make one tc classifer and unify it
with nftables so we don't have multiple things in the stack to offload.

> * Avoid exposing low-level frontend details to the backend, such as
>   structure layouts. If the frontend needs to be updated to support a
>   new software feature, it is desiderable that such changes don't
>   trigger large updates to every driver supporting offloads.
> 
> * Having a common parser for every frontend, instead of allowing each
>   backend driver to re-invent the wheel with its own parser, this is
>   just spreading out complexity all over the place.


I think we should make helper routines and build a library for driver
writers instead. This way each driver writer can call stub functions
to do things like validate u32 against a model.h header or something
along these lines. This keeps flexibility IMO and lets us optimize
the backends for the specific hardware. Remember we eventually need
to get to 100k's of updates a second to compete with user space APIs
if you want this to work on high-end devices. If you don't get here
it wont be usable for a class of systems and we will just lift the
functions into user space for performance reasons. I'm sceptical that
converting from shiny classifier to software IR and then hw IR is
going to be efficient when I can go direct from classifier to hw IR.

Also the IR here only maps easily to a specific set of hardware. For
example it doesn't really map well to some of my more programmable
devices that want to consume "programs" more like BPF. So I don't see
any universal IR being possible. At best you get a set of IRs
dependent on the underlying architecture.

> 
> A summary picture of the infrastructure looks like this:
> 
>                 parser
>         tc-u32 -------
>                        \            jit
>      tc-flower -------------- ast ------> Backend driver
>                        /
>            nft -------
> 
> So the idea is that every frontend implements a parser that builds the
> ast, then this ast is passed via ndo to the driver. The parser is common
> to everyone, is part of the common core infrastructure.

Same argument as above with some helper libs going from u32 to driver
is really minimal code. And what are the performance impact of taking
a simple classifier like flower to ast to backend.

Looking at the similarities between u32 and nft should we unify them in
the stack so that,

   nft ---- jit(u32)

and just run nft inside u32? Or

   nft ---- jit(ebpf)

and just offload u32 or ebpf? or

    u32 --- jit(nft)

any one of those combinations :)

> 
> The tc-u32 parser is a bit complicated because of having the matching
> spread out in different rules through links, but John already came up a
> basic parser than should be placed in the frontend so everyone can
> improve it to generate more expressive ast.
> 
> Note: I don't have access to ixgbe hardware, so I have validated this
> patchset by splicing main parts of the ixgbe backend jit code in simple
> debugging patches that I have here. Quite rudimentary but it has passed
> some basic tests, may still have gotten anything broken. Anyway, the
> main goal is to generate debate on this.
> 

Yep lets debate the general approach. I can deal with ixgbe details. My
reaction is this seems overkill I only see offloads popping up for
flower, u32, nft, and bpf at the moment. Flower doesn't need heavy
overhead due to its simplicity. u32 can be done with some basic helpers
for devices like ixgbe. I think nft can be done the same. bpf is its own
beast that doesn't map well to another IR its more or less its own IR
anyways.

If we need/want a IR I would go for something instruction based
like ebpf over this.

> Comments welcome, thanks.
> 
> Pablo Neira Ayuso (3):
>   net: ixgbe: add struct igxbe_filter
>   net: intermediate representation for jit translation
>   net: convert tc_u32 to use the intermediate representation
> 
>  drivers/net/ethernet/intel/ixgbe/ixgbe.h       |   4 -
>  drivers/net/ethernet/intel/ixgbe/ixgbe_main.c  | 249 +++++++++---------
>  drivers/net/ethernet/intel/ixgbe/ixgbe_model.h | 112 --------
>  include/net/ir.h                               | 173 +++++++++++++
>  include/net/pkt_cls.h                          |   3 +
>  net/core/Makefile                              |   2 +-
>  net/core/ir.c                                  | 219 ++++++++++++++++
>  net/sched/cls_u32.c                            | 344 +++++++++++++++++++++++++
>  8 files changed, 866 insertions(+), 240 deletions(-)
>  delete mode 100644 drivers/net/ethernet/intel/ixgbe/ixgbe_model.h
>  create mode 100644 include/net/ir.h
>  create mode 100644 net/core/ir.c
> 

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

* Re: [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation
  2016-02-25 17:37 ` [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation Pablo Neira Ayuso
@ 2016-02-25 20:37   ` John Fastabend
  2016-02-26 14:24     ` Pablo Neira Ayuso
  0 siblings, 1 reply; 19+ messages in thread
From: John Fastabend @ 2016-02-25 20:37 UTC (permalink / raw)
  To: Pablo Neira Ayuso, netdev; +Cc: davem, jiri, horms

On 16-02-25 09:37 AM, Pablo Neira Ayuso wrote:
> This patch moves the u32 parser from the ixgbe that John has made to the
> core u32. This parser has been adapted to build the intermediate
> representation.
> 
> To store the parsing information, this patch introduces a parse table
> object, one per device, so we don't need to store the parsing states in
> the adapter, which is the major dependency with previous patches.
> 
> Since u32 allows rules connected via links, the u32 parser tracks this
> links and then generates the intermediate representation that is passed
> to the ixgbe driver.

It also supports hash tables and loops.

> 
> New drivers will only have to implement the jit translation code based
> on the intermediate representation. With some extra work, I think it
> should be possible to generalize the existing tc specific ndo action so
> it can be used by other frontends.
> 
> I tried to stick to John's original u32 frontend parser as much as
> possible, adapting it to build the intermediate representation.
> 
> After this change, we don't expose the tc action structure layout and
> other similar frontend details to the backend anymore to the backend
> anymore. I think this is good since changes in the frontend should not
> need to be propagated to the 1..n drivers supporting u32 offloads. In
> that sense, this helps to keep the frontend software representation
> separated from low-level backend driver details.
> 
> After this patch, it should be possible to put the tc_cls_u32_knode
> structure into diet since we only need the handle (as unique id) and the
> ast tree.
> 

On ixgbe this is true but going forward I can support hash functions
so you need the divisor, prio, and handle minimally. I'm not sure how
to do hash tables for example in this IR yet. Might be there I'm still
trying to grok the IR details.

> I couldn't send any more incremental changes to update previous work
> since the u32 parser and the internal representation were put together,
> that why this patch is slightly large.
> 
> Signed-off-by: Pablo Neira Ayuso <pablo@netfilter.org>
> ---
>  drivers/net/ethernet/intel/ixgbe/ixgbe.h       |   4 -
>  drivers/net/ethernet/intel/ixgbe/ixgbe_main.c  | 216 ++++++++--------
>  drivers/net/ethernet/intel/ixgbe/ixgbe_model.h | 112 --------
>  include/net/pkt_cls.h                          |   3 +
>  net/sched/cls_u32.c                            | 344 +++++++++++++++++++++++++

Feels like a lot of code to me when the original direct u32 to hw
was reasonably small and the flower implementation is small as well.

[...]

>  
> +static int ixgbe_tcp_jit(struct net_ir_jit_ctx *ctx,
> +			 const struct net_ir_expr *expr,
> +			 void *data)
> +{
> +	struct ixgbe_filter *f = (struct ixgbe_filter *)data;
> +	struct net_ir_expr *right = expr->relational.right;
> +	struct net_ir_expr *payload;
> +	u32 mask = 0xffffffff;
> +
> +	if (expr->relational.left->type == NET_IR_EXPR_BINOP) {
> +		payload = expr->relational.left->binop.left;
> +		mask = expr->relational.left->binop.right->value.data;
> +	} else {
> +		payload = expr->relational.left;
> +	}
> +
> +	switch (payload->payload.offset) {

I don't like this because it hardcodes the offset pieces into the
driver. The reason I used a model.h header file with its structures
in ixgbe is the next step is to expose the parser so the model can be
updated. So that when the hardware parser changes the data structure
can be changed via firmware call. The style of coding here wont work
for that so we still need the model.h file and for loop.

> +	case offsetof(struct tcphdr, source):
> +		f->input->filter.formatted.src_port = right->value.data & 0xffff;
> +		f->mask.formatted.src_port = mask & 0xffff;
> +		break;
> +	case offsetof(struct tcphdr, dest):
> +		f->input->filter.formatted.dst_port = right->value.data & 0xffff;
> +		f->mask.formatted.dst_port = mask & 0xffff;
> +		break;
> +	default:
> +		return -EOPNOTSUPP;
> +	}
> +	return 0;
> +}
> +
> +static struct net_ir_proto_desc ixgbe_tcp_desc = {
> +	.base		= NET_IR_PAYLOAD_TRANSPORT_HDR,
> +	.protonum	= IPPROTO_TCP,

hmm I'm having trouble tracking down how this is used but the point of
a lot of ongoing work is to not be dependent on any specific protocol
values. The IPPROTO_TCP here looks suspicious why do we need this? It
seems to imply I would need to define a IPPROTO_FOO if I wanted to add
a new protocol.

> +	.jit		= ixgbe_tcp_jit,
> +};
> +
> +static int ixgbe_ipv4_jit(struct net_ir_jit_ctx *ctx,
> +			  const struct net_ir_expr *expr,
> +			  void *data)
> +{
> +	struct ixgbe_filter *f = (struct ixgbe_filter *)data;
> +	struct net_ir_expr *right = expr->relational.right;
> +	struct net_ir_expr *payload;
> +	u32 mask = 0xffffffff;
> +
> +	if (expr->relational.left->type == NET_IR_EXPR_BINOP) {
> +		payload = expr->relational.left->binop.left;
> +		mask = expr->relational.left->binop.right->value.data;
> +	} else {
> +		payload = expr->relational.left;
> +	}
> +
> +	switch (payload->payload.offset) {

same comment here it depends on iphdr, saddr, etc. This is fundamentally
broken in my opinion. I shouldn't be using 'struct' and specific keys to
find address. I want this to be agnostic of protocol so when my users
wants to add protocol foo I can do it without recompiling the
driver/kernel.

> +	case offsetof(struct iphdr, saddr):
> +		f->input->filter.formatted.src_ip[0] = right->value.data;
> +		f->mask.formatted.src_ip[0] = mask;
> +		break;
> +	case offsetof(struct iphdr, daddr):
> +		f->input->filter.formatted.dst_ip[0] = right->value.data;
> +		f->mask.formatted.dst_ip[0] = mask;
> +		break;
> +	case offsetof(struct iphdr, protocol):
> +		net_ir_jit_update_pctx(ctx, NET_IR_PAYLOAD_TRANSPORT_HDR,
> +				       right->value.data);
> +		break;
> +	default:
> +		return -EOPNOTSUPP;
> +	}
> +	return 0;
> +}
> +
> +static struct net_ir_proto_desc ixgbe_ipv4_desc = {
> +	.base		= NET_IR_PAYLOAD_NETWORK_HDR,
> +	.jit		= ixgbe_ipv4_jit,
> +	.protocols	= {
> +		&ixgbe_tcp_desc,
> +		NULL
> +	},
> +};
> +

[...]

>  static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
>  				  __be16 protocol,
>  				  struct tc_cls_u32_offload *cls)
>  {
>  	u32 loc = cls->knode.handle & 0xfffff;
>  	struct ixgbe_hw *hw = &adapter->hw;
> -	struct ixgbe_mat_field *field_ptr;
>  	struct ixgbe_filter f;
> -#ifdef CONFIG_NET_CLS_ACT
> -	const struct tc_action *a;
> -#endif
> -	int i, err = 0;
> +	int err = 0;
>  	u32 handle;
>  
>  	memset(&f.mask, 0, sizeof(union ixgbe_atr_input));
>  	handle = cls->knode.handle;
>  
> -	/* At the moment cls_u32 jumps to transport layer and skips past
> -	 * L2 headers. The canonical method to match L2 frames is to use
> -	 * negative values. However this is error prone at best but really
> -	 * just broken because there is no way to "know" what sort of hdr
> -	 * is in front of the transport layer. Fix cls_u32 to support L2
> -	 * headers when needed.
> -	 */
> -	if (protocol != htons(ETH_P_IP))

This was hardcoded originally just to find the inital offset going
forward I want to patch u32 to start at the link layer to avoid this.

[...]

> -		if (!found_entry)
> -			goto err_out;
> -	}
> +	if (net_ir_jit(&cls->knode.ast, &ixgbe_desc, &f) < 0)
> +		return -EINVAL;
>  

OK but this takes a cls_u32 offload engine that was running on a handful
of LOC + a header file description that in the future will be read out
of firmware and replaces it with an entire infrastructure. I'm not
convinced the added complexity is worth it.

[...]

> --- a/drivers/net/ethernet/intel/ixgbe/ixgbe_model.h
> +++ /dev/null
> @@ -1,112 +0,0 @@
> -/*******************************************************************************
> - *
> - * Intel 10 Gigabit PCI Express Linux drive
> - * Copyright(c) 2016 Intel Corporation.
> - *
> - * This program is free software; you can redistribute it and/or modify it
> - * under the terms and conditions of the GNU General Public License,
> - * version 2, as published by the Free Software Foundation.
> - *
> - * This program is distributed in the hope it will be useful, but WITHOUT
> - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
> - * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
> - * more details.
> - *
> - * You should have received a copy of the GNU General Public License along
> - * with this program.  If not, see <http://www.gnu.org/licenses/>.
> - *
> - * The full GNU General Public License is included in this distribution in
> - * the file called "COPYING".
> - *
> - * Contact Information:
> - * e1000-devel Mailing List <e1000-devel@lists.sourceforge.net>
> - * Intel Corporation, 5200 N.E. Elam Young Parkway, Hillsboro, OR 97124-6497
> - *
> - ******************************************************************************/
> -
> -#ifndef _IXGBE_MODEL_H_
> -#define _IXGBE_MODEL_H_
> -
> -#include "ixgbe.h"
> -#include "ixgbe_type.h"
> -
> -struct ixgbe_mat_field {
> -	unsigned int off;
> -	unsigned int mask;
> -	int (*val)(struct ixgbe_fdir_filter *input,
> -		   union ixgbe_atr_input *mask,
> -		   u32 val, u32 m);
> -	unsigned int type;
> -};

As noted above the intent is to make the parser programmable so
something like this model needs to exist for firmware to populate.

[...]

> -#endif /* _IXGBE_MODEL_H_ */
> diff --git a/include/net/pkt_cls.h b/include/net/pkt_cls.h
> index 2121df5..c276ba2 100644
> --- a/include/net/pkt_cls.h
> +++ b/include/net/pkt_cls.h
> @@ -358,6 +358,8 @@ tcf_match_indev(struct sk_buff *skb, int ifindex)
>  }
>  #endif /* CONFIG_NET_CLS_IND */
>  
> +#include <net/ir.h>
> +
>  struct tc_cls_u32_knode {
>  	struct tcf_exts *exts;
>  	struct tc_u32_sel *sel;
> @@ -366,6 +368,7 @@ struct tc_cls_u32_knode {
>  	u32 mask;
>  	u32 link_handle;
>  	u8 fshift;
> +	struct net_ir_ast ast;
>  };
>  
>  struct tc_cls_u32_hnode {
> diff --git a/net/sched/cls_u32.c b/net/sched/cls_u32.c
> index d54bc94..b79b4675 100644
> --- a/net/sched/cls_u32.c
> +++ b/net/sched/cls_u32.c
> @@ -44,6 +44,8 @@
>  #include <net/act_api.h>
>  #include <net/pkt_cls.h>
>  #include <linux/netdevice.h>
> +#include <net/tc_act/tc_gact.h>
> +#include <net/ir.h>
>  
>  struct tc_u_knode {
>  	struct tc_u_knode __rcu	*next;
> @@ -442,11 +444,208 @@ static void u32_remove_hw_knode(struct tcf_proto *tp, u32 handle)
>  	}
>  }
>  
> +static struct net_ir_stmt *
> +u32_payload_stmt_alloc(enum net_ir_payload_bases base, u32 offset, u32 value,
> +		       u32 mask)
> +{
> +	struct net_ir_expr *expr, *payload, *binop;
> +	struct net_ir_stmt *stmt;
> +
> +	expr = net_ir_expr_alloc(NET_IR_EXPR_RELATIONAL);
> +	if (!expr)
> +		return NULL;
> +
> +	expr->op = NET_IR_OP_EQ;
> +
> +	payload = net_ir_expr_alloc(NET_IR_EXPR_PAYLOAD);

A lot of memory allocation and what not done in the add/del path of the
rule set seems like its going to cause a performance impact. Yes I know
this is already slow slow slow but this adds one more thing to fix when
we get around to optimizing the updates.

Otherwise it seems to be a fairly faithful lifting of my ixgbe parser
into the IR.

[...]

>  	offload.type = TC_SETUP_CLSU32;
>  	offload.cls_u32 = &u32_offload;
> @@ -457,6 +656,19 @@ static void u32_replace_hw_hnode(struct tcf_proto *tp, struct tc_u_hnode *h)
>  		offload.cls_u32->hnode.handle = h->handle;
>  		offload.cls_u32->hnode.prio = h->prio;
>  
> +		/* No support hash tables at the moment so abort when given
> +		 * hash tables.
> +		 */

Cool so you cuaght this but how does the AST support hash tables? Some
of my above comments are around this.

> +		if (h->divisor > 0)
> +			return;
> +
> +		p = u32_parser_tables_get(tp);
> +		if (!p)
> +			return;
> +
> +		set_bit(TC_U32_USERHTID(offload.cls_u32->hnode.handle),
> +			&p->tables);
> +
>  		dev->netdev_ops->ndo_setup_tc(dev, tp->q->handle,
>  					      tp->protocol, &offload);
>  	}
> @@ -467,6 +679,7 @@ static void u32_clear_hw_hnode(struct tcf_proto *tp, struct tc_u_hnode *h)
>  	struct net_device *dev = tp->q->dev_queue->dev;
>  	struct tc_cls_u32_offload u32_offload = {0};
>  	struct tc_to_netdev offload;
> +	struct u32_parser_table *p;
>  
>  	offload.type = TC_SETUP_CLSU32;
>  	offload.cls_u32 = &u32_offload;
> @@ -477,11 +690,135 @@ static void u32_clear_hw_hnode(struct tcf_proto *tp, struct tc_u_hnode *h)
>  		offload.cls_u32->hnode.handle = h->handle;
>  		offload.cls_u32->hnode.prio = h->prio;
>  
> +		p = u32_parser_tables_get(tp);
> +		if (!p)
> +			return;
> +
> +		clear_bit(TC_U32_USERHTID(offload.cls_u32->hnode.handle),
> +			  &p->tables);
> +
>  		dev->netdev_ops->ndo_setup_tc(dev, tp->q->handle,
>  					      tp->protocol, &offload);
>  	}
>  }
>  

So I'm sorry but after reviewing this for a few hours this morning I'm
not convinced the complexity is worth the payoff. It seems to make the
code a bit heavy and the in-driver code path isn't actually any smaller.

Sure I get flower for free but with a bunch of overhead so I don't like
that and flower implementation is also simple in driver. Also bpf wont
fit in this model so I still need an ebpf parser.

So really this just unifies nft and u32 at the cost of another layer
of abstraction which I don't think provides much value. It seems like
I could get to the same place with a few helper functions for driver
writers to use without the overhead/code complexity of a bunch of new
pointers, allocated objects, needless translations and IR code that
now needs to be maintained and reviewed and supported.

My preference is to hold this work until we get a few more driver
implementation for multiple things and see if that can scale easily
as I suspect it might without having to over-engineer it.

Thanks,
.John

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

* Re: [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion
  2016-02-25 18:11 ` [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion John Fastabend
@ 2016-02-26 11:47   ` Pablo Neira Ayuso
  2016-02-26 15:42     ` John Fastabend
  2016-02-26 17:26     ` David Miller
  0 siblings, 2 replies; 19+ messages in thread
From: Pablo Neira Ayuso @ 2016-02-26 11:47 UTC (permalink / raw)
  To: John Fastabend; +Cc: netdev, davem, jiri, horms

On Thu, Feb 25, 2016 at 10:11:36AM -0800, John Fastabend wrote:
> On 16-02-25 09:37 AM, Pablo Neira Ayuso wrote:
> > Hi,
> > 
> > This patchset contains the core infrastructure for the generic
> > intermediate representation that I presented during NetDev 1.1's
> > nftables switchdev talk. This includes the basic infrastructure to
> > convert the tc cls_u32 based on John's parser.
> > 
> > The main goals of this patchset are:
> > 
> > * Provide an unified abstract syntax tree (ast) that can be passed to
> >   the backend driver for translation to internal representation. Then,
> >   based on the backend description, generate the internal
> >   representation. This should reduce the amount of code to maintain
> >   in the driver since every frontend, via parser, generates the ast
> >   that is consumed by the driver through one single ndo indirection.
> > 
> 
> The trouble here is we lose a lot of the goodness of any particular
> classifier this way. For example flower can be implemented with a
> single jump table and this should translate into updates per second
> being very high.
>
> cls_u32 on the other hand is going to require more state and I believe
> will be slightly slower in the updates per second benchmark because
> it requires a set of jump tables.

The parser on the backend is not the way to go.  I would expect it
will not take long time until we see other people copying/pasting it
and adapting it to their need just because they don't find the way/put
the care to make it fit in the infrastructure / abstract it the right
way.

... And then we will get complexity spread out all over the place, in
every single driver. This will be unmaintainable.

The frontend parser should be generic to everyone, it should be placed
in the core, so everyone will take care of improving it.

Let's make a summary picture to clarify this:


            core                               drivers
          -------                             ---------
        1 per frontend                       1 per driver

  [ people from all vendors ]           [     code consuming    ]
  [  working to improve     ]           [   IR representation   ]
  [   the common parser     ]
  [     and improve IR      ]

     Complexity goes here
       in the form of
       infrastructure

> My general argument here is _if_ we can build one universal ast why
> don't we do this higher up the stack. I'm a bit confused on why we
> think (a) all classifiers/nftables/etc can be pushed on to a single
> IR and (b) all hw can consume a single IR if we actually believe this
> lets whack the upper layers and make one tc classifer and unify it
> with nftables so we don't have multiple things in the stack to offload.

Everyone I talked to during Netconf agreed that we want to offload all
frontends. These battles on imposing frontends to people take us
nowhere.

So this "let's whack upper layers and make one" is basically out of
question.

> > * Avoid exposing low-level frontend details to the backend, such as
> >   structure layouts. If the frontend needs to be updated to support a
> >   new software feature, it is desiderable that such changes don't
> >   trigger large updates to every driver supporting offloads.
> > 
> > * Having a common parser for every frontend, instead of allowing each
> >   backend driver to re-invent the wheel with its own parser, this is
> >   just spreading out complexity all over the place.
> 
> 
> I think we should make helper routines and build a library for driver
> writers instead. This way each driver writer can call stub functions
> to do things like validate u32 against a model.h header or something
> along these lines. This keeps flexibility IMO and lets us optimize
> the backends for the specific hardware. Remember we eventually need
> to get to 100k's of updates a second to compete with user space APIs
> if you want this to work on high-end devices. If you don't get here
> it wont be usable for a class of systems and we will just lift the
> functions into user space for performance reasons. I'm sceptical that
> converting from shiny classifier to software IR and then hw IR is
> going to be efficient when I can go direct from classifier to hw IR.

To me this sounds like trying to microptimize the control plane code
by reducing the amount of code to translate it into hardware.
However, this comes at the cost of increasing the overall complexity
in the backend.

I think we have to make an effort to provide generic solutions, Linux
is a generic purpose stack.

Look at the problem from another perspective based on this statement:

"ioctl()/setsockopt()/system call of choice plus fixed layout
structures are faster control planes than Netlink."

Answer: Yes, that's probably right, they require way less code to
build and to parse a message. However, *we are telling everyone that
people should use Netlink".

Why?

Because it helps us avoid exposing the low level details of our
implementation and that makes it more more extensible in the kind of
evolutionary development that we have.

Should we microoptimize the control plane path through avoiding
abstractions? I don't think so.

Another example, look at this structure (what we currently have in
net-next):

 struct tc_cls_u32_knode {
        struct tcf_exts *exts;
        struct tc_u32_sel *sel;
        u32 handle;
        u32 val;
        u32 mask;
        u32 link_handle;
        u8 fshift;
 };

Downside is we expose frontend details to the backend.

* You're exposing structure layouts to the backend, (see
  tcf_exts struct). Another clear symptom to me that this is not the
  way to go are those #ifdefs in the backend driver code to check if the
  tc action API is set. If someone needs to extend the tc action API and
  we get lots of driver using this in the future, that poor guy
  extending tc action will have to mangle quite a lot of driver code.

* Why the driver should care if tc-u32 uses link_handles to connect rules
  and such?

So better pass the driver something generic that it can translate.
With common IR (after one more patches I have here) this will look
like:

 struct tc_cls_u32_knode {
        u32                     handle;
        struct net_ir_ast       ast;
 };

> Also the IR here only maps easily to a specific set of hardware. For
> example it doesn't really map well to some of my more programmable
> devices that want to consume "programs" more like BPF. So I don't see
> any universal IR being possible. At best you get a set of IRs
> dependent on the underlying architecture.

Universal sounds too ambicious, I called this generic. Generic in the
sense of trying to unify and provide as much common infrastructure as
possible. We probably cannot escape having some specificities for some
of the frontends we have, so we'll have to pass this specific info as
decoration to the driver, but that's should not be an excuse to skip
having something common for everyone IMO.

Anyway I'd suggest you go have a look at the IR and see what you send
incremental patches to improve what you cannot represent in a generic
way.

> > A summary picture of the infrastructure looks like this:
> > 
> >                 parser
> >         tc-u32 -------
> >                        \            jit
> >      tc-flower -------------- ast ------> Backend driver
> >                        /
> >            nft -------
> > 
> > So the idea is that every frontend implements a parser that builds the
> > ast, then this ast is passed via ndo to the driver. The parser is common
> > to everyone, is part of the common core infrastructure.
> 
> Same argument as above with some helper libs going from u32 to driver
> is really minimal code. And what are the performance impact of taking
> a simple classifier like flower to ast to backend.
> 
> Looking at the similarities between u32 and nft should we unify them in
> the stack so that,
> 
>    nft ---- jit(u32)
> 
> and just run nft inside u32? Or
> 
>    nft ---- jit(ebpf)
> 
> and just offload u32 or ebpf? or
> 
>     u32 --- jit(nft)
> 
> any one of those combinations :)

As a said above, let's keep this discussion away from the "let's
provide a unified frontend" because takes us nowhere.

> > The tc-u32 parser is a bit complicated because of having the matching
> > spread out in different rules through links, but John already came up a
> > basic parser than should be placed in the frontend so everyone can
> > improve it to generate more expressive ast.
> > 
> > Note: I don't have access to ixgbe hardware, so I have validated this
> > patchset by splicing main parts of the ixgbe backend jit code in simple
> > debugging patches that I have here. Quite rudimentary but it has passed
> > some basic tests, may still have gotten anything broken. Anyway, the
> > main goal is to generate debate on this.
> > 
> 
> Yep lets debate the general approach. I can deal with ixgbe details.
> My reaction is this seems overkill I only see offloads popping up for
> flower, u32, nft, and bpf at the moment. Flower doesn't need heavy
> overhead due to its simplicity. u32 can be done with some basic helpers
> for devices like ixgbe. I think nft can be done the same. bpf is its own
> beast that doesn't map well to another IR its more or less its own IR
> anyways.
> 
> If we need/want a IR I would go for something instruction based
> like ebpf over this.

bpf is a final representation from a compiler perspective, it is
bytecode, I don't think it is meant to be designed as an intermediate
representation.

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

* Re: [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation
  2016-02-25 20:37   ` John Fastabend
@ 2016-02-26 14:24     ` Pablo Neira Ayuso
  2016-02-26 14:53       ` John Fastabend
  0 siblings, 1 reply; 19+ messages in thread
From: Pablo Neira Ayuso @ 2016-02-26 14:24 UTC (permalink / raw)
  To: John Fastabend; +Cc: netdev, davem, jiri, horms

On Thu, Feb 25, 2016 at 12:37:38PM -0800, John Fastabend wrote:
> On 16-02-25 09:37 AM, Pablo Neira Ayuso wrote:
> > This patch moves the u32 parser from the ixgbe that John has made to the
> > core u32. This parser has been adapted to build the intermediate
> > representation.
> > 
> > To store the parsing information, this patch introduces a parse table
> > object, one per device, so we don't need to store the parsing states in
> > the adapter, which is the major dependency with previous patches.
> > 
> > Since u32 allows rules connected via links, the u32 parser tracks this
> > links and then generates the intermediate representation that is passed
> > to the ixgbe driver.
> 
> It also supports hash tables and loops.

Then, it is a matter of extending the tc-u32 parser and the IR to
support whatever you need.

> > New drivers will only have to implement the jit translation code based
> > on the intermediate representation. With some extra work, I think it
> > should be possible to generalize the existing tc specific ndo action so
> > it can be used by other frontends.
> > 
> > I tried to stick to John's original u32 frontend parser as much as
> > possible, adapting it to build the intermediate representation.
> > 
> > After this change, we don't expose the tc action structure layout and
> > other similar frontend details to the backend anymore to the backend
> > anymore. I think this is good since changes in the frontend should not
> > need to be propagated to the 1..n drivers supporting u32 offloads. In
> > that sense, this helps to keep the frontend software representation
> > separated from low-level backend driver details.
> > 
> > After this patch, it should be possible to put the tc_cls_u32_knode
> > structure into diet since we only need the handle (as unique id) and the
> > ast tree.
> > 
> 
> On ixgbe this is true but going forward I can support hash functions
> so you need the divisor, prio, and handle minimally. I'm not sure how
> to do hash tables for example in this IR yet. Might be there I'm still
> trying to grok the IR details.

We just need a description with the list elements that you want to
place in the hashtable.

The matching can be expressed in ast by adding a list of elements that
are part of the hashtable:

                 relational
                    / \
                   /   \
                  /     \
           payload      list of elements
  (offset, base, len)       |
                            \
                             --> e1 --> e2 --> ... --> en

> > I couldn't send any more incremental changes to update previous work
> > since the u32 parser and the internal representation were put together,
> > that why this patch is slightly large.
> > 
> > Signed-off-by: Pablo Neira Ayuso <pablo@netfilter.org>
> > ---
> >  drivers/net/ethernet/intel/ixgbe/ixgbe.h       |   4 -
> >  drivers/net/ethernet/intel/ixgbe/ixgbe_main.c  | 216 ++++++++--------
> >  drivers/net/ethernet/intel/ixgbe/ixgbe_model.h | 112 --------
> >  include/net/pkt_cls.h                          |   3 +
> >  net/sched/cls_u32.c                            | 344 +++++++++++++++++++++++++
> 
> Feels like a lot of code to me when the original direct u32 to hw
> was reasonably small and the flower implementation is small as well.

This is your tc-u32 parser plus simple boiler plate code to build the
IR that will be common to everyone. Without this, other drivers will
have to track u32 links and so on.

Other than that, we'll end up with 1..N different u32 parsers in the
backend.

> [...]
> 
> >  
> > +static int ixgbe_tcp_jit(struct net_ir_jit_ctx *ctx,
> > +			 const struct net_ir_expr *expr,
> > +			 void *data)
> > +{
> > +	struct ixgbe_filter *f = (struct ixgbe_filter *)data;
> > +	struct net_ir_expr *right = expr->relational.right;
> > +	struct net_ir_expr *payload;
> > +	u32 mask = 0xffffffff;
> > +
> > +	if (expr->relational.left->type == NET_IR_EXPR_BINOP) {
> > +		payload = expr->relational.left->binop.left;
> > +		mask = expr->relational.left->binop.right->value.data;
> > +	} else {
> > +		payload = expr->relational.left;
> > +	}
> > +
> > +	switch (payload->payload.offset) {
> 
> I don't like this because it hardcodes the offset pieces into the
> driver. The reason I used a model.h header file with its structures
> in ixgbe is the next step is to expose the parser so the model can be
> updated. So that when the hardware parser changes the data structure
> can be changed via firmware call. The style of coding here wont work
> for that so we still need the model.h file and for loop.

Model update via firmware?

I think it's been already several conference rounds insisting on the
fact that we don't want infrastructure that allows to insert binary
blobs/sdks.

I don't think this is the way to go.

> [...]
> 
> >  static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
> >  				  __be16 protocol,
> >  				  struct tc_cls_u32_offload *cls)
> >  {
> >  	u32 loc = cls->knode.handle & 0xfffff;
> >  	struct ixgbe_hw *hw = &adapter->hw;
> > -	struct ixgbe_mat_field *field_ptr;
> >  	struct ixgbe_filter f;
> > -#ifdef CONFIG_NET_CLS_ACT
> > -	const struct tc_action *a;
> > -#endif
> > -	int i, err = 0;
> > +	int err = 0;
> >  	u32 handle;
> >  
> >  	memset(&f.mask, 0, sizeof(union ixgbe_atr_input));
> >  	handle = cls->knode.handle;
> >  
> > -	/* At the moment cls_u32 jumps to transport layer and skips past
> > -	 * L2 headers. The canonical method to match L2 frames is to use
> > -	 * negative values. However this is error prone at best but really
> > -	 * just broken because there is no way to "know" what sort of hdr
> > -	 * is in front of the transport layer. Fix cls_u32 to support L2
> > -	 * headers when needed.
> > -	 */
> > -	if (protocol != htons(ETH_P_IP))
> 
> This was hardcoded originally just to find the inital offset going
> forward I want to patch u32 to start at the link layer to avoid
> this.

Then, you extend the tc-u32 parser, but from the frontend core
infrastructure code, so everyone we'll benefit for this incremental
update.

> [...]
> 
> > -		if (!found_entry)
> > -			goto err_out;
> > -	}
> > +	if (net_ir_jit(&cls->knode.ast, &ixgbe_desc, &f) < 0)
> > +		return -EINVAL;
> >  
> 
> OK but this takes a cls_u32 offload engine that was running on a handful
> of LOC + a header file description that in the future will be read out
> of firmware and replaces it with an entire infrastructure. I'm not
> convinced the added complexity is worth it.

It's all about consolidating infrastructure.

> [...]
> 
> > --- a/drivers/net/ethernet/intel/ixgbe/ixgbe_model.h
> > +++ /dev/null
> > @@ -1,112 +0,0 @@
> > -/*******************************************************************************
> > - *
> > - * Intel 10 Gigabit PCI Express Linux drive
> > - * Copyright(c) 2016 Intel Corporation.
> > - *
> > - * This program is free software; you can redistribute it and/or modify it
> > - * under the terms and conditions of the GNU General Public License,
> > - * version 2, as published by the Free Software Foundation.
> > - *
> > - * This program is distributed in the hope it will be useful, but WITHOUT
> > - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
> > - * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
> > - * more details.
> > - *
> > - * You should have received a copy of the GNU General Public License along
> > - * with this program.  If not, see <http://www.gnu.org/licenses/>.
> > - *
> > - * The full GNU General Public License is included in this distribution in
> > - * the file called "COPYING".
> > - *
> > - * Contact Information:
> > - * e1000-devel Mailing List <e1000-devel@lists.sourceforge.net>
> > - * Intel Corporation, 5200 N.E. Elam Young Parkway, Hillsboro, OR 97124-6497
> > - *
> > - ******************************************************************************/
> > -
> > -#ifndef _IXGBE_MODEL_H_
> > -#define _IXGBE_MODEL_H_
> > -
> > -#include "ixgbe.h"
> > -#include "ixgbe_type.h"
> > -
> > -struct ixgbe_mat_field {
> > -	unsigned int off;
> > -	unsigned int mask;
> > -	int (*val)(struct ixgbe_fdir_filter *input,
> > -		   union ixgbe_atr_input *mask,
> > -		   u32 val, u32 m);
> > -	unsigned int type;
> > -};
> 
> As noted above the intent is to make the parser programmable so
> something like this model needs to exist for firmware to populate.

Oh, again references to update via firmware.

[...]
> My preference is to hold this work until we get a few more driver
> implementation for multiple things and see if that can scale easily
> as I suspect it might without having to over-engineer it.

I don't agree, that's simply preventing by now other frontends and
drivers to get offload support.

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

* Re: [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation
  2016-02-26 14:24     ` Pablo Neira Ayuso
@ 2016-02-26 14:53       ` John Fastabend
  2016-02-26 16:02         ` Pablo Neira Ayuso
  2016-02-26 17:34         ` David Miller
  0 siblings, 2 replies; 19+ messages in thread
From: John Fastabend @ 2016-02-26 14:53 UTC (permalink / raw)
  To: Pablo Neira Ayuso; +Cc: netdev, davem, jiri, horms

On 16-02-26 06:24 AM, Pablo Neira Ayuso wrote:
> On Thu, Feb 25, 2016 at 12:37:38PM -0800, John Fastabend wrote:
>> On 16-02-25 09:37 AM, Pablo Neira Ayuso wrote:
>>> This patch moves the u32 parser from the ixgbe that John has made to the
>>> core u32. This parser has been adapted to build the intermediate
>>> representation.
>>>
>>> To store the parsing information, this patch introduces a parse table
>>> object, one per device, so we don't need to store the parsing states in
>>> the adapter, which is the major dependency with previous patches.
>>>
>>> Since u32 allows rules connected via links, the u32 parser tracks this
>>> links and then generates the intermediate representation that is passed
>>> to the ixgbe driver.
>>
>> It also supports hash tables and loops.
> 
> Then, it is a matter of extending the tc-u32 parser and the IR to
> support whatever you need.

sure but its easier for me to just consume u32 at the moment. And I
am concerned about performance overhead with this IR. I'm going to need
to have a performant solution eventually and I don't like converting
into one IR only to go into another.

> 
>>> New drivers will only have to implement the jit translation code based
>>> on the intermediate representation. With some extra work, I think it
>>> should be possible to generalize the existing tc specific ndo action so
>>> it can be used by other frontends.
>>>
>>> I tried to stick to John's original u32 frontend parser as much as
>>> possible, adapting it to build the intermediate representation.
>>>
>>> After this change, we don't expose the tc action structure layout and
>>> other similar frontend details to the backend anymore to the backend
>>> anymore. I think this is good since changes in the frontend should not
>>> need to be propagated to the 1..n drivers supporting u32 offloads. In
>>> that sense, this helps to keep the frontend software representation
>>> separated from low-level backend driver details.
>>>
>>> After this patch, it should be possible to put the tc_cls_u32_knode
>>> structure into diet since we only need the handle (as unique id) and the
>>> ast tree.
>>>
>>
>> On ixgbe this is true but going forward I can support hash functions
>> so you need the divisor, prio, and handle minimally. I'm not sure how
>> to do hash tables for example in this IR yet. Might be there I'm still
>> trying to grok the IR details.
> 
> We just need a description with the list elements that you want to
> place in the hashtable.
> 
> The matching can be expressed in ast by adding a list of elements that
> are part of the hashtable:
> 
>                  relational
>                     / \
>                    /   \
>                   /     \
>            payload      list of elements
>   (offset, base, len)       |
>                             \
>                              --> e1 --> e2 --> ... --> en

Sure it can be done but taking the u32 directly and flower directly is
so easy I don't see the need for the complexity.

> 
>>> I couldn't send any more incremental changes to update previous work
>>> since the u32 parser and the internal representation were put together,
>>> that why this patch is slightly large.
>>>
>>> Signed-off-by: Pablo Neira Ayuso <pablo@netfilter.org>
>>> ---
>>>  drivers/net/ethernet/intel/ixgbe/ixgbe.h       |   4 -
>>>  drivers/net/ethernet/intel/ixgbe/ixgbe_main.c  | 216 ++++++++--------
>>>  drivers/net/ethernet/intel/ixgbe/ixgbe_model.h | 112 --------
>>>  include/net/pkt_cls.h                          |   3 +
>>>  net/sched/cls_u32.c                            | 344 +++++++++++++++++++++++++
>>
>> Feels like a lot of code to me when the original direct u32 to hw
>> was reasonably small and the flower implementation is small as well.
> 
> This is your tc-u32 parser plus simple boiler plate code to build the
> IR that will be common to everyone. Without this, other drivers will
> have to track u32 links and so on.
> 

But you lost the model piece which allows me to easily change the
hardware underneath the driver.

> Other than that, we'll end up with 1..N different u32 parsers in the
> backend.
> 

well if folks like the way I wrote the u32 parser we could standardize
the model and header for u32 and give them helper functions. But by
writing your own model you can optimize it. Anyways at the moment it
looks like we will only have two the Intel and mlx ones I haven't
seen anyone else working on this yet.

>> [...]
>>
>>>  
>>> +static int ixgbe_tcp_jit(struct net_ir_jit_ctx *ctx,
>>> +			 const struct net_ir_expr *expr,
>>> +			 void *data)
>>> +{
>>> +	struct ixgbe_filter *f = (struct ixgbe_filter *)data;
>>> +	struct net_ir_expr *right = expr->relational.right;
>>> +	struct net_ir_expr *payload;
>>> +	u32 mask = 0xffffffff;
>>> +
>>> +	if (expr->relational.left->type == NET_IR_EXPR_BINOP) {
>>> +		payload = expr->relational.left->binop.left;
>>> +		mask = expr->relational.left->binop.right->value.data;
>>> +	} else {
>>> +		payload = expr->relational.left;
>>> +	}
>>> +
>>> +	switch (payload->payload.offset) {
>>
>> I don't like this because it hardcodes the offset pieces into the
>> driver. The reason I used a model.h header file with its structures
>> in ixgbe is the next step is to expose the parser so the model can be
>> updated. So that when the hardware parser changes the data structure
>> can be changed via firmware call. The style of coding here wont work
>> for that so we still need the model.h file and for loop.
> 
> Model update via firmware?
> 
> I think it's been already several conference rounds insisting on the
> fact that we don't want infrastructure that allows to insert binary
> blobs/sdks.

Its a firmware/ucode update we do this already today and the
infrastructure to do it has been in the kernel for a long time.
Firmware has always been a binary blob it has to be. All this means is
we have to read the parser out at init() time of the driver instead of
using a static model.h header. Hardware is not going to be static
anymore.

Even if its based on the device ID its the same problem I wouldn't
want a bunch of if (device X) do this else (device y) do that and
so on.

> 
> I don't think this is the way to go.
> 
>> [...]
>>
>>>  static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
>>>  				  __be16 protocol,
>>>  				  struct tc_cls_u32_offload *cls)
>>>  {
>>>  	u32 loc = cls->knode.handle & 0xfffff;
>>>  	struct ixgbe_hw *hw = &adapter->hw;
>>> -	struct ixgbe_mat_field *field_ptr;
>>>  	struct ixgbe_filter f;
>>> -#ifdef CONFIG_NET_CLS_ACT
>>> -	const struct tc_action *a;
>>> -#endif
>>> -	int i, err = 0;
>>> +	int err = 0;
>>>  	u32 handle;
>>>  
>>>  	memset(&f.mask, 0, sizeof(union ixgbe_atr_input));
>>>  	handle = cls->knode.handle;
>>>  
>>> -	/* At the moment cls_u32 jumps to transport layer and skips past
>>> -	 * L2 headers. The canonical method to match L2 frames is to use
>>> -	 * negative values. However this is error prone at best but really
>>> -	 * just broken because there is no way to "know" what sort of hdr
>>> -	 * is in front of the transport layer. Fix cls_u32 to support L2
>>> -	 * headers when needed.
>>> -	 */
>>> -	if (protocol != htons(ETH_P_IP))
>>
>> This was hardcoded originally just to find the inital offset going
>> forward I want to patch u32 to start at the link layer to avoid
>> this.
> 
> Then, you extend the tc-u32 parser, but from the frontend core
> infrastructure code, so everyone we'll benefit for this incremental
> update.
> 

We will push these patches into u32 anyways and it doesn't need the IR
to benefit everyone.

>> [...]
>>
>>> -		if (!found_entry)
>>> -			goto err_out;
>>> -	}
>>> +	if (net_ir_jit(&cls->knode.ast, &ixgbe_desc, &f) < 0)
>>> +		return -EINVAL;
>>>  
>>
>> OK but this takes a cls_u32 offload engine that was running on a handful
>> of LOC + a header file description that in the future will be read out
>> of firmware and replaces it with an entire infrastructure. I'm not
>> convinced the added complexity is worth it.
> 
> It's all about consolidating infrastructure.
> 

sure I just think we are getting ahead of ourselves here and
consolidating a single driver and single classifier and creating a lot
of code along the way when the current implementation is simple IMO with
this I have to follow an AST and pointers and a bunch of allocs. As the
guy writing the driver I see no need for it yet.

>> [...]
>>
>>> --- a/drivers/net/ethernet/intel/ixgbe/ixgbe_model.h
>>> +++ /dev/null
>>> @@ -1,112 +0,0 @@
>>> -/*******************************************************************************
>>> - *
>>> - * Intel 10 Gigabit PCI Express Linux drive
>>> - * Copyright(c) 2016 Intel Corporation.
>>> - *
>>> - * This program is free software; you can redistribute it and/or modify it
>>> - * under the terms and conditions of the GNU General Public License,
>>> - * version 2, as published by the Free Software Foundation.
>>> - *
>>> - * This program is distributed in the hope it will be useful, but WITHOUT
>>> - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
>>> - * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
>>> - * more details.
>>> - *
>>> - * You should have received a copy of the GNU General Public License along
>>> - * with this program.  If not, see <http://www.gnu.org/licenses/>.
>>> - *
>>> - * The full GNU General Public License is included in this distribution in
>>> - * the file called "COPYING".
>>> - *
>>> - * Contact Information:
>>> - * e1000-devel Mailing List <e1000-devel@lists.sourceforge.net>
>>> - * Intel Corporation, 5200 N.E. Elam Young Parkway, Hillsboro, OR 97124-6497
>>> - *
>>> - ******************************************************************************/
>>> -
>>> -#ifndef _IXGBE_MODEL_H_
>>> -#define _IXGBE_MODEL_H_
>>> -
>>> -#include "ixgbe.h"
>>> -#include "ixgbe_type.h"
>>> -
>>> -struct ixgbe_mat_field {
>>> -	unsigned int off;
>>> -	unsigned int mask;
>>> -	int (*val)(struct ixgbe_fdir_filter *input,
>>> -		   union ixgbe_atr_input *mask,
>>> -		   u32 val, u32 m);
>>> -	unsigned int type;
>>> -};
>>
>> As noted above the intent is to make the parser programmable so
>> something like this model needs to exist for firmware to populate.
> 
> Oh, again references to update via firmware.
> 
> [...]
>> My preference is to hold this work until we get a few more driver
>> implementation for multiple things and see if that can scale easily
>> as I suspect it might without having to over-engineer it.
> 
> I don't agree, that's simply preventing by now other frontends and
> drivers to get offload support.
> 

I don't think so at all. The driver support is simple at the moment
and the interface is still evolving in the frontend. What is this
preventing now? I can submit my flower implementation but the mlx folks
wanted to do it. The other concern about performance is also a big
one for me. And the AST as it is doesn't solve my eBPF case either so
maybe we should just lower everything to eBPF and have one IR instead
of two if we really want a universal IR.

.John

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

* Re: [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion
  2016-02-26 11:47   ` Pablo Neira Ayuso
@ 2016-02-26 15:42     ` John Fastabend
  2016-02-26 16:19       ` Pablo Neira Ayuso
  2016-02-26 17:26     ` David Miller
  1 sibling, 1 reply; 19+ messages in thread
From: John Fastabend @ 2016-02-26 15:42 UTC (permalink / raw)
  To: Pablo Neira Ayuso; +Cc: netdev, davem, jiri, horms

On 16-02-26 03:47 AM, Pablo Neira Ayuso wrote:
> On Thu, Feb 25, 2016 at 10:11:36AM -0800, John Fastabend wrote:
>> On 16-02-25 09:37 AM, Pablo Neira Ayuso wrote:
>>> Hi,
>>>
>>> This patchset contains the core infrastructure for the generic
>>> intermediate representation that I presented during NetDev 1.1's
>>> nftables switchdev talk. This includes the basic infrastructure to
>>> convert the tc cls_u32 based on John's parser.
>>>
>>> The main goals of this patchset are:
>>>
>>> * Provide an unified abstract syntax tree (ast) that can be passed to
>>>   the backend driver for translation to internal representation. Then,
>>>   based on the backend description, generate the internal
>>>   representation. This should reduce the amount of code to maintain
>>>   in the driver since every frontend, via parser, generates the ast
>>>   that is consumed by the driver through one single ndo indirection.
>>>
>>
>> The trouble here is we lose a lot of the goodness of any particular
>> classifier this way. For example flower can be implemented with a
>> single jump table and this should translate into updates per second
>> being very high.
>>
>> cls_u32 on the other hand is going to require more state and I believe
>> will be slightly slower in the updates per second benchmark because
>> it requires a set of jump tables.
> 

sorry responding out of order I missed this when I scanned my email
this morning.

> The parser on the backend is not the way to go.  I would expect it
> will not take long time until we see other people copying/pasting it
> and adapting it to their need just because they don't find the way/put
> the care to make it fit in the infrastructure / abstract it the right
> way.
> 
> ... And then we will get complexity spread out all over the place, in
> every single driver. This will be unmaintainable.
> 
> The frontend parser should be generic to everyone, it should be placed
> in the core, so everyone will take care of improving it.
> 
> Let's make a summary picture to clarify this:
> 
> 
>             core                               drivers
>           -------                             ---------
>         1 per frontend                       1 per driver
> 
>   [ people from all vendors ]           [     code consuming    ]
>   [  working to improve     ]           [   IR representation   ]
>   [   the common parser     ]
>   [     and improve IR      ]
> 
>      Complexity goes here
>        in the form of
>        infrastructure
> 

I understand the ideal here but I don't see the need for it yet. I'm
writing the ebpf parser now and evolving the u32 frontend to support
new use cases. I guess all the added complexity in this series comes
from nft? My big concern is performance and I just don't believe
its much complexity to take u32,flower,etc into the driver. Or
if we don't like that flower to u32 can be done with a static jump
table you don't need ASTs and such for that. Maybe it would help to see
the nft implementation as that seems to be the only reason to do this
in my mind. And I'm not so keen to cause overhead on the tc side to
support nft. With your AST I still need a eBPF hook so having three
hooks seems fine to me nft, tc, and ebpf.

>> My general argument here is _if_ we can build one universal ast why
>> don't we do this higher up the stack. I'm a bit confused on why we
>> think (a) all classifiers/nftables/etc can be pushed on to a single
>> IR and (b) all hw can consume a single IR if we actually believe this
>> lets whack the upper layers and make one tc classifer and unify it
>> with nftables so we don't have multiple things in the stack to offload.
> 
> Everyone I talked to during Netconf agreed that we want to offload all
> frontends. These battles on imposing frontends to people take us
> nowhere.
> 
> So this "let's whack upper layers and make one" is basically out of
> question.

Sure it was a bit rhetorical, but if we did we would have the IR by
nature of it being the one thing to offload ;)

> 
>>> * Avoid exposing low-level frontend details to the backend, such as
>>>   structure layouts. If the frontend needs to be updated to support a
>>>   new software feature, it is desiderable that such changes don't
>>>   trigger large updates to every driver supporting offloads.
>>>
>>> * Having a common parser for every frontend, instead of allowing each
>>>   backend driver to re-invent the wheel with its own parser, this is
>>>   just spreading out complexity all over the place.
>>
>>
>> I think we should make helper routines and build a library for driver
>> writers instead. This way each driver writer can call stub functions
>> to do things like validate u32 against a model.h header or something
>> along these lines. This keeps flexibility IMO and lets us optimize
>> the backends for the specific hardware. Remember we eventually need
>> to get to 100k's of updates a second to compete with user space APIs
>> if you want this to work on high-end devices. If you don't get here
>> it wont be usable for a class of systems and we will just lift the
>> functions into user space for performance reasons. I'm sceptical that
>> converting from shiny classifier to software IR and then hw IR is
>> going to be efficient when I can go direct from classifier to hw IR.
> 
> To me this sounds like trying to microptimize the control plane code
> by reducing the amount of code to translate it into hardware.
> However, this comes at the cost of increasing the overall complexity
> in the backend.
> 

Fine by me I can deal with it.

> I think we have to make an effort to provide generic solutions, Linux
> is a generic purpose stack.
> 

I'm OK with that but not at the expense of making your high-end devices
not usable. And this feature is really IMO targeted at high-end devices
for the moment.

> Look at the problem from another perspective based on this statement:
> 
> "ioctl()/setsockopt()/system call of choice plus fixed layout
> structures are faster control planes than Netlink."
> 
> Answer: Yes, that's probably right, they require way less code to
> build and to parse a message. However, *we are telling everyone that
> people should use Netlink".
> 
> Why?
> 
> Because it helps us avoid exposing the low level details of our
> implementation and that makes it more more extensible in the kind of
> evolutionary development that we have.
> 
> Should we microoptimize the control plane path through avoiding
> abstractions? I don't think so.
> 

Bottom line I need at least 50k updates per second or so to be
competitive ideally I want to get to 100k or more. Without this
its not usable for a bunch of use cases.

> Another example, look at this structure (what we currently have in
> net-next):
> 
>  struct tc_cls_u32_knode {
>         struct tcf_exts *exts;
>         struct tc_u32_sel *sel;
>         u32 handle;
>         u32 val;
>         u32 mask;
>         u32 link_handle;
>         u8 fshift;
>  };
> 
> Downside is we expose frontend details to the backend.
> 
> * You're exposing structure layouts to the backend, (see
>   tcf_exts struct). Another clear symptom to me that this is not the
>   way to go are those #ifdefs in the backend driver code to check if the
>   tc action API is set. If someone needs to extend the tc action API and
>   we get lots of driver using this in the future, that poor guy
>   extending tc action will have to mangle quite a lot of driver code.
> 

Agreed the ifdefs are likely not ideal.

> * Why the driver should care if tc-u32 uses link_handles to connect rules
>   and such?
> 
> So better pass the driver something generic that it can translate.
> With common IR (after one more patches I have here) this will look
> like:
> 
>  struct tc_cls_u32_knode {
>         u32                     handle;
>         struct net_ir_ast       ast;
>  };
> 
>> Also the IR here only maps easily to a specific set of hardware. For
>> example it doesn't really map well to some of my more programmable
>> devices that want to consume "programs" more like BPF. So I don't see
>> any universal IR being possible. At best you get a set of IRs
>> dependent on the underlying architecture.
> 
> Universal sounds too ambicious, I called this generic. Generic in the
> sense of trying to unify and provide as much common infrastructure as
> possible. We probably cannot escape having some specificities for some
> of the frontends we have, so we'll have to pass this specific info as
> decoration to the driver, but that's should not be an excuse to skip
> having something common for everyone IMO.
> 
> Anyway I'd suggest you go have a look at the IR and see what you send
> incremental patches to improve what you cannot represent in a generic
> way.
> 

I looked at it and want to know what it does to performance. Also I
think ebpf is a better IR if we want an IR so why not lower it to ebpf
if we want the generic object.

>>> A summary picture of the infrastructure looks like this:
>>>
>>>                 parser
>>>         tc-u32 -------
>>>                        \            jit
>>>      tc-flower -------------- ast ------> Backend driver
>>>                        /
>>>            nft -------
>>>
>>> So the idea is that every frontend implements a parser that builds the
>>> ast, then this ast is passed via ndo to the driver. The parser is common
>>> to everyone, is part of the common core infrastructure.
>>
>> Same argument as above with some helper libs going from u32 to driver
>> is really minimal code. And what are the performance impact of taking
>> a simple classifier like flower to ast to backend.
>>
>> Looking at the similarities between u32 and nft should we unify them in
>> the stack so that,
>>
>>    nft ---- jit(u32)
>>
>> and just run nft inside u32? Or
>>
>>    nft ---- jit(ebpf)
>>
>> and just offload u32 or ebpf? or
>>
>>     u32 --- jit(nft)
>>
>> any one of those combinations :)
> 
> As a said above, let's keep this discussion away from the "let's
> provide a unified frontend" because takes us nowhere.

Sure.

> 
>>> The tc-u32 parser is a bit complicated because of having the matching
>>> spread out in different rules through links, but John already came up a
>>> basic parser than should be placed in the frontend so everyone can
>>> improve it to generate more expressive ast.
>>>
>>> Note: I don't have access to ixgbe hardware, so I have validated this
>>> patchset by splicing main parts of the ixgbe backend jit code in simple
>>> debugging patches that I have here. Quite rudimentary but it has passed
>>> some basic tests, may still have gotten anything broken. Anyway, the
>>> main goal is to generate debate on this.
>>>
>>
>> Yep lets debate the general approach. I can deal with ixgbe details.
>> My reaction is this seems overkill I only see offloads popping up for
>> flower, u32, nft, and bpf at the moment. Flower doesn't need heavy
>> overhead due to its simplicity. u32 can be done with some basic helpers
>> for devices like ixgbe. I think nft can be done the same. bpf is its own
>> beast that doesn't map well to another IR its more or less its own IR
>> anyways.
>>
>> If we need/want a IR I would go for something instruction based
>> like ebpf over this.
> 
> bpf is a final representation from a compiler perspective, it is
> bytecode, I don't think it is meant to be designed as an intermediate
> representation.
> 

Well its just a set of instructions and each driver writer would write
their own jit just the same as the AST. Also it would save me having to
write two jits one for NFT-IR and one for BPF so I would prefer to just
do one and use it as the generic IR its not so bad to convert u32 into
it for example. If we need two ebpf-jit, nft-jit I mind as well have
three tc-jit, nft-jit, and ebpf-jit and optimize each one for their
targeted front-ends.

Thanks,
John

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

* Re: [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation
  2016-02-26 14:53       ` John Fastabend
@ 2016-02-26 16:02         ` Pablo Neira Ayuso
  2016-02-26 16:34           ` John Fastabend
  2016-02-26 17:38           ` David Miller
  2016-02-26 17:34         ` David Miller
  1 sibling, 2 replies; 19+ messages in thread
From: Pablo Neira Ayuso @ 2016-02-26 16:02 UTC (permalink / raw)
  To: John Fastabend; +Cc: netdev, davem, jiri, horms

On Fri, Feb 26, 2016 at 06:53:11AM -0800, John Fastabend wrote:
> On 16-02-26 06:24 AM, Pablo Neira Ayuso wrote:
> > On Thu, Feb 25, 2016 at 12:37:38PM -0800, John Fastabend wrote:
> >> On 16-02-25 09:37 AM, Pablo Neira Ayuso wrote:
> >>> This patch moves the u32 parser from the ixgbe that John has made to the
> >>> core u32. This parser has been adapted to build the intermediate
> >>> representation.
> >>>
> >>> To store the parsing information, this patch introduces a parse table
> >>> object, one per device, so we don't need to store the parsing states in
> >>> the adapter, which is the major dependency with previous patches.
> >>>
> >>> Since u32 allows rules connected via links, the u32 parser tracks this
> >>> links and then generates the intermediate representation that is passed
> >>> to the ixgbe driver.
> >>
> >> It also supports hash tables and loops.
> > 
> > Then, it is a matter of extending the tc-u32 parser and the IR to
> > support whatever you need.
> 
> sure but its easier for me to just consume u32 at the moment. And I
> am concerned about performance overhead with this IR. I'm going to need
> to have a performant solution eventually and I don't like converting
> into one IR only to go into another.

Just because you want to early microoptimize this thing by saving a
little of extra code that runs from the control plane path.

> >>> New drivers will only have to implement the jit translation code based
> >>> on the intermediate representation. With some extra work, I think it
> >>> should be possible to generalize the existing tc specific ndo action so
> >>> it can be used by other frontends.
> >>>
> >>> I tried to stick to John's original u32 frontend parser as much as
> >>> possible, adapting it to build the intermediate representation.
> >>>
> >>> After this change, we don't expose the tc action structure layout and
> >>> other similar frontend details to the backend anymore to the backend
> >>> anymore. I think this is good since changes in the frontend should not
> >>> need to be propagated to the 1..n drivers supporting u32 offloads. In
> >>> that sense, this helps to keep the frontend software representation
> >>> separated from low-level backend driver details.
> >>>
> >>> After this patch, it should be possible to put the tc_cls_u32_knode
> >>> structure into diet since we only need the handle (as unique id) and the
> >>> ast tree.
> >>>
> >>
> >> On ixgbe this is true but going forward I can support hash functions
> >> so you need the divisor, prio, and handle minimally. I'm not sure how
> >> to do hash tables for example in this IR yet. Might be there I'm still
> >> trying to grok the IR details.
> > 
> > We just need a description with the list elements that you want to
> > place in the hashtable.
> > 
> > The matching can be expressed in ast by adding a list of elements that
> > are part of the hashtable:
> > 
> >                  relational
> >                     / \
> >                    /   \
> >                   /     \
> >            payload      list of elements
> >   (offset, base, len)       |
> >                             \
> >                              --> e1 --> e2 --> ... --> en
> 
> Sure it can be done but taking the u32 directly and flower directly is
> so easy I don't see the need for the complexity.

You prefer to push the complexity on the driver side. Then, the
complexity will spread all over the place.

> >>> I couldn't send any more incremental changes to update previous work
> >>> since the u32 parser and the internal representation were put together,
> >>> that why this patch is slightly large.
> >>>
> >>> Signed-off-by: Pablo Neira Ayuso <pablo@netfilter.org>
> >>> ---
> >>>  drivers/net/ethernet/intel/ixgbe/ixgbe.h       |   4 -
> >>>  drivers/net/ethernet/intel/ixgbe/ixgbe_main.c  | 216 ++++++++--------
> >>>  drivers/net/ethernet/intel/ixgbe/ixgbe_model.h | 112 --------
> >>>  include/net/pkt_cls.h                          |   3 +
> >>>  net/sched/cls_u32.c                            | 344 +++++++++++++++++++++++++
> >>
> >> Feels like a lot of code to me when the original direct u32 to hw
> >> was reasonably small and the flower implementation is small as well.
> > 
> > This is your tc-u32 parser plus simple boiler plate code to build the
> > IR that will be common to everyone. Without this, other drivers will
> > have to track u32 links and so on.
> 
> But you lost the model piece which allows me to easily change the
> hardware underneath the driver.

You want to hide hardware capabilities behind that model.

> > Other than that, we'll end up with 1..N different u32 parsers in the
> > backend.
> 
> well if folks like the way I wrote the u32 parser we could standardize
> the model and header for u32 and give them helper functions.

I see, so now you agree with me that the u32 parser should be moved to
the core, that's good.

> But by writing your own model you can optimize it.

I see, but you mean you keep using this ugly IR representation:

 struct tc_cls_u32_knode {
        struct tcf_exts *exts;
        struct tc_u32_sel *sel;
        u32 handle;
        u32 val;
        u32 mask;
        u32 link_handle;
        u8 fshift;
 };

that exposes the tc frontend details. Sorry, but this is not nice.

> Anyways at the moment it looks like we will only have two the Intel
> and mlx ones I haven't seen anyone else working on this yet.

I don't see any convincing argument so far to stick to your approach
and not to use some kind of generic IR.

And even if you have concerns on the generic IR, you can propose
amendments for it to improve it since that would be core
infrastructure.

> >> [...]
> >>
> >>>  
> >>> +static int ixgbe_tcp_jit(struct net_ir_jit_ctx *ctx,
> >>> +			 const struct net_ir_expr *expr,
> >>> +			 void *data)
> >>> +{
> >>> +	struct ixgbe_filter *f = (struct ixgbe_filter *)data;
> >>> +	struct net_ir_expr *right = expr->relational.right;
> >>> +	struct net_ir_expr *payload;
> >>> +	u32 mask = 0xffffffff;
> >>> +
> >>> +	if (expr->relational.left->type == NET_IR_EXPR_BINOP) {
> >>> +		payload = expr->relational.left->binop.left;
> >>> +		mask = expr->relational.left->binop.right->value.data;
> >>> +	} else {
> >>> +		payload = expr->relational.left;
> >>> +	}
> >>> +
> >>> +	switch (payload->payload.offset) {
> >>
> >> I don't like this because it hardcodes the offset pieces into the
> >> driver. The reason I used a model.h header file with its structures
> >> in ixgbe is the next step is to expose the parser so the model can be
> >> updated. So that when the hardware parser changes the data structure
> >> can be changed via firmware call. The style of coding here wont work
> >> for that so we still need the model.h file and for loop.
> > 
> > Model update via firmware?
> > 
> > I think it's been already several conference rounds insisting on the
> > fact that we don't want infrastructure that allows to insert binary
> > blobs/sdks.
> 
> Its a firmware/ucode update we do this already today and the
> infrastructure to do it has been in the kernel for a long time.
> Firmware has always been a binary blob it has to be. All this means is
> we have to read the parser out at init() time of the driver instead of
> using a static model.h header. Hardware is not going to be static
> anymore.
>
> Even if its based on the device ID its the same problem I wouldn't
> want a bunch of if (device X) do this else (device y) do that and
> so on.

That is code that you can keep in your driver side. I don't think
that's a argument not to have a consolidated infrastructure that is
better for everyone.

> > I don't think this is the way to go.
> > 
> >> [...]
> >>
> >>>  static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
> >>>  				  __be16 protocol,
> >>>  				  struct tc_cls_u32_offload *cls)
> >>>  {
> >>>  	u32 loc = cls->knode.handle & 0xfffff;
> >>>  	struct ixgbe_hw *hw = &adapter->hw;
> >>> -	struct ixgbe_mat_field *field_ptr;
> >>>  	struct ixgbe_filter f;
> >>> -#ifdef CONFIG_NET_CLS_ACT
> >>> -	const struct tc_action *a;
> >>> -#endif
> >>> -	int i, err = 0;
> >>> +	int err = 0;
> >>>  	u32 handle;
> >>>  
> >>>  	memset(&f.mask, 0, sizeof(union ixgbe_atr_input));
> >>>  	handle = cls->knode.handle;
> >>>  
> >>> -	/* At the moment cls_u32 jumps to transport layer and skips past
> >>> -	 * L2 headers. The canonical method to match L2 frames is to use
> >>> -	 * negative values. However this is error prone at best but really
> >>> -	 * just broken because there is no way to "know" what sort of hdr
> >>> -	 * is in front of the transport layer. Fix cls_u32 to support L2
> >>> -	 * headers when needed.
> >>> -	 */
> >>> -	if (protocol != htons(ETH_P_IP))
> >>
> >> This was hardcoded originally just to find the inital offset going
> >> forward I want to patch u32 to start at the link layer to avoid
> >> this.
> > 
> > Then, you extend the tc-u32 parser, but from the frontend core
> > infrastructure code, so everyone we'll benefit for this incremental
> > update.
> 
> We will push these patches into u32 anyways and it doesn't need the IR
> to benefit everyone.

I see, now you're convinced to push the tc-u32 parser to the core
infrastructure.

> >> [...]
> >>
> >>> -		if (!found_entry)
> >>> -			goto err_out;
> >>> -	}
> >>> +	if (net_ir_jit(&cls->knode.ast, &ixgbe_desc, &f) < 0)
> >>> +		return -EINVAL;
> >>>  
> >>
> >> OK but this takes a cls_u32 offload engine that was running on a handful
> >> of LOC + a header file description that in the future will be read out
> >> of firmware and replaces it with an entire infrastructure. I'm not
> >> convinced the added complexity is worth it.
> > 
> > It's all about consolidating infrastructure.
> > 
> 
> sure I just think we are getting ahead of ourselves here and
> consolidating a single driver and single classifier and creating a lot
> of code along the way when the current implementation is simple IMO with
> this I have to follow an AST and pointers and a bunch of allocs. As the
> guy writing the driver I see no need for it yet.

This is not creating a lot of code.

My patch is basically moving your u32 parser into the core, this is
probably the largest piece of code in this patchset.

The remaining bits are just a simple infrastructure to build an AST
and walk over it.

$ wc -l net/core/ir.c
273 net/core/ir.c

> >> [...]
> >>
> >>> --- a/drivers/net/ethernet/intel/ixgbe/ixgbe_model.h
> >>> +++ /dev/null
> >>> @@ -1,112 +0,0 @@
> >>> -/*******************************************************************************
> >>> - *
> >>> - * Intel 10 Gigabit PCI Express Linux drive
> >>> - * Copyright(c) 2016 Intel Corporation.
> >>> - *
> >>> - * This program is free software; you can redistribute it and/or modify it
> >>> - * under the terms and conditions of the GNU General Public License,
> >>> - * version 2, as published by the Free Software Foundation.
> >>> - *
> >>> - * This program is distributed in the hope it will be useful, but WITHOUT
> >>> - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
> >>> - * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
> >>> - * more details.
> >>> - *
> >>> - * You should have received a copy of the GNU General Public License along
> >>> - * with this program.  If not, see <http://www.gnu.org/licenses/>.
> >>> - *
> >>> - * The full GNU General Public License is included in this distribution in
> >>> - * the file called "COPYING".
> >>> - *
> >>> - * Contact Information:
> >>> - * e1000-devel Mailing List <e1000-devel@lists.sourceforge.net>
> >>> - * Intel Corporation, 5200 N.E. Elam Young Parkway, Hillsboro, OR 97124-6497
> >>> - *
> >>> - ******************************************************************************/
> >>> -
> >>> -#ifndef _IXGBE_MODEL_H_
> >>> -#define _IXGBE_MODEL_H_
> >>> -
> >>> -#include "ixgbe.h"
> >>> -#include "ixgbe_type.h"
> >>> -
> >>> -struct ixgbe_mat_field {
> >>> -	unsigned int off;
> >>> -	unsigned int mask;
> >>> -	int (*val)(struct ixgbe_fdir_filter *input,
> >>> -		   union ixgbe_atr_input *mask,
> >>> -		   u32 val, u32 m);
> >>> -	unsigned int type;
> >>> -};
> >>
> >> As noted above the intent is to make the parser programmable so
> >> something like this model needs to exist for firmware to populate.
> > 
> > Oh, again references to update via firmware.
> > 
> > [...]
> >> My preference is to hold this work until we get a few more driver
> >> implementation for multiple things and see if that can scale easily
> >> as I suspect it might without having to over-engineer it.
> > 
> > I don't agree, that's simply preventing by now other frontends and
> > drivers to get offload support.
> 
> I don't think so at all. The driver support is simple at the moment
> and the interface is still evolving in the frontend. What is this
> preventing now? I can submit my flower implementation but the mlx folks
> wanted to do it. The other concern about performance is also a big
> one for me. And the AST as it is doesn't solve my eBPF case either so
> maybe we should just lower everything to eBPF and have one IR instead
> of two if we really want a universal IR.

I think you can push a restricted version of eBPF, the frontend parser
won't be nice given that it will have to deal with all its complexity
(compared to the smaller complexity of other existing frontends).
AFAIK, there is no hardware fully supporting eBPF but some
variants/limited versions of bpf-like things, and I don't know of any
standarization effort on this from vendors. So to me you're worring on
things that don't exist yet.

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

* Re: [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion
  2016-02-26 15:42     ` John Fastabend
@ 2016-02-26 16:19       ` Pablo Neira Ayuso
  2016-02-26 16:46         ` John Fastabend
                           ` (2 more replies)
  0 siblings, 3 replies; 19+ messages in thread
From: Pablo Neira Ayuso @ 2016-02-26 16:19 UTC (permalink / raw)
  To: John Fastabend; +Cc: netdev, davem, jiri, horms

On Fri, Feb 26, 2016 at 07:42:25AM -0800, John Fastabend wrote:
> On 16-02-26 03:47 AM, Pablo Neira Ayuso wrote:
> > The parser on the backend is not the way to go.  I would expect it
> > will not take long time until we see other people copying/pasting it
> > and adapting it to their need just because they don't find the way/put
> > the care to make it fit in the infrastructure / abstract it the right
> > way.
> > 
> > ... And then we will get complexity spread out all over the place, in
> > every single driver. This will be unmaintainable.
> > 
> > The frontend parser should be generic to everyone, it should be placed
> > in the core, so everyone will take care of improving it.
> > 
> > Let's make a summary picture to clarify this:
> > 
> > 
> >             core                               drivers
> >           -------                             ---------
> >         1 per frontend                       1 per driver
> > 
> >   [ people from all vendors ]           [     code consuming    ]
> >   [  working to improve     ]           [   IR representation   ]
> >   [   the common parser     ]
> >   [     and improve IR      ]
> > 
> >      Complexity goes here
> >        in the form of
> >        infrastructure
> > 
> 
> I understand the ideal here but I don't see the need for it yet. I'm
> writing the ebpf parser now and evolving the u32 frontend to support
> new use cases. I guess all the added complexity in this series comes
> from nft? My big concern is performance and I just don't believe
> its much complexity to take u32,flower,etc into the driver. Or
> if we don't like that flower to u32 can be done with a static jump
> table you don't need ASTs and such for that.

Well, you can probably make a in-kernel compiler-like thing without
ASTs, but that doesn't mean it will look nice.

> Maybe it would help to see the nft implementation as that seems to
> be the only reason to do this in my mind. And I'm not so keen to
> cause overhead on the tc side to support nft. With your AST I still
> need a eBPF hook so having three hooks seems fine to me nft, tc, and
> ebpf.

I see no reason to have as many hooks as frontends to start with. If
you find limitations with the IR that are unfixable for any of the
existing frontends in the future, then we can add direct hook as final
solution.

But going the other way around, ie. adding one hook per frontend when
we can already represent several of them with this representation,
that makes no sense to me.

> >>> * Avoid exposing low-level frontend details to the backend, such as
> >>>   structure layouts. If the frontend needs to be updated to support a
> >>>   new software feature, it is desiderable that such changes don't
> >>>   trigger large updates to every driver supporting offloads.
> >>>
> >>> * Having a common parser for every frontend, instead of allowing each
> >>>   backend driver to re-invent the wheel with its own parser, this is
> >>>   just spreading out complexity all over the place.
> >>
> >>
> >> I think we should make helper routines and build a library for driver
> >> writers instead. This way each driver writer can call stub functions
> >> to do things like validate u32 against a model.h header or something
> >> along these lines. This keeps flexibility IMO and lets us optimize
> >> the backends for the specific hardware. Remember we eventually need
> >> to get to 100k's of updates a second to compete with user space APIs
> >> if you want this to work on high-end devices. If you don't get here
> >> it wont be usable for a class of systems and we will just lift the
> >> functions into user space for performance reasons. I'm sceptical that
> >> converting from shiny classifier to software IR and then hw IR is
> >> going to be efficient when I can go direct from classifier to hw IR.
> > 
> > To me this sounds like trying to microptimize the control plane code
> > by reducing the amount of code to translate it into hardware.
> > However, this comes at the cost of increasing the overall complexity
> > in the backend.
> 
> Fine by me I can deal with it.

Good.

> > I think we have to make an effort to provide generic solutions, Linux
> > is a generic purpose stack.
> 
> I'm OK with that but not at the expense of making your high-end devices
> not usable. And this feature is really IMO targeted at high-end devices
> for the moment.
>
> > Look at the problem from another perspective based on this statement:
> > 
> > "ioctl()/setsockopt()/system call of choice plus fixed layout
> > structures are faster control planes than Netlink."
> > 
> > Answer: Yes, that's probably right, they require way less code to
> > build and to parse a message. However, *we are telling everyone that
> > people should use Netlink".
> > 
> > Why?
> > 
> > Because it helps us avoid exposing the low level details of our
> > implementation and that makes it more more extensible in the kind of
> > evolutionary development that we have.
> > 
> > Should we microoptimize the control plane path through avoiding
> > abstractions? I don't think so.
> > 
> 
> Bottom line I need at least 50k updates per second or so to be
> competitive ideally I want to get to 100k or more. Without this
> its not usable for a bunch of use cases.

Good, I'm all for reaching those numbers, we can optimize the generic
IR if this ever becomes the bottleneck.

> > Another example, look at this structure (what we currently have in
> > net-next):
> > 
> >  struct tc_cls_u32_knode {
> >         struct tcf_exts *exts;
> >         struct tc_u32_sel *sel;
> >         u32 handle;
> >         u32 val;
> >         u32 mask;
> >         u32 link_handle;
> >         u8 fshift;
> >  };
> > 
> > Downside is we expose frontend details to the backend.
> > 
> > * You're exposing structure layouts to the backend, (see
> >   tcf_exts struct). Another clear symptom to me that this is not the
> >   way to go are those #ifdefs in the backend driver code to check if the
> >   tc action API is set. If someone needs to extend the tc action API and
> >   we get lots of driver using this in the future, that poor guy
> >   extending tc action will have to mangle quite a lot of driver code.
> > 
> 
> Agreed the ifdefs are likely not ideal.

Thanks.

> > * Why the driver should care if tc-u32 uses link_handles to connect rules
> >   and such?
> > 
> > So better pass the driver something generic that it can translate.
> > With common IR (after one more patches I have here) this will look
> > like:
> > 
> >  struct tc_cls_u32_knode {
> >         u32                     handle;
> >         struct net_ir_ast       ast;
> >  };
> > 
> >> Also the IR here only maps easily to a specific set of hardware. For
> >> example it doesn't really map well to some of my more programmable
> >> devices that want to consume "programs" more like BPF. So I don't see
> >> any universal IR being possible. At best you get a set of IRs
> >> dependent on the underlying architecture.
> > 
> > Universal sounds too ambicious, I called this generic. Generic in the
> > sense of trying to unify and provide as much common infrastructure as
> > possible. We probably cannot escape having some specificities for some
> > of the frontends we have, so we'll have to pass this specific info as
> > decoration to the driver, but that's should not be an excuse to skip
> > having something common for everyone IMO.
> > 
> > Anyway I'd suggest you go have a look at the IR and see what you send
> > incremental patches to improve what you cannot represent in a generic
> > way.
> > 
> 
> I looked at it and want to know what it does to performance. Also I
> think ebpf is a better IR if we want an IR so why not lower it to ebpf
> if we want the generic object.

So you indicate that my simple IR representation adds complexity, and
you want to fix that by using a bytecode-based final representation
instead? Sorry, that doesn't make sense to me.

> >>> The tc-u32 parser is a bit complicated because of having the matching
> >>> spread out in different rules through links, but John already came up a
> >>> basic parser than should be placed in the frontend so everyone can
> >>> improve it to generate more expressive ast.
> >>>
> >>> Note: I don't have access to ixgbe hardware, so I have validated this
> >>> patchset by splicing main parts of the ixgbe backend jit code in simple
> >>> debugging patches that I have here. Quite rudimentary but it has passed
> >>> some basic tests, may still have gotten anything broken. Anyway, the
> >>> main goal is to generate debate on this.
> >>>
> >>
> >> Yep lets debate the general approach. I can deal with ixgbe details.
> >> My reaction is this seems overkill I only see offloads popping up for
> >> flower, u32, nft, and bpf at the moment. Flower doesn't need heavy
> >> overhead due to its simplicity. u32 can be done with some basic helpers
> >> for devices like ixgbe. I think nft can be done the same. bpf is its own
> >> beast that doesn't map well to another IR its more or less its own IR
> >> anyways.
> >>
> >> If we need/want a IR I would go for something instruction based
> >> like ebpf over this.
> > 
> > bpf is a final representation from a compiler perspective, it is
> > bytecode, I don't think it is meant to be designed as an intermediate
> > representation.
> 
> Well its just a set of instructions and each driver writer would write
> their own jit just the same as the AST. Also it would save me having to
> write two jits one for NFT-IR and one for BPF so I would prefer to just
> do one and use it as the generic IR its not so bad to convert u32 into
> it for example. If we need two ebpf-jit, nft-jit I mind as well have
> three tc-jit, nft-jit, and ebpf-jit and optimize each one for their
> targeted front-ends.

I told during Netconf that you can use this IR infrastructure to
generate bpf on the backend side. That would basically allow jitting
in software every frontend, but that is a different front. Anyway I
don't understand you want to use bpf as IR. That's a final
representation that is quite complex because of quite wide instruction
set. This is just adding more complexity which is exactly one of your
concerns.

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

* Re: [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation
  2016-02-26 16:02         ` Pablo Neira Ayuso
@ 2016-02-26 16:34           ` John Fastabend
  2016-02-26 17:38           ` David Miller
  1 sibling, 0 replies; 19+ messages in thread
From: John Fastabend @ 2016-02-26 16:34 UTC (permalink / raw)
  To: Pablo Neira Ayuso; +Cc: netdev, davem, jiri, horms

On 16-02-26 08:02 AM, Pablo Neira Ayuso wrote:
> On Fri, Feb 26, 2016 at 06:53:11AM -0800, John Fastabend wrote:
>> On 16-02-26 06:24 AM, Pablo Neira Ayuso wrote:
>>> On Thu, Feb 25, 2016 at 12:37:38PM -0800, John Fastabend wrote:
>>>> On 16-02-25 09:37 AM, Pablo Neira Ayuso wrote:
>>>>> This patch moves the u32 parser from the ixgbe that John has made to the
>>>>> core u32. This parser has been adapted to build the intermediate
>>>>> representation.
>>>>>
>>>>> To store the parsing information, this patch introduces a parse table
>>>>> object, one per device, so we don't need to store the parsing states in
>>>>> the adapter, which is the major dependency with previous patches.
>>>>>
>>>>> Since u32 allows rules connected via links, the u32 parser tracks this
>>>>> links and then generates the intermediate representation that is passed
>>>>> to the ixgbe driver.
>>>>
>>>> It also supports hash tables and loops.
>>>
>>> Then, it is a matter of extending the tc-u32 parser and the IR to
>>> support whatever you need.
>>
>> sure but its easier for me to just consume u32 at the moment. And I
>> am concerned about performance overhead with this IR. I'm going to need
>> to have a performant solution eventually and I don't like converting
>> into one IR only to go into another.
> 
> Just because you want to early microoptimize this thing by saving a
> little of extra code that runs from the control plane path.

But the entire parser + hardware setup is only 149 lines of code plus
a header file. And 45 lines of it hardware specific code and 15 lines
of it are comments. So the entire piece that takes u32 to hardware is
less than 100 lines of code. I'm going to take that block of code and
share it across all my drivers and just write models for each one. After
I make some firmware changes I'll just read the model in from the
firmware and I wont even need that piece. If/when other driver writers
want to use that core loop I created I'll lift it into some helper
routine anyone can call.

I'll do the flower classifier with a static map from keys to u32 if
folks think that is valuable. And that will only be a handful of lines
of code.

So I guess your right I'm not arguing so much against some IR if its
warranted just that the IR proposed here with AST and all seems
unnecessary for offloading front-ends that are so simple when I have
one more or less already. It seems to push the notion of an AST from
nft onto 'tc' which IMO has no need for it.

> 
>>>>> New drivers will only have to implement the jit translation code based
>>>>> on the intermediate representation. With some extra work, I think it
>>>>> should be possible to generalize the existing tc specific ndo action so
>>>>> it can be used by other frontends.
>>>>>
>>>>> I tried to stick to John's original u32 frontend parser as much as
>>>>> possible, adapting it to build the intermediate representation.
>>>>>
>>>>> After this change, we don't expose the tc action structure layout and
>>>>> other similar frontend details to the backend anymore to the backend
>>>>> anymore. I think this is good since changes in the frontend should not
>>>>> need to be propagated to the 1..n drivers supporting u32 offloads. In
>>>>> that sense, this helps to keep the frontend software representation
>>>>> separated from low-level backend driver details.
>>>>>
>>>>> After this patch, it should be possible to put the tc_cls_u32_knode
>>>>> structure into diet since we only need the handle (as unique id) and the
>>>>> ast tree.
>>>>>
>>>>
>>>> On ixgbe this is true but going forward I can support hash functions
>>>> so you need the divisor, prio, and handle minimally. I'm not sure how
>>>> to do hash tables for example in this IR yet. Might be there I'm still
>>>> trying to grok the IR details.
>>>
>>> We just need a description with the list elements that you want to
>>> place in the hashtable.
>>>
>>> The matching can be expressed in ast by adding a list of elements that
>>> are part of the hashtable:
>>>
>>>                  relational
>>>                     / \
>>>                    /   \
>>>                   /     \
>>>            payload      list of elements
>>>   (offset, base, len)       |
>>>                             \
>>>                              --> e1 --> e2 --> ... --> en
>>
>> Sure it can be done but taking the u32 directly and flower directly is
>> so easy I don't see the need for the complexity.
> 
> You prefer to push the complexity on the driver side. Then, the
> complexity will spread all over the place.

As I noted above the complexity is small.

> 
>>>>> I couldn't send any more incremental changes to update previous work
>>>>> since the u32 parser and the internal representation were put together,
>>>>> that why this patch is slightly large.
>>>>>
>>>>> Signed-off-by: Pablo Neira Ayuso <pablo@netfilter.org>
>>>>> ---
>>>>>  drivers/net/ethernet/intel/ixgbe/ixgbe.h       |   4 -
>>>>>  drivers/net/ethernet/intel/ixgbe/ixgbe_main.c  | 216 ++++++++--------
>>>>>  drivers/net/ethernet/intel/ixgbe/ixgbe_model.h | 112 --------
>>>>>  include/net/pkt_cls.h                          |   3 +
>>>>>  net/sched/cls_u32.c                            | 344 +++++++++++++++++++++++++
>>>>
>>>> Feels like a lot of code to me when the original direct u32 to hw
>>>> was reasonably small and the flower implementation is small as well.
>>>
>>> This is your tc-u32 parser plus simple boiler plate code to build the
>>> IR that will be common to everyone. Without this, other drivers will
>>> have to track u32 links and so on.
>>
>> But you lost the model piece which allows me to easily change the
>> hardware underneath the driver.
> 
> You want to hide hardware capabilities behind that model.

No the capabilities are there read the model file. And u32 supports
arbitrary offset:value:mask pairs the file is just exposing the
capabilities in a somewhat general form for devices that are less
flexible and can not consume arbitrary tuples.

Not trying to hide anything... but I don't want to recompile my
kernel just because some user creates a new header type or field.

Anwyays it is sort of independent of IR or not. But I noted it because I
don't want to see things like match ipv4 field X this is a step
backwards.

> 
>>> Other than that, we'll end up with 1..N different u32 parsers in the
>>> backend.
>>
>> well if folks like the way I wrote the u32 parser we could standardize
>> the model and header for u32 and give them helper functions.
> 
> I see, so now you agree with me that the u32 parser should be moved to
> the core, that's good.

Sure as I noted above I am more arguing that the AST IR proposed here is
a bit more than is needed for tc. And seems to be driven entirely by
nft and as I sort of refine my thinking around this through
this thread I'm not sure we need to unify nft/tc at the bottom of the
stack. If nft needs a AST object that is one thing but its not needed
to get tc into the hardware.

Further I've only implemented a single driver at this point so I don't
have anything to actually use a IR as I write the next driver I'll
load the u32 parser and pieces into the core where they are helpful.
This seems pragmatic to me vs building something that has no users yet.

> 
>> But by writing your own model you can optimize it.
> 
> I see, but you mean you keep using this ugly IR representation:
> 
>  struct tc_cls_u32_knode {
>         struct tcf_exts *exts;
>         struct tc_u32_sel *sel;
>         u32 handle;
>         u32 val;
>         u32 mask;
>         u32 link_handle;
>         u8 fshift;
>  };

Yep, but I don't think its so ugly and its simple, easy to understand,
and easy to translate into hardware and to/from the tc frontends.

> 
> that exposes the tc frontend details. Sorry, but this is not nice.
> 

meh pragmatically it works and is easy to translate into and out of.
So rename the structures if that helps. Call it struct tc_actions and
struct tc_selectors.

>> Anyways at the moment it looks like we will only have two the Intel
>> and mlx ones I haven't seen anyone else working on this yet.
> 
> I don't see any convincing argument so far to stick to your approach
> and not to use some kind of generic IR.
> 
> And even if you have concerns on the generic IR, you can propose
> amendments for it to improve it since that would be core
> infrastructure.
> 

If we insist on IR I would either like to use a tc-IR that is basically
what I have today. Or really create a generic IR using eBPF so that I
don't have two. This AST thing feels like lots of complexity for no
good reason. But I don't see any reason to do this until I implement
my next driver where I can use the IR. So I'm going to start working
on the i40e driver soon and can add it then when its needed.

[...]

>>
>> Its a firmware/ucode update we do this already today and the
>> infrastructure to do it has been in the kernel for a long time.
>> Firmware has always been a binary blob it has to be. All this means is
>> we have to read the parser out at init() time of the driver instead of
>> using a static model.h header. Hardware is not going to be static
>> anymore.
>>
>> Even if its based on the device ID its the same problem I wouldn't
>> want a bunch of if (device X) do this else (device y) do that and
>> so on.
> 
> That is code that you can keep in your driver side. I don't think
> that's a argument not to have a consolidated infrastructure that is
> better for everyone.
> 

I think you are right my comment was something of a false argument it
is sort of independent of if you have an IR or not. It just points out
your translation of my ixgbe code was not optimal in my opinion an
really says nothing about your IR.

>>> I don't think this is the way to go.
>>>
>>>> [...]
>>>>
>>>>>  static int ixgbe_configure_clsu32(struct ixgbe_adapter *adapter,
>>>>>  				  __be16 protocol,
>>>>>  				  struct tc_cls_u32_offload *cls)
>>>>>  {
>>>>>  	u32 loc = cls->knode.handle & 0xfffff;
>>>>>  	struct ixgbe_hw *hw = &adapter->hw;
>>>>> -	struct ixgbe_mat_field *field_ptr;
>>>>>  	struct ixgbe_filter f;
>>>>> -#ifdef CONFIG_NET_CLS_ACT
>>>>> -	const struct tc_action *a;
>>>>> -#endif
>>>>> -	int i, err = 0;
>>>>> +	int err = 0;
>>>>>  	u32 handle;
>>>>>  
>>>>>  	memset(&f.mask, 0, sizeof(union ixgbe_atr_input));
>>>>>  	handle = cls->knode.handle;
>>>>>  
>>>>> -	/* At the moment cls_u32 jumps to transport layer and skips past
>>>>> -	 * L2 headers. The canonical method to match L2 frames is to use
>>>>> -	 * negative values. However this is error prone at best but really
>>>>> -	 * just broken because there is no way to "know" what sort of hdr
>>>>> -	 * is in front of the transport layer. Fix cls_u32 to support L2
>>>>> -	 * headers when needed.
>>>>> -	 */
>>>>> -	if (protocol != htons(ETH_P_IP))
>>>>
>>>> This was hardcoded originally just to find the inital offset going
>>>> forward I want to patch u32 to start at the link layer to avoid
>>>> this.
>>>
>>> Then, you extend the tc-u32 parser, but from the frontend core
>>> infrastructure code, so everyone we'll benefit for this incremental
>>> update.
>>
>> We will push these patches into u32 anyways and it doesn't need the IR
>> to benefit everyone.
> 
> I see, now you're convinced to push the tc-u32 parser to the core
> infrastructure.

Yep I'm convinced plus I need to enable other drivers. By and large
I'm more arguing over what IR I guess. I want a light-weight simple
one and you propose something a bit heavier in my opinion.

> 
>>>> [...]
>>>>
>>>>> -		if (!found_entry)
>>>>> -			goto err_out;
>>>>> -	}
>>>>> +	if (net_ir_jit(&cls->knode.ast, &ixgbe_desc, &f) < 0)
>>>>> +		return -EINVAL;
>>>>>  
>>>>
>>>> OK but this takes a cls_u32 offload engine that was running on a handful
>>>> of LOC + a header file description that in the future will be read out
>>>> of firmware and replaces it with an entire infrastructure. I'm not
>>>> convinced the added complexity is worth it.
>>>
>>> It's all about consolidating infrastructure.
>>>
>>
>> sure I just think we are getting ahead of ourselves here and
>> consolidating a single driver and single classifier and creating a lot
>> of code along the way when the current implementation is simple IMO with
>> this I have to follow an AST and pointers and a bunch of allocs. As the
>> guy writing the driver I see no need for it yet.
> 
> This is not creating a lot of code.
> 
> My patch is basically moving your u32 parser into the core, this is
> probably the largest piece of code in this patchset.
> 
> The remaining bits are just a simple infrastructure to build an AST
> and walk over it.
> 
> $ wc -l net/core/ir.c
> 273 net/core/ir.c

OK but why do we need the AST at all? It isn't needed. Why not just take
what we have and literally move it into a core structure?

> 
>>>> [...]
>>>>
>>>>> --- a/drivers/net/ethernet/intel/ixgbe/ixgbe_model.h
>>>>> +++ /dev/null
>>>>> @@ -1,112 +0,0 @@
>>>>> -/*******************************************************************************
>>>>> - *
>>>>> - * Intel 10 Gigabit PCI Express Linux drive
>>>>> - * Copyright(c) 2016 Intel Corporation.
>>>>> - *
>>>>> - * This program is free software; you can redistribute it and/or modify it
>>>>> - * under the terms and conditions of the GNU General Public License,
>>>>> - * version 2, as published by the Free Software Foundation.
>>>>> - *
>>>>> - * This program is distributed in the hope it will be useful, but WITHOUT
>>>>> - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
>>>>> - * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
>>>>> - * more details.
>>>>> - *
>>>>> - * You should have received a copy of the GNU General Public License along
>>>>> - * with this program.  If not, see <http://www.gnu.org/licenses/>.
>>>>> - *
>>>>> - * The full GNU General Public License is included in this distribution in
>>>>> - * the file called "COPYING".
>>>>> - *
>>>>> - * Contact Information:
>>>>> - * e1000-devel Mailing List <e1000-devel@lists.sourceforge.net>
>>>>> - * Intel Corporation, 5200 N.E. Elam Young Parkway, Hillsboro, OR 97124-6497
>>>>> - *
>>>>> - ******************************************************************************/
>>>>> -
>>>>> -#ifndef _IXGBE_MODEL_H_
>>>>> -#define _IXGBE_MODEL_H_
>>>>> -
>>>>> -#include "ixgbe.h"
>>>>> -#include "ixgbe_type.h"
>>>>> -
>>>>> -struct ixgbe_mat_field {
>>>>> -	unsigned int off;
>>>>> -	unsigned int mask;
>>>>> -	int (*val)(struct ixgbe_fdir_filter *input,
>>>>> -		   union ixgbe_atr_input *mask,
>>>>> -		   u32 val, u32 m);
>>>>> -	unsigned int type;
>>>>> -};
>>>>
>>>> As noted above the intent is to make the parser programmable so
>>>> something like this model needs to exist for firmware to populate.
>>>
>>> Oh, again references to update via firmware.
>>>
>>> [...]
>>>> My preference is to hold this work until we get a few more driver
>>>> implementation for multiple things and see if that can scale easily
>>>> as I suspect it might without having to over-engineer it.
>>>
>>> I don't agree, that's simply preventing by now other frontends and
>>> drivers to get offload support.
>>
>> I don't think so at all. The driver support is simple at the moment
>> and the interface is still evolving in the frontend. What is this
>> preventing now? I can submit my flower implementation but the mlx folks
>> wanted to do it. The other concern about performance is also a big
>> one for me. And the AST as it is doesn't solve my eBPF case either so
>> maybe we should just lower everything to eBPF and have one IR instead
>> of two if we really want a universal IR.
> 
> I think you can push a restricted version of eBPF, the frontend parser
> won't be nice given that it will have to deal with all its complexity
> (compared to the smaller complexity of other existing frontends).
> AFAIK, there is no hardware fully supporting eBPF but some
> variants/limited versions of bpf-like things, and I don't know of any
> standarization effort on this from vendors. So to me you're worring on
> things that don't exist yet.
> 

Well they exist but don't have a Linux implementation because we haven't
built one yet. But I think I agree it might be worth keeping ebpf
separate for now.

Thanks,
John

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

* Re: [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion
  2016-02-26 16:19       ` Pablo Neira Ayuso
@ 2016-02-26 16:46         ` John Fastabend
  2016-02-26 17:40         ` David Miller
  2016-02-26 18:53         ` Alexei Starovoitov
  2 siblings, 0 replies; 19+ messages in thread
From: John Fastabend @ 2016-02-26 16:46 UTC (permalink / raw)
  To: Pablo Neira Ayuso; +Cc: netdev, davem, jiri, horms

On 16-02-26 08:19 AM, Pablo Neira Ayuso wrote:
> On Fri, Feb 26, 2016 at 07:42:25AM -0800, John Fastabend wrote:
>> On 16-02-26 03:47 AM, Pablo Neira Ayuso wrote:
>>> The parser on the backend is not the way to go.  I would expect it
>>> will not take long time until we see other people copying/pasting it
>>> and adapting it to their need just because they don't find the way/put
>>> the care to make it fit in the infrastructure / abstract it the right
>>> way.
>>>
>>> ... And then we will get complexity spread out all over the place, in
>>> every single driver. This will be unmaintainable.
>>>
>>> The frontend parser should be generic to everyone, it should be placed
>>> in the core, so everyone will take care of improving it.
>>>
>>> Let's make a summary picture to clarify this:
>>>
>>>
>>>             core                               drivers
>>>           -------                             ---------
>>>         1 per frontend                       1 per driver
>>>
>>>   [ people from all vendors ]           [     code consuming    ]
>>>   [  working to improve     ]           [   IR representation   ]
>>>   [   the common parser     ]
>>>   [     and improve IR      ]
>>>
>>>      Complexity goes here
>>>        in the form of
>>>        infrastructure
>>>
>>
>> I understand the ideal here but I don't see the need for it yet. I'm
>> writing the ebpf parser now and evolving the u32 frontend to support
>> new use cases. I guess all the added complexity in this series comes
>> from nft? My big concern is performance and I just don't believe
>> its much complexity to take u32,flower,etc into the driver. Or
>> if we don't like that flower to u32 can be done with a static jump
>> table you don't need ASTs and such for that.
> 
> Well, you can probably make a in-kernel compiler-like thing without
> ASTs, but that doesn't mean it will look nice.
> 

Sorry looks like I'm creating divergent threads on 3/3 as well. But
maybe unify them here.

I sort of like what we have now and it doesn't use ASTs.

>> Maybe it would help to see the nft implementation as that seems to
>> be the only reason to do this in my mind. And I'm not so keen to
>> cause overhead on the tc side to support nft. With your AST I still
>> need a eBPF hook so having three hooks seems fine to me nft, tc, and
>> ebpf.
> 
> I see no reason to have as many hooks as frontends to start with. If
> you find limitations with the IR that are unfixable for any of the
> existing frontends in the future, then we can add direct hook as final
> solution.
> 
> But going the other way around, ie. adding one hook per frontend when
> we can already represent several of them with this representation,
> that makes no sense to me.
> 

OK but I see no reason to build the most generic IR possible lets build
the most minimal IR possible for the set of frontends supported. Which
at the moment is just tc-u32. And what is embedded in the driver now.

>>>>> * Avoid exposing low-level frontend details to the backend, such as
>>>>>   structure layouts. If the frontend needs to be updated to support a
>>>>>   new software feature, it is desiderable that such changes don't
>>>>>   trigger large updates to every driver supporting offloads.
>>>>>
>>>>> * Having a common parser for every frontend, instead of allowing each
>>>>>   backend driver to re-invent the wheel with its own parser, this is
>>>>>   just spreading out complexity all over the place.
>>>>
>>>>
>>>> I think we should make helper routines and build a library for driver
>>>> writers instead. This way each driver writer can call stub functions
>>>> to do things like validate u32 against a model.h header or something
>>>> along these lines. This keeps flexibility IMO and lets us optimize
>>>> the backends for the specific hardware. Remember we eventually need
>>>> to get to 100k's of updates a second to compete with user space APIs
>>>> if you want this to work on high-end devices. If you don't get here
>>>> it wont be usable for a class of systems and we will just lift the
>>>> functions into user space for performance reasons. I'm sceptical that
>>>> converting from shiny classifier to software IR and then hw IR is
>>>> going to be efficient when I can go direct from classifier to hw IR.
>>>
>>> To me this sounds like trying to microptimize the control plane code
>>> by reducing the amount of code to translate it into hardware.
>>> However, this comes at the cost of increasing the overall complexity
>>> in the backend.
>>
>> Fine by me I can deal with it.
> 
> Good.
> 
>>> I think we have to make an effort to provide generic solutions, Linux
>>> is a generic purpose stack.
>>
>> I'm OK with that but not at the expense of making your high-end devices
>> not usable. And this feature is really IMO targeted at high-end devices
>> for the moment.
>>
>>> Look at the problem from another perspective based on this statement:
>>>
>>> "ioctl()/setsockopt()/system call of choice plus fixed layout
>>> structures are faster control planes than Netlink."
>>>
>>> Answer: Yes, that's probably right, they require way less code to
>>> build and to parse a message. However, *we are telling everyone that
>>> people should use Netlink".
>>>
>>> Why?
>>>
>>> Because it helps us avoid exposing the low level details of our
>>> implementation and that makes it more more extensible in the kind of
>>> evolutionary development that we have.
>>>
>>> Should we microoptimize the control plane path through avoiding
>>> abstractions? I don't think so.
>>>
>>
>> Bottom line I need at least 50k updates per second or so to be
>> competitive ideally I want to get to 100k or more. Without this
>> its not usable for a bunch of use cases.
> 
> Good, I'm all for reaching those numbers, we can optimize the generic
> IR if this ever becomes the bottleneck.
> 
>>> Another example, look at this structure (what we currently have in
>>> net-next):
>>>
>>>  struct tc_cls_u32_knode {
>>>         struct tcf_exts *exts;
>>>         struct tc_u32_sel *sel;
>>>         u32 handle;
>>>         u32 val;
>>>         u32 mask;
>>>         u32 link_handle;
>>>         u8 fshift;
>>>  };
>>>
>>> Downside is we expose frontend details to the backend.
>>>
>>> * You're exposing structure layouts to the backend, (see
>>>   tcf_exts struct). Another clear symptom to me that this is not the
>>>   way to go are those #ifdefs in the backend driver code to check if the
>>>   tc action API is set. If someone needs to extend the tc action API and
>>>   we get lots of driver using this in the future, that poor guy
>>>   extending tc action will have to mangle quite a lot of driver code.
>>>
>>
>> Agreed the ifdefs are likely not ideal.
> 
> Thanks.
> 
>>> * Why the driver should care if tc-u32 uses link_handles to connect rules
>>>   and such?
>>>
>>> So better pass the driver something generic that it can translate.
>>> With common IR (after one more patches I have here) this will look
>>> like:
>>>
>>>  struct tc_cls_u32_knode {
>>>         u32                     handle;
>>>         struct net_ir_ast       ast;
>>>  };
>>>
>>>> Also the IR here only maps easily to a specific set of hardware. For
>>>> example it doesn't really map well to some of my more programmable
>>>> devices that want to consume "programs" more like BPF. So I don't see
>>>> any universal IR being possible. At best you get a set of IRs
>>>> dependent on the underlying architecture.
>>>
>>> Universal sounds too ambicious, I called this generic. Generic in the
>>> sense of trying to unify and provide as much common infrastructure as
>>> possible. We probably cannot escape having some specificities for some
>>> of the frontends we have, so we'll have to pass this specific info as
>>> decoration to the driver, but that's should not be an excuse to skip
>>> having something common for everyone IMO.
>>>
>>> Anyway I'd suggest you go have a look at the IR and see what you send
>>> incremental patches to improve what you cannot represent in a generic
>>> way.
>>>
>>
>> I looked at it and want to know what it does to performance. Also I
>> think ebpf is a better IR if we want an IR so why not lower it to ebpf
>> if we want the generic object.
> 
> So you indicate that my simple IR representation adds complexity, and
> you want to fix that by using a bytecode-based final representation
> instead? Sorry, that doesn't make sense to me.
> 

I'm going through two sides here either really double down and build
a generic IR which for me needs to be something like eBPF or build
the simplest possible IR.

>>>>> The tc-u32 parser is a bit complicated because of having the matching
>>>>> spread out in different rules through links, but John already came up a
>>>>> basic parser than should be placed in the frontend so everyone can
>>>>> improve it to generate more expressive ast.
>>>>>
>>>>> Note: I don't have access to ixgbe hardware, so I have validated this
>>>>> patchset by splicing main parts of the ixgbe backend jit code in simple
>>>>> debugging patches that I have here. Quite rudimentary but it has passed
>>>>> some basic tests, may still have gotten anything broken. Anyway, the
>>>>> main goal is to generate debate on this.
>>>>>
>>>>
>>>> Yep lets debate the general approach. I can deal with ixgbe details.
>>>> My reaction is this seems overkill I only see offloads popping up for
>>>> flower, u32, nft, and bpf at the moment. Flower doesn't need heavy
>>>> overhead due to its simplicity. u32 can be done with some basic helpers
>>>> for devices like ixgbe. I think nft can be done the same. bpf is its own
>>>> beast that doesn't map well to another IR its more or less its own IR
>>>> anyways.
>>>>
>>>> If we need/want a IR I would go for something instruction based
>>>> like ebpf over this.
>>>
>>> bpf is a final representation from a compiler perspective, it is
>>> bytecode, I don't think it is meant to be designed as an intermediate
>>> representation.
>>
>> Well its just a set of instructions and each driver writer would write
>> their own jit just the same as the AST. Also it would save me having to
>> write two jits one for NFT-IR and one for BPF so I would prefer to just
>> do one and use it as the generic IR its not so bad to convert u32 into
>> it for example. If we need two ebpf-jit, nft-jit I mind as well have
>> three tc-jit, nft-jit, and ebpf-jit and optimize each one for their
>> targeted front-ends.
> 
> I told during Netconf that you can use this IR infrastructure to
> generate bpf on the backend side. That would basically allow jitting
> in software every frontend, but that is a different front. Anyway I
> don't understand you want to use bpf as IR. That's a final
> representation that is quite complex because of quite wide instruction
> set. This is just adding more complexity which is exactly one of your
> concerns.
> 

Yeah I'm having two arguments one if I have a IR maybe I should really
try to build the generic IR which means eBPF and absorb all the
complexity that involves because I need the eBPF-jit anyways I'm going
to hit this problem regardless of IR or no IR.

The second line of thought is well if we want an IR and admit it wont
cover some frontends anyways lets just make it as simple and dumb
as possible.

So.. how to make progress. Can you give me a couple days to go over
your patches with some more of this background thread in my head.
I would like to implement at least one more of my devices backends
so when I push generic code it at least has two users so that it
has some point.

Then let me see if I can lift my code into some core library. And
finally I'll try to see if I can be convinced if this AST thing is
really necessary. Maybe I just need to do that exercise to see why
this AST is needed because I don't see it now.

Also why are you after this? Do you have an nft implementation on
top of it? Can we see those patches? Are you just after abstraction
for abstractions sake? Without multiple frontends its hard to justify
building these abstractions.

.John

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

* Re: [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion
  2016-02-26 11:47   ` Pablo Neira Ayuso
  2016-02-26 15:42     ` John Fastabend
@ 2016-02-26 17:26     ` David Miller
  1 sibling, 0 replies; 19+ messages in thread
From: David Miller @ 2016-02-26 17:26 UTC (permalink / raw)
  To: pablo; +Cc: john.fastabend, netdev, jiri, horms

From: Pablo Neira Ayuso <pablo@netfilter.org>
Date: Fri, 26 Feb 2016 12:47:14 +0100

> The frontend parser should be generic to everyone, it should be
> placed in the core, so everyone will take care of improving it.

Generic unfortunately means information loss and lots of wasted work.

And this is one of John's points I think.

If we have an existing abstraction that fits directly into what the
hardware can do, such as the u32 classifiers, translating this back
and forth into an intermediate representation is going to be at best
wasted work and sometimes missing cases that can be loaded into hardware.

I really see zero value in "generic" intermediate languages for this
kind of stuff.

I don't want to have to translate a set of u32 rules into some other
format if u32 is what lots of hardware can do directly already.

Pablo, you're coming at the from another angle, you're starting from
the perspective of nftables which has this nice abstraction and IR.
But no hardware directly offloads nftables IR.  So as an offload
strategy nftables needs this IR to hardware translation layer.

But u32 and others absolutely will not, and we should not force them
to.

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

* Re: [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation
  2016-02-26 14:53       ` John Fastabend
  2016-02-26 16:02         ` Pablo Neira Ayuso
@ 2016-02-26 17:34         ` David Miller
  1 sibling, 0 replies; 19+ messages in thread
From: David Miller @ 2016-02-26 17:34 UTC (permalink / raw)
  To: john.fastabend; +Cc: pablo, netdev, jiri, horms

From: John Fastabend <john.fastabend@gmail.com>
Date: Fri, 26 Feb 2016 06:53:11 -0800

> sure but its easier for me to just consume u32 at the moment. And I
> am concerned about performance overhead with this IR. I'm going to need
> to have a performant solution eventually and I don't like converting
> into one IR only to go into another.

+1

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

* Re: [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation
  2016-02-26 16:02         ` Pablo Neira Ayuso
  2016-02-26 16:34           ` John Fastabend
@ 2016-02-26 17:38           ` David Miller
  1 sibling, 0 replies; 19+ messages in thread
From: David Miller @ 2016-02-26 17:38 UTC (permalink / raw)
  To: pablo; +Cc: john.fastabend, netdev, jiri, horms

From: Pablo Neira Ayuso <pablo@netfilter.org>
Date: Fri, 26 Feb 2016 17:02:22 +0100

> Just because you want to early microoptimize this thing by saving a
> little of extra code that runs from the control plane path.

I don't think that's what he is doing at all.

We have classes of classifier etc. offloads that need IR, and we have
those that don't.

There is nothing wrong with making this distinction and making our
design based upon that observation.

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

* Re: [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion
  2016-02-26 16:19       ` Pablo Neira Ayuso
  2016-02-26 16:46         ` John Fastabend
@ 2016-02-26 17:40         ` David Miller
  2016-02-26 18:53         ` Alexei Starovoitov
  2 siblings, 0 replies; 19+ messages in thread
From: David Miller @ 2016-02-26 17:40 UTC (permalink / raw)
  To: pablo; +Cc: john.fastabend, netdev, jiri, horms

From: Pablo Neira Ayuso <pablo@netfilter.org>
Date: Fri, 26 Feb 2016 17:19:48 +0100

> I see no reason to have as many hooks as frontends to start with. If
> you find limitations with the IR that are unfixable for any of the
> existing frontends in the future, then we can add direct hook as final
> solution.

I see no problem with adding many hooks, one for each class of things
we'd like to offload.  Stuff neading IR vs. stuff that does not.

And IR is "unfixable" for the latter case in that it will always be by
definition pure overhead if the cards can do this stuff directly, and
they can.

I do not encourage anything, in any way whatsoever, to try and genericize
all of this stuff into a generic framework.  That is wasted work in my
opinion.

You find an IR useful for nftables offloads, great!  But I do not see it
being useful nor desirable for u32, flower, et al.

Thanks.

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

* Re: [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion
  2016-02-26 16:19       ` Pablo Neira Ayuso
  2016-02-26 16:46         ` John Fastabend
  2016-02-26 17:40         ` David Miller
@ 2016-02-26 18:53         ` Alexei Starovoitov
  2 siblings, 0 replies; 19+ messages in thread
From: Alexei Starovoitov @ 2016-02-26 18:53 UTC (permalink / raw)
  To: Pablo Neira Ayuso; +Cc: John Fastabend, netdev, davem, jiri, horms

On Fri, Feb 26, 2016 at 05:19:48PM +0100, Pablo Neira Ayuso wrote:
> 
> Good, I'm all for reaching those numbers, we can optimize the generic
> IR if this ever becomes the bottleneck.

The 'generic IR' got mentioned hundred times in this thread,
but what was proposed is not generic. It doesn't even
fully fit u32. Here is why:

> This structure contains a protocol description (defined by struct
> net_ir_proto_desc) that is the initial node of the protocol graph that
> describes the protocol translation. This initial node starts from lower
> supported layer as base (eg. link-layer) then describing the upper
> protocols up to the transport protocols through the following structure:
> 
>  struct net_ir_proto_desc {
>        enum net_ir_payload_bases               base;
>        u32                                     protonum;
>        int                                     (*jit)(struct net_ir_jit_ctx *ctx,
>                                                       const struct net_ir_expr *expr,
>                                                       void *data);
>        const struct net_ir_proto_desc          *protocols[];
>  };

The above representation has builtin concept of protocols, whereas
u32 is protocol agnostic and fits this particular intel nic better.

>  struct net_ir_jit_desc {
>        enum net_ir_payload_bases               base;
>        const struct net_ir_proto_desc          *proto_desc;
>        int                                     (*verdict)(struct net_ir_jit_ctx *ctx,
>                                                           enum net_ir_stmt_verdict verdict,
>                                                           void *data);
>  };

imo the above is a misuse of JIT abbreviation.
Typically JIT means compiling to machine code that can be executed
directly. Converting one representation to another is not really JIT.
Also IR stands for _intermediate_ representation. It is a transitional
state when compiler converts high level language into machine code.
In this case the proposed format is protocol specific syntax tree,
so probably should be called as such.

imo the HW guys should be free to pick whatever representation
we have today and offload it. If u32 is convenient and applies
to HW architecture better, the driver should take u32 tree and
map it to HW (which is what was done already). When/If another
HW comes along with similar HW architecture we can generalize
and reuse u32->ixgbe code. And it should be done by developers
who actually have the HW and can test on it. Trying to 'generalize'
u32->ixgbe code without 2nd HW is not going to be successful.

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

end of thread, other threads:[~2016-02-26 18:53 UTC | newest]

Thread overview: 19+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2016-02-25 17:37 [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion Pablo Neira Ayuso
2016-02-25 17:37 ` [PATCH RFC 1/3] net: ixgbe: add struct igxbe_filter Pablo Neira Ayuso
2016-02-25 17:37 ` [PATCH RFC 2/3] net: intermediate representation for jit translation Pablo Neira Ayuso
2016-02-25 17:37 ` [PATCH RFC 3/3] net: convert tc_u32 to use the intermediate representation Pablo Neira Ayuso
2016-02-25 20:37   ` John Fastabend
2016-02-26 14:24     ` Pablo Neira Ayuso
2016-02-26 14:53       ` John Fastabend
2016-02-26 16:02         ` Pablo Neira Ayuso
2016-02-26 16:34           ` John Fastabend
2016-02-26 17:38           ` David Miller
2016-02-26 17:34         ` David Miller
2016-02-25 18:11 ` [PATCH RFC 0/3] intermediate representation for jit and cls_u32 conversion John Fastabend
2016-02-26 11:47   ` Pablo Neira Ayuso
2016-02-26 15:42     ` John Fastabend
2016-02-26 16:19       ` Pablo Neira Ayuso
2016-02-26 16:46         ` John Fastabend
2016-02-26 17:40         ` David Miller
2016-02-26 18:53         ` Alexei Starovoitov
2016-02-26 17:26     ` David Miller

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