netdev.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH net-next 00/14] devlink: code split and structured instance walk
@ 2023-01-04  4:16 Jakub Kicinski
  2023-01-04  4:16 ` [PATCH net-next 01/14] devlink: move code to a dedicated directory Jakub Kicinski
                   ` (13 more replies)
  0 siblings, 14 replies; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

Split devlink.c into a handful of files, trying to keep the "core"
code away from all the command-specific implementations.
The core code has been quite scattered until now. Going forward we can
consider using a source file per-subobject, I think that it's quite
beneficial to newcomers (based on relative ease with which folks
contribute to ethtool vs devlink). But this series doesn't split
everything out, yet - partially due to backporting concerns,
but mostly due to lack of time. Bulk of the netlink command
handling is left in a leftover.c file.

Introduce a context structure for dumps, and use it to store
the devlink instance ID of the last dumped devlink instance.
This means we don't have to restart the walk from 0 each time.

Finally - introduce a "structured walk". A centralized dump handler
in devlink/netlink.c which walks the devlink instances, deals with
refcounting/locking, simplifying the per-object implementations quite
a bit. Inspired by the ethtool code.

RFC: https://lore.kernel.org/all/20221215020155.1619839-1-kuba@kernel.org/

v1:
 - re-route the locking fix via net
 - rename basic.c -> leftover.c

Jakub Kicinski (14):
  devlink: move code to a dedicated directory
  devlink: split out core code
  devlink: split out netlink code
  netlink: add macro for checking dump ctx size
  devlink: use an explicit structure for dump context
  devlink: remove start variables from dumps
  devlink: drop the filter argument from devlinks_xa_find_get
  devlink: health: combine loops in dump
  devlink: restart dump based on devlink instance ids (simple)
  devlink: restart dump based on devlink instance ids (nested)
  devlink: restart dump based on devlink instance ids (function)
  devlink: uniformly take the devlink instance lock in the dump loop
  devlink: add by-instance dump infra
  devlink: convert remaining dumps to the by-instance scheme

 include/linux/netlink.h                    |    4 +
 net/Makefile                               |    1 +
 net/core/Makefile                          |    1 -
 net/devlink/Makefile                       |    3 +
 net/devlink/core.c                         |  345 +++++
 net/devlink/devl_internal.h                |  205 +++
 net/{core/devlink.c => devlink/leftover.c} | 1460 +++++---------------
 net/devlink/netlink.c                      |  241 ++++
 net/netfilter/nf_conntrack_netlink.c       |    2 +-
 9 files changed, 1157 insertions(+), 1105 deletions(-)
 create mode 100644 net/devlink/Makefile
 create mode 100644 net/devlink/core.c
 create mode 100644 net/devlink/devl_internal.h
 rename net/{core/devlink.c => devlink/leftover.c} (90%)
 create mode 100644 net/devlink/netlink.c

-- 
2.38.1


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

* [PATCH net-next 01/14] devlink: move code to a dedicated directory
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04  4:16 ` [PATCH net-next 02/14] devlink: split out core code Jakub Kicinski
                   ` (12 subsequent siblings)
  13 siblings, 0 replies; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

The devlink code is hard to navigate with 13kLoC in one file.
I really like the way Michal split the ethtool into per-command
files and core. It'd probably be too much to split it all up,
but we can at least separate the core parts out of the per-cmd
implementations and put it in a directory so that new commands
can be separate files.

Move the code, subsequent commit will do a partial split.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
v1: rename basic.c -> leftover.c
---
 net/Makefile                               | 1 +
 net/core/Makefile                          | 1 -
 net/devlink/Makefile                       | 3 +++
 net/{core/devlink.c => devlink/leftover.c} | 0
 4 files changed, 4 insertions(+), 1 deletion(-)
 create mode 100644 net/devlink/Makefile
 rename net/{core/devlink.c => devlink/leftover.c} (100%)

diff --git a/net/Makefile b/net/Makefile
index 6a62e5b27378..0914bea9c335 100644
--- a/net/Makefile
+++ b/net/Makefile
@@ -23,6 +23,7 @@ obj-$(CONFIG_BPFILTER)		+= bpfilter/
 obj-$(CONFIG_PACKET)		+= packet/
 obj-$(CONFIG_NET_KEY)		+= key/
 obj-$(CONFIG_BRIDGE)		+= bridge/
+obj-$(CONFIG_NET_DEVLINK)	+= devlink/
 obj-$(CONFIG_NET_DSA)		+= dsa/
 obj-$(CONFIG_ATALK)		+= appletalk/
 obj-$(CONFIG_X25)		+= x25/
diff --git a/net/core/Makefile b/net/core/Makefile
index 5857cec87b83..10edd66a8a37 100644
--- a/net/core/Makefile
+++ b/net/core/Makefile
@@ -33,7 +33,6 @@ obj-$(CONFIG_LWTUNNEL) += lwtunnel.o
 obj-$(CONFIG_LWTUNNEL_BPF) += lwt_bpf.o
 obj-$(CONFIG_DST_CACHE) += dst_cache.o
 obj-$(CONFIG_HWBM) += hwbm.o
-obj-$(CONFIG_NET_DEVLINK) += devlink.o
 obj-$(CONFIG_GRO_CELLS) += gro_cells.o
 obj-$(CONFIG_FAILOVER) += failover.o
 obj-$(CONFIG_NET_SOCK_MSG) += skmsg.o
diff --git a/net/devlink/Makefile b/net/devlink/Makefile
new file mode 100644
index 000000000000..3a60959f71ee
--- /dev/null
+++ b/net/devlink/Makefile
@@ -0,0 +1,3 @@
+# SPDX-License-Identifier: GPL-2.0
+
+obj-y := leftover.o
diff --git a/net/core/devlink.c b/net/devlink/leftover.c
similarity index 100%
rename from net/core/devlink.c
rename to net/devlink/leftover.c
-- 
2.38.1


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

* [PATCH net-next 02/14] devlink: split out core code
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
  2023-01-04  4:16 ` [PATCH net-next 01/14] devlink: move code to a dedicated directory Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04  9:50   ` Jiri Pirko
  2023-01-04  4:16 ` [PATCH net-next 03/14] devlink: split out netlink code Jakub Kicinski
                   ` (11 subsequent siblings)
  13 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

Move core code into a separate file. It's spread around the main
file which makes refactoring and figuring out how devlink works
harder.

Move the xarray, all the most core devlink instance code out like
locking, ref counting, alloc, register, etc. Leave port stuff in
basic.c, if we want to move port code it'd probably be to its own file.

Rename devlink_netdevice_event() to make it clear that it only touches
ports (that's the only change which isn't a pure code move).

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/Makefile        |   2 +-
 net/devlink/core.c          | 347 ++++++++++++++++++++++++++++
 net/devlink/devl_internal.h | 117 ++++++++++
 net/devlink/leftover.c      | 444 +-----------------------------------
 4 files changed, 476 insertions(+), 434 deletions(-)
 create mode 100644 net/devlink/core.c
 create mode 100644 net/devlink/devl_internal.h

diff --git a/net/devlink/Makefile b/net/devlink/Makefile
index 3a60959f71ee..aff7da844e5d 100644
--- a/net/devlink/Makefile
+++ b/net/devlink/Makefile
@@ -1,3 +1,3 @@
 # SPDX-License-Identifier: GPL-2.0
 
-obj-y := leftover.o
+obj-y := leftover.o core.o
diff --git a/net/devlink/core.c b/net/devlink/core.c
new file mode 100644
index 000000000000..c084eafa17fb
--- /dev/null
+++ b/net/devlink/core.c
@@ -0,0 +1,347 @@
+// SPDX-License-Identifier: GPL-2.0-or-later
+/*
+ * Copyright (c) 2016 Mellanox Technologies. All rights reserved.
+ * Copyright (c) 2016 Jiri Pirko <jiri@mellanox.com>
+ */
+
+#include <net/genetlink.h>
+
+#include "devl_internal.h"
+
+DEFINE_XARRAY_FLAGS(devlinks, XA_FLAGS_ALLOC);
+
+void *devlink_priv(struct devlink *devlink)
+{
+	return &devlink->priv;
+}
+EXPORT_SYMBOL_GPL(devlink_priv);
+
+struct devlink *priv_to_devlink(void *priv)
+{
+	return container_of(priv, struct devlink, priv);
+}
+EXPORT_SYMBOL_GPL(priv_to_devlink);
+
+struct device *devlink_to_dev(const struct devlink *devlink)
+{
+	return devlink->dev;
+}
+EXPORT_SYMBOL_GPL(devlink_to_dev);
+
+struct net *devlink_net(const struct devlink *devlink)
+{
+	return read_pnet(&devlink->_net);
+}
+EXPORT_SYMBOL_GPL(devlink_net);
+
+void devl_assert_locked(struct devlink *devlink)
+{
+	lockdep_assert_held(&devlink->lock);
+}
+EXPORT_SYMBOL_GPL(devl_assert_locked);
+
+#ifdef CONFIG_LOCKDEP
+/* For use in conjunction with LOCKDEP only e.g. rcu_dereference_protected() */
+bool devl_lock_is_held(struct devlink *devlink)
+{
+	return lockdep_is_held(&devlink->lock);
+}
+EXPORT_SYMBOL_GPL(devl_lock_is_held);
+#endif
+
+void devl_lock(struct devlink *devlink)
+{
+	mutex_lock(&devlink->lock);
+}
+EXPORT_SYMBOL_GPL(devl_lock);
+
+int devl_trylock(struct devlink *devlink)
+{
+	return mutex_trylock(&devlink->lock);
+}
+EXPORT_SYMBOL_GPL(devl_trylock);
+
+void devl_unlock(struct devlink *devlink)
+{
+	mutex_unlock(&devlink->lock);
+}
+EXPORT_SYMBOL_GPL(devl_unlock);
+
+struct devlink *__must_check devlink_try_get(struct devlink *devlink)
+{
+	if (refcount_inc_not_zero(&devlink->refcount))
+		return devlink;
+	return NULL;
+}
+
+static void __devlink_put_rcu(struct rcu_head *head)
+{
+	struct devlink *devlink = container_of(head, struct devlink, rcu);
+
+	complete(&devlink->comp);
+}
+
+void devlink_put(struct devlink *devlink)
+{
+	if (refcount_dec_and_test(&devlink->refcount))
+		/* Make sure unregister operation that may await the completion
+		 * is unblocked only after all users are after the end of
+		 * RCU grace period.
+		 */
+		call_rcu(&devlink->rcu, __devlink_put_rcu);
+}
+
+static struct devlink *
+devlinks_xa_find_get(struct net *net, unsigned long *indexp, xa_mark_t filter,
+		     void * (*xa_find_fn)(struct xarray *, unsigned long *,
+					  unsigned long, xa_mark_t))
+{
+	struct devlink *devlink;
+
+	rcu_read_lock();
+retry:
+	devlink = xa_find_fn(&devlinks, indexp, ULONG_MAX, DEVLINK_REGISTERED);
+	if (!devlink)
+		goto unlock;
+
+	/* In case devlink_unregister() was already called and "unregistering"
+	 * mark was set, do not allow to get a devlink reference here.
+	 * This prevents live-lock of devlink_unregister() wait for completion.
+	 */
+	if (xa_get_mark(&devlinks, *indexp, DEVLINK_UNREGISTERING))
+		goto retry;
+
+	/* For a possible retry, the xa_find_after() should be always used */
+	xa_find_fn = xa_find_after;
+	if (!devlink_try_get(devlink))
+		goto retry;
+	if (!net_eq(devlink_net(devlink), net)) {
+		devlink_put(devlink);
+		goto retry;
+	}
+unlock:
+	rcu_read_unlock();
+	return devlink;
+}
+
+struct devlink *
+devlinks_xa_find_get_first(struct net *net, unsigned long *indexp,
+			   xa_mark_t filter)
+{
+	return devlinks_xa_find_get(net, indexp, filter, xa_find);
+}
+
+struct devlink *
+devlinks_xa_find_get_next(struct net *net, unsigned long *indexp,
+			  xa_mark_t filter)
+{
+	return devlinks_xa_find_get(net, indexp, filter, xa_find_after);
+}
+
+/**
+ *	devlink_set_features - Set devlink supported features
+ *
+ *	@devlink: devlink
+ *	@features: devlink support features
+ *
+ *	This interface allows us to set reload ops separatelly from
+ *	the devlink_alloc.
+ */
+void devlink_set_features(struct devlink *devlink, u64 features)
+{
+	ASSERT_DEVLINK_NOT_REGISTERED(devlink);
+
+	WARN_ON(features & DEVLINK_F_RELOAD &&
+		!devlink_reload_supported(devlink->ops));
+	devlink->features = features;
+}
+EXPORT_SYMBOL_GPL(devlink_set_features);
+
+/**
+ *	devlink_register - Register devlink instance
+ *
+ *	@devlink: devlink
+ */
+void devlink_register(struct devlink *devlink)
+{
+	ASSERT_DEVLINK_NOT_REGISTERED(devlink);
+	/* Make sure that we are in .probe() routine */
+
+	xa_set_mark(&devlinks, devlink->index, DEVLINK_REGISTERED);
+	devlink_notify_register(devlink);
+}
+EXPORT_SYMBOL_GPL(devlink_register);
+
+/**
+ *	devlink_unregister - Unregister devlink instance
+ *
+ *	@devlink: devlink
+ */
+void devlink_unregister(struct devlink *devlink)
+{
+	ASSERT_DEVLINK_REGISTERED(devlink);
+	/* Make sure that we are in .remove() routine */
+
+	xa_set_mark(&devlinks, devlink->index, DEVLINK_UNREGISTERING);
+	devlink_put(devlink);
+	wait_for_completion(&devlink->comp);
+
+	devlink_notify_unregister(devlink);
+	xa_clear_mark(&devlinks, devlink->index, DEVLINK_REGISTERED);
+	xa_clear_mark(&devlinks, devlink->index, DEVLINK_UNREGISTERING);
+}
+EXPORT_SYMBOL_GPL(devlink_unregister);
+
+/**
+ *	devlink_alloc_ns - Allocate new devlink instance resources
+ *	in specific namespace
+ *
+ *	@ops: ops
+ *	@priv_size: size of user private data
+ *	@net: net namespace
+ *	@dev: parent device
+ *
+ *	Allocate new devlink instance resources, including devlink index
+ *	and name.
+ */
+struct devlink *devlink_alloc_ns(const struct devlink_ops *ops,
+				 size_t priv_size, struct net *net,
+				 struct device *dev)
+{
+	struct devlink *devlink;
+	static u32 last_id;
+	int ret;
+
+	WARN_ON(!ops || !dev);
+	if (!devlink_reload_actions_valid(ops))
+		return NULL;
+
+	devlink = kzalloc(sizeof(*devlink) + priv_size, GFP_KERNEL);
+	if (!devlink)
+		return NULL;
+
+	ret = xa_alloc_cyclic(&devlinks, &devlink->index, devlink, xa_limit_31b,
+			      &last_id, GFP_KERNEL);
+	if (ret < 0)
+		goto err_xa_alloc;
+
+	devlink->netdevice_nb.notifier_call = devlink_port_netdevice_event;
+	ret = register_netdevice_notifier_net(net, &devlink->netdevice_nb);
+	if (ret)
+		goto err_register_netdevice_notifier;
+
+	devlink->dev = dev;
+	devlink->ops = ops;
+	xa_init_flags(&devlink->ports, XA_FLAGS_ALLOC);
+	xa_init_flags(&devlink->snapshot_ids, XA_FLAGS_ALLOC);
+	write_pnet(&devlink->_net, net);
+	INIT_LIST_HEAD(&devlink->rate_list);
+	INIT_LIST_HEAD(&devlink->linecard_list);
+	INIT_LIST_HEAD(&devlink->sb_list);
+	INIT_LIST_HEAD_RCU(&devlink->dpipe_table_list);
+	INIT_LIST_HEAD(&devlink->resource_list);
+	INIT_LIST_HEAD(&devlink->param_list);
+	INIT_LIST_HEAD(&devlink->region_list);
+	INIT_LIST_HEAD(&devlink->reporter_list);
+	INIT_LIST_HEAD(&devlink->trap_list);
+	INIT_LIST_HEAD(&devlink->trap_group_list);
+	INIT_LIST_HEAD(&devlink->trap_policer_list);
+	lockdep_register_key(&devlink->lock_key);
+	mutex_init(&devlink->lock);
+	lockdep_set_class(&devlink->lock, &devlink->lock_key);
+	mutex_init(&devlink->reporters_lock);
+	mutex_init(&devlink->linecards_lock);
+	refcount_set(&devlink->refcount, 1);
+	init_completion(&devlink->comp);
+
+	return devlink;
+
+err_register_netdevice_notifier:
+	xa_erase(&devlinks, devlink->index);
+err_xa_alloc:
+	kfree(devlink);
+	return NULL;
+}
+EXPORT_SYMBOL_GPL(devlink_alloc_ns);
+
+/**
+ *	devlink_free - Free devlink instance resources
+ *
+ *	@devlink: devlink
+ */
+void devlink_free(struct devlink *devlink)
+{
+	ASSERT_DEVLINK_NOT_REGISTERED(devlink);
+
+	mutex_destroy(&devlink->linecards_lock);
+	mutex_destroy(&devlink->reporters_lock);
+	mutex_destroy(&devlink->lock);
+	lockdep_unregister_key(&devlink->lock_key);
+	WARN_ON(!list_empty(&devlink->trap_policer_list));
+	WARN_ON(!list_empty(&devlink->trap_group_list));
+	WARN_ON(!list_empty(&devlink->trap_list));
+	WARN_ON(!list_empty(&devlink->reporter_list));
+	WARN_ON(!list_empty(&devlink->region_list));
+	WARN_ON(!list_empty(&devlink->param_list));
+	WARN_ON(!list_empty(&devlink->resource_list));
+	WARN_ON(!list_empty(&devlink->dpipe_table_list));
+	WARN_ON(!list_empty(&devlink->sb_list));
+	WARN_ON(!list_empty(&devlink->rate_list));
+	WARN_ON(!list_empty(&devlink->linecard_list));
+	WARN_ON(!xa_empty(&devlink->ports));
+
+	xa_destroy(&devlink->snapshot_ids);
+	xa_destroy(&devlink->ports);
+
+	WARN_ON_ONCE(unregister_netdevice_notifier_net(devlink_net(devlink),
+						       &devlink->netdevice_nb));
+
+	xa_erase(&devlinks, devlink->index);
+
+	kfree(devlink);
+}
+EXPORT_SYMBOL_GPL(devlink_free);
+
+static void __net_exit devlink_pernet_pre_exit(struct net *net)
+{
+	struct devlink *devlink;
+	u32 actions_performed;
+	unsigned long index;
+	int err;
+
+	/* In case network namespace is getting destroyed, reload
+	 * all devlink instances from this namespace into init_net.
+	 */
+	devlinks_xa_for_each_registered_get(net, index, devlink) {
+		WARN_ON(!(devlink->features & DEVLINK_F_RELOAD));
+		mutex_lock(&devlink->lock);
+		err = devlink_reload(devlink, &init_net,
+				     DEVLINK_RELOAD_ACTION_DRIVER_REINIT,
+				     DEVLINK_RELOAD_LIMIT_UNSPEC,
+				     &actions_performed, NULL);
+		mutex_unlock(&devlink->lock);
+		if (err && err != -EOPNOTSUPP)
+			pr_warn("Failed to reload devlink instance into init_net\n");
+		devlink_put(devlink);
+	}
+}
+
+static struct pernet_operations devlink_pernet_ops __net_initdata = {
+	.pre_exit = devlink_pernet_pre_exit,
+};
+
+static int __init devlink_init(void)
+{
+	int err;
+
+	err = genl_register_family(&devlink_nl_family);
+	if (err)
+		goto out;
+	err = register_pernet_subsys(&devlink_pernet_ops);
+
+out:
+	WARN_ON(err);
+	return err;
+}
+
+subsys_initcall(devlink_init);
diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
new file mode 100644
index 000000000000..0cca1f92d733
--- /dev/null
+++ b/net/devlink/devl_internal.h
@@ -0,0 +1,117 @@
+// SPDX-License-Identifier: GPL-2.0-or-later
+/* Copyright (c) 2016 Mellanox Technologies. All rights reserved.
+ * Copyright (c) 2016 Jiri Pirko <jiri@mellanox.com>
+ */
+
+#include <linux/mutex.h>
+#include <linux/netdevice.h>
+#include <linux/notifier.h>
+#include <linux/types.h>
+#include <linux/xarray.h>
+#include <net/devlink.h>
+#include <net/net_namespace.h>
+
+#define DEVLINK_REGISTERED XA_MARK_1
+#define DEVLINK_UNREGISTERING XA_MARK_2
+
+#define DEVLINK_RELOAD_STATS_ARRAY_SIZE \
+	(__DEVLINK_RELOAD_LIMIT_MAX * __DEVLINK_RELOAD_ACTION_MAX)
+
+struct devlink_dev_stats {
+	u32 reload_stats[DEVLINK_RELOAD_STATS_ARRAY_SIZE];
+	u32 remote_reload_stats[DEVLINK_RELOAD_STATS_ARRAY_SIZE];
+};
+
+struct devlink {
+	u32 index;
+	struct xarray ports;
+	struct list_head rate_list;
+	struct list_head sb_list;
+	struct list_head dpipe_table_list;
+	struct list_head resource_list;
+	struct list_head param_list;
+	struct list_head region_list;
+	struct list_head reporter_list;
+	struct mutex reporters_lock; /* protects reporter_list */
+	struct devlink_dpipe_headers *dpipe_headers;
+	struct list_head trap_list;
+	struct list_head trap_group_list;
+	struct list_head trap_policer_list;
+	struct list_head linecard_list;
+	struct mutex linecards_lock; /* protects linecard_list */
+	const struct devlink_ops *ops;
+	u64 features;
+	struct xarray snapshot_ids;
+	struct devlink_dev_stats stats;
+	struct device *dev;
+	possible_net_t _net;
+	/* Serializes access to devlink instance specific objects such as
+	 * port, sb, dpipe, resource, params, region, traps and more.
+	 */
+	struct mutex lock;
+	struct lock_class_key lock_key;
+	u8 reload_failed:1;
+	refcount_t refcount;
+	struct completion comp;
+	struct rcu_head rcu;
+	struct notifier_block netdevice_nb;
+	char priv[] __aligned(NETDEV_ALIGN);
+};
+
+extern struct xarray devlinks;
+extern struct genl_family devlink_nl_family;
+
+/* devlink instances are open to the access from the user space after
+ * devlink_register() call. Such logical barrier allows us to have certain
+ * expectations related to locking.
+ *
+ * Before *_register() - we are in initialization stage and no parallel
+ * access possible to the devlink instance. All drivers perform that phase
+ * by implicitly holding device_lock.
+ *
+ * After *_register() - users and driver can access devlink instance at
+ * the same time.
+ */
+#define ASSERT_DEVLINK_REGISTERED(d)                                           \
+	WARN_ON_ONCE(!xa_get_mark(&devlinks, (d)->index, DEVLINK_REGISTERED))
+#define ASSERT_DEVLINK_NOT_REGISTERED(d)                                       \
+	WARN_ON_ONCE(xa_get_mark(&devlinks, (d)->index, DEVLINK_REGISTERED))
+
+/* Iterate over devlink pointers which were possible to get reference to.
+ * devlink_put() needs to be called for each iterated devlink pointer
+ * in loop body in order to release the reference.
+ */
+#define devlinks_xa_for_each_get(net, index, devlink, filter)		\
+	for (index = 0,							\
+	     devlink = devlinks_xa_find_get_first(net, &index, filter);	\
+	     devlink; devlink = devlinks_xa_find_get_next(net, &index, filter))
+
+#define devlinks_xa_for_each_registered_get(net, index, devlink)	\
+	devlinks_xa_for_each_get(net, index, devlink, DEVLINK_REGISTERED)
+
+struct devlink *
+devlinks_xa_find_get_first(struct net *net, unsigned long *indexp,
+			   xa_mark_t filter);
+struct devlink *
+devlinks_xa_find_get_next(struct net *net, unsigned long *indexp,
+			  xa_mark_t filter);
+
+/* Netlink */
+void devlink_notify_unregister(struct devlink *devlink);
+void devlink_notify_register(struct devlink *devlink);
+
+/* Ports */
+int devlink_port_netdevice_event(struct notifier_block *nb,
+				 unsigned long event, void *ptr);
+
+/* Reload */
+bool devlink_reload_actions_valid(const struct devlink_ops *ops);
+int devlink_reload(struct devlink *devlink, struct net *dest_net,
+		   enum devlink_reload_action action,
+		   enum devlink_reload_limit limit,
+		   u32 *actions_performed, struct netlink_ext_ack *extack);
+
+static inline bool devlink_reload_supported(const struct devlink_ops *ops)
+{
+	return ops->reload_down && ops->reload_up;
+}
diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
index 032d6d0a5ce6..05f2e75b3a03 100644
--- a/net/devlink/leftover.c
+++ b/net/devlink/leftover.c
@@ -31,52 +31,7 @@
 #define CREATE_TRACE_POINTS
 #include <trace/events/devlink.h>
 
