bpf.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [RFC PATCH bpf-next v2 0/2] bpf: adding map batch processing support
@ 2019-09-06 22:54 Yonghong Song
  2019-09-06 22:54 ` [RFC PATCH bpf-next v2 1/2] " Yonghong Song
  2019-09-06 22:54 ` [RFC PATCH bpf-next v2 2/2] tools/bpf: test bpf_map_lookup_and_delete_batch() Yonghong Song
  0 siblings, 2 replies; 5+ messages in thread
From: Yonghong Song @ 2019-09-06 22:54 UTC (permalink / raw)
  To: ast, daniel, netdev, bpf
  Cc: kernel-team, Jakub Kicinski, Brian Vazquez, Stanislav Fomichev

Previous discussion at:
  https://lore.kernel.org/bpf/7ba9b492-8a08-a1d0-9c6e-03be4b8e5e07@fb.com/T/#t

Previous approach tries to use existing per-map looks like
bpf_map_{get_next_key, lookup_elem, update_elem, delete_elem}
to implement a batching process.

It has a series drawback when the prev_key used by bpf_map_get_next_key()
is not in hash table. In that case, as the hash table has no idea where
the `prev_key` has been placed in the bucket before deletion, currently,
it returns the first key. This makes batch processing may see
duplicated elements, or in worst case if the hash table has heavy
update/delete, the batch processing may never finish.

This RFC patch set implements bucket based batching for hashtab.
That is, for lookup/delete, either the whole bucket is processed
or none of elements in the bucket is processed. Forward progress
is also guaranteed as long as user provides enough buffer.

This RFC also serves as a base for discussion at upcoming
LPC2019 BPF Microconference.

Changelogs:
   v1 -> RFC v2:
     . To address the bpf_map_get_next_key() issue where
       if a key is not available the first key will be returned,
       implement per-map batch operations for hashtab/lru_hashtab,
       using bucket lock, as suggested by Alexei.

Cc: Jakub Kicinski <jakub.kicinski@netronome.com>
Cc: Brian Vazquez <brianvv@google.com>
Cc: Stanislav Fomichev <sdf@google.com>

Yonghong Song (2):
  bpf: adding map batch processing support
  tools/bpf: test bpf_map_lookup_and_delete_batch()

 include/linux/bpf.h                           |   9 +
 include/uapi/linux/bpf.h                      |  22 ++
 kernel/bpf/hashtab.c                          | 324 ++++++++++++++++++
 kernel/bpf/syscall.c                          |  68 ++++
 tools/include/uapi/linux/bpf.h                |  22 ++
 tools/lib/bpf/bpf.c                           |  59 ++++
 tools/lib/bpf/bpf.h                           |  13 +
 tools/lib/bpf/libbpf.map                      |   4 +
 .../map_tests/map_lookup_and_delete_batch.c   | 155 +++++++++
 9 files changed, 676 insertions(+)
 create mode 100644 tools/testing/selftests/bpf/map_tests/map_lookup_and_delete_batch.c

-- 
2.17.1


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

* [RFC PATCH bpf-next v2 1/2] bpf: adding map batch processing support
  2019-09-06 22:54 [RFC PATCH bpf-next v2 0/2] bpf: adding map batch processing support Yonghong Song
