[<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