-#define DEVLINK_RELOAD_STATS_ARRAY_SIZE \
-	(__DEVLINK_RELOAD_LIMIT_MAX * __DEVLINK_RELOAD_ACTION_MAX)
-
-struct devlink_dev_stats {
-	u32 reload_stats[DEVLINK_RELOAD_STATS_ARRAY_SIZE];
-	u32 remote_reload_stats[DEVLINK_RELOAD_STATS_ARRAY_SIZE];
-};
-
-struct devlink {
-	u32 index;
-	struct xarray ports;
-	struct list_head rate_list;
-	struct list_head sb_list;
-	struct list_head dpipe_table_list;
-	struct list_head resource_list;
-	struct list_head param_list;
-	struct list_head region_list;
-	struct list_head reporter_list;
-	struct mutex reporters_lock; /* protects reporter_list */
-	struct devlink_dpipe_headers *dpipe_headers;
-	struct list_head trap_list;
-	struct list_head trap_group_list;
-	struct list_head trap_policer_list;
-	struct list_head linecard_list;
-	struct mutex linecards_lock; /* protects linecard_list */
-	const struct devlink_ops *ops;
-	u64 features;
-	struct xarray snapshot_ids;
-	struct devlink_dev_stats stats;
-	struct device *dev;
-	possible_net_t _net;
-	/* Serializes access to devlink instance specific objects such as
-	 * port, sb, dpipe, resource, params, region, traps and more.
-	 */
-	struct mutex lock;
-	struct lock_class_key lock_key;
-	u8 reload_failed:1;
-	refcount_t refcount;
-	struct completion comp;
-	struct rcu_head rcu;
-	struct notifier_block netdevice_nb;
-	char priv[] __aligned(NETDEV_ALIGN);
-};
-
-struct devlink_linecard_ops;
-struct devlink_linecard_type;
+#include "devl_internal.h"
 
 struct devlink_linecard {
 	struct list_head list;
@@ -122,24 +77,6 @@ struct devlink_resource {
 	void *occ_get_priv;
 };
 
-void *devlink_priv(struct devlink *devlink)
-{
-	return &devlink->priv;
-}
-EXPORT_SYMBOL_GPL(devlink_priv);
-
-struct devlink *priv_to_devlink(void *priv)
-{
-	return container_of(priv, struct devlink, priv);
-}
-EXPORT_SYMBOL_GPL(priv_to_devlink);
-
-struct device *devlink_to_dev(const struct devlink *devlink)
-{
-	return devlink->dev;
-}
-EXPORT_SYMBOL_GPL(devlink_to_dev);
-
 static struct devlink_dpipe_field devlink_dpipe_fields_ethernet[] = {
 	{
 		.name = "destination mac",
@@ -211,148 +148,6 @@ static const struct nla_policy devlink_selftest_nl_policy[DEVLINK_ATTR_SELFTEST_
 	[DEVLINK_ATTR_SELFTEST_ID_FLASH] = { .type = NLA_FLAG },
 };
 
-static DEFINE_XARRAY_FLAGS(devlinks, XA_FLAGS_ALLOC);
-#define DEVLINK_REGISTERED XA_MARK_1
-#define DEVLINK_UNREGISTERING XA_MARK_2
-
-/* devlink instances are open to the access from the user space after
- * devlink_register() call. Such logical barrier allows us to have certain
- * expectations related to locking.
- *
- * Before *_register() - we are in initialization stage and no parallel
- * access possible to the devlink instance. All drivers perform that phase
- * by implicitly holding device_lock.
- *
- * After *_register() - users and driver can access devlink instance at
- * the same time.
- */
-#define ASSERT_DEVLINK_REGISTERED(d)                                           \
-	WARN_ON_ONCE(!xa_get_mark(&devlinks, (d)->index, DEVLINK_REGISTERED))
-#define ASSERT_DEVLINK_NOT_REGISTERED(d)                                       \
-	WARN_ON_ONCE(xa_get_mark(&devlinks, (d)->index, DEVLINK_REGISTERED))
-
-struct net *devlink_net(const struct devlink *devlink)
-{
-	return read_pnet(&devlink->_net);
-}
-EXPORT_SYMBOL_GPL(devlink_net);
-
-static void __devlink_put_rcu(struct rcu_head *head)
-{
-	struct devlink *devlink = container_of(head, struct devlink, rcu);
-
-	complete(&devlink->comp);
-}
-
-void devlink_put(struct devlink *devlink)
-{
-	if (refcount_dec_and_test(&devlink->refcount))
-		/* Make sure unregister operation that may await the completion
-		 * is unblocked only after all users are after the end of
-		 * RCU grace period.
-		 */
-		call_rcu(&devlink->rcu, __devlink_put_rcu);
-}
-
-struct devlink *__must_check devlink_try_get(struct devlink *devlink)
-{
-	if (refcount_inc_not_zero(&devlink->refcount))
-		return devlink;
-	return NULL;
-}
-
-void devl_assert_locked(struct devlink *devlink)
-{
-	lockdep_assert_held(&devlink->lock);
-}
-EXPORT_SYMBOL_GPL(devl_assert_locked);
-
-#ifdef CONFIG_LOCKDEP
-/* For use in conjunction with LOCKDEP only e.g. rcu_dereference_protected() */
-bool devl_lock_is_held(struct devlink *devlink)
-{
-	return lockdep_is_held(&devlink->lock);
-}
-EXPORT_SYMBOL_GPL(devl_lock_is_held);
-#endif
-
-void devl_lock(struct devlink *devlink)
-{
-	mutex_lock(&devlink->lock);
-}
-EXPORT_SYMBOL_GPL(devl_lock);
-
-int devl_trylock(struct devlink *devlink)
-{
-	return mutex_trylock(&devlink->lock);
-}
-EXPORT_SYMBOL_GPL(devl_trylock);
-
-void devl_unlock(struct devlink *devlink)
-{
-	mutex_unlock(&devlink->lock);
-}
-EXPORT_SYMBOL_GPL(devl_unlock);
-
-static struct devlink *
-devlinks_xa_find_get(struct net *net, unsigned long *indexp, xa_mark_t filter,
-		     void * (*xa_find_fn)(struct xarray *, unsigned long *,
-					  unsigned long, xa_mark_t))
-{
-	struct devlink *devlink;
-
-	rcu_read_lock();
-retry:
-	devlink = xa_find_fn(&devlinks, indexp, ULONG_MAX, DEVLINK_REGISTERED);
-	if (!devlink)
-		goto unlock;
-
-	/* In case devlink_unregister() was already called and "unregistering"
-	 * mark was set, do not allow to get a devlink reference here.
-	 * This prevents live-lock of devlink_unregister() wait for completion.
-	 */
-	if (xa_get_mark(&devlinks, *indexp, DEVLINK_UNREGISTERING))
-		goto retry;
-
-	/* For a possible retry, the xa_find_after() should be always used */
-	xa_find_fn = xa_find_after;
-	if (!devlink_try_get(devlink))
-		goto retry;
-	if (!net_eq(devlink_net(devlink), net)) {
-		devlink_put(devlink);
-		goto retry;
-	}
-unlock:
-	rcu_read_unlock();
-	return devlink;
-}
-
-static struct devlink *devlinks_xa_find_get_first(struct net *net,
-						  unsigned long *indexp,
-						  xa_mark_t filter)
-{
-	return devlinks_xa_find_get(net, indexp, filter, xa_find);
-}
-
-static struct devlink *devlinks_xa_find_get_next(struct net *net,
-						 unsigned long *indexp,
-						 xa_mark_t filter)
-{
-	return devlinks_xa_find_get(net, indexp, filter, xa_find_after);
-}
-
-/* Iterate over devlink pointers which were possible to get reference to.
- * devlink_put() needs to be called for each iterated devlink pointer
- * in loop body in order to release the reference.
- */
-#define devlinks_xa_for_each_get(net, index, devlink, filter)			\
-	for (index = 0,								\
-	     devlink = devlinks_xa_find_get_first(net, &index, filter);		\
-	     devlink; devlink = devlinks_xa_find_get_next(net, &index, filter))
-
-#define devlinks_xa_for_each_registered_get(net, index, devlink)		\
-	devlinks_xa_for_each_get(net, index, devlink, DEVLINK_REGISTERED)
-
 static struct devlink *devlink_get_from_attrs(struct net *net,
 					      struct nlattr **attrs)
 {
@@ -917,8 +712,6 @@ static void devlink_nl_post_doit(const struct genl_split_ops *ops,
 	devlink_put(devlink);
 }
 
-static struct genl_family devlink_nl_family;
-
 enum devlink_multicast_groups {
 	DEVLINK_MCGRP_CONFIG,
 };
@@ -4653,11 +4446,6 @@ static void devlink_ns_change_notify(struct devlink *devlink,
 		devlink_notify(devlink, DEVLINK_CMD_DEL);
 }
 
-static bool devlink_reload_supported(const struct devlink_ops *ops)
-{
-	return ops->reload_down && ops->reload_up;
-}
-
 static void devlink_reload_failed_set(struct devlink *devlink,
 				      bool reload_failed)
 {
@@ -4725,9 +4513,10 @@ void devlink_remote_reload_actions_performed(struct devlink *devlink,
 }
 EXPORT_SYMBOL_GPL(devlink_remote_reload_actions_performed);
 
-static int devlink_reload(struct devlink *devlink, struct net *dest_net,
-			  enum devlink_reload_action action, enum devlink_reload_limit limit,
-			  u32 *actions_performed, struct netlink_ext_ack *extack)
+int devlink_reload(struct devlink *devlink, struct net *dest_net,
+		   enum devlink_reload_action action,
+		   enum devlink_reload_limit limit,
+		   u32 *actions_performed, struct netlink_ext_ack *extack)
 {
 	u32 remote_reload_stats[DEVLINK_RELOAD_STATS_ARRAY_SIZE];
 	struct net *curr_net;
@@ -9877,7 +9666,7 @@ static const struct genl_small_ops devlink_nl_ops[] = {
 	},
 };
 
-static struct genl_family devlink_nl_family __ro_after_init = {
+struct genl_family devlink_nl_family __ro_after_init = {
 	.name		= DEVLINK_GENL_NAME,
 	.version	= DEVLINK_GENL_VERSION,
 	.maxattr	= DEVLINK_ATTR_MAX,
@@ -9894,7 +9683,7 @@ static struct genl_family devlink_nl_family __ro_after_init = {
 	.n_mcgrps	= ARRAY_SIZE(devlink_nl_mcgrps),
 };
 
-static bool devlink_reload_actions_valid(const struct devlink_ops *ops)
+bool devlink_reload_actions_valid(const struct devlink_ops *ops)
 {
 	const struct devlink_reload_combination *comb;
 	int i;
@@ -9923,100 +9712,6 @@ static bool devlink_reload_actions_valid(const struct devlink_ops *ops)
 	return true;
 }
 
-/**
- *	devlink_set_features - Set devlink supported features
- *
- *	@devlink: devlink
- *	@features: devlink support features
- *
- *	This interface allows us to set reload ops separatelly from
- *	the devlink_alloc.
- */
-void devlink_set_features(struct devlink *devlink, u64 features)
-{
-	ASSERT_DEVLINK_NOT_REGISTERED(devlink);
-
-	WARN_ON(features & DEVLINK_F_RELOAD &&
-		!devlink_reload_supported(devlink->ops));
-	devlink->features = features;
-}
-EXPORT_SYMBOL_GPL(devlink_set_features);
-
-static int devlink_netdevice_event(struct notifier_block *nb,
-				   unsigned long event, void *ptr);
-
-/**
- *	devlink_alloc_ns - Allocate new devlink instance resources
- *	in specific namespace
- *
- *	@ops: ops
- *	@priv_size: size of user private data
- *	@net: net namespace
- *	@dev: parent device
- *
- *	Allocate new devlink instance resources, including devlink index
- *	and name.
- */
-struct devlink *devlink_alloc_ns(const struct devlink_ops *ops,
-				 size_t priv_size, struct net *net,
-				 struct device *dev)
-{
-	struct devlink *devlink;
-	static u32 last_id;
-	int ret;
-
-	WARN_ON(!ops || !dev);
-	if (!devlink_reload_actions_valid(ops))
-		return NULL;
-
-	devlink = kzalloc(sizeof(*devlink) + priv_size, GFP_KERNEL);
-	if (!devlink)
-		return NULL;
-
-	ret = xa_alloc_cyclic(&devlinks, &devlink->index, devlink, xa_limit_31b,
-			      &last_id, GFP_KERNEL);
-	if (ret < 0)
-		goto err_xa_alloc;
-
-	devlink->netdevice_nb.notifier_call = devlink_netdevice_event;
-	ret = register_netdevice_notifier_net(net, &devlink->netdevice_nb);
-	if (ret)
-		goto err_register_netdevice_notifier;
-
-	devlink->dev = dev;
-	devlink->ops = ops;
-	xa_init_flags(&devlink->ports, XA_FLAGS_ALLOC);
-	xa_init_flags(&devlink->snapshot_ids, XA_FLAGS_ALLOC);
-	write_pnet(&devlink->_net, net);
-	INIT_LIST_HEAD(&devlink->rate_list);
-	INIT_LIST_HEAD(&devlink->linecard_list);
-	INIT_LIST_HEAD(&devlink->sb_list);
-	INIT_LIST_HEAD_RCU(&devlink->dpipe_table_list);
-	INIT_LIST_HEAD(&devlink->resource_list);
-	INIT_LIST_HEAD(&devlink->param_list);
-	INIT_LIST_HEAD(&devlink->region_list);
-	INIT_LIST_HEAD(&devlink->reporter_list);
-	INIT_LIST_HEAD(&devlink->trap_list);
-	INIT_LIST_HEAD(&devlink->trap_group_list);
-	INIT_LIST_HEAD(&devlink->trap_policer_list);
-	lockdep_register_key(&devlink->lock_key);
-	mutex_init(&devlink->lock);
-	lockdep_set_class(&devlink->lock, &devlink->lock_key);
-	mutex_init(&devlink->reporters_lock);
-	mutex_init(&devlink->linecards_lock);
-	refcount_set(&devlink->refcount, 1);
-	init_completion(&devlink->comp);
-
-	return devlink;
-
-err_register_netdevice_notifier:
-	xa_erase(&devlinks, devlink->index);
-err_xa_alloc:
-	kfree(devlink);
-	return NULL;
-}
-EXPORT_SYMBOL_GPL(devlink_alloc_ns);
-
 static void
 devlink_trap_policer_notify(struct devlink *devlink,
 			    const struct devlink_trap_policer_item *policer_item,
@@ -10029,7 +9724,7 @@ static void devlink_trap_notify(struct devlink *devlink,
 				const struct devlink_trap_item *trap_item,
 				enum devlink_command cmd);
 
-static void devlink_notify_register(struct devlink *devlink)
+void devlink_notify_register(struct devlink *devlink)
 {
 	struct devlink_trap_policer_item *policer_item;
 	struct devlink_trap_group_item *group_item;
@@ -10070,7 +9765,7 @@ static void devlink_notify_register(struct devlink *devlink)
 				     DEVLINK_CMD_PARAM_NEW);
 }
 
-static void devlink_notify_unregister(struct devlink *devlink)
+void devlink_notify_unregister(struct devlink *devlink)
 {
 	struct devlink_trap_policer_item *policer_item;
 	struct devlink_trap_group_item *group_item;
@@ -10107,79 +9802,6 @@ static void devlink_notify_unregister(struct devlink *devlink)
 	devlink_notify(devlink, DEVLINK_CMD_DEL);
 }
 
-/**
- *	devlink_register - Register devlink instance
- *
- *	@devlink: devlink
- */
-void devlink_register(struct devlink *devlink)
-{
-	ASSERT_DEVLINK_NOT_REGISTERED(devlink);
-	/* Make sure that we are in .probe() routine */
-
-	xa_set_mark(&devlinks, devlink->index, DEVLINK_REGISTERED);
-	devlink_notify_register(devlink);
-}
-EXPORT_SYMBOL_GPL(devlink_register);
-
-/**
- *	devlink_unregister - Unregister devlink instance
- *
- *	@devlink: devlink
- */
-void devlink_unregister(struct devlink *devlink)
-{
-	ASSERT_DEVLINK_REGISTERED(devlink);
-	/* Make sure that we are in .remove() routine */
-
-	xa_set_mark(&devlinks, devlink->index, DEVLINK_UNREGISTERING);
-	devlink_put(devlink);
-	wait_for_completion(&devlink->comp);
-
-	devlink_notify_unregister(devlink);
-	xa_clear_mark(&devlinks, devlink->index, DEVLINK_REGISTERED);
-	xa_clear_mark(&devlinks, devlink->index, DEVLINK_UNREGISTERING);
-}
-EXPORT_SYMBOL_GPL(devlink_unregister);
-
-/**
- *	devlink_free - Free devlink instance resources
- *
- *	@devlink: devlink
- */
-void devlink_free(struct devlink *devlink)
-{
-	ASSERT_DEVLINK_NOT_REGISTERED(devlink);
-
-	mutex_destroy(&devlink->linecards_lock);
-	mutex_destroy(&devlink->reporters_lock);
-	mutex_destroy(&devlink->lock);
-	lockdep_unregister_key(&devlink->lock_key);
-	WARN_ON(!list_empty(&devlink->trap_policer_list));
-	WARN_ON(!list_empty(&devlink->trap_group_list));
-	WARN_ON(!list_empty(&devlink->trap_list));
-	WARN_ON(!list_empty(&devlink->reporter_list));
-	WARN_ON(!list_empty(&devlink->region_list));
-	WARN_ON(!list_empty(&devlink->param_list));
-	WARN_ON(!list_empty(&devlink->resource_list));
-	WARN_ON(!list_empty(&devlink->dpipe_table_list));
-	WARN_ON(!list_empty(&devlink->sb_list));
-	WARN_ON(!list_empty(&devlink->rate_list));
-	WARN_ON(!list_empty(&devlink->linecard_list));
-	WARN_ON(!xa_empty(&devlink->ports));
-
-	xa_destroy(&devlink->snapshot_ids);
-	xa_destroy(&devlink->ports);
-
-	WARN_ON_ONCE(unregister_netdevice_notifier_net(devlink_net(devlink),
-						       &devlink->netdevice_nb));
-
-	xa_erase(&devlinks, devlink->index);
-
-	kfree(devlink);
-}
-EXPORT_SYMBOL_GPL(devlink_free);
-
 static void devlink_port_type_warn(struct work_struct *work)
 {
 	WARN(true, "Type was not set for devlink port.");
@@ -10480,8 +10102,8 @@ void devlink_port_type_clear(struct devlink_port *devlink_port)
 }
 EXPORT_SYMBOL_GPL(devlink_port_type_clear);
 
-static int devlink_netdevice_event(struct notifier_block *nb,
-				   unsigned long event, void *ptr)
+int devlink_port_netdevice_event(struct notifier_block *nb,
+				 unsigned long event, void *ptr)
 {
 	struct net_device *netdev = netdev_notifier_info_to_dev(ptr);
 	struct devlink_port *devlink_port = netdev->devlink_port;
@@ -12983,47 +12605,3 @@ int devlink_compat_switch_id_get(struct net_device *dev,
 
 	return 0;
 }
-
-static void __net_exit devlink_pernet_pre_exit(struct net *net)
-{
-	struct devlink *devlink;
-	u32 actions_performed;
-	unsigned long index;
-	int err;
-
-	/* In case network namespace is getting destroyed, reload
-	 * all devlink instances from this namespace into init_net.
-	 */
-	devlinks_xa_for_each_registered_get(net, index, devlink) {
-		WARN_ON(!(devlink->features & DEVLINK_F_RELOAD));
-		mutex_lock(&devlink->lock);
-		err = devlink_reload(devlink, &init_net,
-				     DEVLINK_RELOAD_ACTION_DRIVER_REINIT,
-				     DEVLINK_RELOAD_LIMIT_UNSPEC,
-				     &actions_performed, NULL);
-		mutex_unlock(&devlink->lock);
-		if (err && err != -EOPNOTSUPP)
-			pr_warn("Failed to reload devlink instance into init_net\n");
-		devlink_put(devlink);
-	}
-}
-
-static struct pernet_operations devlink_pernet_ops __net_initdata = {
-	.pre_exit = devlink_pernet_pre_exit,
-};
-
-static int __init devlink_init(void)
-{
-	int err;
-
-	err = genl_register_family(&devlink_nl_family);
-	if (err)
-		goto out;
-	err = register_pernet_subsys(&devlink_pernet_ops);
-
-out:
-	WARN_ON(err);
-	return err;
-}
-
-subsys_initcall(devlink_init);
-- 
2.38.1


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

* [PATCH net-next 03/14] devlink: split out netlink code
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
  2023-01-04  4:16 ` [PATCH net-next 01/14] devlink: move code to a dedicated directory Jakub Kicinski
  2023-01-04  4:16 ` [PATCH net-next 02/14] devlink: split out core code Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-05  9:03   ` Jiri Pirko
  2023-01-04  4:16 ` [PATCH net-next 04/14] netlink: add macro for checking dump ctx size Jakub Kicinski
                   ` (10 subsequent siblings)
  13 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

Move out the netlink glue into a separate file.
Leave the ops in the old file because we'd have to export a ton
of functions. Going forward we should switch to split ops which
will let us to put the new ops in the netlink.c file.

Pure code move, no functional changes.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/Makefile        |   2 +-
 net/devlink/devl_internal.h |  31 ++++++
 net/devlink/leftover.c      | 212 ++----------------------------------
 net/devlink/netlink.c       | 195 +++++++++++++++++++++++++++++++++
 4 files changed, 235 insertions(+), 205 deletions(-)
 create mode 100644 net/devlink/netlink.c

diff --git a/net/devlink/Makefile b/net/devlink/Makefile
index aff7da844e5d..1b1eeac59cb3 100644
--- a/net/devlink/Makefile
+++ b/net/devlink/Makefile
@@ -1,3 +1,3 @@
 # SPDX-License-Identifier: GPL-2.0
 
-obj-y := leftover.o core.o
+obj-y := leftover.o core.o netlink.o
diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
index 0cca1f92d733..bc7df9b0f775 100644
--- a/net/devlink/devl_internal.h
+++ b/net/devlink/devl_internal.h
@@ -97,6 +97,20 @@ devlinks_xa_find_get_next(struct net *net, unsigned long *indexp,
 			  xa_mark_t filter);
 
 /* Netlink */
+#define DEVLINK_NL_FLAG_NEED_PORT		BIT(0)
+#define DEVLINK_NL_FLAG_NEED_DEVLINK_OR_PORT	BIT(1)
+#define DEVLINK_NL_FLAG_NEED_RATE		BIT(2)
+#define DEVLINK_NL_FLAG_NEED_RATE_NODE		BIT(3)
+#define DEVLINK_NL_FLAG_NEED_LINECARD		BIT(4)
+
+enum devlink_multicast_groups {
+	DEVLINK_MCGRP_CONFIG,
+};
+
+extern const struct genl_small_ops devlink_nl_ops[56];
+
+struct devlink *devlink_get_from_attrs(struct net *net, struct nlattr **attrs);
+
 void devlink_notify_unregister(struct devlink *devlink);
 void devlink_notify_register(struct devlink *devlink);
 
@@ -104,6 +118,9 @@ void devlink_notify_register(struct devlink *devlink);
 int devlink_port_netdevice_event(struct notifier_block *nb,
 				 unsigned long event, void *ptr);
 
+struct devlink_port *
+devlink_port_get_from_info(struct devlink *devlink, struct genl_info *info);
+
 /* Reload */
 bool devlink_reload_actions_valid(const struct devlink_ops *ops);
 int devlink_reload(struct devlink *devlink, struct net *dest_net,
@@ -115,3 +132,17 @@ static inline bool devlink_reload_supported(const struct devlink_ops *ops)
 {
 	return ops->reload_down && ops->reload_up;
 }
+
+/* Line cards */
+struct devlink_linecard;
+
+struct devlink_linecard *
+devlink_linecard_get_from_info(struct devlink *devlink, struct genl_info *info);
+void devlink_linecard_put(struct devlink_linecard *linecard);
+
+/* Rates */
+struct devlink_rate *
+devlink_rate_get_from_info(struct devlink *devlink, struct genl_info *info);
+struct devlink_rate *
+devlink_rate_node_get_from_info(struct devlink *devlink,
+				struct genl_info *info);
diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
index 05f2e75b3a03..e01ba7999b91 100644
--- a/net/devlink/leftover.c
+++ b/net/devlink/leftover.c
@@ -148,30 +148,6 @@ static const struct nla_policy devlink_selftest_nl_policy[DEVLINK_ATTR_SELFTEST_
 	[DEVLINK_ATTR_SELFTEST_ID_FLASH] = { .type = NLA_FLAG },
 };
 
-static struct devlink *devlink_get_from_attrs(struct net *net,
-					      struct nlattr **attrs)
-{
-	struct devlink *devlink;
-	unsigned long index;
-	char *busname;
-	char *devname;
-
-	if (!attrs[DEVLINK_ATTR_BUS_NAME] || !attrs[DEVLINK_ATTR_DEV_NAME])
-		return ERR_PTR(-EINVAL);
-
-	busname = nla_data(attrs[DEVLINK_ATTR_BUS_NAME]);
-	devname = nla_data(attrs[DEVLINK_ATTR_DEV_NAME]);
-
-	devlinks_xa_for_each_registered_get(net, index, devlink) {
-		if (strcmp(devlink->dev->bus->name, busname) == 0 &&
-		    strcmp(dev_name(devlink->dev), devname) == 0)
-			return devlink;
-		devlink_put(devlink);
-	}
-
-	return ERR_PTR(-ENODEV);
-}
-
 #define ASSERT_DEVLINK_PORT_REGISTERED(devlink_port)				\
 	WARN_ON_ONCE(!(devlink_port)->registered)
 #define ASSERT_DEVLINK_PORT_NOT_REGISTERED(devlink_port)			\
@@ -200,8 +176,8 @@ static struct devlink_port *devlink_port_get_from_attrs(struct devlink *devlink,
 	return ERR_PTR(-EINVAL);
 }
 
-static struct devlink_port *devlink_port_get_from_info(struct devlink *devlink,
-						       struct genl_info *info)
+struct devlink_port *devlink_port_get_from_info(struct devlink *devlink,
+						struct genl_info *info)
 {
 	return devlink_port_get_from_attrs(devlink, info->attrs);
 }
@@ -261,13 +237,13 @@ devlink_rate_node_get_from_attrs(struct devlink *devlink, struct nlattr **attrs)
 	return devlink_rate_node_get_by_name(devlink, rate_node_name);
 }
 
-static struct devlink_rate *
+struct devlink_rate *
 devlink_rate_node_get_from_info(struct devlink *devlink, struct genl_info *info)
 {
 	return devlink_rate_node_get_from_attrs(devlink, info->attrs);
 }
 
-static struct devlink_rate *
+struct devlink_rate *
 devlink_rate_get_from_info(struct devlink *devlink, struct genl_info *info)
 {
 	struct nlattr **attrs = info->attrs;
@@ -318,13 +294,13 @@ devlink_linecard_get_from_attrs(struct devlink *devlink, struct nlattr **attrs)
 	return ERR_PTR(-EINVAL);
 }
 
-static struct devlink_linecard *
+struct devlink_linecard *
 devlink_linecard_get_from_info(struct devlink *devlink, struct genl_info *info)
 {
 	return devlink_linecard_get_from_attrs(devlink, info->attrs);
 }
 
-static void devlink_linecard_put(struct devlink_linecard *linecard)
+void devlink_linecard_put(struct devlink_linecard *linecard)
 {
 	if (refcount_dec_and_test(&linecard->refcount)) {
 		mutex_destroy(&linecard->state_lock);
@@ -633,93 +609,6 @@ devlink_region_snapshot_get_by_id(struct devlink_region *region, u32 id)
 	return NULL;
 }
 
-#define DEVLINK_NL_FLAG_NEED_PORT		BIT(0)
-#define DEVLINK_NL_FLAG_NEED_DEVLINK_OR_PORT	BIT(1)
-#define DEVLINK_NL_FLAG_NEED_RATE		BIT(2)
-#define DEVLINK_NL_FLAG_NEED_RATE_NODE		BIT(3)
-#define DEVLINK_NL_FLAG_NEED_LINECARD		BIT(4)
-
-static int devlink_nl_pre_doit(const struct genl_split_ops *ops,
-			       struct sk_buff *skb, struct genl_info *info)
-{
-	struct devlink_linecard *linecard;
-	struct devlink_port *devlink_port;
-	struct devlink *devlink;
-	int err;
-
-	devlink = devlink_get_from_attrs(genl_info_net(info), info->attrs);
-	if (IS_ERR(devlink))
-		return PTR_ERR(devlink);
-	devl_lock(devlink);
-	info->user_ptr[0] = devlink;
-	if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_PORT) {
-		devlink_port = devlink_port_get_from_info(devlink, info);
-		if (IS_ERR(devlink_port)) {
-			err = PTR_ERR(devlink_port);
-			goto unlock;
-		}
-		info->user_ptr[1] = devlink_port;
-	} else if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_DEVLINK_OR_PORT) {
-		devlink_port = devlink_port_get_from_info(devlink, info);
-		if (!IS_ERR(devlink_port))
-			info->user_ptr[1] = devlink_port;
-	} else if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_RATE) {
-		struct devlink_rate *devlink_rate;
-
-		devlink_rate = devlink_rate_get_from_info(devlink, info);
-		if (IS_ERR(devlink_rate)) {
-			err = PTR_ERR(devlink_rate);
-			goto unlock;
-		}
-		info->user_ptr[1] = devlink_rate;
-	} else if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_RATE_NODE) {
-		struct devlink_rate *rate_node;
-
-		rate_node = devlink_rate_node_get_from_info(devlink, info);
-		if (IS_ERR(rate_node)) {
-			err = PTR_ERR(rate_node);
-			goto unlock;
-		}
-		info->user_ptr[1] = rate_node;
-	} else if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_LINECARD) {
-		linecard = devlink_linecard_get_from_info(devlink, info);
-		if (IS_ERR(linecard)) {
-			err = PTR_ERR(linecard);
-			goto unlock;
-		}
-		info->user_ptr[1] = linecard;
-	}
-	return 0;
-
-unlock:
-	devl_unlock(devlink);
-	devlink_put(devlink);
-	return err;
-}
-
-static void devlink_nl_post_doit(const struct genl_split_ops *ops,
-				 struct sk_buff *skb, struct genl_info *info)
-{
-	struct devlink_linecard *linecard;
-	struct devlink *devlink;
-
-	devlink = info->user_ptr[0];
-	if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_LINECARD) {
-		linecard = info->user_ptr[1];
-		devlink_linecard_put(linecard);
-	}
-	devl_unlock(devlink);
-	devlink_put(devlink);
-}
-
-enum devlink_multicast_groups {
-	DEVLINK_MCGRP_CONFIG,
-};
-
-static const struct genl_multicast_group devlink_nl_mcgrps[] = {
-	[DEVLINK_MCGRP_CONFIG] = { .name = DEVLINK_GENL_MCGRP_CONFIG_NAME },
-};
-
 static int devlink_nl_put_handle(struct sk_buff *msg, struct devlink *devlink)
 {
 	if (nla_put_string(msg, DEVLINK_ATTR_BUS_NAME, devlink->dev->bus->name))
@@ -9234,76 +9123,7 @@ static int devlink_nl_cmd_trap_policer_set_doit(struct sk_buff *skb,
 	return devlink_trap_policer_set(devlink, policer_item, info);
 }
 
-static const struct nla_policy devlink_nl_policy[DEVLINK_ATTR_MAX + 1] = {
-	[DEVLINK_ATTR_UNSPEC] = { .strict_start_type =
-		DEVLINK_ATTR_TRAP_POLICER_ID },
-	[DEVLINK_ATTR_BUS_NAME] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_DEV_NAME] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_PORT_INDEX] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_PORT_TYPE] = NLA_POLICY_RANGE(NLA_U16, DEVLINK_PORT_TYPE_AUTO,
-						    DEVLINK_PORT_TYPE_IB),
-	[DEVLINK_ATTR_PORT_SPLIT_COUNT] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_SB_INDEX] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_SB_POOL_INDEX] = { .type = NLA_U16 },
-	[DEVLINK_ATTR_SB_POOL_TYPE] = { .type = NLA_U8 },
-	[DEVLINK_ATTR_SB_POOL_SIZE] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_SB_POOL_THRESHOLD_TYPE] = { .type = NLA_U8 },
-	[DEVLINK_ATTR_SB_THRESHOLD] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_SB_TC_INDEX] = { .type = NLA_U16 },
-	[DEVLINK_ATTR_ESWITCH_MODE] = NLA_POLICY_RANGE(NLA_U16, DEVLINK_ESWITCH_MODE_LEGACY,
-						       DEVLINK_ESWITCH_MODE_SWITCHDEV),
-	[DEVLINK_ATTR_ESWITCH_INLINE_MODE] = { .type = NLA_U8 },
-	[DEVLINK_ATTR_ESWITCH_ENCAP_MODE] = { .type = NLA_U8 },
-	[DEVLINK_ATTR_DPIPE_TABLE_NAME] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_DPIPE_TABLE_COUNTERS_ENABLED] = { .type = NLA_U8 },
-	[DEVLINK_ATTR_RESOURCE_ID] = { .type = NLA_U64},
-	[DEVLINK_ATTR_RESOURCE_SIZE] = { .type = NLA_U64},
-	[DEVLINK_ATTR_PARAM_NAME] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_PARAM_TYPE] = { .type = NLA_U8 },
-	[DEVLINK_ATTR_PARAM_VALUE_CMODE] = { .type = NLA_U8 },
-	[DEVLINK_ATTR_REGION_NAME] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_REGION_SNAPSHOT_ID] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_REGION_CHUNK_ADDR] = { .type = NLA_U64 },
-	[DEVLINK_ATTR_REGION_CHUNK_LEN] = { .type = NLA_U64 },
-	[DEVLINK_ATTR_HEALTH_REPORTER_NAME] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_HEALTH_REPORTER_GRACEFUL_PERIOD] = { .type = NLA_U64 },
-	[DEVLINK_ATTR_HEALTH_REPORTER_AUTO_RECOVER] = { .type = NLA_U8 },
-	[DEVLINK_ATTR_FLASH_UPDATE_FILE_NAME] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_FLASH_UPDATE_COMPONENT] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_FLASH_UPDATE_OVERWRITE_MASK] =
-		NLA_POLICY_BITFIELD32(DEVLINK_SUPPORTED_FLASH_OVERWRITE_SECTIONS),
-	[DEVLINK_ATTR_TRAP_NAME] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_TRAP_ACTION] = { .type = NLA_U8 },
-	[DEVLINK_ATTR_TRAP_GROUP_NAME] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_NETNS_PID] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_NETNS_FD] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_NETNS_ID] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_HEALTH_REPORTER_AUTO_DUMP] = { .type = NLA_U8 },
-	[DEVLINK_ATTR_TRAP_POLICER_ID] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_TRAP_POLICER_RATE] = { .type = NLA_U64 },
-	[DEVLINK_ATTR_TRAP_POLICER_BURST] = { .type = NLA_U64 },
-	[DEVLINK_ATTR_PORT_FUNCTION] = { .type = NLA_NESTED },
-	[DEVLINK_ATTR_RELOAD_ACTION] = NLA_POLICY_RANGE(NLA_U8, DEVLINK_RELOAD_ACTION_DRIVER_REINIT,
-							DEVLINK_RELOAD_ACTION_MAX),
-	[DEVLINK_ATTR_RELOAD_LIMITS] = NLA_POLICY_BITFIELD32(DEVLINK_RELOAD_LIMITS_VALID_MASK),
-	[DEVLINK_ATTR_PORT_FLAVOUR] = { .type = NLA_U16 },
-	[DEVLINK_ATTR_PORT_PCI_PF_NUMBER] = { .type = NLA_U16 },
-	[DEVLINK_ATTR_PORT_PCI_SF_NUMBER] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_PORT_CONTROLLER_NUMBER] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_RATE_TYPE] = { .type = NLA_U16 },
-	[DEVLINK_ATTR_RATE_TX_SHARE] = { .type = NLA_U64 },
-	[DEVLINK_ATTR_RATE_TX_MAX] = { .type = NLA_U64 },
-	[DEVLINK_ATTR_RATE_NODE_NAME] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_RATE_PARENT_NODE_NAME] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_LINECARD_INDEX] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_LINECARD_TYPE] = { .type = NLA_NUL_STRING },
-	[DEVLINK_ATTR_SELFTESTS] = { .type = NLA_NESTED },
-	[DEVLINK_ATTR_RATE_TX_PRIORITY] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_RATE_TX_WEIGHT] = { .type = NLA_U32 },
-	[DEVLINK_ATTR_REGION_DIRECT] = { .type = NLA_FLAG },
-};
-
-static const struct genl_small_ops devlink_nl_ops[] = {
+const struct genl_small_ops devlink_nl_ops[56] = {
 	{
 		.cmd = DEVLINK_CMD_GET,
 		.validate = GENL_DONT_VALIDATE_STRICT | GENL_DONT_VALIDATE_DUMP,
@@ -9664,23 +9484,7 @@ static const struct genl_small_ops devlink_nl_ops[] = {
 		.doit = devlink_nl_cmd_selftests_run,
 		.flags = GENL_ADMIN_PERM,
 	},
-};
-
-struct genl_family devlink_nl_family __ro_after_init = {
-	.name		= DEVLINK_GENL_NAME,
-	.version	= DEVLINK_GENL_VERSION,
-	.maxattr	= DEVLINK_ATTR_MAX,
-	.policy = devlink_nl_policy,
-	.netnsok	= true,
-	.parallel_ops	= true,
-	.pre_doit	= devlink_nl_pre_doit,
-	.post_doit	= devlink_nl_post_doit,
-	.module		= THIS_MODULE,
-	.small_ops	= devlink_nl_ops,
-	.n_small_ops	= ARRAY_SIZE(devlink_nl_ops),
-	.resv_start_op	= DEVLINK_CMD_SELFTESTS_RUN + 1,
-	.mcgrps		= devlink_nl_mcgrps,
-	.n_mcgrps	= ARRAY_SIZE(devlink_nl_mcgrps),
+	/* -- No new ops here! Use split ops going forward! -- */
 };
 
 bool devlink_reload_actions_valid(const struct devlink_ops *ops)
diff --git a/net/devlink/netlink.c b/net/devlink/netlink.c
new file mode 100644
index 000000000000..ce1a7d674d14
--- /dev/null
+++ b/net/devlink/netlink.c
@@ -0,0 +1,195 @@
+// SPDX-License-Identifier: GPL-2.0-or-later
+/*
+ * Copyright (c) 2016 Mellanox Technologies. All rights reserved.
+ * Copyright (c) 2016 Jiri Pirko <jiri@mellanox.com>
+ */
+
+#include <net/genetlink.h>
+
+#include "devl_internal.h"
+
+static const struct genl_multicast_group devlink_nl_mcgrps[] = {
+	[DEVLINK_MCGRP_CONFIG] = { .name = DEVLINK_GENL_MCGRP_CONFIG_NAME },
+};
+
+static const struct nla_policy devlink_nl_policy[DEVLINK_ATTR_MAX + 1] = {
+	[DEVLINK_ATTR_UNSPEC] = { .strict_start_type =
+		DEVLINK_ATTR_TRAP_POLICER_ID },
+	[DEVLINK_ATTR_BUS_NAME] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_DEV_NAME] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_PORT_INDEX] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_PORT_TYPE] = NLA_POLICY_RANGE(NLA_U16, DEVLINK_PORT_TYPE_AUTO,
+						    DEVLINK_PORT_TYPE_IB),
+	[DEVLINK_ATTR_PORT_SPLIT_COUNT] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_SB_INDEX] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_SB_POOL_INDEX] = { .type = NLA_U16 },
+	[DEVLINK_ATTR_SB_POOL_TYPE] = { .type = NLA_U8 },
+	[DEVLINK_ATTR_SB_POOL_SIZE] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_SB_POOL_THRESHOLD_TYPE] = { .type = NLA_U8 },
+	[DEVLINK_ATTR_SB_THRESHOLD] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_SB_TC_INDEX] = { .type = NLA_U16 },
+	[DEVLINK_ATTR_ESWITCH_MODE] = NLA_POLICY_RANGE(NLA_U16, DEVLINK_ESWITCH_MODE_LEGACY,
+						       DEVLINK_ESWITCH_MODE_SWITCHDEV),
+	[DEVLINK_ATTR_ESWITCH_INLINE_MODE] = { .type = NLA_U8 },
+	[DEVLINK_ATTR_ESWITCH_ENCAP_MODE] = { .type = NLA_U8 },
+	[DEVLINK_ATTR_DPIPE_TABLE_NAME] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_DPIPE_TABLE_COUNTERS_ENABLED] = { .type = NLA_U8 },
+	[DEVLINK_ATTR_RESOURCE_ID] = { .type = NLA_U64},
+	[DEVLINK_ATTR_RESOURCE_SIZE] = { .type = NLA_U64},
+	[DEVLINK_ATTR_PARAM_NAME] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_PARAM_TYPE] = { .type = NLA_U8 },
+	[DEVLINK_ATTR_PARAM_VALUE_CMODE] = { .type = NLA_U8 },
+	[DEVLINK_ATTR_REGION_NAME] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_REGION_SNAPSHOT_ID] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_REGION_CHUNK_ADDR] = { .type = NLA_U64 },
+	[DEVLINK_ATTR_REGION_CHUNK_LEN] = { .type = NLA_U64 },
+	[DEVLINK_ATTR_HEALTH_REPORTER_NAME] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_HEALTH_REPORTER_GRACEFUL_PERIOD] = { .type = NLA_U64 },
+	[DEVLINK_ATTR_HEALTH_REPORTER_AUTO_RECOVER] = { .type = NLA_U8 },
+	[DEVLINK_ATTR_FLASH_UPDATE_FILE_NAME] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_FLASH_UPDATE_COMPONENT] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_FLASH_UPDATE_OVERWRITE_MASK] =
+		NLA_POLICY_BITFIELD32(DEVLINK_SUPPORTED_FLASH_OVERWRITE_SECTIONS),
+	[DEVLINK_ATTR_TRAP_NAME] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_TRAP_ACTION] = { .type = NLA_U8 },
+	[DEVLINK_ATTR_TRAP_GROUP_NAME] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_NETNS_PID] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_NETNS_FD] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_NETNS_ID] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_HEALTH_REPORTER_AUTO_DUMP] = { .type = NLA_U8 },
+	[DEVLINK_ATTR_TRAP_POLICER_ID] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_TRAP_POLICER_RATE] = { .type = NLA_U64 },
+	[DEVLINK_ATTR_TRAP_POLICER_BURST] = { .type = NLA_U64 },
+	[DEVLINK_ATTR_PORT_FUNCTION] = { .type = NLA_NESTED },
+	[DEVLINK_ATTR_RELOAD_ACTION] = NLA_POLICY_RANGE(NLA_U8, DEVLINK_RELOAD_ACTION_DRIVER_REINIT,
+							DEVLINK_RELOAD_ACTION_MAX),
+	[DEVLINK_ATTR_RELOAD_LIMITS] = NLA_POLICY_BITFIELD32(DEVLINK_RELOAD_LIMITS_VALID_MASK),
+	[DEVLINK_ATTR_PORT_FLAVOUR] = { .type = NLA_U16 },
+	[DEVLINK_ATTR_PORT_PCI_PF_NUMBER] = { .type = NLA_U16 },
+	[DEVLINK_ATTR_PORT_PCI_SF_NUMBER] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_PORT_CONTROLLER_NUMBER] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_RATE_TYPE] = { .type = NLA_U16 },
+	[DEVLINK_ATTR_RATE_TX_SHARE] = { .type = NLA_U64 },
+	[DEVLINK_ATTR_RATE_TX_MAX] = { .type = NLA_U64 },
+	[DEVLINK_ATTR_RATE_NODE_NAME] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_RATE_PARENT_NODE_NAME] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_LINECARD_INDEX] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_LINECARD_TYPE] = { .type = NLA_NUL_STRING },
+	[DEVLINK_ATTR_SELFTESTS] = { .type = NLA_NESTED },
+	[DEVLINK_ATTR_RATE_TX_PRIORITY] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_RATE_TX_WEIGHT] = { .type = NLA_U32 },
+	[DEVLINK_ATTR_REGION_DIRECT] = { .type = NLA_FLAG },
+};
+
+struct devlink *devlink_get_from_attrs(struct net *net, struct nlattr **attrs)
+{
+	struct devlink *devlink;
+	unsigned long index;
+	char *busname;
+	char *devname;
+
+	if (!attrs[DEVLINK_ATTR_BUS_NAME] || !attrs[DEVLINK_ATTR_DEV_NAME])
+		return ERR_PTR(-EINVAL);
+
+	busname = nla_data(attrs[DEVLINK_ATTR_BUS_NAME]);
+	devname = nla_data(attrs[DEVLINK_ATTR_DEV_NAME]);
+
+	devlinks_xa_for_each_registered_get(net, index, devlink) {
+		if (strcmp(devlink->dev->bus->name, busname) == 0 &&
+		    strcmp(dev_name(devlink->dev), devname) == 0)
+			return devlink;
+		devlink_put(devlink);
+	}
+
+	return ERR_PTR(-ENODEV);
+}
+
+static int devlink_nl_pre_doit(const struct genl_split_ops *ops,
+			       struct sk_buff *skb, struct genl_info *info)
+{
+	struct devlink_linecard *linecard;
+	struct devlink_port *devlink_port;
+	struct devlink *devlink;
+	int err;
+
+	devlink = devlink_get_from_attrs(genl_info_net(info), info->attrs);
+	if (IS_ERR(devlink))
+		return PTR_ERR(devlink);
+	devl_lock(devlink);
+	info->user_ptr[0] = devlink;
+	if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_PORT) {
+		devlink_port = devlink_port_get_from_info(devlink, info);
+		if (IS_ERR(devlink_port)) {
+			err = PTR_ERR(devlink_port);
+			goto unlock;
+		}
+		info->user_ptr[1] = devlink_port;
+	} else if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_DEVLINK_OR_PORT) {
+		devlink_port = devlink_port_get_from_info(devlink, info);
+		if (!IS_ERR(devlink_port))
+			info->user_ptr[1] = devlink_port;
+	} else if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_RATE) {
+		struct devlink_rate *devlink_rate;
+
+		devlink_rate = devlink_rate_get_from_info(devlink, info);
+		if (IS_ERR(devlink_rate)) {
+			err = PTR_ERR(devlink_rate);
+			goto unlock;
+		}
+		info->user_ptr[1] = devlink_rate;
+	} else if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_RATE_NODE) {
+		struct devlink_rate *rate_node;
+
+		rate_node = devlink_rate_node_get_from_info(devlink, info);
+		if (IS_ERR(rate_node)) {
+			err = PTR_ERR(rate_node);
+			goto unlock;
+		}
+		info->user_ptr[1] = rate_node;
+	} else if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_LINECARD) {
+		linecard = devlink_linecard_get_from_info(devlink, info);
+		if (IS_ERR(linecard)) {
+			err = PTR_ERR(linecard);
+			goto unlock;
+		}
+		info->user_ptr[1] = linecard;
+	}
+	return 0;
+
+unlock:
+	devl_unlock(devlink);
+	devlink_put(devlink);
+	return err;
+}
+
+static void devlink_nl_post_doit(const struct genl_split_ops *ops,
+				 struct sk_buff *skb, struct genl_info *info)
+{
+	struct devlink_linecard *linecard;
+	struct devlink *devlink;
+
+	devlink = info->user_ptr[0];
+	if (ops->internal_flags & DEVLINK_NL_FLAG_NEED_LINECARD) {
+		linecard = info->user_ptr[1];
+		devlink_linecard_put(linecard);
+	}
+	devl_unlock(devlink);
+	devlink_put(devlink);
+}
+
+struct genl_family devlink_nl_family __ro_after_init = {
+	.name		= DEVLINK_GENL_NAME,
+	.version	= DEVLINK_GENL_VERSION,
+	.maxattr	= DEVLINK_ATTR_MAX,
+	.policy		= devlink_nl_policy,
+	.netnsok	= true,
+	.parallel_ops	= true,
+	.pre_doit	= devlink_nl_pre_doit,
+	.post_doit	= devlink_nl_post_doit,
+	.module		= THIS_MODULE,
+	.small_ops	= devlink_nl_ops,
+	.n_small_ops	= ARRAY_SIZE(devlink_nl_ops),
+	.resv_start_op	= DEVLINK_CMD_SELFTESTS_RUN + 1,
+	.mcgrps		= devlink_nl_mcgrps,
+	.n_mcgrps	= ARRAY_SIZE(devlink_nl_mcgrps),
+};
-- 
2.38.1


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

* [PATCH net-next 04/14] netlink: add macro for checking dump ctx size
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
                   ` (2 preceding siblings ...)
  2023-01-04  4:16 ` [PATCH net-next 03/14] devlink: split out netlink code Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04  9:51   ` Jiri Pirko
  2023-01-04  4:16 ` [PATCH net-next 05/14] devlink: use an explicit structure for dump context Jakub Kicinski
                   ` (9 subsequent siblings)
  13 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem
  Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski,
	pablo, kadlec, fw, johannes, ecree.xilinx, netfilter-devel,
	coreteam

We encourage casting struct netlink_callback::ctx to a local
struct (in a comment above the field). Provide a convenience
macro for checking if the local struct fits into the ctx.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
CC: pablo@netfilter.org
CC: kadlec@netfilter.org
CC: fw@strlen.de
CC: johannes@sipsolutions.net
CC: ecree.xilinx@gmail.com
CC: netfilter-devel@vger.kernel.org
CC: coreteam@netfilter.org
---
 include/linux/netlink.h              | 4 ++++
 net/netfilter/nf_conntrack_netlink.c | 2 +-
 2 files changed, 5 insertions(+), 1 deletion(-)

diff --git a/include/linux/netlink.h b/include/linux/netlink.h
index d81bde5a5844..38f6334f408c 100644
--- a/include/linux/netlink.h
+++ b/include/linux/netlink.h
@@ -263,6 +263,10 @@ struct netlink_callback {
 	};
 };
 
+#define NL_ASSET_DUMP_CTX_FITS(type_name)				\
+	BUILD_BUG_ON(sizeof(type_name) >				\
+		     sizeof_field(struct netlink_callback, ctx))
+
 struct netlink_notify {
 	struct net *net;
 	u32 portid;
diff --git a/net/netfilter/nf_conntrack_netlink.c b/net/netfilter/nf_conntrack_netlink.c
index 1286ae7d4609..90672e293e57 100644
--- a/net/netfilter/nf_conntrack_netlink.c
+++ b/net/netfilter/nf_conntrack_netlink.c
@@ -3866,7 +3866,7 @@ static int __init ctnetlink_init(void)
 {
 	int ret;
 
-	BUILD_BUG_ON(sizeof(struct ctnetlink_list_dump_ctx) > sizeof_field(struct netlink_callback, ctx));
+	NL_ASSET_DUMP_CTX_FITS(struct ctnetlink_list_dump_ctx);
 
 	ret = nfnetlink_subsys_register(&ctnl_subsys);
 	if (ret < 0) {
-- 
2.38.1


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

* [PATCH net-next 05/14] devlink: use an explicit structure for dump context
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
                   ` (3 preceding siblings ...)
  2023-01-04  4:16 ` [PATCH net-next 04/14] netlink: add macro for checking dump ctx size Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04 10:04   ` Jiri Pirko
  2023-01-04  4:16 ` [PATCH net-next 06/14] devlink: remove start variables from dumps Jakub Kicinski
                   ` (8 subsequent siblings)
  13 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

Create a dump context structure instead of using cb->args
as an unsigned long array. This is a pure conversion which
is intended to be as much of a noop as possible.
Subsequent changes will use this to simplify the code.

The two non-trivial parts are:
 - devlink_nl_cmd_health_reporter_dump_get_dumpit() checks args[0]
   to see if devlink_fmsg_dumpit() has already been called (whether
   this is the first msg), but doesn't use the exact value, so we
   can drop the local variable there already
 - devlink_nl_cmd_region_read_dumpit() uses args[0] for address
   but we'll use args[1] now, shouldn't matter

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/devl_internal.h | 23 +++++++++
 net/devlink/leftover.c      | 98 ++++++++++++++++++++++---------------
 2 files changed, 81 insertions(+), 40 deletions(-)

diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
index bc7df9b0f775..91059311f18d 100644
--- a/net/devlink/devl_internal.h
+++ b/net/devlink/devl_internal.h
@@ -107,6 +107,21 @@ enum devlink_multicast_groups {
 	DEVLINK_MCGRP_CONFIG,
 };
 
+/* state held across netlink dumps */
+struct devlink_nl_dump_state {
+	int idx;
+	union {
+		/* DEVLINK_CMD_REGION_READ */
+		struct {
+			u64 start_offset;
+		};
+		/* DEVLINK_CMD_HEALTH_REPORTER_DUMP_GET */
+		struct {
+			u64 dump_ts;
+		};
+	};
+};
+
 extern const struct genl_small_ops devlink_nl_ops[56];
 
 struct devlink *devlink_get_from_attrs(struct net *net, struct nlattr **attrs);
@@ -114,6 +129,14 @@ struct devlink *devlink_get_from_attrs(struct net *net, struct nlattr **attrs);
 void devlink_notify_unregister(struct devlink *devlink);
 void devlink_notify_register(struct devlink *devlink);
 
+static inline struct devlink_nl_dump_state *
+devl_dump_state(struct netlink_callback *cb)
+{
+	NL_ASSET_DUMP_CTX_FITS(struct devlink_nl_dump_state);
+
+	return (struct devlink_nl_dump_state *)cb->ctx;
+}
+
 /* Ports */
 int devlink_port_netdevice_event(struct notifier_block *nb,
 				 unsigned long event, void *ptr);
diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
index e01ba7999b91..bcc930b7cfcf 100644
--- a/net/devlink/leftover.c
+++ b/net/devlink/leftover.c
@@ -1222,9 +1222,10 @@ static void devlink_rate_notify(struct devlink_rate *devlink_rate,
 static int devlink_nl_cmd_rate_get_dumpit(struct sk_buff *msg,
 					  struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_rate *devlink_rate;
 	struct devlink *devlink;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -1256,7 +1257,7 @@ static int devlink_nl_cmd_rate_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -1317,8 +1318,9 @@ static int devlink_nl_cmd_get_doit(struct sk_buff *skb, struct genl_info *info)
 static int devlink_nl_cmd_get_dumpit(struct sk_buff *msg,
 				     struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
@@ -1342,7 +1344,7 @@ static int devlink_nl_cmd_get_dumpit(struct sk_buff *msg,
 		idx++;
 	}
 out:
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -1371,10 +1373,11 @@ static int devlink_nl_cmd_port_get_doit(struct sk_buff *skb,
 static int devlink_nl_cmd_port_get_dumpit(struct sk_buff *msg,
 					  struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
 	struct devlink_port *devlink_port;
 	unsigned long index, port_index;
-	int start = cb->args[0];
+	int start = dump->idx;
 	int idx = 0;
 	int err;
 
@@ -1401,7 +1404,7 @@ static int devlink_nl_cmd_port_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -2150,9 +2153,10 @@ static int devlink_nl_cmd_linecard_get_doit(struct sk_buff *skb,
 static int devlink_nl_cmd_linecard_get_dumpit(struct sk_buff *msg,
 					      struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_linecard *linecard;
 	struct devlink *devlink;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
@@ -2183,7 +2187,7 @@ static int devlink_nl_cmd_linecard_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -2412,9 +2416,10 @@ static int devlink_nl_cmd_sb_get_doit(struct sk_buff *skb,
 static int devlink_nl_cmd_sb_get_dumpit(struct sk_buff *msg,
 					struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
 	struct devlink_sb *devlink_sb;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
@@ -2442,7 +2447,7 @@ static int devlink_nl_cmd_sb_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -2554,9 +2559,10 @@ static int __sb_pool_get_dumpit(struct sk_buff *msg, int start, int *p_idx,
 static int devlink_nl_cmd_sb_pool_get_dumpit(struct sk_buff *msg,
 					     struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
 	struct devlink_sb *devlink_sb;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -2587,7 +2593,7 @@ static int devlink_nl_cmd_sb_pool_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -2769,9 +2775,10 @@ static int __sb_port_pool_get_dumpit(struct sk_buff *msg, int start, int *p_idx,
 static int devlink_nl_cmd_sb_port_pool_get_dumpit(struct sk_buff *msg,
 						  struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
 	struct devlink_sb *devlink_sb;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -2802,7 +2809,7 @@ static int devlink_nl_cmd_sb_port_pool_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -3012,9 +3019,10 @@ static int
 devlink_nl_cmd_sb_tc_pool_bind_get_dumpit(struct sk_buff *msg,
 					  struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
 	struct devlink_sb *devlink_sb;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -3046,7 +3054,7 @@ devlink_nl_cmd_sb_tc_pool_bind_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -4871,8 +4879,9 @@ static int devlink_nl_cmd_selftests_get_doit(struct sk_buff *skb,
 static int devlink_nl_cmd_selftests_get_dumpit(struct sk_buff *msg,
 					       struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -4899,7 +4908,7 @@ static int devlink_nl_cmd_selftests_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -5351,9 +5360,10 @@ static void devlink_param_notify(struct devlink *devlink,
 static int devlink_nl_cmd_param_get_dumpit(struct sk_buff *msg,
 					   struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_param_item *param_item;
 	struct devlink *devlink;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -5386,7 +5396,7 @@ static int devlink_nl_cmd_param_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -6095,8 +6105,9 @@ static int devlink_nl_cmd_region_get_devlink_dumpit(struct sk_buff *msg,
 static int devlink_nl_cmd_region_get_dumpit(struct sk_buff *msg,
 					    struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -6109,7 +6120,7 @@ static int devlink_nl_cmd_region_get_dumpit(struct sk_buff *msg,
 			goto out;
 	}
 out:
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -6394,6 +6405,7 @@ static int devlink_nl_cmd_region_read_dumpit(struct sk_buff *skb,
 					     struct netlink_callback *cb)
 {
 	const struct genl_dumpit_info *info = genl_dumpit_info(cb);
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct nlattr *chunks_attr, *region_attr, *snapshot_attr;
 	u64 ret_offset, start_offset, end_offset = U64_MAX;
 	struct nlattr **attrs = info->attrs;
@@ -6407,7 +6419,7 @@ static int devlink_nl_cmd_region_read_dumpit(struct sk_buff *skb,
 	void *hdr;
 	int err;
 
-	start_offset = *((u64 *)&cb->args[0]);
+	start_offset = dump->start_offset;
 
 	devlink = devlink_get_from_attrs(sock_net(cb->skb->sk), attrs);
 	if (IS_ERR(devlink))
@@ -6546,7 +6558,7 @@ static int devlink_nl_cmd_region_read_dumpit(struct sk_buff *skb,
 		goto nla_put_failure;
 	}
 
-	*((u64 *)&cb->args[0]) = ret_offset;
+	dump->start_offset = ret_offset;
 
 	nla_nest_end(skb, chunks_attr);
 	genlmsg_end(skb, hdr);
@@ -6745,8 +6757,9 @@ static int devlink_nl_cmd_info_get_doit(struct sk_buff *skb,
 static int devlink_nl_cmd_info_get_dumpit(struct sk_buff *msg,
 					  struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -6775,7 +6788,7 @@ static int devlink_nl_cmd_info_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -7344,7 +7357,8 @@ static int devlink_fmsg_dumpit(struct devlink_fmsg *fmsg, struct sk_buff *skb,
 			       struct netlink_callback *cb,
 			       enum devlink_command cmd)
 {
-	int index = cb->args[0];
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
+	int index = dump->idx;
 	int tmp_index = index;
 	void *hdr;
 	int err;
@@ -7360,7 +7374,7 @@ static int devlink_fmsg_dumpit(struct devlink_fmsg *fmsg, struct sk_buff *skb,
 	if ((err && err != -EMSGSIZE) || tmp_index == index)
 		goto nla_put_failure;
 
-	cb->args[0] = index;
+	dump->idx = index;
 	genlmsg_end(skb, hdr);
 	return skb->len;
 
@@ -7911,11 +7925,12 @@ static int
 devlink_nl_cmd_health_reporter_get_dumpit(struct sk_buff *msg,
 					  struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_health_reporter *reporter;
 	unsigned long index, port_index;
 	struct devlink_port *port;
 	struct devlink *devlink;
-	int start = cb->args[0];
+	int start = dump->idx;
 	int idx = 0;
 	int err;
 
@@ -7970,7 +7985,7 @@ devlink_nl_cmd_health_reporter_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -8082,8 +8097,8 @@ static int
 devlink_nl_cmd_health_reporter_dump_get_dumpit(struct sk_buff *skb,
 					       struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_health_reporter *reporter;
-	u64 start = cb->args[0];
 	int err;
 
 	reporter = devlink_health_reporter_get_from_cb(cb);
@@ -8095,13 +8110,13 @@ devlink_nl_cmd_health_reporter_dump_get_dumpit(struct sk_buff *skb,
 		goto out;
 	}
 	mutex_lock(&reporter->dump_lock);
-	if (!start) {
+	if (!dump->idx) {
 		err = devlink_health_do_dump(reporter, NULL, cb->extack);
 		if (err)
 			goto unlock;
-		cb->args[1] = reporter->dump_ts;
+		dump->dump_ts = reporter->dump_ts;
 	}
-	if (!reporter->dump_fmsg || cb->args[1] != reporter->dump_ts) {
+	if (!reporter->dump_fmsg || dump->dump_ts != reporter->dump_ts) {
 		NL_SET_ERR_MSG_MOD(cb->extack, "Dump trampled, please retry");
 		err = -EAGAIN;
 		goto unlock;
@@ -8495,9 +8510,10 @@ static int devlink_nl_cmd_trap_get_doit(struct sk_buff *skb,
 static int devlink_nl_cmd_trap_get_dumpit(struct sk_buff *msg,
 					  struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_trap_item *trap_item;
 	struct devlink *devlink;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
@@ -8525,7 +8541,7 @@ static int devlink_nl_cmd_trap_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -8710,11 +8726,12 @@ static int devlink_nl_cmd_trap_group_get_doit(struct sk_buff *skb,
 static int devlink_nl_cmd_trap_group_get_dumpit(struct sk_buff *msg,
 						struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	enum devlink_command cmd = DEVLINK_CMD_TRAP_GROUP_NEW;
 	struct devlink_trap_group_item *group_item;
 	u32 portid = NETLINK_CB(cb->skb).portid;
 	struct devlink *devlink;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
@@ -8743,7 +8760,7 @@ static int devlink_nl_cmd_trap_group_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
@@ -9014,11 +9031,12 @@ static int devlink_nl_cmd_trap_policer_get_doit(struct sk_buff *skb,
 static int devlink_nl_cmd_trap_policer_get_dumpit(struct sk_buff *msg,
 						  struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	enum devlink_command cmd = DEVLINK_CMD_TRAP_POLICER_NEW;
 	struct devlink_trap_policer_item *policer_item;
 	u32 portid = NETLINK_CB(cb->skb).portid;
 	struct devlink *devlink;
-	int start = cb->args[0];
+	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
@@ -9047,7 +9065,7 @@ static int devlink_nl_cmd_trap_policer_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	cb->args[0] = idx;
+	dump->idx = idx;
 	return msg->len;
 }
 
-- 
2.38.1


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

* [PATCH net-next 06/14] devlink: remove start variables from dumps
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
                   ` (4 preceding siblings ...)
  2023-01-04  4:16 ` [PATCH net-next 05/14] devlink: use an explicit structure for dump context Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04  4:16 ` [PATCH net-next 07/14] devlink: drop the filter argument from devlinks_xa_find_get Jakub Kicinski
                   ` (7 subsequent siblings)
  13 siblings, 0 replies; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

The start variables made the code clearer when we had to access
cb->args[0] directly, as the name args doesn't explain much.
Now that we use a structure to hold state this seems no longer
needed.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/leftover.c | 55 +++++++++++++++---------------------------
 1 file changed, 19 insertions(+), 36 deletions(-)

diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
index bcc930b7cfcf..9aac82dc7269 100644
--- a/net/devlink/leftover.c
+++ b/net/devlink/leftover.c
@@ -1225,7 +1225,6 @@ static int devlink_nl_cmd_rate_get_dumpit(struct sk_buff *msg,
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_rate *devlink_rate;
 	struct devlink *devlink;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -1236,7 +1235,7 @@ static int devlink_nl_cmd_rate_get_dumpit(struct sk_buff *msg,
 			enum devlink_command cmd = DEVLINK_CMD_RATE_NEW;
 			u32 id = NETLINK_CB(cb->skb).portid;
 
-			if (idx < start) {
+			if (idx < dump->idx) {
 				idx++;
 				continue;
 			}
@@ -1320,13 +1319,12 @@ static int devlink_nl_cmd_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
 
 	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
-		if (idx < start) {
+		if (idx < dump->idx) {
 			idx++;
 			devlink_put(devlink);
 			continue;
@@ -1377,14 +1375,13 @@ static int devlink_nl_cmd_port_get_dumpit(struct sk_buff *msg,
 	struct devlink *devlink;
 	struct devlink_port *devlink_port;
 	unsigned long index, port_index;
-	int start = dump->idx;
 	int idx = 0;
 	int err;
 
 	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
 		devl_lock(devlink);
 		xa_for_each(&devlink->ports, port_index, devlink_port) {
-			if (idx < start) {
+			if (idx < dump->idx) {
 				idx++;
 				continue;
 			}
@@ -2156,7 +2153,6 @@ static int devlink_nl_cmd_linecard_get_dumpit(struct sk_buff *msg,
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_linecard *linecard;
 	struct devlink *devlink;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
@@ -2164,7 +2160,7 @@ static int devlink_nl_cmd_linecard_get_dumpit(struct sk_buff *msg,
 	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
 		mutex_lock(&devlink->linecards_lock);
 		list_for_each_entry(linecard, &devlink->linecard_list, list) {
-			if (idx < start) {
+			if (idx < dump->idx) {
 				idx++;
 				continue;
 			}
@@ -2419,7 +2415,6 @@ static int devlink_nl_cmd_sb_get_dumpit(struct sk_buff *msg,
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
 	struct devlink_sb *devlink_sb;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
@@ -2427,7 +2422,7 @@ static int devlink_nl_cmd_sb_get_dumpit(struct sk_buff *msg,
 	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
 		devl_lock(devlink);
 		list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
-			if (idx < start) {
+			if (idx < dump->idx) {
 				idx++;
 				continue;
 			}
@@ -2562,7 +2557,6 @@ static int devlink_nl_cmd_sb_pool_get_dumpit(struct sk_buff *msg,
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
 	struct devlink_sb *devlink_sb;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -2573,8 +2567,8 @@ static int devlink_nl_cmd_sb_pool_get_dumpit(struct sk_buff *msg,
 
 		devl_lock(devlink);
 		list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
-			err = __sb_pool_get_dumpit(msg, start, &idx, devlink,
-						   devlink_sb,
+			err = __sb_pool_get_dumpit(msg, dump->idx, &idx,
+						   devlink, devlink_sb,
 						   NETLINK_CB(cb->skb).portid,
 						   cb->nlh->nlmsg_seq);
 			if (err == -EOPNOTSUPP) {
@@ -2778,7 +2772,6 @@ static int devlink_nl_cmd_sb_port_pool_get_dumpit(struct sk_buff *msg,
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
 	struct devlink_sb *devlink_sb;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -2789,7 +2782,7 @@ static int devlink_nl_cmd_sb_port_pool_get_dumpit(struct sk_buff *msg,
 
 		devl_lock(devlink);
 		list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
-			err = __sb_port_pool_get_dumpit(msg, start, &idx,
+			err = __sb_port_pool_get_dumpit(msg, dump->idx, &idx,
 							devlink, devlink_sb,
 							NETLINK_CB(cb->skb).portid,
 							cb->nlh->nlmsg_seq);
@@ -3022,7 +3015,6 @@ devlink_nl_cmd_sb_tc_pool_bind_get_dumpit(struct sk_buff *msg,
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
 	struct devlink_sb *devlink_sb;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -3033,9 +3025,8 @@ devlink_nl_cmd_sb_tc_pool_bind_get_dumpit(struct sk_buff *msg,
 
 		devl_lock(devlink);
 		list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
-			err = __sb_tc_pool_bind_get_dumpit(msg, start, &idx,
-							   devlink,
-							   devlink_sb,
+			err = __sb_tc_pool_bind_get_dumpit(msg, dump->idx, &idx,
+							   devlink, devlink_sb,
 							   NETLINK_CB(cb->skb).portid,
 							   cb->nlh->nlmsg_seq);
 			if (err == -EOPNOTSUPP) {
@@ -4881,13 +4872,12 @@ static int devlink_nl_cmd_selftests_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
 
 	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
-		if (idx < start || !devlink->ops->selftest_check)
+		if (idx < dump->idx || !devlink->ops->selftest_check)
 			goto inc;
 
 		devl_lock(devlink);
@@ -5363,7 +5353,6 @@ static int devlink_nl_cmd_param_get_dumpit(struct sk_buff *msg,
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_param_item *param_item;
 	struct devlink *devlink;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
@@ -5371,7 +5360,7 @@ static int devlink_nl_cmd_param_get_dumpit(struct sk_buff *msg,
 	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
 		devl_lock(devlink);
 		list_for_each_entry(param_item, &devlink->param_list, list) {
-			if (idx < start) {
+			if (idx < dump->idx) {
 				idx++;
 				continue;
 			}
@@ -6107,14 +6096,13 @@ static int devlink_nl_cmd_region_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
 
 	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
 		err = devlink_nl_cmd_region_get_devlink_dumpit(msg, cb, devlink,
-							       &idx, start);
+							       &idx, dump->idx);
 		devlink_put(devlink);
 		if (err)
 			goto out;
@@ -6759,13 +6747,12 @@ static int devlink_nl_cmd_info_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err = 0;
 
 	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
-		if (idx < start)
+		if (idx < dump->idx)
 			goto inc;
 
 		devl_lock(devlink);
@@ -7930,7 +7917,6 @@ devlink_nl_cmd_health_reporter_get_dumpit(struct sk_buff *msg,
 	unsigned long index, port_index;
 	struct devlink_port *port;
 	struct devlink *devlink;
-	int start = dump->idx;
 	int idx = 0;
 	int err;
 
@@ -7938,7 +7924,7 @@ devlink_nl_cmd_health_reporter_get_dumpit(struct sk_buff *msg,
 		mutex_lock(&devlink->reporters_lock);
 		list_for_each_entry(reporter, &devlink->reporter_list,
 				    list) {
-			if (idx < start) {
+			if (idx < dump->idx) {
 				idx++;
 				continue;
 			}
@@ -7962,7 +7948,7 @@ devlink_nl_cmd_health_reporter_get_dumpit(struct sk_buff *msg,
 		xa_for_each(&devlink->ports, port_index, port) {
 			mutex_lock(&port->reporters_lock);
 			list_for_each_entry(reporter, &port->reporter_list, list) {
-				if (idx < start) {
+				if (idx < dump->idx) {
 					idx++;
 					continue;
 				}
@@ -8513,7 +8499,6 @@ static int devlink_nl_cmd_trap_get_dumpit(struct sk_buff *msg,
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_trap_item *trap_item;
 	struct devlink *devlink;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
@@ -8521,7 +8506,7 @@ static int devlink_nl_cmd_trap_get_dumpit(struct sk_buff *msg,
 	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
 		devl_lock(devlink);
 		list_for_each_entry(trap_item, &devlink->trap_list, list) {
-			if (idx < start) {
+			if (idx < dump->idx) {
 				idx++;
 				continue;
 			}
@@ -8731,7 +8716,6 @@ static int devlink_nl_cmd_trap_group_get_dumpit(struct sk_buff *msg,
 	struct devlink_trap_group_item *group_item;
 	u32 portid = NETLINK_CB(cb->skb).portid;
 	struct devlink *devlink;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
@@ -8740,7 +8724,7 @@ static int devlink_nl_cmd_trap_group_get_dumpit(struct sk_buff *msg,
 		devl_lock(devlink);
 		list_for_each_entry(group_item, &devlink->trap_group_list,
 				    list) {
-			if (idx < start) {
+			if (idx < dump->idx) {
 				idx++;
 				continue;
 			}
@@ -9036,7 +9020,6 @@ static int devlink_nl_cmd_trap_policer_get_dumpit(struct sk_buff *msg,
 	struct devlink_trap_policer_item *policer_item;
 	u32 portid = NETLINK_CB(cb->skb).portid;
 	struct devlink *devlink;
-	int start = dump->idx;
 	unsigned long index;
 	int idx = 0;
 	int err;
@@ -9045,7 +9028,7 @@ static int devlink_nl_cmd_trap_policer_get_dumpit(struct sk_buff *msg,
 		devl_lock(devlink);
 		list_for_each_entry(policer_item, &devlink->trap_policer_list,
 				    list) {
-			if (idx < start) {
+			if (idx < dump->idx) {
 				idx++;
 				continue;
 			}
-- 
2.38.1


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

* [PATCH net-next 07/14] devlink: drop the filter argument from devlinks_xa_find_get
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
                   ` (5 preceding siblings ...)
  2023-01-04  4:16 ` [PATCH net-next 06/14] devlink: remove start variables from dumps Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04 10:05   ` Jiri Pirko
  2023-01-04  4:16 ` [PATCH net-next 08/14] devlink: health: combine loops in dump Jakub Kicinski
                   ` (6 subsequent siblings)
  13 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

Looks like devlinks_xa_find_get() was intended to get the mark
from the @filter argument. It doesn't actually use @filter, passing
DEVLINK_REGISTERED to xa_find_fn() directly. Walking marks other
than registered is unlikely so drop @filter argument completely.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/core.c          | 12 +++++-------
 net/devlink/devl_internal.h | 15 +++++----------
 2 files changed, 10 insertions(+), 17 deletions(-)

diff --git a/net/devlink/core.c b/net/devlink/core.c
index c084eafa17fb..3a99bf84632e 100644
--- a/net/devlink/core.c
+++ b/net/devlink/core.c
@@ -92,7 +92,7 @@ void devlink_put(struct devlink *devlink)
 }
 
 static struct devlink *
-devlinks_xa_find_get(struct net *net, unsigned long *indexp, xa_mark_t filter,
+devlinks_xa_find_get(struct net *net, unsigned long *indexp,
 		     void * (*xa_find_fn)(struct xarray *, unsigned long *,
 					  unsigned long, xa_mark_t))
 {
@@ -125,17 +125,15 @@ devlinks_xa_find_get(struct net *net, unsigned long *indexp, xa_mark_t filter,
 }
 
 struct devlink *
-devlinks_xa_find_get_first(struct net *net, unsigned long *indexp,
-			   xa_mark_t filter)
+devlinks_xa_find_get_first(struct net *net, unsigned long *indexp)
 {
-	return devlinks_xa_find_get(net, indexp, filter, xa_find);
+	return devlinks_xa_find_get(net, indexp, xa_find);
 }
 
 struct devlink *
-devlinks_xa_find_get_next(struct net *net, unsigned long *indexp,
-			  xa_mark_t filter)
+devlinks_xa_find_get_next(struct net *net, unsigned long *indexp)
 {
-	return devlinks_xa_find_get(net, indexp, filter, xa_find_after);
+	return devlinks_xa_find_get(net, indexp, xa_find_after);
 }
 
 /**
diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
index 91059311f18d..ee98f3bdcd33 100644
--- a/net/devlink/devl_internal.h
+++ b/net/devlink/devl_internal.h
@@ -81,20 +81,15 @@ extern struct genl_family devlink_nl_family;
  * devlink_put() needs to be called for each iterated devlink pointer
  * in loop body in order to release the reference.
  */
-#define devlinks_xa_for_each_get(net, index, devlink, filter)		\
-	for (index = 0,							\
-	     devlink = devlinks_xa_find_get_first(net, &index, filter);	\
-	     devlink; devlink = devlinks_xa_find_get_next(net, &index, filter))
-
 #define devlinks_xa_for_each_registered_get(net, index, devlink)	\
-	devlinks_xa_for_each_get(net, index, devlink, DEVLINK_REGISTERED)
+	for (index = 0,							\
+	     devlink = devlinks_xa_find_get_first(net, &index);	\
+	     devlink; devlink = devlinks_xa_find_get_next(net, &index))
 
 struct devlink *
-devlinks_xa_find_get_first(struct net *net, unsigned long *indexp,
-			   xa_mark_t filter);
+devlinks_xa_find_get_first(struct net *net, unsigned long *indexp);
 struct devlink *
-devlinks_xa_find_get_next(struct net *net, unsigned long *indexp,
-			  xa_mark_t filter);
+devlinks_xa_find_get_next(struct net *net, unsigned long *indexp);
 
 /* Netlink */
 #define DEVLINK_NL_FLAG_NEED_PORT		BIT(0)
-- 
2.38.1


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

* [PATCH net-next 08/14] devlink: health: combine loops in dump
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
                   ` (6 preceding siblings ...)
  2023-01-04  4:16 ` [PATCH net-next 07/14] devlink: drop the filter argument from devlinks_xa_find_get Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04 10:06   ` Jiri Pirko
  2023-01-04  4:16 ` [PATCH net-next 09/14] devlink: restart dump based on devlink instance ids (simple) Jakub Kicinski
                   ` (5 subsequent siblings)
  13 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

Walk devlink instances only once. Dump the instance reporters
and port reporters before moving to the next instance.
User space should not depend on ordering of messages.

This will make improving stability of the walk easier.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/leftover.c | 3 ---
 1 file changed, 3 deletions(-)

diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
index 9aac82dc7269..e3cfb64990b4 100644
--- a/net/devlink/leftover.c
+++ b/net/devlink/leftover.c
@@ -7940,10 +7940,7 @@ devlink_nl_cmd_health_reporter_get_dumpit(struct sk_buff *msg,
 			idx++;
 		}
 		mutex_unlock(&devlink->reporters_lock);
-		devlink_put(devlink);
-	}
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
 		devl_lock(devlink);
 		xa_for_each(&devlink->ports, port_index, port) {
 			mutex_lock(&port->reporters_lock);
-- 
2.38.1


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

* [PATCH net-next 09/14] devlink: restart dump based on devlink instance ids (simple)
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
                   ` (7 preceding siblings ...)
  2023-01-04  4:16 ` [PATCH net-next 08/14] devlink: health: combine loops in dump Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04 14:18   ` Jiri Pirko
  2023-01-04  4:16 ` [PATCH net-next 10/14] devlink: restart dump based on devlink instance ids (nested) Jakub Kicinski
                   ` (4 subsequent siblings)
  13 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

xarray gives each devlink instance an id and allows us to restart
walk based on that id quite neatly. This is nice both from the
perspective of code brevity and from the stability of the dump
(devlink instances disappearing from before the resumption point
will not cause inconsistent dumps).

This patch takes care of simple cases where dump->idx counts
devlink instances only.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/core.c          |  2 +-
 net/devlink/devl_internal.h | 14 ++++++++++++++
 net/devlink/leftover.c      | 36 ++++++++----------------------------
 3 files changed, 23 insertions(+), 29 deletions(-)

diff --git a/net/devlink/core.c b/net/devlink/core.c
index 3a99bf84632e..371d6821315d 100644
--- a/net/devlink/core.c
+++ b/net/devlink/core.c
@@ -91,7 +91,7 @@ void devlink_put(struct devlink *devlink)
 		call_rcu(&devlink->rcu, __devlink_put_rcu);
 }
 
-static struct devlink *
+struct devlink *
 devlinks_xa_find_get(struct net *net, unsigned long *indexp,
 		     void * (*xa_find_fn)(struct xarray *, unsigned long *,
 					  unsigned long, xa_mark_t))
diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
index ee98f3bdcd33..a567ff77601d 100644
--- a/net/devlink/devl_internal.h
+++ b/net/devlink/devl_internal.h
@@ -87,6 +87,10 @@ extern struct genl_family devlink_nl_family;
 	     devlink; devlink = devlinks_xa_find_get_next(net, &index))
 
 struct devlink *
+devlinks_xa_find_get(struct net *net, unsigned long *indexp,
+		     void * (*xa_find_fn)(struct xarray *, unsigned long *,
+					  unsigned long, xa_mark_t));
+struct devlink *
 devlinks_xa_find_get_first(struct net *net, unsigned long *indexp);
 struct devlink *
 devlinks_xa_find_get_next(struct net *net, unsigned long *indexp);
@@ -104,6 +108,7 @@ enum devlink_multicast_groups {
 
 /* state held across netlink dumps */
 struct devlink_nl_dump_state {
+	unsigned long instance;
 	int idx;
 	union {
 		/* DEVLINK_CMD_REGION_READ */
@@ -117,6 +122,15 @@ struct devlink_nl_dump_state {
 	};
 };
 
+/* Iterate over devlink pointers which were possible to get reference to.
+ * devlink_put() needs to be called for each iterated devlink pointer
+ * in loop body in order to release the reference.
+ */
+#define devlink_dump_for_each_instance_get(msg, dump, devlink)		\
+	for (; (devlink = devlinks_xa_find_get(sock_net(msg->sk),	\
+					       &dump->instance, xa_find)); \
+	     dump->instance++)
+
 extern const struct genl_small_ops devlink_nl_ops[56];
 
 struct devlink *devlink_get_from_attrs(struct net *net, struct nlattr **attrs);
diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
index e3cfb64990b4..0f24b321b0bb 100644
--- a/net/devlink/leftover.c
+++ b/net/devlink/leftover.c
@@ -1319,17 +1319,9 @@ static int devlink_nl_cmd_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	unsigned long index;
-	int idx = 0;
 	int err;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
-		if (idx < dump->idx) {
-			idx++;
-			devlink_put(devlink);
-			continue;
-		}
-
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
 		devl_lock(devlink);
 		err = devlink_nl_fill(msg, devlink, DEVLINK_CMD_NEW,
 				      NETLINK_CB(cb->skb).portid,
@@ -1339,10 +1331,8 @@ static int devlink_nl_cmd_get_dumpit(struct sk_buff *msg,
 
 		if (err)
 			goto out;
-		idx++;
 	}
 out:
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -4872,13 +4862,13 @@ static int devlink_nl_cmd_selftests_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	unsigned long index;
-	int idx = 0;
 	int err = 0;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
-		if (idx < dump->idx || !devlink->ops->selftest_check)
-			goto inc;
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		if (!devlink->ops->selftest_check) {
+			devlink_put(devlink);
+			continue;
+		}
 
 		devl_lock(devlink);
 		err = devlink_nl_selftests_fill(msg, devlink,
@@ -4890,15 +4880,13 @@ static int devlink_nl_cmd_selftests_get_dumpit(struct sk_buff *msg,
 			devlink_put(devlink);
 			break;
 		}
-inc:
-		idx++;
+
 		devlink_put(devlink);
 	}
 
 	if (err != -EMSGSIZE)
 		return err;
 
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -6747,14 +6735,9 @@ static int devlink_nl_cmd_info_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	unsigned long index;
-	int idx = 0;
 	int err = 0;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
-		if (idx < dump->idx)
-			goto inc;
-
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
 		devl_lock(devlink);
 		err = devlink_nl_info_fill(msg, devlink, DEVLINK_CMD_INFO_GET,
 					   NETLINK_CB(cb->skb).portid,
@@ -6767,15 +6750,12 @@ static int devlink_nl_cmd_info_get_dumpit(struct sk_buff *msg,
 			devlink_put(devlink);
 			break;
 		}
-inc:
-		idx++;
 		devlink_put(devlink);
 	}
 
 	if (err != -EMSGSIZE)
 		return err;
 
-	dump->idx = idx;
 	return msg->len;
 }
 
-- 
2.38.1


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

* [PATCH net-next 10/14] devlink: restart dump based on devlink instance ids (nested)
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
                   ` (8 preceding siblings ...)
  2023-01-04  4:16 ` [PATCH net-next 09/14] devlink: restart dump based on devlink instance ids (simple) Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04 15:47   ` Jiri Pirko
  2023-01-04  4:16 ` [PATCH net-next 11/14] devlink: restart dump based on devlink instance ids (function) Jakub Kicinski
                   ` (3 subsequent siblings)
  13 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

Use xarray id for cases of simple sub-object iteration.
We'll now use the dump->instance for the devlink instances
and dump->idx for subobject index.

Moving the definition of idx into the inner loop makes sense,
so while at it also move other sub-object local variables into
the loop.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/devl_internal.h |  2 +-
 net/devlink/leftover.c      | 93 +++++++++++++++++++------------------
 2 files changed, 49 insertions(+), 46 deletions(-)

diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
index a567ff77601d..5adac38454fd 100644
--- a/net/devlink/devl_internal.h
+++ b/net/devlink/devl_internal.h
@@ -129,7 +129,7 @@ struct devlink_nl_dump_state {
 #define devlink_dump_for_each_instance_get(msg, dump, devlink)		\
 	for (; (devlink = devlinks_xa_find_get(sock_net(msg->sk),	\
 					       &dump->instance, xa_find)); \
-	     dump->instance++)
+	     dump->instance++, dump->idx = 0)
 
 extern const struct genl_small_ops devlink_nl_ops[56];
 
diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
index 0f24b321b0bb..028a763feb50 100644
--- a/net/devlink/leftover.c
+++ b/net/devlink/leftover.c
@@ -1223,13 +1223,13 @@ static int devlink_nl_cmd_rate_get_dumpit(struct sk_buff *msg,
 					  struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink_rate *devlink_rate;
 	struct devlink *devlink;
-	unsigned long index;
-	int idx = 0;
 	int err = 0;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		struct devlink_rate *devlink_rate;
+		int idx = 0;
+
 		devl_lock(devlink);
 		list_for_each_entry(devlink_rate, &devlink->rate_list, list) {
 			enum devlink_command cmd = DEVLINK_CMD_RATE_NEW;
@@ -1245,6 +1245,7 @@ static int devlink_nl_cmd_rate_get_dumpit(struct sk_buff *msg,
 			if (err) {
 				devl_unlock(devlink);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 			idx++;
@@ -1256,7 +1257,6 @@ static int devlink_nl_cmd_rate_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -1363,12 +1363,13 @@ static int devlink_nl_cmd_port_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	struct devlink_port *devlink_port;
-	unsigned long index, port_index;
-	int idx = 0;
 	int err;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		struct devlink_port *devlink_port;
+		unsigned long port_index;
+		int idx = 0;
+
 		devl_lock(devlink);
 		xa_for_each(&devlink->ports, port_index, devlink_port) {
 			if (idx < dump->idx) {
@@ -1383,6 +1384,7 @@ static int devlink_nl_cmd_port_get_dumpit(struct sk_buff *msg,
 			if (err) {
 				devl_unlock(devlink);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 			idx++;
@@ -1391,7 +1393,6 @@ static int devlink_nl_cmd_port_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -2143,11 +2144,11 @@ static int devlink_nl_cmd_linecard_get_dumpit(struct sk_buff *msg,
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_linecard *linecard;
 	struct devlink *devlink;
-	unsigned long index;
-	int idx = 0;
 	int err;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		int idx = 0;
+
 		mutex_lock(&devlink->linecards_lock);
 		list_for_each_entry(linecard, &devlink->linecard_list, list) {
 			if (idx < dump->idx) {
@@ -2165,6 +2166,7 @@ static int devlink_nl_cmd_linecard_get_dumpit(struct sk_buff *msg,
 			if (err) {
 				mutex_unlock(&devlink->linecards_lock);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 			idx++;
@@ -2173,7 +2175,6 @@ static int devlink_nl_cmd_linecard_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -2404,12 +2405,12 @@ static int devlink_nl_cmd_sb_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	struct devlink_sb *devlink_sb;
-	unsigned long index;
-	int idx = 0;
 	int err;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		struct devlink_sb *devlink_sb;
+		int idx = 0;
+
 		devl_lock(devlink);
 		list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
 			if (idx < dump->idx) {
@@ -2424,6 +2425,7 @@ static int devlink_nl_cmd_sb_get_dumpit(struct sk_buff *msg,
 			if (err) {
 				devl_unlock(devlink);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 			idx++;
@@ -2432,7 +2434,6 @@ static int devlink_nl_cmd_sb_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -5339,13 +5340,13 @@ static int devlink_nl_cmd_param_get_dumpit(struct sk_buff *msg,
 					   struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink_param_item *param_item;
 	struct devlink *devlink;
-	unsigned long index;
-	int idx = 0;
 	int err = 0;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		struct devlink_param_item *param_item;
+		int idx = 0;
+
 		devl_lock(devlink);
 		list_for_each_entry(param_item, &devlink->param_list, list) {
 			if (idx < dump->idx) {
@@ -5362,6 +5363,7 @@ static int devlink_nl_cmd_param_get_dumpit(struct sk_buff *msg,
 			} else if (err) {
 				devl_unlock(devlink);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 			idx++;
@@ -5373,7 +5375,6 @@ static int devlink_nl_cmd_param_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -7893,14 +7894,15 @@ devlink_nl_cmd_health_reporter_get_dumpit(struct sk_buff *msg,
 					  struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink_health_reporter *reporter;
-	unsigned long index, port_index;
-	struct devlink_port *port;
 	struct devlink *devlink;
-	int idx = 0;
 	int err;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		struct devlink_health_reporter *reporter;
+		struct devlink_port *port;
+		unsigned long port_index;
+		int idx = 0;
+
 		mutex_lock(&devlink->reporters_lock);
 		list_for_each_entry(reporter, &devlink->reporter_list,
 				    list) {
@@ -7915,6 +7917,7 @@ devlink_nl_cmd_health_reporter_get_dumpit(struct sk_buff *msg,
 			if (err) {
 				mutex_unlock(&devlink->reporters_lock);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 			idx++;
@@ -7938,6 +7941,7 @@ devlink_nl_cmd_health_reporter_get_dumpit(struct sk_buff *msg,
 					mutex_unlock(&port->reporters_lock);
 					devl_unlock(devlink);
 					devlink_put(devlink);
+					dump->idx = idx;
 					goto out;
 				}
 				idx++;
@@ -7948,7 +7952,6 @@ devlink_nl_cmd_health_reporter_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -8474,13 +8477,13 @@ static int devlink_nl_cmd_trap_get_dumpit(struct sk_buff *msg,
 					  struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink_trap_item *trap_item;
 	struct devlink *devlink;
-	unsigned long index;
-	int idx = 0;
 	int err;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		struct devlink_trap_item *trap_item;
+		int idx = 0;
+
 		devl_lock(devlink);
 		list_for_each_entry(trap_item, &devlink->trap_list, list) {
 			if (idx < dump->idx) {
@@ -8495,6 +8498,7 @@ static int devlink_nl_cmd_trap_get_dumpit(struct sk_buff *msg,
 			if (err) {
 				devl_unlock(devlink);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 			idx++;
@@ -8503,7 +8507,6 @@ static int devlink_nl_cmd_trap_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -8690,14 +8693,14 @@ static int devlink_nl_cmd_trap_group_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	enum devlink_command cmd = DEVLINK_CMD_TRAP_GROUP_NEW;
-	struct devlink_trap_group_item *group_item;
 	u32 portid = NETLINK_CB(cb->skb).portid;
 	struct devlink *devlink;
-	unsigned long index;
-	int idx = 0;
 	int err;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		struct devlink_trap_group_item *group_item;
+		int idx = 0;
+
 		devl_lock(devlink);
 		list_for_each_entry(group_item, &devlink->trap_group_list,
 				    list) {
@@ -8713,6 +8716,7 @@ static int devlink_nl_cmd_trap_group_get_dumpit(struct sk_buff *msg,
 			if (err) {
 				devl_unlock(devlink);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 			idx++;
@@ -8721,7 +8725,6 @@ static int devlink_nl_cmd_trap_group_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -8994,14 +8997,14 @@ static int devlink_nl_cmd_trap_policer_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	enum devlink_command cmd = DEVLINK_CMD_TRAP_POLICER_NEW;
-	struct devlink_trap_policer_item *policer_item;
 	u32 portid = NETLINK_CB(cb->skb).portid;
 	struct devlink *devlink;
-	unsigned long index;
-	int idx = 0;
 	int err;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		struct devlink_trap_policer_item *policer_item;
+		int idx = 0;
+
 		devl_lock(devlink);
 		list_for_each_entry(policer_item, &devlink->trap_policer_list,
 				    list) {
@@ -9017,6 +9020,7 @@ static int devlink_nl_cmd_trap_policer_get_dumpit(struct sk_buff *msg,
 			if (err) {
 				devl_unlock(devlink);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 			idx++;
@@ -9025,7 +9029,6 @@ static int devlink_nl_cmd_trap_policer_get_dumpit(struct sk_buff *msg,
 		devlink_put(devlink);
 	}
 out:
-	dump->idx = idx;
 	return msg->len;
 }
 
-- 
2.38.1


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

* [PATCH net-next 11/14] devlink: restart dump based on devlink instance ids (function)
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
                   ` (9 preceding siblings ...)
  2023-01-04  4:16 ` [PATCH net-next 10/14] devlink: restart dump based on devlink instance ids (nested) Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04 15:51   ` Jiri Pirko
  2023-01-04  4:16 ` [PATCH net-next 12/14] devlink: uniformly take the devlink instance lock in the dump loop Jakub Kicinski
                   ` (2 subsequent siblings)
  13 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

Use xarray id for cases of sub-objects which are iterated in
a function.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/leftover.c | 41 +++++++++++++++++++++--------------------
 1 file changed, 21 insertions(+), 20 deletions(-)

diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
index 028a763feb50..d01089b65ddc 100644
--- a/net/devlink/leftover.c
+++ b/net/devlink/leftover.c
@@ -2547,12 +2547,12 @@ static int devlink_nl_cmd_sb_pool_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	struct devlink_sb *devlink_sb;
-	unsigned long index;
-	int idx = 0;
 	int err = 0;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		struct devlink_sb *devlink_sb;
+		int idx = 0;
+
 		if (!devlink->ops->sb_pool_get)
 			goto retry;
 
@@ -2567,6 +2567,7 @@ static int devlink_nl_cmd_sb_pool_get_dumpit(struct sk_buff *msg,
 			} else if (err) {
 				devl_unlock(devlink);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 		}
@@ -2578,7 +2579,6 @@ static int devlink_nl_cmd_sb_pool_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -2762,12 +2762,12 @@ static int devlink_nl_cmd_sb_port_pool_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	struct devlink_sb *devlink_sb;
-	unsigned long index;
-	int idx = 0;
 	int err = 0;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		struct devlink_sb *devlink_sb;
+		int idx = 0;
+
 		if (!devlink->ops->sb_port_pool_get)
 			goto retry;
 
@@ -2782,6 +2782,7 @@ static int devlink_nl_cmd_sb_port_pool_get_dumpit(struct sk_buff *msg,
 			} else if (err) {
 				devl_unlock(devlink);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 		}
@@ -2793,7 +2794,6 @@ static int devlink_nl_cmd_sb_port_pool_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -3005,12 +3005,12 @@ devlink_nl_cmd_sb_tc_pool_bind_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	struct devlink_sb *devlink_sb;
-	unsigned long index;
-	int idx = 0;
 	int err = 0;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		struct devlink_sb *devlink_sb;
+		int idx = 0;
+
 		if (!devlink->ops->sb_tc_pool_bind_get)
 			goto retry;
 
@@ -3025,6 +3025,7 @@ devlink_nl_cmd_sb_tc_pool_bind_get_dumpit(struct sk_buff *msg,
 			} else if (err) {
 				devl_unlock(devlink);
 				devlink_put(devlink);
+				dump->idx = idx;
 				goto out;
 			}
 		}
@@ -3036,7 +3037,6 @@ devlink_nl_cmd_sb_tc_pool_bind_get_dumpit(struct sk_buff *msg,
 	if (err != -EMSGSIZE)
 		return err;
 
-	dump->idx = idx;
 	return msg->len;
 }
 
@@ -6085,19 +6085,20 @@ static int devlink_nl_cmd_region_get_dumpit(struct sk_buff *msg,
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink *devlink;
-	unsigned long index;
-	int idx = 0;
 	int err = 0;
 
-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		int idx = 0;
+
 		err = devlink_nl_cmd_region_get_devlink_dumpit(msg, cb, devlink,
 							       &idx, dump->idx);
 		devlink_put(devlink);
-		if (err)
+		if (err) {
+			dump->idx = idx;
 			goto out;
+		}
 	}
 out:
-	dump->idx = idx;
 	return msg->len;
 }
 
-- 
2.38.1


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

* [PATCH net-next 12/14] devlink: uniformly take the devlink instance lock in the dump loop
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
                   ` (10 preceding siblings ...)
  2023-01-04  4:16 ` [PATCH net-next 11/14] devlink: restart dump based on devlink instance ids (function) Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04 15:52   ` Jiri Pirko
  2023-01-04  4:16 ` [PATCH net-next 13/14] devlink: add by-instance dump infra Jakub Kicinski
  2023-01-04  4:16 ` [PATCH net-next 14/14] devlink: convert remaining dumps to the by-instance scheme Jakub Kicinski
  13 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

Move the lock taking out of devlink_nl_cmd_region_get_devlink_dumpit().
This way all dumps will take the instance lock in the main iteration
loop directly, making refactoring and reading the code easier.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/leftover.c | 13 ++++++-------
 1 file changed, 6 insertions(+), 7 deletions(-)

diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
index d01089b65ddc..c6ad8133fc23 100644
--- a/net/devlink/leftover.c
+++ b/net/devlink/leftover.c
@@ -6050,9 +6050,8 @@ static int devlink_nl_cmd_region_get_devlink_dumpit(struct sk_buff *msg,
 	struct devlink_region *region;
 	struct devlink_port *port;
 	unsigned long port_index;
-	int err = 0;
+	int err;
 
-	devl_lock(devlink);
 	list_for_each_entry(region, &devlink->region_list, list) {
 		if (*idx < start) {
 			(*idx)++;
@@ -6064,7 +6063,7 @@ static int devlink_nl_cmd_region_get_devlink_dumpit(struct sk_buff *msg,
 					     cb->nlh->nlmsg_seq,
 					     NLM_F_MULTI, region);
 		if (err)
-			goto out;
+			return err;
 		(*idx)++;
 	}
 
@@ -6072,12 +6071,10 @@ static int devlink_nl_cmd_region_get_devlink_dumpit(struct sk_buff *msg,
 		err = devlink_nl_cmd_region_get_port_dumpit(msg, cb, port, idx,
 							    start);
 		if (err)
-			goto out;
+			return err;
 	}
 
-out:
-	devl_unlock(devlink);
-	return err;
+	return 0;
 }
 
 static int devlink_nl_cmd_region_get_dumpit(struct sk_buff *msg,
@@ -6090,8 +6087,10 @@ static int devlink_nl_cmd_region_get_dumpit(struct sk_buff *msg,
 	devlink_dump_for_each_instance_get(msg, dump, devlink) {
 		int idx = 0;
 
+		devl_lock(devlink);
 		err = devlink_nl_cmd_region_get_devlink_dumpit(msg, cb, devlink,
 							       &idx, dump->idx);
+		devl_unlock(devlink);
 		devlink_put(devlink);
 		if (err) {
 			dump->idx = idx;
-- 
2.38.1


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

* [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
                   ` (11 preceding siblings ...)
  2023-01-04  4:16 ` [PATCH net-next 12/14] devlink: uniformly take the devlink instance lock in the dump loop Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  2023-01-04 16:50   ` Jiri Pirko
  2023-01-04  4:16 ` [PATCH net-next 14/14] devlink: convert remaining dumps to the by-instance scheme Jakub Kicinski
  13 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

Most dumpit implementations walk the devlink instances.
This requires careful lock taking and reference dropping.
Factor the loop out and provide just a callback to handle
a single instance dump.

Convert one user as an example, other users converted
in the next change.

Slightly inspired by ethtool netlink code.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/devl_internal.h | 10 +++++++
 net/devlink/leftover.c      | 55 ++++++++++++++++---------------------
 net/devlink/netlink.c       | 33 ++++++++++++++++++++++
 3 files changed, 67 insertions(+), 31 deletions(-)

diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
index 5adac38454fd..e49b82dd77cd 100644
--- a/net/devlink/devl_internal.h
+++ b/net/devlink/devl_internal.h
@@ -122,6 +122,11 @@ struct devlink_nl_dump_state {
 	};
 };
 
+struct devlink_gen_cmd {
+	int (*dump_one)(struct sk_buff *msg, struct devlink *devlink,
+			struct netlink_callback *cb);
+};
+
 /* Iterate over devlink pointers which were possible to get reference to.
  * devlink_put() needs to be called for each iterated devlink pointer
  * in loop body in order to release the reference.
@@ -138,6 +143,9 @@ struct devlink *devlink_get_from_attrs(struct net *net, struct nlattr **attrs);
 void devlink_notify_unregister(struct devlink *devlink);
 void devlink_notify_register(struct devlink *devlink);
 
+int devlink_instance_iter_dump(struct sk_buff *msg,
+			       struct netlink_callback *cb);
+
 static inline struct devlink_nl_dump_state *
 devl_dump_state(struct netlink_callback *cb)
 {
@@ -173,6 +181,8 @@ devlink_linecard_get_from_info(struct devlink *devlink, struct genl_info *info);
 void devlink_linecard_put(struct devlink_linecard *linecard);
 
 /* Rates */
+extern const struct devlink_gen_cmd devl_gen_rate_get;
+
 struct devlink_rate *
 devlink_rate_get_from_info(struct devlink *devlink, struct genl_info *info);
 struct devlink_rate *
diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
index c6ad8133fc23..f18d8dcf9751 100644
--- a/net/devlink/leftover.c
+++ b/net/devlink/leftover.c
@@ -1219,47 +1219,40 @@ static void devlink_rate_notify(struct devlink_rate *devlink_rate,
 				0, DEVLINK_MCGRP_CONFIG, GFP_KERNEL);
 }
 
-static int devlink_nl_cmd_rate_get_dumpit(struct sk_buff *msg,
-					  struct netlink_callback *cb)
+static int
+devlink_nl_cmd_rate_get_dumpinst(struct sk_buff *msg, struct devlink *devlink,
+				 struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
+	struct devlink_rate *devlink_rate;
+	int idx = 0;
 	int err = 0;
 
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		struct devlink_rate *devlink_rate;
-		int idx = 0;
-
-		devl_lock(devlink);
-		list_for_each_entry(devlink_rate, &devlink->rate_list, list) {
-			enum devlink_command cmd = DEVLINK_CMD_RATE_NEW;
-			u32 id = NETLINK_CB(cb->skb).portid;
+	list_for_each_entry(devlink_rate, &devlink->rate_list, list) {
+		enum devlink_command cmd = DEVLINK_CMD_RATE_NEW;
+		u32 id = NETLINK_CB(cb->skb).portid;
 
-			if (idx < dump->idx) {
-				idx++;
-				continue;
-			}
-			err = devlink_nl_rate_fill(msg, devlink_rate, cmd, id,
-						   cb->nlh->nlmsg_seq,
-						   NLM_F_MULTI, NULL);
-			if (err) {
-				devl_unlock(devlink);
-				devlink_put(devlink);
-				dump->idx = idx;
-				goto out;
-			}
+		if (idx < dump->idx) {
 			idx++;
+			continue;
 		}
-		devl_unlock(devlink);
-		devlink_put(devlink);
+		err = devlink_nl_rate_fill(msg, devlink_rate, cmd, id,
+					   cb->nlh->nlmsg_seq,
+					   NLM_F_MULTI, NULL);
+		if (err) {
+			dump->idx = idx;
+			break;
+		}
+		idx++;
 	}
-out:
-	if (err != -EMSGSIZE)
-		return err;
 
-	return msg->len;
+	return err;
 }
 
+const struct devlink_gen_cmd devl_gen_rate_get = {
+	.dump_one		= devlink_nl_cmd_rate_get_dumpinst,
+};
+
 static int devlink_nl_cmd_rate_get_doit(struct sk_buff *skb,
 					struct genl_info *info)
 {
@@ -9130,7 +9123,7 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 	{
 		.cmd = DEVLINK_CMD_RATE_GET,
 		.doit = devlink_nl_cmd_rate_get_doit,
-		.dumpit = devlink_nl_cmd_rate_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		.internal_flags = DEVLINK_NL_FLAG_NEED_RATE,
 		/* can be retrieved by unprivileged users */
 	},
diff --git a/net/devlink/netlink.c b/net/devlink/netlink.c
index ce1a7d674d14..fcf10c288480 100644
--- a/net/devlink/netlink.c
+++ b/net/devlink/netlink.c
@@ -5,6 +5,7 @@
  */
 
 #include <net/genetlink.h>
+#include <net/sock.h>
 
 #include "devl_internal.h"
 
@@ -177,6 +178,38 @@ static void devlink_nl_post_doit(const struct genl_split_ops *ops,
 	devlink_put(devlink);
 }
 
+static const struct devlink_gen_cmd *devl_gen_cmds[] = {
+	[DEVLINK_CMD_RATE_GET]		= &devl_gen_rate_get,
+};
+
+int devlink_instance_iter_dump(struct sk_buff *msg, struct netlink_callback *cb)
+{
+	const struct genl_dumpit_info *info = genl_dumpit_info(cb);
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
+	const struct devlink_gen_cmd *cmd;
+	struct devlink *devlink;
+	int err = 0;
+
+	cmd = devl_gen_cmds[info->op.cmd];
+
+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
+		devl_lock(devlink);
+		err = cmd->dump_one(msg, devlink, cb);
+		devl_unlock(devlink);
+		devlink_put(devlink);
+
+		if (err)
+			break;
+
+		/* restart sub-object walk for the next instance */
+		dump->idx = 0;
+	}
+
+	if (err != -EMSGSIZE)
+		return err;
+	return msg->len;
+}
+
 struct genl_family devlink_nl_family __ro_after_init = {
 	.name		= DEVLINK_GENL_NAME,
 	.version	= DEVLINK_GENL_VERSION,
-- 
2.38.1


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

* [PATCH net-next 14/14] devlink: convert remaining dumps to the by-instance scheme
  2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
                   ` (12 preceding siblings ...)
  2023-01-04  4:16 ` [PATCH net-next 13/14] devlink: add by-instance dump infra Jakub Kicinski
@ 2023-01-04  4:16 ` Jakub Kicinski
  13 siblings, 0 replies; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-04  4:16 UTC (permalink / raw)
  To: davem; +Cc: netdev, edumazet, pabeni, jacob.e.keller, jiri, Jakub Kicinski

Soon we'll have to check if a devlink instance is alive after
locking it. Convert to the by-instance dumping scheme to make
refactoring easier.

Most of the subobject code no longer has to worry about any devlink
locking / lifetime rules (the only ones that still do are the two subject
types which stubbornly use their own locking). Both dump and do callbacks
are given a devlink instance which is already locked and good-to-access
(do from the .pre_doit handler, dump from the new dump indirection).

Note that we'll now check presence of an op (e.g. for sb_pool_get)
under the devlink instance lock, that will soon be necessary anyway,
because we don't hold refs on the driver modules so the memory
in which ops live may be gone for a dead instance, after upcoming
locking changes.

Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
Signed-off-by: Jakub Kicinski <kuba@kernel.org>
---
 net/devlink/devl_internal.h |  15 +
 net/devlink/leftover.c      | 686 +++++++++++++++---------------------
 net/devlink/netlink.c       |  13 +
 3 files changed, 320 insertions(+), 394 deletions(-)

diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
index e49b82dd77cd..1d7ab11f2f7e 100644
--- a/net/devlink/devl_internal.h
+++ b/net/devlink/devl_internal.h
@@ -154,6 +154,21 @@ devl_dump_state(struct netlink_callback *cb)
 	return (struct devlink_nl_dump_state *)cb->ctx;
 }
 
+/* gen cmds */
+extern const struct devlink_gen_cmd devl_gen_inst;
+extern const struct devlink_gen_cmd devl_gen_port;
+extern const struct devlink_gen_cmd devl_gen_sb;
+extern const struct devlink_gen_cmd devl_gen_sb_pool;
+extern const struct devlink_gen_cmd devl_gen_sb_port_pool;
+extern const struct devlink_gen_cmd devl_gen_sb_tc_pool_bind;
+extern const struct devlink_gen_cmd devl_gen_selftests;
+extern const struct devlink_gen_cmd devl_gen_param;
+extern const struct devlink_gen_cmd devl_gen_region;
+extern const struct devlink_gen_cmd devl_gen_info;
+extern const struct devlink_gen_cmd devl_gen_trap;
+extern const struct devlink_gen_cmd devl_gen_trap_group;
+extern const struct devlink_gen_cmd devl_gen_trap_policer;
+
 /* Ports */
 int devlink_port_netdevice_event(struct notifier_block *nb,
 				 unsigned long event, void *ptr);
diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
index f18d8dcf9751..ceac4343698b 100644
--- a/net/devlink/leftover.c
+++ b/net/devlink/leftover.c
@@ -1307,28 +1307,19 @@ static int devlink_nl_cmd_get_doit(struct sk_buff *skb, struct genl_info *info)
 	return genlmsg_reply(msg, info);
 }
 
-static int devlink_nl_cmd_get_dumpit(struct sk_buff *msg,
-				     struct netlink_callback *cb)
+static int
+devlink_nl_cmd_get_dumpinst(struct sk_buff *msg, struct devlink *devlink,
+			    struct netlink_callback *cb)
 {
-	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
-	int err;
-
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		devl_lock(devlink);
-		err = devlink_nl_fill(msg, devlink, DEVLINK_CMD_NEW,
-				      NETLINK_CB(cb->skb).portid,
-				      cb->nlh->nlmsg_seq, NLM_F_MULTI);
-		devl_unlock(devlink);
-		devlink_put(devlink);
-
-		if (err)
-			goto out;
-	}
-out:
-	return msg->len;
+	return devlink_nl_fill(msg, devlink, DEVLINK_CMD_NEW,
+			       NETLINK_CB(cb->skb).portid,
+			       cb->nlh->nlmsg_seq, NLM_F_MULTI);
 }
 
+const struct devlink_gen_cmd devl_gen_inst = {
+	.dump_one		= devlink_nl_cmd_get_dumpinst,
+};
+
 static int devlink_nl_cmd_port_get_doit(struct sk_buff *skb,
 					struct genl_info *info)
 {
@@ -1351,44 +1342,40 @@ static int devlink_nl_cmd_port_get_doit(struct sk_buff *skb,
 	return genlmsg_reply(msg, info);
 }
 
-static int devlink_nl_cmd_port_get_dumpit(struct sk_buff *msg,
-					  struct netlink_callback *cb)
+static int
+devlink_nl_cmd_port_get_dumpinst(struct sk_buff *msg, struct devlink *devlink,
+				 struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
-	int err;
-
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		struct devlink_port *devlink_port;
-		unsigned long port_index;
-		int idx = 0;
+	struct devlink_port *devlink_port;
+	unsigned long port_index;
+	int idx = 0;
+	int err = 0;
 
-		devl_lock(devlink);
-		xa_for_each(&devlink->ports, port_index, devlink_port) {
-			if (idx < dump->idx) {
-				idx++;
-				continue;
-			}
-			err = devlink_nl_port_fill(msg, devlink_port,
-						   DEVLINK_CMD_NEW,
-						   NETLINK_CB(cb->skb).portid,
-						   cb->nlh->nlmsg_seq,
-						   NLM_F_MULTI, cb->extack);
-			if (err) {
-				devl_unlock(devlink);
-				devlink_put(devlink);
-				dump->idx = idx;
-				goto out;
-			}
+	xa_for_each(&devlink->ports, port_index, devlink_port) {
+		if (idx < dump->idx) {
 			idx++;
+			continue;
 		}
-		devl_unlock(devlink);
-		devlink_put(devlink);
+		err = devlink_nl_port_fill(msg, devlink_port,
+					   DEVLINK_CMD_NEW,
+					   NETLINK_CB(cb->skb).portid,
+					   cb->nlh->nlmsg_seq,
+					   NLM_F_MULTI, cb->extack);
+		if (err) {
+			dump->idx = idx;
+			break;
+		}
+		idx++;
 	}
-out:
-	return msg->len;
+
+	return err;
 }
 
+const struct devlink_gen_cmd devl_gen_port = {
+	.dump_one		= devlink_nl_cmd_port_get_dumpinst,
+};
+
 static int devlink_port_type_set(struct devlink_port *devlink_port,
 				 enum devlink_port_type port_type)
 
@@ -2393,43 +2380,39 @@ static int devlink_nl_cmd_sb_get_doit(struct sk_buff *skb,
 	return genlmsg_reply(msg, info);
 }
 
-static int devlink_nl_cmd_sb_get_dumpit(struct sk_buff *msg,
-					struct netlink_callback *cb)
+static int
+devlink_nl_cmd_sb_get_dumpinst(struct sk_buff *msg, struct devlink *devlink,
+			       struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
-	int err;
-
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		struct devlink_sb *devlink_sb;
-		int idx = 0;
+	struct devlink_sb *devlink_sb;
+	int idx = 0;
+	int err = 0;
 
-		devl_lock(devlink);
-		list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
-			if (idx < dump->idx) {
-				idx++;
-				continue;
-			}
-			err = devlink_nl_sb_fill(msg, devlink, devlink_sb,
-						 DEVLINK_CMD_SB_NEW,
-						 NETLINK_CB(cb->skb).portid,
-						 cb->nlh->nlmsg_seq,
-						 NLM_F_MULTI);
-			if (err) {
-				devl_unlock(devlink);
-				devlink_put(devlink);
-				dump->idx = idx;
-				goto out;
-			}
+	list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
+		if (idx < dump->idx) {
 			idx++;
+			continue;
 		}
-		devl_unlock(devlink);
-		devlink_put(devlink);
+		err = devlink_nl_sb_fill(msg, devlink, devlink_sb,
+					 DEVLINK_CMD_SB_NEW,
+					 NETLINK_CB(cb->skb).portid,
+					 cb->nlh->nlmsg_seq,
+					 NLM_F_MULTI);
+		if (err) {
+			dump->idx = idx;
+			break;
+		}
+		idx++;
 	}
-out:
-	return msg->len;
+
+	return err;
 }
 
+const struct devlink_gen_cmd devl_gen_sb = {
+	.dump_one		= devlink_nl_cmd_sb_get_dumpinst,
+};
+
 static int devlink_nl_sb_pool_fill(struct sk_buff *msg, struct devlink *devlink,
 				   struct devlink_sb *devlink_sb,
 				   u16 pool_index, enum devlink_command cmd,
@@ -2535,46 +2518,39 @@ static int __sb_pool_get_dumpit(struct sk_buff *msg, int start, int *p_idx,
 	return 0;
 }
 
-static int devlink_nl_cmd_sb_pool_get_dumpit(struct sk_buff *msg,
-					     struct netlink_callback *cb)
+static int
+devlink_nl_cmd_sb_pool_get_dumpinst(struct sk_buff *msg,
+				    struct devlink *devlink,
+				    struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
+	struct devlink_sb *devlink_sb;
 	int err = 0;
+	int idx = 0;
 
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		struct devlink_sb *devlink_sb;
-		int idx = 0;
-
-		if (!devlink->ops->sb_pool_get)
-			goto retry;
+	if (!devlink->ops->sb_pool_get)
+		return 0;
 
-		devl_lock(devlink);
-		list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
-			err = __sb_pool_get_dumpit(msg, dump->idx, &idx,
-						   devlink, devlink_sb,
-						   NETLINK_CB(cb->skb).portid,
-						   cb->nlh->nlmsg_seq);
-			if (err == -EOPNOTSUPP) {
-				err = 0;
-			} else if (err) {
-				devl_unlock(devlink);
-				devlink_put(devlink);
-				dump->idx = idx;
-				goto out;
-			}
+	list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
+		err = __sb_pool_get_dumpit(msg, dump->idx, &idx,
+					   devlink, devlink_sb,
+					   NETLINK_CB(cb->skb).portid,
+					   cb->nlh->nlmsg_seq);
+		if (err == -EOPNOTSUPP) {
+			err = 0;
+		} else if (err) {
+			dump->idx = idx;
+			break;
 		}
-		devl_unlock(devlink);
-retry:
-		devlink_put(devlink);
 	}
-out:
-	if (err != -EMSGSIZE)
-		return err;
 
-	return msg->len;
+	return err;
 }
 
+const struct devlink_gen_cmd devl_gen_sb_pool = {
+	.dump_one		= devlink_nl_cmd_sb_pool_get_dumpinst,
+};
+
 static int devlink_sb_pool_set(struct devlink *devlink, unsigned int sb_index,
 			       u16 pool_index, u32 size,
 			       enum devlink_sb_threshold_type threshold_type,
@@ -2750,46 +2726,39 @@ static int __sb_port_pool_get_dumpit(struct sk_buff *msg, int start, int *p_idx,
 	return 0;
 }
 
-static int devlink_nl_cmd_sb_port_pool_get_dumpit(struct sk_buff *msg,
-						  struct netlink_callback *cb)
+static int
+devlink_nl_cmd_sb_port_pool_get_dumpinst(struct sk_buff *msg,
+					 struct devlink *devlink,
+					 struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
+	struct devlink_sb *devlink_sb;
+	int idx = 0;
 	int err = 0;
 
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		struct devlink_sb *devlink_sb;
-		int idx = 0;
-
-		if (!devlink->ops->sb_port_pool_get)
-			goto retry;
+	if (!devlink->ops->sb_port_pool_get)
+		return 0;
 
-		devl_lock(devlink);
-		list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
-			err = __sb_port_pool_get_dumpit(msg, dump->idx, &idx,
-							devlink, devlink_sb,
-							NETLINK_CB(cb->skb).portid,
-							cb->nlh->nlmsg_seq);
-			if (err == -EOPNOTSUPP) {
-				err = 0;
-			} else if (err) {
-				devl_unlock(devlink);
-				devlink_put(devlink);
-				dump->idx = idx;
-				goto out;
-			}
+	list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
+		err = __sb_port_pool_get_dumpit(msg, dump->idx, &idx,
+						devlink, devlink_sb,
+						NETLINK_CB(cb->skb).portid,
+						cb->nlh->nlmsg_seq);
+		if (err == -EOPNOTSUPP) {
+			err = 0;
+		} else if (err) {
+			dump->idx = idx;
+			break;
 		}
-		devl_unlock(devlink);
-retry:
-		devlink_put(devlink);
 	}
-out:
-	if (err != -EMSGSIZE)
-		return err;
 
-	return msg->len;
+	return err;
 }
 
+const struct devlink_gen_cmd devl_gen_sb_port_pool = {
+	.dump_one		= devlink_nl_cmd_sb_port_pool_get_dumpinst,
+};
+
 static int devlink_sb_port_pool_set(struct devlink_port *devlink_port,
 				    unsigned int sb_index, u16 pool_index,
 				    u32 threshold,
@@ -2993,46 +2962,38 @@ static int __sb_tc_pool_bind_get_dumpit(struct sk_buff *msg,
 }
 
 static int
-devlink_nl_cmd_sb_tc_pool_bind_get_dumpit(struct sk_buff *msg,
-					  struct netlink_callback *cb)
+devlink_nl_cmd_sb_tc_pool_bind_get_dumpinst(struct sk_buff *msg,
+					    struct devlink *devlink,
+					    struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
+	struct devlink_sb *devlink_sb;
+	int idx = 0;
 	int err = 0;
 
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		struct devlink_sb *devlink_sb;
-		int idx = 0;
-
-		if (!devlink->ops->sb_tc_pool_bind_get)
-			goto retry;
+	if (!devlink->ops->sb_tc_pool_bind_get)
+		return 0;
 
-		devl_lock(devlink);
-		list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
-			err = __sb_tc_pool_bind_get_dumpit(msg, dump->idx, &idx,
-							   devlink, devlink_sb,
-							   NETLINK_CB(cb->skb).portid,
-							   cb->nlh->nlmsg_seq);
-			if (err == -EOPNOTSUPP) {
-				err = 0;
-			} else if (err) {
-				devl_unlock(devlink);
-				devlink_put(devlink);
-				dump->idx = idx;
-				goto out;
-			}
+	list_for_each_entry(devlink_sb, &devlink->sb_list, list) {
+		err = __sb_tc_pool_bind_get_dumpit(msg, dump->idx, &idx,
+						   devlink, devlink_sb,
+						   NETLINK_CB(cb->skb).portid,
+						   cb->nlh->nlmsg_seq);
+		if (err == -EOPNOTSUPP) {
+			err = 0;
+		} else if (err) {
+			dump->idx = idx;
+			break;
 		}
-		devl_unlock(devlink);
-retry:
-		devlink_put(devlink);
 	}
-out:
-	if (err != -EMSGSIZE)
-		return err;
 
-	return msg->len;
+	return err;
 }
 
+const struct devlink_gen_cmd devl_gen_sb_tc_pool_bind = {
+	.dump_one		= devlink_nl_cmd_sb_tc_pool_bind_get_dumpinst,
+};
+
 static int devlink_sb_tc_pool_bind_set(struct devlink_port *devlink_port,
 				       unsigned int sb_index, u16 tc_index,
 				       enum devlink_sb_pool_type pool_type,
@@ -4851,39 +4812,24 @@ static int devlink_nl_cmd_selftests_get_doit(struct sk_buff *skb,
 	return genlmsg_reply(msg, info);
 }
 
-static int devlink_nl_cmd_selftests_get_dumpit(struct sk_buff *msg,
-					       struct netlink_callback *cb)
+static int
+devlink_nl_cmd_selftests_get_dumpinst(struct sk_buff *msg,
+				      struct devlink *devlink,
+				      struct netlink_callback *cb)
 {
-	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
-	int err = 0;
-
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		if (!devlink->ops->selftest_check) {
-			devlink_put(devlink);
-			continue;
-		}
-
-		devl_lock(devlink);
-		err = devlink_nl_selftests_fill(msg, devlink,
-						NETLINK_CB(cb->skb).portid,
-						cb->nlh->nlmsg_seq, NLM_F_MULTI,
-						cb->extack);
-		devl_unlock(devlink);
-		if (err) {
-			devlink_put(devlink);
-			break;
-		}
-
-		devlink_put(devlink);
-	}
-
-	if (err != -EMSGSIZE)
-		return err;
+	if (!devlink->ops->selftest_check)
+		return 0;
 
-	return msg->len;
+	return devlink_nl_selftests_fill(msg, devlink,
+					 NETLINK_CB(cb->skb).portid,
+					 cb->nlh->nlmsg_seq, NLM_F_MULTI,
+					 cb->extack);
 }
 
+const struct devlink_gen_cmd devl_gen_selftests = {
+	.dump_one		= devlink_nl_cmd_selftests_get_dumpinst,
+};
+
 static int devlink_selftest_result_put(struct sk_buff *skb, unsigned int id,
 				       enum devlink_selftest_status test_status)
 {
@@ -5329,48 +5275,41 @@ static void devlink_param_notify(struct devlink *devlink,
 				msg, 0, DEVLINK_MCGRP_CONFIG, GFP_KERNEL);
 }
 
-static int devlink_nl_cmd_param_get_dumpit(struct sk_buff *msg,
-					   struct netlink_callback *cb)
+static int
+devlink_nl_cmd_param_get_dumpinst(struct sk_buff *msg, struct devlink *devlink,
+				  struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
+	struct devlink_param_item *param_item;
+	int idx = 0;
 	int err = 0;
 
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		struct devlink_param_item *param_item;
-		int idx = 0;
-
-		devl_lock(devlink);
-		list_for_each_entry(param_item, &devlink->param_list, list) {
-			if (idx < dump->idx) {
-				idx++;
-				continue;
-			}
-			err = devlink_nl_param_fill(msg, devlink, 0, param_item,
-						    DEVLINK_CMD_PARAM_GET,
-						    NETLINK_CB(cb->skb).portid,
-						    cb->nlh->nlmsg_seq,
-						    NLM_F_MULTI);
-			if (err == -EOPNOTSUPP) {
-				err = 0;
-			} else if (err) {
-				devl_unlock(devlink);
-				devlink_put(devlink);
-				dump->idx = idx;
-				goto out;
-			}
+	list_for_each_entry(param_item, &devlink->param_list, list) {
+		if (idx < dump->idx) {
 			idx++;
+			continue;
 		}
-		devl_unlock(devlink);
-		devlink_put(devlink);
+		err = devlink_nl_param_fill(msg, devlink, 0, param_item,
+					    DEVLINK_CMD_PARAM_GET,
+					    NETLINK_CB(cb->skb).portid,
+					    cb->nlh->nlmsg_seq,
+					    NLM_F_MULTI);
+		if (err == -EOPNOTSUPP) {
+			err = 0;
+		} else if (err) {
+			dump->idx = idx;
+			break;
+		}
+		idx++;
 	}
-out:
-	if (err != -EMSGSIZE)
-		return err;
 
-	return msg->len;
+	return err;
 }
 
+const struct devlink_gen_cmd devl_gen_param = {
+	.dump_one		= devlink_nl_cmd_param_get_dumpinst,
+};
+
 static int
 devlink_param_type_get_from_info(struct genl_info *info,
 				 enum devlink_param_type *param_type)
@@ -6034,20 +5973,20 @@ static int devlink_nl_cmd_region_get_port_dumpit(struct sk_buff *msg,
 	return err;
 }
 
-static int devlink_nl_cmd_region_get_devlink_dumpit(struct sk_buff *msg,
-						    struct netlink_callback *cb,
-						    struct devlink *devlink,
-						    int *idx,
-						    int start)
+static int
+devlink_nl_cmd_region_get_dumpinst(struct sk_buff *msg, struct devlink *devlink,
+				   struct netlink_callback *cb)
 {
+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
 	struct devlink_region *region;
 	struct devlink_port *port;
 	unsigned long port_index;
+	int idx = 0;
 	int err;
 
 	list_for_each_entry(region, &devlink->region_list, list) {
-		if (*idx < start) {
-			(*idx)++;
+		if (idx < dump->idx) {
+			idx++;
 			continue;
 		}
 		err = devlink_nl_region_fill(msg, devlink,
@@ -6055,44 +5994,28 @@ static int devlink_nl_cmd_region_get_devlink_dumpit(struct sk_buff *msg,
 					     NETLINK_CB(cb->skb).portid,
 					     cb->nlh->nlmsg_seq,
 					     NLM_F_MULTI, region);
-		if (err)
+		if (err) {
+			dump->idx = idx;
 			return err;
-		(*idx)++;
+		}
+		idx++;
 	}
 
 	xa_for_each(&devlink->ports, port_index, port) {
-		err = devlink_nl_cmd_region_get_port_dumpit(msg, cb, port, idx,
-							    start);
-		if (err)
+		err = devlink_nl_cmd_region_get_port_dumpit(msg, cb, port, &idx,
+							    dump->idx);
+		if (err) {
+			dump->idx = idx;
 			return err;
+		}
 	}
 
 	return 0;
 }
 
-static int devlink_nl_cmd_region_get_dumpit(struct sk_buff *msg,
-					    struct netlink_callback *cb)
-{
-	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
-	int err = 0;
-
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		int idx = 0;
-
-		devl_lock(devlink);
-		err = devlink_nl_cmd_region_get_devlink_dumpit(msg, cb, devlink,
-							       &idx, dump->idx);
-		devl_unlock(devlink);
-		devlink_put(devlink);
-		if (err) {
-			dump->idx = idx;
-			goto out;
-		}
-	}
-out:
-	return msg->len;
-}
+const struct devlink_gen_cmd devl_gen_region = {
+	.dump_one		= devlink_nl_cmd_region_get_dumpinst,
+};
 
 static int devlink_nl_cmd_region_del(struct sk_buff *skb,
 				     struct genl_info *info)
@@ -6724,35 +6647,25 @@ static int devlink_nl_cmd_info_get_doit(struct sk_buff *skb,
 	return genlmsg_reply(msg, info);
 }
 
-static int devlink_nl_cmd_info_get_dumpit(struct sk_buff *msg,
-					  struct netlink_callback *cb)
+static int
+devlink_nl_cmd_info_get_dumpinst(struct sk_buff *msg, struct devlink *devlink,
+				 struct netlink_callback *cb)
 {
-	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
-	int err = 0;
-
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		devl_lock(devlink);
-		err = devlink_nl_info_fill(msg, devlink, DEVLINK_CMD_INFO_GET,
-					   NETLINK_CB(cb->skb).portid,
-					   cb->nlh->nlmsg_seq, NLM_F_MULTI,
-					   cb->extack);
-		devl_unlock(devlink);
-		if (err == -EOPNOTSUPP)
-			err = 0;
-		else if (err) {
-			devlink_put(devlink);
-			break;
-		}
-		devlink_put(devlink);
-	}
-
-	if (err != -EMSGSIZE)
-		return err;
+	int err;
 
-	return msg->len;
+	err = devlink_nl_info_fill(msg, devlink, DEVLINK_CMD_INFO_GET,
+				   NETLINK_CB(cb->skb).portid,
+				   cb->nlh->nlmsg_seq, NLM_F_MULTI,
+				   cb->extack);
+	if (err == -EOPNOTSUPP)
+		err = 0;
+	return err;
 }
 
+const struct devlink_gen_cmd devl_gen_info = {
+	.dump_one		= devlink_nl_cmd_info_get_dumpinst,
+};
+
 struct devlink_fmsg_item {
 	struct list_head list;
 	int attrtype;
@@ -8466,43 +8379,39 @@ static int devlink_nl_cmd_trap_get_doit(struct sk_buff *skb,
 	return err;
 }
 
-static int devlink_nl_cmd_trap_get_dumpit(struct sk_buff *msg,
-					  struct netlink_callback *cb)
+static int
+devlink_nl_cmd_trap_get_dumpinst(struct sk_buff *msg, struct devlink *devlink,
+				 struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	struct devlink *devlink;
-	int err;
-
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		struct devlink_trap_item *trap_item;
-		int idx = 0;
+	struct devlink_trap_item *trap_item;
+	int idx = 0;
+	int err = 0;
 
-		devl_lock(devlink);
-		list_for_each_entry(trap_item, &devlink->trap_list, list) {
-			if (idx < dump->idx) {
-				idx++;
-				continue;
-			}
-			err = devlink_nl_trap_fill(msg, devlink, trap_item,
-						   DEVLINK_CMD_TRAP_NEW,
-						   NETLINK_CB(cb->skb).portid,
-						   cb->nlh->nlmsg_seq,
-						   NLM_F_MULTI);
-			if (err) {
-				devl_unlock(devlink);
-				devlink_put(devlink);
-				dump->idx = idx;
-				goto out;
-			}
+	list_for_each_entry(trap_item, &devlink->trap_list, list) {
+		if (idx < dump->idx) {
 			idx++;
+			continue;
 		}
-		devl_unlock(devlink);
-		devlink_put(devlink);
+		err = devlink_nl_trap_fill(msg, devlink, trap_item,
+					   DEVLINK_CMD_TRAP_NEW,
+					   NETLINK_CB(cb->skb).portid,
+					   cb->nlh->nlmsg_seq,
+					   NLM_F_MULTI);
+		if (err) {
+			dump->idx = idx;
+			break;
+		}
+		idx++;
 	}
-out:
-	return msg->len;
+
+	return err;
 }
 
+const struct devlink_gen_cmd devl_gen_trap = {
+	.dump_one		= devlink_nl_cmd_trap_get_dumpinst,
+};
+
 static int __devlink_trap_action_set(struct devlink *devlink,
 				     struct devlink_trap_item *trap_item,
 				     enum devlink_trap_action trap_action,
@@ -8681,46 +8590,41 @@ static int devlink_nl_cmd_trap_group_get_doit(struct sk_buff *skb,
 	return err;
 }
 
-static int devlink_nl_cmd_trap_group_get_dumpit(struct sk_buff *msg,
-						struct netlink_callback *cb)
+static int
+devlink_nl_cmd_trap_group_get_dumpinst(struct sk_buff *msg,
+				       struct devlink *devlink,
+				       struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	enum devlink_command cmd = DEVLINK_CMD_TRAP_GROUP_NEW;
-	u32 portid = NETLINK_CB(cb->skb).portid;
-	struct devlink *devlink;
-	int err;
+	struct devlink_trap_group_item *group_item;
+	int idx = 0;
+	int err = 0;
 
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		struct devlink_trap_group_item *group_item;
-		int idx = 0;
 
-		devl_lock(devlink);
-		list_for_each_entry(group_item, &devlink->trap_group_list,
-				    list) {
-			if (idx < dump->idx) {
-				idx++;
-				continue;
-			}
-			err = devlink_nl_trap_group_fill(msg, devlink,
-							 group_item, cmd,
-							 portid,
-							 cb->nlh->nlmsg_seq,
-							 NLM_F_MULTI);
-			if (err) {
-				devl_unlock(devlink);
-				devlink_put(devlink);
-				dump->idx = idx;
-				goto out;
-			}
+	list_for_each_entry(group_item, &devlink->trap_group_list, list) {
+		if (idx < dump->idx) {
 			idx++;
+			continue;
 		}
-		devl_unlock(devlink);
-		devlink_put(devlink);
+		err = devlink_nl_trap_group_fill(msg, devlink, group_item,
+						 DEVLINK_CMD_TRAP_GROUP_NEW,
+						 NETLINK_CB(cb->skb).portid,
+						 cb->nlh->nlmsg_seq,
+						 NLM_F_MULTI);
+		if (err) {
+			dump->idx = idx;
+			break;
+		}
+		idx++;
 	}
-out:
-	return msg->len;
+
+	return err;
 }
 
+const struct devlink_gen_cmd devl_gen_trap_group = {
+	.dump_one		= devlink_nl_cmd_trap_group_get_dumpinst,
+};
+
 static int
 __devlink_trap_group_action_set(struct devlink *devlink,
 				struct devlink_trap_group_item *group_item,
@@ -8985,46 +8889,40 @@ static int devlink_nl_cmd_trap_policer_get_doit(struct sk_buff *skb,
 	return err;
 }
 
-static int devlink_nl_cmd_trap_policer_get_dumpit(struct sk_buff *msg,
-						  struct netlink_callback *cb)
+static int
+devlink_nl_cmd_trap_policer_get_dumpinst(struct sk_buff *msg,
+					 struct devlink *devlink,
+					 struct netlink_callback *cb)
 {
 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
-	enum devlink_command cmd = DEVLINK_CMD_TRAP_POLICER_NEW;
-	u32 portid = NETLINK_CB(cb->skb).portid;
-	struct devlink *devlink;
-	int err;
-
-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
-		struct devlink_trap_policer_item *policer_item;
-		int idx = 0;
+	struct devlink_trap_policer_item *policer_item;
+	int idx = 0;
+	int err = 0;
 
-		devl_lock(devlink);
-		list_for_each_entry(policer_item, &devlink->trap_policer_list,
-				    list) {
-			if (idx < dump->idx) {
-				idx++;
-				continue;
-			}
-			err = devlink_nl_trap_policer_fill(msg, devlink,
-							   policer_item, cmd,
-							   portid,
-							   cb->nlh->nlmsg_seq,
-							   NLM_F_MULTI);
-			if (err) {
-				devl_unlock(devlink);
-				devlink_put(devlink);
-				dump->idx = idx;
-				goto out;
-			}
+	list_for_each_entry(policer_item, &devlink->trap_policer_list, list) {
+		if (idx < dump->idx) {
 			idx++;
+			continue;
 		}
-		devl_unlock(devlink);
-		devlink_put(devlink);
+		err = devlink_nl_trap_policer_fill(msg, devlink, policer_item,
+						   DEVLINK_CMD_TRAP_POLICER_NEW,
+						   NETLINK_CB(cb->skb).portid,
+						   cb->nlh->nlmsg_seq,
+						   NLM_F_MULTI);
+		if (err) {
+			dump->idx = idx;
+			break;
+		}
+		idx++;
 	}
-out:
-	return msg->len;
+
+	return err;
 }
 
+const struct devlink_gen_cmd devl_gen_trap_policer = {
+	.dump_one		= devlink_nl_cmd_trap_policer_get_dumpinst,
+};
+
 static int
 devlink_trap_policer_set(struct devlink *devlink,
 			 struct devlink_trap_policer_item *policer_item,
@@ -9102,14 +9000,14 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 		.cmd = DEVLINK_CMD_GET,
 		.validate = GENL_DONT_VALIDATE_STRICT | GENL_DONT_VALIDATE_DUMP,
 		.doit = devlink_nl_cmd_get_doit,
-		.dumpit = devlink_nl_cmd_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		/* can be retrieved by unprivileged users */
 	},
 	{
 		.cmd = DEVLINK_CMD_PORT_GET,
 		.validate = GENL_DONT_VALIDATE_STRICT | GENL_DONT_VALIDATE_DUMP,
 		.doit = devlink_nl_cmd_port_get_doit,
-		.dumpit = devlink_nl_cmd_port_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		.internal_flags = DEVLINK_NL_FLAG_NEED_PORT,
 		/* can be retrieved by unprivileged users */
 	},
@@ -9185,14 +9083,14 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 		.cmd = DEVLINK_CMD_SB_GET,
 		.validate = GENL_DONT_VALIDATE_STRICT | GENL_DONT_VALIDATE_DUMP,
 		.doit = devlink_nl_cmd_sb_get_doit,
-		.dumpit = devlink_nl_cmd_sb_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		/* can be retrieved by unprivileged users */
 	},
 	{
 		.cmd = DEVLINK_CMD_SB_POOL_GET,
 		.validate = GENL_DONT_VALIDATE_STRICT | GENL_DONT_VALIDATE_DUMP,
 		.doit = devlink_nl_cmd_sb_pool_get_doit,
-		.dumpit = devlink_nl_cmd_sb_pool_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		/* can be retrieved by unprivileged users */
 	},
 	{
@@ -9205,7 +9103,7 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 		.cmd = DEVLINK_CMD_SB_PORT_POOL_GET,
 		.validate = GENL_DONT_VALIDATE_STRICT | GENL_DONT_VALIDATE_DUMP,
 		.doit = devlink_nl_cmd_sb_port_pool_get_doit,
-		.dumpit = devlink_nl_cmd_sb_port_pool_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		.internal_flags = DEVLINK_NL_FLAG_NEED_PORT,
 		/* can be retrieved by unprivileged users */
 	},
@@ -9220,7 +9118,7 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 		.cmd = DEVLINK_CMD_SB_TC_POOL_BIND_GET,
 		.validate = GENL_DONT_VALIDATE_STRICT | GENL_DONT_VALIDATE_DUMP,
 		.doit = devlink_nl_cmd_sb_tc_pool_bind_get_doit,
-		.dumpit = devlink_nl_cmd_sb_tc_pool_bind_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		.internal_flags = DEVLINK_NL_FLAG_NEED_PORT,
 		/* can be retrieved by unprivileged users */
 	},
@@ -9301,7 +9199,7 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 		.cmd = DEVLINK_CMD_PARAM_GET,
 		.validate = GENL_DONT_VALIDATE_STRICT | GENL_DONT_VALIDATE_DUMP,
 		.doit = devlink_nl_cmd_param_get_doit,
-		.dumpit = devlink_nl_cmd_param_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		/* can be retrieved by unprivileged users */
 	},
 	{
@@ -9329,7 +9227,7 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 		.cmd = DEVLINK_CMD_REGION_GET,
 		.validate = GENL_DONT_VALIDATE_STRICT | GENL_DONT_VALIDATE_DUMP,
 		.doit = devlink_nl_cmd_region_get_doit,
-		.dumpit = devlink_nl_cmd_region_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		.flags = GENL_ADMIN_PERM,
 	},
 	{
@@ -9355,7 +9253,7 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 		.cmd = DEVLINK_CMD_INFO_GET,
 		.validate = GENL_DONT_VALIDATE_STRICT | GENL_DONT_VALIDATE_DUMP,
 		.doit = devlink_nl_cmd_info_get_doit,
-		.dumpit = devlink_nl_cmd_info_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		/* can be retrieved by unprivileged users */
 	},
 	{
@@ -9417,7 +9315,7 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 	{
 		.cmd = DEVLINK_CMD_TRAP_GET,
 		.doit = devlink_nl_cmd_trap_get_doit,
-		.dumpit = devlink_nl_cmd_trap_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		/* can be retrieved by unprivileged users */
 	},
 	{
@@ -9428,7 +9326,7 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 	{
 		.cmd = DEVLINK_CMD_TRAP_GROUP_GET,
 		.doit = devlink_nl_cmd_trap_group_get_doit,
-		.dumpit = devlink_nl_cmd_trap_group_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		/* can be retrieved by unprivileged users */
 	},
 	{
@@ -9439,7 +9337,7 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 	{
 		.cmd = DEVLINK_CMD_TRAP_POLICER_GET,
 		.doit = devlink_nl_cmd_trap_policer_get_doit,
-		.dumpit = devlink_nl_cmd_trap_policer_get_dumpit,
+		.dumpit = devlink_instance_iter_dump,
 		/* can be retrieved by unprivileged users */
 	},
 	{
@@ -9450,7 +9348,7 @@ const struct genl_small_ops devlink_nl_ops[56] = {
 	{
 		.cmd = DEVLINK_CMD_SELFTESTS_GET,
 		.doit = devlink_nl_cmd_selftests_get_doit,
-		.dumpit = devlink_nl_cmd_selftests_get_dumpit
+		.dumpit = devlink_instance_iter_dump,
 		/* can be retrieved by unprivileged users */
 	},
 	{
diff --git a/net/devlink/netlink.c b/net/devlink/netlink.c
index fcf10c288480..7eb39ccb2ae7 100644
--- a/net/devlink/netlink.c
+++ b/net/devlink/netlink.c
@@ -179,7 +179,20 @@ static void devlink_nl_post_doit(const struct genl_split_ops *ops,
 }
 
 static const struct devlink_gen_cmd *devl_gen_cmds[] = {
+	[DEVLINK_CMD_GET]		= &devl_gen_inst,
+	[DEVLINK_CMD_PORT_GET]		= &devl_gen_port,
+	[DEVLINK_CMD_SB_GET]		= &devl_gen_sb,
+	[DEVLINK_CMD_SB_POOL_GET]	= &devl_gen_sb_pool,
+	[DEVLINK_CMD_SB_PORT_POOL_GET]	= &devl_gen_sb_port_pool,
+	[DEVLINK_CMD_SB_TC_POOL_BIND_GET] = &devl_gen_sb_tc_pool_bind,
+	[DEVLINK_CMD_PARAM_GET]		= &devl_gen_param,
+	[DEVLINK_CMD_REGION_GET]	= &devl_gen_region,
+	[DEVLINK_CMD_INFO_GET]		= &devl_gen_info,
 	[DEVLINK_CMD_RATE_GET]		= &devl_gen_rate_get,
+	[DEVLINK_CMD_TRAP_GET]		= &devl_gen_trap,
+	[DEVLINK_CMD_TRAP_GROUP_GET]	= &devl_gen_trap_group,
+	[DEVLINK_CMD_TRAP_POLICER_GET]	= &devl_gen_trap_policer,
+	[DEVLINK_CMD_SELFTESTS_GET]	= &devl_gen_selftests,
 };
 
 int devlink_instance_iter_dump(struct sk_buff *msg, struct netlink_callback *cb)
-- 
2.38.1


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

* Re: [PATCH net-next 02/14] devlink: split out core code
  2023-01-04  4:16 ` [PATCH net-next 02/14] devlink: split out core code Jakub Kicinski
@ 2023-01-04  9:50   ` Jiri Pirko
  2023-01-05  2:10     ` Jakub Kicinski
  0 siblings, 1 reply; 42+ messages in thread
From: Jiri Pirko @ 2023-01-04  9:50 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Wed, Jan 04, 2023 at 05:16:24AM CET, kuba@kernel.org wrote:
>Move core code into a separate file. It's spread around the main
>file which makes refactoring and figuring out how devlink works
>harder.
>
>Move the xarray, all the most core devlink instance code out like
>locking, ref counting, alloc, register, etc. Leave port stuff in
>basic.c, if we want to move port code it'd probably be to its own file.

leftover.c

>
>Rename devlink_netdevice_event() to make it clear that it only touches
>ports (that's the only change which isn't a pure code move).

Did you do any other changes on the move? I believe that for patches
like this that move a lot of code it is beneficial to move the code "as
is". The changes could be done in a separate patches, for the ease of
review purposes. Could you please?


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

* Re: [PATCH net-next 04/14] netlink: add macro for checking dump ctx size
  2023-01-04  4:16 ` [PATCH net-next 04/14] netlink: add macro for checking dump ctx size Jakub Kicinski
@ 2023-01-04  9:51   ` Jiri Pirko
  0 siblings, 0 replies; 42+ messages in thread
From: Jiri Pirko @ 2023-01-04  9:51 UTC (permalink / raw)
  To: Jakub Kicinski
  Cc: davem, netdev, edumazet, pabeni, jacob.e.keller, pablo, kadlec,
	fw, johannes, ecree.xilinx, netfilter-devel, coreteam

Wed, Jan 04, 2023 at 05:16:26AM CET, kuba@kernel.org wrote:
>We encourage casting struct netlink_callback::ctx to a local
>struct (in a comment above the field). Provide a convenience
>macro for checking if the local struct fits into the ctx.
>
>Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
>Signed-off-by: Jakub Kicinski <kuba@kernel.org>

Reviewed-by: Jiri Pirko <jiri@nvidia.com>

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

* Re: [PATCH net-next 05/14] devlink: use an explicit structure for dump context
  2023-01-04  4:16 ` [PATCH net-next 05/14] devlink: use an explicit structure for dump context Jakub Kicinski
@ 2023-01-04 10:04   ` Jiri Pirko
  2023-01-05  2:22     ` Jakub Kicinski
  0 siblings, 1 reply; 42+ messages in thread
From: Jiri Pirko @ 2023-01-04 10:04 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Wed, Jan 04, 2023 at 05:16:27AM CET, kuba@kernel.org wrote:
>Create a dump context structure instead of using cb->args
>as an unsigned long array. This is a pure conversion which
>is intended to be as much of a noop as possible.
>Subsequent changes will use this to simplify the code.
>
>The two non-trivial parts are:
> - devlink_nl_cmd_health_reporter_dump_get_dumpit() checks args[0]
>   to see if devlink_fmsg_dumpit() has already been called (whether
>   this is the first msg), but doesn't use the exact value, so we
>   can drop the local variable there already
> - devlink_nl_cmd_region_read_dumpit() uses args[0] for address
>   but we'll use args[1] now, shouldn't matter

I don't follow this. Where do you use args[1]? you mean
dump->start_offset? If yes, it does not matter at all and I think
mentioning that only confuses reader (as it did for me).


>
>Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
>Signed-off-by: Jakub Kicinski <kuba@kernel.org>
>---
> net/devlink/devl_internal.h | 23 +++++++++
> net/devlink/leftover.c      | 98 ++++++++++++++++++++++---------------
> 2 files changed, 81 insertions(+), 40 deletions(-)
>
>diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
>index bc7df9b0f775..91059311f18d 100644
>--- a/net/devlink/devl_internal.h
>+++ b/net/devlink/devl_internal.h
>@@ -107,6 +107,21 @@ enum devlink_multicast_groups {
> 	DEVLINK_MCGRP_CONFIG,
> };
> 
>+/* state held across netlink dumps */
>+struct devlink_nl_dump_state {
>+	int idx;
>+	union {
>+		/* DEVLINK_CMD_REGION_READ */
>+		struct {
>+			u64 start_offset;
>+		};
>+		/* DEVLINK_CMD_HEALTH_REPORTER_DUMP_GET */
>+		struct {
>+			u64 dump_ts;
>+		};
>+	};
>+};
>+
> extern const struct genl_small_ops devlink_nl_ops[56];
> 
> struct devlink *devlink_get_from_attrs(struct net *net, struct nlattr **attrs);
>@@ -114,6 +129,14 @@ struct devlink *devlink_get_from_attrs(struct net *net, struct nlattr **attrs);
> void devlink_notify_unregister(struct devlink *devlink);
> void devlink_notify_register(struct devlink *devlink);
> 
>+static inline struct devlink_nl_dump_state *
>+devl_dump_state(struct netlink_callback *cb)

What is the convesion you established again regarding "devl_" and
"devlink_" prefixes? I don't find it written down anywhere and honestly
it confuses me a bit.



>+{
>+	NL_ASSET_DUMP_CTX_FITS(struct devlink_nl_dump_state);
>+
>+	return (struct devlink_nl_dump_state *)cb->ctx;
>+}
>+
> /* Ports */
> int devlink_port_netdevice_event(struct notifier_block *nb,
> 				 unsigned long event, void *ptr);
>diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
>index e01ba7999b91..bcc930b7cfcf 100644
>--- a/net/devlink/leftover.c
>+++ b/net/devlink/leftover.c
>@@ -1222,9 +1222,10 @@ static void devlink_rate_notify(struct devlink_rate *devlink_rate,
> static int devlink_nl_cmd_rate_get_dumpit(struct sk_buff *msg,
> 					  struct netlink_callback *cb)
> {
>+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);

Could this be named "state" or "dump_state"? "dump" is not what it is.



> 	struct devlink_rate *devlink_rate;
> 	struct devlink *devlink;
>-	int start = cb->args[0];
>+	int start = dump->idx;
> 	unsigned long index;
> 	int idx = 0;
> 	int err = 0;

[..]


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

* Re: [PATCH net-next 07/14] devlink: drop the filter argument from devlinks_xa_find_get
  2023-01-04  4:16 ` [PATCH net-next 07/14] devlink: drop the filter argument from devlinks_xa_find_get Jakub Kicinski
@ 2023-01-04 10:05   ` Jiri Pirko
  0 siblings, 0 replies; 42+ messages in thread
From: Jiri Pirko @ 2023-01-04 10:05 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Wed, Jan 04, 2023 at 05:16:29AM CET, kuba@kernel.org wrote:
>Looks like devlinks_xa_find_get() was intended to get the mark
>from the @filter argument. It doesn't actually use @filter, passing
>DEVLINK_REGISTERED to xa_find_fn() directly. Walking marks other
>than registered is unlikely so drop @filter argument completely.
>
>Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
>Signed-off-by: Jakub Kicinski <kuba@kernel.org>

Reviewed-by: Jiri Pirko <jiri@nvidia.com>

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

* Re: [PATCH net-next 08/14] devlink: health: combine loops in dump
  2023-01-04  4:16 ` [PATCH net-next 08/14] devlink: health: combine loops in dump Jakub Kicinski
@ 2023-01-04 10:06   ` Jiri Pirko
  0 siblings, 0 replies; 42+ messages in thread
From: Jiri Pirko @ 2023-01-04 10:06 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Wed, Jan 04, 2023 at 05:16:30AM CET, kuba@kernel.org wrote:
>Walk devlink instances only once. Dump the instance reporters
>and port reporters before moving to the next instance.
>User space should not depend on ordering of messages.
>
>This will make improving stability of the walk easier.
>
>Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
>Signed-off-by: Jakub Kicinski <kuba@kernel.org>

Reviewed-by: Jiri Pirko <jiri@nvidia.com>

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

* Re: [PATCH net-next 09/14] devlink: restart dump based on devlink instance ids (simple)
  2023-01-04  4:16 ` [PATCH net-next 09/14] devlink: restart dump based on devlink instance ids (simple) Jakub Kicinski
@ 2023-01-04 14:18   ` Jiri Pirko
  2023-01-05  3:21     ` Jakub Kicinski
  0 siblings, 1 reply; 42+ messages in thread
From: Jiri Pirko @ 2023-01-04 14:18 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Wed, Jan 04, 2023 at 05:16:31AM CET, kuba@kernel.org wrote:
>xarray gives each devlink instance an id and allows us to restart
>walk based on that id quite neatly. This is nice both from the
>perspective of code brevity and from the stability of the dump
>(devlink instances disappearing from before the resumption point
>will not cause inconsistent dumps).
>
>This patch takes care of simple cases where dump->idx counts
>devlink instances only.
>
>Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
>Signed-off-by: Jakub Kicinski <kuba@kernel.org>
>---
> net/devlink/core.c          |  2 +-
> net/devlink/devl_internal.h | 14 ++++++++++++++
> net/devlink/leftover.c      | 36 ++++++++----------------------------
> 3 files changed, 23 insertions(+), 29 deletions(-)
>
>diff --git a/net/devlink/core.c b/net/devlink/core.c
>index 3a99bf84632e..371d6821315d 100644
>--- a/net/devlink/core.c
>+++ b/net/devlink/core.c
>@@ -91,7 +91,7 @@ void devlink_put(struct devlink *devlink)
> 		call_rcu(&devlink->rcu, __devlink_put_rcu);
> }
> 
>-static struct devlink *
>+struct devlink *
> devlinks_xa_find_get(struct net *net, unsigned long *indexp,
> 		     void * (*xa_find_fn)(struct xarray *, unsigned long *,
> 					  unsigned long, xa_mark_t))
>diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
>index ee98f3bdcd33..a567ff77601d 100644
>--- a/net/devlink/devl_internal.h
>+++ b/net/devlink/devl_internal.h
>@@ -87,6 +87,10 @@ extern struct genl_family devlink_nl_family;
> 	     devlink; devlink = devlinks_xa_find_get_next(net, &index))
> 
> struct devlink *
>+devlinks_xa_find_get(struct net *net, unsigned long *indexp,
>+		     void * (*xa_find_fn)(struct xarray *, unsigned long *,
>+					  unsigned long, xa_mark_t));
>+struct devlink *
> devlinks_xa_find_get_first(struct net *net, unsigned long *indexp);
> struct devlink *
> devlinks_xa_find_get_next(struct net *net, unsigned long *indexp);
>@@ -104,6 +108,7 @@ enum devlink_multicast_groups {
> 
> /* state held across netlink dumps */
> struct devlink_nl_dump_state {
>+	unsigned long instance;
> 	int idx;
> 	union {
> 		/* DEVLINK_CMD_REGION_READ */
>@@ -117,6 +122,15 @@ struct devlink_nl_dump_state {
> 	};
> };
> 
>+/* Iterate over devlink pointers which were possible to get reference to.
>+ * devlink_put() needs to be called for each iterated devlink pointer
>+ * in loop body in order to release the reference.
>+ */
>+#define devlink_dump_for_each_instance_get(msg, dump, devlink)		\
>+	for (; (devlink = devlinks_xa_find_get(sock_net(msg->sk),	\

I undestand that the "dump" is zeroed at the beginning of dumpit call,
however, if you call this helper multiple times, the second iteration
would't not work.

Perhaps better to initialize instance=0 at the beginning of the loop to
make this helper calls behaviour independent on context.


>+					       &dump->instance, xa_find)); \
>+	     dump->instance++)
>+
> extern const struct genl_small_ops devlink_nl_ops[56];
> 
> struct devlink *devlink_get_from_attrs(struct net *net, struct nlattr **attrs);
>diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
>index e3cfb64990b4..0f24b321b0bb 100644
>--- a/net/devlink/leftover.c
>+++ b/net/devlink/leftover.c
>@@ -1319,17 +1319,9 @@ static int devlink_nl_cmd_get_dumpit(struct sk_buff *msg,
> {
> 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
> 	struct devlink *devlink;
>-	unsigned long index;
>-	int idx = 0;
> 	int err;
> 
>-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
>-		if (idx < dump->idx) {
>-			idx++;
>-			devlink_put(devlink);
>-			continue;
>-		}
>-
>+	devlink_dump_for_each_instance_get(msg, dump, devlink) {

The name suggests on the first sight that you are iterating some dump,
which is slightly confusing. Perhaps better to have
"devlinks_xa_for_each_" in the prefix somehow?

	devlinks_xa_for_each_registered_get_dumping()

I know it is long :)


> 		devl_lock(devlink);
> 		err = devlink_nl_fill(msg, devlink, DEVLINK_CMD_NEW,
> 				      NETLINK_CB(cb->skb).portid,
>@@ -1339,10 +1331,8 @@ static int devlink_nl_cmd_get_dumpit(struct sk_buff *msg,
> 
> 		if (err)
> 			goto out;
>-		idx++;
> 	}
> out:
>-	dump->idx = idx;
> 	return msg->len;
> }
> 
>@@ -4872,13 +4862,13 @@ static int devlink_nl_cmd_selftests_get_dumpit(struct sk_buff *msg,
> {
> 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
> 	struct devlink *devlink;
>-	unsigned long index;
>-	int idx = 0;
> 	int err = 0;
> 
>-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
>-		if (idx < dump->idx || !devlink->ops->selftest_check)
>-			goto inc;
>+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
>+		if (!devlink->ops->selftest_check) {
>+			devlink_put(devlink);
>+			continue;
>+		}
> 
> 		devl_lock(devlink);
> 		err = devlink_nl_selftests_fill(msg, devlink,
>@@ -4890,15 +4880,13 @@ static int devlink_nl_cmd_selftests_get_dumpit(struct sk_buff *msg,
> 			devlink_put(devlink);
> 			break;
> 		}
>-inc:
>-		idx++;
>+
> 		devlink_put(devlink);
> 	}
> 
> 	if (err != -EMSGSIZE)
> 		return err;
> 
>-	dump->idx = idx;
> 	return msg->len;
> }
> 
>@@ -6747,14 +6735,9 @@ static int devlink_nl_cmd_info_get_dumpit(struct sk_buff *msg,
> {
> 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
> 	struct devlink *devlink;
>-	unsigned long index;
>-	int idx = 0;
> 	int err = 0;
> 
>-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
>-		if (idx < dump->idx)
>-			goto inc;
>-
>+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
> 		devl_lock(devlink);
> 		err = devlink_nl_info_fill(msg, devlink, DEVLINK_CMD_INFO_GET,
> 					   NETLINK_CB(cb->skb).portid,
>@@ -6767,15 +6750,12 @@ static int devlink_nl_cmd_info_get_dumpit(struct sk_buff *msg,
> 			devlink_put(devlink);
> 			break;
> 		}
>-inc:
>-		idx++;
> 		devlink_put(devlink);
> 	}
> 
> 	if (err != -EMSGSIZE)
> 		return err;
> 
>-	dump->idx = idx;
> 	return msg->len;
> }
> 
>-- 
>2.38.1
>

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

* Re: [PATCH net-next 10/14] devlink: restart dump based on devlink instance ids (nested)
  2023-01-04  4:16 ` [PATCH net-next 10/14] devlink: restart dump based on devlink instance ids (nested) Jakub Kicinski
@ 2023-01-04 15:47   ` Jiri Pirko
  0 siblings, 0 replies; 42+ messages in thread
From: Jiri Pirko @ 2023-01-04 15:47 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Wed, Jan 04, 2023 at 05:16:32AM CET, kuba@kernel.org wrote:
>Use xarray id for cases of simple sub-object iteration.
>We'll now use the dump->instance for the devlink instances
>and dump->idx for subobject index.
>
>Moving the definition of idx into the inner loop makes sense,
>so while at it also move other sub-object local variables into
>the loop.
>
>Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
>Signed-off-by: Jakub Kicinski <kuba@kernel.org>

Reviewed-by: Jiri Pirko <jiri@nvidia.com>

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

* Re: [PATCH net-next 11/14] devlink: restart dump based on devlink instance ids (function)
  2023-01-04  4:16 ` [PATCH net-next 11/14] devlink: restart dump based on devlink instance ids (function) Jakub Kicinski
@ 2023-01-04 15:51   ` Jiri Pirko
  0 siblings, 0 replies; 42+ messages in thread
From: Jiri Pirko @ 2023-01-04 15:51 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Wed, Jan 04, 2023 at 05:16:33AM CET, kuba@kernel.org wrote:
>Use xarray id for cases of sub-objects which are iterated in
>a function.
>
>Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
>Signed-off-by: Jakub Kicinski <kuba@kernel.org>

Reviewed-by: Jiri Pirko <jiri@nvidia.com>

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

* Re: [PATCH net-next 12/14] devlink: uniformly take the devlink instance lock in the dump loop
  2023-01-04  4:16 ` [PATCH net-next 12/14] devlink: uniformly take the devlink instance lock in the dump loop Jakub Kicinski
@ 2023-01-04 15:52   ` Jiri Pirko
  0 siblings, 0 replies; 42+ messages in thread
From: Jiri Pirko @ 2023-01-04 15:52 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Wed, Jan 04, 2023 at 05:16:34AM CET, kuba@kernel.org wrote:
>Move the lock taking out of devlink_nl_cmd_region_get_devlink_dumpit().
>This way all dumps will take the instance lock in the main iteration
>loop directly, making refactoring and reading the code easier.
>
>Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
>Signed-off-by: Jakub Kicinski <kuba@kernel.org>

Reviewed-by: Jiri Pirko <jiri@nvidia.com>

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

* Re: [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-04  4:16 ` [PATCH net-next 13/14] devlink: add by-instance dump infra Jakub Kicinski
@ 2023-01-04 16:50   ` Jiri Pirko
  2023-01-05  3:46     ` Jakub Kicinski
  0 siblings, 1 reply; 42+ messages in thread
From: Jiri Pirko @ 2023-01-04 16:50 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Wed, Jan 04, 2023 at 05:16:35AM CET, kuba@kernel.org wrote:
>Most dumpit implementations walk the devlink instances.
>This requires careful lock taking and reference dropping.
>Factor the loop out and provide just a callback to handle
>a single instance dump.
>
>Convert one user as an example, other users converted
>in the next change.
>
>Slightly inspired by ethtool netlink code.
>
>Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
>Signed-off-by: Jakub Kicinski <kuba@kernel.org>
>---
> net/devlink/devl_internal.h | 10 +++++++
> net/devlink/leftover.c      | 55 ++++++++++++++++---------------------
> net/devlink/netlink.c       | 33 ++++++++++++++++++++++
> 3 files changed, 67 insertions(+), 31 deletions(-)
>
>diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
>index 5adac38454fd..e49b82dd77cd 100644
>--- a/net/devlink/devl_internal.h
>+++ b/net/devlink/devl_internal.h
>@@ -122,6 +122,11 @@ struct devlink_nl_dump_state {
> 	};
> };
> 
>+struct devlink_gen_cmd {

What is "gen"? Generic netlink? Not sure why perhaps "nl" would be fine
to be consistent with the rest of the code?
Why "cmd"? That looks a bit odd to me.



>+	int (*dump_one)(struct sk_buff *msg, struct devlink *devlink,
>+			struct netlink_callback *cb);

Do you plan to have more callbacks here? If no, wouldn't it be better
to just have typedef and assign the pointer to the dump_one in devl_gen_cmds
array?


>+};
>+
> /* Iterate over devlink pointers which were possible to get reference to.
>  * devlink_put() needs to be called for each iterated devlink pointer
>  * in loop body in order to release the reference.
>@@ -138,6 +143,9 @@ struct devlink *devlink_get_from_attrs(struct net *net, struct nlattr **attrs);
> void devlink_notify_unregister(struct devlink *devlink);
> void devlink_notify_register(struct devlink *devlink);
> 
>+int devlink_instance_iter_dump(struct sk_buff *msg,
>+			       struct netlink_callback *cb);
>+
> static inline struct devlink_nl_dump_state *
> devl_dump_state(struct netlink_callback *cb)
> {
>@@ -173,6 +181,8 @@ devlink_linecard_get_from_info(struct devlink *devlink, struct genl_info *info);
> void devlink_linecard_put(struct devlink_linecard *linecard);
> 
> /* Rates */
>+extern const struct devlink_gen_cmd devl_gen_rate_get;

The struct name is *_cmd, not sure why the variable name is *_get
Shouldn't it be rather devl_gen_cmd_rate?

Still sounds a bit odd.


>+
> struct devlink_rate *
> devlink_rate_get_from_info(struct devlink *devlink, struct genl_info *info);
> struct devlink_rate *
>diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
>index c6ad8133fc23..f18d8dcf9751 100644
>--- a/net/devlink/leftover.c
>+++ b/net/devlink/leftover.c
>@@ -1219,47 +1219,40 @@ static void devlink_rate_notify(struct devlink_rate *devlink_rate,
> 				0, DEVLINK_MCGRP_CONFIG, GFP_KERNEL);
> }
> 
>-static int devlink_nl_cmd_rate_get_dumpit(struct sk_buff *msg,
>-					  struct netlink_callback *cb)
>+static int
>+devlink_nl_cmd_rate_get_dumpinst(struct sk_buff *msg, struct devlink *devlink,
>+				 struct netlink_callback *cb)
> {
> 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
>-	struct devlink *devlink;
>+	struct devlink_rate *devlink_rate;
>+	int idx = 0;
> 	int err = 0;
> 
>-	devlink_dump_for_each_instance_get(msg, dump, devlink) {
>-		struct devlink_rate *devlink_rate;
>-		int idx = 0;
>-
>-		devl_lock(devlink);
>-		list_for_each_entry(devlink_rate, &devlink->rate_list, list) {
>-			enum devlink_command cmd = DEVLINK_CMD_RATE_NEW;
>-			u32 id = NETLINK_CB(cb->skb).portid;
>+	list_for_each_entry(devlink_rate, &devlink->rate_list, list) {
>+		enum devlink_command cmd = DEVLINK_CMD_RATE_NEW;
>+		u32 id = NETLINK_CB(cb->skb).portid;
> 
>-			if (idx < dump->idx) {
>-				idx++;
>-				continue;
>-			}
>-			err = devlink_nl_rate_fill(msg, devlink_rate, cmd, id,
>-						   cb->nlh->nlmsg_seq,
>-						   NLM_F_MULTI, NULL);
>-			if (err) {
>-				devl_unlock(devlink);
>-				devlink_put(devlink);
>-				dump->idx = idx;
>-				goto out;
>-			}
>+		if (idx < dump->idx) {
> 			idx++;
>+			continue;
> 		}
>-		devl_unlock(devlink);
>-		devlink_put(devlink);
>+		err = devlink_nl_rate_fill(msg, devlink_rate, cmd, id,
>+					   cb->nlh->nlmsg_seq,
>+					   NLM_F_MULTI, NULL);
>+		if (err) {
>+			dump->idx = idx;
>+			break;
>+		}
>+		idx++;
> 	}
>-out:
>-	if (err != -EMSGSIZE)
>-		return err;
> 
>-	return msg->len;
>+	return err;
> }
> 
>+const struct devlink_gen_cmd devl_gen_rate_get = {
>+	.dump_one		= devlink_nl_cmd_rate_get_dumpinst,

dump_one/dumpinst inconsistency in names


>+};
>+
> static int devlink_nl_cmd_rate_get_doit(struct sk_buff *skb,
> 					struct genl_info *info)
> {
>@@ -9130,7 +9123,7 @@ const struct genl_small_ops devlink_nl_ops[56] = {
> 	{
> 		.cmd = DEVLINK_CMD_RATE_GET,
> 		.doit = devlink_nl_cmd_rate_get_doit,
>-		.dumpit = devlink_nl_cmd_rate_get_dumpit,
>+		.dumpit = devlink_instance_iter_dump,

again, inconsistency:
devlink_instance_iter_dumpit


> 		.internal_flags = DEVLINK_NL_FLAG_NEED_RATE,
> 		/* can be retrieved by unprivileged users */

Unrelated to this patch, I wonder, why you didn't move devlink_nl_ops
along with the rest of the netlink code to netlink.c?


> 	},
>diff --git a/net/devlink/netlink.c b/net/devlink/netlink.c
>index ce1a7d674d14..fcf10c288480 100644
>--- a/net/devlink/netlink.c
>+++ b/net/devlink/netlink.c
>@@ -5,6 +5,7 @@
>  */
> 
> #include <net/genetlink.h>
>+#include <net/sock.h>
> 
> #include "devl_internal.h"
> 
>@@ -177,6 +178,38 @@ static void devlink_nl_post_doit(const struct genl_split_ops *ops,
> 	devlink_put(devlink);
> }
> 
>+static const struct devlink_gen_cmd *devl_gen_cmds[] = {
>+	[DEVLINK_CMD_RATE_GET]		= &devl_gen_rate_get,

static const devlink_nl_dump_one_t *devlink_nl_dump_one[] = {
	[DEVLINK_CMD_RATE_GET]	= &devl_nl_rate_dump_one,
}
Maybe? (not sure how the devlink/devl should be used here though)



>+};
>+
>+int devlink_instance_iter_dump(struct sk_buff *msg, struct netlink_callback *cb)
>+{
>+	const struct genl_dumpit_info *info = genl_dumpit_info(cb);
>+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
>+	const struct devlink_gen_cmd *cmd;
>+	struct devlink *devlink;
>+	int err = 0;
>+
>+	cmd = devl_gen_cmds[info->op.cmd];
>+
>+	devlink_dump_for_each_instance_get(msg, dump, devlink) {
>+		devl_lock(devlink);
>+		err = cmd->dump_one(msg, devlink, cb);
>+		devl_unlock(devlink);
>+		devlink_put(devlink);
>+
>+		if (err)
>+			break;
>+
>+		/* restart sub-object walk for the next instance */
>+		dump->idx = 0;
>+	}
>+
>+	if (err != -EMSGSIZE)
>+		return err;
>+	return msg->len;
>+}
>+
> struct genl_family devlink_nl_family __ro_after_init = {
> 	.name		= DEVLINK_GENL_NAME,
> 	.version	= DEVLINK_GENL_VERSION,
>-- 
>2.38.1
>

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

* Re: [PATCH net-next 02/14] devlink: split out core code
  2023-01-04  9:50   ` Jiri Pirko
@ 2023-01-05  2:10     ` Jakub Kicinski
  2023-01-06 23:55       ` Jacob Keller
  0 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-05  2:10 UTC (permalink / raw)
  To: Jiri Pirko; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

On Wed, 4 Jan 2023 10:50:32 +0100 Jiri Pirko wrote:
> >Rename devlink_netdevice_event() to make it clear that it only touches
> >ports (that's the only change which isn't a pure code move).  
> 
> Did you do any other changes on the move? 

Please read the paragraph you're quoting again.
I specifically addressed this question.

> I believe that for patches like this that move a lot of code it is
> beneficial to move the code "as is". The changes could be done in a
> separate patches, for the ease of review purposes.

I obviously know that. That's why patch 1 and patch 2 are separate.
The line between what warrants a separate patch and what doesn't
is somewhat subjective.

> Could you please?

Sure.. :/

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

* Re: [PATCH net-next 05/14] devlink: use an explicit structure for dump context
  2023-01-04 10:04   ` Jiri Pirko
@ 2023-01-05  2:22     ` Jakub Kicinski
  0 siblings, 0 replies; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-05  2:22 UTC (permalink / raw)
  To: Jiri Pirko; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

On Wed, 4 Jan 2023 11:04:01 +0100 Jiri Pirko wrote:
> Wed, Jan 04, 2023 at 05:16:27AM CET, kuba@kernel.org wrote:
> >Create a dump context structure instead of using cb->args
> >as an unsigned long array. This is a pure conversion which
> >is intended to be as much of a noop as possible.
> >Subsequent changes will use this to simplify the code.
> >
> >The two non-trivial parts are:
> > - devlink_nl_cmd_health_reporter_dump_get_dumpit() checks args[0]
> >   to see if devlink_fmsg_dumpit() has already been called (whether
> >   this is the first msg), but doesn't use the exact value, so we
> >   can drop the local variable there already
> > - devlink_nl_cmd_region_read_dumpit() uses args[0] for address
> >   but we'll use args[1] now, shouldn't matter  
> 
> I don't follow this. Where do you use args[1]? you mean
> dump->start_offset?

Yes, it used to be stored at the start of the args/cb buffer,
now it's stored after index, even tho index is not used.

> If yes, it does not matter at all and I think
> mentioning that only confuses reader (as it did for me).

I think that's fine. I want a note for myself that I knew the "binary
layouts" are changed.

> >diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
> >index e01ba7999b91..bcc930b7cfcf 100644
> >--- a/net/devlink/leftover.c
> >+++ b/net/devlink/leftover.c
> >@@ -1222,9 +1222,10 @@ static void devlink_rate_notify(struct devlink_rate *devlink_rate,
> > static int devlink_nl_cmd_rate_get_dumpit(struct sk_buff *msg,
> > 					  struct netlink_callback *cb)
> > {
> >+	struct devlink_nl_dump_state *dump = devl_dump_state(cb);  
> 
> Could this be named "state" or "dump_state"? "dump" is not what it is.

Sure...

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

* Re: [PATCH net-next 09/14] devlink: restart dump based on devlink instance ids (simple)
  2023-01-04 14:18   ` Jiri Pirko
@ 2023-01-05  3:21     ` Jakub Kicinski
  0 siblings, 0 replies; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-05  3:21 UTC (permalink / raw)
  To: Jiri Pirko; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

On Wed, 4 Jan 2023 15:18:54 +0100 Jiri Pirko wrote:
> >@@ -117,6 +122,15 @@ struct devlink_nl_dump_state {
> > 	};
> > };
> > 
> >+/* Iterate over devlink pointers which were possible to get reference to.
> >+ * devlink_put() needs to be called for each iterated devlink pointer
> >+ * in loop body in order to release the reference.
> >+ */
> >+#define devlink_dump_for_each_instance_get(msg, dump, devlink)		\
> >+	for (; (devlink = devlinks_xa_find_get(sock_net(msg->sk),	\  
> 
> I undestand that the "dump" is zeroed at the beginning of dumpit call,
> however, if you call this helper multiple times, the second iteration
> would't not work.
> 
> Perhaps better to initialize instance=0 at the beginning of the loop to
> make this helper calls behaviour independent on context.

Mmm. dump is zeroed when the operation starts. If there are multiple
->dumpit calls / skbs - the dump will not get re-started.

Restarting the instance walk would be tricky in practice (the
subsequent ->dumpit calls would need to know if they are pre- or post-
restarted) - so I don't think we should anticipate having to do this.

Let me rewrite to comment to make the dumpit-only-ness explicit.

> >+					       &dump->instance, xa_find)); \
> >+	     dump->instance++)
> >+
> > extern const struct genl_small_ops devlink_nl_ops[56];
> > 
> > struct devlink *devlink_get_from_attrs(struct net *net, struct nlattr **attrs);
> >diff --git a/net/devlink/leftover.c b/net/devlink/leftover.c
> >index e3cfb64990b4..0f24b321b0bb 100644
> >--- a/net/devlink/leftover.c
> >+++ b/net/devlink/leftover.c
> >@@ -1319,17 +1319,9 @@ static int devlink_nl_cmd_get_dumpit(struct sk_buff *msg,
> > {
> > 	struct devlink_nl_dump_state *dump = devl_dump_state(cb);
> > 	struct devlink *devlink;
> >-	unsigned long index;
> >-	int idx = 0;
> > 	int err;
> > 
> >-	devlinks_xa_for_each_registered_get(sock_net(msg->sk), index, devlink) {
> >-		if (idx < dump->idx) {
> >-			idx++;
> >-			devlink_put(devlink);
> >-			continue;
> >-		}
> >-
> >+	devlink_dump_for_each_instance_get(msg, dump, devlink) {  
> 
> The name suggests on the first sight that you are iterating some dump,
> which is slightly confusing. Perhaps better to have
> "devlinks_xa_for_each_" in the prefix somehow?
> 
> 	devlinks_xa_for_each_registered_get_dumping()
> 
> I know it is long :)

It's only usable in netlink dumps and with clear semantic implications.
More generic name is not appropriate IMO.
Maybe after you remove the line card locks we could remove the macro
completely? It only has 3 users after my patches - on in the generic
dump handling and one for the subobjects which have their own locks.

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

* Re: [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-04 16:50   ` Jiri Pirko
@ 2023-01-05  3:46     ` Jakub Kicinski
  2023-01-05  9:02       ` Jiri Pirko
  2023-01-06 11:25       ` Jiri Pirko
  0 siblings, 2 replies; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-05  3:46 UTC (permalink / raw)
  To: Jiri Pirko; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

On Wed, 4 Jan 2023 17:50:33 +0100 Jiri Pirko wrote:
> Wed, Jan 04, 2023 at 05:16:35AM CET, kuba@kernel.org wrote:
> >Most dumpit implementations walk the devlink instances.
> >This requires careful lock taking and reference dropping.
> >Factor the loop out and provide just a callback to handle
> >a single instance dump.
> >
> >Convert one user as an example, other users converted
> >in the next change.
> >
> >Slightly inspired by ethtool netlink code.

> >diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
> >index 5adac38454fd..e49b82dd77cd 100644
> >--- a/net/devlink/devl_internal.h
> >+++ b/net/devlink/devl_internal.h
> >@@ -122,6 +122,11 @@ struct devlink_nl_dump_state {
> > 	};
> > };
> > 
> >+struct devlink_gen_cmd {  
> 
> What is "gen"? Generic netlink?

Generic devlink command. In other words the implementation 
is straightforward enough to factor out the common parts.

> Not sure why perhaps "nl" would be fine to be consistent with the
> rest of the code? Why "cmd"? That looks a bit odd to me.
> 
> >+	int (*dump_one)(struct sk_buff *msg, struct devlink
> >*devlink,
> >+			struct netlink_callback *cb);  
> 
> Do you plan to have more callbacks here? If no, wouldn't it be better
> to just have typedef and assign the pointer to the dump_one in
> devl_gen_cmds array?

If I find the time - yes, more refactoring is possible.

> >+};
> >+
> > /* Iterate over devlink pointers which were possible to get
> > reference to.
> >  * devlink_put() needs to be called for each iterated devlink
> > pointer
> >  * in loop body in order to release the reference.
> >@@ -138,6 +143,9 @@ struct devlink *devlink_get_from_attrs(struct
> >net *net, struct nlattr **attrs);
> > void devlink_notify_unregister(struct devlink *devlink);
> > void devlink_notify_register(struct devlink *devlink);
> > 
> >+int devlink_instance_iter_dump(struct sk_buff *msg,
> >+			       struct netlink_callback *cb);
> >+
> > static inline struct devlink_nl_dump_state *
> > devl_dump_state(struct netlink_callback *cb)
> > {
> >@@ -173,6 +181,8 @@ devlink_linecard_get_from_info(struct devlink
> >*devlink, struct genl_info *info);
> > void devlink_linecard_put(struct devlink_linecard *linecard);
> > 
> > /* Rates */
> >+extern const struct devlink_gen_cmd devl_gen_rate_get;  
> 
> The struct name is *_cmd, not sure why the variable name is *_get
> Shouldn't it be rather devl_gen_cmd_rate?

It is the implementation of get.. there's also a set command.. 
which would be under a different index...

> >+			dump->idx = idx;
> >+			break;
> >+		}
> >+		idx++;
> > 	}
> >-out:
> >-	if (err != -EMSGSIZE)
> >-		return err;
> > 
> >-	return msg->len;
> >+	return err;
> > }
> > 
> >+const struct devlink_gen_cmd devl_gen_rate_get = {
> >+	.dump_one		=
> >devlink_nl_cmd_rate_get_dumpinst,  
> 
> dump_one/dumpinst inconsistency in names

Sure...

> >+};
> >+
> > static int devlink_nl_cmd_rate_get_doit(struct sk_buff *skb,
> > 					struct genl_info *info)
> > {
> >@@ -9130,7 +9123,7 @@ const struct genl_small_ops devlink_nl_ops[56]
> >= {
> > 	{
> > 		.cmd = DEVLINK_CMD_RATE_GET,
> > 		.doit = devlink_nl_cmd_rate_get_doit,
> >-		.dumpit = devlink_nl_cmd_rate_get_dumpit,
> >+		.dumpit = devlink_instance_iter_dump,  
> 
> again, inconsistency:
> devlink_instance_iter_dumpit

You mean it doesn't have nl, cmd, dump_one in the name?
Could you *please* at least say what you want the names to be if you're
sending all those subjective nit picks? :/

I'll call it devlink_nl_instance_iter_dump

> > 		.internal_flags = DEVLINK_NL_FLAG_NEED_RATE,
> > 		/* can be retrieved by unprivileged users */  
> 
> Unrelated to this patch, I wonder, why you didn't move devlink_nl_ops
> along with the rest of the netlink code to netlink.c?

It's explained in the commit message for patch 3 :/

> > 	},
> >diff --git a/net/devlink/netlink.c b/net/devlink/netlink.c
> >index ce1a7d674d14..fcf10c288480 100644
> >--- a/net/devlink/netlink.c
> >+++ b/net/devlink/netlink.c
> >@@ -5,6 +5,7 @@
> >  */
> > 
> > #include <net/genetlink.h>
> >+#include <net/sock.h>
> > 
> > #include "devl_internal.h"
> > 
> >@@ -177,6 +178,38 @@ static void devlink_nl_post_doit(const struct
> >genl_split_ops *ops,
> > 	devlink_put(devlink);
> > }
> > 
> >+static const struct devlink_gen_cmd *devl_gen_cmds[] = {
> >+	[DEVLINK_CMD_RATE_GET]		= &devl_gen_rate_get,
> > 
> 
> static const devlink_nl_dump_one_t *devlink_nl_dump_one[] = {
> 	[DEVLINK_CMD_RATE_GET]	= &devl_nl_rate_dump_one,
> }
> Maybe? (not sure how the devlink/devl should be used here though)

Nope.

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

* Re: [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-05  3:46     ` Jakub Kicinski
@ 2023-01-05  9:02       ` Jiri Pirko
  2023-01-05 18:24         ` Jakub Kicinski
  2023-01-06 11:25       ` Jiri Pirko
  1 sibling, 1 reply; 42+ messages in thread
From: Jiri Pirko @ 2023-01-05  9:02 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Thu, Jan 05, 2023 at 04:46:04AM CET, kuba@kernel.org wrote:
>On Wed, 4 Jan 2023 17:50:33 +0100 Jiri Pirko wrote:
>> Wed, Jan 04, 2023 at 05:16:35AM CET, kuba@kernel.org wrote:
>> >Most dumpit implementations walk the devlink instances.
>> >This requires careful lock taking and reference dropping.
>> >Factor the loop out and provide just a callback to handle
>> >a single instance dump.
>> >
>> >Convert one user as an example, other users converted
>> >in the next change.
>> >
>> >Slightly inspired by ethtool netlink code.
>
>> >diff --git a/net/devlink/devl_internal.h b/net/devlink/devl_internal.h
>> >index 5adac38454fd..e49b82dd77cd 100644
>> >--- a/net/devlink/devl_internal.h
>> >+++ b/net/devlink/devl_internal.h
>> >@@ -122,6 +122,11 @@ struct devlink_nl_dump_state {
>> > 	};
>> > };
>> > 
>> >+struct devlink_gen_cmd {  
>> 
>> What is "gen"? Generic netlink?
>
>Generic devlink command. In other words the implementation 
>is straightforward enough to factor out the common parts.

Could it be "genl" then?


>
>> Not sure why perhaps "nl" would be fine to be consistent with the
>> rest of the code? Why "cmd"? That looks a bit odd to me.
>> 
>> >+	int (*dump_one)(struct sk_buff *msg, struct devlink
>> >*devlink,
>> >+			struct netlink_callback *cb);  
>> 
>> Do you plan to have more callbacks here? If no, wouldn't it be better
>> to just have typedef and assign the pointer to the dump_one in
>> devl_gen_cmds array?
>
>If I find the time - yes, more refactoring is possible.

Could you elaborate a bit more about that?


>
>> >+};
>> >+
>> > /* Iterate over devlink pointers which were possible to get
>> > reference to.
>> >  * devlink_put() needs to be called for each iterated devlink
>> > pointer
>> >  * in loop body in order to release the reference.
>> >@@ -138,6 +143,9 @@ struct devlink *devlink_get_from_attrs(struct
>> >net *net, struct nlattr **attrs);
>> > void devlink_notify_unregister(struct devlink *devlink);
>> > void devlink_notify_register(struct devlink *devlink);
>> > 
>> >+int devlink_instance_iter_dump(struct sk_buff *msg,
>> >+			       struct netlink_callback *cb);
>> >+
>> > static inline struct devlink_nl_dump_state *
>> > devl_dump_state(struct netlink_callback *cb)
>> > {
>> >@@ -173,6 +181,8 @@ devlink_linecard_get_from_info(struct devlink
>> >*devlink, struct genl_info *info);
>> > void devlink_linecard_put(struct devlink_linecard *linecard);
>> > 
>> > /* Rates */
>> >+extern const struct devlink_gen_cmd devl_gen_rate_get;  
>> 
>> The struct name is *_cmd, not sure why the variable name is *_get
>> Shouldn't it be rather devl_gen_cmd_rate?
>
>It is the implementation of get.. there's also a set command.. 
>which would be under a different index...

I see.


>
>> >+			dump->idx = idx;
>> >+			break;
>> >+		}
>> >+		idx++;
>> > 	}
>> >-out:
>> >-	if (err != -EMSGSIZE)
>> >-		return err;
>> > 
>> >-	return msg->len;
>> >+	return err;
>> > }
>> > 
>> >+const struct devlink_gen_cmd devl_gen_rate_get = {
>> >+	.dump_one		=
>> >devlink_nl_cmd_rate_get_dumpinst,  
>> 
>> dump_one/dumpinst inconsistency in names
>
>Sure...
>
>> >+};
>> >+
>> > static int devlink_nl_cmd_rate_get_doit(struct sk_buff *skb,
>> > 					struct genl_info *info)
>> > {
>> >@@ -9130,7 +9123,7 @@ const struct genl_small_ops devlink_nl_ops[56]
>> >= {
>> > 	{
>> > 		.cmd = DEVLINK_CMD_RATE_GET,
>> > 		.doit = devlink_nl_cmd_rate_get_doit,
>> >-		.dumpit = devlink_nl_cmd_rate_get_dumpit,
>> >+		.dumpit = devlink_instance_iter_dump,  
>> 
>> again, inconsistency:
>> devlink_instance_iter_dumpit
>
>You mean it doesn't have nl, cmd, dump_one in the name?
>Could you *please* at least say what you want the names to be if you're
>sending all those subjective nit picks? :/

Well, I provided a suggested name, not sure why that was not clear.
The point was s/dump/dumpit/ to match the op name.

I'm sorry you find this a subjective nitpick. I believe it is important
to maintain some naming consistency in order to make code readable.
Nothing subjective about it.


>
>I'll call it devlink_nl_instance_iter_dump

devlink_nl_instance_iter_dumpit please.


>
>> > 		.internal_flags = DEVLINK_NL_FLAG_NEED_RATE,
>> > 		/* can be retrieved by unprivileged users */  
>> 
>> Unrelated to this patch, I wonder, why you didn't move devlink_nl_ops
>> along with the rest of the netlink code to netlink.c?
>
>It's explained in the commit message for patch 3 :/

I missed that, sorry.


>
>> > 	},
>> >diff --git a/net/devlink/netlink.c b/net/devlink/netlink.c
>> >index ce1a7d674d14..fcf10c288480 100644
>> >--- a/net/devlink/netlink.c
>> >+++ b/net/devlink/netlink.c
>> >@@ -5,6 +5,7 @@
>> >  */
>> > 
>> > #include <net/genetlink.h>
>> >+#include <net/sock.h>
>> > 
>> > #include "devl_internal.h"
>> > 
>> >@@ -177,6 +178,38 @@ static void devlink_nl_post_doit(const struct
>> >genl_split_ops *ops,
>> > 	devlink_put(devlink);
>> > }
>> > 
>> >+static const struct devlink_gen_cmd *devl_gen_cmds[] = {
>> >+	[DEVLINK_CMD_RATE_GET]		= &devl_gen_rate_get,
>> > 
>> 
>> static const devlink_nl_dump_one_t *devlink_nl_dump_one[] = {
>> 	[DEVLINK_CMD_RATE_GET]	= &devl_nl_rate_dump_one,
>> }
>> Maybe? (not sure how the devlink/devl should be used here though)
>
>Nope.

Awesome.


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

* Re: [PATCH net-next 03/14] devlink: split out netlink code
  2023-01-04  4:16 ` [PATCH net-next 03/14] devlink: split out netlink code Jakub Kicinski
@ 2023-01-05  9:03   ` Jiri Pirko
  2023-01-05 18:20     ` Jakub Kicinski
  0 siblings, 1 reply; 42+ messages in thread
From: Jiri Pirko @ 2023-01-05  9:03 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Wed, Jan 04, 2023 at 05:16:25AM CET, kuba@kernel.org wrote:
>Move out the netlink glue into a separate file.
>Leave the ops in the old file because we'd have to export a ton
>of functions. Going forward we should switch to split ops which

What do you mean by "split ops"?


>will let us to put the new ops in the netlink.c file.
>
>Pure code move, no functional changes.

[...]

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

* Re: [PATCH net-next 03/14] devlink: split out netlink code
  2023-01-05  9:03   ` Jiri Pirko
@ 2023-01-05 18:20     ` Jakub Kicinski
  2023-01-06  8:47       ` Jiri Pirko
  0 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-05 18:20 UTC (permalink / raw)
  To: Jiri Pirko; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

On Thu, 5 Jan 2023 10:03:53 +0100 Jiri Pirko wrote:
> Wed, Jan 04, 2023 at 05:16:25AM CET, kuba@kernel.org wrote:
> >Move out the netlink glue into a separate file.
> >Leave the ops in the old file because we'd have to export a ton
> >of functions. Going forward we should switch to split ops which  
> 
> What do you mean by "split ops"?

struct genl_split_ops

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

* Re: [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-05  9:02       ` Jiri Pirko
@ 2023-01-05 18:24         ` Jakub Kicinski
  2023-01-06  8:56           ` Jiri Pirko
  0 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-05 18:24 UTC (permalink / raw)
  To: Jiri Pirko; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

On Thu, 5 Jan 2023 10:02:54 +0100 Jiri Pirko wrote:
> Thu, Jan 05, 2023 at 04:46:04AM CET, kuba@kernel.org wrote:
> >> What is "gen"? Generic netlink?  
> >
> >Generic devlink command. In other words the implementation 
> >is straightforward enough to factor out the common parts.  
> 
> Could it be "genl" then?

Why? What other kind of command is there?
The distinction is weird vs generic, not genl vs IDK-what.

> >> Do you plan to have more callbacks here? If no, wouldn't it be better
> >> to just have typedef and assign the pointer to the dump_one in
> >> devl_gen_cmds array?  
> >
> >If I find the time - yes, more refactoring is possible.  
> 
> Could you elaborate a bit more about that?

If I recall I was thinking about adding a "fill" op and policy related
info to the structure. The details would fall into place during coding..

> >You mean it doesn't have nl, cmd, dump_one in the name?
> >Could you *please* at least say what you want the names to be if you're
> >sending all those subjective nit picks? :/  
> 
> Well, I provided a suggested name, not sure why that was not clear.
> The point was s/dump/dumpit/ to match the op name.

Oh, just the "it" at the end? Sorry, I don't see the point.

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

* Re: [PATCH net-next 03/14] devlink: split out netlink code
  2023-01-05 18:20     ` Jakub Kicinski
@ 2023-01-06  8:47       ` Jiri Pirko
  0 siblings, 0 replies; 42+ messages in thread
From: Jiri Pirko @ 2023-01-06  8:47 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Thu, Jan 05, 2023 at 07:20:58PM CET, kuba@kernel.org wrote:
>On Thu, 5 Jan 2023 10:03:53 +0100 Jiri Pirko wrote:
>> Wed, Jan 04, 2023 at 05:16:25AM CET, kuba@kernel.org wrote:
>> >Move out the netlink glue into a separate file.
>> >Leave the ops in the old file because we'd have to export a ton
>> >of functions. Going forward we should switch to split ops which  
>> 
>> What do you mean by "split ops"?
>
>struct genl_split_ops

I see.

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

* Re: [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-05 18:24         ` Jakub Kicinski
@ 2023-01-06  8:56           ` Jiri Pirko
  2023-01-06 21:12             ` Jakub Kicinski
  0 siblings, 1 reply; 42+ messages in thread
From: Jiri Pirko @ 2023-01-06  8:56 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Thu, Jan 05, 2023 at 07:24:37PM CET, kuba@kernel.org wrote:
>On Thu, 5 Jan 2023 10:02:54 +0100 Jiri Pirko wrote:
>> Thu, Jan 05, 2023 at 04:46:04AM CET, kuba@kernel.org wrote:
>> >> What is "gen"? Generic netlink?  
>> >
>> >Generic devlink command. In other words the implementation 
>> >is straightforward enough to factor out the common parts.  
>> 
>> Could it be "genl" then?
>
>Why? What other kind of command is there?
>The distinction is weird vs generic, not genl vs IDK-what.

Compare outputs of:
git grep _gen_ net/
git grep _genl_ net/

My point is to see consistent naming scheme. I know you don't care about
that much, but I believe it helps readability and code understanding.
What is the downside? I'm not really sure why you are against it.

>
>> >> Do you plan to have more callbacks here? If no, wouldn't it be better
>> >> to just have typedef and assign the pointer to the dump_one in
>> >> devl_gen_cmds array?  
>> >
>> >If I find the time - yes, more refactoring is possible.  
>> 
>> Could you elaborate a bit more about that?
>
>If I recall I was thinking about adding a "fill" op and policy related
>info to the structure. The details would fall into place during coding..
>
>> >You mean it doesn't have nl, cmd, dump_one in the name?
>> >Could you *please* at least say what you want the names to be if you're
>> >sending all those subjective nit picks? :/  
>> 
>> Well, I provided a suggested name, not sure why that was not clear.
>> The point was s/dump/dumpit/ to match the op name.
>
>Oh, just the "it" at the end? Sorry, I don't see the point.

The point is simple. Ops is a struct of callback by name X. If someone
implements this ops struct, it is nice to assign the callbacks functions
of name y_X so it is obvious from the first sight, what the function
is related to.

I'm not sure what's wrong about having this sort of consistency. I
believe that you as a maintainer should rather enforce it than to be
against it. Am I missing something?

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

* Re: [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-05  3:46     ` Jakub Kicinski
  2023-01-05  9:02       ` Jiri Pirko
@ 2023-01-06 11:25       ` Jiri Pirko
  1 sibling, 0 replies; 42+ messages in thread
From: Jiri Pirko @ 2023-01-06 11:25 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Thu, Jan 05, 2023 at 04:46:04AM CET, kuba@kernel.org wrote:
>On Wed, 4 Jan 2023 17:50:33 +0100 Jiri Pirko wrote:
>> Wed, Jan 04, 2023 at 05:16:35AM CET, kuba@kernel.org wrote:

[...]

>> >@@ -173,6 +181,8 @@ devlink_linecard_get_from_info(struct devlink
>> >*devlink, struct genl_info *info);
>> > void devlink_linecard_put(struct devlink_linecard *linecard);
>> > 
>> > /* Rates */
>> >+extern const struct devlink_gen_cmd devl_gen_rate_get;  

The rest of the commands (next patch) you put in a different place, so
this is alone here.


>> 
>> The struct name is *_cmd, not sure why the variable name is *_get
>> Shouldn't it be rather devl_gen_cmd_rate?
>
>It is the implementation of get.. there's also a set command.. 
>which would be under a different index...

The rest of the commands (next patch) you omitted the "_get" suffix.


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

* Re: [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-06  8:56           ` Jiri Pirko
@ 2023-01-06 21:12             ` Jakub Kicinski
  2023-01-07  9:23               ` Jiri Pirko
  0 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-06 21:12 UTC (permalink / raw)
  To: Jiri Pirko; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

On Fri, 6 Jan 2023 09:56:36 +0100 Jiri Pirko wrote:
>> Oh, just the "it" at the end? Sorry, I don't see the point.  
> 
> The point is simple. Ops is a struct of callback by name X. If someone
> implements this ops struct, it is nice to assign the callbacks functions
> of name y_X so it is obvious from the first sight, what the function
> is related to.
> 
> I'm not sure what's wrong about having this sort of consistency. I
> believe that you as a maintainer should rather enforce it than to be
> against it. Am I missing something?

IMO you have a tendency to form names by concatenating adjacent
information rather than reflecting on what matters to the reader.
I believe the low readability of the devlink code is sufficient 
evidence to disagree with that direction.

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

* Re: [PATCH net-next 02/14] devlink: split out core code
  2023-01-05  2:10     ` Jakub Kicinski
@ 2023-01-06 23:55       ` Jacob Keller
  0 siblings, 0 replies; 42+ messages in thread
From: Jacob Keller @ 2023-01-06 23:55 UTC (permalink / raw)
  To: Jakub Kicinski, Jiri Pirko; +Cc: davem, netdev, edumazet, pabeni



On 1/4/2023 6:10 PM, Jakub Kicinski wrote:
> On Wed, 4 Jan 2023 10:50:32 +0100 Jiri Pirko wrote:
>>> Rename devlink_netdevice_event() to make it clear that it only touches
>>> ports (that's the only change which isn't a pure code move).  
>>
>> Did you do any other changes on the move? 
> 
> Please read the paragraph you're quoting again.
> I specifically addressed this question.
> 
>> I believe that for patches like this that move a lot of code it is
>> beneficial to move the code "as is". The changes could be done in a
>> separate patches, for the ease of review purposes.
> 
> I obviously know that. That's why patch 1 and patch 2 are separate.
> The line between what warrants a separate patch and what doesn't
> is somewhat subjective.
> 
>> Could you please?
> 
> Sure.. :/

I am fine with it as-is, but it would be easier to review without the
change.

I typically review such changes using git diff's --color-moved-lines
option to make it easier to see what changed vs not and ensure it lines
up with what was described.

Thanks,
Jake

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

* Re: [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-06 21:12             ` Jakub Kicinski
@ 2023-01-07  9:23               ` Jiri Pirko
  2023-01-09 19:49                 ` Jakub Kicinski
  0 siblings, 1 reply; 42+ messages in thread
From: Jiri Pirko @ 2023-01-07  9:23 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Fri, Jan 06, 2023 at 10:12:14PM CET, kuba@kernel.org wrote:
>On Fri, 6 Jan 2023 09:56:36 +0100 Jiri Pirko wrote:
>>> Oh, just the "it" at the end? Sorry, I don't see the point.  
>> 
>> The point is simple. Ops is a struct of callback by name X. If someone
>> implements this ops struct, it is nice to assign the callbacks functions
>> of name y_X so it is obvious from the first sight, what the function
>> is related to.
>> 
>> I'm not sure what's wrong about having this sort of consistency. I
>> believe that you as a maintainer should rather enforce it than to be
>> against it. Am I missing something?
>
>IMO you have a tendency to form names by concatenating adjacent
>information rather than reflecting on what matters to the reader.
>I believe the low readability of the devlink code is sufficient 
>evidence to disagree with that direction.

Hmm.
1) What is wrong of having:
   .dumpit = devlink_instance_iter_dumpit
   instead of
   .dumpit = devlink_instance_iter_dump
   ?
   How exactly that decreases readability?

2) Why exactly you find devlink code hard to read?


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

* Re: [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-07  9:23               ` Jiri Pirko
@ 2023-01-09 19:49                 ` Jakub Kicinski
  2023-01-10 14:31                   ` Jiri Pirko
  0 siblings, 1 reply; 42+ messages in thread
From: Jakub Kicinski @ 2023-01-09 19:49 UTC (permalink / raw)
  To: Jiri Pirko; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

On Sat, 7 Jan 2023 10:23:48 +0100 Jiri Pirko wrote:
> Hmm.
> 1) What is wrong of having:
>    .dumpit = devlink_instance_iter_dumpit
>    instead of
>    .dumpit = devlink_instance_iter_dump
>    ?
>    How exactly that decreases readability?

The "it" at the end of the function name is there because do is a C
keyword, so we can't call the do callback do, we must call it doit.

The further from netlink core we get the more this is an API wart 
and the less it makes sense. 
instance iter dump is closer to plain English.

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

* Re: [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-09 19:49                 ` Jakub Kicinski
@ 2023-01-10 14:31                   ` Jiri Pirko
  2023-01-12  0:13                     ` Jacob Keller
  0 siblings, 1 reply; 42+ messages in thread
From: Jiri Pirko @ 2023-01-10 14:31 UTC (permalink / raw)
  To: Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni, jacob.e.keller

Mon, Jan 09, 2023 at 08:49:49PM CET, kuba@kernel.org wrote:
>On Sat, 7 Jan 2023 10:23:48 +0100 Jiri Pirko wrote:
>> Hmm.
>> 1) What is wrong of having:
>>    .dumpit = devlink_instance_iter_dumpit
>>    instead of
>>    .dumpit = devlink_instance_iter_dump
>>    ?
>>    How exactly that decreases readability?
>
>The "it" at the end of the function name is there because do is a C
>keyword, so we can't call the do callback do, we must call it doit.
>
>The further from netlink core we get the more this is an API wart 
>and the less it makes sense. 
>instance iter dump is closer to plain English.

Hmm, I guess if you are not happy about the callback name, you should
change it, to ".dump" in this case. My point the the naming consistency
between the callback name and the function assigned. But nevermind.

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

* Re: [PATCH net-next 13/14] devlink: add by-instance dump infra
  2023-01-10 14:31                   ` Jiri Pirko
@ 2023-01-12  0:13                     ` Jacob Keller
  0 siblings, 0 replies; 42+ messages in thread
From: Jacob Keller @ 2023-01-12  0:13 UTC (permalink / raw)
  To: Jiri Pirko, Jakub Kicinski; +Cc: davem, netdev, edumazet, pabeni



On 1/10/2023 6:31 AM, Jiri Pirko wrote:
> Mon, Jan 09, 2023 at 08:49:49PM CET, kuba@kernel.org wrote:
>> On Sat, 7 Jan 2023 10:23:48 +0100 Jiri Pirko wrote:
>>> Hmm.
>>> 1) What is wrong of having:
>>>    .dumpit = devlink_instance_iter_dumpit
>>>    instead of
>>>    .dumpit = devlink_instance_iter_dump
>>>    ?
>>>    How exactly that decreases readability?
>>
>> The "it" at the end of the function name is there because do is a C
>> keyword, so we can't call the do callback do, we must call it doit.
>>
>> The further from netlink core we get the more this is an API wart 
>> and the less it makes sense. 
>> instance iter dump is closer to plain English.
> 
> Hmm, I guess if you are not happy about the callback name, you should
> change it, to ".dump" in this case. My point the the naming consistency
> between the callback name and the function assigned. But nevermind.

+1 for having the callback part and the name match. I don't particularly
care if its .dump or .dumpit, but I do like having the callback match
the struct member. That being said, its not really a huge deal.

Thanks,
Jake

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

end of thread, other threads:[~2023-01-12  0:14 UTC | newest]

Thread overview: 42+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2023-01-04  4:16 [PATCH net-next 00/14] devlink: code split and structured instance walk Jakub Kicinski
2023-01-04  4:16 ` [PATCH net-next 01/14] devlink: move code to a dedicated directory Jakub Kicinski
2023-01-04  4:16 ` [PATCH net-next 02/14] devlink: split out core code Jakub Kicinski
2023-01-04  9:50   ` Jiri Pirko
2023-01-05  2:10     ` Jakub Kicinski
2023-01-06 23:55       ` Jacob Keller
2023-01-04  4:16 ` [PATCH net-next 03/14] devlink: split out netlink code Jakub Kicinski
2023-01-05  9:03   ` Jiri Pirko
2023-01-05 18:20     ` Jakub Kicinski
2023-01-06  8:47       ` Jiri Pirko
2023-01-04  4:16 ` [PATCH net-next 04/14] netlink: add macro for checking dump ctx size Jakub Kicinski
2023-01-04  9:51   ` Jiri Pirko
2023-01-04  4:16 ` [PATCH net-next 05/14] devlink: use an explicit structure for dump context Jakub Kicinski
2023-01-04 10:04   ` Jiri Pirko
2023-01-05  2:22     ` Jakub Kicinski
2023-01-04  4:16 ` [PATCH net-next 06/14] devlink: remove start variables from dumps Jakub Kicinski
2023-01-04  4:16 ` [PATCH net-next 07/14] devlink: drop the filter argument from devlinks_xa_find_get Jakub Kicinski
2023-01-04 10:05   ` Jiri Pirko
2023-01-04  4:16 ` [PATCH net-next 08/14] devlink: health: combine loops in dump Jakub Kicinski
2023-01-04 10:06   ` Jiri Pirko
2023-01-04  4:16 ` [PATCH net-next 09/14] devlink: restart dump based on devlink instance ids (simple) Jakub Kicinski
2023-01-04 14:18   ` Jiri Pirko
2023-01-05  3:21     ` Jakub Kicinski
2023-01-04  4:16 ` [PATCH net-next 10/14] devlink: restart dump based on devlink instance ids (nested) Jakub Kicinski
2023-01-04 15:47   ` Jiri Pirko
2023-01-04  4:16 ` [PATCH net-next 11/14] devlink: restart dump based on devlink instance ids (function) Jakub Kicinski
2023-01-04 15:51   ` Jiri Pirko
2023-01-04  4:16 ` [PATCH net-next 12/14] devlink: uniformly take the devlink instance lock in the dump loop Jakub Kicinski
2023-01-04 15:52   ` Jiri Pirko
2023-01-04  4:16 ` [PATCH net-next 13/14] devlink: add by-instance dump infra Jakub Kicinski
2023-01-04 16:50   ` Jiri Pirko
2023-01-05  3:46     ` Jakub Kicinski
2023-01-05  9:02       ` Jiri Pirko
2023-01-05 18:24         ` Jakub Kicinski
2023-01-06  8:56           ` Jiri Pirko
2023-01-06 21:12             ` Jakub Kicinski
2023-01-07  9:23               ` Jiri Pirko
2023-01-09 19:49                 ` Jakub Kicinski
2023-01-10 14:31                   ` Jiri Pirko
2023-01-12  0:13                     ` Jacob Keller
2023-01-06 11:25       ` Jiri Pirko
2023-01-04  4:16 ` [PATCH net-next 14/14] devlink: convert remaining dumps to the by-instance scheme Jakub Kicinski

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