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=NpZ5gjSUJ93BN1c8x1Qfmac6-gKcRxRdAztME2snyCaQ@mail.gmail.com>
Date: Thu, 14 Aug 2025 14:14:37 -0700
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, 
	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, wajdi.k.feghali@...el.com, vinodh.gopal@...el.com
Subject: Re: [PATCH v11 24/24] mm: zswap: Batched zswap_compress() with
 compress batching of large folios.

On Thu, Jul 31, 2025 at 9:36 PM Kanchana P Sridhar
<kanchana.p.sridhar@...el.com> wrote:
>
> This patch introduces a new unified implementation of zswap_compress()
> for compressors that do and do not support batching. This eliminates
> code duplication and facilitates maintainability of the code with the
> introduction of compress batching.
>
> The vectorized implementation of calling the earlier zswap_compress()
> sequentially, one page at a time in zswap_store_pages(), is replaced
> with this new version of zswap_compress() that accepts multiple pages to
> compress as a batch.
>
> If the compressor does not support batching, each page in the batch is
> compressed and stored sequentially.
>
> If the compressor supports batching, for e.g., 'deflate-iaa', the Intel
> IAA hardware accelerator, the batch is compressed in parallel in
> hardware by setting the acomp_ctx->req->kernel_data to contain the
> necessary batching data before calling crypto_acomp_compress(). If all
> requests in the batch are compressed without errors, the compressed
> buffers are then stored in zpool.
>
> Another important change this patch makes is with the acomp_ctx mutex
> locking in zswap_compress(). Earlier, the mutex was held per page's
> compression. With the new code, [un]locking the mutex per page caused
> regressions for software compressors when testing with usemem
> (30 processes) and also kernel compilation with 'allmod' config. The
> regressions were more eggregious when PMD folios were stored. The
> implementation in this commit locks/unlocks the mutex once per batch,
> that resolves the regression.
>
> The use of prefetchw() for zswap entries and likely()/unlikely()
> annotations prevent regressions with software compressors like zstd, and
> generally improve non-batching compressors' performance with the
> batching code by ~3%.
>
> Architectural considerations for the zswap batching framework:
> ==============================================================
> We have designed the zswap batching framework to be
> hardware-agnostic. It has no dependencies on Intel-specific features and
> can be leveraged by any hardware accelerator or software-based
> compressor. In other words, the framework is open and inclusive by
> design.
>
> Other ongoing work that can use batching:
> =========================================
> This patch-series demonstrates the performance benefits of compress
> batching when used in zswap_store() of large folios. shrink_folio_list()
> "reclaim batching" of any-order folios is the major next work that uses
> the zswap compress batching framework: our testing of kernel_compilation
> with writeback and the zswap shrinker indicates 10X fewer pages get
> written back when we reclaim 32 folios as a batch, as compared to one
> folio at a time: this is with deflate-iaa and with zstd. We expect to
> submit a patch-series with this data and the resulting performance
> improvements shortly. Reclaim batching relieves memory pressure faster
> than reclaiming one folio at a time, hence alleviates the need to scan
> slab memory for writeback.
>
> Nhat has given ideas on using batching with the ongoing kcompressd work,
> as well as beneficially using decompression batching & block IO batching
> to improve zswap writeback efficiency.
>
> Experiments that combine zswap compress batching, reclaim batching,
> swapin_readahead() decompression batching of prefetched pages, and
> writeback batching show that 0 pages are written back with deflate-iaa
> and zstd. For comparison, the baselines for these compressors see
> 200K-800K pages written to disk (kernel compilation 'allmod' config).
>
> To summarize, these are future clients of the batching framework:
>
>    - shrink_folio_list() reclaim batching of multiple folios:
>        Implemented, will submit patch-series.
>    - zswap writeback with decompress batching:
>        Implemented, will submit patch-series.
>    - zram:
>        Implemented, will submit patch-series.
>    - kcompressd:
>        Not yet implemented.
>    - file systems:
>        Not yet implemented.
>    - swapin_readahead() decompression batching of prefetched pages:
>        Implemented, will submit patch-series.
>
> Additionally, any place we have folios that need to be compressed, can
> potentially be parallelized.
>
> Signed-off-by: Kanchana P Sridhar <kanchana.p.sridhar@...el.com>
> ---
>  mm/swap.h  |  23 ++++++
>  mm/zswap.c | 201 ++++++++++++++++++++++++++++++++++++++---------------
>  2 files changed, 168 insertions(+), 56 deletions(-)
>
> diff --git a/mm/swap.h b/mm/swap.h
> index 911ad5ff0f89f..2afbf00f59fea 100644
> --- a/mm/swap.h
> +++ b/mm/swap.h
> @@ -11,6 +11,29 @@ extern int page_cluster;
>  #include <linux/swapops.h> /* for swp_offset */
>  #include <linux/blk_types.h> /* for bio_end_io_t */
>
> +/* linux/mm/zswap.c */
> +/*
> + * A compression algorithm that wants to batch compressions/decompressions
> + * must define its own internal data structures that exactly mirror
> + * @struct swap_batch_comp_data and @struct swap_batch_decomp_data.
> + */
> +struct swap_batch_comp_data {
> +       struct page **pages;
> +       u8 **dsts;
> +       unsigned int *dlens;
> +       int *errors;
> +       u8 nr_comps;
> +};
> +
> +struct swap_batch_decomp_data {
> +       u8 **srcs;
> +       struct page **pages;
> +       unsigned int *slens;
> +       unsigned int *dlens;
> +       int *errors;
> +       u8 nr_decomps;
> +};

