lists.openwall.net   lists  /  announce  owl-users  owl-dev  john-users  john-dev  passwdqc-users  yescrypt  popa3d-users  /  oss-security  kernel-hardening  musl  sabotage  tlsify  passwords  /  crypt-dev  xvendor  /  Bugtraq  Full-Disclosure  linux-kernel  linux-netdev  linux-ext4  linux-hardening  linux-cve-announce  PHC 
Open Source and information security mailing list archives
 
Hash Suite: Windows password security audit tool. GUI, reports in PDF.
[<prev] [next>] [<thread-prev] [thread-next>] [day] [month] [year] [list]
Message-ID: <CAKEwX=PAo6s6L=WL1va6edcGznQcYWe8Xrtt-zBFdAo05wS-AA@mail.gmail.com>
Date: Fri, 30 Jan 2026 16:33:28 -0800
From: Nhat Pham <nphamcs@...il.com>
To: Kanchana P Sridhar <kanchana.p.sridhar@...el.com>
Cc: linux-kernel@...r.kernel.org, linux-mm@...ck.org, hannes@...xchg.org, 
	yosry.ahmed@...ux.dev, chengming.zhou@...ux.dev, usamaarif642@...il.com, 
	ryan.roberts@....com, 21cnbao@...il.com, ying.huang@...ux.alibaba.com, 
	akpm@...ux-foundation.org, senozhatsky@...omium.org, sj@...nel.org, 
	kasong@...cent.com, linux-crypto@...r.kernel.org, herbert@...dor.apana.org.au, 
	davem@...emloft.net, clabbe@...libre.com, ardb@...nel.org, 
	ebiggers@...gle.com, surenb@...gle.com, kristen.c.accardi@...el.com, 
	vinicius.gomes@...el.com, giovanni.cabiddu@...el.com, 
	wajdi.k.feghali@...el.com
Subject: Re: [PATCH v14 25/26] mm: zswap: Store large folios in batches.