@ 2019-09-06 22:54 ` Yonghong Song
  2019-09-23 23:20   ` Brian Vazquez
  2019-09-06 22:54 ` [RFC PATCH bpf-next v2 2/2] tools/bpf: test bpf_map_lookup_and_delete_batch() Yonghong Song
  1 sibling, 1 reply; 5+ messages in thread
From: Yonghong Song @ 2019-09-06 22:54 UTC (permalink / raw)
  To: ast, daniel, netdev, bpf
  Cc: kernel-team, Jakub Kicinski, Brian Vazquez, Stanislav Fomichev

Brian Vazquez has proposed BPF_MAP_DUMP command to look up more than one
map entries per syscall.
  https://lore.kernel.org/bpf/CABCgpaU3xxX6CMMxD+1knApivtc2jLBHysDXw-0E9bQEL0qC3A@mail.gmail.com/T/#t

During discussion, we found more use cases can be supported in a similar
map operation batching framework. For example, batched map lookup and delete,
which can be really helpful for bcc.
  https://github.com/iovisor/bcc/blob/master/tools/tcptop.py#L233-L243
  https://github.com/iovisor/bcc/blob/master/tools/slabratetop.py#L129-L138

Also, in bcc, we have API to delete all entries in a map.
  https://github.com/iovisor/bcc/blob/master/src/cc/api/BPFTable.h#L257-L264

For map update, batched operations also useful as sometimes applications need
to populate initial maps with more than one entry. For example, the below
example is from kernel/samples/bpf/xdp_redirect_cpu_user.c:
  https://github.com/torvalds/linux/blob/master/samples/bpf/xdp_redirect_cpu_user.c#L543-L550

This patch addresses all the above use cases. To make uapi stable, it also
covers other potential use cases. For bpf syscall subcommands are introduced:
        BPF_MAP_LOOKUP_BATCH
        BPF_MAP_LOOKUP_AND_DELETE_BATCH
        BPF_MAP_UPDATE_BATCH
        BPF_MAP_DELETE_BATCH

The UAPI attribute structure looks like:

    struct { /* struct used by BPF_MAP_*_BATCH commands */
            __u64           batch;  /* input/output:
                                     * input: start batch,
                                     *        0 to start from beginning.
                                     * output: next start batch,
                                     *         0 to end batching.
                                     */
            __aligned_u64   keys;
            __aligned_u64   values;
            __u32           count;  /* input/output:
                                     * input: # of elements keys/values.
                                     * output: # of filled elements.
                                     */
            __u32           map_fd;
            __u64           elem_flags;
            __u64           flags;
    } batch;

An opaque value 'batch' is used for user/kernel space communication
for where in the map to start the operation for lookup/lookup_and_delete/delete.
  input 'batch' = 0: to start the operation from the beginning of the map.
  output 'batch': if not 0, the next input for batch operation.

For lookup/lookup_and_delete:
  operation: lookup/lookup_and_delete starting from a particular 'batch'.
  return:
     'batch'       'count'     return code     meaning
      0            0           0               Done. Nothing left
      0            0           -ENOSPC         no space to handle batch 0
      > 0          0           -ENOSPC         no space to handle 'batch'
      > 0          > 0         0               stopped right before 'batch'
Note that:
  (1). Even if return code is 0 and return 'count' > 0, the return 'count' may
       not be equal to input 'count'. This happens when there is no enough space
       to handle a batch.
  (2). If the return code is an error and not -EFAULT,
       'batch' indicates the batch has issues and 'count' indicates the number
       of elements successfully processed.

For delete:
  operation: deletion starting from a particular 'batch'.
  return: 0 means everything is deleted from 'batch'.
          error code means something deletion not happening.

For update:
  operation: update 'count' number of elements in 'keys'/'values'.
  return: 0 means successful updates for all elements.
          error code, if not -EFAULT, 'count' is the number of successful updates.

Signed-off-by: Yonghong Song <yhs@fb.com>
---
 include/linux/bpf.h      |   9 ++
 include/uapi/linux/bpf.h |  22 +++
 kernel/bpf/hashtab.c     | 324 +++++++++++++++++++++++++++++++++++++++
 kernel/bpf/syscall.c     |  68 ++++++++
 4 files changed, 423 insertions(+)

diff --git a/include/linux/bpf.h b/include/linux/bpf.h
index 5b9d22338606..3c1302e8e2d4 100644
--- a/include/linux/bpf.h
+++ b/include/linux/bpf.h
@@ -37,6 +37,15 @@ struct bpf_map_ops {
 	int (*map_get_next_key)(struct bpf_map *map, void *key, void *next_key);
 	void (*map_release_uref)(struct bpf_map *map);
 	void *(*map_lookup_elem_sys_only)(struct bpf_map *map, void *key);
+	int (*map_lookup_batch)(struct bpf_map *map, const union bpf_attr *attr,
+				union bpf_attr __user *uattr);
+	int (*map_lookup_and_delete_batch)(struct bpf_map *map,
+					   const union bpf_attr *attr,
+					   union bpf_attr __user *uattr);
+	int (*map_update_batch)(struct bpf_map *map, const union bpf_attr *attr,
+				union bpf_attr __user *uattr);
+	int (*map_delete_batch)(struct bpf_map *map, const union bpf_attr *attr,
+				union bpf_attr __user *uattr);
 
 	/* funcs callable from userspace and from eBPF programs */
 	void *(*map_lookup_elem)(struct bpf_map *map, void *key);
diff --git a/include/uapi/linux/bpf.h b/include/uapi/linux/bpf.h
index 5d2fb183ee2d..9d4f76073dd9 100644
--- a/include/uapi/linux/bpf.h
+++ b/include/uapi/linux/bpf.h
@@ -107,6 +107,10 @@ enum bpf_cmd {
 	BPF_MAP_LOOKUP_AND_DELETE_ELEM,
 	BPF_MAP_FREEZE,
 	BPF_BTF_GET_NEXT_ID,
+	BPF_MAP_LOOKUP_BATCH,
+	BPF_MAP_LOOKUP_AND_DELETE_BATCH,
+	BPF_MAP_UPDATE_BATCH,
+	BPF_MAP_DELETE_BATCH,
 };
 
 enum bpf_map_type {
@@ -396,6 +400,24 @@ union bpf_attr {
 		__u64		flags;
 	};
 
+	struct { /* struct used by BPF_MAP_*_BATCH commands */
+		__u64		batch;	/* input/output:
+					 * input: start batch,
+					 *        0 to start from beginning.
+					 * output: next start batch,
+					 *         0 to end batching.
+					 */
+		__aligned_u64	keys;
+		__aligned_u64	values;
+		__u32		count;	/* input/output:
+					 * input: # of elements keys/values.
+					 * output: # of filled elements.
+					 */
+		__u32		map_fd;
+		__u64		elem_flags;
+		__u64		flags;
+	} batch;
+
 	struct { /* anonymous struct used by BPF_PROG_LOAD command */
 		__u32		prog_type;	/* one of enum bpf_prog_type */
 		__u32		insn_cnt;
diff --git a/kernel/bpf/hashtab.c b/kernel/bpf/hashtab.c
index 22066a62c8c9..ee7b90200f4d 100644
--- a/kernel/bpf/hashtab.c
+++ b/kernel/bpf/hashtab.c
@@ -1232,6 +1232,322 @@ static void htab_map_seq_show_elem(struct bpf_map *map, void *key,
 	rcu_read_unlock();
 }
 
+static int
+__htab_map_lookup_and_delete_batch(struct bpf_map *map,
+				   const union bpf_attr *attr,
+				   union bpf_attr __user *uattr,
+				   bool do_delete, bool is_lru_map)
+{
+	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
+	u32 bucket_cnt, total, key_size, value_size, roundup_key_size;
+	void *keys = NULL, *values = NULL, *value, *dst_key, *dst_val;
+	u64 elem_map_flags, map_flags;
+	struct hlist_nulls_head *head;
+	void __user *ukeys, *uvalues;
+	struct hlist_nulls_node *n;
+	u32 batch, max_count;
+	unsigned long flags;
+	struct htab_elem *l;
+	struct bucket *b;
+	int ret = 0;
+
+	max_count = attr->batch.count;
+	if (!max_count)
+		return 0;
+
+	elem_map_flags = attr->batch.elem_flags;
+	if ((elem_map_flags & ~BPF_F_LOCK) ||
+	    ((elem_map_flags & BPF_F_LOCK) && !map_value_has_spin_lock(map)))
+		return -EINVAL;
+
+	map_flags = attr->batch.flags;
+	if (map_flags)
+		return -EINVAL;
+
+	batch = (u32)attr->batch.batch;
+	if (batch >= htab->n_buckets)
+		return -EINVAL;
+
+	/* We cannot do copy_from_user or copy_to_user inside
+	 * the rcu_read_lock. Allocate enough space here.
+	 */
+	key_size = htab->map.key_size;
+	roundup_key_size = round_up(htab->map.key_size, 8);
+	value_size = htab->map.value_size;
+	keys = kvmalloc(key_size * max_count, GFP_USER | __GFP_NOWARN);
+	values = kvmalloc(value_size * max_count, GFP_USER | __GFP_NOWARN);
+	if (!keys || !values) {
+		ret = -ENOMEM;
+		goto out;
+	}
+
+	dst_key = keys;
+	dst_val = values;
+	total = 0;
+
+	preempt_disable();
+	this_cpu_inc(bpf_prog_active);
+	rcu_read_lock();
+
+again:
+	b = &htab->buckets[batch];
+	head = &b->head;
+	raw_spin_lock_irqsave(&b->lock, flags);
+
+	bucket_cnt = 0;
+	hlist_nulls_for_each_entry_rcu(l, n, head, hash_node)
+		bucket_cnt++;
+
+	if (bucket_cnt > (max_count - total)) {
+		if (total == 0)
+			ret = -ENOSPC;
+		goto after_loop;
+	}
+
+	hlist_nulls_for_each_entry_rcu(l, n, head, hash_node) {
+		memcpy(dst_key, l->key, key_size);
+
+		value = l->key + roundup_key_size;
+		if (elem_map_flags & BPF_F_LOCK)
+			copy_map_value_locked(map, dst_val, value, true);
+		else
+			copy_map_value(map, dst_val, value);
+		check_and_init_map_lock(map, dst_val);
+
+		dst_key += key_size;
+		dst_val += value_size;
+		total++;
+	}
+
+	if (do_delete) {
+		hlist_nulls_for_each_entry_rcu(l, n, head, hash_node) {
+			hlist_nulls_del_rcu(&l->hash_node);
+			if (is_lru_map)
+				bpf_lru_push_free(&htab->lru, &l->lru_node);
+			else
+				free_htab_elem(htab, l);
+		}
+	}
+
+	batch++;
+	if (batch >= htab->n_buckets) {
+		batch = 0;
+		goto after_loop;
+	}
+
+	raw_spin_unlock_irqrestore(&b->lock, flags);
+	goto again;
+
+after_loop:
+	raw_spin_unlock_irqrestore(&b->lock, flags);
+
+	rcu_read_unlock();
+	this_cpu_dec(bpf_prog_active);
+	preempt_enable();
+
+	/* copy data back to user */
+	ukeys = u64_to_user_ptr(attr->batch.keys);
+	uvalues = u64_to_user_ptr(attr->batch.values);
+	if (put_user(batch, &uattr->batch.batch) ||
+	    copy_to_user(ukeys, keys, total * key_size) ||
+	    copy_to_user(uvalues, values, total * value_size) ||
+	    put_user(total, &uattr->batch.count))
+		ret = -EFAULT;
+
+out:
+	kvfree(keys);
+	kvfree(values);
+	return ret;
+}
+
+static int
+__htab_map_update_batch(struct bpf_map *map, const union bpf_attr *attr,
+			union bpf_attr __user *uattr, bool is_lru_map)
+{
+	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
+	u32 count, max_count, key_size, roundup_key_size, value_size;
+	u64 elem_map_flags, map_flags;
+	void __user *ukey, *uvalue;
+	void *key, *value;
+	int ret = 0;
+
+	max_count = attr->batch.count;
+	if (!max_count)
+		return 0;
+
+	elem_map_flags = attr->batch.elem_flags;
+	if ((elem_map_flags & BPF_F_LOCK) && !map_value_has_spin_lock(map))
+		return -EINVAL;
+
+	map_flags = attr->batch.flags;
+	if (map_flags)
+		return -EINVAL;
+
+	key_size = htab->map.key_size;
+	roundup_key_size = round_up(htab->map.key_size, 8);
+	value_size = htab->map.value_size;
+	key = kmalloc(key_size, GFP_USER | __GFP_NOWARN);
+	value = kmalloc(value_size, GFP_USER | __GFP_NOWARN);
+	if (!key || !value) {
+		ret = -ENOMEM;
+		goto out;
+	}
+
+	ukey = u64_to_user_ptr(attr->batch.keys);
+	uvalue = u64_to_user_ptr(attr->batch.values);
+	for (count = 0; count < max_count; count++) {
+		if (copy_from_user(key, ukey + count * key_size, key_size) ||
+		    copy_from_user(value, uvalue + count * value_size, value_size)) {
+			ret = -EFAULT;
+			break;
+		}
+
+		preempt_disable();
+		__this_cpu_inc(bpf_prog_active);
+		rcu_read_lock();
+		if (is_lru_map)
+			ret = htab_lru_map_update_elem(map, key, value, elem_map_flags);
+		else
+			ret = htab_map_update_elem(map, key, value, elem_map_flags);
+		rcu_read_unlock();
+		__this_cpu_dec(bpf_prog_active);
+		preempt_enable();
+
+		if (ret) {
+			if (put_user(count, &uattr->batch.count))
+				ret = -EFAULT;
+			break;
+		}
+	}
+
+out:
+	kfree(key);
+	kfree(value);
+	return ret;
+}
+
+static int
+__htab_map_delete_batch(struct bpf_map *map,
+			const union bpf_attr *attr,
+			union bpf_attr __user *uattr,
+			bool is_lru_map)
+{
+	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
+	u64 elem_map_flags, map_flags;
+	struct hlist_nulls_head *head;
+	struct hlist_nulls_node *n;
+	u32 batch, max_count;
+	unsigned long flags;
+	struct htab_elem *l;
+	struct bucket *b;
+
+	elem_map_flags = attr->batch.elem_flags;
+	map_flags = attr->batch.flags;
+	if (elem_map_flags || map_flags)
+		return -EINVAL;
+
+	max_count = attr->batch.count;
+	batch = (u32)attr->batch.batch;
+	if (max_count || batch >= htab->n_buckets)
+		return -EINVAL;
+
+	preempt_disable();
+	__this_cpu_inc(bpf_prog_active);
+	rcu_read_lock();
+
+again:
+	b = &htab->buckets[batch];
+	head = &b->head;
+	raw_spin_lock_irqsave(&b->lock, flags);
+
+	hlist_nulls_for_each_entry_rcu(l, n, head, hash_node) {
+		hlist_nulls_del_rcu(&l->hash_node);
+		if (is_lru_map)
+			bpf_lru_push_free(&htab->lru, &l->lru_node);
+		else
+			free_htab_elem(htab, l);
+	}
+
+	batch++;
+	if (batch >= htab->n_buckets)
+		goto out;
+
+	raw_spin_unlock_irqrestore(&b->lock, flags);
+	goto again;
+
+out:
+	raw_spin_unlock_irqrestore(&b->lock, flags);
+	rcu_read_unlock();
+	__this_cpu_dec(bpf_prog_active);
+	preempt_enable();
+
+	return 0;
+}
+
+static int
+htab_map_lookup_batch(struct bpf_map *map, const union bpf_attr *attr,
+		      union bpf_attr __user *uattr)
+{
+	return __htab_map_lookup_and_delete_batch(map, attr, uattr, false,
+						  false);
+}
+
+static int
+htab_map_lookup_and_delete_batch(struct bpf_map *map,
+				 const union bpf_attr *attr,
+				 union bpf_attr __user *uattr)
+{
+	return __htab_map_lookup_and_delete_batch(map, attr, uattr, true,
+						  false);
+}
+
+static int
+htab_map_update_batch(struct bpf_map *map, const union bpf_attr *attr,
+		      union bpf_attr __user *uattr)
+{
+	return __htab_map_update_batch(map, attr, uattr, false);
+}
+
+static int
+htab_map_delete_batch(struct bpf_map *map,
+		      const union bpf_attr *attr,
+		      union bpf_attr __user *uattr)
+{
+	return __htab_map_delete_batch(map, attr, uattr, false);
+}
+
+static int
+htab_lru_map_lookup_batch(struct bpf_map *map, const union bpf_attr *attr,
+			  union bpf_attr __user *uattr)
+{
+	return __htab_map_lookup_and_delete_batch(map, attr, uattr, false,
+						  true);
+}
+
+static int
+htab_lru_map_lookup_and_delete_batch(struct bpf_map *map,
+				     const union bpf_attr *attr,
+				     union bpf_attr __user *uattr)
+{
+	return __htab_map_lookup_and_delete_batch(map, attr, uattr, true,
+						  true);
+}
+
+static int
+htab_lru_map_update_batch(struct bpf_map *map, const union bpf_attr *attr,
+			  union bpf_attr __user *uattr)
+{
+	return __htab_map_update_batch(map, attr, uattr, true);
+}
+
+static int
+htab_lru_map_delete_batch(struct bpf_map *map,
+			  const union bpf_attr *attr,
+			  union bpf_attr __user *uattr)
+{
+	return __htab_map_delete_batch(map, attr, uattr, true);
+}
+
 const struct bpf_map_ops htab_map_ops = {
 	.map_alloc_check = htab_map_alloc_check,
 	.map_alloc = htab_map_alloc,
@@ -1242,6 +1558,10 @@ const struct bpf_map_ops htab_map_ops = {
 	.map_delete_elem = htab_map_delete_elem,
 	.map_gen_lookup = htab_map_gen_lookup,
 	.map_seq_show_elem = htab_map_seq_show_elem,
+	.map_lookup_batch = htab_map_lookup_batch,
+	.map_lookup_and_delete_batch = htab_map_lookup_and_delete_batch,
+	.map_update_batch = htab_map_update_batch,
+	.map_delete_batch = htab_map_delete_batch,
 };
 
 const struct bpf_map_ops htab_lru_map_ops = {
@@ -1255,6 +1575,10 @@ const struct bpf_map_ops htab_lru_map_ops = {
 	.map_delete_elem = htab_lru_map_delete_elem,
 	.map_gen_lookup = htab_lru_map_gen_lookup,
 	.map_seq_show_elem = htab_map_seq_show_elem,
+	.map_lookup_batch = htab_lru_map_lookup_batch,
+	.map_lookup_and_delete_batch = htab_lru_map_lookup_and_delete_batch,
+	.map_update_batch = htab_lru_map_update_batch,
+	.map_delete_batch = htab_lru_map_delete_batch,
 };
 
 /* Called from eBPF program */
diff --git a/kernel/bpf/syscall.c b/kernel/bpf/syscall.c
index ca60eafa6922..e83bdf7efbd8 100644
--- a/kernel/bpf/syscall.c
+++ b/kernel/bpf/syscall.c
@@ -2816,6 +2816,62 @@ static int bpf_task_fd_query(const union bpf_attr *attr,
 	return err;
 }
 
+#define BPF_MAP_BATCH_LAST_FIELD batch.flags
+
+#define BPF_DO_BATCH(fn)			\
+	do {					\
+		if (!fn) {			\
+			err = -ENOTSUPP;	\
+			goto err_put;		\
+		}				\
+		err = fn(map, attr, uattr);	\
+	} while(0)
+
+static int bpf_map_do_batch(const union bpf_attr *attr,
+			    union bpf_attr __user *uattr,
+			    int cmd)
+{
+	struct bpf_map *map;
+	int err, ufd;
+	struct fd f;
+
+	if (CHECK_ATTR(BPF_MAP_BATCH))
+		return -EINVAL;
+
+	ufd = attr->batch.map_fd;
+	f = fdget(ufd);
+	map = __bpf_map_get(f);
+	if (IS_ERR(map))
+		return PTR_ERR(map);
+
+	if ((cmd == BPF_MAP_LOOKUP_BATCH ||
+	     cmd == BPF_MAP_LOOKUP_AND_DELETE_BATCH) &&
+	    !(map_get_sys_perms(map, f) & FMODE_CAN_READ)) {
+		err = -EPERM;
+		goto err_put;
+	}
+
+	if (cmd != BPF_MAP_LOOKUP_BATCH &&
+	    !(map_get_sys_perms(map, f) & FMODE_CAN_WRITE)) {
+		err = -EPERM;
+		goto err_put;
+	}
+
+	if (cmd == BPF_MAP_LOOKUP_BATCH) {
+		BPF_DO_BATCH(map->ops->map_lookup_batch);
+	} else if (cmd == BPF_MAP_LOOKUP_AND_DELETE_BATCH) {
+		BPF_DO_BATCH(map->ops->map_lookup_and_delete_batch);
+	} else if (cmd == BPF_MAP_UPDATE_BATCH) {
+		BPF_DO_BATCH(map->ops->map_update_batch);
+	} else {
+		BPF_DO_BATCH(map->ops->map_delete_batch);
+	}
+
+err_put:
+	fdput(f);
+	return err;
+}
+
 SYSCALL_DEFINE3(bpf, int, cmd, union bpf_attr __user *, uattr, unsigned int, size)
 {
 	union bpf_attr attr = {};
@@ -2913,6 +2969,18 @@ SYSCALL_DEFINE3(bpf, int, cmd, union bpf_attr __user *, uattr, unsigned int, siz
 	case BPF_MAP_LOOKUP_AND_DELETE_ELEM:
 		err = map_lookup_and_delete_elem(&attr);
 		break;
+	case BPF_MAP_LOOKUP_BATCH:
+		err = bpf_map_do_batch(&attr, uattr, BPF_MAP_LOOKUP_BATCH);
+		break;
+	case BPF_MAP_LOOKUP_AND_DELETE_BATCH:
+		err = bpf_map_do_batch(&attr, uattr, BPF_MAP_LOOKUP_AND_DELETE_BATCH);
+		break;
+	case BPF_MAP_UPDATE_BATCH:
+		err = bpf_map_do_batch(&attr, uattr, BPF_MAP_UPDATE_BATCH);
+		break;
+	case BPF_MAP_DELETE_BATCH:
+		err = bpf_map_do_batch(&attr, uattr, BPF_MAP_DELETE_BATCH);
+		break;
 	default:
 		err = -EINVAL;
 		break;
-- 
2.17.1


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

* [RFC PATCH bpf-next v2 2/2] tools/bpf: test bpf_map_lookup_and_delete_batch()
  2019-09-06 22:54 [RFC PATCH bpf-next v2 0/2] bpf: adding map batch processing support Yonghong Song
  2019-09-06 22:54 ` [RFC PATCH bpf-next v2 1/2] " Yonghong Song