This struct is not being used yet right? I assume this is used for
batch zswap load and writeback etc.

Can we introduce them when those series are sent out? Just to limit
the amount of reviewing here :)

> +
>  /* linux/mm/page_io.c */
>  int sio_pool_init(void);
>  struct swap_iocb;
> diff --git a/mm/zswap.c b/mm/zswap.c
> index 8ca69c3f30df2..c30c1f325f573 100644
> --- a/mm/zswap.c
> +++ b/mm/zswap.c
> @@ -35,6 +35,7 @@
>  #include <linux/pagemap.h>
>  #include <linux/workqueue.h>
>  #include <linux/list_lru.h>
> +#include <linux/prefetch.h>
>
>  #include "swap.h"
>  #include "internal.h"
> @@ -988,71 +989,163 @@ static int zswap_cpu_comp_prepare(unsigned int cpu, struct hlist_node *node)
>         return ret;
>  }
>
> -static bool zswap_compress(struct page *page, struct zswap_entry *entry,
> -                          struct zswap_pool *pool)
> +/*
> + * Unified code path for compressors that do and do not support batching. This
> + * procedure will compress multiple @nr_pages in @folio starting from the
> + * @start index.
> + *
> + * It is assumed that @nr_pages <= ZSWAP_MAX_BATCH_SIZE. zswap_store() makes
> + * sure of this by design.

Maybe add a VM_WARN_ON_ONCE(nr_pages <= ZSWAP_MAX_BATCH_SIZE); in
zswap_store_pages() to codify this design choice?

> + *
> + * @nr_pages can be in (1, ZSWAP_MAX_BATCH_SIZE] even if the compressor does not
> + * support batching.
> + *
> + * If @pool->compr_batch_size is 1, each page is processed sequentially.
> + *
> + * If @pool->compr_batch_size is > 1, compression batching is invoked, except if
> + * @nr_pages is 1: if so, we call the fully synchronous non-batching
> + * crypto_acomp API.
> + *
> + * In both cases, if all compressions are successful, the compressed buffers
> + * are stored in zpool.
> + *
> + * A few important changes made to not regress and in fact improve
> + * compression performance with non-batching software compressors, using this
> + * new/batching code:
> + *
> + * 1) acomp_ctx mutex locking:
> + *    Earlier, the mutex was held per page compression. With the new code,
> + *    [un]locking the mutex per page caused regressions for software
> + *    compressors. We now lock the mutex once per batch, which resolves the
> + *    regression.

Makes sense, yeah.

> + *
> + * 2) The prefetchw() and likely()/unlikely() annotations prevent
> + *    regressions with software compressors like zstd, and generally improve
> + *    non-batching compressors' performance with the batching code by ~3%.
> + */
> +static bool zswap_compress(struct folio *folio, long start, unsigned int nr_pages,
> +                          struct zswap_entry *entries[], struct zswap_pool *pool,
> +                          int node_id)
>  {
>         struct crypto_acomp_ctx *acomp_ctx;
>         struct scatterlist input, output;
> -       int comp_ret = 0, alloc_ret = 0;
> -       unsigned int dlen = PAGE_SIZE;
> -       unsigned long handle;
> -       struct zpool *zpool;
> +       struct zpool *zpool = pool->zpool;
> +
> +       unsigned int dlens[ZSWAP_MAX_BATCH_SIZE];
> +       int errors[ZSWAP_MAX_BATCH_SIZE];
> +
> +       unsigned int nr_comps = min(nr_pages, pool->compr_batch_size);
> +       unsigned int i, j;
> +       int err;
>         gfp_t gfp;
> -       u8 *dst;
> +
> +       gfp = GFP_NOWAIT | __GFP_NORETRY | __GFP_HIGHMEM | __GFP_MOVABLE;
>
>         acomp_ctx = raw_cpu_ptr(pool->acomp_ctx);
>
>         mutex_lock(&acomp_ctx->mutex);
>
> -       dst = acomp_ctx->buffers[0];
> -       sg_init_table(&input, 1);
> -       sg_set_page(&input, page, PAGE_SIZE, 0);
> -
>         /*
> -        * We need PAGE_SIZE * 2 here since there maybe over-compression case,
> -        * and hardware-accelerators may won't check the dst buffer size, so
> -        * giving the dst buffer with enough length to avoid buffer overflow.
> +        * Note:
> +        * [i] refers to the incoming batch space and is used to
> +        *     index into the folio pages, @entries and @errors.
>          */
> -       sg_init_one(&output, dst, PAGE_SIZE * 2);
> -       acomp_request_set_params(acomp_ctx->req, &input, &output, PAGE_SIZE, dlen);
> +       for (i = 0; i < nr_pages; i += nr_comps) {
> +               if (nr_comps == 1) {
> +                       sg_init_table(&input, 1);
> +                       sg_set_page(&input, folio_page(folio, start + i), PAGE_SIZE, 0);
>
> -       /*
> -        * it maybe looks a little bit silly that we send an asynchronous request,
> -        * then wait for its completion synchronously. This makes the process look
> -        * synchronous in fact.
> -        * Theoretically, acomp supports users send multiple acomp requests in one
> -        * acomp instance, then get those requests done simultaneously. but in this
> -        * case, zswap actually does store and load page by page, there is no
> -        * existing method to send the second page before the first page is done
> -        * in one thread doing zwap.
> -        * but in different threads running on different cpu, we have different
> -        * acomp instance, so multiple threads can do (de)compression in parallel.
> -        */
> -       comp_ret = crypto_wait_req(crypto_acomp_compress(acomp_ctx->req), &acomp_ctx->wait);
> -       dlen = acomp_ctx->req->dlen;
> -       if (comp_ret)
> -               goto unlock;
> +                       /*
> +                        * We need PAGE_SIZE * 2 here since there maybe over-compression case,
> +                        * and hardware-accelerators may won't check the dst buffer size, so
> +                        * giving the dst buffer with enough length to avoid buffer overflow.
> +                        */
> +                       sg_init_one(&output, acomp_ctx->buffers[0], PAGE_SIZE * 2);
> +                       acomp_request_set_params(acomp_ctx->req, &input,
> +                                                &output, PAGE_SIZE, PAGE_SIZE);
> +
> +                       errors[i] = crypto_wait_req(crypto_acomp_compress(acomp_ctx->req),
> +                                                   &acomp_ctx->wait);
> +                       if (unlikely(errors[i]))
> +                               goto compress_error;
> +
> +                       dlens[i] = acomp_ctx->req->dlen;
> +               } else {
> +                       struct page *pages[ZSWAP_MAX_BATCH_SIZE];
> +                       unsigned int k;
> +
> +                       for (k = 0; k < nr_pages; ++k)
> +                               pages[k] = folio_page(folio, start + k);
> +
> +                       struct swap_batch_comp_data batch_comp_data = {
> +                               .pages = pages,
> +                               .dsts = acomp_ctx->buffers,
> +                               .dlens = dlens,
> +                               .errors = errors,
> +                               .nr_comps = nr_pages,
> +                       };
> +
> +                       acomp_ctx->req->kernel_data = &batch_comp_data;
> +
> +                       if (unlikely(crypto_acomp_compress(acomp_ctx->req)))
> +                               goto compress_error;

I assume this is a new crypto API?

I'll let Herbert decide whether this makes sense :)

> +               }
>
> -       zpool = pool->zpool;
> -       gfp = GFP_NOWAIT | __GFP_NORETRY | __GFP_HIGHMEM | __GFP_MOVABLE;
> -       alloc_ret = zpool_malloc(zpool, dlen, gfp, &handle, page_to_nid(page));
> -       if (alloc_ret)
> -               goto unlock;
> -
> -       zpool_obj_write(zpool, handle, dst, dlen);
> -       entry->handle = handle;
> -       entry->length = dlen;
> -
> -unlock:
> -       if (comp_ret == -ENOSPC || alloc_ret == -ENOSPC)
> -               zswap_reject_compress_poor++;
> -       else if (comp_ret)
> -               zswap_reject_compress_fail++;
> -       else if (alloc_ret)
> -               zswap_reject_alloc_fail++;
> +               /*
> +                * All @nr_comps pages were successfully compressed.
> +                * Store the pages in zpool.
> +                *
> +                * Note:
> +                * [j] refers to the incoming batch space and is used to
> +                *     index into the folio pages, @entries, @dlens and @errors.
> +                * [k] refers to the @acomp_ctx space, as determined by
> +                *     @pool->compr_batch_size, and is used to index into
> +                *     @acomp_ctx->buffers.
> +                */
> +               for (j = i; j < i + nr_comps; ++j) {
> +                       unsigned int k = j - i;
> +                       unsigned long handle;
> +
> +                       /*
> +                        * prefetchw() minimizes cache-miss latency by
> +                        * moving the zswap entry to the cache before it
> +                        * is written to; reducing sys time by ~1.5% for
> +                        * non-batching software compressors.
> +                        */
> +                       prefetchw(entries[j]);
> +                       err = zpool_malloc(zpool, dlens[j], gfp, &handle, node_id);
> +
> +                       if (unlikely(err)) {
> +                               if (err == -ENOSPC)
> +                                       zswap_reject_compress_poor++;
> +                               else
> +                                       zswap_reject_alloc_fail++;
> +
> +                               goto err_unlock;
> +                       }
> +
> +                       zpool_obj_write(zpool, handle, acomp_ctx->buffers[k], dlens[j]);
> +                       entries[j]->handle = handle;
> +                       entries[j]->length = dlens[j];
> +               }
> +       } /* finished compress and store nr_pages. */
>
>         mutex_unlock(&acomp_ctx->mutex);
> -       return comp_ret == 0 && alloc_ret == 0;
> +       return true;
> +
> +compress_error:
> +       for (j = i; j < i + nr_comps; ++j) {
> +               if (errors[j]) {
> +                       if (errors[j] == -ENOSPC)
> +                               zswap_reject_compress_poor++;
> +                       else
> +                               zswap_reject_compress_fail++;
> +               }
> +       }
> +
> +err_unlock:
> +       mutex_unlock(&acomp_ctx->mutex);
> +       return false;
>  }
>
>  static bool zswap_decompress(struct zswap_entry *entry, struct folio *folio)
> @@ -1590,12 +1683,8 @@ static bool zswap_store_pages(struct folio *folio,
>                 INIT_LIST_HEAD(&entries[i]->lru);
>         }
>
> -       for (i = 0; i < nr_pages; ++i) {
> -               struct page *page = folio_page(folio, start + i);
> -
> -               if (!zswap_compress(page, entries[i], pool))
> -                       goto store_pages_failed;
> -       }
> +       if (unlikely(!zswap_compress(folio, start, nr_pages, entries, pool, node_id)))
> +               goto store_pages_failed;
>
>         for (i = 0; i < nr_pages; ++i) {
>                 struct zswap_entry *old, *entry = entries[i];
> --
> 2.27.0
>

Powered by blists - more mailing lists

Powered by Openwall GNU/*/Linux Powered by OpenVZ