On Sat, Jan 24, 2026 at 7:36 PM Kanchana P Sridhar
<kanchana.p.sridhar@...el.com> wrote:
>
> Support batching when storing large folios in zswap. If the underlying
> compressor supports batching (e.g. hardware parallel compression),
> allocate multiple compression buffers, otherwise allocate one. The
> number of buffers is bounded by a new constant, ZSWAP_MAX_BATCH_SIZE, to
> limit the memory overhead. For existing software compressors, the only
> extra overhead is the extra 'buffers' pointer, so 8 bytes per-CPU on
> x86_64.
>
> Only the first buffer is currently used, but subsequent changes will use
> the remaining buffers for hardware compression batching.
>
> Regardless of compression batching, always process large folios in
> batches. For hardware compressors, the batch size is the compressor
> batch size, otherwise ZSWAP_MAX_BATCH_SIZE is used.
>
> zswap_store_page() is replaced with zswap_store_pages(), which processes
> a batch of pages and allows for batching optimizations. For now, only
> optimize allocating entries by using batch allocations from the slab
> cache.
>
> Since batch allocations do not support specifying a node id, store the
> node id in the zswap entry instead of relying on the zswap_entry being
> allocated on the same node. The size of the zswap_entry remains
> unchanged as 'referenced' is lumped in with the 'length' (as it doesn't
> need a full unsigned int anyway).
>
> Avoid repeatedly calling mem_cgroup_zswap_writeback_enabled() for every
> page and only call it once for the folio, since the entire folio is
> charged to a single memcg.
>
> Suggested-by: Nhat Pham <nphamcs@...il.com>
> Suggested-by: Yosry Ahmed <yosry.ahmed@...ux.dev>
> Signed-off-by: Kanchana P Sridhar <kanchana.p.sridhar@...el.com>
> ---
>  mm/zswap.c | 351 +++++++++++++++++++++++++++++++++++++----------------
>  1 file changed, 248 insertions(+), 103 deletions(-)
>
> diff --git a/mm/zswap.c b/mm/zswap.c
> index 0d56390342b7..6a22add63220 100644
> --- a/mm/zswap.c
> +++ b/mm/zswap.c
> @@ -82,6 +82,11 @@ static bool zswap_pool_reached_full;
>
>  #define ZSWAP_PARAM_UNSET ""
>
> +/* Limit the batch size to limit per-CPU memory usage for dst buffers. */
> +#define ZSWAP_MAX_BATCH_SIZE 8U
> +#define ZSWAP_ENTRY_SPARE_4BYTES 32U
> +#define ZSWAP_ENTRY_REF_BIT 1U
> +
>  static int zswap_setup(void);
>
>  /* Enable/disable zswap */
> @@ -139,7 +144,7 @@ struct crypto_acomp_ctx {
>         struct crypto_acomp *acomp;
>         struct acomp_req *req;
>         struct crypto_wait wait;
> -       u8 *buffer;
> +       u8 **buffers;
>         struct mutex mutex;
>  };
>
> @@ -148,6 +153,9 @@ struct crypto_acomp_ctx {
>   * The only case where lru_lock is not acquired while holding tree.lock is
>   * when a zswap_entry is taken off the lru for writeback, in that case it
>   * needs to be verified that it's still valid in the tree.
> + *
> + * @compr_batch_size: The max batch size of the compression algorithm,
> + *                    bounded by ZSWAP_MAX_BATCH_SIZE.
>   */
>  struct zswap_pool {
>         struct zs_pool *zs_pool;
> @@ -157,6 +165,7 @@ struct zswap_pool {
>         struct work_struct release_work;
>         struct hlist_node node;
>         char tfm_name[CRYPTO_MAX_ALG_NAME];
> +       u8 compr_batch_size;
>  };
>
>  /* Global LRU lists shared by all zswap pools. */
> @@ -181,6 +190,7 @@ static struct shrinker *zswap_shrinker;
>   *              writeback logic. The entry is only reclaimed by the writeback
>   *              logic if referenced is unset. See comments in the shrinker
>   *              section for context.
> + * nid - NUMA node id of the page for which this is the zswap entry.
>   * pool - the zswap_pool the entry's data is in
>   * handle - zsmalloc allocation handle that stores the compressed page data
>   * objcg - the obj_cgroup that the compressed memory is charged to
> @@ -188,8 +198,11 @@ static struct shrinker *zswap_shrinker;
>   */
>  struct zswap_entry {
>         swp_entry_t swpentry;
> -       unsigned int length;
> -       bool referenced;
> +       struct {
> +               unsigned int length:(ZSWAP_ENTRY_SPARE_4BYTES - ZSWAP_ENTRY_REF_BIT);
> +               bool referenced:ZSWAP_ENTRY_REF_BIT;

Hmm I thought Yosry confirmed that using values directly rather than
macros (i.e 32 and 1 instead of ZSWAP_ENTRY_SPARE_4BYTES and
ZSWAP_ENTRY_REF_BIT) was the convention? :)

https://lore.kernel.org/linux-mm/gnm6hcqlzna4p3unrad2sur7pnyovr7f2sfuiufzweu2zbfb2r@ia422moyti7v/

I was just copying zsmalloc's format ;) Anyway, either way a fixlet
should be sufficient. No big deal...

> +       };
> +       int nid;
>         struct zswap_pool *pool;
>         unsigned long handle;
>         struct obj_cgroup *objcg;
> @@ -241,8 +254,10 @@ static inline struct xarray *swap_zswap_tree(swp_entry_t swp)
>  **********************************/
>  static void __zswap_pool_empty(struct percpu_ref *ref);
>
> -static void acomp_ctx_dealloc(struct crypto_acomp_ctx *acomp_ctx)
> +static void acomp_ctx_dealloc(struct crypto_acomp_ctx *acomp_ctx, u8 nr_buffers)
>  {
> +       u8 i;
> +
>         if (IS_ERR_OR_NULL(acomp_ctx))
>                 return;
>
> @@ -252,7 +267,11 @@ static void acomp_ctx_dealloc(struct crypto_acomp_ctx *acomp_ctx)
>         if (!IS_ERR_OR_NULL(acomp_ctx->acomp))
>                 crypto_free_acomp(acomp_ctx->acomp);
>
> -       kfree(acomp_ctx->buffer);
> +       if (acomp_ctx->buffers) {
> +               for (i = 0; i < nr_buffers; ++i)
> +                       kfree(acomp_ctx->buffers[i]);
> +               kfree(acomp_ctx->buffers);
> +       }
>  }
>
>  static struct zswap_pool *zswap_pool_create(char *compressor)
> @@ -264,6 +283,7 @@ static struct zswap_pool *zswap_pool_create(char *compressor)
>         if (!zswap_has_pool && !strcmp(compressor, ZSWAP_PARAM_UNSET))
>                 return NULL;
>
> +       /* Many things rely on the zero-initialization. */
>         pool = kzalloc(sizeof(*pool), GFP_KERNEL);
>         if (!pool)
>                 return NULL;
> @@ -316,7 +336,9 @@ static struct zswap_pool *zswap_pool_create(char *compressor)
>
>  cpuhp_add_fail:
>         for_each_possible_cpu(cpu)
> -               acomp_ctx_dealloc(per_cpu_ptr(pool->acomp_ctx, cpu));
> +               acomp_ctx_dealloc(per_cpu_ptr(pool->acomp_ctx, cpu),
> +                                 pool->compr_batch_size);
> +
>  error:
>         if (pool->acomp_ctx)
>                 free_percpu(pool->acomp_ctx);
> @@ -354,7 +376,8 @@ static void zswap_pool_destroy(struct zswap_pool *pool)
>         cpuhp_state_remove_instance(CPUHP_MM_ZSWP_POOL_PREPARE, &pool->node);
>
>         for_each_possible_cpu(cpu)
> -               acomp_ctx_dealloc(per_cpu_ptr(pool->acomp_ctx, cpu));
> +               acomp_ctx_dealloc(per_cpu_ptr(pool->acomp_ctx, cpu),
> +                                 pool->compr_batch_size);
>
>         free_percpu(pool->acomp_ctx);
>
> @@ -645,14 +668,8 @@ static inline struct mem_cgroup *mem_cgroup_from_entry(struct zswap_entry *entry
>  }
>  #endif
>
> -static inline int entry_to_nid(struct zswap_entry *entry)
> -{
> -       return page_to_nid(virt_to_page(entry));
> -}
> -
>  static void zswap_lru_add(struct list_lru *list_lru, struct zswap_entry *entry)
>  {
> -       int nid = entry_to_nid(entry);
>         struct mem_cgroup *memcg;
>
>         /*
> @@ -669,19 +686,18 @@ static void zswap_lru_add(struct list_lru *list_lru, struct zswap_entry *entry)
>         rcu_read_lock();
>         memcg = mem_cgroup_from_entry(entry);
>         /* will always succeed */
> -       list_lru_add(list_lru, &entry->lru, nid, memcg);
> +       list_lru_add(list_lru, &entry->lru, entry->nid, memcg);
>         rcu_read_unlock();
>  }
>
>  static void zswap_lru_del(struct list_lru *list_lru, struct zswap_entry *entry)
>  {
> -       int nid = entry_to_nid(entry);
>         struct mem_cgroup *memcg;
>
>         rcu_read_lock();
>         memcg = mem_cgroup_from_entry(entry);
>         /* will always succeed */
> -       list_lru_del(list_lru, &entry->lru, nid, memcg);
> +       list_lru_del(list_lru, &entry->lru, entry->nid, memcg);
>         rcu_read_unlock();
>  }
>
> @@ -741,6 +757,56 @@ static void zswap_entry_cache_free(struct zswap_entry *entry)
>         kmem_cache_free(zswap_entry_cache, entry);
>  }
>
> +static __always_inline void zswap_entries_cache_free_batch(
> +       struct zswap_entry **entries,
> +       u8 nr_entries)
> +{
> +       /*
> +        * It is okay to use this to free entries allocated separately
> +        * by zswap_entry_cache_alloc().
> +        */
> +       kmem_cache_free_bulk(zswap_entry_cache, nr_entries, (void **)entries);
> +}
> +
> +static __always_inline bool zswap_entries_cache_alloc_batch(
> +       struct zswap_entry **entries,
> +       u8 nr_entries,
> +       gfp_t gfp,
> +       int nid)
> +{
> +       int nr_alloc = kmem_cache_alloc_bulk(zswap_entry_cache, gfp,
> +                                            nr_entries, (void **)entries);
> +
> +       /*
> +        * kmem_cache_alloc_bulk() should return @nr_entries on success
> +        * and 0 on failure.
> +        */
> +       if (likely(nr_alloc == nr_entries))
> +               return true;
> +
> +       if (WARN_ON_ONCE(unlikely(nr_alloc && (nr_alloc != nr_entries)))) {
> +               zswap_reject_kmemcache_fail++;
> +               zswap_entries_cache_free_batch(entries, nr_alloc);
> +               nr_alloc = 0;
> +       }

Can partial allocation happen? I checked a couple callers of
kmem_cache_alloc_bulk() and none of them check the case nr_alloc &&
nr_alloc != nr_entries.

In fact, one caller (__io_alloc_req_refill() in io_uring/io_uring.c)
even explicitly document:

    ret = kmem_cache_alloc_bulk(req_cachep, gfp, ARRAY_SIZE(reqs), reqs);

    /*
    * Bulk alloc is all-or-nothing. If we fail to get a batch,
    * retry single alloc to be on the safe side.
    */
    if (unlikely(ret <= 0)) {
        reqs[0] = kmem_cache_alloc(req_cachep, gfp);
        if (!reqs[0])
            return false;
        ret = 1;
    }

Other callsers don't even bother checking the negative case (i.e ret <
0) - only the 0 case. I'm not terribly familiar with bulk allocation
though. Please fact check me :)