@ 2019-09-06 22:54 ` Yonghong Song
  1 sibling, 0 replies; 5+ messages in thread
From: Yonghong Song @ 2019-09-06 22:54 UTC (permalink / raw)
  To: ast, daniel, netdev, bpf
  Cc: kernel-team, Jakub Kicinski, Brian Vazquez, Stanislav Fomichev

Added four libbpf API functions to support map batch operations:
  . int bpf_map_delete_batch( ... )
  . int bpf_map_lookup_batch( ... )
  . int bpf_map_lookup_and_delete_batch( ... )
  . int bpf_map_update_batch( ... )

Tested bpf_map_lookup_and_delete_batch() and bpf_map_update_batch()
functionality.
  $ ./test_maps
  ...
  test_map_lookup_and_delete_batch:PASS
  ...

Note that I clumped uapi header sync patch, libbpf patch
and tests patch together considering this is a RFC patch.
Will do proper formating once it is out of RFC stage.

Signed-off-by: Yonghong Song <yhs@fb.com>
---
 tools/include/uapi/linux/bpf.h                |  22 +++
 tools/lib/bpf/bpf.c                           |  59 +++++++
 tools/lib/bpf/bpf.h                           |  13 ++
 tools/lib/bpf/libbpf.map                      |   4 +
 .../map_tests/map_lookup_and_delete_batch.c   | 155 ++++++++++++++++++
 5 files changed, 253 insertions(+)
 create mode 100644 tools/testing/selftests/bpf/map_tests/map_lookup_and_delete_batch.c

diff --git a/tools/include/uapi/linux/bpf.h b/tools/include/uapi/linux/bpf.h
index 5d2fb183ee2d..9d4f76073dd9 100644
--- a/tools/include/uapi/linux/bpf.h
+++ b/tools/include/uapi/linux/bpf.h
@@ -107,6 +107,10 @@ enum bpf_cmd {
 	BPF_MAP_LOOKUP_AND_DELETE_ELEM,
 	BPF_MAP_FREEZE,
 	BPF_BTF_GET_NEXT_ID,
+	BPF_MAP_LOOKUP_BATCH,
+	BPF_MAP_LOOKUP_AND_DELETE_BATCH,
+	BPF_MAP_UPDATE_BATCH,
+	BPF_MAP_DELETE_BATCH,
 };
 
 enum bpf_map_type {
@@ -396,6 +400,24 @@ union bpf_attr {
 		__u64		flags;
 	};
 
+	struct { /* struct used by BPF_MAP_*_BATCH commands */
+		__u64		batch;	/* input/output:
+					 * input: start batch,
+					 *        0 to start from beginning.
+					 * output: next start batch,
+					 *         0 to end batching.
+					 */
+		__aligned_u64	keys;
+		__aligned_u64	values;
+		__u32		count;	/* input/output:
+					 * input: # of elements keys/values.
+					 * output: # of filled elements.
+					 */
+		__u32		map_fd;
+		__u64		elem_flags;
+		__u64		flags;
+	} batch;
+
 	struct { /* anonymous struct used by BPF_PROG_LOAD command */
 		__u32		prog_type;	/* one of enum bpf_prog_type */
 		__u32		insn_cnt;
diff --git a/tools/lib/bpf/bpf.c b/tools/lib/bpf/bpf.c
index cbb933532981..367bdcb3c62b 100644
--- a/tools/lib/bpf/bpf.c
+++ b/tools/lib/bpf/bpf.c
@@ -438,6 +438,65 @@ int bpf_map_freeze(int fd)
 	return sys_bpf(BPF_MAP_FREEZE, &attr, sizeof(attr));
 }
 
+static int bpf_map_batch_common(int cmd, int fd, __u64 *batch,
+				void *keys, void *values,
+				__u32 *count, __u64 elem_flags,
+				__u64 flags)
+{
+	union bpf_attr attr = {};
+	int ret;
+
+	attr.batch.map_fd = fd;
+	if (batch)
+		attr.batch.batch = *batch;
+	attr.batch.keys = ptr_to_u64(keys);
+	attr.batch.values = ptr_to_u64(values);
+	if (count)
+		attr.batch.count = *count;
+	attr.batch.elem_flags = elem_flags;
+	attr.batch.flags = flags;
+
+	ret = sys_bpf(cmd, &attr, sizeof(attr));
+	if (batch)
+		*batch = attr.batch.batch;
+	if (count)
+		*count = attr.batch.count;
+
+	return ret;
+}
+
+int bpf_map_delete_batch(int fd, __u64 *batch, __u32 *count, __u64 elem_flags,
+			 __u64 flags)
+{
+	return bpf_map_batch_common(BPF_MAP_DELETE_BATCH, fd, batch,
+				    NULL, NULL, count, elem_flags, flags);
+}
+
+int bpf_map_lookup_batch(int fd, __u64 *batch, void *keys, void *values,
+			 __u32 *count, __u64 elem_flags, __u64 flags)
+{
+	return bpf_map_batch_common(BPF_MAP_LOOKUP_BATCH, fd, batch,
+				    keys, values, count, elem_flags, flags);
+}
+
+int bpf_map_lookup_and_delete_batch(int fd, __u64 *batch,
+				    void *keys, void *values,
+				    __u32 *count, __u64 elem_flags,
+				    __u64 flags)
+{
+	return bpf_map_batch_common(BPF_MAP_LOOKUP_AND_DELETE_BATCH,
+				    fd, batch, keys, values,
+				    count, elem_flags, flags);
+}
+
+int bpf_map_update_batch(int fd, void *keys, void *values, __u32 *count,
+			 __u64 elem_flags, __u64 flags)
+{
+	return bpf_map_batch_common(BPF_MAP_UPDATE_BATCH,
+				    fd, NULL, keys, values,
+				    count, elem_flags, flags);
+}
+
 int bpf_obj_pin(int fd, const char *pathname)
 {
 	union bpf_attr attr;
diff --git a/tools/lib/bpf/bpf.h b/tools/lib/bpf/bpf.h
index 0db01334740f..37211840f345 100644
--- a/tools/lib/bpf/bpf.h
+++ b/tools/lib/bpf/bpf.h
@@ -120,6 +120,19 @@ LIBBPF_API int bpf_map_lookup_and_delete_elem(int fd, const void *key,
 LIBBPF_API int bpf_map_delete_elem(int fd, const void *key);
 LIBBPF_API int bpf_map_get_next_key(int fd, const void *key, void *next_key);
 LIBBPF_API int bpf_map_freeze(int fd);
+LIBBPF_API int bpf_map_delete_batch(int fd, __u64 *batch, __u32 *count,
+				    __u64 elem_flags, __u64 flags);
+LIBBPF_API int bpf_map_lookup_batch(int fd, __u64 *batch, void *keys,
+				    void *values, __u32 *count,
+				    __u64 elem_flags, __u64 flags);
+LIBBPF_API int bpf_map_lookup_and_delete_batch(int fd, __u64 *batch,
+					       void *keys, void *values,
+					       __u32 *count, __u64 elem_flags,
+					       __u64 flags);
+LIBBPF_API int bpf_map_update_batch(int fd, void *keys, void *values,
+				    __u32 *count, __u64 elem_flags,
+				    __u64 flags);
+
 LIBBPF_API int bpf_obj_pin(int fd, const char *pathname);
 LIBBPF_API int bpf_obj_get(const char *pathname);
 LIBBPF_API int bpf_prog_attach(int prog_fd, int attachable_fd,
diff --git a/tools/lib/bpf/libbpf.map b/tools/lib/bpf/libbpf.map
index d04c7cb623ed..739bd9f76e50 100644
--- a/tools/lib/bpf/libbpf.map
+++ b/tools/lib/bpf/libbpf.map
@@ -189,4 +189,8 @@ LIBBPF_0.0.4 {
 LIBBPF_0.0.5 {
 	global:
 		bpf_btf_get_next_id;
+		bpf_map_delete_batch;
+		bpf_map_lookup_and_delete_batch;
+		bpf_map_lookup_batch;
+		bpf_map_update_batch;
 } LIBBPF_0.0.4;
diff --git a/tools/testing/selftests/bpf/map_tests/map_lookup_and_delete_batch.c b/tools/testing/selftests/bpf/map_tests/map_lookup_and_delete_batch.c
new file mode 100644
index 000000000000..dd906b1de595
--- /dev/null
+++ b/tools/testing/selftests/bpf/map_tests/map_lookup_and_delete_batch.c
@@ -0,0 +1,155 @@
+// SPDX-License-Identifier: GPL-2.0
+/* Copyright (c) 2019 Facebook  */
+#include <stdio.h>
+#include <errno.h>
+#include <string.h>
+
+#include <bpf/bpf.h>
+#include <bpf/libbpf.h>
+
+#include <test_maps.h>
+
+static void map_batch_update(int map_fd, __u32 max_entries, int *keys,
+			     int *values)
+{
+	int i, err;
+
+	for (i = 0; i < max_entries; i++) {
+		keys[i] = i + 1;
+		values[i] = i + 2;
+	}
+
+	err = bpf_map_update_batch(map_fd, keys, values, &max_entries, 0, 0);
+	CHECK(err, "bpf_map_update_batch()", "error:%s\n", strerror(errno));
+}
+
+static void map_batch_verify(int *visited, __u32 max_entries,
+			     int *keys, int *values)
+{
+	int i;
+
+	memset(visited, 0, max_entries * sizeof(*visited));
+	for (i = 0; i < max_entries; i++) {
+		CHECK(keys[i] + 1 != values[i], "key/value checking",
+		      "error: i %d key %d value %d\n", i, keys[i], values[i]);
+		visited[i] = 1;
+	}
+	for (i = 0; i < max_entries; i++) {
+		CHECK(visited[i] != 1, "visited checking",
+		      "error: keys array at index %d missing\n", i);
+	}
+}
+
+void test_map_lookup_and_delete_batch(void)
+{
+	struct bpf_create_map_attr xattr = {
+		.name = "hash_map",
+		.map_type = BPF_MAP_TYPE_HASH,
+		.key_size = sizeof(int),
+		.value_size = sizeof(int),
+	};
+	int map_fd, *keys, *values, *visited, key;
+	__u32 count, total, total_success;
+	const __u32 max_entries = 10;
+	int err, i, step;
+	bool nospace_err;
+	__u64 batch = 0;
+
+	xattr.max_entries = max_entries;
+	map_fd = bpf_create_map_xattr(&xattr);
+	CHECK(map_fd == -1,
+	      "bpf_create_map_xattr()", "error:%s\n", strerror(errno));
+
+	keys = malloc(max_entries * sizeof(int));
+	values = malloc(max_entries * sizeof(int));
+	visited = malloc(max_entries * sizeof(int));
+	CHECK(!keys || !values || !visited, "malloc()", "error:%s\n", strerror(errno));
+
+	/* test 1: lookup/delete an empty hash table, success */
+	count = max_entries;
+	err = bpf_map_lookup_and_delete_batch(map_fd, &batch, keys, values,
+					      &count, 0, 0);
+	CHECK(err, "empty map", "error: %s\n", strerror(errno));
+	CHECK(batch || count, "empty map", "batch = %lld, count = %u\n", batch, count);
+
+	/* populate elements to the map */
+	map_batch_update(map_fd, max_entries, keys, values);
+
+	/* test 2: lookup/delete with count = 0, success */
+	batch = 0;
+	count = 0;
+	err = bpf_map_lookup_and_delete_batch(map_fd, &batch, keys, values,
+					      &count, 0, 0);
+	CHECK(err, "count = 0", "error: %s\n", strerror(errno));
+
+	/* test 3: lookup/delete with count = max_entries, success */
+	memset(keys, 0, max_entries * sizeof(*keys));
+	memset(values, 0, max_entries * sizeof(*values));
+	count = max_entries;
+	batch = 0;
+	err = bpf_map_lookup_and_delete_batch(map_fd, &batch, keys,
+					      values, &count, 0, 0);
+	CHECK(err, "count = max_entries", "error: %s\n", strerror(errno));
+	CHECK(count != max_entries || batch != 0, "count = max_entries",
+	      "count = %u, max_entries = %u, batch = %lld\n",
+	      count, max_entries, batch);
+	map_batch_verify(visited, max_entries, keys, values);
+
+	/* bpf_map_get_next_key() should return -ENOENT for an empty map. */
+	err = bpf_map_get_next_key(map_fd, NULL, &key);
+	CHECK(!err, "bpf_map_get_next_key()", "error: %s\n", strerror(errno));
+
+	/* test 4: lookup/delete in a loop with various steps. */
+	total_success = 0;
+	for (step = 1; step < max_entries; step++) {
+		map_batch_update(map_fd, max_entries, keys, values);
+		memset(keys, 0, max_entries * sizeof(*keys));
+		memset(values, 0, max_entries * sizeof(*values));
+		batch = 0;
+		total = 0;
+		i = 0;
+		/* iteratively lookup/delete elements with 'step' elements each */
+		count = step;
+		nospace_err = false;
+		while (true) {
+			err = bpf_map_lookup_and_delete_batch(map_fd, &batch,
+							      keys + total,
+							      values + total,
+							      &count, 0, 0);
+			/* It is possible that we are failing due to buffer size
+			 * not big enough. In such cases, let us just exit and
+			 * go with large steps. Not that a buffer size with
+			 * max_entries should always work.
+			 */
+			if (err && errno == ENOSPC) {
+				nospace_err = true;
+				break;
+			}
+
+			CHECK(err, "lookup/delete with steps", "error: %s\n",
+			      strerror(errno));
+
+			total += count;
+			if (batch == 0)
+				break;
+
+			i++;
+		}
+
+		if (nospace_err == true)
+			continue;
+
+		CHECK(total != max_entries, "lookup/delete with steps",
+		      "total = %u, max_entries = %u\n", total, max_entries);
+
+		map_batch_verify(visited, max_entries, keys, values);
+		err = bpf_map_get_next_key(map_fd, NULL, &key);
+		CHECK(!err, "bpf_map_get_next_key()", "error: %s\n", strerror(errno));
+
+		total_success++;
+	}
+
+	CHECK(total_success == 0, "check total_success", "unexpected failure\n");
+
+	printf("%s:PASS\n", __func__);
+}
-- 
2.17.1


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

* Re: [RFC PATCH bpf-next v2 1/2] bpf: adding map batch processing support
  2019-09-06 22:54 ` [RFC PATCH bpf-next v2 1/2] " Yonghong Song
