From: "Sridhar, Kanchana P" <kanchana.p.sridhar@intel.com>
To: Yosry Ahmed <yosry.ahmed@linux.dev>
Cc: "linux-kernel@vger.kernel.org" <linux-kernel@vger.kernel.org>,
"linux-mm@kvack.org" <linux-mm@kvack.org>,
"hannes@cmpxchg.org" <hannes@cmpxchg.org>,
"nphamcs@gmail.com" <nphamcs@gmail.com>,
"chengming.zhou@linux.dev" <chengming.zhou@linux.dev>,
"usamaarif642@gmail.com" <usamaarif642@gmail.com>,
"ryan.roberts@arm.com" <ryan.roberts@arm.com>,
"21cnbao@gmail.com" <21cnbao@gmail.com>,
"ying.huang@linux.alibaba.com" <ying.huang@linux.alibaba.com>,
"akpm@linux-foundation.org" <akpm@linux-foundation.org>,
"senozhatsky@chromium.org" <senozhatsky@chromium.org>,
"sj@kernel.org" <sj@kernel.org>,
"kasong@tencent.com" <kasong@tencent.com>,
"linux-crypto@vger.kernel.org" <linux-crypto@vger.kernel.org>,
"herbert@gondor.apana.org.au" <herbert@gondor.apana.org.au>,
"davem@davemloft.net" <davem@davemloft.net>,
"clabbe@baylibre.com" <clabbe@baylibre.com>,
"ardb@kernel.org" <ardb@kernel.org>,
"ebiggers@google.com" <ebiggers@google.com>,
"surenb@google.com" <surenb@google.com>,
"Accardi, Kristen C" <kristen.c.accardi@intel.com>,
"Gomes, Vinicius" <vinicius.gomes@intel.com>,
"Feghali, Wajdi K" <wajdi.k.feghali@intel.com>,
"Gopal, Vinodh" <vinodh.gopal@intel.com>,
"Sridhar, Kanchana P" <kanchana.p.sridhar@intel.com>
Subject: RE: [PATCH v13 21/22] mm: zswap: zswap_store() will process a large folio in batches.
Date: Fri, 12 Dec 2025 01:43:54 +0000 [thread overview]
Message-ID: <SJ2PR11MB8472875DB2900920EB2C51DCC9AEA@SJ2PR11MB8472.namprd11.prod.outlook.com> (raw)
In-Reply-To: <3i4jpzank53niagxzddc3vmzi5s3o5kmernzitktyavd5jwvp5@ytxzww57ux7a>
> -----Original Message-----
> From: Yosry Ahmed <yosry.ahmed@linux.dev>
> Sent: Thursday, November 13, 2025 12:51 PM
> To: Sridhar, Kanchana P <kanchana.p.sridhar@intel.com>
> Cc: linux-kernel@vger.kernel.org; linux-mm@kvack.org;
> hannes@cmpxchg.org; 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; sj@kernel.org; kasong@tencent.com; 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; Accardi, Kristen C
> <kristen.c.accardi@intel.com>; Gomes, Vinicius <vinicius.gomes@intel.com>;
> Feghali, Wajdi K <wajdi.k.feghali@intel.com>; Gopal, Vinodh
> <vinodh.gopal@intel.com>
> Subject: Re: [PATCH v13 21/22] mm: zswap: zswap_store() will process a
> large folio in batches.
>
> On Tue, Nov 04, 2025 at 01:12:34AM -0800, Kanchana P Sridhar wrote:
>
> Subject:
>
> "mm: zswap: Store large folios in batches"
>
> > This patch makes two major changes:
> >
> > First, we allocate pool batching resources if the compressor supports
> > batching:
> >
> > This patch sets up zswap for allocating per-CPU resources optimally
> > for non-batching and batching compressors.
> >
> > A new ZSWAP_MAX_BATCH_SIZE constant is defined as 8U, to set an upper
> > limit on the number of pages in large folios that will be batch
> > compressed.
> >
> > It is up to the compressor to manage multiple requests, as needed, to
> > accomplish batch parallelism. zswap only needs to allocate the per-CPU
> > dst buffers according to the batch size supported by the compressor.
> >
> > A "u8 compr_batch_size" member is added to "struct zswap_pool", as per
> > Yosry's suggestion. pool->compr_batch_size is set as the minimum of
> > the compressor's max batch-size and ZSWAP_MAX_BATCH_SIZE.
> Accordingly,
> > pool->compr_batch_size compression dst buffers are allocated in the
> > per-CPU acomp_ctx.
> >
> > zswap does not use more than one dst buffer yet. Follow-up patches
> > will actually utilize the multiple acomp_ctx buffers for batch
> > compression/decompression of multiple pages.
> >
> > Thus, ZSWAP_MAX_BATCH_SIZE limits the amount of extra memory used
> for
> > batching. There is a small extra memory overhead of allocating
> > the acomp_ctx->buffers array for compressors that do not support
> > batching: On x86_64, the overhead is 1 pointer per-CPU (i.e. 8 bytes).
>
> Support batching when storing large folios in zswap. If the underlying
> compressor supports batching (e.g. HW 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 HW compression batching.
>
> >
> > Next, we store the folio in batches:
> >
> > This patch modifies zswap_store() to store a batch of pages in large
> > folios at a time, instead of storing one page at a time. It does this by
> > calling a new procedure zswap_store_pages() with a range of indices in
> > the folio: for batching compressors, this range contains up to
> > pool->compr_batch_size pages. For non-batching compressors, we send up
> > to ZSWAP_MAX_BATCH_SIZE pages to be sequentially compressed and
> stored
> > in zswap_store_pages().
> >
> > zswap_store_pages() implements all the computes done earlier in
> > zswap_store_page() for a single-page, for multiple pages in a folio,
> > namely the "batch":
> >
> > 1) It starts by allocating all zswap entries required to store the
> > batch. New procedures, zswap_entries_cache_alloc_batch() and
> > zswap_entries_cache_free_batch() call kmem_cache_[free]alloc_bulk()
> > to optimize the performance of this step.
> >
> > 2) The entry doesn't have to be allocated on the same node as the page
> > being stored in zswap: we let the slab allocator decide this in
> > kmem_cache_alloc_bulk(). However, to make sure the current zswap
> > LRU list/shrinker behavior is preserved, we store the folio's nid as
> > a new @nid member in the entry to enable adding it to the correct
> > LRU list (and deleting it from the right LRU list). This ensures
> > that when the folio's allocating NUMA node is under memory
> > pressure, the entries corresponding to its pages are written back.
> >
> > The memory footprint of struct zswap_entry remains unchanged at
> > 56 bytes with the addition of the "int nid" member by condensing
> > "length" and "referenced" into 4 bytes using bit fields and using
> > the 4 bytes available after "referenced" for the "int nid". Thanks
> > to Nhat and Yosry for these suggestions!
> >
> > 3) Next, the entries fields are written, computes that need to be happen
> > anyway, without modifying the zswap xarray/LRU publishing order. This
> > avoids bringing the entries into the cache for writing in different
> > code blocks within this procedure, hence improves latency.
> >
> > 4) Next, it calls zswap_compress() to sequentially compress each page in
> > the batch.
> >
> > 5) Finally, it adds the batch's zswap entries to the xarray and LRU,
> > charges zswap memory and increments zswap stats.
> >
> > 6) The error handling and cleanup required for all failure scenarios
> > that can occur while storing a batch in zswap are consolidated to a
> > single "store_pages_failed" label in zswap_store_pages(). Here again,
> > we optimize performance by calling kmem_cache_free_bulk().
>
> Regardless of compression batching, always process large folios in
> batches. For HW 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.
Ok, will change this accordingly, thanks.
>
> >
> > This commit also makes a minor optimization in zswap_compress(), that
> > takes a "bool wb_enabled" argument; computed once in zswap_store()
> > rather than for each page in the folio.
> >
> > Suggested-by: Nhat Pham <nphamcs@gmail.com>
> > Suggested-by: Yosry Ahmed <yosry.ahmed@linux.dev>
> > Signed-off-by: Kanchana P Sridhar <kanchana.p.sridhar@intel.com>
> > ---
> > mm/zswap.c | 336 ++++++++++++++++++++++++++++++++++++-------------
> ----
> > 1 file changed, 232 insertions(+), 104 deletions(-)
> >
> > diff --git a/mm/zswap.c b/mm/zswap.c
> > index cb384eb7c815..257567edc587 100644
> > --- a/mm/zswap.c
> > +++ b/mm/zswap.c
> > @@ -82,6 +82,9 @@ 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
> > +
> > static int zswap_setup(void);
> >
> > /* Enable/disable zswap */
> > @@ -139,7 +142,7 @@ struct crypto_acomp_ctx {
> > struct crypto_acomp *acomp;
> > struct acomp_req *req;
> > struct crypto_wait wait;
> > - u8 *buffer;
> > + u8 **buffers;
> > struct mutex mutex;
> > bool is_sleepable;
> > };
> > @@ -149,6 +152,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;
> > @@ -158,6 +164,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. */
> > @@ -182,6 +189,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
> > @@ -189,8 +197,11 @@ static struct shrinker *zswap_shrinker;
> > */
> > struct zswap_entry {
> > swp_entry_t swpentry;
> > - unsigned int length;
> > - bool referenced;
> > + struct {
> > + unsigned int length:31;
> > + bool referenced:1;
> > + };
> > + int nid;
> > struct zswap_pool *pool;
> > unsigned long handle;
> > struct obj_cgroup *objcg;
> > @@ -242,8 +253,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;
> >
> > @@ -253,7 +266,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)
> > @@ -265,6 +282,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;
> > @@ -315,7 +333,9 @@ static struct zswap_pool *zswap_pool_create(char
> *compressor)
> > 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);
> > +
> > error:
> > if (pool->acomp_ctx)
> > free_percpu(pool->acomp_ctx);
> > @@ -353,7 +373,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);
> >
> > @@ -644,14 +665,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;
> >
> > /*
> > @@ -668,19 +683,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();
> > }
> >
> > @@ -740,6 +754,29 @@ static void zswap_entry_cache_free(struct
> zswap_entry *entry)
> > kmem_cache_free(zswap_entry_cache, entry);
> > }
> >
>
> Instead of this:
>
> > +/*
> > + * Returns 0 if kmem_cache_alloc_bulk() failed and a positive number
> otherwise.
> > + * The code for __kmem_cache_alloc_bulk() indicates that this positive
> number
> > + * will be the @size requested, i.e., @nr_entries.
> > + */
> > +static __always_inline int zswap_entries_cache_alloc_batch(void
> **entries,
> > + unsigned int
> nr_entries,
> > + gfp_t gfp)
> > +{
> > + int nr_alloc = kmem_cache_alloc_bulk(zswap_entry_cache, gfp,
> > + nr_entries, entries);
> > +
>
> Add this here:
> /*
> * kmem_cache_alloc_bulk() should return nr_entries on success
> * and 0 on failure.
> */
>
Sure.
> > + WARN_ON(!nr_alloc || (nr_alloc != nr_entries));
>
> WARN_ON_ONCE() is sufficient, and why do we WARN if
> kmem_cache_alloc_bulk() fails? I thought that was expected in some
> cases.
I can change this to a WARN_ON_ONCE(). The code for kmem_cache_alloc_bulk()
makes sure that either all entries are allocated, or none are allocated
(partial allocations are freed and 0 returned in case of the latter). It can be expected
to fail based on this.
I believe there was an earlier comment for which I added the WARN_ON? I can
either change this to WARN_ON_ONCE() or drop the WARN_ON_ONCE(), since
we anyway have a fallback mechanism.
>
> > +
> > + return nr_alloc;
> > +}
> > +
>
> Please document that it's okay use this to free entries allocated
> separately by zswap_entry_cache_alloc().
Sure.
>
> > +static __always_inline void zswap_entries_cache_free_batch(void
> **entries,
> > + unsigned int
> nr_entries)
> > +{
> > + kmem_cache_free_bulk(zswap_entry_cache, nr_entries, entries);
> > +}
> > +
> > /*
> > * Carries out the common pattern of freeing an entry's zsmalloc allocation,
> > * freeing the entry itself, and decrementing the number of stored pages.
> > @@ -766,7 +803,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
> > @@ -775,11 +814,7 @@ static int zswap_cpu_comp_prepare(unsigned int
> cpu, struct hlist_node *node)
> > if (!IS_ERR_OR_NULL(acomp_ctx->acomp))
> > 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));
> > @@ -788,20 +823,39 @@ static int zswap_cpu_comp_prepare(unsigned int
> cpu, struct hlist_node *node)
> > }
> > acomp_ctx->is_sleepable = acomp_is_async(acomp_ctx->acomp);
> >
> > + /*
> > + * 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));
> > +
> > 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);
> >
> > @@ -811,12 +865,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;
> > @@ -830,7 +884,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);
> >
> > @@ -860,8 +914,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;
> > }
> > @@ -906,7 +959,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, acomp_ctx-
> >buffer);
> > + obj = zs_obj_read_begin(pool->zs_pool, entry->handle, acomp_ctx-
> >buffers[0]);
> >
> > /* zswap entries of length PAGE_SIZE are not compressed. */
> > if (entry->length == PAGE_SIZE) {
> > @@ -916,15 +969,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);
> > @@ -1378,95 +1431,156 @@ 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 obj_cgroup *objcg,
> > + struct zswap_pool *pool,
> > + int nid,
> > + bool wb_enabled)
> > {
> > - swp_entry_t page_swpentry = page_swap_entry(page);
> > - struct zswap_entry *entry, *old;
> > -
> > - /* allocate entry */
> > - entry = zswap_entry_cache_alloc(GFP_KERNEL, page_to_nid(page));
> > - if (!entry) {
> > - zswap_reject_kmemcache_fail++;
> > - return false;
> > + struct zswap_entry *entries[ZSWAP_MAX_BATCH_SIZE];
> > + u8 i, store_fail_idx = 0, nr_pages = end - start;
> > +
> > + VM_WARN_ON_ONCE(nr_pages > ZSWAP_MAX_BATCH_SIZE);
> > +
> > + if (unlikely(!zswap_entries_cache_alloc_batch((void **)&entries[0],
>
> Is this equivalent to just passing in 'entries'?
It is, however, I wanted to keep this equivalent to the failure case call to
zswap_entries_cache_free_batch(), that passes in the address of the
batch index that failed xarray store.
>
> > + nr_pages, GFP_KERNEL)))
> {
> > + for (i = 0; i < nr_pages; ++i) {
> > + entries[i] = zswap_entry_cache_alloc(GFP_KERNEL,
> nid);
> > +
> > + if (unlikely(!entries[i])) {
> > + zswap_reject_kmemcache_fail++;
> > + /*
> > + * While handling this error, we only need to
> > + * call zswap_entries_cache_free_batch() for
> > + * entries[0 .. @i-1].
> > + */
> > + nr_pages = i;
> > + goto store_pages_failed;
> > + }
> > + }
>
>
> Maybe move the fallback loop into zswap_entries_cache_alloc_batch()?
I could, however, I would need to modify the API to return the error index "i",
so that the "goto store_pages_failed" works. Imo, inlining this makes the error
handling more apparent, but let me know.
>
> > }
> >
> > - if (!zswap_compress(page, entry, pool))
> > - goto compress_failed;
> > + /*
> > + * We colocate entry initialization as much as possible here to
> > + * minimize potential cache misses.
>
> s/colocate/co-locate
>
> Please only keep the portion above and drop the rest of the comment.
Ok.
>
> > + *
> > + * With kmem_cache_alloc_bulk(), the batch's entries will be created
> > + * on the NUMA node of the CPU on which zswap_store() is called,
> which
> > + * might not be the same as @nid, the NUMA node on which @folio
> was
> > + * allocated. In order for the @folio's entries to be written back when
> > + * @nid experiences memory pressure, we store @nid in @entry-
> >nid.
> > + * This ensures that the entry is added to and deleted from the LRU
> > + * list of the correct node, namely @nid.
> > + */
> > + 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;
> > }
> [..]
next prev parent reply other threads:[~2025-12-12 1:44 UTC|newest]
Thread overview: 79+ messages / expand[flat|nested] mbox.gz Atom feed top
2025-11-04 9:12 [PATCH v13 00/22] zswap compression batching with optimized iaa_crypto driver Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 01/22] crypto: iaa - Reorganize the iaa_crypto driver code Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 02/22] crypto: iaa - New architecture for IAA device WQ comp/decomp usage & core mapping Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 03/22] crypto: iaa - Simplify, consistency of function parameters, minor stats bug fix Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 04/22] crypto: iaa - Descriptor allocation timeouts with mitigations Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 05/22] crypto: iaa - iaa_wq uses percpu_refs for get/put reference counting Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 06/22] crypto: iaa - Simplify the code flow in iaa_compress() and iaa_decompress() Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 07/22] crypto: iaa - Refactor hardware descriptor setup into separate procedures Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 08/22] crypto: iaa - Simplified, efficient job submissions for non-irq mode Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 09/22] crypto: iaa - Deprecate exporting add/remove IAA compression modes Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 10/22] crypto: iaa - Expect a single scatterlist for a [de]compress request's src/dst Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 11/22] crypto: iaa - Rearchitect iaa_crypto to have clean interfaces with crypto_acomp Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 12/22] crypto: acomp - Define a unit_size in struct acomp_req to enable batching Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 13/22] crypto: iaa - IAA Batching for parallel compressions/decompressions Kanchana P Sridhar
2025-11-14 9:59 ` Herbert Xu
2025-11-16 18:53 ` Sridhar, Kanchana P
2025-11-17 3:12 ` Herbert Xu
2025-11-17 5:47 ` Sridhar, Kanchana P
2025-11-04 9:12 ` [PATCH v13 14/22] crypto: iaa - Enable async mode and make it the default Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 15/22] crypto: iaa - Disable iaa_verify_compress by default Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 16/22] crypto: iaa - Submit the two largest source buffers first in decompress batching Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 17/22] crypto: iaa - Add deflate-iaa-dynamic compression mode Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 18/22] crypto: acomp - Add crypto_acomp_batch_size() to get an algorithm's batch-size Kanchana P Sridhar
2025-11-04 9:12 ` [PATCH v13 19/22] mm: zswap: Per-CPU acomp_ctx resources exist from pool creation to deletion Kanchana P Sridhar
2025-11-13 20:24 ` Yosry Ahmed
2025-12-12 0:55 ` Sridhar, Kanchana P
2025-12-12 1:06 ` Yosry Ahmed
2025-12-12 1:58 ` Sridhar, Kanchana P
2025-12-12 2:47 ` Yosry Ahmed
2025-12-12 4:32 ` Sridhar, Kanchana P
2025-12-12 18:17 ` Sridhar, Kanchana P
2025-12-12 18:43 ` Yosry Ahmed
2025-12-12 20:53 ` Sridhar, Kanchana P
2025-12-12 22:25 ` Yosry Ahmed
2025-12-13 19:53 ` Sridhar, Kanchana P
2025-11-04 9:12 ` [PATCH v13 20/22] mm: zswap: Consistently use IS_ERR_OR_NULL() to check acomp_ctx resources Kanchana P Sridhar
2025-11-13 20:25 ` Yosry Ahmed
2025-12-12 1:07 ` Sridhar, Kanchana P
2025-11-04 9:12 ` [PATCH v13 21/22] mm: zswap: zswap_store() will process a large folio in batches Kanchana P Sridhar
2025-11-06 17:45 ` Nhat Pham
2025-11-07 2:28 ` Sridhar, Kanchana P
2025-11-13 20:52 ` Yosry Ahmed
2025-11-13 20:51 ` Yosry Ahmed
2025-12-12 1:43 ` Sridhar, Kanchana P [this message]
2025-12-12 4:40 ` Yosry Ahmed
2025-12-12 18:03 ` Sridhar, Kanchana P
2025-11-04 9:12 ` [PATCH v13 22/22] mm: zswap: Batched zswap_compress() with compress batching of large folios Kanchana P Sridhar
2025-11-13 21:34 ` Yosry Ahmed
2025-11-13 23:55 ` Sridhar, Kanchana P
2025-11-14 0:46 ` Yosry Ahmed
2025-12-19 2:29 ` Sridhar, Kanchana P
2025-12-19 15:26 ` Yosry Ahmed
2025-12-19 19:03 ` Sridhar, Kanchana P
2025-11-14 5:52 ` Yosry Ahmed
2025-11-14 6:43 ` Sridhar, Kanchana P
2025-11-14 15:37 ` Yosry Ahmed
2025-11-14 19:23 ` Sridhar, Kanchana P
2025-11-14 19:44 ` Yosry Ahmed
2025-11-14 19:59 ` Sridhar, Kanchana P
2025-11-14 20:49 ` Yosry Ahmed
2025-11-26 5:46 ` Herbert Xu
2025-11-26 6:34 ` Yosry Ahmed
2025-11-26 20:05 ` Sridhar, Kanchana P
2025-12-08 3:23 ` Herbert Xu
2025-12-08 4:17 ` Sridhar, Kanchana P
2025-12-08 4:24 ` Herbert Xu
2025-12-08 4:33 ` Sridhar, Kanchana P
2025-12-09 1:15 ` Yosry Ahmed
2025-12-09 2:32 ` Herbert Xu
2025-12-09 16:55 ` Yosry Ahmed
2025-12-09 17:21 ` Sridhar, Kanchana P
2025-12-09 17:31 ` Yosry Ahmed
2025-12-09 19:38 ` Sridhar, Kanchana P
2025-12-10 16:01 ` Yosry Ahmed
2025-12-10 18:47 ` Sridhar, Kanchana P
2025-12-10 4:28 ` Herbert Xu
2025-12-10 5:36 ` Sridhar, Kanchana P
2025-12-10 15:53 ` Yosry Ahmed
2025-11-13 18:14 ` [PATCH v13 00/22] zswap compression batching with optimized iaa_crypto driver 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=SJ2PR11MB8472875DB2900920EB2C51DCC9AEA@SJ2PR11MB8472.namprd11.prod.outlook.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=kasong@tencent.com \
--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=sj@kernel.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