> +
> +       if (unlikely(!nr_alloc)) {
> +               unsigned int i;
> +
> +               for (i = 0; i < nr_entries; ++i) {
> +                       entries[i] = zswap_entry_cache_alloc(GFP_KERNEL, nid);
> +
> +                       if (unlikely(!entries[i])) {
> +                               zswap_reject_kmemcache_fail++;
> +                               zswap_entries_cache_free_batch(entries, i);
> +                               return false;
> +                       }
> +               }
> +       }
> +
> +       return true;
> +}
> +
>  /*
>   * Carries out the common pattern of freeing an entry's zsmalloc allocation,
>   * freeing the entry itself, and decrementing the number of stored pages.
> @@ -767,7 +833,9 @@ static int zswap_cpu_comp_prepare(unsigned int cpu, struct hlist_node *node)
>  {
>         struct zswap_pool *pool = hlist_entry(node, struct zswap_pool, node);
>         struct crypto_acomp_ctx *acomp_ctx = per_cpu_ptr(pool->acomp_ctx, cpu);
> +       int nid = cpu_to_node(cpu);
>         int ret = -ENOMEM;
> +       u8 i;
>
>         /*
>          * To handle cases where the CPU goes through online-offline-online
> @@ -778,11 +846,7 @@ static int zswap_cpu_comp_prepare(unsigned int cpu, struct hlist_node *node)
>                 return 0;
>         }
>
> -       acomp_ctx->buffer = kmalloc_node(PAGE_SIZE, GFP_KERNEL, cpu_to_node(cpu));
> -       if (!acomp_ctx->buffer)
> -               return ret;
> -
> -       acomp_ctx->acomp = crypto_alloc_acomp_node(pool->tfm_name, 0, 0, cpu_to_node(cpu));
> +       acomp_ctx->acomp = crypto_alloc_acomp_node(pool->tfm_name, 0, 0, nid);
>         if (IS_ERR_OR_NULL(acomp_ctx->acomp)) {
>                 pr_err("could not alloc crypto acomp %s : %ld\n",
>                                 pool->tfm_name, PTR_ERR(acomp_ctx->acomp));
> @@ -790,20 +854,39 @@ static int zswap_cpu_comp_prepare(unsigned int cpu, struct hlist_node *node)
>                 goto fail;
>         }
>
> +       /*
> +        * Allocate up to ZSWAP_MAX_BATCH_SIZE dst buffers if the
> +        * compressor supports batching.
> +        */
> +       pool->compr_batch_size = min(ZSWAP_MAX_BATCH_SIZE,
> +                                    crypto_acomp_batch_size(acomp_ctx->acomp));
> +