@ 2019-09-23 23:20   ` Brian Vazquez
  2019-09-24 16:28     ` Yonghong Song
  0 siblings, 1 reply; 5+ messages in thread
From: Brian Vazquez @ 2019-09-23 23:20 UTC (permalink / raw)
  To: Yonghong Song
  Cc: ast, Daniel Borkmann, netdev, bpf, kernel-team, Jakub Kicinski,
	Brian Vazquez, Stanislav Fomichev

Hi Yonghong, thanks for working on this!

I have some concerns about this implementation but overall I think
this might work for our use case too!

On Sun, Sep 8, 2019 at 1:11 AM Yonghong Song <yhs@fb.com> wrote:
>
> Brian Vazquez has proposed BPF_MAP_DUMP command to look up more than one
> map entries per syscall.
>   https://lore.kernel.org/bpf/CABCgpaU3xxX6CMMxD+1knApivtc2jLBHysDXw-0E9bQEL0qC3A@mail.gmail.com/T/#t
>
> During discussion, we found more use cases can be supported in a similar
> map operation batching framework. For example, batched map lookup and delete,
> which can be really helpful for bcc.
>   https://github.com/iovisor/bcc/blob/master/tools/tcptop.py#L233-L243
>   https://github.com/iovisor/bcc/blob/master/tools/slabratetop.py#L129-L138
>
> Also, in bcc, we have API to delete all entries in a map.
>   https://github.com/iovisor/bcc/blob/master/src/cc/api/BPFTable.h#L257-L264
>
> For map update, batched operations also useful as sometimes applications need
> to populate initial maps with more than one entry. For example, the below
> example is from kernel/samples/bpf/xdp_redirect_cpu_user.c:
>   https://github.com/torvalds/linux/blob/master/samples/bpf/xdp_redirect_cpu_user.c#L543-L550
>
> This patch addresses all the above use cases. To make uapi stable, it also
> covers other potential use cases. For bpf syscall subcommands are introduced:
>         BPF_MAP_LOOKUP_BATCH
>         BPF_MAP_LOOKUP_AND_DELETE_BATCH
>         BPF_MAP_UPDATE_BATCH
>         BPF_MAP_DELETE_BATCH
>
> The UAPI attribute structure looks like:
>
>     struct { /* struct used by BPF_MAP_*_BATCH commands */
>             __u64           batch;  /* input/output:
>                                      * input: start batch,
>                                      *        0 to start from beginning.
>                                      * output: next start batch,
>                                      *         0 to end batching.
>                                      */
>             __aligned_u64   keys;
>             __aligned_u64   values;
>             __u32           count;  /* input/output:
>                                      * input: # of elements keys/values.
>                                      * output: # of filled elements.
>                                      */
>             __u32           map_fd;
>             __u64           elem_flags;
>             __u64           flags;
>     } batch;
>
> An opaque value 'batch' is used for user/kernel space communication
> for where in the map to start the operation for lookup/lookup_and_delete/delete.
>   input 'batch' = 0: to start the operation from the beginning of the map.
>   output 'batch': if not 0, the next input for batch operation.
>
> For lookup/lookup_and_delete:
>   operation: lookup/lookup_and_delete starting from a particular 'batch'.
>   return:
>      'batch'       'count'     return code     meaning
>       0            0           0               Done. Nothing left
>       0            0           -ENOSPC         no space to handle batch 0
>       > 0          0           -ENOSPC         no space to handle 'batch'
>       > 0          > 0         0               stopped right before 'batch'
> Note that:
>   (1). Even if return code is 0 and return 'count' > 0, the return 'count' may
>        not be equal to input 'count'. This happens when there is no enough space
>        to handle a batch.
>   (2). If the return code is an error and not -EFAULT,
>        'batch' indicates the batch has issues and 'count' indicates the number
>        of elements successfully processed.
>
> For delete:
>   operation: deletion starting from a particular 'batch'.
>   return: 0 means everything is deleted from 'batch'.
>           error code means something deletion not happening.
>
> For update:
>   operation: update 'count' number of elements in 'keys'/'values'.
>   return: 0 means successful updates for all elements.
>           error code, if not -EFAULT, 'count' is the number of successful updates.
>
> Signed-off-by: Yonghong Song <yhs@fb.com>
> ---
>  include/linux/bpf.h      |   9 ++
>  include/uapi/linux/bpf.h |  22 +++
>  kernel/bpf/hashtab.c     | 324 +++++++++++++++++++++++++++++++++++++++
>  kernel/bpf/syscall.c     |  68 ++++++++
>  4 files changed, 423 insertions(+)
>
> diff --git a/include/linux/bpf.h b/include/linux/bpf.h
> index 5b9d22338606..3c1302e8e2d4 100644
> --- a/include/linux/bpf.h
> +++ b/include/linux/bpf.h
> @@ -37,6 +37,15 @@ struct bpf_map_ops {
>         int (*map_get_next_key)(struct bpf_map *map, void *key, void *next_key);
>         void (*map_release_uref)(struct bpf_map *map);
>         void *(*map_lookup_elem_sys_only)(struct bpf_map *map, void *key);
> +       int (*map_lookup_batch)(struct bpf_map *map, const union bpf_attr *attr,
> +                               union bpf_attr __user *uattr);
> +       int (*map_lookup_and_delete_batch)(struct bpf_map *map,
> +                                          const union bpf_attr *attr,
> +                                          union bpf_attr __user *uattr);
> +       int (*map_update_batch)(struct bpf_map *map, const union bpf_attr *attr,
> +                               union bpf_attr __user *uattr);
> +       int (*map_delete_batch)(struct bpf_map *map, const union bpf_attr *attr,
> +                               union bpf_attr __user *uattr);
>
>         /* funcs callable from userspace and from eBPF programs */
>         void *(*map_lookup_elem)(struct bpf_map *map, void *key);
> diff --git a/include/uapi/linux/bpf.h b/include/uapi/linux/bpf.h
> index 5d2fb183ee2d..9d4f76073dd9 100644
> --- a/include/uapi/linux/bpf.h
> +++ b/include/uapi/linux/bpf.h
> @@ -107,6 +107,10 @@ enum bpf_cmd {
>         BPF_MAP_LOOKUP_AND_DELETE_ELEM,
>         BPF_MAP_FREEZE,
>         BPF_BTF_GET_NEXT_ID,
> +       BPF_MAP_LOOKUP_BATCH,
> +       BPF_MAP_LOOKUP_AND_DELETE_BATCH,
> +       BPF_MAP_UPDATE_BATCH,
> +       BPF_MAP_DELETE_BATCH,
>  };
>
>  enum bpf_map_type {
> @@ -396,6 +400,24 @@ union bpf_attr {
>                 __u64           flags;
>         };
>
> +       struct { /* struct used by BPF_MAP_*_BATCH commands */
> +               __u64           batch;  /* input/output:
> +                                        * input: start batch,
> +                                        *        0 to start from beginning.
> +                                        * output: next start batch,
> +                                        *         0 to end batching.
> +                                        */
> +               __aligned_u64   keys;
> +               __aligned_u64   values;
> +               __u32           count;  /* input/output:
> +                                        * input: # of elements keys/values.
> +                                        * output: # of filled elements.
> +                                        */
> +               __u32           map_fd;
> +               __u64           elem_flags;
> +               __u64           flags;
> +       } batch;
> +
>         struct { /* anonymous struct used by BPF_PROG_LOAD command */
>                 __u32           prog_type;      /* one of enum bpf_prog_type */
>                 __u32           insn_cnt;
> diff --git a/kernel/bpf/hashtab.c b/kernel/bpf/hashtab.c
> index 22066a62c8c9..ee7b90200f4d 100644
> --- a/kernel/bpf/hashtab.c
> +++ b/kernel/bpf/hashtab.c
> @@ -1232,6 +1232,322 @@ static void htab_map_seq_show_elem(struct bpf_map *map, void *key,
>         rcu_read_unlock();
>  }
>
> +static int
> +__htab_map_lookup_and_delete_batch(struct bpf_map *map,
> +                                  const union bpf_attr *attr,
> +                                  union bpf_attr __user *uattr,
> +                                  bool do_delete, bool is_lru_map)
> +{
> +       struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
> +       u32 bucket_cnt, total, key_size, value_size, roundup_key_size;
> +       void *keys = NULL, *values = NULL, *value, *dst_key, *dst_val;
> +       u64 elem_map_flags, map_flags;
> +       struct hlist_nulls_head *head;
> +       void __user *ukeys, *uvalues;
> +       struct hlist_nulls_node *n;
> +       u32 batch, max_count;
> +       unsigned long flags;
> +       struct htab_elem *l;
> +       struct bucket *b;
> +       int ret = 0;
> +
> +       max_count = attr->batch.count;
> +       if (!max_count)
> +               return 0;
> +
> +       elem_map_flags = attr->batch.elem_flags;
> +       if ((elem_map_flags & ~BPF_F_LOCK) ||
> +           ((elem_map_flags & BPF_F_LOCK) && !map_value_has_spin_lock(map)))
> +               return -EINVAL;
> +
> +       map_flags = attr->batch.flags;
> +       if (map_flags)
> +               return -EINVAL;
> +
> +       batch = (u32)attr->batch.batch;
> +       if (batch >= htab->n_buckets)
> +               return -EINVAL;
> +
> +       /* We cannot do copy_from_user or copy_to_user inside
> +        * the rcu_read_lock. Allocate enough space here.
> +        */
> +       key_size = htab->map.key_size;
> +       roundup_key_size = round_up(htab->map.key_size, 8);
> +       value_size = htab->map.value_size;
> +       keys = kvmalloc(key_size * max_count, GFP_USER | __GFP_NOWARN);
> +       values = kvmalloc(value_size * max_count, GFP_USER | __GFP_NOWARN);
> +       if (!keys || !values) {
> +               ret = -ENOMEM;
> +               goto out;
> +       }
> +
> +       dst_key = keys;
> +       dst_val = values;
> +       total = 0;
> +
> +       preempt_disable();
> +       this_cpu_inc(bpf_prog_active);
> +       rcu_read_lock();
> +
> +again:
> +       b = &htab->buckets[batch];
> +       head = &b->head;
> +       raw_spin_lock_irqsave(&b->lock, flags);
> +

Would it be possible to avoid that lock when we're not deleting (just
batching lookup)? To be honest I don't how much impact would have to
grab that lock when concurrent additions are happening in a bpf
program.

> +       bucket_cnt = 0;
> +       hlist_nulls_for_each_entry_rcu(l, n, head, hash_node)
> +               bucket_cnt++;
> +
> +       if (bucket_cnt > (max_count - total)) {
> +               if (total == 0)
> +                       ret = -ENOSPC;
> +               goto after_loop;
> +       }
> +
> +       hlist_nulls_for_each_entry_rcu(l, n, head, hash_node) {
> +               memcpy(dst_key, l->key, key_size);
> +
> +               value = l->key + roundup_key_size;
> +               if (elem_map_flags & BPF_F_LOCK)
> +                       copy_map_value_locked(map, dst_val, value, true);
> +               else
> +                       copy_map_value(map, dst_val, value);
> +               check_and_init_map_lock(map, dst_val);
> +
> +               dst_key += key_size;
> +               dst_val += value_size;
> +               total++;
> +       }
> +
> +       if (do_delete) {
> +               hlist_nulls_for_each_entry_rcu(l, n, head, hash_node) {
> +                       hlist_nulls_del_rcu(&l->hash_node);
> +                       if (is_lru_map)
> +                               bpf_lru_push_free(&htab->lru, &l->lru_node);
> +                       else
> +                               free_htab_elem(htab, l);
> +               }
> +       }
> +
> +       batch++;
> +       if (batch >= htab->n_buckets) {
> +               batch = 0;
> +               goto after_loop;
> +       }
> +
> +       raw_spin_unlock_irqrestore(&b->lock, flags);
> +       goto again;
> +
> +after_loop:
> +       raw_spin_unlock_irqrestore(&b->lock, flags);
> +
> +       rcu_read_unlock();
> +       this_cpu_dec(bpf_prog_active);
> +       preempt_enable();
> +
> +       /* copy data back to user */
> +       ukeys = u64_to_user_ptr(attr->batch.keys);
> +       uvalues = u64_to_user_ptr(attr->batch.values);
> +       if (put_user(batch, &uattr->batch.batch) ||
> +           copy_to_user(ukeys, keys, total * key_size) ||
> +           copy_to_user(uvalues, values, total * value_size) ||
> +           put_user(total, &uattr->batch.count))
> +               ret = -EFAULT;
> +
> +out:
> +       kvfree(keys);
> +       kvfree(values);
> +       return ret;
> +}
> +
> +static int
> +__htab_map_update_batch(struct bpf_map *map, const union bpf_attr *attr,
> +                       union bpf_attr __user *uattr, bool is_lru_map)
> +{
> +       struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
> +       u32 count, max_count, key_size, roundup_key_size, value_size;
> +       u64 elem_map_flags, map_flags;
> +       void __user *ukey, *uvalue;
> +       void *key, *value;
> +       int ret = 0;
> +
> +       max_count = attr->batch.count;
> +       if (!max_count)
> +               return 0;
> +
> +       elem_map_flags = attr->batch.elem_flags;
> +       if ((elem_map_flags & BPF_F_LOCK) && !map_value_has_spin_lock(map))
> +               return -EINVAL;
> +
> +       map_flags = attr->batch.flags;
> +       if (map_flags)
> +               return -EINVAL;
> +
> +       key_size = htab->map.key_size;
> +       roundup_key_size = round_up(htab->map.key_size, 8);
> +       value_size = htab->map.value_size;
> +       key = kmalloc(key_size, GFP_USER | __GFP_NOWARN);
> +       value = kmalloc(value_size, GFP_USER | __GFP_NOWARN);
> +       if (!key || !value) {
> +               ret = -ENOMEM;
> +               goto out;
> +       }
> +
> +       ukey = u64_to_user_ptr(attr->batch.keys);
> +       uvalue = u64_to_user_ptr(attr->batch.values);
> +       for (count = 0; count < max_count; count++) {
> +               if (copy_from_user(key, ukey + count * key_size, key_size) ||
> +                   copy_from_user(value, uvalue + count * value_size, value_size)) {
> +                       ret = -EFAULT;
> +                       break;
> +               }
> +
> +               preempt_disable();
> +               __this_cpu_inc(bpf_prog_active);
> +               rcu_read_lock();
> +               if (is_lru_map)
> +                       ret = htab_lru_map_update_elem(map, key, value, elem_map_flags);
> +               else
> +                       ret = htab_map_update_elem(map, key, value, elem_map_flags);
> +               rcu_read_unlock();
> +               __this_cpu_dec(bpf_prog_active);
> +               preempt_enable();
> +
> +               if (ret) {
> +                       if (put_user(count, &uattr->batch.count))
> +                               ret = -EFAULT;
> +                       break;
> +               }
> +       }
> +
> +out:
> +       kfree(key);
> +       kfree(value);
> +       return ret;
> +}
> +
> +static int
> +__htab_map_delete_batch(struct bpf_map *map,
> +                       const union bpf_attr *attr,
> +                       union bpf_attr __user *uattr,
> +                       bool is_lru_map)
> +{
> +       struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
> +       u64 elem_map_flags, map_flags;
> +       struct hlist_nulls_head *head;
> +       struct hlist_nulls_node *n;
> +       u32 batch, max_count;
> +       unsigned long flags;
> +       struct htab_elem *l;
> +       struct bucket *b;
> +
> +       elem_map_flags = attr->batch.elem_flags;
> +       map_flags = attr->batch.flags;
> +       if (elem_map_flags || map_flags)
> +               return -EINVAL;
> +
> +       max_count = attr->batch.count;
> +       batch = (u32)attr->batch.batch;
> +       if (max_count || batch >= htab->n_buckets)
> +               return -EINVAL;
> +
> +       preempt_disable();
> +       __this_cpu_inc(bpf_prog_active);
> +       rcu_read_lock();
> +
> +again:
> +       b = &htab->buckets[batch];
> +       head = &b->head;
> +       raw_spin_lock_irqsave(&b->lock, flags);
> +
> +       hlist_nulls_for_each_entry_rcu(l, n, head, hash_node) {
> +               hlist_nulls_del_rcu(&l->hash_node);
> +               if (is_lru_map)
> +                       bpf_lru_push_free(&htab->lru, &l->lru_node);
> +               else
> +                       free_htab_elem(htab, l);
> +       }
> +
> +       batch++;
> +       if (batch >= htab->n_buckets)
> +               goto out;
> +
> +       raw_spin_unlock_irqrestore(&b->lock, flags);
> +       goto again;
> +
> +out:
> +       raw_spin_unlock_irqrestore(&b->lock, flags);
> +       rcu_read_unlock();
> +       __this_cpu_dec(bpf_prog_active);
> +       preempt_enable();
> +
> +       return 0;
> +}
> +
> +static int
> +htab_map_lookup_batch(struct bpf_map *map, const union bpf_attr *attr,
> +                     union bpf_attr __user *uattr)
> +{
> +       return __htab_map_lookup_and_delete_batch(map, attr, uattr, false,
> +                                                 false);
> +}
> +
> +static int
> +htab_map_lookup_and_delete_batch(struct bpf_map *map,
> +                                const union bpf_attr *attr,
> +                                union bpf_attr __user *uattr)
> +{
> +       return __htab_map_lookup_and_delete_batch(map, attr, uattr, true,
> +                                                 false);
> +}
> +
> +static int
> +htab_map_update_batch(struct bpf_map *map, const union bpf_attr *attr,
> +                     union bpf_attr __user *uattr)
> +{
> +       return __htab_map_update_batch(map, attr, uattr, false);
> +}
> +
> +static int
> +htab_map_delete_batch(struct bpf_map *map,
> +                     const union bpf_attr *attr,
> +                     union bpf_attr __user *uattr)
> +{
> +       return __htab_map_delete_batch(map, attr, uattr, false);
> +}
> +
> +static int
> +htab_lru_map_lookup_batch(struct bpf_map *map, const union bpf_attr *attr,
> +                         union bpf_attr __user *uattr)
> +{
> +       return __htab_map_lookup_and_delete_batch(map, attr, uattr, false,
> +                                                 true);
> +}
> +
> +static int
> +htab_lru_map_lookup_and_delete_batch(struct bpf_map *map,
> +                                    const union bpf_attr *attr,
> +                                    union bpf_attr __user *uattr)
> +{
> +       return __htab_map_lookup_and_delete_batch(map, attr, uattr, true,
> +                                                 true);
> +}
> +
> +static int
> +htab_lru_map_update_batch(struct bpf_map *map, const union bpf_attr *attr,
> +                         union bpf_attr __user *uattr)
> +{
> +       return __htab_map_update_batch(map, attr, uattr, true);
> +}
> +
> +static int
> +htab_lru_map_delete_batch(struct bpf_map *map,
> +                         const union bpf_attr *attr,
> +                         union bpf_attr __user *uattr)
> +{
> +       return __htab_map_delete_batch(map, attr, uattr, true);
> +}
> +
>  const struct bpf_map_ops htab_map_ops = {
>         .map_alloc_check = htab_map_alloc_check,
>         .map_alloc = htab_map_alloc,
> @@ -1242,6 +1558,10 @@ const struct bpf_map_ops htab_map_ops = {
>         .map_delete_elem = htab_map_delete_elem,
>         .map_gen_lookup = htab_map_gen_lookup,
>         .map_seq_show_elem = htab_map_seq_show_elem,
> +       .map_lookup_batch = htab_map_lookup_batch,
> +       .map_lookup_and_delete_batch = htab_map_lookup_and_delete_batch,
> +       .map_update_batch = htab_map_update_batch,
> +       .map_delete_batch = htab_map_delete_batch,
>  };
>
>  const struct bpf_map_ops htab_lru_map_ops = {
> @@ -1255,6 +1575,10 @@ const struct bpf_map_ops htab_lru_map_ops = {
>         .map_delete_elem = htab_lru_map_delete_elem,
>         .map_gen_lookup = htab_lru_map_gen_lookup,
>         .map_seq_show_elem = htab_map_seq_show_elem,
> +       .map_lookup_batch = htab_lru_map_lookup_batch,
> +       .map_lookup_and_delete_batch = htab_lru_map_lookup_and_delete_batch,
> +       .map_update_batch = htab_lru_map_update_batch,
> +       .map_delete_batch = htab_lru_map_delete_batch,
>  };
>
>  /* Called from eBPF program */
> diff --git a/kernel/bpf/syscall.c b/kernel/bpf/syscall.c
> index ca60eafa6922..e83bdf7efbd8 100644
> --- a/kernel/bpf/syscall.c
> +++ b/kernel/bpf/syscall.c
> @@ -2816,6 +2816,62 @@ static int bpf_task_fd_query(const union bpf_attr *attr,
>         return err;
>  }
>
> +#define BPF_MAP_BATCH_LAST_FIELD batch.flags
> +
> +#define BPF_DO_BATCH(fn)                       \
> +       do {                                    \
> +               if (!fn) {                      \
> +                       err = -ENOTSUPP;        \
> +                       goto err_put;           \
> +               }                               \
> +               err = fn(map, attr, uattr);     \
> +       } while(0)
> +
> +static int bpf_map_do_batch(const union bpf_attr *attr,
> +                           union bpf_attr __user *uattr,
> +                           int cmd)
> +{
> +       struct bpf_map *map;
> +       int err, ufd;
> +       struct fd f;
> +
> +       if (CHECK_ATTR(BPF_MAP_BATCH))
> +               return -EINVAL;
> +
> +       ufd = attr->batch.map_fd;
> +       f = fdget(ufd);
> +       map = __bpf_map_get(f);
> +       if (IS_ERR(map))
> +               return PTR_ERR(map);
> +
> +       if ((cmd == BPF_MAP_LOOKUP_BATCH ||
> +            cmd == BPF_MAP_LOOKUP_AND_DELETE_BATCH) &&
> +           !(map_get_sys_perms(map, f) & FMODE_CAN_READ)) {
> +               err = -EPERM;
> +               goto err_put;
> +       }
> +
> +       if (cmd != BPF_MAP_LOOKUP_BATCH &&
> +           !(map_get_sys_perms(map, f) & FMODE_CAN_WRITE)) {
> +               err = -EPERM;
> +               goto err_put;
> +       }
> +
> +       if (cmd == BPF_MAP_LOOKUP_BATCH) {
> +               BPF_DO_BATCH(map->ops->map_lookup_batch);
> +       } else if (cmd == BPF_MAP_LOOKUP_AND_DELETE_BATCH) {
> +               BPF_DO_BATCH(map->ops->map_lookup_and_delete_batch);
> +       } else if (cmd == BPF_MAP_UPDATE_BATCH) {
> +               BPF_DO_BATCH(map->ops->map_update_batch);
> +       } else {
> +               BPF_DO_BATCH(map->ops->map_delete_batch);
> +       }
> +
> +err_put:
> +       fdput(f);
> +       return err;
> +}
> +
>  SYSCALL_DEFINE3(bpf, int, cmd, union bpf_attr __user *, uattr, unsigned int, size)
>  {
>         union bpf_attr attr = {};
> @@ -2913,6 +2969,18 @@ SYSCALL_DEFINE3(bpf, int, cmd, union bpf_attr __user *, uattr, unsigned int, siz
>         case BPF_MAP_LOOKUP_AND_DELETE_ELEM:
>                 err = map_lookup_and_delete_elem(&attr);
>                 break;
> +       case BPF_MAP_LOOKUP_BATCH:
> +               err = bpf_map_do_batch(&attr, uattr, BPF_MAP_LOOKUP_BATCH);
> +               break;
> +       case BPF_MAP_LOOKUP_AND_DELETE_BATCH:
> +               err = bpf_map_do_batch(&attr, uattr, BPF_MAP_LOOKUP_AND_DELETE_BATCH);
> +               break;
> +       case BPF_MAP_UPDATE_BATCH:
> +               err = bpf_map_do_batch(&attr, uattr, BPF_MAP_UPDATE_BATCH);
> +               break;
> +       case BPF_MAP_DELETE_BATCH:
> +               err = bpf_map_do_batch(&attr, uattr, BPF_MAP_DELETE_BATCH);
> +               break;
>         default:
>                 err = -EINVAL;
>                 break;
> --
> 2.17.1
>

In general it'd be great if we could express the old functions
(get_next_key, lookup, delete) in terms of this new ones so we  avoid
having twice the code.

Also to be honest I don't see how batching updates would be useful,
maybe just try to do batching lookup and lookup_and_delete?

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

* Re: [RFC PATCH bpf-next v2 1/2] bpf: adding map batch processing support
  2019-09-23 23:20   ` Brian Vazquez
@ 2019-09-24 16:28     ` Yonghong Song
  0 siblings, 0 replies; 5+ messages in thread
From: Yonghong Song @ 2019-09-24 16:28 UTC (permalink / raw)
  To: Brian Vazquez
  Cc: Alexei Starovoitov, Daniel Borkmann, netdev, bpf, Kernel Team,
	Jakub Kicinski, Brian Vazquez, Stanislav Fomichev



On 9/23/19 4:20 PM, Brian Vazquez wrote:
> Hi Yonghong, thanks for working on this!
> 
> I have some concerns about this implementation but overall I think
> this might work for our use case too!

Thanks for reviewing the patch. I will get back to this very soon.

> 
> On Sun, Sep 8, 2019 at 1:11 AM Yonghong Song <yhs@fb.com> wrote:
>>
>> Brian Vazquez has proposed BPF_MAP_DUMP command to look up more than one
>> map entries per syscall.
>>    https://lore.kernel.org/bpf/CABCgpaU3xxX6CMMxD+1knApivtc2jLBHysDXw-0E9bQEL0qC3A@mail.gmail.com/T/#t
>>
>> During discussion, we found more use cases can be supported in a similar
>> map operation batching framework. For example, batched map lookup and delete,
>> which can be really helpful for bcc.
>>    https://github.com/iovisor/bcc/blob/master/tools/tcptop.py#L233-L243
>>    https://github.com/iovisor/bcc/blob/master/tools/slabratetop.py#L129-L138
>>
>> Also, in bcc, we have API to delete all entries in a map.
>>    https://github.com/iovisor/bcc/blob/master/src/cc/api/BPFTable.h#L257-L264
>>
>> For map update, batched operations also useful as sometimes applications need
>> to populate initial maps with more than one entry. For example, the below
>> example is from kernel/samples/bpf/xdp_redirect_cpu_user.c:
>>    https://github.com/torvalds/linux/blob/master/samples/bpf/xdp_redirect_cpu_user.c#L543-L550
>>
>> This patch addresses all the above use cases. To make uapi stable, it also
>> covers other potential use cases. For bpf syscall subcommands are introduced:
>>          BPF_MAP_LOOKUP_BATCH
>>          BPF_MAP_LOOKUP_AND_DELETE_BATCH
>>          BPF_MAP_UPDATE_BATCH
>>          BPF_MAP_DELETE_BATCH
>>
>> The UAPI attribute structure looks like:
>>
>>      struct { /* struct used by BPF_MAP_*_BATCH commands */
>>              __u64           batch;  /* input/output:
>>                                       * input: start batch,
>>                                       *        0 to start from beginning.
>>                                       * output: next start batch,
>>                                       *         0 to end batching.
>>                                       */
>>              __aligned_u64   keys;
>>              __aligned_u64   values;
>>              __u32           count;  /* input/output:
>>                                       * input: # of elements keys/values.
>>                                       * output: # of filled elements.
>>                                       */
>>              __u32           map_fd;
>>              __u64           elem_flags;
>>              __u64           flags;
>>      } batch;
>>
>> An opaque value 'batch' is used for user/kernel space communication
>> for where in the map to start the operation for lookup/lookup_and_delete/delete.
>>    input 'batch' = 0: to start the operation from the beginning of the map.
>>    output 'batch': if not 0, the next input for batch operation.
>>
>> For lookup/lookup_and_delete:
>>    operation: lookup/lookup_and_delete starting from a particular 'batch'.
>>    return:
>>       'batch'       'count'     return code     meaning
>>        0            0           0               Done. Nothing left
>>        0            0           -ENOSPC         no space to handle batch 0
>>        > 0          0           -ENOSPC         no space to handle 'batch'
>>        > 0          > 0         0               stopped right before 'batch'
>> Note that:
>>    (1). Even if return code is 0 and return 'count' > 0, the return 'count' may
>>         not be equal to input 'count'. This happens when there is no enough space
>>         to handle a batch.
>>    (2). If the return code is an error and not -EFAULT,
>>         'batch' indicates the batch has issues and 'count' indicates the number
>>         of elements successfully processed.
>>
>> For delete:
>>    operation: deletion starting from a particular 'batch'.
>>    return: 0 means everything is deleted from 'batch'.
>>            error code means something deletion not happening.
>>
>> For update:
>>    operation: update 'count' number of elements in 'keys'/'values'.
>>    return: 0 means successful updates for all elements.
>>            error code, if not -EFAULT, 'count' is the number of successful updates.
>>
>> Signed-off-by: Yonghong Song <yhs@fb.com>
>> ---
>>   include/linux/bpf.h      |   9 ++
>>   include/uapi/linux/bpf.h |  22 +++
>>   kernel/bpf/hashtab.c     | 324 +++++++++++++++++++++++++++++++++++++++
>>   kernel/bpf/syscall.c     |  68 ++++++++
>>   4 files changed, 423 insertions(+)
>>
[...]
>> +static int
>> +__htab_map_lookup_and_delete_batch(struct bpf_map *map,
>> +                                  const union bpf_attr *attr,
>> +                                  union bpf_attr __user *uattr,
>> +                                  bool do_delete, bool is_lru_map)
>> +{
>> +       struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
>> +       u32 bucket_cnt, total, key_size, value_size, roundup_key_size;
>> +       void *keys = NULL, *values = NULL, *value, *dst_key, *dst_val;
>> +       u64 elem_map_flags, map_flags;
>> +       struct hlist_nulls_head *head;
>> +       void __user *ukeys, *uvalues;
>> +       struct hlist_nulls_node *n;
>> +       u32 batch, max_count;
>> +       unsigned long flags;
>> +       struct htab_elem *l;
>> +       struct bucket *b;
>> +       int ret = 0;
>> +
>> +       max_count = attr->batch.count;
>> +       if (!max_count)
>> +               return 0;
>> +
>> +       elem_map_flags = attr->batch.elem_flags;
>> +       if ((elem_map_flags & ~BPF_F_LOCK) ||
>> +           ((elem_map_flags & BPF_F_LOCK) && !map_value_has_spin_lock(map)))
>> +               return -EINVAL;
>> +
>> +       map_flags = attr->batch.flags;
>> +       if (map_flags)
>> +               return -EINVAL;
>> +
>> +       batch = (u32)attr->batch.batch;
>> +       if (batch >= htab->n_buckets)
>> +               return -EINVAL;
>> +
>> +       /* We cannot do copy_from_user or copy_to_user inside
>> +        * the rcu_read_lock. Allocate enough space here.
>> +        */
>> +       key_size = htab->map.key_size;
>> +       roundup_key_size = round_up(htab->map.key_size, 8);
>> +       value_size = htab->map.value_size;
>> +       keys = kvmalloc(key_size * max_count, GFP_USER | __GFP_NOWARN);
>> +       values = kvmalloc(value_size * max_count, GFP_USER | __GFP_NOWARN);
>> +       if (!keys || !values) {
>> +               ret = -ENOMEM;
>> +               goto out;
>> +       }
>> +
>> +       dst_key = keys;
>> +       dst_val = values;
>> +       total = 0;
>> +
>> +       preempt_disable();
>> +       this_cpu_inc(bpf_prog_active);
>> +       rcu_read_lock();
>> +
>> +again:
>> +       b = &htab->buckets[batch];
>> +       head = &b->head;
>> +       raw_spin_lock_irqsave(&b->lock, flags);
>> +
> 
> Would it be possible to avoid that lock when we're not deleting (just
> batching lookup)? To be honest I don't how much impact would have to
> grab that lock when concurrent additions are happening in a bpf
> program.

Yes, it is possible.
Another reason I took the lock to ensure that when I tested
that bucket_cnt is smaller or equal to the remaining buffer size,
it should remain that way during subsequent elem copying.
Without lock, if in parallel some updates added some
elements and there will be no enough space to copy.

But on the other hand, I think it is okay to just stop
when the buffer is full and miss a few elements.

> 
>> +       bucket_cnt = 0;
>> +       hlist_nulls_for_each_entry_rcu(l, n, head, hash_node)
>> +               bucket_cnt++;
>> +
>> +       if (bucket_cnt > (max_count - total)) {
>> +               if (total == 0)
>> +                       ret = -ENOSPC;
>> +               goto after_loop;
>> +       }
>> +
>> +       hlist_nulls_for_each_entry_rcu(l, n, head, hash_node) {
>> +               memcpy(dst_key, l->key, key_size);
>> +
>> +               value = l->key + roundup_key_size;
>> +               if (elem_map_flags & BPF_F_LOCK)
>> +                       copy_map_value_locked(map, dst_val, value, true);
>> +               else
>> +                       copy_map_value(map, dst_val, value);
>> +               check_and_init_map_lock(map, dst_val);
>> +
>> +               dst_key += key_size;
>> +               dst_val += value_size;
>> +               total++;
>> +       }
>> +
>> +       if (do_delete) {
>> +               hlist_nulls_for_each_entry_rcu(l, n, head, hash_node) {
>> +                       hlist_nulls_del_rcu(&l->hash_node);
>> +                       if (is_lru_map)
>> +                               bpf_lru_push_free(&htab->lru, &l->lru_node);
>> +                       else
>> +                               free_htab_elem(htab, l);
>> +               }
>> +       }
>> +
>> +       batch++;
>> +       if (batch >= htab->n_buckets) {
>> +               batch = 0;
>> +               goto after_loop;
>> +       }
>> +
>> +       raw_spin_unlock_irqrestore(&b->lock, flags);
>> +       goto again;
>> +
>> +after_loop:
>> +       raw_spin_unlock_irqrestore(&b->lock, flags);
>> +
>> +       rcu_read_unlock();
>> +       this_cpu_dec(bpf_prog_active);
>> +       preempt_enable();
>> +
>> +       /* copy data back to user */
>> +       ukeys = u64_to_user_ptr(attr->batch.keys);
>> +       uvalues = u64_to_user_ptr(attr->batch.values);
>> +       if (put_user(batch, &uattr->batch.batch) ||
>> +           copy_to_user(ukeys, keys, total * key_size) ||
>> +           copy_to_user(uvalues, values, total * value_size) ||
>> +           put_user(total, &uattr->batch.count))
>> +               ret = -EFAULT;
>> +
>> +out:
>> +       kvfree(keys);
>> +       kvfree(values);
>> +       return ret;
>> +}
>> +
>> +static int
>> +__htab_map_update_batch(struct bpf_map *map, const union bpf_attr *attr,
>> +                       union bpf_attr __user *uattr, bool is_lru_map)
>> +{
>> +       struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
>> +       u32 count, max_count, key_size, roundup_key_size, value_size;
>> +       u64 elem_map_flags, map_flags;
>> +       void __user *ukey, *uvalue;
>> +       void *key, *value;
>> +       int ret = 0;
>> +
>> +       max_count = attr->batch.count;
>> +       if (!max_count)
>> +               return 0;
>> +
>> +       elem_map_flags = attr->batch.elem_flags;
>> +       if ((elem_map_flags & BPF_F_LOCK) && !map_value_has_spin_lock(map))
>> +               return -EINVAL;
>> +
>> +       map_flags = attr->batch.flags;
>> +       if (map_flags)
>> +               return -EINVAL;
>> +
>> +       key_size = htab->map.key_size;
>> +       roundup_key_size = round_up(htab->map.key_size, 8);
>> +       value_size = htab->map.value_size;
>> +       key = kmalloc(key_size, GFP_USER | __GFP_NOWARN);
>> +       value = kmalloc(value_size, GFP_USER | __GFP_NOWARN);
>> +       if (!key || !value) {
>> +               ret = -ENOMEM;
>> +               goto out;
>> +       }
>> +
>> +       ukey = u64_to_user_ptr(attr->batch.keys);
>> +       uvalue = u64_to_user_ptr(attr->batch.values);
>> +       for (count = 0; count < max_count; count++) {
>> +               if (copy_from_user(key, ukey + count * key_size, key_size) ||
>> +                   copy_from_user(value, uvalue + count * value_size, value_size)) {
>> +                       ret = -EFAULT;
>> +                       break;
>> +               }
>> +
>> +               preempt_disable();
>> +               __this_cpu_inc(bpf_prog_active);
>> +               rcu_read_lock();
>> +               if (is_lru_map)
>> +                       ret = htab_lru_map_update_elem(map, key, value, elem_map_flags);
>> +               else
>> +                       ret = htab_map_update_elem(map, key, value, elem_map_flags);
>> +               rcu_read_unlock();
>> +               __this_cpu_dec(bpf_prog_active);
>> +               preempt_enable();
>> +
>> +               if (ret) {
>> +                       if (put_user(count, &uattr->batch.count))
>> +                               ret = -EFAULT;
>> +                       break;
>> +               }
>> +       }
>> +
>> +out:
>> +       kfree(key);
>> +       kfree(value);
>> +       return ret;
>> +}
>> +
>> +static int
>> +__htab_map_delete_batch(struct bpf_map *map,
>> +                       const union bpf_attr *attr,
>> +                       union bpf_attr __user *uattr,
>> +                       bool is_lru_map)
>> +{
>> +       struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
>> +       u64 elem_map_flags, map_flags;
>> +       struct hlist_nulls_head *head;
>> +       struct hlist_nulls_node *n;
>> +       u32 batch, max_count;
>> +       unsigned long flags;
>> +       struct htab_elem *l;
>> +       struct bucket *b;
>> +
>> +       elem_map_flags = attr->batch.elem_flags;
>> +       map_flags = attr->batch.flags;
>> +       if (elem_map_flags || map_flags)
>> +               return -EINVAL;
>> +
>> +       max_count = attr->batch.count;
>> +       batch = (u32)attr->batch.batch;
>> +       if (max_count || batch >= htab->n_buckets)
>> +               return -EINVAL;
>> +
>> +       preempt_disable();
>> +       __this_cpu_inc(bpf_prog_active);
>> +       rcu_read_lock();
>> +
>> +again:
>> +       b = &htab->buckets[batch];
>> +       head = &b->head;
>> +       raw_spin_lock_irqsave(&b->lock, flags);
>> +
>> +       hlist_nulls_for_each_entry_rcu(l, n, head, hash_node) {
>> +               hlist_nulls_del_rcu(&l->hash_node);
>> +               if (is_lru_map)
>> +                       bpf_lru_push_free(&htab->lru, &l->lru_node);
>> +               else
>> +                       free_htab_elem(htab, l);
>> +       }
>> +
>> +       batch++;
>> +       if (batch >= htab->n_buckets)
>> +               goto out;
>> +
>> +       raw_spin_unlock_irqrestore(&b->lock, flags);
>> +       goto again;
>> +
>> +out:
>> +       raw_spin_unlock_irqrestore(&b->lock, flags);
>> +       rcu_read_unlock();
>> +       __this_cpu_dec(bpf_prog_active);
>> +       preempt_enable();
>> +
>> +       return 0;
>> +}
>> +
>> +static int
>> +htab_map_lookup_batch(struct bpf_map *map, const union bpf_attr *attr,
>> +                     union bpf_attr __user *uattr)
>> +{
>> +       return __htab_map_lookup_and_delete_batch(map, attr, uattr, false,
>> +                                                 false);
>> +}
>> +
>> +static int
>> +htab_map_lookup_and_delete_batch(struct bpf_map *map,
>> +                                const union bpf_attr *attr,
>> +                                union bpf_attr __user *uattr)
>> +{
>> +       return __htab_map_lookup_and_delete_batch(map, attr, uattr, true,
>> +                                                 false);
>> +}
>> +
>> +static int
>> +htab_map_update_batch(struct bpf_map *map, const union bpf_attr *attr,
>> +                     union bpf_attr __user *uattr)
>> +{
>> +       return __htab_map_update_batch(map, attr, uattr, false);
>> +}
>> +
>> +static int
>> +htab_map_delete_batch(struct bpf_map *map,
>> +                     const union bpf_attr *attr,
>> +                     union bpf_attr __user *uattr)
>> +{
>> +       return __htab_map_delete_batch(map, attr, uattr, false);
>> +}
>> +
>> +static int
>> +htab_lru_map_lookup_batch(struct bpf_map *map, const union bpf_attr *attr,
>> +                         union bpf_attr __user *uattr)
>> +{
>> +       return __htab_map_lookup_and_delete_batch(map, attr, uattr, false,
>> +                                                 true);
>> +}
>> +
>> +static int
>> +htab_lru_map_lookup_and_delete_batch(struct bpf_map *map,
>> +                                    const union bpf_attr *attr,
>> +                                    union bpf_attr __user *uattr)
>> +{
>> +       return __htab_map_lookup_and_delete_batch(map, attr, uattr, true,
>> +                                                 true);
>> +}
>> +
>> +static int
>> +htab_lru_map_update_batch(struct bpf_map *map, const union bpf_attr *attr,
>> +                         union bpf_attr __user *uattr)
>> +{
>> +       return __htab_map_update_batch(map, attr, uattr, true);
>> +}
>> +
>> +static int
>> +htab_lru_map_delete_batch(struct bpf_map *map,
>> +                         const union bpf_attr *attr,
>> +                         union bpf_attr __user *uattr)
>> +{
>> +       return __htab_map_delete_batch(map, attr, uattr, true);
>> +}
>> +
>>   const struct bpf_map_ops htab_map_ops = {
>>          .map_alloc_check = htab_map_alloc_check,
>>          .map_alloc = htab_map_alloc,
>> @@ -1242,6 +1558,10 @@ const struct bpf_map_ops htab_map_ops = {
>>          .map_delete_elem = htab_map_delete_elem,
>>          .map_gen_lookup = htab_map_gen_lookup,
>>          .map_seq_show_elem = htab_map_seq_show_elem,
>> +       .map_lookup_batch = htab_map_lookup_batch,
>> +       .map_lookup_and_delete_batch = htab_map_lookup_and_delete_batch,
>> +       .map_update_batch = htab_map_update_batch,
>> +       .map_delete_batch = htab_map_delete_batch,
>>   };
>>
>>   const struct bpf_map_ops htab_lru_map_ops = {
>> @@ -1255,6 +1575,10 @@ const struct bpf_map_ops htab_lru_map_ops = {
>>          .map_delete_elem = htab_lru_map_delete_elem,
>>          .map_gen_lookup = htab_lru_map_gen_lookup,
>>          .map_seq_show_elem = htab_map_seq_show_elem,
>> +       .map_lookup_batch = htab_lru_map_lookup_batch,
>> +       .map_lookup_and_delete_batch = htab_lru_map_lookup_and_delete_batch,
>> +       .map_update_batch = htab_lru_map_update_batch,
>> +       .map_delete_batch = htab_lru_map_delete_batch,
>>   };
>>
>>   /* Called from eBPF program */
>> diff --git a/kernel/bpf/syscall.c b/kernel/bpf/syscall.c
>> index ca60eafa6922..e83bdf7efbd8 100644
>> --- a/kernel/bpf/syscall.c
>> +++ b/kernel/bpf/syscall.c
>> @@ -2816,6 +2816,62 @@ static int bpf_task_fd_query(const union bpf_attr *attr,
>>          return err;
>>   }
>>
>> +#define BPF_MAP_BATCH_LAST_FIELD batch.flags
>> +
>> +#define BPF_DO_BATCH(fn)                       \
>> +       do {                                    \
>> +               if (!fn) {                      \
>> +                       err = -ENOTSUPP;        \
>> +                       goto err_put;           \
>> +               }                               \
>> +               err = fn(map, attr, uattr);     \
>> +       } while(0)
>> +
>> +static int bpf_map_do_batch(const union bpf_attr *attr,
>> +                           union bpf_attr __user *uattr,
>> +                           int cmd)
>> +{
>> +       struct bpf_map *map;
>> +       int err, ufd;
>> +       struct fd f;
>> +
>> +       if (CHECK_ATTR(BPF_MAP_BATCH))
>> +               return -EINVAL;
>> +
>> +       ufd = attr->batch.map_fd;
>> +       f = fdget(ufd);
>> +       map = __bpf_map_get(f);
>> +       if (IS_ERR(map))
>> +               return PTR_ERR(map);
>> +
>> +       if ((cmd == BPF_MAP_LOOKUP_BATCH ||
>> +            cmd == BPF_MAP_LOOKUP_AND_DELETE_BATCH) &&
>> +           !(map_get_sys_perms(map, f) & FMODE_CAN_READ)) {
>> +               err = -EPERM;
>> +               goto err_put;
>> +       }
>> +
>> +       if (cmd != BPF_MAP_LOOKUP_BATCH &&
>> +           !(map_get_sys_perms(map, f) & FMODE_CAN_WRITE)) {
>> +               err = -EPERM;
>> +               goto err_put;
>> +       }
>> +
>> +       if (cmd == BPF_MAP_LOOKUP_BATCH) {
>> +               BPF_DO_BATCH(map->ops->map_lookup_batch);
>> +       } else if (cmd == BPF_MAP_LOOKUP_AND_DELETE_BATCH) {
>> +               BPF_DO_BATCH(map->ops->map_lookup_and_delete_batch);
>> +       } else if (cmd == BPF_MAP_UPDATE_BATCH) {
>> +               BPF_DO_BATCH(map->ops->map_update_batch);
>> +       } else {
>> +               BPF_DO_BATCH(map->ops->map_delete_batch);
>> +       }
>> +
>> +err_put:
>> +       fdput(f);
>> +       return err;
>> +}
>> +
>>   SYSCALL_DEFINE3(bpf, int, cmd, union bpf_attr __user *, uattr, unsigned int, size)
>>   {
>>          union bpf_attr attr = {};
>> @@ -2913,6 +2969,18 @@ SYSCALL_DEFINE3(bpf, int, cmd, union bpf_attr __user *, uattr, unsigned int, siz
>>          case BPF_MAP_LOOKUP_AND_DELETE_ELEM:
>>                  err = map_lookup_and_delete_elem(&attr);
>>                  break;
>> +       case BPF_MAP_LOOKUP_BATCH:
>> +               err = bpf_map_do_batch(&attr, uattr, BPF_MAP_LOOKUP_BATCH);
>> +               break;
>> +       case BPF_MAP_LOOKUP_AND_DELETE_BATCH:
>> +               err = bpf_map_do_batch(&attr, uattr, BPF_MAP_LOOKUP_AND_DELETE_BATCH);
>> +               break;
>> +       case BPF_MAP_UPDATE_BATCH:
>> +               err = bpf_map_do_batch(&attr, uattr, BPF_MAP_UPDATE_BATCH);
>> +               break;
>> +       case BPF_MAP_DELETE_BATCH:
>> +               err = bpf_map_do_batch(&attr, uattr, BPF_MAP_DELETE_BATCH);
>> +               break;
>>          default:
>>                  err = -EINVAL;
>>                  break;
>> --
>> 2.17.1
>>
> 
> In general it'd be great if we could express the old functions
> (get_next_key, lookup, delete) in terms of this new ones so we  avoid
> having twice the code.

They do share some initial checkings at least. Let me check how I can do 
this.

> 
> Also to be honest I don't see how batching updates would be useful,
> maybe just try to do batching lookup and lookup_and_delete?

We do have a use case for batching updates.
Internally we have a bpf based firewall. it uses map-in-map.
The inner map is a hash table with thousands of entries.
Periodically some user space alerts will trigger firewall
rule changes. In this case, the new map is prepared
and thousands of bpf update syscalls are called to
update the new map. Once the new map is ready,
a final update into outer map will enable using the
new map.

Replacing these thousands map update syscalls with just
one map batch update is still a very good optimization.

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

end of thread, other threads:[~2019-09-24 16:28 UTC | newest]

Thread overview: 5+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2019-09-06 22:54 [RFC PATCH bpf-next v2 0/2] bpf: adding map batch processing support Yonghong Song
2019-09-06 22:54 ` [RFC PATCH bpf-next v2 1/2] " Yonghong Song
2019-09-23 23:20   ` Brian Vazquez
2019-09-24 16:28     ` Yonghong Song
2019-09-06 22:54 ` [RFC PATCH bpf-next v2 2/2] tools/bpf: test bpf_map_lookup_and_delete_batch() Yonghong Song

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).