cachepc-linux

Fork of AMDESE/linux with modifications for CachePC side-channel attack
git clone https://git.sinitax.com/sinitax/cachepc-linux
Log | Files | Refs | README | LICENSE | sfeed.txt

hashtab.c (65779B)


      1// SPDX-License-Identifier: GPL-2.0-only
      2/* Copyright (c) 2011-2014 PLUMgrid, http://plumgrid.com
      3 * Copyright (c) 2016 Facebook
      4 */
      5#include <linux/bpf.h>
      6#include <linux/btf.h>
      7#include <linux/jhash.h>
      8#include <linux/filter.h>
      9#include <linux/rculist_nulls.h>
     10#include <linux/random.h>
     11#include <uapi/linux/btf.h>
     12#include <linux/rcupdate_trace.h>
     13#include <linux/btf_ids.h>
     14#include "percpu_freelist.h"
     15#include "bpf_lru_list.h"
     16#include "map_in_map.h"
     17
     18#define HTAB_CREATE_FLAG_MASK						\
     19	(BPF_F_NO_PREALLOC | BPF_F_NO_COMMON_LRU | BPF_F_NUMA_NODE |	\
     20	 BPF_F_ACCESS_MASK | BPF_F_ZERO_SEED)
     21
     22#define BATCH_OPS(_name)			\
     23	.map_lookup_batch =			\
     24	_name##_map_lookup_batch,		\
     25	.map_lookup_and_delete_batch =		\
     26	_name##_map_lookup_and_delete_batch,	\
     27	.map_update_batch =			\
     28	generic_map_update_batch,		\
     29	.map_delete_batch =			\
     30	generic_map_delete_batch
     31
     32/*
     33 * The bucket lock has two protection scopes:
     34 *
     35 * 1) Serializing concurrent operations from BPF programs on different
     36 *    CPUs
     37 *
     38 * 2) Serializing concurrent operations from BPF programs and sys_bpf()
     39 *
     40 * BPF programs can execute in any context including perf, kprobes and
     41 * tracing. As there are almost no limits where perf, kprobes and tracing
     42 * can be invoked from the lock operations need to be protected against
     43 * deadlocks. Deadlocks can be caused by recursion and by an invocation in
     44 * the lock held section when functions which acquire this lock are invoked
     45 * from sys_bpf(). BPF recursion is prevented by incrementing the per CPU
     46 * variable bpf_prog_active, which prevents BPF programs attached to perf
     47 * events, kprobes and tracing to be invoked before the prior invocation
     48 * from one of these contexts completed. sys_bpf() uses the same mechanism
     49 * by pinning the task to the current CPU and incrementing the recursion
     50 * protection across the map operation.
     51 *
     52 * This has subtle implications on PREEMPT_RT. PREEMPT_RT forbids certain
     53 * operations like memory allocations (even with GFP_ATOMIC) from atomic
     54 * contexts. This is required because even with GFP_ATOMIC the memory
     55 * allocator calls into code paths which acquire locks with long held lock
     56 * sections. To ensure the deterministic behaviour these locks are regular
     57 * spinlocks, which are converted to 'sleepable' spinlocks on RT. The only
     58 * true atomic contexts on an RT kernel are the low level hardware
     59 * handling, scheduling, low level interrupt handling, NMIs etc. None of
     60 * these contexts should ever do memory allocations.
     61 *
     62 * As regular device interrupt handlers and soft interrupts are forced into
     63 * thread context, the existing code which does
     64 *   spin_lock*(); alloc(GPF_ATOMIC); spin_unlock*();
     65 * just works.
     66 *
     67 * In theory the BPF locks could be converted to regular spinlocks as well,
     68 * but the bucket locks and percpu_freelist locks can be taken from
     69 * arbitrary contexts (perf, kprobes, tracepoints) which are required to be
     70 * atomic contexts even on RT. These mechanisms require preallocated maps,
     71 * so there is no need to invoke memory allocations within the lock held
     72 * sections.
     73 *
     74 * BPF maps which need dynamic allocation are only used from (forced)
     75 * thread context on RT and can therefore use regular spinlocks which in
     76 * turn allows to invoke memory allocations from the lock held section.
     77 *
     78 * On a non RT kernel this distinction is neither possible nor required.
     79 * spinlock maps to raw_spinlock and the extra code is optimized out by the
     80 * compiler.
     81 */
     82struct bucket {
     83	struct hlist_nulls_head head;
     84	union {
     85		raw_spinlock_t raw_lock;
     86		spinlock_t     lock;
     87	};
     88};
     89
     90#define HASHTAB_MAP_LOCK_COUNT 8
     91#define HASHTAB_MAP_LOCK_MASK (HASHTAB_MAP_LOCK_COUNT - 1)
     92
     93struct bpf_htab {
     94	struct bpf_map map;
     95	struct bucket *buckets;
     96	void *elems;
     97	union {
     98		struct pcpu_freelist freelist;
     99		struct bpf_lru lru;
    100	};
    101	struct htab_elem *__percpu *extra_elems;
    102	atomic_t count;	/* number of elements in this hashtable */
    103	u32 n_buckets;	/* number of hash buckets */
    104	u32 elem_size;	/* size of each element in bytes */
    105	u32 hashrnd;
    106	struct lock_class_key lockdep_key;
    107	int __percpu *map_locked[HASHTAB_MAP_LOCK_COUNT];
    108};
    109
    110/* each htab element is struct htab_elem + key + value */
    111struct htab_elem {
    112	union {
    113		struct hlist_nulls_node hash_node;
    114		struct {
    115			void *padding;
    116			union {
    117				struct bpf_htab *htab;
    118				struct pcpu_freelist_node fnode;
    119				struct htab_elem *batch_flink;
    120			};
    121		};
    122	};
    123	union {
    124		struct rcu_head rcu;
    125		struct bpf_lru_node lru_node;
    126	};
    127	u32 hash;
    128	char key[] __aligned(8);
    129};
    130
    131static inline bool htab_is_prealloc(const struct bpf_htab *htab)
    132{
    133	return !(htab->map.map_flags & BPF_F_NO_PREALLOC);
    134}
    135
    136static inline bool htab_use_raw_lock(const struct bpf_htab *htab)
    137{
    138	return (!IS_ENABLED(CONFIG_PREEMPT_RT) || htab_is_prealloc(htab));
    139}
    140
    141static void htab_init_buckets(struct bpf_htab *htab)
    142{
    143	unsigned int i;
    144
    145	for (i = 0; i < htab->n_buckets; i++) {
    146		INIT_HLIST_NULLS_HEAD(&htab->buckets[i].head, i);
    147		if (htab_use_raw_lock(htab)) {
    148			raw_spin_lock_init(&htab->buckets[i].raw_lock);
    149			lockdep_set_class(&htab->buckets[i].raw_lock,
    150					  &htab->lockdep_key);
    151		} else {
    152			spin_lock_init(&htab->buckets[i].lock);
    153			lockdep_set_class(&htab->buckets[i].lock,
    154					  &htab->lockdep_key);
    155		}
    156		cond_resched();
    157	}
    158}
    159
    160static inline int htab_lock_bucket(const struct bpf_htab *htab,
    161				   struct bucket *b, u32 hash,
    162				   unsigned long *pflags)
    163{
    164	unsigned long flags;
    165
    166	hash = hash & HASHTAB_MAP_LOCK_MASK;
    167
    168	migrate_disable();
    169	if (unlikely(__this_cpu_inc_return(*(htab->map_locked[hash])) != 1)) {
    170		__this_cpu_dec(*(htab->map_locked[hash]));
    171		migrate_enable();
    172		return -EBUSY;
    173	}
    174
    175	if (htab_use_raw_lock(htab))
    176		raw_spin_lock_irqsave(&b->raw_lock, flags);
    177	else
    178		spin_lock_irqsave(&b->lock, flags);
    179	*pflags = flags;
    180
    181	return 0;
    182}
    183
    184static inline void htab_unlock_bucket(const struct bpf_htab *htab,
    185				      struct bucket *b, u32 hash,
    186				      unsigned long flags)
    187{
    188	hash = hash & HASHTAB_MAP_LOCK_MASK;
    189	if (htab_use_raw_lock(htab))
    190		raw_spin_unlock_irqrestore(&b->raw_lock, flags);
    191	else
    192		spin_unlock_irqrestore(&b->lock, flags);
    193	__this_cpu_dec(*(htab->map_locked[hash]));
    194	migrate_enable();
    195}
    196
    197static bool htab_lru_map_delete_node(void *arg, struct bpf_lru_node *node);
    198
    199static bool htab_is_lru(const struct bpf_htab *htab)
    200{
    201	return htab->map.map_type == BPF_MAP_TYPE_LRU_HASH ||
    202		htab->map.map_type == BPF_MAP_TYPE_LRU_PERCPU_HASH;
    203}
    204
    205static bool htab_is_percpu(const struct bpf_htab *htab)
    206{
    207	return htab->map.map_type == BPF_MAP_TYPE_PERCPU_HASH ||
    208		htab->map.map_type == BPF_MAP_TYPE_LRU_PERCPU_HASH;
    209}
    210
    211static inline void htab_elem_set_ptr(struct htab_elem *l, u32 key_size,
    212				     void __percpu *pptr)
    213{
    214	*(void __percpu **)(l->key + key_size) = pptr;
    215}
    216
    217static inline void __percpu *htab_elem_get_ptr(struct htab_elem *l, u32 key_size)
    218{
    219	return *(void __percpu **)(l->key + key_size);
    220}
    221
    222static void *fd_htab_map_get_ptr(const struct bpf_map *map, struct htab_elem *l)
    223{
    224	return *(void **)(l->key + roundup(map->key_size, 8));
    225}
    226
    227static struct htab_elem *get_htab_elem(struct bpf_htab *htab, int i)
    228{
    229	return (struct htab_elem *) (htab->elems + i * (u64)htab->elem_size);
    230}
    231
    232static bool htab_has_extra_elems(struct bpf_htab *htab)
    233{
    234	return !htab_is_percpu(htab) && !htab_is_lru(htab);
    235}
    236
    237static void htab_free_prealloced_timers(struct bpf_htab *htab)
    238{
    239	u32 num_entries = htab->map.max_entries;
    240	int i;
    241
    242	if (!map_value_has_timer(&htab->map))
    243		return;
    244	if (htab_has_extra_elems(htab))
    245		num_entries += num_possible_cpus();
    246
    247	for (i = 0; i < num_entries; i++) {
    248		struct htab_elem *elem;
    249
    250		elem = get_htab_elem(htab, i);
    251		bpf_timer_cancel_and_free(elem->key +
    252					  round_up(htab->map.key_size, 8) +
    253					  htab->map.timer_off);
    254		cond_resched();
    255	}
    256}
    257
    258static void htab_free_prealloced_kptrs(struct bpf_htab *htab)
    259{
    260	u32 num_entries = htab->map.max_entries;
    261	int i;
    262
    263	if (!map_value_has_kptrs(&htab->map))
    264		return;
    265	if (htab_has_extra_elems(htab))
    266		num_entries += num_possible_cpus();
    267
    268	for (i = 0; i < num_entries; i++) {
    269		struct htab_elem *elem;
    270
    271		elem = get_htab_elem(htab, i);
    272		bpf_map_free_kptrs(&htab->map, elem->key + round_up(htab->map.key_size, 8));
    273		cond_resched();
    274	}
    275}
    276
    277static void htab_free_elems(struct bpf_htab *htab)
    278{
    279	int i;
    280
    281	if (!htab_is_percpu(htab))
    282		goto free_elems;
    283
    284	for (i = 0; i < htab->map.max_entries; i++) {
    285		void __percpu *pptr;
    286
    287		pptr = htab_elem_get_ptr(get_htab_elem(htab, i),
    288					 htab->map.key_size);
    289		free_percpu(pptr);
    290		cond_resched();
    291	}
    292free_elems:
    293	bpf_map_area_free(htab->elems);
    294}
    295
    296/* The LRU list has a lock (lru_lock). Each htab bucket has a lock
    297 * (bucket_lock). If both locks need to be acquired together, the lock
    298 * order is always lru_lock -> bucket_lock and this only happens in
    299 * bpf_lru_list.c logic. For example, certain code path of
    300 * bpf_lru_pop_free(), which is called by function prealloc_lru_pop(),
    301 * will acquire lru_lock first followed by acquiring bucket_lock.
    302 *
    303 * In hashtab.c, to avoid deadlock, lock acquisition of
    304 * bucket_lock followed by lru_lock is not allowed. In such cases,
    305 * bucket_lock needs to be released first before acquiring lru_lock.
    306 */
    307static struct htab_elem *prealloc_lru_pop(struct bpf_htab *htab, void *key,
    308					  u32 hash)
    309{
    310	struct bpf_lru_node *node = bpf_lru_pop_free(&htab->lru, hash);
    311	struct htab_elem *l;
    312
    313	if (node) {
    314		u32 key_size = htab->map.key_size;
    315
    316		l = container_of(node, struct htab_elem, lru_node);
    317		memcpy(l->key, key, key_size);
    318		check_and_init_map_value(&htab->map,
    319					 l->key + round_up(key_size, 8));
    320		return l;
    321	}
    322
    323	return NULL;
    324}
    325
    326static int prealloc_init(struct bpf_htab *htab)
    327{
    328	u32 num_entries = htab->map.max_entries;
    329	int err = -ENOMEM, i;
    330
    331	if (htab_has_extra_elems(htab))
    332		num_entries += num_possible_cpus();
    333
    334	htab->elems = bpf_map_area_alloc((u64)htab->elem_size * num_entries,
    335					 htab->map.numa_node);
    336	if (!htab->elems)
    337		return -ENOMEM;
    338
    339	if (!htab_is_percpu(htab))
    340		goto skip_percpu_elems;
    341
    342	for (i = 0; i < num_entries; i++) {
    343		u32 size = round_up(htab->map.value_size, 8);
    344		void __percpu *pptr;
    345
    346		pptr = bpf_map_alloc_percpu(&htab->map, size, 8,
    347					    GFP_USER | __GFP_NOWARN);
    348		if (!pptr)
    349			goto free_elems;
    350		htab_elem_set_ptr(get_htab_elem(htab, i), htab->map.key_size,
    351				  pptr);
    352		cond_resched();
    353	}
    354
    355skip_percpu_elems:
    356	if (htab_is_lru(htab))
    357		err = bpf_lru_init(&htab->lru,
    358				   htab->map.map_flags & BPF_F_NO_COMMON_LRU,
    359				   offsetof(struct htab_elem, hash) -
    360				   offsetof(struct htab_elem, lru_node),
    361				   htab_lru_map_delete_node,
    362				   htab);
    363	else
    364		err = pcpu_freelist_init(&htab->freelist);
    365
    366	if (err)
    367		goto free_elems;
    368
    369	if (htab_is_lru(htab))
    370		bpf_lru_populate(&htab->lru, htab->elems,
    371				 offsetof(struct htab_elem, lru_node),
    372				 htab->elem_size, num_entries);
    373	else
    374		pcpu_freelist_populate(&htab->freelist,
    375				       htab->elems + offsetof(struct htab_elem, fnode),
    376				       htab->elem_size, num_entries);
    377
    378	return 0;
    379
    380free_elems:
    381	htab_free_elems(htab);
    382	return err;
    383}
    384
    385static void prealloc_destroy(struct bpf_htab *htab)
    386{
    387	htab_free_elems(htab);
    388
    389	if (htab_is_lru(htab))
    390		bpf_lru_destroy(&htab->lru);
    391	else
    392		pcpu_freelist_destroy(&htab->freelist);
    393}
    394
    395static int alloc_extra_elems(struct bpf_htab *htab)
    396{
    397	struct htab_elem *__percpu *pptr, *l_new;
    398	struct pcpu_freelist_node *l;
    399	int cpu;
    400
    401	pptr = bpf_map_alloc_percpu(&htab->map, sizeof(struct htab_elem *), 8,
    402				    GFP_USER | __GFP_NOWARN);
    403	if (!pptr)
    404		return -ENOMEM;
    405
    406	for_each_possible_cpu(cpu) {
    407		l = pcpu_freelist_pop(&htab->freelist);
    408		/* pop will succeed, since prealloc_init()
    409		 * preallocated extra num_possible_cpus elements
    410		 */
    411		l_new = container_of(l, struct htab_elem, fnode);
    412		*per_cpu_ptr(pptr, cpu) = l_new;
    413	}
    414	htab->extra_elems = pptr;
    415	return 0;
    416}
    417
    418/* Called from syscall */
    419static int htab_map_alloc_check(union bpf_attr *attr)
    420{
    421	bool percpu = (attr->map_type == BPF_MAP_TYPE_PERCPU_HASH ||
    422		       attr->map_type == BPF_MAP_TYPE_LRU_PERCPU_HASH);
    423	bool lru = (attr->map_type == BPF_MAP_TYPE_LRU_HASH ||
    424		    attr->map_type == BPF_MAP_TYPE_LRU_PERCPU_HASH);
    425	/* percpu_lru means each cpu has its own LRU list.
    426	 * it is different from BPF_MAP_TYPE_PERCPU_HASH where
    427	 * the map's value itself is percpu.  percpu_lru has
    428	 * nothing to do with the map's value.
    429	 */
    430	bool percpu_lru = (attr->map_flags & BPF_F_NO_COMMON_LRU);
    431	bool prealloc = !(attr->map_flags & BPF_F_NO_PREALLOC);
    432	bool zero_seed = (attr->map_flags & BPF_F_ZERO_SEED);
    433	int numa_node = bpf_map_attr_numa_node(attr);
    434
    435	BUILD_BUG_ON(offsetof(struct htab_elem, htab) !=
    436		     offsetof(struct htab_elem, hash_node.pprev));
    437	BUILD_BUG_ON(offsetof(struct htab_elem, fnode.next) !=
    438		     offsetof(struct htab_elem, hash_node.pprev));
    439
    440	if (lru && !bpf_capable())
    441		/* LRU implementation is much complicated than other
    442		 * maps.  Hence, limit to CAP_BPF.
    443		 */
    444		return -EPERM;
    445
    446	if (zero_seed && !capable(CAP_SYS_ADMIN))
    447		/* Guard against local DoS, and discourage production use. */
    448		return -EPERM;
    449
    450	if (attr->map_flags & ~HTAB_CREATE_FLAG_MASK ||
    451	    !bpf_map_flags_access_ok(attr->map_flags))
    452		return -EINVAL;
    453
    454	if (!lru && percpu_lru)
    455		return -EINVAL;
    456
    457	if (lru && !prealloc)
    458		return -ENOTSUPP;
    459
    460	if (numa_node != NUMA_NO_NODE && (percpu || percpu_lru))
    461		return -EINVAL;
    462
    463	/* check sanity of attributes.
    464	 * value_size == 0 may be allowed in the future to use map as a set
    465	 */
    466	if (attr->max_entries == 0 || attr->key_size == 0 ||
    467	    attr->value_size == 0)
    468		return -EINVAL;
    469
    470	if ((u64)attr->key_size + attr->value_size >= KMALLOC_MAX_SIZE -
    471	   sizeof(struct htab_elem))
    472		/* if key_size + value_size is bigger, the user space won't be
    473		 * able to access the elements via bpf syscall. This check
    474		 * also makes sure that the elem_size doesn't overflow and it's
    475		 * kmalloc-able later in htab_map_update_elem()
    476		 */
    477		return -E2BIG;
    478
    479	return 0;
    480}
    481
    482static struct bpf_map *htab_map_alloc(union bpf_attr *attr)
    483{
    484	bool percpu = (attr->map_type == BPF_MAP_TYPE_PERCPU_HASH ||
    485		       attr->map_type == BPF_MAP_TYPE_LRU_PERCPU_HASH);
    486	bool lru = (attr->map_type == BPF_MAP_TYPE_LRU_HASH ||
    487		    attr->map_type == BPF_MAP_TYPE_LRU_PERCPU_HASH);
    488	/* percpu_lru means each cpu has its own LRU list.
    489	 * it is different from BPF_MAP_TYPE_PERCPU_HASH where
    490	 * the map's value itself is percpu.  percpu_lru has
    491	 * nothing to do with the map's value.
    492	 */
    493	bool percpu_lru = (attr->map_flags & BPF_F_NO_COMMON_LRU);
    494	bool prealloc = !(attr->map_flags & BPF_F_NO_PREALLOC);
    495	struct bpf_htab *htab;
    496	int err, i;
    497
    498	htab = kzalloc(sizeof(*htab), GFP_USER | __GFP_ACCOUNT);
    499	if (!htab)
    500		return ERR_PTR(-ENOMEM);
    501
    502	lockdep_register_key(&htab->lockdep_key);
    503
    504	bpf_map_init_from_attr(&htab->map, attr);
    505
    506	if (percpu_lru) {
    507		/* ensure each CPU's lru list has >=1 elements.
    508		 * since we are at it, make each lru list has the same
    509		 * number of elements.
    510		 */
    511		htab->map.max_entries = roundup(attr->max_entries,
    512						num_possible_cpus());
    513		if (htab->map.max_entries < attr->max_entries)
    514			htab->map.max_entries = rounddown(attr->max_entries,
    515							  num_possible_cpus());
    516	}
    517
    518	/* hash table size must be power of 2 */
    519	htab->n_buckets = roundup_pow_of_two(htab->map.max_entries);
    520
    521	htab->elem_size = sizeof(struct htab_elem) +
    522			  round_up(htab->map.key_size, 8);
    523	if (percpu)
    524		htab->elem_size += sizeof(void *);
    525	else
    526		htab->elem_size += round_up(htab->map.value_size, 8);
    527
    528	err = -E2BIG;
    529	/* prevent zero size kmalloc and check for u32 overflow */
    530	if (htab->n_buckets == 0 ||
    531	    htab->n_buckets > U32_MAX / sizeof(struct bucket))
    532		goto free_htab;
    533
    534	err = -ENOMEM;
    535	htab->buckets = bpf_map_area_alloc(htab->n_buckets *
    536					   sizeof(struct bucket),
    537					   htab->map.numa_node);
    538	if (!htab->buckets)
    539		goto free_htab;
    540
    541	for (i = 0; i < HASHTAB_MAP_LOCK_COUNT; i++) {
    542		htab->map_locked[i] = bpf_map_alloc_percpu(&htab->map,
    543							   sizeof(int),
    544							   sizeof(int),
    545							   GFP_USER);
    546		if (!htab->map_locked[i])
    547			goto free_map_locked;
    548	}
    549
    550	if (htab->map.map_flags & BPF_F_ZERO_SEED)
    551		htab->hashrnd = 0;
    552	else
    553		htab->hashrnd = get_random_int();
    554
    555	htab_init_buckets(htab);
    556
    557	if (prealloc) {
    558		err = prealloc_init(htab);
    559		if (err)
    560			goto free_map_locked;
    561
    562		if (!percpu && !lru) {
    563			/* lru itself can remove the least used element, so
    564			 * there is no need for an extra elem during map_update.
    565			 */
    566			err = alloc_extra_elems(htab);
    567			if (err)
    568				goto free_prealloc;
    569		}
    570	}
    571
    572	return &htab->map;
    573
    574free_prealloc:
    575	prealloc_destroy(htab);
    576free_map_locked:
    577	for (i = 0; i < HASHTAB_MAP_LOCK_COUNT; i++)
    578		free_percpu(htab->map_locked[i]);
    579	bpf_map_area_free(htab->buckets);
    580free_htab:
    581	lockdep_unregister_key(&htab->lockdep_key);
    582	kfree(htab);
    583	return ERR_PTR(err);
    584}
    585
    586static inline u32 htab_map_hash(const void *key, u32 key_len, u32 hashrnd)
    587{
    588	return jhash(key, key_len, hashrnd);
    589}
    590
    591static inline struct bucket *__select_bucket(struct bpf_htab *htab, u32 hash)
    592{
    593	return &htab->buckets[hash & (htab->n_buckets - 1)];
    594}
    595
    596static inline struct hlist_nulls_head *select_bucket(struct bpf_htab *htab, u32 hash)
    597{
    598	return &__select_bucket(htab, hash)->head;
    599}
    600
    601/* this lookup function can only be called with bucket lock taken */
    602static struct htab_elem *lookup_elem_raw(struct hlist_nulls_head *head, u32 hash,
    603					 void *key, u32 key_size)
    604{
    605	struct hlist_nulls_node *n;
    606	struct htab_elem *l;
    607
    608	hlist_nulls_for_each_entry_rcu(l, n, head, hash_node)
    609		if (l->hash == hash && !memcmp(&l->key, key, key_size))
    610			return l;
    611
    612	return NULL;
    613}
    614
    615/* can be called without bucket lock. it will repeat the loop in
    616 * the unlikely event when elements moved from one bucket into another
    617 * while link list is being walked
    618 */
    619static struct htab_elem *lookup_nulls_elem_raw(struct hlist_nulls_head *head,
    620					       u32 hash, void *key,
    621					       u32 key_size, u32 n_buckets)
    622{
    623	struct hlist_nulls_node *n;
    624	struct htab_elem *l;
    625
    626again:
    627	hlist_nulls_for_each_entry_rcu(l, n, head, hash_node)
    628		if (l->hash == hash && !memcmp(&l->key, key, key_size))
    629			return l;
    630
    631	if (unlikely(get_nulls_value(n) != (hash & (n_buckets - 1))))
    632		goto again;
    633
    634	return NULL;
    635}
    636
    637/* Called from syscall or from eBPF program directly, so
    638 * arguments have to match bpf_map_lookup_elem() exactly.
    639 * The return value is adjusted by BPF instructions
    640 * in htab_map_gen_lookup().
    641 */
    642static void *__htab_map_lookup_elem(struct bpf_map *map, void *key)
    643{
    644	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
    645	struct hlist_nulls_head *head;
    646	struct htab_elem *l;
    647	u32 hash, key_size;
    648
    649	WARN_ON_ONCE(!rcu_read_lock_held() && !rcu_read_lock_trace_held() &&
    650		     !rcu_read_lock_bh_held());
    651
    652	key_size = map->key_size;
    653
    654	hash = htab_map_hash(key, key_size, htab->hashrnd);
    655
    656	head = select_bucket(htab, hash);
    657
    658	l = lookup_nulls_elem_raw(head, hash, key, key_size, htab->n_buckets);
    659
    660	return l;
    661}
    662
    663static void *htab_map_lookup_elem(struct bpf_map *map, void *key)
    664{
    665	struct htab_elem *l = __htab_map_lookup_elem(map, key);
    666
    667	if (l)
    668		return l->key + round_up(map->key_size, 8);
    669
    670	return NULL;
    671}
    672
    673/* inline bpf_map_lookup_elem() call.
    674 * Instead of:
    675 * bpf_prog
    676 *   bpf_map_lookup_elem
    677 *     map->ops->map_lookup_elem
    678 *       htab_map_lookup_elem
    679 *         __htab_map_lookup_elem
    680 * do:
    681 * bpf_prog
    682 *   __htab_map_lookup_elem
    683 */
    684static int htab_map_gen_lookup(struct bpf_map *map, struct bpf_insn *insn_buf)
    685{
    686	struct bpf_insn *insn = insn_buf;
    687	const int ret = BPF_REG_0;
    688
    689	BUILD_BUG_ON(!__same_type(&__htab_map_lookup_elem,
    690		     (void *(*)(struct bpf_map *map, void *key))NULL));
    691	*insn++ = BPF_EMIT_CALL(__htab_map_lookup_elem);
    692	*insn++ = BPF_JMP_IMM(BPF_JEQ, ret, 0, 1);
    693	*insn++ = BPF_ALU64_IMM(BPF_ADD, ret,
    694				offsetof(struct htab_elem, key) +
    695				round_up(map->key_size, 8));
    696	return insn - insn_buf;
    697}
    698
    699static __always_inline void *__htab_lru_map_lookup_elem(struct bpf_map *map,
    700							void *key, const bool mark)
    701{
    702	struct htab_elem *l = __htab_map_lookup_elem(map, key);
    703
    704	if (l) {
    705		if (mark)
    706			bpf_lru_node_set_ref(&l->lru_node);
    707		return l->key + round_up(map->key_size, 8);
    708	}
    709
    710	return NULL;
    711}
    712
    713static void *htab_lru_map_lookup_elem(struct bpf_map *map, void *key)
    714{
    715	return __htab_lru_map_lookup_elem(map, key, true);
    716}
    717
    718static void *htab_lru_map_lookup_elem_sys(struct bpf_map *map, void *key)
    719{
    720	return __htab_lru_map_lookup_elem(map, key, false);
    721}
    722
    723static int htab_lru_map_gen_lookup(struct bpf_map *map,
    724				   struct bpf_insn *insn_buf)
    725{
    726	struct bpf_insn *insn = insn_buf;
    727	const int ret = BPF_REG_0;
    728	const int ref_reg = BPF_REG_1;
    729
    730	BUILD_BUG_ON(!__same_type(&__htab_map_lookup_elem,
    731		     (void *(*)(struct bpf_map *map, void *key))NULL));
    732	*insn++ = BPF_EMIT_CALL(__htab_map_lookup_elem);
    733	*insn++ = BPF_JMP_IMM(BPF_JEQ, ret, 0, 4);
    734	*insn++ = BPF_LDX_MEM(BPF_B, ref_reg, ret,
    735			      offsetof(struct htab_elem, lru_node) +
    736			      offsetof(struct bpf_lru_node, ref));
    737	*insn++ = BPF_JMP_IMM(BPF_JNE, ref_reg, 0, 1);
    738	*insn++ = BPF_ST_MEM(BPF_B, ret,
    739			     offsetof(struct htab_elem, lru_node) +
    740			     offsetof(struct bpf_lru_node, ref),
    741			     1);
    742	*insn++ = BPF_ALU64_IMM(BPF_ADD, ret,
    743				offsetof(struct htab_elem, key) +
    744				round_up(map->key_size, 8));
    745	return insn - insn_buf;
    746}
    747
    748static void check_and_free_fields(struct bpf_htab *htab,
    749				  struct htab_elem *elem)
    750{
    751	void *map_value = elem->key + round_up(htab->map.key_size, 8);
    752
    753	if (map_value_has_timer(&htab->map))
    754		bpf_timer_cancel_and_free(map_value + htab->map.timer_off);
    755	if (map_value_has_kptrs(&htab->map))
    756		bpf_map_free_kptrs(&htab->map, map_value);
    757}
    758
    759/* It is called from the bpf_lru_list when the LRU needs to delete
    760 * older elements from the htab.
    761 */
    762static bool htab_lru_map_delete_node(void *arg, struct bpf_lru_node *node)
    763{
    764	struct bpf_htab *htab = arg;
    765	struct htab_elem *l = NULL, *tgt_l;
    766	struct hlist_nulls_head *head;
    767	struct hlist_nulls_node *n;
    768	unsigned long flags;
    769	struct bucket *b;
    770	int ret;
    771
    772	tgt_l = container_of(node, struct htab_elem, lru_node);
    773	b = __select_bucket(htab, tgt_l->hash);
    774	head = &b->head;
    775
    776	ret = htab_lock_bucket(htab, b, tgt_l->hash, &flags);
    777	if (ret)
    778		return false;
    779
    780	hlist_nulls_for_each_entry_rcu(l, n, head, hash_node)
    781		if (l == tgt_l) {
    782			hlist_nulls_del_rcu(&l->hash_node);
    783			check_and_free_fields(htab, l);
    784			break;
    785		}
    786
    787	htab_unlock_bucket(htab, b, tgt_l->hash, flags);
    788
    789	return l == tgt_l;
    790}
    791
    792/* Called from syscall */
    793static int htab_map_get_next_key(struct bpf_map *map, void *key, void *next_key)
    794{
    795	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
    796	struct hlist_nulls_head *head;
    797	struct htab_elem *l, *next_l;
    798	u32 hash, key_size;
    799	int i = 0;
    800
    801	WARN_ON_ONCE(!rcu_read_lock_held());
    802
    803	key_size = map->key_size;
    804
    805	if (!key)
    806		goto find_first_elem;
    807
    808	hash = htab_map_hash(key, key_size, htab->hashrnd);
    809
    810	head = select_bucket(htab, hash);
    811
    812	/* lookup the key */
    813	l = lookup_nulls_elem_raw(head, hash, key, key_size, htab->n_buckets);
    814
    815	if (!l)
    816		goto find_first_elem;
    817
    818	/* key was found, get next key in the same bucket */
    819	next_l = hlist_nulls_entry_safe(rcu_dereference_raw(hlist_nulls_next_rcu(&l->hash_node)),
    820				  struct htab_elem, hash_node);
    821
    822	if (next_l) {
    823		/* if next elem in this hash list is non-zero, just return it */
    824		memcpy(next_key, next_l->key, key_size);
    825		return 0;
    826	}
    827
    828	/* no more elements in this hash list, go to the next bucket */
    829	i = hash & (htab->n_buckets - 1);
    830	i++;
    831
    832find_first_elem:
    833	/* iterate over buckets */
    834	for (; i < htab->n_buckets; i++) {
    835		head = select_bucket(htab, i);
    836
    837		/* pick first element in the bucket */
    838		next_l = hlist_nulls_entry_safe(rcu_dereference_raw(hlist_nulls_first_rcu(head)),
    839					  struct htab_elem, hash_node);
    840		if (next_l) {
    841			/* if it's not empty, just return it */
    842			memcpy(next_key, next_l->key, key_size);
    843			return 0;
    844		}
    845	}
    846
    847	/* iterated over all buckets and all elements */
    848	return -ENOENT;
    849}
    850
    851static void htab_elem_free(struct bpf_htab *htab, struct htab_elem *l)
    852{
    853	if (htab->map.map_type == BPF_MAP_TYPE_PERCPU_HASH)
    854		free_percpu(htab_elem_get_ptr(l, htab->map.key_size));
    855	check_and_free_fields(htab, l);
    856	kfree(l);
    857}
    858
    859static void htab_elem_free_rcu(struct rcu_head *head)
    860{
    861	struct htab_elem *l = container_of(head, struct htab_elem, rcu);
    862	struct bpf_htab *htab = l->htab;
    863
    864	htab_elem_free(htab, l);
    865}
    866
    867static void htab_put_fd_value(struct bpf_htab *htab, struct htab_elem *l)
    868{
    869	struct bpf_map *map = &htab->map;
    870	void *ptr;
    871
    872	if (map->ops->map_fd_put_ptr) {
    873		ptr = fd_htab_map_get_ptr(map, l);
    874		map->ops->map_fd_put_ptr(ptr);
    875	}
    876}
    877
    878static void free_htab_elem(struct bpf_htab *htab, struct htab_elem *l)
    879{
    880	htab_put_fd_value(htab, l);
    881
    882	if (htab_is_prealloc(htab)) {
    883		check_and_free_fields(htab, l);
    884		__pcpu_freelist_push(&htab->freelist, &l->fnode);
    885	} else {
    886		atomic_dec(&htab->count);
    887		l->htab = htab;
    888		call_rcu(&l->rcu, htab_elem_free_rcu);
    889	}
    890}
    891
    892static void pcpu_copy_value(struct bpf_htab *htab, void __percpu *pptr,
    893			    void *value, bool onallcpus)
    894{
    895	if (!onallcpus) {
    896		/* copy true value_size bytes */
    897		memcpy(this_cpu_ptr(pptr), value, htab->map.value_size);
    898	} else {
    899		u32 size = round_up(htab->map.value_size, 8);
    900		int off = 0, cpu;
    901
    902		for_each_possible_cpu(cpu) {
    903			bpf_long_memcpy(per_cpu_ptr(pptr, cpu),
    904					value + off, size);
    905			off += size;
    906		}
    907	}
    908}
    909
    910static void pcpu_init_value(struct bpf_htab *htab, void __percpu *pptr,
    911			    void *value, bool onallcpus)
    912{
    913	/* When using prealloc and not setting the initial value on all cpus,
    914	 * zero-fill element values for other cpus (just as what happens when
    915	 * not using prealloc). Otherwise, bpf program has no way to ensure
    916	 * known initial values for cpus other than current one
    917	 * (onallcpus=false always when coming from bpf prog).
    918	 */
    919	if (htab_is_prealloc(htab) && !onallcpus) {
    920		u32 size = round_up(htab->map.value_size, 8);
    921		int current_cpu = raw_smp_processor_id();
    922		int cpu;
    923
    924		for_each_possible_cpu(cpu) {
    925			if (cpu == current_cpu)
    926				bpf_long_memcpy(per_cpu_ptr(pptr, cpu), value,
    927						size);
    928			else
    929				memset(per_cpu_ptr(pptr, cpu), 0, size);
    930		}
    931	} else {
    932		pcpu_copy_value(htab, pptr, value, onallcpus);
    933	}
    934}
    935
    936static bool fd_htab_map_needs_adjust(const struct bpf_htab *htab)
    937{
    938	return htab->map.map_type == BPF_MAP_TYPE_HASH_OF_MAPS &&
    939	       BITS_PER_LONG == 64;
    940}
    941
    942static struct htab_elem *alloc_htab_elem(struct bpf_htab *htab, void *key,
    943					 void *value, u32 key_size, u32 hash,
    944					 bool percpu, bool onallcpus,
    945					 struct htab_elem *old_elem)
    946{
    947	u32 size = htab->map.value_size;
    948	bool prealloc = htab_is_prealloc(htab);
    949	struct htab_elem *l_new, **pl_new;
    950	void __percpu *pptr;
    951
    952	if (prealloc) {
    953		if (old_elem) {
    954			/* if we're updating the existing element,
    955			 * use per-cpu extra elems to avoid freelist_pop/push
    956			 */
    957			pl_new = this_cpu_ptr(htab->extra_elems);
    958			l_new = *pl_new;
    959			htab_put_fd_value(htab, old_elem);
    960			*pl_new = old_elem;
    961		} else {
    962			struct pcpu_freelist_node *l;
    963
    964			l = __pcpu_freelist_pop(&htab->freelist);
    965			if (!l)
    966				return ERR_PTR(-E2BIG);
    967			l_new = container_of(l, struct htab_elem, fnode);
    968		}
    969	} else {
    970		if (atomic_inc_return(&htab->count) > htab->map.max_entries)
    971			if (!old_elem) {
    972				/* when map is full and update() is replacing
    973				 * old element, it's ok to allocate, since
    974				 * old element will be freed immediately.
    975				 * Otherwise return an error
    976				 */
    977				l_new = ERR_PTR(-E2BIG);
    978				goto dec_count;
    979			}
    980		l_new = bpf_map_kmalloc_node(&htab->map, htab->elem_size,
    981					     GFP_ATOMIC | __GFP_NOWARN,
    982					     htab->map.numa_node);
    983		if (!l_new) {
    984			l_new = ERR_PTR(-ENOMEM);
    985			goto dec_count;
    986		}
    987		check_and_init_map_value(&htab->map,
    988					 l_new->key + round_up(key_size, 8));
    989	}
    990
    991	memcpy(l_new->key, key, key_size);
    992	if (percpu) {
    993		size = round_up(size, 8);
    994		if (prealloc) {
    995			pptr = htab_elem_get_ptr(l_new, key_size);
    996		} else {
    997			/* alloc_percpu zero-fills */
    998			pptr = bpf_map_alloc_percpu(&htab->map, size, 8,
    999						    GFP_ATOMIC | __GFP_NOWARN);
   1000			if (!pptr) {
   1001				kfree(l_new);
   1002				l_new = ERR_PTR(-ENOMEM);
   1003				goto dec_count;
   1004			}
   1005		}
   1006
   1007		pcpu_init_value(htab, pptr, value, onallcpus);
   1008
   1009		if (!prealloc)
   1010			htab_elem_set_ptr(l_new, key_size, pptr);
   1011	} else if (fd_htab_map_needs_adjust(htab)) {
   1012		size = round_up(size, 8);
   1013		memcpy(l_new->key + round_up(key_size, 8), value, size);
   1014	} else {
   1015		copy_map_value(&htab->map,
   1016			       l_new->key + round_up(key_size, 8),
   1017			       value);
   1018	}
   1019
   1020	l_new->hash = hash;
   1021	return l_new;
   1022dec_count:
   1023	atomic_dec(&htab->count);
   1024	return l_new;
   1025}
   1026
   1027static int check_flags(struct bpf_htab *htab, struct htab_elem *l_old,
   1028		       u64 map_flags)
   1029{
   1030	if (l_old && (map_flags & ~BPF_F_LOCK) == BPF_NOEXIST)
   1031		/* elem already exists */
   1032		return -EEXIST;
   1033
   1034	if (!l_old && (map_flags & ~BPF_F_LOCK) == BPF_EXIST)
   1035		/* elem doesn't exist, cannot update it */
   1036		return -ENOENT;
   1037
   1038	return 0;
   1039}
   1040
   1041/* Called from syscall or from eBPF program */
   1042static int htab_map_update_elem(struct bpf_map *map, void *key, void *value,
   1043				u64 map_flags)
   1044{
   1045	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   1046	struct htab_elem *l_new = NULL, *l_old;
   1047	struct hlist_nulls_head *head;
   1048	unsigned long flags;
   1049	struct bucket *b;
   1050	u32 key_size, hash;
   1051	int ret;
   1052
   1053	if (unlikely((map_flags & ~BPF_F_LOCK) > BPF_EXIST))
   1054		/* unknown flags */
   1055		return -EINVAL;
   1056
   1057	WARN_ON_ONCE(!rcu_read_lock_held() && !rcu_read_lock_trace_held() &&
   1058		     !rcu_read_lock_bh_held());
   1059
   1060	key_size = map->key_size;
   1061
   1062	hash = htab_map_hash(key, key_size, htab->hashrnd);
   1063
   1064	b = __select_bucket(htab, hash);
   1065	head = &b->head;
   1066
   1067	if (unlikely(map_flags & BPF_F_LOCK)) {
   1068		if (unlikely(!map_value_has_spin_lock(map)))
   1069			return -EINVAL;
   1070		/* find an element without taking the bucket lock */
   1071		l_old = lookup_nulls_elem_raw(head, hash, key, key_size,
   1072					      htab->n_buckets);
   1073		ret = check_flags(htab, l_old, map_flags);
   1074		if (ret)
   1075			return ret;
   1076		if (l_old) {
   1077			/* grab the element lock and update value in place */
   1078			copy_map_value_locked(map,
   1079					      l_old->key + round_up(key_size, 8),
   1080					      value, false);
   1081			return 0;
   1082		}
   1083		/* fall through, grab the bucket lock and lookup again.
   1084		 * 99.9% chance that the element won't be found,
   1085		 * but second lookup under lock has to be done.
   1086		 */
   1087	}
   1088
   1089	ret = htab_lock_bucket(htab, b, hash, &flags);
   1090	if (ret)
   1091		return ret;
   1092
   1093	l_old = lookup_elem_raw(head, hash, key, key_size);
   1094
   1095	ret = check_flags(htab, l_old, map_flags);
   1096	if (ret)
   1097		goto err;
   1098
   1099	if (unlikely(l_old && (map_flags & BPF_F_LOCK))) {
   1100		/* first lookup without the bucket lock didn't find the element,
   1101		 * but second lookup with the bucket lock found it.
   1102		 * This case is highly unlikely, but has to be dealt with:
   1103		 * grab the element lock in addition to the bucket lock
   1104		 * and update element in place
   1105		 */
   1106		copy_map_value_locked(map,
   1107				      l_old->key + round_up(key_size, 8),
   1108				      value, false);
   1109		ret = 0;
   1110		goto err;
   1111	}
   1112
   1113	l_new = alloc_htab_elem(htab, key, value, key_size, hash, false, false,
   1114				l_old);
   1115	if (IS_ERR(l_new)) {
   1116		/* all pre-allocated elements are in use or memory exhausted */
   1117		ret = PTR_ERR(l_new);
   1118		goto err;
   1119	}
   1120
   1121	/* add new element to the head of the list, so that
   1122	 * concurrent search will find it before old elem
   1123	 */
   1124	hlist_nulls_add_head_rcu(&l_new->hash_node, head);
   1125	if (l_old) {
   1126		hlist_nulls_del_rcu(&l_old->hash_node);
   1127		if (!htab_is_prealloc(htab))
   1128			free_htab_elem(htab, l_old);
   1129		else
   1130			check_and_free_fields(htab, l_old);
   1131	}
   1132	ret = 0;
   1133err:
   1134	htab_unlock_bucket(htab, b, hash, flags);
   1135	return ret;
   1136}
   1137
   1138static void htab_lru_push_free(struct bpf_htab *htab, struct htab_elem *elem)
   1139{
   1140	check_and_free_fields(htab, elem);
   1141	bpf_lru_push_free(&htab->lru, &elem->lru_node);
   1142}
   1143
   1144static int htab_lru_map_update_elem(struct bpf_map *map, void *key, void *value,
   1145				    u64 map_flags)
   1146{
   1147	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   1148	struct htab_elem *l_new, *l_old = NULL;
   1149	struct hlist_nulls_head *head;
   1150	unsigned long flags;
   1151	struct bucket *b;
   1152	u32 key_size, hash;
   1153	int ret;
   1154
   1155	if (unlikely(map_flags > BPF_EXIST))
   1156		/* unknown flags */
   1157		return -EINVAL;
   1158
   1159	WARN_ON_ONCE(!rcu_read_lock_held() && !rcu_read_lock_trace_held() &&
   1160		     !rcu_read_lock_bh_held());
   1161
   1162	key_size = map->key_size;
   1163
   1164	hash = htab_map_hash(key, key_size, htab->hashrnd);
   1165
   1166	b = __select_bucket(htab, hash);
   1167	head = &b->head;
   1168
   1169	/* For LRU, we need to alloc before taking bucket's
   1170	 * spinlock because getting free nodes from LRU may need
   1171	 * to remove older elements from htab and this removal
   1172	 * operation will need a bucket lock.
   1173	 */
   1174	l_new = prealloc_lru_pop(htab, key, hash);
   1175	if (!l_new)
   1176		return -ENOMEM;
   1177	copy_map_value(&htab->map,
   1178		       l_new->key + round_up(map->key_size, 8), value);
   1179
   1180	ret = htab_lock_bucket(htab, b, hash, &flags);
   1181	if (ret)
   1182		return ret;
   1183
   1184	l_old = lookup_elem_raw(head, hash, key, key_size);
   1185
   1186	ret = check_flags(htab, l_old, map_flags);
   1187	if (ret)
   1188		goto err;
   1189
   1190	/* add new element to the head of the list, so that
   1191	 * concurrent search will find it before old elem
   1192	 */
   1193	hlist_nulls_add_head_rcu(&l_new->hash_node, head);
   1194	if (l_old) {
   1195		bpf_lru_node_set_ref(&l_new->lru_node);
   1196		hlist_nulls_del_rcu(&l_old->hash_node);
   1197	}
   1198	ret = 0;
   1199
   1200err:
   1201	htab_unlock_bucket(htab, b, hash, flags);
   1202
   1203	if (ret)
   1204		htab_lru_push_free(htab, l_new);
   1205	else if (l_old)
   1206		htab_lru_push_free(htab, l_old);
   1207
   1208	return ret;
   1209}
   1210
   1211static int __htab_percpu_map_update_elem(struct bpf_map *map, void *key,
   1212					 void *value, u64 map_flags,
   1213					 bool onallcpus)
   1214{
   1215	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   1216	struct htab_elem *l_new = NULL, *l_old;
   1217	struct hlist_nulls_head *head;
   1218	unsigned long flags;
   1219	struct bucket *b;
   1220	u32 key_size, hash;
   1221	int ret;
   1222
   1223	if (unlikely(map_flags > BPF_EXIST))
   1224		/* unknown flags */
   1225		return -EINVAL;
   1226
   1227	WARN_ON_ONCE(!rcu_read_lock_held() && !rcu_read_lock_trace_held() &&
   1228		     !rcu_read_lock_bh_held());
   1229
   1230	key_size = map->key_size;
   1231
   1232	hash = htab_map_hash(key, key_size, htab->hashrnd);
   1233
   1234	b = __select_bucket(htab, hash);
   1235	head = &b->head;
   1236
   1237	ret = htab_lock_bucket(htab, b, hash, &flags);
   1238	if (ret)
   1239		return ret;
   1240
   1241	l_old = lookup_elem_raw(head, hash, key, key_size);
   1242
   1243	ret = check_flags(htab, l_old, map_flags);
   1244	if (ret)
   1245		goto err;
   1246
   1247	if (l_old) {
   1248		/* per-cpu hash map can update value in-place */
   1249		pcpu_copy_value(htab, htab_elem_get_ptr(l_old, key_size),
   1250				value, onallcpus);
   1251	} else {
   1252		l_new = alloc_htab_elem(htab, key, value, key_size,
   1253					hash, true, onallcpus, NULL);
   1254		if (IS_ERR(l_new)) {
   1255			ret = PTR_ERR(l_new);
   1256			goto err;
   1257		}
   1258		hlist_nulls_add_head_rcu(&l_new->hash_node, head);
   1259	}
   1260	ret = 0;
   1261err:
   1262	htab_unlock_bucket(htab, b, hash, flags);
   1263	return ret;
   1264}
   1265
   1266static int __htab_lru_percpu_map_update_elem(struct bpf_map *map, void *key,
   1267					     void *value, u64 map_flags,
   1268					     bool onallcpus)
   1269{
   1270	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   1271	struct htab_elem *l_new = NULL, *l_old;
   1272	struct hlist_nulls_head *head;
   1273	unsigned long flags;
   1274	struct bucket *b;
   1275	u32 key_size, hash;
   1276	int ret;
   1277
   1278	if (unlikely(map_flags > BPF_EXIST))
   1279		/* unknown flags */
   1280		return -EINVAL;
   1281
   1282	WARN_ON_ONCE(!rcu_read_lock_held() && !rcu_read_lock_trace_held() &&
   1283		     !rcu_read_lock_bh_held());
   1284
   1285	key_size = map->key_size;
   1286
   1287	hash = htab_map_hash(key, key_size, htab->hashrnd);
   1288
   1289	b = __select_bucket(htab, hash);
   1290	head = &b->head;
   1291
   1292	/* For LRU, we need to alloc before taking bucket's
   1293	 * spinlock because LRU's elem alloc may need
   1294	 * to remove older elem from htab and this removal
   1295	 * operation will need a bucket lock.
   1296	 */
   1297	if (map_flags != BPF_EXIST) {
   1298		l_new = prealloc_lru_pop(htab, key, hash);
   1299		if (!l_new)
   1300			return -ENOMEM;
   1301	}
   1302
   1303	ret = htab_lock_bucket(htab, b, hash, &flags);
   1304	if (ret)
   1305		return ret;
   1306
   1307	l_old = lookup_elem_raw(head, hash, key, key_size);
   1308
   1309	ret = check_flags(htab, l_old, map_flags);
   1310	if (ret)
   1311		goto err;
   1312
   1313	if (l_old) {
   1314		bpf_lru_node_set_ref(&l_old->lru_node);
   1315
   1316		/* per-cpu hash map can update value in-place */
   1317		pcpu_copy_value(htab, htab_elem_get_ptr(l_old, key_size),
   1318				value, onallcpus);
   1319	} else {
   1320		pcpu_init_value(htab, htab_elem_get_ptr(l_new, key_size),
   1321				value, onallcpus);
   1322		hlist_nulls_add_head_rcu(&l_new->hash_node, head);
   1323		l_new = NULL;
   1324	}
   1325	ret = 0;
   1326err:
   1327	htab_unlock_bucket(htab, b, hash, flags);
   1328	if (l_new)
   1329		bpf_lru_push_free(&htab->lru, &l_new->lru_node);
   1330	return ret;
   1331}
   1332
   1333static int htab_percpu_map_update_elem(struct bpf_map *map, void *key,
   1334				       void *value, u64 map_flags)
   1335{
   1336	return __htab_percpu_map_update_elem(map, key, value, map_flags, false);
   1337}
   1338
   1339static int htab_lru_percpu_map_update_elem(struct bpf_map *map, void *key,
   1340					   void *value, u64 map_flags)
   1341{
   1342	return __htab_lru_percpu_map_update_elem(map, key, value, map_flags,
   1343						 false);
   1344}
   1345
   1346/* Called from syscall or from eBPF program */
   1347static int htab_map_delete_elem(struct bpf_map *map, void *key)
   1348{
   1349	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   1350	struct hlist_nulls_head *head;
   1351	struct bucket *b;
   1352	struct htab_elem *l;
   1353	unsigned long flags;
   1354	u32 hash, key_size;
   1355	int ret;
   1356
   1357	WARN_ON_ONCE(!rcu_read_lock_held() && !rcu_read_lock_trace_held() &&
   1358		     !rcu_read_lock_bh_held());
   1359
   1360	key_size = map->key_size;
   1361
   1362	hash = htab_map_hash(key, key_size, htab->hashrnd);
   1363	b = __select_bucket(htab, hash);
   1364	head = &b->head;
   1365
   1366	ret = htab_lock_bucket(htab, b, hash, &flags);
   1367	if (ret)
   1368		return ret;
   1369
   1370	l = lookup_elem_raw(head, hash, key, key_size);
   1371
   1372	if (l) {
   1373		hlist_nulls_del_rcu(&l->hash_node);
   1374		free_htab_elem(htab, l);
   1375	} else {
   1376		ret = -ENOENT;
   1377	}
   1378
   1379	htab_unlock_bucket(htab, b, hash, flags);
   1380	return ret;
   1381}
   1382
   1383static int htab_lru_map_delete_elem(struct bpf_map *map, void *key)
   1384{
   1385	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   1386	struct hlist_nulls_head *head;
   1387	struct bucket *b;
   1388	struct htab_elem *l;
   1389	unsigned long flags;
   1390	u32 hash, key_size;
   1391	int ret;
   1392
   1393	WARN_ON_ONCE(!rcu_read_lock_held() && !rcu_read_lock_trace_held() &&
   1394		     !rcu_read_lock_bh_held());
   1395
   1396	key_size = map->key_size;
   1397
   1398	hash = htab_map_hash(key, key_size, htab->hashrnd);
   1399	b = __select_bucket(htab, hash);
   1400	head = &b->head;
   1401
   1402	ret = htab_lock_bucket(htab, b, hash, &flags);
   1403	if (ret)
   1404		return ret;
   1405
   1406	l = lookup_elem_raw(head, hash, key, key_size);
   1407
   1408	if (l)
   1409		hlist_nulls_del_rcu(&l->hash_node);
   1410	else
   1411		ret = -ENOENT;
   1412
   1413	htab_unlock_bucket(htab, b, hash, flags);
   1414	if (l)
   1415		htab_lru_push_free(htab, l);
   1416	return ret;
   1417}
   1418
   1419static void delete_all_elements(struct bpf_htab *htab)
   1420{
   1421	int i;
   1422
   1423	for (i = 0; i < htab->n_buckets; i++) {
   1424		struct hlist_nulls_head *head = select_bucket(htab, i);
   1425		struct hlist_nulls_node *n;
   1426		struct htab_elem *l;
   1427
   1428		hlist_nulls_for_each_entry_safe(l, n, head, hash_node) {
   1429			hlist_nulls_del_rcu(&l->hash_node);
   1430			htab_elem_free(htab, l);
   1431		}
   1432	}
   1433}
   1434
   1435static void htab_free_malloced_timers(struct bpf_htab *htab)
   1436{
   1437	int i;
   1438
   1439	rcu_read_lock();
   1440	for (i = 0; i < htab->n_buckets; i++) {
   1441		struct hlist_nulls_head *head = select_bucket(htab, i);
   1442		struct hlist_nulls_node *n;
   1443		struct htab_elem *l;
   1444
   1445		hlist_nulls_for_each_entry(l, n, head, hash_node) {
   1446			/* We don't reset or free kptr on uref dropping to zero,
   1447			 * hence just free timer.
   1448			 */
   1449			bpf_timer_cancel_and_free(l->key +
   1450						  round_up(htab->map.key_size, 8) +
   1451						  htab->map.timer_off);
   1452		}
   1453		cond_resched_rcu();
   1454	}
   1455	rcu_read_unlock();
   1456}
   1457
   1458static void htab_map_free_timers(struct bpf_map *map)
   1459{
   1460	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   1461
   1462	/* We don't reset or free kptr on uref dropping to zero. */
   1463	if (!map_value_has_timer(&htab->map))
   1464		return;
   1465	if (!htab_is_prealloc(htab))
   1466		htab_free_malloced_timers(htab);
   1467	else
   1468		htab_free_prealloced_timers(htab);
   1469}
   1470
   1471/* Called when map->refcnt goes to zero, either from workqueue or from syscall */
   1472static void htab_map_free(struct bpf_map *map)
   1473{
   1474	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   1475	int i;
   1476
   1477	/* bpf_free_used_maps() or close(map_fd) will trigger this map_free callback.
   1478	 * bpf_free_used_maps() is called after bpf prog is no longer executing.
   1479	 * There is no need to synchronize_rcu() here to protect map elements.
   1480	 */
   1481
   1482	/* some of free_htab_elem() callbacks for elements of this map may
   1483	 * not have executed. Wait for them.
   1484	 */
   1485	rcu_barrier();
   1486	if (!htab_is_prealloc(htab)) {
   1487		delete_all_elements(htab);
   1488	} else {
   1489		htab_free_prealloced_kptrs(htab);
   1490		prealloc_destroy(htab);
   1491	}
   1492
   1493	bpf_map_free_kptr_off_tab(map);
   1494	free_percpu(htab->extra_elems);
   1495	bpf_map_area_free(htab->buckets);
   1496	for (i = 0; i < HASHTAB_MAP_LOCK_COUNT; i++)
   1497		free_percpu(htab->map_locked[i]);
   1498	lockdep_unregister_key(&htab->lockdep_key);
   1499	kfree(htab);
   1500}
   1501
   1502static void htab_map_seq_show_elem(struct bpf_map *map, void *key,
   1503				   struct seq_file *m)
   1504{
   1505	void *value;
   1506
   1507	rcu_read_lock();
   1508
   1509	value = htab_map_lookup_elem(map, key);
   1510	if (!value) {
   1511		rcu_read_unlock();
   1512		return;
   1513	}
   1514
   1515	btf_type_seq_show(map->btf, map->btf_key_type_id, key, m);
   1516	seq_puts(m, ": ");
   1517	btf_type_seq_show(map->btf, map->btf_value_type_id, value, m);
   1518	seq_puts(m, "\n");
   1519
   1520	rcu_read_unlock();
   1521}
   1522
   1523static int __htab_map_lookup_and_delete_elem(struct bpf_map *map, void *key,
   1524					     void *value, bool is_lru_map,
   1525					     bool is_percpu, u64 flags)
   1526{
   1527	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   1528	struct hlist_nulls_head *head;
   1529	unsigned long bflags;
   1530	struct htab_elem *l;
   1531	u32 hash, key_size;
   1532	struct bucket *b;
   1533	int ret;
   1534
   1535	key_size = map->key_size;
   1536
   1537	hash = htab_map_hash(key, key_size, htab->hashrnd);
   1538	b = __select_bucket(htab, hash);
   1539	head = &b->head;
   1540
   1541	ret = htab_lock_bucket(htab, b, hash, &bflags);
   1542	if (ret)
   1543		return ret;
   1544
   1545	l = lookup_elem_raw(head, hash, key, key_size);
   1546	if (!l) {
   1547		ret = -ENOENT;
   1548	} else {
   1549		if (is_percpu) {
   1550			u32 roundup_value_size = round_up(map->value_size, 8);
   1551			void __percpu *pptr;
   1552			int off = 0, cpu;
   1553
   1554			pptr = htab_elem_get_ptr(l, key_size);
   1555			for_each_possible_cpu(cpu) {
   1556				bpf_long_memcpy(value + off,
   1557						per_cpu_ptr(pptr, cpu),
   1558						roundup_value_size);
   1559				off += roundup_value_size;
   1560			}
   1561		} else {
   1562			u32 roundup_key_size = round_up(map->key_size, 8);
   1563
   1564			if (flags & BPF_F_LOCK)
   1565				copy_map_value_locked(map, value, l->key +
   1566						      roundup_key_size,
   1567						      true);
   1568			else
   1569				copy_map_value(map, value, l->key +
   1570					       roundup_key_size);
   1571			check_and_init_map_value(map, value);
   1572		}
   1573
   1574		hlist_nulls_del_rcu(&l->hash_node);
   1575		if (!is_lru_map)
   1576			free_htab_elem(htab, l);
   1577	}
   1578
   1579	htab_unlock_bucket(htab, b, hash, bflags);
   1580
   1581	if (is_lru_map && l)
   1582		htab_lru_push_free(htab, l);
   1583
   1584	return ret;
   1585}
   1586
   1587static int htab_map_lookup_and_delete_elem(struct bpf_map *map, void *key,
   1588					   void *value, u64 flags)
   1589{
   1590	return __htab_map_lookup_and_delete_elem(map, key, value, false, false,
   1591						 flags);
   1592}
   1593
   1594static int htab_percpu_map_lookup_and_delete_elem(struct bpf_map *map,
   1595						  void *key, void *value,
   1596						  u64 flags)
   1597{
   1598	return __htab_map_lookup_and_delete_elem(map, key, value, false, true,
   1599						 flags);
   1600}
   1601
   1602static int htab_lru_map_lookup_and_delete_elem(struct bpf_map *map, void *key,
   1603					       void *value, u64 flags)
   1604{
   1605	return __htab_map_lookup_and_delete_elem(map, key, value, true, false,
   1606						 flags);
   1607}
   1608
   1609static int htab_lru_percpu_map_lookup_and_delete_elem(struct bpf_map *map,
   1610						      void *key, void *value,
   1611						      u64 flags)
   1612{
   1613	return __htab_map_lookup_and_delete_elem(map, key, value, true, true,
   1614						 flags);
   1615}
   1616
   1617static int
   1618__htab_map_lookup_and_delete_batch(struct bpf_map *map,
   1619				   const union bpf_attr *attr,
   1620				   union bpf_attr __user *uattr,
   1621				   bool do_delete, bool is_lru_map,
   1622				   bool is_percpu)
   1623{
   1624	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   1625	u32 bucket_cnt, total, key_size, value_size, roundup_key_size;
   1626	void *keys = NULL, *values = NULL, *value, *dst_key, *dst_val;
   1627	void __user *uvalues = u64_to_user_ptr(attr->batch.values);
   1628	void __user *ukeys = u64_to_user_ptr(attr->batch.keys);
   1629	void __user *ubatch = u64_to_user_ptr(attr->batch.in_batch);
   1630	u32 batch, max_count, size, bucket_size, map_id;
   1631	struct htab_elem *node_to_free = NULL;
   1632	u64 elem_map_flags, map_flags;
   1633	struct hlist_nulls_head *head;
   1634	struct hlist_nulls_node *n;
   1635	unsigned long flags = 0;
   1636	bool locked = false;
   1637	struct htab_elem *l;
   1638	struct bucket *b;
   1639	int ret = 0;
   1640
   1641	elem_map_flags = attr->batch.elem_flags;
   1642	if ((elem_map_flags & ~BPF_F_LOCK) ||
   1643	    ((elem_map_flags & BPF_F_LOCK) && !map_value_has_spin_lock(map)))
   1644		return -EINVAL;
   1645
   1646	map_flags = attr->batch.flags;
   1647	if (map_flags)
   1648		return -EINVAL;
   1649
   1650	max_count = attr->batch.count;
   1651	if (!max_count)
   1652		return 0;
   1653
   1654	if (put_user(0, &uattr->batch.count))
   1655		return -EFAULT;
   1656
   1657	batch = 0;
   1658	if (ubatch && copy_from_user(&batch, ubatch, sizeof(batch)))
   1659		return -EFAULT;
   1660
   1661	if (batch >= htab->n_buckets)
   1662		return -ENOENT;
   1663
   1664	key_size = htab->map.key_size;
   1665	roundup_key_size = round_up(htab->map.key_size, 8);
   1666	value_size = htab->map.value_size;
   1667	size = round_up(value_size, 8);
   1668	if (is_percpu)
   1669		value_size = size * num_possible_cpus();
   1670	total = 0;
   1671	/* while experimenting with hash tables with sizes ranging from 10 to
   1672	 * 1000, it was observed that a bucket can have up to 5 entries.
   1673	 */
   1674	bucket_size = 5;
   1675
   1676alloc:
   1677	/* We cannot do copy_from_user or copy_to_user inside
   1678	 * the rcu_read_lock. Allocate enough space here.
   1679	 */
   1680	keys = kvmalloc_array(key_size, bucket_size, GFP_USER | __GFP_NOWARN);
   1681	values = kvmalloc_array(value_size, bucket_size, GFP_USER | __GFP_NOWARN);
   1682	if (!keys || !values) {
   1683		ret = -ENOMEM;
   1684		goto after_loop;
   1685	}
   1686
   1687again:
   1688	bpf_disable_instrumentation();
   1689	rcu_read_lock();
   1690again_nocopy:
   1691	dst_key = keys;
   1692	dst_val = values;
   1693	b = &htab->buckets[batch];
   1694	head = &b->head;
   1695	/* do not grab the lock unless need it (bucket_cnt > 0). */
   1696	if (locked) {
   1697		ret = htab_lock_bucket(htab, b, batch, &flags);
   1698		if (ret)
   1699			goto next_batch;
   1700	}
   1701
   1702	bucket_cnt = 0;
   1703	hlist_nulls_for_each_entry_rcu(l, n, head, hash_node)
   1704		bucket_cnt++;
   1705
   1706	if (bucket_cnt && !locked) {
   1707		locked = true;
   1708		goto again_nocopy;
   1709	}
   1710
   1711	if (bucket_cnt > (max_count - total)) {
   1712		if (total == 0)
   1713			ret = -ENOSPC;
   1714		/* Note that since bucket_cnt > 0 here, it is implicit
   1715		 * that the locked was grabbed, so release it.
   1716		 */
   1717		htab_unlock_bucket(htab, b, batch, flags);
   1718		rcu_read_unlock();
   1719		bpf_enable_instrumentation();
   1720		goto after_loop;
   1721	}
   1722
   1723	if (bucket_cnt > bucket_size) {
   1724		bucket_size = bucket_cnt;
   1725		/* Note that since bucket_cnt > 0 here, it is implicit
   1726		 * that the locked was grabbed, so release it.
   1727		 */
   1728		htab_unlock_bucket(htab, b, batch, flags);
   1729		rcu_read_unlock();
   1730		bpf_enable_instrumentation();
   1731		kvfree(keys);
   1732		kvfree(values);
   1733		goto alloc;
   1734	}
   1735
   1736	/* Next block is only safe to run if you have grabbed the lock */
   1737	if (!locked)
   1738		goto next_batch;
   1739
   1740	hlist_nulls_for_each_entry_safe(l, n, head, hash_node) {
   1741		memcpy(dst_key, l->key, key_size);
   1742
   1743		if (is_percpu) {
   1744			int off = 0, cpu;
   1745			void __percpu *pptr;
   1746
   1747			pptr = htab_elem_get_ptr(l, map->key_size);
   1748			for_each_possible_cpu(cpu) {
   1749				bpf_long_memcpy(dst_val + off,
   1750						per_cpu_ptr(pptr, cpu), size);
   1751				off += size;
   1752			}
   1753		} else {
   1754			value = l->key + roundup_key_size;
   1755			if (map->map_type == BPF_MAP_TYPE_HASH_OF_MAPS) {
   1756				struct bpf_map **inner_map = value;
   1757
   1758				 /* Actual value is the id of the inner map */
   1759				map_id = map->ops->map_fd_sys_lookup_elem(*inner_map);
   1760				value = &map_id;
   1761			}
   1762
   1763			if (elem_map_flags & BPF_F_LOCK)
   1764				copy_map_value_locked(map, dst_val, value,
   1765						      true);
   1766			else
   1767				copy_map_value(map, dst_val, value);
   1768			check_and_init_map_value(map, dst_val);
   1769		}
   1770		if (do_delete) {
   1771			hlist_nulls_del_rcu(&l->hash_node);
   1772
   1773			/* bpf_lru_push_free() will acquire lru_lock, which
   1774			 * may cause deadlock. See comments in function
   1775			 * prealloc_lru_pop(). Let us do bpf_lru_push_free()
   1776			 * after releasing the bucket lock.
   1777			 */
   1778			if (is_lru_map) {
   1779				l->batch_flink = node_to_free;
   1780				node_to_free = l;
   1781			} else {
   1782				free_htab_elem(htab, l);
   1783			}
   1784		}
   1785		dst_key += key_size;
   1786		dst_val += value_size;
   1787	}
   1788
   1789	htab_unlock_bucket(htab, b, batch, flags);
   1790	locked = false;
   1791
   1792	while (node_to_free) {
   1793		l = node_to_free;
   1794		node_to_free = node_to_free->batch_flink;
   1795		htab_lru_push_free(htab, l);
   1796	}
   1797
   1798next_batch:
   1799	/* If we are not copying data, we can go to next bucket and avoid
   1800	 * unlocking the rcu.
   1801	 */
   1802	if (!bucket_cnt && (batch + 1 < htab->n_buckets)) {
   1803		batch++;
   1804		goto again_nocopy;
   1805	}
   1806
   1807	rcu_read_unlock();
   1808	bpf_enable_instrumentation();
   1809	if (bucket_cnt && (copy_to_user(ukeys + total * key_size, keys,
   1810	    key_size * bucket_cnt) ||
   1811	    copy_to_user(uvalues + total * value_size, values,
   1812	    value_size * bucket_cnt))) {
   1813		ret = -EFAULT;
   1814		goto after_loop;
   1815	}
   1816
   1817	total += bucket_cnt;
   1818	batch++;
   1819	if (batch >= htab->n_buckets) {
   1820		ret = -ENOENT;
   1821		goto after_loop;
   1822	}
   1823	goto again;
   1824
   1825after_loop:
   1826	if (ret == -EFAULT)
   1827		goto out;
   1828
   1829	/* copy # of entries and next batch */
   1830	ubatch = u64_to_user_ptr(attr->batch.out_batch);
   1831	if (copy_to_user(ubatch, &batch, sizeof(batch)) ||
   1832	    put_user(total, &uattr->batch.count))
   1833		ret = -EFAULT;
   1834
   1835out:
   1836	kvfree(keys);
   1837	kvfree(values);
   1838	return ret;
   1839}
   1840
   1841static int
   1842htab_percpu_map_lookup_batch(struct bpf_map *map, const union bpf_attr *attr,
   1843			     union bpf_attr __user *uattr)
   1844{
   1845	return __htab_map_lookup_and_delete_batch(map, attr, uattr, false,
   1846						  false, true);
   1847}
   1848
   1849static int
   1850htab_percpu_map_lookup_and_delete_batch(struct bpf_map *map,
   1851					const union bpf_attr *attr,
   1852					union bpf_attr __user *uattr)
   1853{
   1854	return __htab_map_lookup_and_delete_batch(map, attr, uattr, true,
   1855						  false, true);
   1856}
   1857
   1858static int
   1859htab_map_lookup_batch(struct bpf_map *map, const union bpf_attr *attr,
   1860		      union bpf_attr __user *uattr)
   1861{
   1862	return __htab_map_lookup_and_delete_batch(map, attr, uattr, false,
   1863						  false, false);
   1864}
   1865
   1866static int
   1867htab_map_lookup_and_delete_batch(struct bpf_map *map,
   1868				 const union bpf_attr *attr,
   1869				 union bpf_attr __user *uattr)
   1870{
   1871	return __htab_map_lookup_and_delete_batch(map, attr, uattr, true,
   1872						  false, false);
   1873}
   1874
   1875static int
   1876htab_lru_percpu_map_lookup_batch(struct bpf_map *map,
   1877				 const union bpf_attr *attr,
   1878				 union bpf_attr __user *uattr)
   1879{
   1880	return __htab_map_lookup_and_delete_batch(map, attr, uattr, false,
   1881						  true, true);
   1882}
   1883
   1884static int
   1885htab_lru_percpu_map_lookup_and_delete_batch(struct bpf_map *map,
   1886					    const union bpf_attr *attr,
   1887					    union bpf_attr __user *uattr)
   1888{
   1889	return __htab_map_lookup_and_delete_batch(map, attr, uattr, true,
   1890						  true, true);
   1891}
   1892
   1893static int
   1894htab_lru_map_lookup_batch(struct bpf_map *map, const union bpf_attr *attr,
   1895			  union bpf_attr __user *uattr)
   1896{
   1897	return __htab_map_lookup_and_delete_batch(map, attr, uattr, false,
   1898						  true, false);
   1899}
   1900
   1901static int
   1902htab_lru_map_lookup_and_delete_batch(struct bpf_map *map,
   1903				     const union bpf_attr *attr,
   1904				     union bpf_attr __user *uattr)
   1905{
   1906	return __htab_map_lookup_and_delete_batch(map, attr, uattr, true,
   1907						  true, false);
   1908}
   1909
   1910struct bpf_iter_seq_hash_map_info {
   1911	struct bpf_map *map;
   1912	struct bpf_htab *htab;
   1913	void *percpu_value_buf; // non-zero means percpu hash
   1914	u32 bucket_id;
   1915	u32 skip_elems;
   1916};
   1917
   1918static struct htab_elem *
   1919bpf_hash_map_seq_find_next(struct bpf_iter_seq_hash_map_info *info,
   1920			   struct htab_elem *prev_elem)
   1921{
   1922	const struct bpf_htab *htab = info->htab;
   1923	u32 skip_elems = info->skip_elems;
   1924	u32 bucket_id = info->bucket_id;
   1925	struct hlist_nulls_head *head;
   1926	struct hlist_nulls_node *n;
   1927	struct htab_elem *elem;
   1928	struct bucket *b;
   1929	u32 i, count;
   1930
   1931	if (bucket_id >= htab->n_buckets)
   1932		return NULL;
   1933
   1934	/* try to find next elem in the same bucket */
   1935	if (prev_elem) {
   1936		/* no update/deletion on this bucket, prev_elem should be still valid
   1937		 * and we won't skip elements.
   1938		 */
   1939		n = rcu_dereference_raw(hlist_nulls_next_rcu(&prev_elem->hash_node));
   1940		elem = hlist_nulls_entry_safe(n, struct htab_elem, hash_node);
   1941		if (elem)
   1942			return elem;
   1943
   1944		/* not found, unlock and go to the next bucket */
   1945		b = &htab->buckets[bucket_id++];
   1946		rcu_read_unlock();
   1947		skip_elems = 0;
   1948	}
   1949
   1950	for (i = bucket_id; i < htab->n_buckets; i++) {
   1951		b = &htab->buckets[i];
   1952		rcu_read_lock();
   1953
   1954		count = 0;
   1955		head = &b->head;
   1956		hlist_nulls_for_each_entry_rcu(elem, n, head, hash_node) {
   1957			if (count >= skip_elems) {
   1958				info->bucket_id = i;
   1959				info->skip_elems = count;
   1960				return elem;
   1961			}
   1962			count++;
   1963		}
   1964
   1965		rcu_read_unlock();
   1966		skip_elems = 0;
   1967	}
   1968
   1969	info->bucket_id = i;
   1970	info->skip_elems = 0;
   1971	return NULL;
   1972}
   1973
   1974static void *bpf_hash_map_seq_start(struct seq_file *seq, loff_t *pos)
   1975{
   1976	struct bpf_iter_seq_hash_map_info *info = seq->private;
   1977	struct htab_elem *elem;
   1978
   1979	elem = bpf_hash_map_seq_find_next(info, NULL);
   1980	if (!elem)
   1981		return NULL;
   1982
   1983	if (*pos == 0)
   1984		++*pos;
   1985	return elem;
   1986}
   1987
   1988static void *bpf_hash_map_seq_next(struct seq_file *seq, void *v, loff_t *pos)
   1989{
   1990	struct bpf_iter_seq_hash_map_info *info = seq->private;
   1991
   1992	++*pos;
   1993	++info->skip_elems;
   1994	return bpf_hash_map_seq_find_next(info, v);
   1995}
   1996
   1997static int __bpf_hash_map_seq_show(struct seq_file *seq, struct htab_elem *elem)
   1998{
   1999	struct bpf_iter_seq_hash_map_info *info = seq->private;
   2000	u32 roundup_key_size, roundup_value_size;
   2001	struct bpf_iter__bpf_map_elem ctx = {};
   2002	struct bpf_map *map = info->map;
   2003	struct bpf_iter_meta meta;
   2004	int ret = 0, off = 0, cpu;
   2005	struct bpf_prog *prog;
   2006	void __percpu *pptr;
   2007
   2008	meta.seq = seq;
   2009	prog = bpf_iter_get_info(&meta, elem == NULL);
   2010	if (prog) {
   2011		ctx.meta = &meta;
   2012		ctx.map = info->map;
   2013		if (elem) {
   2014			roundup_key_size = round_up(map->key_size, 8);
   2015			ctx.key = elem->key;
   2016			if (!info->percpu_value_buf) {
   2017				ctx.value = elem->key + roundup_key_size;
   2018			} else {
   2019				roundup_value_size = round_up(map->value_size, 8);
   2020				pptr = htab_elem_get_ptr(elem, map->key_size);
   2021				for_each_possible_cpu(cpu) {
   2022					bpf_long_memcpy(info->percpu_value_buf + off,
   2023							per_cpu_ptr(pptr, cpu),
   2024							roundup_value_size);
   2025					off += roundup_value_size;
   2026				}
   2027				ctx.value = info->percpu_value_buf;
   2028			}
   2029		}
   2030		ret = bpf_iter_run_prog(prog, &ctx);
   2031	}
   2032
   2033	return ret;
   2034}
   2035
   2036static int bpf_hash_map_seq_show(struct seq_file *seq, void *v)
   2037{
   2038	return __bpf_hash_map_seq_show(seq, v);
   2039}
   2040
   2041static void bpf_hash_map_seq_stop(struct seq_file *seq, void *v)
   2042{
   2043	if (!v)
   2044		(void)__bpf_hash_map_seq_show(seq, NULL);
   2045	else
   2046		rcu_read_unlock();
   2047}
   2048
   2049static int bpf_iter_init_hash_map(void *priv_data,
   2050				  struct bpf_iter_aux_info *aux)
   2051{
   2052	struct bpf_iter_seq_hash_map_info *seq_info = priv_data;
   2053	struct bpf_map *map = aux->map;
   2054	void *value_buf;
   2055	u32 buf_size;
   2056
   2057	if (map->map_type == BPF_MAP_TYPE_PERCPU_HASH ||
   2058	    map->map_type == BPF_MAP_TYPE_LRU_PERCPU_HASH) {
   2059		buf_size = round_up(map->value_size, 8) * num_possible_cpus();
   2060		value_buf = kmalloc(buf_size, GFP_USER | __GFP_NOWARN);
   2061		if (!value_buf)
   2062			return -ENOMEM;
   2063
   2064		seq_info->percpu_value_buf = value_buf;
   2065	}
   2066
   2067	seq_info->map = map;
   2068	seq_info->htab = container_of(map, struct bpf_htab, map);
   2069	return 0;
   2070}
   2071
   2072static void bpf_iter_fini_hash_map(void *priv_data)
   2073{
   2074	struct bpf_iter_seq_hash_map_info *seq_info = priv_data;
   2075
   2076	kfree(seq_info->percpu_value_buf);
   2077}
   2078
   2079static const struct seq_operations bpf_hash_map_seq_ops = {
   2080	.start	= bpf_hash_map_seq_start,
   2081	.next	= bpf_hash_map_seq_next,
   2082	.stop	= bpf_hash_map_seq_stop,
   2083	.show	= bpf_hash_map_seq_show,
   2084};
   2085
   2086static const struct bpf_iter_seq_info iter_seq_info = {
   2087	.seq_ops		= &bpf_hash_map_seq_ops,
   2088	.init_seq_private	= bpf_iter_init_hash_map,
   2089	.fini_seq_private	= bpf_iter_fini_hash_map,
   2090	.seq_priv_size		= sizeof(struct bpf_iter_seq_hash_map_info),
   2091};
   2092
   2093static int bpf_for_each_hash_elem(struct bpf_map *map, bpf_callback_t callback_fn,
   2094				  void *callback_ctx, u64 flags)
   2095{
   2096	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   2097	struct hlist_nulls_head *head;
   2098	struct hlist_nulls_node *n;
   2099	struct htab_elem *elem;
   2100	u32 roundup_key_size;
   2101	int i, num_elems = 0;
   2102	void __percpu *pptr;
   2103	struct bucket *b;
   2104	void *key, *val;
   2105	bool is_percpu;
   2106	u64 ret = 0;
   2107
   2108	if (flags != 0)
   2109		return -EINVAL;
   2110
   2111	is_percpu = htab_is_percpu(htab);
   2112
   2113	roundup_key_size = round_up(map->key_size, 8);
   2114	/* disable migration so percpu value prepared here will be the
   2115	 * same as the one seen by the bpf program with bpf_map_lookup_elem().
   2116	 */
   2117	if (is_percpu)
   2118		migrate_disable();
   2119	for (i = 0; i < htab->n_buckets; i++) {
   2120		b = &htab->buckets[i];
   2121		rcu_read_lock();
   2122		head = &b->head;
   2123		hlist_nulls_for_each_entry_rcu(elem, n, head, hash_node) {
   2124			key = elem->key;
   2125			if (is_percpu) {
   2126				/* current cpu value for percpu map */
   2127				pptr = htab_elem_get_ptr(elem, map->key_size);
   2128				val = this_cpu_ptr(pptr);
   2129			} else {
   2130				val = elem->key + roundup_key_size;
   2131			}
   2132			num_elems++;
   2133			ret = callback_fn((u64)(long)map, (u64)(long)key,
   2134					  (u64)(long)val, (u64)(long)callback_ctx, 0);
   2135			/* return value: 0 - continue, 1 - stop and return */
   2136			if (ret) {
   2137				rcu_read_unlock();
   2138				goto out;
   2139			}
   2140		}
   2141		rcu_read_unlock();
   2142	}
   2143out:
   2144	if (is_percpu)
   2145		migrate_enable();
   2146	return num_elems;
   2147}
   2148
   2149BTF_ID_LIST_SINGLE(htab_map_btf_ids, struct, bpf_htab)
   2150const struct bpf_map_ops htab_map_ops = {
   2151	.map_meta_equal = bpf_map_meta_equal,
   2152	.map_alloc_check = htab_map_alloc_check,
   2153	.map_alloc = htab_map_alloc,
   2154	.map_free = htab_map_free,
   2155	.map_get_next_key = htab_map_get_next_key,
   2156	.map_release_uref = htab_map_free_timers,
   2157	.map_lookup_elem = htab_map_lookup_elem,
   2158	.map_lookup_and_delete_elem = htab_map_lookup_and_delete_elem,
   2159	.map_update_elem = htab_map_update_elem,
   2160	.map_delete_elem = htab_map_delete_elem,
   2161	.map_gen_lookup = htab_map_gen_lookup,
   2162	.map_seq_show_elem = htab_map_seq_show_elem,
   2163	.map_set_for_each_callback_args = map_set_for_each_callback_args,
   2164	.map_for_each_callback = bpf_for_each_hash_elem,
   2165	BATCH_OPS(htab),
   2166	.map_btf_id = &htab_map_btf_ids[0],
   2167	.iter_seq_info = &iter_seq_info,
   2168};
   2169
   2170const struct bpf_map_ops htab_lru_map_ops = {
   2171	.map_meta_equal = bpf_map_meta_equal,
   2172	.map_alloc_check = htab_map_alloc_check,
   2173	.map_alloc = htab_map_alloc,
   2174	.map_free = htab_map_free,
   2175	.map_get_next_key = htab_map_get_next_key,
   2176	.map_release_uref = htab_map_free_timers,
   2177	.map_lookup_elem = htab_lru_map_lookup_elem,
   2178	.map_lookup_and_delete_elem = htab_lru_map_lookup_and_delete_elem,
   2179	.map_lookup_elem_sys_only = htab_lru_map_lookup_elem_sys,
   2180	.map_update_elem = htab_lru_map_update_elem,
   2181	.map_delete_elem = htab_lru_map_delete_elem,
   2182	.map_gen_lookup = htab_lru_map_gen_lookup,
   2183	.map_seq_show_elem = htab_map_seq_show_elem,
   2184	.map_set_for_each_callback_args = map_set_for_each_callback_args,
   2185	.map_for_each_callback = bpf_for_each_hash_elem,
   2186	BATCH_OPS(htab_lru),
   2187	.map_btf_id = &htab_map_btf_ids[0],
   2188	.iter_seq_info = &iter_seq_info,
   2189};
   2190
   2191/* Called from eBPF program */
   2192static void *htab_percpu_map_lookup_elem(struct bpf_map *map, void *key)
   2193{
   2194	struct htab_elem *l = __htab_map_lookup_elem(map, key);
   2195
   2196	if (l)
   2197		return this_cpu_ptr(htab_elem_get_ptr(l, map->key_size));
   2198	else
   2199		return NULL;
   2200}
   2201
   2202static void *htab_percpu_map_lookup_percpu_elem(struct bpf_map *map, void *key, u32 cpu)
   2203{
   2204	struct htab_elem *l;
   2205
   2206	if (cpu >= nr_cpu_ids)
   2207		return NULL;
   2208
   2209	l = __htab_map_lookup_elem(map, key);
   2210	if (l)
   2211		return per_cpu_ptr(htab_elem_get_ptr(l, map->key_size), cpu);
   2212	else
   2213		return NULL;
   2214}
   2215
   2216static void *htab_lru_percpu_map_lookup_elem(struct bpf_map *map, void *key)
   2217{
   2218	struct htab_elem *l = __htab_map_lookup_elem(map, key);
   2219
   2220	if (l) {
   2221		bpf_lru_node_set_ref(&l->lru_node);
   2222		return this_cpu_ptr(htab_elem_get_ptr(l, map->key_size));
   2223	}
   2224
   2225	return NULL;
   2226}
   2227
   2228static void *htab_lru_percpu_map_lookup_percpu_elem(struct bpf_map *map, void *key, u32 cpu)
   2229{
   2230	struct htab_elem *l;
   2231
   2232	if (cpu >= nr_cpu_ids)
   2233		return NULL;
   2234
   2235	l = __htab_map_lookup_elem(map, key);
   2236	if (l) {
   2237		bpf_lru_node_set_ref(&l->lru_node);
   2238		return per_cpu_ptr(htab_elem_get_ptr(l, map->key_size), cpu);
   2239	}
   2240
   2241	return NULL;
   2242}
   2243
   2244int bpf_percpu_hash_copy(struct bpf_map *map, void *key, void *value)
   2245{
   2246	struct htab_elem *l;
   2247	void __percpu *pptr;
   2248	int ret = -ENOENT;
   2249	int cpu, off = 0;
   2250	u32 size;
   2251
   2252	/* per_cpu areas are zero-filled and bpf programs can only
   2253	 * access 'value_size' of them, so copying rounded areas
   2254	 * will not leak any kernel data
   2255	 */
   2256	size = round_up(map->value_size, 8);
   2257	rcu_read_lock();
   2258	l = __htab_map_lookup_elem(map, key);
   2259	if (!l)
   2260		goto out;
   2261	/* We do not mark LRU map element here in order to not mess up
   2262	 * eviction heuristics when user space does a map walk.
   2263	 */
   2264	pptr = htab_elem_get_ptr(l, map->key_size);
   2265	for_each_possible_cpu(cpu) {
   2266		bpf_long_memcpy(value + off,
   2267				per_cpu_ptr(pptr, cpu), size);
   2268		off += size;
   2269	}
   2270	ret = 0;
   2271out:
   2272	rcu_read_unlock();
   2273	return ret;
   2274}
   2275
   2276int bpf_percpu_hash_update(struct bpf_map *map, void *key, void *value,
   2277			   u64 map_flags)
   2278{
   2279	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   2280	int ret;
   2281
   2282	rcu_read_lock();
   2283	if (htab_is_lru(htab))
   2284		ret = __htab_lru_percpu_map_update_elem(map, key, value,
   2285							map_flags, true);
   2286	else
   2287		ret = __htab_percpu_map_update_elem(map, key, value, map_flags,
   2288						    true);
   2289	rcu_read_unlock();
   2290
   2291	return ret;
   2292}
   2293
   2294static void htab_percpu_map_seq_show_elem(struct bpf_map *map, void *key,
   2295					  struct seq_file *m)
   2296{
   2297	struct htab_elem *l;
   2298	void __percpu *pptr;
   2299	int cpu;
   2300
   2301	rcu_read_lock();
   2302
   2303	l = __htab_map_lookup_elem(map, key);
   2304	if (!l) {
   2305		rcu_read_unlock();
   2306		return;
   2307	}
   2308
   2309	btf_type_seq_show(map->btf, map->btf_key_type_id, key, m);
   2310	seq_puts(m, ": {\n");
   2311	pptr = htab_elem_get_ptr(l, map->key_size);
   2312	for_each_possible_cpu(cpu) {
   2313		seq_printf(m, "\tcpu%d: ", cpu);
   2314		btf_type_seq_show(map->btf, map->btf_value_type_id,
   2315				  per_cpu_ptr(pptr, cpu), m);
   2316		seq_puts(m, "\n");
   2317	}
   2318	seq_puts(m, "}\n");
   2319
   2320	rcu_read_unlock();
   2321}
   2322
   2323const struct bpf_map_ops htab_percpu_map_ops = {
   2324	.map_meta_equal = bpf_map_meta_equal,
   2325	.map_alloc_check = htab_map_alloc_check,
   2326	.map_alloc = htab_map_alloc,
   2327	.map_free = htab_map_free,
   2328	.map_get_next_key = htab_map_get_next_key,
   2329	.map_lookup_elem = htab_percpu_map_lookup_elem,
   2330	.map_lookup_and_delete_elem = htab_percpu_map_lookup_and_delete_elem,
   2331	.map_update_elem = htab_percpu_map_update_elem,
   2332	.map_delete_elem = htab_map_delete_elem,
   2333	.map_lookup_percpu_elem = htab_percpu_map_lookup_percpu_elem,
   2334	.map_seq_show_elem = htab_percpu_map_seq_show_elem,
   2335	.map_set_for_each_callback_args = map_set_for_each_callback_args,
   2336	.map_for_each_callback = bpf_for_each_hash_elem,
   2337	BATCH_OPS(htab_percpu),
   2338	.map_btf_id = &htab_map_btf_ids[0],
   2339	.iter_seq_info = &iter_seq_info,
   2340};
   2341
   2342const struct bpf_map_ops htab_lru_percpu_map_ops = {
   2343	.map_meta_equal = bpf_map_meta_equal,
   2344	.map_alloc_check = htab_map_alloc_check,
   2345	.map_alloc = htab_map_alloc,
   2346	.map_free = htab_map_free,
   2347	.map_get_next_key = htab_map_get_next_key,
   2348	.map_lookup_elem = htab_lru_percpu_map_lookup_elem,
   2349	.map_lookup_and_delete_elem = htab_lru_percpu_map_lookup_and_delete_elem,
   2350	.map_update_elem = htab_lru_percpu_map_update_elem,
   2351	.map_delete_elem = htab_lru_map_delete_elem,
   2352	.map_lookup_percpu_elem = htab_lru_percpu_map_lookup_percpu_elem,
   2353	.map_seq_show_elem = htab_percpu_map_seq_show_elem,
   2354	.map_set_for_each_callback_args = map_set_for_each_callback_args,
   2355	.map_for_each_callback = bpf_for_each_hash_elem,
   2356	BATCH_OPS(htab_lru_percpu),
   2357	.map_btf_id = &htab_map_btf_ids[0],
   2358	.iter_seq_info = &iter_seq_info,
   2359};
   2360
   2361static int fd_htab_map_alloc_check(union bpf_attr *attr)
   2362{
   2363	if (attr->value_size != sizeof(u32))
   2364		return -EINVAL;
   2365	return htab_map_alloc_check(attr);
   2366}
   2367
   2368static void fd_htab_map_free(struct bpf_map *map)
   2369{
   2370	struct bpf_htab *htab = container_of(map, struct bpf_htab, map);
   2371	struct hlist_nulls_node *n;
   2372	struct hlist_nulls_head *head;
   2373	struct htab_elem *l;
   2374	int i;
   2375
   2376	for (i = 0; i < htab->n_buckets; i++) {
   2377		head = select_bucket(htab, i);
   2378
   2379		hlist_nulls_for_each_entry_safe(l, n, head, hash_node) {
   2380			void *ptr = fd_htab_map_get_ptr(map, l);
   2381
   2382			map->ops->map_fd_put_ptr(ptr);
   2383		}
   2384	}
   2385
   2386	htab_map_free(map);
   2387}
   2388
   2389/* only called from syscall */
   2390int bpf_fd_htab_map_lookup_elem(struct bpf_map *map, void *key, u32 *value)
   2391{
   2392	void **ptr;
   2393	int ret = 0;
   2394
   2395	if (!map->ops->map_fd_sys_lookup_elem)
   2396		return -ENOTSUPP;
   2397
   2398	rcu_read_lock();
   2399	ptr = htab_map_lookup_elem(map, key);
   2400	if (ptr)
   2401		*value = map->ops->map_fd_sys_lookup_elem(READ_ONCE(*ptr));
   2402	else
   2403		ret = -ENOENT;
   2404	rcu_read_unlock();
   2405
   2406	return ret;
   2407}
   2408
   2409/* only called from syscall */
   2410int bpf_fd_htab_map_update_elem(struct bpf_map *map, struct file *map_file,
   2411				void *key, void *value, u64 map_flags)
   2412{
   2413	void *ptr;
   2414	int ret;
   2415	u32 ufd = *(u32 *)value;
   2416
   2417	ptr = map->ops->map_fd_get_ptr(map, map_file, ufd);
   2418	if (IS_ERR(ptr))
   2419		return PTR_ERR(ptr);
   2420
   2421	ret = htab_map_update_elem(map, key, &ptr, map_flags);
   2422	if (ret)
   2423		map->ops->map_fd_put_ptr(ptr);
   2424
   2425	return ret;
   2426}
   2427
   2428static struct bpf_map *htab_of_map_alloc(union bpf_attr *attr)
   2429{
   2430	struct bpf_map *map, *inner_map_meta;
   2431
   2432	inner_map_meta = bpf_map_meta_alloc(attr->inner_map_fd);
   2433	if (IS_ERR(inner_map_meta))
   2434		return inner_map_meta;
   2435
   2436	map = htab_map_alloc(attr);
   2437	if (IS_ERR(map)) {
   2438		bpf_map_meta_free(inner_map_meta);
   2439		return map;
   2440	}
   2441
   2442	map->inner_map_meta = inner_map_meta;
   2443
   2444	return map;
   2445}
   2446
   2447static void *htab_of_map_lookup_elem(struct bpf_map *map, void *key)
   2448{
   2449	struct bpf_map **inner_map  = htab_map_lookup_elem(map, key);
   2450
   2451	if (!inner_map)
   2452		return NULL;
   2453
   2454	return READ_ONCE(*inner_map);
   2455}
   2456
   2457static int htab_of_map_gen_lookup(struct bpf_map *map,
   2458				  struct bpf_insn *insn_buf)
   2459{
   2460	struct bpf_insn *insn = insn_buf;
   2461	const int ret = BPF_REG_0;
   2462
   2463	BUILD_BUG_ON(!__same_type(&__htab_map_lookup_elem,
   2464		     (void *(*)(struct bpf_map *map, void *key))NULL));
   2465	*insn++ = BPF_EMIT_CALL(__htab_map_lookup_elem);
   2466	*insn++ = BPF_JMP_IMM(BPF_JEQ, ret, 0, 2);
   2467	*insn++ = BPF_ALU64_IMM(BPF_ADD, ret,
   2468				offsetof(struct htab_elem, key) +
   2469				round_up(map->key_size, 8));
   2470	*insn++ = BPF_LDX_MEM(BPF_DW, ret, ret, 0);
   2471
   2472	return insn - insn_buf;
   2473}
   2474
   2475static void htab_of_map_free(struct bpf_map *map)
   2476{
   2477	bpf_map_meta_free(map->inner_map_meta);
   2478	fd_htab_map_free(map);
   2479}
   2480
   2481const struct bpf_map_ops htab_of_maps_map_ops = {
   2482	.map_alloc_check = fd_htab_map_alloc_check,
   2483	.map_alloc = htab_of_map_alloc,
   2484	.map_free = htab_of_map_free,
   2485	.map_get_next_key = htab_map_get_next_key,
   2486	.map_lookup_elem = htab_of_map_lookup_elem,
   2487	.map_delete_elem = htab_map_delete_elem,
   2488	.map_fd_get_ptr = bpf_map_fd_get_ptr,
   2489	.map_fd_put_ptr = bpf_map_fd_put_ptr,
   2490	.map_fd_sys_lookup_elem = bpf_map_fd_sys_lookup_elem,
   2491	.map_gen_lookup = htab_of_map_gen_lookup,
   2492	.map_check_btf = map_check_no_btf,
   2493	BATCH_OPS(htab),
   2494	.map_btf_id = &htab_map_btf_ids[0],
   2495};