I asssume this is going to be 0 for zstd?

>         acomp_ctx->req = acomp_request_alloc(acomp_ctx->acomp);
> +
>         if (IS_ERR_OR_NULL(acomp_ctx->req)) {
>                 pr_err("could not alloc crypto acomp_request %s\n",
>                        pool->tfm_name);
>                 goto fail;
>         }
>
> -       crypto_init_wait(&acomp_ctx->wait);
> +       acomp_ctx->buffers = kcalloc_node(pool->compr_batch_size, sizeof(u8 *),
> +                                         GFP_KERNEL, nid);
> +       if (!acomp_ctx->buffers)
> +               goto fail;
> +
> +       for (i = 0; i < pool->compr_batch_size; ++i) {
> +               acomp_ctx->buffers[i] = kmalloc_node(PAGE_SIZE, GFP_KERNEL, nid);
> +               if (!acomp_ctx->buffers[i])
> +                       goto fail;
> +       }
>
>         /*
>          * if the backend of acomp is async zip, crypto_req_done() will wakeup
>          * crypto_wait_req(); if the backend of acomp is scomp, the callback
>          * won't be called, crypto_wait_req() will return without blocking.
>          */
> +       crypto_init_wait(&acomp_ctx->wait);
> +
>         acomp_request_set_callback(acomp_ctx->req, CRYPTO_TFM_REQ_MAY_BACKLOG,
>                                    crypto_req_done, &acomp_ctx->wait);
>
> @@ -813,12 +896,12 @@ static int zswap_cpu_comp_prepare(unsigned int cpu, struct hlist_node *node)
>         return 0;
>
>  fail:
> -       acomp_ctx_dealloc(acomp_ctx);
> +       acomp_ctx_dealloc(acomp_ctx, pool->compr_batch_size);
>         return ret;
>  }
>
>  static bool zswap_compress(struct page *page, struct zswap_entry *entry,
> -                          struct zswap_pool *pool)
> +                          struct zswap_pool *pool, bool wb_enabled)
>  {
>         struct crypto_acomp_ctx *acomp_ctx;
>         struct scatterlist input, output;
> @@ -832,7 +915,7 @@ static bool zswap_compress(struct page *page, struct zswap_entry *entry,
>         acomp_ctx = raw_cpu_ptr(pool->acomp_ctx);
>         mutex_lock(&acomp_ctx->mutex);
>
> -       dst = acomp_ctx->buffer;
> +       dst = acomp_ctx->buffers[0];
>         sg_init_table(&input, 1);
>         sg_set_page(&input, page, PAGE_SIZE, 0);
>
> @@ -862,8 +945,7 @@ static bool zswap_compress(struct page *page, struct zswap_entry *entry,
>          * to the active LRU list in the case.
>          */
>         if (comp_ret || !dlen || dlen >= PAGE_SIZE) {
> -               if (!mem_cgroup_zswap_writeback_enabled(
> -                                       folio_memcg(page_folio(page)))) {
> +               if (!wb_enabled) {
>                         comp_ret = comp_ret ? comp_ret : -EINVAL;
>                         goto unlock;
>                 }
> @@ -909,7 +991,7 @@ static bool zswap_decompress(struct zswap_entry *entry, struct folio *folio)
>         acomp_ctx = raw_cpu_ptr(pool->acomp_ctx);
>         mutex_lock(&acomp_ctx->mutex);
>         obj = zs_obj_read_begin(pool->zs_pool, entry->handle, entry->length,
> -                               acomp_ctx->buffer);
> +                               acomp_ctx->buffers[0]);
>
>         /* zswap entries of length PAGE_SIZE are not compressed. */
>         if (entry->length == PAGE_SIZE) {
> @@ -919,15 +1001,15 @@ static bool zswap_decompress(struct zswap_entry *entry, struct folio *folio)
>
>         /*
>          * zs_obj_read_begin() might return a kmap address of highmem when
> -        * acomp_ctx->buffer is not used.  However, sg_init_one() does not
> -        * handle highmem addresses, so copy the object to acomp_ctx->buffer.
> +        * acomp_ctx->buffers[0] is not used.  However, sg_init_one() does not
> +        * handle highmem addresses, so copy the object to acomp_ctx->buffers[0].
>          */
>         if (virt_addr_valid(obj)) {
>                 src = obj;
>         } else {
> -               WARN_ON_ONCE(obj == acomp_ctx->buffer);
> -               memcpy(acomp_ctx->buffer, obj, entry->length);
> -               src = acomp_ctx->buffer;
> +               WARN_ON_ONCE(obj == acomp_ctx->buffers[0]);
> +               memcpy(acomp_ctx->buffers[0], obj, entry->length);
> +               src = acomp_ctx->buffers[0];
>         }
>
>         sg_init_one(&input, src, entry->length);
> @@ -1381,95 +1463,136 @@ static void shrink_worker(struct work_struct *w)
>  * main API
>  **********************************/
>
> -static bool zswap_store_page(struct page *page,
> -                            struct obj_cgroup *objcg,
> -                            struct zswap_pool *pool)
> +/*
> + * Store multiple pages in @folio, starting from the page at index @start up to
> + * the page at index @end-1.
> + */
> +static bool zswap_store_pages(struct folio *folio,
> +                             long start,
> +                             long end,
> +                             struct zswap_pool *pool,
> +                             struct crypto_acomp_ctx *acomp_ctx,
> +                             int nid,
> +                             bool wb_enabled,
> +                             struct obj_cgroup *objcg)
>  {
> -       swp_entry_t page_swpentry = page_swap_entry(page);
> -       struct zswap_entry *entry, *old;
> +       struct zswap_entry *entries[ZSWAP_MAX_BATCH_SIZE];
> +       u8 i, store_fail_idx = 0, nr_pages = end - start;
>
> -       /* allocate entry */
> -       entry = zswap_entry_cache_alloc(GFP_KERNEL, page_to_nid(page));
> -       if (!entry) {
> -               zswap_reject_kmemcache_fail++;
> +       VM_WARN_ON_ONCE(nr_pages > ZSWAP_MAX_BATCH_SIZE);
> +
> +       if (unlikely(!zswap_entries_cache_alloc_batch(entries, nr_pages,
> +                                                     GFP_KERNEL, nid)))
>                 return false;
> -       }
>
> -       if (!zswap_compress(page, entry, pool))
> -               goto compress_failed;
> +       /*
> +        * We co-locate entry initialization as much as possible here to
> +        * minimize potential cache misses.
> +        */
> +       for (i = 0; i < nr_pages; ++i) {
> +               entries[i]->handle = (unsigned long)ERR_PTR(-EINVAL);
> +               entries[i]->pool = pool;
> +               entries[i]->swpentry = page_swap_entry(folio_page(folio, start + i));
> +               entries[i]->objcg = objcg;
> +               entries[i]->referenced = true;
> +               entries[i]->nid = nid;
> +               INIT_LIST_HEAD(&entries[i]->lru);
> +       }
>
> -       old = xa_store(swap_zswap_tree(page_swpentry),
> -                      swp_offset(page_swpentry),
> -                      entry, GFP_KERNEL);
> -       if (xa_is_err(old)) {
> -               int err = xa_err(old);
> +       for (i = 0; i < nr_pages; ++i) {
> +               struct page *page = folio_page(folio, start + i);
>
> -               WARN_ONCE(err != -ENOMEM, "unexpected xarray error: %d\n", err);
> -               zswap_reject_alloc_fail++;
> -               goto store_failed;
> +               if (!zswap_compress(page, entries[i], pool, wb_enabled))
> +                       goto store_pages_failed;
>         }
>
> -       /*
> -        * We may have had an existing entry that became stale when
> -        * the folio was redirtied and now the new version is being
> -        * swapped out. Get rid of the old.
> -        */
> -       if (old)
> -               zswap_entry_free(old);
> +       for (i = 0; i < nr_pages; ++i) {
> +               struct zswap_entry *old, *entry = entries[i];
>
> -       /*
> -        * The entry is successfully compressed and stored in the tree, there is
> -        * no further possibility of failure. Grab refs to the pool and objcg,
> -        * charge zswap memory, and increment zswap_stored_pages.
> -        * The opposite actions will be performed by zswap_entry_free()
> -        * when the entry is removed from the tree.
> -        */
> -       zswap_pool_get(pool);
> -       if (objcg) {
> -               obj_cgroup_get(objcg);
> -               obj_cgroup_charge_zswap(objcg, entry->length);
> -       }
> -       atomic_long_inc(&zswap_stored_pages);
> -       if (entry->length == PAGE_SIZE)
> -               atomic_long_inc(&zswap_stored_incompressible_pages);
> +               old = xa_store(swap_zswap_tree(entry->swpentry),
> +                              swp_offset(entry->swpentry),
> +                              entry, GFP_KERNEL);

Future follow-up: perhaps we can use advanced xarray API (xas_*) to
take the lock only once.

> +               if (unlikely(xa_is_err(old))) {
> +                       int err = xa_err(old);
>
> -       /*
> -        * We finish initializing the entry while it's already in xarray.
> -        * This is safe because:
> -        *
> -        * 1. Concurrent stores and invalidations are excluded by folio lock.
> -        *
> -        * 2. Writeback is excluded by the entry not being on the LRU yet.
> -        *    The publishing order matters to prevent writeback from seeing
> -        *    an incoherent entry.
> -        */
> -       entry->pool = pool;
> -       entry->swpentry = page_swpentry;
> -       entry->objcg = objcg;
> -       entry->referenced = true;
> -       if (entry->length) {
> -               INIT_LIST_HEAD(&entry->lru);
> -               zswap_lru_add(&zswap_list_lru, entry);
> +                       WARN_ONCE(err != -ENOMEM, "unexpected xarray error: %d\n", err);
> +                       zswap_reject_alloc_fail++;
> +                       /*
> +                        * Entries up to this point have been stored in the
> +                        * xarray. zswap_store() will erase them from the xarray
> +                        * and call zswap_entry_free(). Local cleanup in
> +                        * 'store_pages_failed' only needs to happen for
> +                        * entries from [@i to @nr_pages).
> +                        */
> +                       store_fail_idx = i;
> +                       goto store_pages_failed;
> +               }
> +
> +               /*
> +                * We may have had an existing entry that became stale when
> +                * the folio was redirtied and now the new version is being
> +                * swapped out. Get rid of the old.
> +                */
> +               if (unlikely(old))
> +                       zswap_entry_free(old);
> +
> +               /*
> +                * The entry is successfully compressed and stored in the tree,
> +                * and further failures will be cleaned up in zswap_store().
> +                * Grab refs to the pool and objcg, charge zswap memory, and
> +                * increment zswap_stored_pages. The opposite actions will be
> +                * performed by zswap_entry_free() when the entry is removed
> +                * from the tree.
> +                */
> +               zswap_pool_get(pool);
> +               if (objcg) {
> +                       obj_cgroup_get(objcg);
> +                       obj_cgroup_charge_zswap(objcg, entry->length);
> +               }
> +               atomic_long_inc(&zswap_stored_pages);
> +               if (entry->length == PAGE_SIZE)
> +                       atomic_long_inc(&zswap_stored_incompressible_pages);
> +
> +               /*
> +                * We finish by adding the entry to the LRU while it's already
> +                * in xarray. This is safe because:
> +                *
> +                * 1. Concurrent stores and invalidations are excluded by folio lock.
> +                *
> +                * 2. Writeback is excluded by the entry not being on the LRU yet.
> +                *    The publishing order matters to prevent writeback from seeing
> +                *    an incoherent entry.
> +                */
> +               if (likely(entry->length))
> +                       zswap_lru_add(&zswap_list_lru, entry);

Hang on - how can entry->length == 0? This is probably holdover from
back when zero pages are still managed in zswap?

Future follow-up work: remove this check if that's the case...

The rest looks solid to me - I'll defer to Yosry and Johannes.

Powered by blists - more mailing lists

Powered by Openwall GNU/*/Linux Powered by OpenVZ