linux-mm.kvack.org archive mirror
 help / color / mirror / Atom feed
From: Kanchana P Sridhar <kanchana.p.sridhar@intel.com>
To: linux-kernel@vger.kernel.org, linux-mm@kvack.org,
	hannes@cmpxchg.org, yosry.ahmed@linux.dev, nphamcs@gmail.com,
	chengming.zhou@linux.dev, usamaarif642@gmail.com,
	ryan.roberts@arm.com, 21cnbao@gmail.com,
	ying.huang@linux.alibaba.com, akpm@linux-foundation.org,
	senozhatsky@chromium.org, linux-crypto@vger.kernel.org,
	herbert@gondor.apana.org.au, davem@davemloft.net,
	clabbe@baylibre.com, ardb@kernel.org, ebiggers@google.com,
	surenb@google.com, kristen.c.accardi@intel.com,
	vinicius.gomes@intel.com
Cc: wajdi.k.feghali@intel.com, vinodh.gopal@intel.com,
	kanchana.p.sridhar@intel.com
Subject: [PATCH v10 25/25] mm: zswap: Batched zswap_compress() with compress batching of large folios.
Date: Thu,  3 Jul 2025 21:23:23 -0700	[thread overview]
Message-ID: <20250704042323.10318-26-kanchana.p.sridhar@intel.com> (raw)
In-Reply-To: <20250704042323.10318-1-kanchana.p.sridhar@intel.com>

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

Signed-off-by: Kanchana P Sridhar <kanchana.p.sridhar@intel.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;
+};
+
 /* linux/mm/page_io.c */
 int sio_pool_init(void);
 struct swap_iocb;
diff --git a/mm/zswap.c b/mm/zswap.c
index e8a9b8ef65b93..b3c2a287610f9 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 void zswap_entry_free(struct zswap_entry *entry)
 /*********************************
 * compressed storage functions
 **********************************/
-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.
+ *
+ * @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.
+ *
+ * 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;
+		}
 
-	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



  parent reply	other threads:[~2025-07-04  4:24 UTC|newest]

Thread overview: 33+ messages / expand[flat|nested]  mbox.gz  Atom feed  top
2025-07-04  4:22 [PATCH v10 00/25] zswap compression batching with optimized iaa_crypto driver Kanchana P Sridhar
2025-07-04  4:22 ` [PATCH v10 01/25] crypto: iaa - Reorganize the iaa_crypto driver code Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 02/25] crypto: iaa - New architecture for IAA device WQ comp/decomp usage & core mapping Kanchana P Sridhar
2025-07-05  6:00   ` kernel test robot
2025-07-04  4:23 ` [PATCH v10 03/25] crypto: iaa - Simplify, consistency of function parameters, minor stats bug fix Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 04/25] crypto: iaa - Descriptor allocation timeouts with mitigations Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 05/25] crypto: iaa - iaa_wq uses percpu_refs for get/put reference counting Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 06/25] crypto: iaa - Simplify the code flow in iaa_compress() and iaa_decompress() Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 07/25] crypto: iaa - Refactor hardware descriptor setup into separate procedures Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 08/25] crypto: iaa - Simplified, efficient job submissions for non-irq mode Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 09/25] crypto: iaa - Deprecate exporting add/remove IAA compression modes Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 10/25] crypto: iaa - Rearchitect the iaa_crypto driver to be usable by zswap and zram Kanchana P Sridhar
2025-07-04 15:56   ` kernel test robot
2025-07-05 14:49   ` kernel test robot
2025-07-04  4:23 ` [PATCH v10 11/25] crypto: iaa - Enablers for submitting descriptors then polling for completion Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 12/25] crypto: acomp - Add "void *kernel_data" in "struct acomp_req" for kernel users Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 13/25] crypto: iaa - IAA Batching for parallel compressions/decompressions Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 14/25] crypto: iaa - Enable async mode and make it the default Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 15/25] crypto: iaa - Disable iaa_verify_compress by default Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 16/25] crypto: iaa - Submit the two largest source buffers first in decompress batching Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 17/25] crypto: iaa - Add deflate-iaa-dynamic compression mode Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 18/25] crypto: acomp - Add crypto_acomp_batch_size() to get an algorithm's batch-size Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 19/25] crypto: iaa - IAA acomp_algs register the get_batch_size() interface Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 20/25] mm: zswap: Move the CPU hotplug procedures under "pool functions" Kanchana P Sridhar
2025-07-04 18:39   ` Nhat Pham
2025-07-04 21:35     ` Sridhar, Kanchana P
2025-07-04  4:23 ` [PATCH v10 21/25] mm: zswap: Per-CPU acomp_ctx resources exist from pool creation to deletion Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 22/25] mm: zswap: Consistently use IS_ERR_OR_NULL() to check acomp_ctx resources Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 23/25] mm: zswap: Allocate pool batching resources if the compressor supports batching Kanchana P Sridhar
2025-07-04  4:23 ` [PATCH v10 24/25] mm: zswap: zswap_store() will process a large folio in batches Kanchana P Sridhar
2025-07-04  4:23 ` Kanchana P Sridhar [this message]
2025-07-04 18:37 ` [PATCH v10 00/25] zswap compression batching with optimized iaa_crypto driver Nhat Pham
2025-07-04 21:39   ` Sridhar, Kanchana P

Reply instructions:

You may reply publicly to this message via plain-text email
using any one of the following methods:

* Save the following mbox file, import it into your mail client,
  and reply-to-all from there: mbox

  Avoid top-posting and favor interleaved quoting:
  https://en.wikipedia.org/wiki/Posting_style#Interleaved_style

* Reply using the --to, --cc, and --in-reply-to
  switches of git-send-email(1):

  git send-email \
    --in-reply-to=20250704042323.10318-26-kanchana.p.sridhar@intel.com \
    --to=kanchana.p.sridhar@intel.com \
    --cc=21cnbao@gmail.com \
    --cc=akpm@linux-foundation.org \
    --cc=ardb@kernel.org \
    --cc=chengming.zhou@linux.dev \
    --cc=clabbe@baylibre.com \
    --cc=davem@davemloft.net \
    --cc=ebiggers@google.com \
    --cc=hannes@cmpxchg.org \
    --cc=herbert@gondor.apana.org.au \
    --cc=kristen.c.accardi@intel.com \
    --cc=linux-crypto@vger.kernel.org \
    --cc=linux-kernel@vger.kernel.org \
    --cc=linux-mm@kvack.org \
    --cc=nphamcs@gmail.com \
    --cc=ryan.roberts@arm.com \
    --cc=senozhatsky@chromium.org \
    --cc=surenb@google.com \
    --cc=usamaarif642@gmail.com \
    --cc=vinicius.gomes@intel.com \
    --cc=vinodh.gopal@intel.com \
    --cc=wajdi.k.feghali@intel.com \
    --cc=ying.huang@linux.alibaba.com \
    --cc=yosry.ahmed@linux.dev \
    /path/to/YOUR_REPLY

  https://kernel.org/pub/software/scm/git/docs/git-send-email.html

* If your mail client supports setting the In-Reply-To header
  via mailto: links, try the mailto: link
Be sure your reply has a Subject: header at the top and a blank line before the message body.
This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox