* [PATCH v8 00/17] zsmalloc/zram: there be preemption
@ 2025-02-21 22:25 Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 01/17] zram: sleepable entry locking Sergey Senozhatsky
` (16 more replies)
0 siblings, 17 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
Currently zram runs compression and decompression in non-preemptible
sections, e.g.
zcomp_stream_get() // grabs CPU local lock
zcomp_compress()
or
zram_slot_lock() // grabs entry spin-lock
zcomp_stream_get() // grabs CPU local lock
zs_map_object() // grabs rwlock and CPU local lock
zcomp_decompress()
Potentially a little troublesome for a number of reasons.
For instance, this makes it impossible to use async compression
algorithms or/and H/W compression algorithms, which can wait for OP
completion or resource availability. This also restricts what
compression algorithms can do internally, for example, zstd can
allocate internal state memory for C/D dictionaries:
do_fsync()
do_writepages()
zram_bio_write()
zram_write_page() // become non-preemptible
zcomp_compress()
zstd_compress()
ZSTD_compress_usingCDict()
ZSTD_compressBegin_usingCDict_internal()
ZSTD_resetCCtx_usingCDict()
ZSTD_resetCCtx_internal()
zstd_custom_alloc() // memory allocation
Not to mention that the system can be configured to maximize
compression ratio at a cost of CPU/HW time (e.g. lz4hc or deflate
with very high compression level) so zram can stay in non-preemptible
section (even under spin-lock or/and rwlock) for an extended period
of time. Aside from compression algorithms, this also restricts what
zram can do. One particular example is zram_write_page() zsmalloc
handle allocation, which has an optimistic allocation (disallowing
direct reclaim) and a pessimistic fallback path, which then forces
zram to compress the page one more time.
This series changes zram to not directly impose atomicity restrictions
on compression algorithms (and on itself), which makes zram write()
fully preemptible; zram read(), sadly, is not always preemptible yet.
There are still indirect atomicity restrictions imposed by zsmalloc().
One notable example is object mapping API, which returns with:
a) local CPU lock held
b) zspage rwlock held
First, zsmalloc's zspage lock is converted from rwlock to a special
type of RW-lookalike look with some extra guarantees/features. Second,
a new handle mapping is introduced which doesn't use per-CPU buffers
(and hence no local CPU lock), does fewer memcpy() calls, but requires
users to provide a pointer to temp buffer for object copy-in (when
needed). Third, zram is converted to the new zsmalloc mapping API
and thus zram read() becomes preemptible.
v7 -> v8
- also provide helpers for lockdep class_lock to remove even
more ifdef-s (Yosry)
- moved zsmalloc lockdep class_lock registration so that on
error we don't un-register a not-yet-registered class
- tweaked some commit messages
Sergey Senozhatsky (17):
zram: sleepable entry locking
zram: permit preemption with active compression stream
zram: remove unused crypto include
zram: remove max_comp_streams device attr
zram: remove second stage of handle allocation
zram: remove writestall zram_stats member
zram: limit max recompress prio to num_active_comps
zram: filter out recomp targets based on priority
zram: rework recompression loop
zsmalloc: rename pool lock
zsmalloc: make zspage lock preemptible
zsmalloc: introduce new object mapping API
zram: switch to new zsmalloc object mapping API
zram: permit reclaim in zstd custom allocator
zram: do not leak page on recompress_store error path
zram: do not leak page on writeback_store error path
zram: add might_sleep to zcomp API
Documentation/ABI/testing/sysfs-block-zram | 8 -
Documentation/admin-guide/blockdev/zram.rst | 36 +-
drivers/block/zram/backend_zstd.c | 11 +-
drivers/block/zram/zcomp.c | 48 ++-
drivers/block/zram/zcomp.h | 8 +-
drivers/block/zram/zram_drv.c | 283 ++++++++--------
drivers/block/zram/zram_drv.h | 22 +-
include/linux/zsmalloc.h | 8 +
mm/zsmalloc.c | 351 ++++++++++++++++----
9 files changed, 488 insertions(+), 287 deletions(-)
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 01/17] zram: sleepable entry locking
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-24 8:19 ` Sebastian Andrzej Siewior
2025-02-21 22:25 ` [PATCH v8 02/17] zram: permit preemption with active compression stream Sergey Senozhatsky
` (15 subsequent siblings)
16 siblings, 1 reply; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
Concurrent modifications of meta table entries is now handled
by per-entry spin-lock. This has a number of shortcomings.
First, this imposes atomic requirements on compression backends.
zram can call both zcomp_compress() and zcomp_decompress() under
entry spin-lock, which implies that we can use only compression
algorithms that don't schedule/sleep/wait during compression and
decompression. This, for instance, makes it impossible to use
some of the ASYNC compression algorithms (H/W compression, etc.)
implementations.
Second, this can potentially trigger watchdogs. For example,
entry re-compression with secondary algorithms is performed
under entry spin-lock. Given that we chain secondary
compression algorithms and that some of them can be configured
for best compression ratio (and worst compression speed) zram
can stay under spin-lock for quite some time.
Having a per-entry mutex (or, for instance, a rw-semaphore)
significantly increases sizeof() of each entry and hence the
meta table. Therefore entry locking returns back to bit
locking, as before, however, this time also preempt-rt friendly,
because if waits-on-bit instead of spinning-on-bit. Lock owners
are also now permitted to schedule, which is a first step on the
path of making zram non-atomic.
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zram_drv.c | 62 ++++++++++++++++++++++++++++-------
drivers/block/zram/zram_drv.h | 20 +++++++----
2 files changed, 65 insertions(+), 17 deletions(-)
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index 9f5020b077c5..37c5651305c2 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -58,19 +58,62 @@ static void zram_free_page(struct zram *zram, size_t index);
static int zram_read_from_zspool(struct zram *zram, struct page *page,
u32 index);
-static int zram_slot_trylock(struct zram *zram, u32 index)
+#ifdef CONFIG_DEBUG_LOCK_ALLOC
+#define slot_dep_map(zram, index) (&(zram)->table[(index)].dep_map)
+#define zram_lock_class(zram) (&(zram)->lock_class)
+#else
+#define slot_dep_map(zram, index) NULL
+#define zram_lock_class(zram) NULL
+#endif
+
+static void zram_slot_lock_init(struct zram *zram, u32 index)
{
- return spin_trylock(&zram->table[index].lock);
+ lockdep_init_map(slot_dep_map(zram, index),
+ "zram->table[index].lock",
+ zram_lock_class(zram), 0);
+}
+
+/*
+ * entry locking rules:
+ *
+ * 1) Lock is exclusive
+ *
+ * 2) lock() function can sleep waiting for the lock
+ *
+ * 3) Lock owner can sleep
+ *
+ * 4) Use TRY lock variant when in atomic context
+ * - must check return value and handle locking failers
+ */
+static __must_check bool zram_slot_trylock(struct zram *zram, u32 index)
+{
+ unsigned long *lock = &zram->table[index].flags;
+
+ if (!test_and_set_bit_lock(ZRAM_ENTRY_LOCK, lock)) {
+ mutex_acquire(slot_dep_map(zram, index), 0, 1, _RET_IP_);
+ lock_acquired(slot_dep_map(zram, index), _RET_IP_);
+ return true;
+ }
+
+ lock_contended(slot_dep_map(zram, index), _RET_IP_);
+ return false;
}
static void zram_slot_lock(struct zram *zram, u32 index)
{
- spin_lock(&zram->table[index].lock);
+ unsigned long *lock = &zram->table[index].flags;
+
+ mutex_acquire(slot_dep_map(zram, index), 0, 0, _RET_IP_);
+ wait_on_bit_lock(lock, ZRAM_ENTRY_LOCK, TASK_UNINTERRUPTIBLE);
+ lock_acquired(slot_dep_map(zram, index), _RET_IP_);
}
static void zram_slot_unlock(struct zram *zram, u32 index)
{
- spin_unlock(&zram->table[index].lock);
+ unsigned long *lock = &zram->table[index].flags;
+
+ mutex_release(slot_dep_map(zram, index), _RET_IP_);
+ clear_and_wake_up_bit(ZRAM_ENTRY_LOCK, lock);
}
static inline bool init_done(struct zram *zram)
@@ -93,7 +136,6 @@ static void zram_set_handle(struct zram *zram, u32 index, unsigned long handle)
zram->table[index].handle = handle;
}
-/* flag operations require table entry bit_spin_lock() being held */
static bool zram_test_flag(struct zram *zram, u32 index,
enum zram_pageflags flag)
{
@@ -1473,15 +1515,11 @@ static bool zram_meta_alloc(struct zram *zram, u64 disksize)
huge_class_size = zs_huge_class_size(zram->mem_pool);
for (index = 0; index < num_pages; index++)
- spin_lock_init(&zram->table[index].lock);
+ zram_slot_lock_init(zram, index);
+
return true;
}
-/*
- * To protect concurrent access to the same index entry,
- * caller should hold this table index entry's bit_spinlock to
- * indicate this index entry is accessing.
- */
static void zram_free_page(struct zram *zram, size_t index)
{
unsigned long handle;
@@ -2625,6 +2663,7 @@ static int zram_add(void)
if (ret)
goto out_cleanup_disk;
+ lockdep_register_key(zram_lock_class(zram));
zram_debugfs_register(zram);
pr_info("Added device: %s\n", zram->disk->disk_name);
return device_id;
@@ -2653,6 +2692,7 @@ static int zram_remove(struct zram *zram)
zram->claim = true;
mutex_unlock(&zram->disk->open_mutex);
+ lockdep_unregister_key(zram_lock_class(zram));
zram_debugfs_unregister(zram);
if (claimed) {
diff --git a/drivers/block/zram/zram_drv.h b/drivers/block/zram/zram_drv.h
index db78d7c01b9a..794c9234e627 100644
--- a/drivers/block/zram/zram_drv.h
+++ b/drivers/block/zram/zram_drv.h
@@ -28,7 +28,6 @@
#define ZRAM_SECTOR_PER_LOGICAL_BLOCK \
(1 << (ZRAM_LOGICAL_BLOCK_SHIFT - SECTOR_SHIFT))
-
/*
* ZRAM is mainly used for memory efficiency so we want to keep memory
* footprint small and thus squeeze size and zram pageflags into a flags
@@ -46,6 +45,7 @@
/* Flags for zram pages (table[page_no].flags) */
enum zram_pageflags {
ZRAM_SAME = ZRAM_FLAG_SHIFT, /* Page consists the same element */
+ ZRAM_ENTRY_LOCK, /* entry access lock bit */
ZRAM_WB, /* page is stored on backing_device */
ZRAM_PP_SLOT, /* Selected for post-processing */
ZRAM_HUGE, /* Incompressible page */
@@ -58,13 +58,18 @@ enum zram_pageflags {
__NR_ZRAM_PAGEFLAGS,
};
-/*-- Data structures */
-
-/* Allocated for each disk page */
+/*
+ * Allocated for each disk page. We use bit-lock (ZRAM_ENTRY_LOCK bit
+ * of flags) to save memory. There can be plenty of entries and standard
+ * locking primitives (e.g. mutex) will significantly increase sizeof()
+ * of each entry and hence of the meta table.
+ */
struct zram_table_entry {
unsigned long handle;
- unsigned int flags;
- spinlock_t lock;
+ unsigned long flags;
+#ifdef CONFIG_DEBUG_LOCK_ALLOC
+ struct lockdep_map dep_map;
+#endif
#ifdef CONFIG_ZRAM_TRACK_ENTRY_ACTIME
ktime_t ac_time;
#endif
@@ -137,5 +142,8 @@ struct zram {
struct dentry *debugfs_dir;
#endif
atomic_t pp_in_progress;
+#ifdef CONFIG_DEBUG_LOCK_ALLOC
+ struct lock_class_key lock_class;
+#endif
};
#endif
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 02/17] zram: permit preemption with active compression stream
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 01/17] zram: sleepable entry locking Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 03/17] zram: remove unused crypto include Sergey Senozhatsky
` (14 subsequent siblings)
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
Currently, per-CPU stream access is done from a non-preemptible
(atomic) section, which imposes the same atomicity requirements on
compression backends as entry spin-lock, and makes it impossible
to use algorithms that can schedule/wait/sleep during compression
and decompression.
Switch to preemptible per-CPU model, similar to the one used
in zswap. Instead of a per-CPU local lock, each stream carries
a mutex which is locked throughout entire time zram uses it
for compression or decompression, so that cpu-dead event waits
for zram to stop using a particular per-CPU stream and release
it.
Suggested-by: Yosry Ahmed <yosry.ahmed@linux.dev>
Reviewed-by: Yosry Ahmed <yosry.ahmed@linux.dev>
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zcomp.c | 41 +++++++++++++++++++++++++----------
drivers/block/zram/zcomp.h | 6 ++---
drivers/block/zram/zram_drv.c | 20 ++++++++---------
3 files changed, 42 insertions(+), 25 deletions(-)
diff --git a/drivers/block/zram/zcomp.c b/drivers/block/zram/zcomp.c
index bb514403e305..53e4c37441be 100644
--- a/drivers/block/zram/zcomp.c
+++ b/drivers/block/zram/zcomp.c
@@ -6,7 +6,7 @@
#include <linux/slab.h>
#include <linux/wait.h>
#include <linux/sched.h>
-#include <linux/cpu.h>
+#include <linux/cpuhotplug.h>
#include <linux/crypto.h>
#include <linux/vmalloc.h>
@@ -109,13 +109,29 @@ ssize_t zcomp_available_show(const char *comp, char *buf)
struct zcomp_strm *zcomp_stream_get(struct zcomp *comp)
{
- local_lock(&comp->stream->lock);
- return this_cpu_ptr(comp->stream);
+ for (;;) {
+ struct zcomp_strm *zstrm = raw_cpu_ptr(comp->stream);
+
+ /*
+ * Inspired by zswap
+ *
+ * stream is returned with ->mutex locked which prevents
+ * cpu_dead() from releasing this stream under us, however
+ * there is still a race window between raw_cpu_ptr() and
+ * mutex_lock(), during which we could have been migrated
+ * from a CPU that has already destroyed its stream. If
+ * so then unlock and re-try on the current CPU.
+ */
+ mutex_lock(&zstrm->lock);
+ if (likely(zstrm->buffer))
+ return zstrm;
+ mutex_unlock(&zstrm->lock);
+ }
}
-void zcomp_stream_put(struct zcomp *comp)
+void zcomp_stream_put(struct zcomp_strm *zstrm)
{
- local_unlock(&comp->stream->lock);
+ mutex_unlock(&zstrm->lock);
}
int zcomp_compress(struct zcomp *comp, struct zcomp_strm *zstrm,
@@ -151,12 +167,9 @@ int zcomp_decompress(struct zcomp *comp, struct zcomp_strm *zstrm,
int zcomp_cpu_up_prepare(unsigned int cpu, struct hlist_node *node)
{
struct zcomp *comp = hlist_entry(node, struct zcomp, node);
- struct zcomp_strm *zstrm;
+ struct zcomp_strm *zstrm = per_cpu_ptr(comp->stream, cpu);
int ret;
- zstrm = per_cpu_ptr(comp->stream, cpu);
- local_lock_init(&zstrm->lock);
-
ret = zcomp_strm_init(comp, zstrm);
if (ret)
pr_err("Can't allocate a compression stream\n");
@@ -166,16 +179,17 @@ int zcomp_cpu_up_prepare(unsigned int cpu, struct hlist_node *node)
int zcomp_cpu_dead(unsigned int cpu, struct hlist_node *node)
{
struct zcomp *comp = hlist_entry(node, struct zcomp, node);
- struct zcomp_strm *zstrm;
+ struct zcomp_strm *zstrm = per_cpu_ptr(comp->stream, cpu);
- zstrm = per_cpu_ptr(comp->stream, cpu);
+ mutex_lock(&zstrm->lock);
zcomp_strm_free(comp, zstrm);
+ mutex_unlock(&zstrm->lock);
return 0;
}
static int zcomp_init(struct zcomp *comp, struct zcomp_params *params)
{
- int ret;
+ int ret, cpu;
comp->stream = alloc_percpu(struct zcomp_strm);
if (!comp->stream)
@@ -186,6 +200,9 @@ static int zcomp_init(struct zcomp *comp, struct zcomp_params *params)
if (ret)
goto cleanup;
+ for_each_possible_cpu(cpu)
+ mutex_init(&per_cpu_ptr(comp->stream, cpu)->lock);
+
ret = cpuhp_state_add_instance(CPUHP_ZCOMP_PREPARE, &comp->node);
if (ret < 0)
goto cleanup;
diff --git a/drivers/block/zram/zcomp.h b/drivers/block/zram/zcomp.h
index ad5762813842..23b8236b9090 100644
--- a/drivers/block/zram/zcomp.h
+++ b/drivers/block/zram/zcomp.h
@@ -3,7 +3,7 @@
#ifndef _ZCOMP_H_
#define _ZCOMP_H_
-#include <linux/local_lock.h>
+#include <linux/mutex.h>
#define ZCOMP_PARAM_NO_LEVEL INT_MIN
@@ -31,7 +31,7 @@ struct zcomp_ctx {
};
struct zcomp_strm {
- local_lock_t lock;
+ struct mutex lock;
/* compression buffer */
void *buffer;
struct zcomp_ctx ctx;
@@ -77,7 +77,7 @@ struct zcomp *zcomp_create(const char *alg, struct zcomp_params *params);
void zcomp_destroy(struct zcomp *comp);
struct zcomp_strm *zcomp_stream_get(struct zcomp *comp);
-void zcomp_stream_put(struct zcomp *comp);
+void zcomp_stream_put(struct zcomp_strm *zstrm);
int zcomp_compress(struct zcomp *comp, struct zcomp_strm *zstrm,
const void *src, unsigned int *dst_len);
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index 37c5651305c2..1b5bb206239f 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -1613,7 +1613,7 @@ static int read_compressed_page(struct zram *zram, struct page *page, u32 index)
ret = zcomp_decompress(zram->comps[prio], zstrm, src, size, dst);
kunmap_local(dst);
zs_unmap_object(zram->mem_pool, handle);
- zcomp_stream_put(zram->comps[prio]);
+ zcomp_stream_put(zstrm);
return ret;
}
@@ -1774,14 +1774,14 @@ static int zram_write_page(struct zram *zram, struct page *page, u32 index)
kunmap_local(mem);
if (unlikely(ret)) {
- zcomp_stream_put(zram->comps[ZRAM_PRIMARY_COMP]);
+ zcomp_stream_put(zstrm);
pr_err("Compression failed! err=%d\n", ret);
zs_free(zram->mem_pool, handle);
return ret;
}
if (comp_len >= huge_class_size) {
- zcomp_stream_put(zram->comps[ZRAM_PRIMARY_COMP]);
+ zcomp_stream_put(zstrm);
return write_incompressible_page(zram, page, index);
}
@@ -1805,7 +1805,7 @@ static int zram_write_page(struct zram *zram, struct page *page, u32 index)
__GFP_HIGHMEM |
__GFP_MOVABLE);
if (IS_ERR_VALUE(handle)) {
- zcomp_stream_put(zram->comps[ZRAM_PRIMARY_COMP]);
+ zcomp_stream_put(zstrm);
atomic64_inc(&zram->stats.writestall);
handle = zs_malloc(zram->mem_pool, comp_len,
GFP_NOIO | __GFP_HIGHMEM |
@@ -1817,7 +1817,7 @@ static int zram_write_page(struct zram *zram, struct page *page, u32 index)
}
if (!zram_can_store_page(zram)) {
- zcomp_stream_put(zram->comps[ZRAM_PRIMARY_COMP]);
+ zcomp_stream_put(zstrm);
zs_free(zram->mem_pool, handle);
return -ENOMEM;
}
@@ -1825,7 +1825,7 @@ static int zram_write_page(struct zram *zram, struct page *page, u32 index)
dst = zs_map_object(zram->mem_pool, handle, ZS_MM_WO);
memcpy(dst, zstrm->buffer, comp_len);
- zcomp_stream_put(zram->comps[ZRAM_PRIMARY_COMP]);
+ zcomp_stream_put(zstrm);
zs_unmap_object(zram->mem_pool, handle);
zram_slot_lock(zram, index);
@@ -1984,7 +1984,7 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
kunmap_local(src);
if (ret) {
- zcomp_stream_put(zram->comps[prio]);
+ zcomp_stream_put(zstrm);
return ret;
}
@@ -1994,7 +1994,7 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
/* Continue until we make progress */
if (class_index_new >= class_index_old ||
(threshold && comp_len_new >= threshold)) {
- zcomp_stream_put(zram->comps[prio]);
+ zcomp_stream_put(zstrm);
continue;
}
@@ -2052,13 +2052,13 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
__GFP_HIGHMEM |
__GFP_MOVABLE);
if (IS_ERR_VALUE(handle_new)) {
- zcomp_stream_put(zram->comps[prio]);
+ zcomp_stream_put(zstrm);
return PTR_ERR((void *)handle_new);
}
dst = zs_map_object(zram->mem_pool, handle_new, ZS_MM_WO);
memcpy(dst, zstrm->buffer, comp_len_new);
- zcomp_stream_put(zram->comps[prio]);
+ zcomp_stream_put(zstrm);
zs_unmap_object(zram->mem_pool, handle_new);
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 03/17] zram: remove unused crypto include
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 01/17] zram: sleepable entry locking Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 02/17] zram: permit preemption with active compression stream Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 04/17] zram: remove max_comp_streams device attr Sergey Senozhatsky
` (13 subsequent siblings)
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
We stopped using crypto API (for the time being), so remove
its include and replace CRYPTO_MAX_ALG_NAME with a local
define.
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zcomp.c | 1 -
drivers/block/zram/zram_drv.c | 4 +++-
drivers/block/zram/zram_drv.h | 1 -
3 files changed, 3 insertions(+), 3 deletions(-)
diff --git a/drivers/block/zram/zcomp.c b/drivers/block/zram/zcomp.c
index 53e4c37441be..cfdde2e0748a 100644
--- a/drivers/block/zram/zcomp.c
+++ b/drivers/block/zram/zcomp.c
@@ -7,7 +7,6 @@
#include <linux/wait.h>
#include <linux/sched.h>
#include <linux/cpuhotplug.h>
-#include <linux/crypto.h>
#include <linux/vmalloc.h>
#include "zcomp.h"
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index 1b5bb206239f..c73d8024f48f 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -44,6 +44,8 @@ static DEFINE_MUTEX(zram_index_mutex);
static int zram_major;
static const char *default_compressor = CONFIG_ZRAM_DEF_COMP;
+#define ZRAM_MAX_ALGO_NAME_SZ 128
+
/* Module params (documentation at end) */
static unsigned int num_devices = 1;
/*
@@ -1154,7 +1156,7 @@ static int __comp_algorithm_store(struct zram *zram, u32 prio, const char *buf)
size_t sz;
sz = strlen(buf);
- if (sz >= CRYPTO_MAX_ALG_NAME)
+ if (sz >= ZRAM_MAX_ALGO_NAME_SZ)
return -E2BIG;
compressor = kstrdup(buf, GFP_KERNEL);
diff --git a/drivers/block/zram/zram_drv.h b/drivers/block/zram/zram_drv.h
index 794c9234e627..2c380ea9a816 100644
--- a/drivers/block/zram/zram_drv.h
+++ b/drivers/block/zram/zram_drv.h
@@ -17,7 +17,6 @@
#include <linux/rwsem.h>
#include <linux/zsmalloc.h>
-#include <linux/crypto.h>
#include "zcomp.h"
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 04/17] zram: remove max_comp_streams device attr
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (2 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 03/17] zram: remove unused crypto include Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 05/17] zram: remove second stage of handle allocation Sergey Senozhatsky
` (12 subsequent siblings)
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
max_comp_streams device attribute has been defunct since
May 2016 when zram switched to per-CPU compression streams,
remove it.
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
Documentation/ABI/testing/sysfs-block-zram | 8 -----
Documentation/admin-guide/blockdev/zram.rst | 36 ++++++---------------
drivers/block/zram/zram_drv.c | 23 -------------
3 files changed, 10 insertions(+), 57 deletions(-)
diff --git a/Documentation/ABI/testing/sysfs-block-zram b/Documentation/ABI/testing/sysfs-block-zram
index 1ef69e0271f9..36c57de0a10a 100644
--- a/Documentation/ABI/testing/sysfs-block-zram
+++ b/Documentation/ABI/testing/sysfs-block-zram
@@ -22,14 +22,6 @@ Description:
device. The reset operation frees all the memory associated
with this device.
-What: /sys/block/zram<id>/max_comp_streams
-Date: February 2014
-Contact: Sergey Senozhatsky <sergey.senozhatsky@gmail.com>
-Description:
- The max_comp_streams file is read-write and specifies the
- number of backend's zcomp_strm compression streams (number of
- concurrent compress operations).
-
What: /sys/block/zram<id>/comp_algorithm
Date: February 2014
Contact: Sergey Senozhatsky <sergey.senozhatsky@gmail.com>
diff --git a/Documentation/admin-guide/blockdev/zram.rst b/Documentation/admin-guide/blockdev/zram.rst
index 714a5171bfc0..7ad4c86f8258 100644
--- a/Documentation/admin-guide/blockdev/zram.rst
+++ b/Documentation/admin-guide/blockdev/zram.rst
@@ -54,7 +54,7 @@ The list of possible return codes:
If you use 'echo', the returned value is set by the 'echo' utility,
and, in general case, something like::
- echo 3 > /sys/block/zram0/max_comp_streams
+ echo foo > /sys/block/zram0/comp_algorithm
if [ $? -ne 0 ]; then
handle_error
fi
@@ -73,21 +73,7 @@ This creates 4 devices: /dev/zram{0,1,2,3}
num_devices parameter is optional and tells zram how many devices should be
pre-created. Default: 1.
-2) Set max number of compression streams
-========================================
-
-Regardless of the value passed to this attribute, ZRAM will always
-allocate multiple compression streams - one per online CPU - thus
-allowing several concurrent compression operations. The number of
-allocated compression streams goes down when some of the CPUs
-become offline. There is no single-compression-stream mode anymore,
-unless you are running a UP system or have only 1 CPU online.
-
-To find out how many streams are currently available::
-
- cat /sys/block/zram0/max_comp_streams
-
-3) Select compression algorithm
+2) Select compression algorithm
===============================
Using comp_algorithm device attribute one can see available and
@@ -107,7 +93,7 @@ Examples::
For the time being, the `comp_algorithm` content shows only compression
algorithms that are supported by zram.
-4) Set compression algorithm parameters: Optional
+3) Set compression algorithm parameters: Optional
=================================================
Compression algorithms may support specific parameters which can be
@@ -138,7 +124,7 @@ better the compression ratio, it even can take negatives values for some
algorithms), for other algorithms `level` is acceleration level (the higher
the value the lower the compression ratio).
-5) Set Disksize
+4) Set Disksize
===============
Set disk size by writing the value to sysfs node 'disksize'.
@@ -158,7 +144,7 @@ There is little point creating a zram of greater than twice the size of memory
since we expect a 2:1 compression ratio. Note that zram uses about 0.1% of the
size of the disk when not in use so a huge zram is wasteful.
-6) Set memory limit: Optional
+5) Set memory limit: Optional
=============================
Set memory limit by writing the value to sysfs node 'mem_limit'.
@@ -177,7 +163,7 @@ Examples::
# To disable memory limit
echo 0 > /sys/block/zram0/mem_limit
-7) Activate
+6) Activate
===========
::
@@ -188,7 +174,7 @@ Examples::
mkfs.ext4 /dev/zram1
mount /dev/zram1 /tmp
-8) Add/remove zram devices
+7) Add/remove zram devices
==========================
zram provides a control interface, which enables dynamic (on-demand) device
@@ -208,7 +194,7 @@ execute::
echo X > /sys/class/zram-control/hot_remove
-9) Stats
+8) Stats
========
Per-device statistics are exported as various nodes under /sys/block/zram<id>/
@@ -228,8 +214,6 @@ mem_limit WO specifies the maximum amount of memory ZRAM can
writeback_limit WO specifies the maximum amount of write IO zram
can write out to backing device as 4KB unit
writeback_limit_enable RW show and set writeback_limit feature
-max_comp_streams RW the number of possible concurrent compress
- operations
comp_algorithm RW show and change the compression algorithm
algorithm_params WO setup compression algorithm parameters
compact WO trigger memory compaction
@@ -310,7 +294,7 @@ a single line of text and contains the following stats separated by whitespace:
Unit: 4K bytes
============== =============================================================
-10) Deactivate
+9) Deactivate
==============
::
@@ -318,7 +302,7 @@ a single line of text and contains the following stats separated by whitespace:
swapoff /dev/zram0
umount /dev/zram1
-11) Reset
+10) Reset
=========
Write any positive value to 'reset' sysfs node::
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index c73d8024f48f..c7bc0c9f3f2f 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -1109,27 +1109,6 @@ static void zram_debugfs_register(struct zram *zram) {};
static void zram_debugfs_unregister(struct zram *zram) {};
#endif
-/*
- * We switched to per-cpu streams and this attr is not needed anymore.
- * However, we will keep it around for some time, because:
- * a) we may revert per-cpu streams in the future
- * b) it's visible to user space and we need to follow our 2 years
- * retirement rule; but we already have a number of 'soon to be
- * altered' attrs, so max_comp_streams need to wait for the next
- * layoff cycle.
- */
-static ssize_t max_comp_streams_show(struct device *dev,
- struct device_attribute *attr, char *buf)
-{
- return scnprintf(buf, PAGE_SIZE, "%d\n", num_online_cpus());
-}
-
-static ssize_t max_comp_streams_store(struct device *dev,
- struct device_attribute *attr, const char *buf, size_t len)
-{
- return len;
-}
-
static void comp_algorithm_set(struct zram *zram, u32 prio, const char *alg)
{
/* Do not free statically defined compression algorithms */
@@ -2546,7 +2525,6 @@ static DEVICE_ATTR_WO(reset);
static DEVICE_ATTR_WO(mem_limit);
static DEVICE_ATTR_WO(mem_used_max);
static DEVICE_ATTR_WO(idle);
-static DEVICE_ATTR_RW(max_comp_streams);
static DEVICE_ATTR_RW(comp_algorithm);
#ifdef CONFIG_ZRAM_WRITEBACK
static DEVICE_ATTR_RW(backing_dev);
@@ -2568,7 +2546,6 @@ static struct attribute *zram_disk_attrs[] = {
&dev_attr_mem_limit.attr,
&dev_attr_mem_used_max.attr,
&dev_attr_idle.attr,
- &dev_attr_max_comp_streams.attr,
&dev_attr_comp_algorithm.attr,
#ifdef CONFIG_ZRAM_WRITEBACK
&dev_attr_backing_dev.attr,
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 05/17] zram: remove second stage of handle allocation
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (3 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 04/17] zram: remove max_comp_streams device attr Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 06/17] zram: remove writestall zram_stats member Sergey Senozhatsky
` (11 subsequent siblings)
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
Previously zram write() was atomic which required us to pass
__GFP_KSWAPD_RECLAIM to zsmalloc handle allocation on a fast
path and attempt a slow path allocation (with recompression)
if the fast path failed.
Since we are not in atomic context anymore we can permit
direct reclaim during handle allocation, and hence can have
a single allocation path. There is no slow path anymore so
we don't unlock per-CPU stream (and don't lose compressed
data) which means that there is no need to do recompression
now (which should reduce CPU and battery usage).
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zram_drv.c | 38 ++++++-----------------------------
1 file changed, 6 insertions(+), 32 deletions(-)
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index c7bc0c9f3f2f..4ccc1a1a8f20 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -1729,11 +1729,11 @@ static int write_incompressible_page(struct zram *zram, struct page *page,
static int zram_write_page(struct zram *zram, struct page *page, u32 index)
{
int ret = 0;
- unsigned long handle = -ENOMEM;
- unsigned int comp_len = 0;
+ unsigned long handle;
+ unsigned int comp_len;
void *dst, *mem;
struct zcomp_strm *zstrm;
- unsigned long element = 0;
+ unsigned long element;
bool same_filled;
/* First, free memory allocated to this slot (if any) */
@@ -1747,7 +1747,6 @@ static int zram_write_page(struct zram *zram, struct page *page, u32 index)
if (same_filled)
return write_same_filled_page(zram, element, index);
-compress_again:
zstrm = zcomp_stream_get(zram->comps[ZRAM_PRIMARY_COMP]);
mem = kmap_local_page(page);
ret = zcomp_compress(zram->comps[ZRAM_PRIMARY_COMP], zstrm,
@@ -1757,7 +1756,6 @@ static int zram_write_page(struct zram *zram, struct page *page, u32 index)
if (unlikely(ret)) {
zcomp_stream_put(zstrm);
pr_err("Compression failed! err=%d\n", ret);
- zs_free(zram->mem_pool, handle);
return ret;
}
@@ -1766,35 +1764,11 @@ static int zram_write_page(struct zram *zram, struct page *page, u32 index)
return write_incompressible_page(zram, page, index);
}
- /*
- * handle allocation has 2 paths:
- * a) fast path is executed with preemption disabled (for
- * per-cpu streams) and has __GFP_DIRECT_RECLAIM bit clear,
- * since we can't sleep;
- * b) slow path enables preemption and attempts to allocate
- * the page with __GFP_DIRECT_RECLAIM bit set. we have to
- * put per-cpu compression stream and, thus, to re-do
- * the compression once handle is allocated.
- *
- * if we have a 'non-null' handle here then we are coming
- * from the slow path and handle has already been allocated.
- */
- if (IS_ERR_VALUE(handle))
- handle = zs_malloc(zram->mem_pool, comp_len,
- __GFP_KSWAPD_RECLAIM |
- __GFP_NOWARN |
- __GFP_HIGHMEM |
- __GFP_MOVABLE);
+ handle = zs_malloc(zram->mem_pool, comp_len,
+ GFP_NOIO | __GFP_HIGHMEM | __GFP_MOVABLE);
if (IS_ERR_VALUE(handle)) {
zcomp_stream_put(zstrm);
- atomic64_inc(&zram->stats.writestall);
- handle = zs_malloc(zram->mem_pool, comp_len,
- GFP_NOIO | __GFP_HIGHMEM |
- __GFP_MOVABLE);
- if (IS_ERR_VALUE(handle))
- return PTR_ERR((void *)handle);
-
- goto compress_again;
+ return PTR_ERR((void *)handle);
}
if (!zram_can_store_page(zram)) {
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 06/17] zram: remove writestall zram_stats member
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (4 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 05/17] zram: remove second stage of handle allocation Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 07/17] zram: limit max recompress prio to num_active_comps Sergey Senozhatsky
` (10 subsequent siblings)
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
There is no zsmalloc handle allocation slow path now and
writestall is not possible any longer. Remove it from
zram_stats.
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zram_drv.c | 3 +--
drivers/block/zram/zram_drv.h | 1 -
2 files changed, 1 insertion(+), 3 deletions(-)
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index 4ccc1a1a8f20..710b10c6e336 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -1443,9 +1443,8 @@ static ssize_t debug_stat_show(struct device *dev,
down_read(&zram->init_lock);
ret = scnprintf(buf, PAGE_SIZE,
- "version: %d\n%8llu %8llu\n",
+ "version: %d\n0 %8llu\n",
version,
- (u64)atomic64_read(&zram->stats.writestall),
(u64)atomic64_read(&zram->stats.miss_free));
up_read(&zram->init_lock);
diff --git a/drivers/block/zram/zram_drv.h b/drivers/block/zram/zram_drv.h
index 2c380ea9a816..59c75154524f 100644
--- a/drivers/block/zram/zram_drv.h
+++ b/drivers/block/zram/zram_drv.h
@@ -84,7 +84,6 @@ struct zram_stats {
atomic64_t huge_pages_since; /* no. of huge pages since zram set up */
atomic64_t pages_stored; /* no. of pages currently stored */
atomic_long_t max_used_pages; /* no. of maximum pages stored */
- atomic64_t writestall; /* no. of write slow paths */
atomic64_t miss_free; /* no. of missed free */
#ifdef CONFIG_ZRAM_WRITEBACK
atomic64_t bd_count; /* no. of pages in backing device */
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 07/17] zram: limit max recompress prio to num_active_comps
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (5 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 06/17] zram: remove writestall zram_stats member Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 08/17] zram: filter out recomp targets based on priority Sergey Senozhatsky
` (9 subsequent siblings)
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
Use the actual number of algorithms zram was configure with
instead of theoretical limit of ZRAM_MAX_COMPS.
Also make sure that min prio is not above max prio.
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zram_drv.c | 15 ++++++++++++---
1 file changed, 12 insertions(+), 3 deletions(-)
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index 710b10c6e336..b32b959046af 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -2031,16 +2031,19 @@ static ssize_t recompress_store(struct device *dev,
struct device_attribute *attr,
const char *buf, size_t len)
{
- u32 prio = ZRAM_SECONDARY_COMP, prio_max = ZRAM_MAX_COMPS;
struct zram *zram = dev_to_zram(dev);
char *args, *param, *val, *algo = NULL;
u64 num_recomp_pages = ULLONG_MAX;
struct zram_pp_ctl *ctl = NULL;
struct zram_pp_slot *pps;
u32 mode = 0, threshold = 0;
+ u32 prio, prio_max;
struct page *page;
ssize_t ret;
+ prio = ZRAM_SECONDARY_COMP;
+ prio_max = zram->num_active_comps;
+
args = skip_spaces(buf);
while (*args) {
args = next_arg(args, ¶m, &val);
@@ -2093,7 +2096,7 @@ static ssize_t recompress_store(struct device *dev,
if (prio == ZRAM_PRIMARY_COMP)
prio = ZRAM_SECONDARY_COMP;
- prio_max = min(prio + 1, ZRAM_MAX_COMPS);
+ prio_max = prio + 1;
continue;
}
}
@@ -2121,7 +2124,7 @@ static ssize_t recompress_store(struct device *dev,
continue;
if (!strcmp(zram->comp_algs[prio], algo)) {
- prio_max = min(prio + 1, ZRAM_MAX_COMPS);
+ prio_max = prio + 1;
found = true;
break;
}
@@ -2133,6 +2136,12 @@ static ssize_t recompress_store(struct device *dev,
}
}
+ prio_max = min(prio_max, (u32)zram->num_active_comps);
+ if (prio >= prio_max) {
+ ret = -EINVAL;
+ goto release_init_lock;
+ }
+
page = alloc_page(GFP_KERNEL);
if (!page) {
ret = -ENOMEM;
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 08/17] zram: filter out recomp targets based on priority
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (6 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 07/17] zram: limit max recompress prio to num_active_comps Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 09/17] zram: rework recompression loop Sergey Senozhatsky
` (8 subsequent siblings)
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
Do no select for post processing slots that are already
compressed with same or higher priority compression
algorithm.
This should save some memory, as previously we would still
put those entries into corresponding post-processing buckets
and filter them out later in recompress_slot().
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zram_drv.c | 25 ++++++++++++++++---------
1 file changed, 16 insertions(+), 9 deletions(-)
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index b32b959046af..92908495c904 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -1827,7 +1827,7 @@ static int zram_bvec_write(struct zram *zram, struct bio_vec *bvec,
#define RECOMPRESS_IDLE (1 << 0)
#define RECOMPRESS_HUGE (1 << 1)
-static int scan_slots_for_recompress(struct zram *zram, u32 mode,
+static int scan_slots_for_recompress(struct zram *zram, u32 mode, u32 prio_max,
struct zram_pp_ctl *ctl)
{
unsigned long nr_pages = zram->disksize >> PAGE_SHIFT;
@@ -1859,6 +1859,10 @@ static int scan_slots_for_recompress(struct zram *zram, u32 mode,
zram_test_flag(zram, index, ZRAM_INCOMPRESSIBLE))
goto next;
+ /* Already compressed with same of higher priority */
+ if (zram_get_priority(zram, index) + 1 >= prio_max)
+ goto next;
+
pps->index = index;
place_pp_slot(zram, ctl, pps);
pps = NULL;
@@ -1915,6 +1919,16 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
zram_clear_flag(zram, index, ZRAM_IDLE);
class_index_old = zs_lookup_class_index(zram->mem_pool, comp_len_old);
+
+ prio = max(prio, zram_get_priority(zram, index) + 1);
+ /*
+ * Recompression slots scan should not select slots that are
+ * already compressed with a higher priority algorithm, but
+ * just in case
+ */
+ if (prio >= prio_max)
+ return 0;
+
/*
* Iterate the secondary comp algorithms list (in order of priority)
* and try to recompress the page.
@@ -1923,13 +1937,6 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
if (!zram->comps[prio])
continue;
- /*
- * Skip if the object is already re-compressed with a higher
- * priority algorithm (or same algorithm).
- */
- if (prio <= zram_get_priority(zram, index))
- continue;
-
num_recomps++;
zstrm = zcomp_stream_get(zram->comps[prio]);
src = kmap_local_page(page);
@@ -2154,7 +2161,7 @@ static ssize_t recompress_store(struct device *dev,
goto release_init_lock;
}
- scan_slots_for_recompress(zram, mode, ctl);
+ scan_slots_for_recompress(zram, mode, prio_max, ctl);
ret = len;
while ((pps = select_pp_slot(ctl))) {
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 09/17] zram: rework recompression loop
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (7 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 08/17] zram: filter out recomp targets based on priority Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 10/17] zsmalloc: rename pool lock Sergey Senozhatsky
` (7 subsequent siblings)
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
This reworks recompression loop handling:
- set a rule that stream-put NULLs the stream pointer
If the loop returns with a non-NULL stream then it's a
successfull recompression, otherwise the stream should
always be NULL.
- do not count the number of recompressions
Mark object as incompressible as soon as the algorithm
with the highest priority failed to compress that object.
- count compression errors as resource usage
Even if compression has failed, we still need to bump
num_recomp_pages counter.
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zram_drv.c | 53 +++++++++++++----------------------
1 file changed, 19 insertions(+), 34 deletions(-)
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index 92908495c904..b96be8576cbc 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -1892,9 +1892,8 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
unsigned int comp_len_new;
unsigned int class_index_old;
unsigned int class_index_new;
- u32 num_recomps = 0;
void *src, *dst;
- int ret;
+ int ret = 0;
handle_old = zram_get_handle(zram, index);
if (!handle_old)
@@ -1937,7 +1936,6 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
if (!zram->comps[prio])
continue;
- num_recomps++;
zstrm = zcomp_stream_get(zram->comps[prio]);
src = kmap_local_page(page);
ret = zcomp_compress(zram->comps[prio], zstrm,
@@ -1946,7 +1944,8 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
if (ret) {
zcomp_stream_put(zstrm);
- return ret;
+ zstrm = NULL;
+ break;
}
class_index_new = zs_lookup_class_index(zram->mem_pool,
@@ -1956,6 +1955,7 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
if (class_index_new >= class_index_old ||
(threshold && comp_len_new >= threshold)) {
zcomp_stream_put(zstrm);
+ zstrm = NULL;
continue;
}
@@ -1963,14 +1963,6 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
break;
}
- /*
- * We did not try to recompress, e.g. when we have only one
- * secondary algorithm and the page is already recompressed
- * using that algorithm
- */
- if (!zstrm)
- return 0;
-
/*
* Decrement the limit (if set) on pages we can recompress, even
* when current recompression was unsuccessful or did not compress
@@ -1980,38 +1972,31 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
if (*num_recomp_pages)
*num_recomp_pages -= 1;
- if (class_index_new >= class_index_old) {
+ /* Compression error */
+ if (ret)
+ return ret;
+
+ if (!zstrm) {
/*
* Secondary algorithms failed to re-compress the page
- * in a way that would save memory, mark the object as
- * incompressible so that we will not try to compress
- * it again.
+ * in a way that would save memory.
*
- * We need to make sure that all secondary algorithms have
- * failed, so we test if the number of recompressions matches
- * the number of active secondary algorithms.
+ * Mark the object incompressible if the max-priority
+ * algorithm couldn't re-compress it.
*/
- if (num_recomps == zram->num_active_comps - 1)
- zram_set_flag(zram, index, ZRAM_INCOMPRESSIBLE);
+ if (prio < zram->num_active_comps)
+ return 0;
+ zram_set_flag(zram, index, ZRAM_INCOMPRESSIBLE);
return 0;
}
- /* Successful recompression but above threshold */
- if (threshold && comp_len_new >= threshold)
- return 0;
-
/*
- * No direct reclaim (slow path) for handle allocation and no
- * re-compression attempt (unlike in zram_write_bvec()) since
- * we already have stored that object in zsmalloc. If we cannot
- * alloc memory for recompressed object then we bail out and
- * simply keep the old (existing) object in zsmalloc.
+ * We are holding per-CPU stream mutex and entry lock so better
+ * avoid direct reclaim. Allocation error is not fatal since
+ * we still have the old object in the mem_pool.
*/
handle_new = zs_malloc(zram->mem_pool, comp_len_new,
- __GFP_KSWAPD_RECLAIM |
- __GFP_NOWARN |
- __GFP_HIGHMEM |
- __GFP_MOVABLE);
+ GFP_NOWAIT | __GFP_HIGHMEM | __GFP_MOVABLE);
if (IS_ERR_VALUE(handle_new)) {
zcomp_stream_put(zstrm);
return PTR_ERR((void *)handle_new);
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 10/17] zsmalloc: rename pool lock
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (8 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 09/17] zram: rework recompression loop Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 11/17] zsmalloc: make zspage lock preemptible Sergey Senozhatsky
` (6 subsequent siblings)
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
The old name comes from the times when the pool did not have
compaction (defragmentation). Rename it to ->lock because these
days it synchronizes not only migration.
Reviewed-by: Yosry Ahmed <yosry.ahmed@linux.dev>
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
mm/zsmalloc.c | 38 +++++++++++++++++++-------------------
1 file changed, 19 insertions(+), 19 deletions(-)
diff --git a/mm/zsmalloc.c b/mm/zsmalloc.c
index 817626a351f8..1424ee73cbb5 100644
--- a/mm/zsmalloc.c
+++ b/mm/zsmalloc.c
@@ -18,7 +18,7 @@
/*
* lock ordering:
* page_lock
- * pool->migrate_lock
+ * pool->lock
* class->lock
* zspage->lock
*/
@@ -223,8 +223,8 @@ struct zs_pool {
#ifdef CONFIG_COMPACTION
struct work_struct free_work;
#endif
- /* protect page/zspage migration */
- rwlock_t migrate_lock;
+ /* protect zspage migration/compaction */
+ rwlock_t lock;
atomic_t compaction_in_progress;
};
@@ -1206,7 +1206,7 @@ void *zs_map_object(struct zs_pool *pool, unsigned long handle,
BUG_ON(in_interrupt());
/* It guarantees it can get zspage from handle safely */
- read_lock(&pool->migrate_lock);
+ read_lock(&pool->lock);
obj = handle_to_obj(handle);
obj_to_location(obj, &zpdesc, &obj_idx);
zspage = get_zspage(zpdesc);
@@ -1218,7 +1218,7 @@ void *zs_map_object(struct zs_pool *pool, unsigned long handle,
* which is smaller granularity.
*/
migrate_read_lock(zspage);
- read_unlock(&pool->migrate_lock);
+ read_unlock(&pool->lock);
class = zspage_class(pool, zspage);
off = offset_in_page(class->size * obj_idx);
@@ -1450,16 +1450,16 @@ void zs_free(struct zs_pool *pool, unsigned long handle)
return;
/*
- * The pool->migrate_lock protects the race with zpage's migration
+ * The pool->lock protects the race with zpage's migration
* so it's safe to get the page from handle.
*/
- read_lock(&pool->migrate_lock);
+ read_lock(&pool->lock);
obj = handle_to_obj(handle);
obj_to_zpdesc(obj, &f_zpdesc);
zspage = get_zspage(f_zpdesc);
class = zspage_class(pool, zspage);
spin_lock(&class->lock);
- read_unlock(&pool->migrate_lock);
+ read_unlock(&pool->lock);
class_stat_sub(class, ZS_OBJS_INUSE, 1);
obj_free(class->size, obj);
@@ -1796,7 +1796,7 @@ static int zs_page_migrate(struct page *newpage, struct page *page,
* The pool migrate_lock protects the race between zpage migration
* and zs_free.
*/
- write_lock(&pool->migrate_lock);
+ write_lock(&pool->lock);
class = zspage_class(pool, zspage);
/*
@@ -1833,7 +1833,7 @@ static int zs_page_migrate(struct page *newpage, struct page *page,
* Since we complete the data copy and set up new zspage structure,
* it's okay to release migration_lock.
*/
- write_unlock(&pool->migrate_lock);
+ write_unlock(&pool->lock);
spin_unlock(&class->lock);
migrate_write_unlock(zspage);
@@ -1956,7 +1956,7 @@ static unsigned long __zs_compact(struct zs_pool *pool,
* protect the race between zpage migration and zs_free
* as well as zpage allocation/free
*/
- write_lock(&pool->migrate_lock);
+ write_lock(&pool->lock);
spin_lock(&class->lock);
while (zs_can_compact(class)) {
int fg;
@@ -1983,14 +1983,14 @@ static unsigned long __zs_compact(struct zs_pool *pool,
src_zspage = NULL;
if (get_fullness_group(class, dst_zspage) == ZS_INUSE_RATIO_100
- || rwlock_is_contended(&pool->migrate_lock)) {
+ || rwlock_is_contended(&pool->lock)) {
putback_zspage(class, dst_zspage);
dst_zspage = NULL;
spin_unlock(&class->lock);
- write_unlock(&pool->migrate_lock);
+ write_unlock(&pool->lock);
cond_resched();
- write_lock(&pool->migrate_lock);
+ write_lock(&pool->lock);
spin_lock(&class->lock);
}
}
@@ -2002,7 +2002,7 @@ static unsigned long __zs_compact(struct zs_pool *pool,
putback_zspage(class, dst_zspage);
spin_unlock(&class->lock);
- write_unlock(&pool->migrate_lock);
+ write_unlock(&pool->lock);
return pages_freed;
}
@@ -2014,10 +2014,10 @@ unsigned long zs_compact(struct zs_pool *pool)
unsigned long pages_freed = 0;
/*
- * Pool compaction is performed under pool->migrate_lock so it is basically
+ * Pool compaction is performed under pool->lock so it is basically
* single-threaded. Having more than one thread in __zs_compact()
- * will increase pool->migrate_lock contention, which will impact other
- * zsmalloc operations that need pool->migrate_lock.
+ * will increase pool->lock contention, which will impact other
+ * zsmalloc operations that need pool->lock.
*/
if (atomic_xchg(&pool->compaction_in_progress, 1))
return 0;
@@ -2139,7 +2139,7 @@ struct zs_pool *zs_create_pool(const char *name)
return NULL;
init_deferred_free(pool);
- rwlock_init(&pool->migrate_lock);
+ rwlock_init(&pool->lock);
atomic_set(&pool->compaction_in_progress, 0);
pool->name = kstrdup(name, GFP_KERNEL);
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 11/17] zsmalloc: make zspage lock preemptible
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (9 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 10/17] zsmalloc: rename pool lock Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-24 8:59 ` Sebastian Andrzej Siewior
2025-02-21 22:25 ` [PATCH v8 12/17] zsmalloc: introduce new object mapping API Sergey Senozhatsky
` (5 subsequent siblings)
16 siblings, 1 reply; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
In order to implement preemptible object mapping we need a zspage lock
that satisfies several preconditions:
- it should be reader-write type of a lock
- it should be possible to hold it from any context, but also being
preemptible if the context allows it
- we never sleep while acquiring but can sleep while holding in read
mode
An rwsemaphore doesn't suffice, due to atomicity requirements, rwlock
doesn't satisfy due to reader-preemptability requirement. It's also
worth to mention, that per-zspage rwsem is a little too memory heavy
(we can easily have double digits megabytes used only on rwsemaphores).
Switch over from rwlock_t to a atomic_t-based implementation of a
reader-writer semaphore that satisfies all of the preconditions.
The spin-lock based zspage_lock is suggested by Hillf Danton.
Suggested-by: Hillf Danton <hdanton@sina.com>
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
mm/zsmalloc.c | 184 +++++++++++++++++++++++++++++++++++---------------
1 file changed, 131 insertions(+), 53 deletions(-)
diff --git a/mm/zsmalloc.c b/mm/zsmalloc.c
index 1424ee73cbb5..03710d71d022 100644
--- a/mm/zsmalloc.c
+++ b/mm/zsmalloc.c
@@ -226,6 +226,9 @@ struct zs_pool {
/* protect zspage migration/compaction */
rwlock_t lock;
atomic_t compaction_in_progress;
+#ifdef CONFIG_DEBUG_LOCK_ALLOC
+ struct lock_class_key lock_class;
+#endif
};
static inline void zpdesc_set_first(struct zpdesc *zpdesc)
@@ -257,6 +260,18 @@ static inline void free_zpdesc(struct zpdesc *zpdesc)
__free_page(page);
}
+#define ZS_PAGE_UNLOCKED 0
+#define ZS_PAGE_WRLOCKED -1
+
+struct zspage_lock {
+ spinlock_t lock;
+ int cnt;
+
+#ifdef CONFIG_DEBUG_LOCK_ALLOC
+ struct lockdep_map dep_map;
+#endif
+};
+
struct zspage {
struct {
unsigned int huge:HUGE_BITS;
@@ -269,7 +284,7 @@ struct zspage {
struct zpdesc *first_zpdesc;
struct list_head list; /* fullness list */
struct zs_pool *pool;
- rwlock_t lock;
+ struct zspage_lock zsl;
};
struct mapping_area {
@@ -279,6 +294,93 @@ struct mapping_area {
enum zs_mapmode vm_mm; /* mapping mode */
};
+#ifdef CONFIG_DEBUG_LOCK_ALLOC
+#define zsl_dep_map(zsl) (&(zsl)->dep_map)
+#define zspool_lock_class(pool) (&(pool)->lock_class)
+#else
+#define zsl_dep_map(zsl) NULL
+#define zspool_lock_class(pool) NULL
+#endif
+
+static void zspage_lock_init(struct zspage *zspage)
+{
+ struct zspage_lock *zsl = &zspage->zsl;
+
+ lockdep_init_map(zsl_dep_map(zsl), "zspage->lock",
+ zspool_lock_class(zspage->pool), 0);
+ spin_lock_init(&zsl->lock);
+ zsl->cnt = ZS_PAGE_UNLOCKED;
+}
+
+/*
+ * The zspage lock can be held from atomic contexts, but it needs to remain
+ * preemptible when held for reading because it remains held outside of those
+ * atomic contexts, otherwise we unnecessarily lose preemptibility.
+ *
+ * To achieve this, the following rules are enforced on readers and writers:
+ *
+ * - Writers are blocked by both writers and readers, while readers are only
+ * blocked by writers (i.e. normal rwlock semantics).
+ *
+ * - Writers are always atomic (to allow readers to spin waiting for them).
+ *
+ * - Writers always use trylock (as the lock may be held be sleeping readers).
+ *
+ * - Readers may spin on the lock (as they can only wait for atomic writers).
+ *
+ * - Readers may sleep while holding the lock (as writes only use trylock).
+ */
+static void zspage_read_lock(struct zspage *zspage)
+{
+ struct zspage_lock *zsl = &zspage->zsl;
+
+ rwsem_acquire_read(zsl_dep_map(zsl), 0, 0, _RET_IP_);
+
+ spin_lock(&zsl->lock);
+ zsl->cnt++;
+ spin_unlock(&zsl->lock);
+
+ lock_acquired(zsl_dep_map(zsl), _RET_IP_);
+}
+
+static void zspage_read_unlock(struct zspage *zspage)
+{
+ struct zspage_lock *zsl = &zspage->zsl;
+
+ rwsem_release(zsl_dep_map(zsl), _RET_IP_);
+
+ spin_lock(&zsl->lock);
+ zsl->cnt--;
+ spin_unlock(&zsl->lock);
+}
+
+static __must_check bool zspage_write_trylock(struct zspage *zspage)
+{
+ struct zspage_lock *zsl = &zspage->zsl;
+
+ spin_lock(&zsl->lock);
+ if (zsl->cnt == ZS_PAGE_UNLOCKED) {
+ zsl->cnt = ZS_PAGE_WRLOCKED;
+ rwsem_acquire(zsl_dep_map(zsl), 0, 1, _RET_IP_);
+ lock_acquired(zsl_dep_map(zsl), _RET_IP_);
+ return true;
+ }
+
+ lock_contended(zsl_dep_map(zsl), _RET_IP_);
+ spin_unlock(&zsl->lock);
+ return false;
+}
+
+static void zspage_write_unlock(struct zspage *zspage)
+{
+ struct zspage_lock *zsl = &zspage->zsl;
+
+ rwsem_release(zsl_dep_map(zsl), _RET_IP_);
+
+ zsl->cnt = ZS_PAGE_UNLOCKED;
+ spin_unlock(&zsl->lock);
+}
+
/* huge object: pages_per_zspage == 1 && maxobj_per_zspage == 1 */
static void SetZsHugePage(struct zspage *zspage)
{
@@ -290,12 +392,6 @@ static bool ZsHugePage(struct zspage *zspage)
return zspage->huge;
}
-static void migrate_lock_init(struct zspage *zspage);
-static void migrate_read_lock(struct zspage *zspage);
-static void migrate_read_unlock(struct zspage *zspage);
-static void migrate_write_lock(struct zspage *zspage);
-static void migrate_write_unlock(struct zspage *zspage);
-
#ifdef CONFIG_COMPACTION
static void kick_deferred_free(struct zs_pool *pool);
static void init_deferred_free(struct zs_pool *pool);
@@ -992,7 +1088,9 @@ static struct zspage *alloc_zspage(struct zs_pool *pool,
return NULL;
zspage->magic = ZSPAGE_MAGIC;
- migrate_lock_init(zspage);
+ zspage->pool = pool;
+ zspage->class = class->index;
+ zspage_lock_init(zspage);
for (i = 0; i < class->pages_per_zspage; i++) {
struct zpdesc *zpdesc;
@@ -1015,8 +1113,6 @@ static struct zspage *alloc_zspage(struct zs_pool *pool,
create_page_chain(class, zspage, zpdescs);
init_zspage(class, zspage);
- zspage->pool = pool;
- zspage->class = class->index;
return zspage;
}
@@ -1217,7 +1313,7 @@ void *zs_map_object(struct zs_pool *pool, unsigned long handle,
* zs_unmap_object API so delegate the locking from class to zspage
* which is smaller granularity.
*/
- migrate_read_lock(zspage);
+ zspage_read_lock(zspage);
read_unlock(&pool->lock);
class = zspage_class(pool, zspage);
@@ -1277,7 +1373,7 @@ void zs_unmap_object(struct zs_pool *pool, unsigned long handle)
}
local_unlock(&zs_map_area.lock);
- migrate_read_unlock(zspage);
+ zspage_read_unlock(zspage);
}
EXPORT_SYMBOL_GPL(zs_unmap_object);
@@ -1671,18 +1767,18 @@ static void lock_zspage(struct zspage *zspage)
/*
* Pages we haven't locked yet can be migrated off the list while we're
* trying to lock them, so we need to be careful and only attempt to
- * lock each page under migrate_read_lock(). Otherwise, the page we lock
+ * lock each page under zspage_read_lock(). Otherwise, the page we lock
* may no longer belong to the zspage. This means that we may wait for
* the wrong page to unlock, so we must take a reference to the page
- * prior to waiting for it to unlock outside migrate_read_lock().
+ * prior to waiting for it to unlock outside zspage_read_lock().
*/
while (1) {
- migrate_read_lock(zspage);
+ zspage_read_lock(zspage);
zpdesc = get_first_zpdesc(zspage);
if (zpdesc_trylock(zpdesc))
break;
zpdesc_get(zpdesc);
- migrate_read_unlock(zspage);
+ zspage_read_unlock(zspage);
zpdesc_wait_locked(zpdesc);
zpdesc_put(zpdesc);
}
@@ -1693,41 +1789,16 @@ static void lock_zspage(struct zspage *zspage)
curr_zpdesc = zpdesc;
} else {
zpdesc_get(zpdesc);
- migrate_read_unlock(zspage);
+ zspage_read_unlock(zspage);
zpdesc_wait_locked(zpdesc);
zpdesc_put(zpdesc);
- migrate_read_lock(zspage);
+ zspage_read_lock(zspage);
}
}
- migrate_read_unlock(zspage);
+ zspage_read_unlock(zspage);
}
#endif /* CONFIG_COMPACTION */
-static void migrate_lock_init(struct zspage *zspage)
-{
- rwlock_init(&zspage->lock);
-}
-
-static void migrate_read_lock(struct zspage *zspage) __acquires(&zspage->lock)
-{
- read_lock(&zspage->lock);
-}
-
-static void migrate_read_unlock(struct zspage *zspage) __releases(&zspage->lock)
-{
- read_unlock(&zspage->lock);
-}
-
-static void migrate_write_lock(struct zspage *zspage)
-{
- write_lock(&zspage->lock);
-}
-
-static void migrate_write_unlock(struct zspage *zspage)
-{
- write_unlock(&zspage->lock);
-}
-
#ifdef CONFIG_COMPACTION
static const struct movable_operations zsmalloc_mops;
@@ -1785,9 +1856,6 @@ static int zs_page_migrate(struct page *newpage, struct page *page,
VM_BUG_ON_PAGE(!zpdesc_is_isolated(zpdesc), zpdesc_page(zpdesc));
- /* We're committed, tell the world that this is a Zsmalloc page. */
- __zpdesc_set_zsmalloc(newzpdesc);
-
/* The page is locked, so this pointer must remain valid */
zspage = get_zspage(zpdesc);
pool = zspage->pool;
@@ -1803,8 +1871,15 @@ static int zs_page_migrate(struct page *newpage, struct page *page,
* the class lock protects zpage alloc/free in the zspage.
*/
spin_lock(&class->lock);
- /* the migrate_write_lock protects zpage access via zs_map_object */
- migrate_write_lock(zspage);
+ /* the zspage write_lock protects zpage access via zs_map_object */
+ if (!zspage_write_trylock(zspage)) {
+ spin_unlock(&class->lock);
+ write_unlock(&pool->lock);
+ return -EINVAL;
+ }
+
+ /* We're committed, tell the world that this is a Zsmalloc page. */
+ __zpdesc_set_zsmalloc(newzpdesc);
offset = get_first_obj_offset(zpdesc);
s_addr = kmap_local_zpdesc(zpdesc);
@@ -1835,7 +1910,7 @@ static int zs_page_migrate(struct page *newpage, struct page *page,
*/
write_unlock(&pool->lock);
spin_unlock(&class->lock);
- migrate_write_unlock(zspage);
+ zspage_write_unlock(zspage);
zpdesc_get(newzpdesc);
if (zpdesc_zone(newzpdesc) != zpdesc_zone(zpdesc)) {
@@ -1971,9 +2046,11 @@ static unsigned long __zs_compact(struct zs_pool *pool,
if (!src_zspage)
break;
- migrate_write_lock(src_zspage);
+ if (!zspage_write_trylock(src_zspage))
+ break;
+
migrate_zspage(pool, src_zspage, dst_zspage);
- migrate_write_unlock(src_zspage);
+ zspage_write_unlock(src_zspage);
fg = putback_zspage(class, src_zspage);
if (fg == ZS_INUSE_RATIO_0) {
@@ -2141,6 +2218,7 @@ struct zs_pool *zs_create_pool(const char *name)
init_deferred_free(pool);
rwlock_init(&pool->lock);
atomic_set(&pool->compaction_in_progress, 0);
+ lockdep_register_key(zspool_lock_class(pool));
pool->name = kstrdup(name, GFP_KERNEL);
if (!pool->name)
@@ -2233,7 +2311,6 @@ struct zs_pool *zs_create_pool(const char *name)
* trigger compaction manually. Thus, ignore return code.
*/
zs_register_shrinker(pool);
-
return pool;
err:
@@ -2270,6 +2347,7 @@ void zs_destroy_pool(struct zs_pool *pool)
kfree(class);
}
+ lockdep_unregister_key(zspool_lock_class(pool));
destroy_cache(pool);
kfree(pool->name);
kfree(pool);
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 12/17] zsmalloc: introduce new object mapping API
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (10 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 11/17] zsmalloc: make zspage lock preemptible Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-24 9:01 ` Sebastian Andrzej Siewior
2025-02-21 22:25 ` [PATCH v8 13/17] zram: switch to new zsmalloc " Sergey Senozhatsky
` (4 subsequent siblings)
16 siblings, 1 reply; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
Current object mapping API is a little cumbersome. First, it's
inconsistent, sometimes it returns with page-faults disabled and
sometimes with page-faults enabled. Second, and most importantly,
it enforces atomicity restrictions on its users. zs_map_object()
has to return a liner object address which is not always possible
because some objects span multiple physical (non-contiguous)
pages. For such objects zsmalloc uses a per-CPU buffer to which
object's data is copied before a pointer to that per-CPU buffer
is returned back to the caller. This leads to another, final,
issue - extra memcpy(). Since the caller gets a pointer to
per-CPU buffer it can memcpy() data only to that buffer, and
during zs_unmap_object() zsmalloc will memcpy() from that per-CPU
buffer to physical pages that object in question spans across.
New API splits functions by access mode:
- zs_obj_read_begin(handle, local_copy)
Returns a pointer to handle memory. For objects that span two
physical pages a local_copy buffer is used to store object's
data before the address is returned to the caller. Otherwise
the object's page is kmap_local mapped directly.
- zs_obj_read_end(handle, buf)
Unmaps the page if it was kmap_local mapped by zs_obj_read_begin().
- zs_obj_write(handle, buf, len)
Copies len-bytes from compression buffer to handle memory
(takes care of objects that span two pages). This does not
need any additional (e.g. per-CPU) buffers and writes the data
directly to zsmalloc pool pages.
In terms of performance, on a synthetic and completely reproducible
test that allocates fixed number of objects of fixed sizes and
iterates over those objects, first mapping in RO then in RW mode:
OLD API
=======
3 first results out of 10
369,205,778 instructions # 0.80 insn per cycle
40,467,926 branches # 113.732 M/sec
369,002,122 instructions # 0.62 insn per cycle
40,426,145 branches # 189.361 M/sec
369,036,706 instructions # 0.63 insn per cycle
40,430,860 branches # 204.105 M/sec
[..]
NEW API
=======
3 first results out of 10
265,799,293 instructions # 0.51 insn per cycle
29,834,567 branches # 170.281 M/sec
265,765,970 instructions # 0.55 insn per cycle
29,829,019 branches # 161.602 M/sec
265,764,702 instructions # 0.51 insn per cycle
29,828,015 branches # 189.677 M/sec
[..]
T-test on all 10 runs
=====================
Difference at 95.0% confidence
-1.03219e+08 +/- 55308.7
-27.9705% +/- 0.0149878%
(Student's t, pooled s = 58864.4)
The old API will stay around until the remaining users switch
to the new one. After that we'll also remove zsmalloc per-CPU
buffer and CPU hotplug handling.
The split of map(RO) and map(WO) into read_{begin/end}/write is
suggested by Yosry Ahmed.
Suggested-by: Yosry Ahmed <yosry.ahmed@linux.dev>
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
include/linux/zsmalloc.h | 8 +++
mm/zsmalloc.c | 129 +++++++++++++++++++++++++++++++++++++++
2 files changed, 137 insertions(+)
diff --git a/include/linux/zsmalloc.h b/include/linux/zsmalloc.h
index a48cd0ffe57d..7d70983cf398 100644
--- a/include/linux/zsmalloc.h
+++ b/include/linux/zsmalloc.h
@@ -58,4 +58,12 @@ unsigned long zs_compact(struct zs_pool *pool);
unsigned int zs_lookup_class_index(struct zs_pool *pool, unsigned int size);
void zs_pool_stats(struct zs_pool *pool, struct zs_pool_stats *stats);
+
+void *zs_obj_read_begin(struct zs_pool *pool, unsigned long handle,
+ void *local_copy);
+void zs_obj_read_end(struct zs_pool *pool, unsigned long handle,
+ void *handle_mem);
+void zs_obj_write(struct zs_pool *pool, unsigned long handle,
+ void *handle_mem, size_t mem_len);
+
#endif
diff --git a/mm/zsmalloc.c b/mm/zsmalloc.c
index 03710d71d022..1288a4120855 100644
--- a/mm/zsmalloc.c
+++ b/mm/zsmalloc.c
@@ -1377,6 +1377,135 @@ void zs_unmap_object(struct zs_pool *pool, unsigned long handle)
}
EXPORT_SYMBOL_GPL(zs_unmap_object);
+void *zs_obj_read_begin(struct zs_pool *pool, unsigned long handle,
+ void *local_copy)
+{
+ struct zspage *zspage;
+ struct zpdesc *zpdesc;
+ unsigned long obj, off;
+ unsigned int obj_idx;
+ struct size_class *class;
+ void *addr;
+
+ WARN_ON(in_interrupt());
+
+ /* Guarantee we can get zspage from handle safely */
+ read_lock(&pool->lock);
+ obj = handle_to_obj(handle);
+ obj_to_location(obj, &zpdesc, &obj_idx);
+ zspage = get_zspage(zpdesc);
+
+ /* Make sure migration doesn't move any pages in this zspage */
+ zspage_read_lock(zspage);
+ read_unlock(&pool->lock);
+
+ class = zspage_class(pool, zspage);
+ off = offset_in_page(class->size * obj_idx);
+
+ if (off + class->size <= PAGE_SIZE) {
+ /* this object is contained entirely within a page */
+ addr = kmap_local_zpdesc(zpdesc);
+ addr += off;
+ } else {
+ size_t sizes[2];
+
+ /* this object spans two pages */
+ sizes[0] = PAGE_SIZE - off;
+ sizes[1] = class->size - sizes[0];
+ addr = local_copy;
+
+ memcpy_from_page(addr, zpdesc_page(zpdesc),
+ off, sizes[0]);
+ zpdesc = get_next_zpdesc(zpdesc);
+ memcpy_from_page(addr + sizes[0],
+ zpdesc_page(zpdesc),
+ 0, sizes[1]);
+ }
+
+ if (!ZsHugePage(zspage))
+ addr += ZS_HANDLE_SIZE;
+
+ return addr;
+}
+EXPORT_SYMBOL_GPL(zs_obj_read_begin);
+
+void zs_obj_read_end(struct zs_pool *pool, unsigned long handle,
+ void *handle_mem)
+{
+ struct zspage *zspage;
+ struct zpdesc *zpdesc;
+ unsigned long obj, off;
+ unsigned int obj_idx;
+ struct size_class *class;
+
+ obj = handle_to_obj(handle);
+ obj_to_location(obj, &zpdesc, &obj_idx);
+ zspage = get_zspage(zpdesc);
+ class = zspage_class(pool, zspage);
+ off = offset_in_page(class->size * obj_idx);
+
+ if (off + class->size <= PAGE_SIZE) {
+ if (!ZsHugePage(zspage))
+ off += ZS_HANDLE_SIZE;
+ handle_mem -= off;
+ kunmap_local(handle_mem);
+ }
+
+ zspage_read_unlock(zspage);
+}
+EXPORT_SYMBOL_GPL(zs_obj_read_end);
+
+void zs_obj_write(struct zs_pool *pool, unsigned long handle,
+ void *handle_mem, size_t mem_len)
+{
+ struct zspage *zspage;
+ struct zpdesc *zpdesc;
+ unsigned long obj, off;
+ unsigned int obj_idx;
+ struct size_class *class;
+
+ WARN_ON(in_interrupt());
+
+ /* Guarantee we can get zspage from handle safely */
+ read_lock(&pool->lock);
+ obj = handle_to_obj(handle);
+ obj_to_location(obj, &zpdesc, &obj_idx);
+ zspage = get_zspage(zpdesc);
+
+ /* Make sure migration doesn't move any pages in this zspage */
+ zspage_read_lock(zspage);
+ read_unlock(&pool->lock);
+
+ class = zspage_class(pool, zspage);
+ off = offset_in_page(class->size * obj_idx);
+
+ if (off + class->size <= PAGE_SIZE) {
+ /* this object is contained entirely within a page */
+ void *dst = kmap_local_zpdesc(zpdesc);
+
+ if (!ZsHugePage(zspage))
+ off += ZS_HANDLE_SIZE;
+ memcpy(dst + off, handle_mem, mem_len);
+ kunmap_local(dst);
+ } else {
+ /* this object spans two pages */
+ size_t sizes[2];
+
+ off += ZS_HANDLE_SIZE;
+ sizes[0] = PAGE_SIZE - off;
+ sizes[1] = mem_len - sizes[0];
+
+ memcpy_to_page(zpdesc_page(zpdesc), off,
+ handle_mem, sizes[0]);
+ zpdesc = get_next_zpdesc(zpdesc);
+ memcpy_to_page(zpdesc_page(zpdesc), 0,
+ handle_mem + sizes[0], sizes[1]);
+ }
+
+ zspage_read_unlock(zspage);
+}
+EXPORT_SYMBOL_GPL(zs_obj_write);
+
/**
* zs_huge_class_size() - Returns the size (in bytes) of the first huge
* zsmalloc &size_class.
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 13/17] zram: switch to new zsmalloc object mapping API
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (11 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 12/17] zsmalloc: introduce new object mapping API Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 14/17] zram: permit reclaim in zstd custom allocator Sergey Senozhatsky
` (3 subsequent siblings)
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
Use new read/write zsmalloc object API. For cases when RO mapped
object spans two physical pages (requires temp buffer) compression
streams now carry around one extra physical page.
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zcomp.c | 4 +++-
drivers/block/zram/zcomp.h | 2 ++
drivers/block/zram/zram_drv.c | 28 ++++++++++------------------
3 files changed, 15 insertions(+), 19 deletions(-)
diff --git a/drivers/block/zram/zcomp.c b/drivers/block/zram/zcomp.c
index cfdde2e0748a..a1d627054bb1 100644
--- a/drivers/block/zram/zcomp.c
+++ b/drivers/block/zram/zcomp.c
@@ -45,6 +45,7 @@ static const struct zcomp_ops *backends[] = {
static void zcomp_strm_free(struct zcomp *comp, struct zcomp_strm *zstrm)
{
comp->ops->destroy_ctx(&zstrm->ctx);
+ vfree(zstrm->local_copy);
vfree(zstrm->buffer);
zstrm->buffer = NULL;
}
@@ -57,12 +58,13 @@ static int zcomp_strm_init(struct zcomp *comp, struct zcomp_strm *zstrm)
if (ret)
return ret;
+ zstrm->local_copy = vzalloc(PAGE_SIZE);
/*
* allocate 2 pages. 1 for compressed data, plus 1 extra for the
* case when compressed size is larger than the original one
*/
zstrm->buffer = vzalloc(2 * PAGE_SIZE);
- if (!zstrm->buffer) {
+ if (!zstrm->buffer || !zstrm->local_copy) {
zcomp_strm_free(comp, zstrm);
return -ENOMEM;
}
diff --git a/drivers/block/zram/zcomp.h b/drivers/block/zram/zcomp.h
index 23b8236b9090..25339ed1e07e 100644
--- a/drivers/block/zram/zcomp.h
+++ b/drivers/block/zram/zcomp.h
@@ -34,6 +34,8 @@ struct zcomp_strm {
struct mutex lock;
/* compression buffer */
void *buffer;
+ /* local copy of handle memory */
+ void *local_copy;
struct zcomp_ctx ctx;
};
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index b96be8576cbc..1ce981ce6f48 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -1566,11 +1566,11 @@ static int read_incompressible_page(struct zram *zram, struct page *page,
void *src, *dst;
handle = zram_get_handle(zram, index);
- src = zs_map_object(zram->mem_pool, handle, ZS_MM_RO);
+ src = zs_obj_read_begin(zram->mem_pool, handle, NULL);
dst = kmap_local_page(page);
copy_page(dst, src);
kunmap_local(dst);
- zs_unmap_object(zram->mem_pool, handle);
+ zs_obj_read_end(zram->mem_pool, handle, src);
return 0;
}
@@ -1588,11 +1588,11 @@ static int read_compressed_page(struct zram *zram, struct page *page, u32 index)
prio = zram_get_priority(zram, index);
zstrm = zcomp_stream_get(zram->comps[prio]);
- src = zs_map_object(zram->mem_pool, handle, ZS_MM_RO);
+ src = zs_obj_read_begin(zram->mem_pool, handle, zstrm->local_copy);
dst = kmap_local_page(page);
ret = zcomp_decompress(zram->comps[prio], zstrm, src, size, dst);
kunmap_local(dst);
- zs_unmap_object(zram->mem_pool, handle);
+ zs_obj_read_end(zram->mem_pool, handle, src);
zcomp_stream_put(zstrm);
return ret;
@@ -1688,7 +1688,7 @@ static int write_incompressible_page(struct zram *zram, struct page *page,
u32 index)
{
unsigned long handle;
- void *src, *dst;
+ void *src;
/*
* This function is called from preemptible context so we don't need
@@ -1705,11 +1705,9 @@ static int write_incompressible_page(struct zram *zram, struct page *page,
return -ENOMEM;
}
- dst = zs_map_object(zram->mem_pool, handle, ZS_MM_WO);
src = kmap_local_page(page);
- memcpy(dst, src, PAGE_SIZE);
+ zs_obj_write(zram->mem_pool, handle, src, PAGE_SIZE);
kunmap_local(src);
- zs_unmap_object(zram->mem_pool, handle);
zram_slot_lock(zram, index);
zram_set_flag(zram, index, ZRAM_HUGE);
@@ -1730,7 +1728,7 @@ static int zram_write_page(struct zram *zram, struct page *page, u32 index)
int ret = 0;
unsigned long handle;
unsigned int comp_len;
- void *dst, *mem;
+ void *mem;
struct zcomp_strm *zstrm;
unsigned long element;
bool same_filled;
@@ -1776,11 +1774,8 @@ static int zram_write_page(struct zram *zram, struct page *page, u32 index)
return -ENOMEM;
}
- dst = zs_map_object(zram->mem_pool, handle, ZS_MM_WO);
-
- memcpy(dst, zstrm->buffer, comp_len);
+ zs_obj_write(zram->mem_pool, handle, zstrm->buffer, comp_len);
zcomp_stream_put(zstrm);
- zs_unmap_object(zram->mem_pool, handle);
zram_slot_lock(zram, index);
zram_set_handle(zram, index, handle);
@@ -1892,7 +1887,7 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
unsigned int comp_len_new;
unsigned int class_index_old;
unsigned int class_index_new;
- void *src, *dst;
+ void *src;
int ret = 0;
handle_old = zram_get_handle(zram, index);
@@ -2002,12 +1997,9 @@ static int recompress_slot(struct zram *zram, u32 index, struct page *page,
return PTR_ERR((void *)handle_new);
}
- dst = zs_map_object(zram->mem_pool, handle_new, ZS_MM_WO);
- memcpy(dst, zstrm->buffer, comp_len_new);
+ zs_obj_write(zram->mem_pool, handle_new, zstrm->buffer, comp_len_new);
zcomp_stream_put(zstrm);
- zs_unmap_object(zram->mem_pool, handle_new);
-
zram_free_page(zram, index);
zram_set_handle(zram, index, handle_new);
zram_set_obj_size(zram, index, comp_len_new);
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 14/17] zram: permit reclaim in zstd custom allocator
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (12 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 13/17] zram: switch to new zsmalloc " Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-24 9:10 ` Sebastian Andrzej Siewior
2025-02-21 22:25 ` [PATCH v8 15/17] zram: do not leak page on recompress_store error path Sergey Senozhatsky
` (2 subsequent siblings)
16 siblings, 1 reply; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
When configured with pre-trained compression/decompression
dictionary support, zstd requires custom memory allocator,
which it calls internally from compression()/decompression()
routines. That means allocation from atomic context (either
under entry spin-lock, or per-CPU local-lock or both). Now,
with non-atomic zram read()/write(), those limitations are
relaxed and we can allow direct and indirect reclaim.
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/backend_zstd.c | 11 +++--------
1 file changed, 3 insertions(+), 8 deletions(-)
diff --git a/drivers/block/zram/backend_zstd.c b/drivers/block/zram/backend_zstd.c
index 1184c0036f44..53431251ea62 100644
--- a/drivers/block/zram/backend_zstd.c
+++ b/drivers/block/zram/backend_zstd.c
@@ -24,19 +24,14 @@ struct zstd_params {
/*
* For C/D dictionaries we need to provide zstd with zstd_custom_mem,
* which zstd uses internally to allocate/free memory when needed.
- *
- * This means that allocator.customAlloc() can be called from zcomp_compress()
- * under local-lock (per-CPU compression stream), in which case we must use
- * GFP_ATOMIC.
- *
- * Another complication here is that we can be configured as a swap device.
*/
static void *zstd_custom_alloc(void *opaque, size_t size)
{
- if (!preemptible())
+ /* Technically this should not happen */
+ if (WARN_ON_ONCE(!preemptible()))
return kvzalloc(size, GFP_ATOMIC);
- return kvzalloc(size, __GFP_KSWAPD_RECLAIM | __GFP_NOWARN);
+ return kvzalloc(size, GFP_NOIO | __GFP_NOWARN);
}
static void zstd_custom_free(void *opaque, void *address)
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 15/17] zram: do not leak page on recompress_store error path
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (13 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 14/17] zram: permit reclaim in zstd custom allocator Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 16/17] zram: do not leak page on writeback_store " Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 17/17] zram: add might_sleep to zcomp API Sergey Senozhatsky
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
Ensure the page used for local object data is freed
on error out path.
Fixes: 3f909a60cec1 ("zram: rework recompress target selection strategy")
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zram_drv.c | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index 1ce981ce6f48..1da329cae8ce 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -2022,7 +2022,7 @@ static ssize_t recompress_store(struct device *dev,
struct zram_pp_slot *pps;
u32 mode = 0, threshold = 0;
u32 prio, prio_max;
- struct page *page;
+ struct page *page = NULL;
ssize_t ret;
prio = ZRAM_SECONDARY_COMP;
@@ -2166,9 +2166,9 @@ static ssize_t recompress_store(struct device *dev,
cond_resched();
}
- __free_page(page);
-
release_init_lock:
+ if (page)
+ __free_page(page);
release_pp_ctl(zram, ctl);
atomic_set(&zram->pp_in_progress, 0);
up_read(&zram->init_lock);
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 16/17] zram: do not leak page on writeback_store error path
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (14 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 15/17] zram: do not leak page on recompress_store error path Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 17/17] zram: add might_sleep to zcomp API Sergey Senozhatsky
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
Ensure the page used for local object data is freed
on error out path.
Fixes: 330edc2bc059 (zram: rework writeback target selection strategy)
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zram_drv.c | 6 ++++--
1 file changed, 4 insertions(+), 2 deletions(-)
diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
index 1da329cae8ce..4e9381b153da 100644
--- a/drivers/block/zram/zram_drv.c
+++ b/drivers/block/zram/zram_drv.c
@@ -792,7 +792,7 @@ static ssize_t writeback_store(struct device *dev,
unsigned long index = 0;
struct bio bio;
struct bio_vec bio_vec;
- struct page *page;
+ struct page *page = NULL;
ssize_t ret = len;
int mode, err;
unsigned long blk_idx = 0;
@@ -934,8 +934,10 @@ static ssize_t writeback_store(struct device *dev,
if (blk_idx)
free_block_bdev(zram, blk_idx);
- __free_page(page);
+
release_init_lock:
+ if (page)
+ __free_page(page);
release_pp_ctl(zram, ctl);
atomic_set(&zram->pp_in_progress, 0);
up_read(&zram->init_lock);
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* [PATCH v8 17/17] zram: add might_sleep to zcomp API
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
` (15 preceding siblings ...)
2025-02-21 22:25 ` [PATCH v8 16/17] zram: do not leak page on writeback_store " Sergey Senozhatsky
@ 2025-02-21 22:25 ` Sergey Senozhatsky
16 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-21 22:25 UTC (permalink / raw)
To: Andrew Morton
Cc: Yosry Ahmed, Hillf Danton, Kairui Song,
Sebastian Andrzej Siewior, Minchan Kim, linux-mm, linux-kernel,
Sergey Senozhatsky
Explicitly state that zcomp compress/decompress must be
called from non-atomic context.
Signed-off-by: Sergey Senozhatsky <senozhatsky@chromium.org>
---
drivers/block/zram/zcomp.c | 2 ++
1 file changed, 2 insertions(+)
diff --git a/drivers/block/zram/zcomp.c b/drivers/block/zram/zcomp.c
index a1d627054bb1..d26a58c67e95 100644
--- a/drivers/block/zram/zcomp.c
+++ b/drivers/block/zram/zcomp.c
@@ -146,6 +146,7 @@ int zcomp_compress(struct zcomp *comp, struct zcomp_strm *zstrm,
};
int ret;
+ might_sleep();
ret = comp->ops->compress(comp->params, &zstrm->ctx, &req);
if (!ret)
*dst_len = req.dst_len;
@@ -162,6 +163,7 @@ int zcomp_decompress(struct zcomp *comp, struct zcomp_strm *zstrm,
.dst_len = PAGE_SIZE,
};
+ might_sleep();
return comp->ops->decompress(comp->params, &zstrm->ctx, &req);
}
--
2.48.1.601.g30ceb7b040-goog
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 01/17] zram: sleepable entry locking
2025-02-21 22:25 ` [PATCH v8 01/17] zram: sleepable entry locking Sergey Senozhatsky
@ 2025-02-24 8:19 ` Sebastian Andrzej Siewior
2025-02-25 4:51 ` Sergey Senozhatsky
0 siblings, 1 reply; 33+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-02-24 8:19 UTC (permalink / raw)
To: Sergey Senozhatsky
Cc: Andrew Morton, Yosry Ahmed, Hillf Danton, Kairui Song,
Minchan Kim, linux-mm, linux-kernel
On 2025-02-22 07:25:32 [+0900], Sergey Senozhatsky wrote:
…
> diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
> index 9f5020b077c5..37c5651305c2 100644
> --- a/drivers/block/zram/zram_drv.c
> +++ b/drivers/block/zram/zram_drv.c
> @@ -58,19 +58,62 @@ static void zram_free_page(struct zram *zram, size_t index);
> static int zram_read_from_zspool(struct zram *zram, struct page *page,
> u32 index);
>
> -static int zram_slot_trylock(struct zram *zram, u32 index)
> +#ifdef CONFIG_DEBUG_LOCK_ALLOC
> +#define slot_dep_map(zram, index) (&(zram)->table[(index)].dep_map)
> +#define zram_lock_class(zram) (&(zram)->lock_class)
> +#else
> +#define slot_dep_map(zram, index) NULL
> +#define zram_lock_class(zram) NULL
> +#endif
That CONFIG_DEBUG_LOCK_ALLOC here is not needed because dep_map as well
as lock_class goes away in !CONFIG_DEBUG_LOCK_ALLOC case.
> +static void zram_slot_lock_init(struct zram *zram, u32 index)
> {
> - return spin_trylock(&zram->table[index].lock);
> + lockdep_init_map(slot_dep_map(zram, index),
> + "zram->table[index].lock",
> + zram_lock_class(zram), 0);
> +}
Why do need zram_lock_class and slot_dep_map? As far as I can tell, you
init both in the same place and you acquire both in the same place.
Therefore it looks like you tell lockdep that you acquire two locks
while it would be enough to do it with one.
> +/*
> + * entry locking rules:
> + *
> + * 1) Lock is exclusive
> + *
> + * 2) lock() function can sleep waiting for the lock
> + *
> + * 3) Lock owner can sleep
> + *
> + * 4) Use TRY lock variant when in atomic context
> + * - must check return value and handle locking failers
> + */
> +static __must_check bool zram_slot_trylock(struct zram *zram, u32 index)
> +{
> + unsigned long *lock = &zram->table[index].flags;
> +
> + if (!test_and_set_bit_lock(ZRAM_ENTRY_LOCK, lock)) {
> + mutex_acquire(slot_dep_map(zram, index), 0, 1, _RET_IP_);
> + lock_acquired(slot_dep_map(zram, index), _RET_IP_);
> + return true;
> + }
> +
> + lock_contended(slot_dep_map(zram, index), _RET_IP_);
> + return false;
> }
>
> static void zram_slot_lock(struct zram *zram, u32 index)
> {
> - spin_lock(&zram->table[index].lock);
> + unsigned long *lock = &zram->table[index].flags;
> +
> + mutex_acquire(slot_dep_map(zram, index), 0, 0, _RET_IP_);
> + wait_on_bit_lock(lock, ZRAM_ENTRY_LOCK, TASK_UNINTERRUPTIBLE);
> + lock_acquired(slot_dep_map(zram, index), _RET_IP_);
This looks odd. The first mutex_acquire() can be invoked twice by two
threads, right? The first thread gets both (mutex_acquire() and
lock_acquired()) while, the second gets mutex_acquire() and blocks on
wait_on_bit_lock()).
> }
>
> static void zram_slot_unlock(struct zram *zram, u32 index)
> {
> - spin_unlock(&zram->table[index].lock);
> + unsigned long *lock = &zram->table[index].flags;
> +
> + mutex_release(slot_dep_map(zram, index), _RET_IP_);
> + clear_and_wake_up_bit(ZRAM_ENTRY_LOCK, lock);
> }
>
> static inline bool init_done(struct zram *zram)
…
> diff --git a/drivers/block/zram/zram_drv.h b/drivers/block/zram/zram_drv.h
> index db78d7c01b9a..794c9234e627 100644
> --- a/drivers/block/zram/zram_drv.h
> +++ b/drivers/block/zram/zram_drv.h
> @@ -58,13 +58,18 @@ enum zram_pageflags {
> __NR_ZRAM_PAGEFLAGS,
> };
>
> -/*-- Data structures */
> -
> -/* Allocated for each disk page */
> +/*
> + * Allocated for each disk page. We use bit-lock (ZRAM_ENTRY_LOCK bit
> + * of flags) to save memory. There can be plenty of entries and standard
> + * locking primitives (e.g. mutex) will significantly increase sizeof()
> + * of each entry and hence of the meta table.
> + */
> struct zram_table_entry {
> unsigned long handle;
> - unsigned int flags;
> - spinlock_t lock;
> + unsigned long flags;
> +#ifdef CONFIG_DEBUG_LOCK_ALLOC
> + struct lockdep_map dep_map;
> +#endif
> #ifdef CONFIG_ZRAM_TRACK_ENTRY_ACTIME
> ktime_t ac_time;
> #endif
> @@ -137,5 +142,8 @@ struct zram {
> struct dentry *debugfs_dir;
> #endif
> atomic_t pp_in_progress;
> +#ifdef CONFIG_DEBUG_LOCK_ALLOC
> + struct lock_class_key lock_class;
> +#endif
As mentioned earlier, no need for CONFIG_DEBUG_LOCK_ALLOC.
> };
> #endif
> --
> 2.48.1.601.g30ceb7b040-goog
>
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 11/17] zsmalloc: make zspage lock preemptible
2025-02-21 22:25 ` [PATCH v8 11/17] zsmalloc: make zspage lock preemptible Sergey Senozhatsky
@ 2025-02-24 8:59 ` Sebastian Andrzej Siewior
2025-02-25 4:28 ` Sergey Senozhatsky
0 siblings, 1 reply; 33+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-02-24 8:59 UTC (permalink / raw)
To: Sergey Senozhatsky
Cc: Andrew Morton, Yosry Ahmed, Hillf Danton, Kairui Song,
Minchan Kim, linux-mm, linux-kernel
On 2025-02-22 07:25:42 [+0900], Sergey Senozhatsky wrote:
> index 1424ee73cbb5..03710d71d022 100644
> --- a/mm/zsmalloc.c
> +++ b/mm/zsmalloc.c
> @@ -226,6 +226,9 @@ struct zs_pool {
> /* protect zspage migration/compaction */
> rwlock_t lock;
> atomic_t compaction_in_progress;
> +#ifdef CONFIG_DEBUG_LOCK_ALLOC
> + struct lock_class_key lock_class;
> +#endif
No ifdef
> };
>
> static inline void zpdesc_set_first(struct zpdesc *zpdesc)
> @@ -279,6 +294,93 @@ struct mapping_area {
> enum zs_mapmode vm_mm; /* mapping mode */
> };
>
> +#ifdef CONFIG_DEBUG_LOCK_ALLOC
> +#define zsl_dep_map(zsl) (&(zsl)->dep_map)
> +#define zspool_lock_class(pool) (&(pool)->lock_class)
> +#else
> +#define zsl_dep_map(zsl) NULL
> +#define zspool_lock_class(pool) NULL
> +#endif
> +
> +static void zspage_lock_init(struct zspage *zspage)
> +{
> + struct zspage_lock *zsl = &zspage->zsl;
> +
> + lockdep_init_map(zsl_dep_map(zsl), "zspage->lock",
> + zspool_lock_class(zspage->pool), 0);
> + spin_lock_init(&zsl->lock);
> + zsl->cnt = ZS_PAGE_UNLOCKED;
> +}
> +
> +/*
> + * The zspage lock can be held from atomic contexts, but it needs to remain
> + * preemptible when held for reading because it remains held outside of those
> + * atomic contexts, otherwise we unnecessarily lose preemptibility.
> + *
> + * To achieve this, the following rules are enforced on readers and writers:
> + *
> + * - Writers are blocked by both writers and readers, while readers are only
> + * blocked by writers (i.e. normal rwlock semantics).
> + *
> + * - Writers are always atomic (to allow readers to spin waiting for them).
> + *
> + * - Writers always use trylock (as the lock may be held be sleeping readers).
> + *
> + * - Readers may spin on the lock (as they can only wait for atomic writers).
> + *
> + * - Readers may sleep while holding the lock (as writes only use trylock).
> + */
> +static void zspage_read_lock(struct zspage *zspage)
> +{
> + struct zspage_lock *zsl = &zspage->zsl;
> +
> + rwsem_acquire_read(zsl_dep_map(zsl), 0, 0, _RET_IP_);
> +
> + spin_lock(&zsl->lock);
> + zsl->cnt++;
> + spin_unlock(&zsl->lock);
How is this working given that a read_lock always increments the
counter? If it is write_locked then a read_lock makes it UNLOCKED.
migrate_read_lock() did block if a writer was pending, this does not or
at least it is not obvious how.
> + lock_acquired(zsl_dep_map(zsl), _RET_IP_);
> +}
> +
Sebastian
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 12/17] zsmalloc: introduce new object mapping API
2025-02-21 22:25 ` [PATCH v8 12/17] zsmalloc: introduce new object mapping API Sergey Senozhatsky
@ 2025-02-24 9:01 ` Sebastian Andrzej Siewior
2025-02-25 4:29 ` Sergey Senozhatsky
0 siblings, 1 reply; 33+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-02-24 9:01 UTC (permalink / raw)
To: Sergey Senozhatsky
Cc: Andrew Morton, Yosry Ahmed, Hillf Danton, Kairui Song,
Minchan Kim, linux-mm, linux-kernel
On 2025-02-22 07:25:43 [+0900], Sergey Senozhatsky wrote:
> index 03710d71d022..1288a4120855 100644
> --- a/mm/zsmalloc.c
> +++ b/mm/zsmalloc.c
> @@ -1377,6 +1377,135 @@ void zs_unmap_object(struct zs_pool *pool, unsigned long handle)
> }
> EXPORT_SYMBOL_GPL(zs_unmap_object);
>
> +void *zs_obj_read_begin(struct zs_pool *pool, unsigned long handle,
> + void *local_copy)
> +{
> + struct zspage *zspage;
> + struct zpdesc *zpdesc;
> + unsigned long obj, off;
> + unsigned int obj_idx;
> + struct size_class *class;
> + void *addr;
> +
> + WARN_ON(in_interrupt());
This macro is deprecated. Could you come up with something else like
!in_task() if needed?
> + /* Guarantee we can get zspage from handle safely */
> + read_lock(&pool->lock);
> + obj = handle_to_obj(handle);
> + obj_to_location(obj, &zpdesc, &obj_idx);
> + zspage = get_zspage(zpdesc);
…
Sebastian
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 14/17] zram: permit reclaim in zstd custom allocator
2025-02-21 22:25 ` [PATCH v8 14/17] zram: permit reclaim in zstd custom allocator Sergey Senozhatsky
@ 2025-02-24 9:10 ` Sebastian Andrzej Siewior
2025-02-25 4:42 ` Sergey Senozhatsky
0 siblings, 1 reply; 33+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-02-24 9:10 UTC (permalink / raw)
To: Sergey Senozhatsky
Cc: Andrew Morton, Yosry Ahmed, Hillf Danton, Kairui Song,
Minchan Kim, linux-mm, linux-kernel
On 2025-02-22 07:25:45 [+0900], Sergey Senozhatsky wrote:
> static void *zstd_custom_alloc(void *opaque, size_t size)
> {
> - if (!preemptible())
> + /* Technically this should not happen */
> + if (WARN_ON_ONCE(!preemptible()))
> return kvzalloc(size, GFP_ATOMIC);
This check works only on preemptible kernels.
If you run this on !PREEMPTIBLE kernels, preemptible() reports always 0
so that WARNING will always trigger there.
> - return kvzalloc(size, __GFP_KSWAPD_RECLAIM | __GFP_NOWARN);
> + return kvzalloc(size, GFP_NOIO | __GFP_NOWARN);
> }
>
> static void zstd_custom_free(void *opaque, void *address)
> --
> 2.48.1.601.g30ceb7b040-goog
>
Sebastian
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 11/17] zsmalloc: make zspage lock preemptible
2025-02-24 8:59 ` Sebastian Andrzej Siewior
@ 2025-02-25 4:28 ` Sergey Senozhatsky
0 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-25 4:28 UTC (permalink / raw)
To: Sebastian Andrzej Siewior
Cc: Sergey Senozhatsky, Andrew Morton, Yosry Ahmed, Hillf Danton,
Kairui Song, Minchan Kim, linux-mm, linux-kernel
On (25/02/24 09:59), Sebastian Andrzej Siewior wrote:
> > +/*
> > + * The zspage lock can be held from atomic contexts, but it needs to remain
> > + * preemptible when held for reading because it remains held outside of those
> > + * atomic contexts, otherwise we unnecessarily lose preemptibility.
> > + *
> > + * To achieve this, the following rules are enforced on readers and writers:
> > + *
> > + * - Writers are blocked by both writers and readers, while readers are only
> > + * blocked by writers (i.e. normal rwlock semantics).
> > + *
> > + * - Writers are always atomic (to allow readers to spin waiting for them).
> > + *
> > + * - Writers always use trylock (as the lock may be held be sleeping readers).
> > + *
> > + * - Readers may spin on the lock (as they can only wait for atomic writers).
> > + *
> > + * - Readers may sleep while holding the lock (as writes only use trylock).
> > + */
> > +static void zspage_read_lock(struct zspage *zspage)
> > +{
> > + struct zspage_lock *zsl = &zspage->zsl;
> > +
> > + rwsem_acquire_read(zsl_dep_map(zsl), 0, 0, _RET_IP_);
> > +
> > + spin_lock(&zsl->lock);
> > + zsl->cnt++;
> > + spin_unlock(&zsl->lock);
>
> How is this working given that a read_lock always increments the
> counter? If it is write_locked then a read_lock makes it UNLOCKED.
If zspage is write-locked then zsl->lock is also locked, because
write-lock returns with zsl->lock acquired and releases it in
write-unlock.
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 12/17] zsmalloc: introduce new object mapping API
2025-02-24 9:01 ` Sebastian Andrzej Siewior
@ 2025-02-25 4:29 ` Sergey Senozhatsky
0 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-25 4:29 UTC (permalink / raw)
To: Sebastian Andrzej Siewior
Cc: Sergey Senozhatsky, Andrew Morton, Yosry Ahmed, Hillf Danton,
Kairui Song, Minchan Kim, linux-mm, linux-kernel
On (25/02/24 10:01), Sebastian Andrzej Siewior wrote:
> Date: Mon, 24 Feb 2025 10:01:54 +0100
> From: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
> To: Sergey Senozhatsky <senozhatsky@chromium.org>
> Cc: Andrew Morton <akpm@linux-foundation.org>, Yosry Ahmed
> <yosry.ahmed@linux.dev>, Hillf Danton <hdanton@sina.com>, Kairui Song
> <ryncsn@gmail.com>, Minchan Kim <minchan@kernel.org>, linux-mm@kvack.org,
> linux-kernel@vger.kernel.org
> Subject: Re: [PATCH v8 12/17] zsmalloc: introduce new object mapping API
> Message-ID: <20250224090154.7Xlsipmu@linutronix.de>
>
> On 2025-02-22 07:25:43 [+0900], Sergey Senozhatsky wrote:
> > index 03710d71d022..1288a4120855 100644
> > --- a/mm/zsmalloc.c
> > +++ b/mm/zsmalloc.c
> > @@ -1377,6 +1377,135 @@ void zs_unmap_object(struct zs_pool *pool, unsigned long handle)
> > }
> > EXPORT_SYMBOL_GPL(zs_unmap_object);
> >
> > +void *zs_obj_read_begin(struct zs_pool *pool, unsigned long handle,
> > + void *local_copy)
> > +{
> > + struct zspage *zspage;
> > + struct zpdesc *zpdesc;
> > + unsigned long obj, off;
> > + unsigned int obj_idx;
> > + struct size_class *class;
> > + void *addr;
> > +
> > + WARN_ON(in_interrupt());
>
> This macro is deprecated. Could you come up with something else like
> !in_task() if needed?
This was derived from the current zsmalloc object mapping API
BUG_ON(in_interrupt());
but converted from BUG_ON() to WARN_ON(). I guess I can drop
entirely tho.
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 14/17] zram: permit reclaim in zstd custom allocator
2025-02-24 9:10 ` Sebastian Andrzej Siewior
@ 2025-02-25 4:42 ` Sergey Senozhatsky
2025-02-26 3:01 ` Sergey Senozhatsky
2025-02-27 13:19 ` Sebastian Andrzej Siewior
0 siblings, 2 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-25 4:42 UTC (permalink / raw)
To: Sebastian Andrzej Siewior
Cc: Sergey Senozhatsky, Andrew Morton, Yosry Ahmed, Hillf Danton,
Kairui Song, Minchan Kim, linux-mm, linux-kernel
On (25/02/24 10:10), Sebastian Andrzej Siewior wrote:
> On 2025-02-22 07:25:45 [+0900], Sergey Senozhatsky wrote:
> > static void *zstd_custom_alloc(void *opaque, size_t size)
> > {
> > - if (!preemptible())
> > + /* Technically this should not happen */
> > + if (WARN_ON_ONCE(!preemptible()))
> > return kvzalloc(size, GFP_ATOMIC);
>
> This check works only on preemptible kernels.
I'm not sure this is true.
> If you run this on !PREEMPTIBLE kernels, preemptible() reports always 0
> so that WARNING will always trigger there.
I thought that preemptible() depends on PREEMPT_COUNT, not on
PREEMPTIBLE, because even on !PREEMPTIBLE preempt-count still
holds hard/soft irq counts, etc.
I ran CONFIG_PREEMPT_NONE=y zram-zstd tests and didn't see any
warnings.
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 01/17] zram: sleepable entry locking
2025-02-24 8:19 ` Sebastian Andrzej Siewior
@ 2025-02-25 4:51 ` Sergey Senozhatsky
2025-02-27 12:05 ` Sebastian Andrzej Siewior
0 siblings, 1 reply; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-25 4:51 UTC (permalink / raw)
To: Sebastian Andrzej Siewior
Cc: Sergey Senozhatsky, Andrew Morton, Yosry Ahmed, Hillf Danton,
Kairui Song, Minchan Kim, linux-mm, linux-kernel
On (25/02/24 09:19), Sebastian Andrzej Siewior wrote:
> > diff --git a/drivers/block/zram/zram_drv.c b/drivers/block/zram/zram_drv.c
> > index 9f5020b077c5..37c5651305c2 100644
> > --- a/drivers/block/zram/zram_drv.c
> > +++ b/drivers/block/zram/zram_drv.c
> > @@ -58,19 +58,62 @@ static void zram_free_page(struct zram *zram, size_t index);
> > static int zram_read_from_zspool(struct zram *zram, struct page *page,
> > u32 index);
> >
> > -static int zram_slot_trylock(struct zram *zram, u32 index)
> > +#ifdef CONFIG_DEBUG_LOCK_ALLOC
> > +#define slot_dep_map(zram, index) (&(zram)->table[(index)].dep_map)
> > +#define zram_lock_class(zram) (&(zram)->lock_class)
> > +#else
> > +#define slot_dep_map(zram, index) NULL
> > +#define zram_lock_class(zram) NULL
> > +#endif
>
> That CONFIG_DEBUG_LOCK_ALLOC here is not needed because dep_map as well
> as lock_class goes away in !CONFIG_DEBUG_LOCK_ALLOC case.
Let me give it a try.
> > +static void zram_slot_lock_init(struct zram *zram, u32 index)
> > {
> > - return spin_trylock(&zram->table[index].lock);
> > + lockdep_init_map(slot_dep_map(zram, index),
> > + "zram->table[index].lock",
> > + zram_lock_class(zram), 0);
> > +}
> Why do need zram_lock_class and slot_dep_map? As far as I can tell, you
> init both in the same place and you acquire both in the same place.
> Therefore it looks like you tell lockdep that you acquire two locks
> while it would be enough to do it with one.
Sorry, I'm not that familiar with lockdep, can you elaborate?
I don't think we can pass NULL as lock-class to lockdep_init_map(),
this should trigger `if (DEBUG_LOCKS_WARN_ON(!key))` as far as I
can tell. I guess it's something else that you are suggesting?
> > static void zram_slot_lock(struct zram *zram, u32 index)
> > {
> > - spin_lock(&zram->table[index].lock);
> > + unsigned long *lock = &zram->table[index].flags;
> > +
> > + mutex_acquire(slot_dep_map(zram, index), 0, 0, _RET_IP_);
> > + wait_on_bit_lock(lock, ZRAM_ENTRY_LOCK, TASK_UNINTERRUPTIBLE);
> > + lock_acquired(slot_dep_map(zram, index), _RET_IP_);
>
> This looks odd. The first mutex_acquire() can be invoked twice by two
> threads, right? The first thread gets both (mutex_acquire() and
> lock_acquired()) while, the second gets mutex_acquire() and blocks on
> wait_on_bit_lock()).
Hmm why is this a problem? ... and I'm pretty sure it was you who
suggested to put mutex_acquire() before wait_on_bit_lock() [1] ;)
[1] https://lore.kernel.org/all/20250206073803.c2tiyIq6@linutronix.de/
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 14/17] zram: permit reclaim in zstd custom allocator
2025-02-25 4:42 ` Sergey Senozhatsky
@ 2025-02-26 3:01 ` Sergey Senozhatsky
2025-02-27 13:19 ` Sebastian Andrzej Siewior
1 sibling, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-26 3:01 UTC (permalink / raw)
To: Sergey Senozhatsky
Cc: Sebastian Andrzej Siewior, Andrew Morton, Yosry Ahmed,
Hillf Danton, Kairui Song, Minchan Kim, linux-mm, linux-kernel
On (25/02/25 13:42), Sergey Senozhatsky wrote:
> On (25/02/24 10:10), Sebastian Andrzej Siewior wrote:
> > On 2025-02-22 07:25:45 [+0900], Sergey Senozhatsky wrote:
> > > static void *zstd_custom_alloc(void *opaque, size_t size)
> > > {
> > > + if (WARN_ON_ONCE(!preemptible()))
Gone.
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 01/17] zram: sleepable entry locking
2025-02-25 4:51 ` Sergey Senozhatsky
@ 2025-02-27 12:05 ` Sebastian Andrzej Siewior
2025-02-27 12:42 ` Sergey Senozhatsky
0 siblings, 1 reply; 33+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-02-27 12:05 UTC (permalink / raw)
To: Sergey Senozhatsky
Cc: Andrew Morton, Yosry Ahmed, Hillf Danton, Kairui Song,
Minchan Kim, linux-mm, linux-kernel
On 2025-02-25 13:51:31 [+0900], Sergey Senozhatsky wrote:
> > > +static void zram_slot_lock_init(struct zram *zram, u32 index)
> > > {
> > > - return spin_trylock(&zram->table[index].lock);
> > > + lockdep_init_map(slot_dep_map(zram, index),
> > > + "zram->table[index].lock",
> > > + zram_lock_class(zram), 0);
> > > +}
> > Why do need zram_lock_class and slot_dep_map? As far as I can tell, you
> > init both in the same place and you acquire both in the same place.
> > Therefore it looks like you tell lockdep that you acquire two locks
> > while it would be enough to do it with one.
>
> Sorry, I'm not that familiar with lockdep, can you elaborate?
> I don't think we can pass NULL as lock-class to lockdep_init_map(),
> this should trigger `if (DEBUG_LOCKS_WARN_ON(!key))` as far as I
> can tell. I guess it's something else that you are suggesting?
ach. Got it. What about
| static void zram_slot_lock_init(struct zram *zram, u32 index)
| {
| static struct lock_class_key __key;
|
| lockdep_init_map(slot_dep_map(zram, index),
| "zram->table[index].lock",
| &__key, 0);
| }
So every lock coming from zram belongs to the same class. Otherwise each
lock coming from zram_slot_lock_init() would belong to a different class
and for lockdep it would look like they are different locks. But they
are used always in the same way.
> > > static void zram_slot_lock(struct zram *zram, u32 index)
> > > {
> > > - spin_lock(&zram->table[index].lock);
> > > + unsigned long *lock = &zram->table[index].flags;
> > > +
> > > + mutex_acquire(slot_dep_map(zram, index), 0, 0, _RET_IP_);
> > > + wait_on_bit_lock(lock, ZRAM_ENTRY_LOCK, TASK_UNINTERRUPTIBLE);
> > > + lock_acquired(slot_dep_map(zram, index), _RET_IP_);
> >
> > This looks odd. The first mutex_acquire() can be invoked twice by two
> > threads, right? The first thread gets both (mutex_acquire() and
> > lock_acquired()) while, the second gets mutex_acquire() and blocks on
> > wait_on_bit_lock()).
>
> Hmm why is this a problem? ... and I'm pretty sure it was you who
> suggested to put mutex_acquire() before wait_on_bit_lock() [1] ;)
Sure. I was confused that you issue it twice. I didn't noticed the d in
lock_acquired(). So you have one for lockdep and one for lockstat. That
is okay ;)
Sebastian
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 01/17] zram: sleepable entry locking
2025-02-27 12:05 ` Sebastian Andrzej Siewior
@ 2025-02-27 12:42 ` Sergey Senozhatsky
2025-02-27 13:04 ` Sergey Senozhatsky
0 siblings, 1 reply; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-27 12:42 UTC (permalink / raw)
To: Sebastian Andrzej Siewior
Cc: Sergey Senozhatsky, Andrew Morton, Yosry Ahmed, Hillf Danton,
Kairui Song, Minchan Kim, linux-mm, linux-kernel
On (25/02/27 13:05), Sebastian Andrzej Siewior wrote:
> > > > +static void zram_slot_lock_init(struct zram *zram, u32 index)
> > > > {
> > > > - return spin_trylock(&zram->table[index].lock);
> > > > + lockdep_init_map(slot_dep_map(zram, index),
> > > > + "zram->table[index].lock",
> > > > + zram_lock_class(zram), 0);
> > > > +}
> > > Why do need zram_lock_class and slot_dep_map? As far as I can tell, you
> > > init both in the same place and you acquire both in the same place.
> > > Therefore it looks like you tell lockdep that you acquire two locks
> > > while it would be enough to do it with one.
> >
> > Sorry, I'm not that familiar with lockdep, can you elaborate?
> > I don't think we can pass NULL as lock-class to lockdep_init_map(),
> > this should trigger `if (DEBUG_LOCKS_WARN_ON(!key))` as far as I
> > can tell. I guess it's something else that you are suggesting?
>
> ach. Got it. What about
>
> | static void zram_slot_lock_init(struct zram *zram, u32 index)
> | {
> | static struct lock_class_key __key;
> |
> | lockdep_init_map(slot_dep_map(zram, index),
> | "zram->table[index].lock",
> | &__key, 0);
> | }
>
> So every lock coming from zram belongs to the same class. Otherwise each
> lock coming from zram_slot_lock_init() would belong to a different class
> and for lockdep it would look like they are different locks. But they
> are used always in the same way.
I see. I thought that they key was "shared" between zram meta table
entries because the key is per-zram device, which sort of made sense
(we can have different zram devices in a system - one swap, a bunch
mounted with various file-systems on them).
I can do a 'static key', one for all zram devices.
> > > > static void zram_slot_lock(struct zram *zram, u32 index)
> > > > {
> > > > - spin_lock(&zram->table[index].lock);
> > > > + unsigned long *lock = &zram->table[index].flags;
> > > > +
> > > > + mutex_acquire(slot_dep_map(zram, index), 0, 0, _RET_IP_);
> > > > + wait_on_bit_lock(lock, ZRAM_ENTRY_LOCK, TASK_UNINTERRUPTIBLE);
> > > > + lock_acquired(slot_dep_map(zram, index), _RET_IP_);
> > >
> > > This looks odd. The first mutex_acquire() can be invoked twice by two
> > > threads, right? The first thread gets both (mutex_acquire() and
> > > lock_acquired()) while, the second gets mutex_acquire() and blocks on
> > > wait_on_bit_lock()).
> >
> > Hmm why is this a problem? ... and I'm pretty sure it was you who
> > suggested to put mutex_acquire() before wait_on_bit_lock() [1] ;)
>
> Sure. I was confused that you issue it twice. I didn't noticed the d in
> lock_acquired(). So you have one for lockdep and one for lockstat. That
> is okay ;)
Cool!
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 01/17] zram: sleepable entry locking
2025-02-27 12:42 ` Sergey Senozhatsky
@ 2025-02-27 13:04 ` Sergey Senozhatsky
2025-02-27 13:12 ` Sebastian Andrzej Siewior
0 siblings, 1 reply; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-27 13:04 UTC (permalink / raw)
To: Sebastian Andrzej Siewior
Cc: Andrew Morton, Yosry Ahmed, Hillf Danton, Kairui Song,
Minchan Kim, linux-mm, linux-kernel, Sergey Senozhatsky
On (25/02/27 21:42), Sergey Senozhatsky wrote:
> > ach. Got it. What about
> >
> > | static void zram_slot_lock_init(struct zram *zram, u32 index)
> > | {
> > | static struct lock_class_key __key;
> > |
> > | lockdep_init_map(slot_dep_map(zram, index),
> > | "zram->table[index].lock",
> > | &__key, 0);
> > | }
> >
> > So every lock coming from zram belongs to the same class. Otherwise each
> > lock coming from zram_slot_lock_init() would belong to a different class
> > and for lockdep it would look like they are different locks. But they
> > are used always in the same way.
>
> I see. I thought that they key was "shared" between zram meta table
> entries because the key is per-zram device, which sort of made sense
> (we can have different zram devices in a system - one swap, a bunch
> mounted with various file-systems on them).
So the lock class is registered dynamically for each zram device
zram_add()
lockdep_register_key(&zram->lock_class);
and then we use that zram->lock_class to init zram->table entries.
We unregister the lock_class during each zram device destruction
zram_remove()
lockdep_unregister_key(&zram->lock_class);
Does this still put zram->table entries into different lock classes?
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 01/17] zram: sleepable entry locking
2025-02-27 13:04 ` Sergey Senozhatsky
@ 2025-02-27 13:12 ` Sebastian Andrzej Siewior
2025-02-27 13:20 ` Sergey Senozhatsky
0 siblings, 1 reply; 33+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-02-27 13:12 UTC (permalink / raw)
To: Sergey Senozhatsky
Cc: Andrew Morton, Yosry Ahmed, Hillf Danton, Kairui Song,
Minchan Kim, linux-mm, linux-kernel
On 2025-02-27 22:04:16 [+0900], Sergey Senozhatsky wrote:
> On (25/02/27 21:42), Sergey Senozhatsky wrote:
> > > ach. Got it. What about
> > >
> > > | static void zram_slot_lock_init(struct zram *zram, u32 index)
> > > | {
> > > | static struct lock_class_key __key;
> > > |
> > > | lockdep_init_map(slot_dep_map(zram, index),
> > > | "zram->table[index].lock",
> > > | &__key, 0);
> > > | }
> > >
> > > So every lock coming from zram belongs to the same class. Otherwise each
> > > lock coming from zram_slot_lock_init() would belong to a different class
> > > and for lockdep it would look like they are different locks. But they
> > > are used always in the same way.
> >
> > I see. I thought that they key was "shared" between zram meta table
> > entries because the key is per-zram device, which sort of made sense
> > (we can have different zram devices in a system - one swap, a bunch
> > mounted with various file-systems on them).
Yes. So usually you do spin_lock_init() and this creates a key at _this_
very position. So every lock initialized at this position shares the
same class/ the same pattern.
> So the lock class is registered dynamically for each zram device
>
> zram_add()
> lockdep_register_key(&zram->lock_class);
>
> and then we use that zram->lock_class to init zram->table entries.
>
> We unregister the lock_class during each zram device destruction
>
> zram_remove()
> lockdep_unregister_key(&zram->lock_class);
>
> Does this still put zram->table entries into different lock classes?
You shouldn't need to register and unregister the lock_class. What you
do should match for instance j_trans_commit_map in fs/jbd2/journal.c or
__key in include/linux/rhashtable.h & lib/rhashtable.c.
At least based on my understanding so far.
Sebastian
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 14/17] zram: permit reclaim in zstd custom allocator
2025-02-25 4:42 ` Sergey Senozhatsky
2025-02-26 3:01 ` Sergey Senozhatsky
@ 2025-02-27 13:19 ` Sebastian Andrzej Siewior
1 sibling, 0 replies; 33+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-02-27 13:19 UTC (permalink / raw)
To: Sergey Senozhatsky
Cc: Andrew Morton, Yosry Ahmed, Hillf Danton, Kairui Song,
Minchan Kim, linux-mm, linux-kernel
On 2025-02-25 13:42:55 [+0900], Sergey Senozhatsky wrote:
> On (25/02/24 10:10), Sebastian Andrzej Siewior wrote:
> > On 2025-02-22 07:25:45 [+0900], Sergey Senozhatsky wrote:
> > > static void *zstd_custom_alloc(void *opaque, size_t size)
> > > {
> > > - if (!preemptible())
> > > + /* Technically this should not happen */
> > > + if (WARN_ON_ONCE(!preemptible()))
> > > return kvzalloc(size, GFP_ATOMIC);
> >
> > This check works only on preemptible kernels.
>
> I'm not sure this is true.
>
> > If you run this on !PREEMPTIBLE kernels, preemptible() reports always 0
> > so that WARNING will always trigger there.
>
> I thought that preemptible() depends on PREEMPT_COUNT, not on
> PREEMPTIBLE, because even on !PREEMPTIBLE preempt-count still
> holds hard/soft irq counts, etc.
Yes. The preempt count is always there to hold NMI/ HARDIRQ/ SOFTIRQ.
However only on a preemptible (that is with PREEMPT_COUNT) kernel
preempt_disable() does something. So on !PREEMPTIBLE kernel you don't
see spin_lock() or preempt_disable() reflect in preempt_count. Unless
you enable debugging which force this option into a non-preemptible
kernel.
> I ran CONFIG_PREEMPT_NONE=y zram-zstd tests and didn't see any
> warnings.
Sebastian
^ permalink raw reply [flat|nested] 33+ messages in thread
* Re: [PATCH v8 01/17] zram: sleepable entry locking
2025-02-27 13:12 ` Sebastian Andrzej Siewior
@ 2025-02-27 13:20 ` Sergey Senozhatsky
0 siblings, 0 replies; 33+ messages in thread
From: Sergey Senozhatsky @ 2025-02-27 13:20 UTC (permalink / raw)
To: Sebastian Andrzej Siewior
Cc: Sergey Senozhatsky, Andrew Morton, Yosry Ahmed, Hillf Danton,
Kairui Song, Minchan Kim, linux-mm, linux-kernel
On (25/02/27 14:12), Sebastian Andrzej Siewior wrote:
> > > I see. I thought that they key was "shared" between zram meta table
> > > entries because the key is per-zram device, which sort of made sense
> > > (we can have different zram devices in a system - one swap, a bunch
> > > mounted with various file-systems on them).
>
> Yes. So usually you do spin_lock_init() and this creates a key at _this_
> very position. So every lock initialized at this position shares the
> same class/ the same pattern.
>
> > So the lock class is registered dynamically for each zram device
> >
> > zram_add()
> > lockdep_register_key(&zram->lock_class);
> >
> > and then we use that zram->lock_class to init zram->table entries.
> >
> > We unregister the lock_class during each zram device destruction
> >
> > zram_remove()
> > lockdep_unregister_key(&zram->lock_class);
> >
> > Does this still put zram->table entries into different lock classes?
>
> You shouldn't need to register and unregister the lock_class. What you
> do should match for instance j_trans_commit_map in fs/jbd2/journal.c or
> __key in include/linux/rhashtable.h & lib/rhashtable.c.
I see, thank you.
Let me try static keys then (in zram and in zsmalloc). Will need
a day or two to re-run the tests, and then will send out an updated
series.
^ permalink raw reply [flat|nested] 33+ messages in thread
end of thread, other threads:[~2025-02-27 13:24 UTC | newest]
Thread overview: 33+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2025-02-21 22:25 [PATCH v8 00/17] zsmalloc/zram: there be preemption Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 01/17] zram: sleepable entry locking Sergey Senozhatsky
2025-02-24 8:19 ` Sebastian Andrzej Siewior
2025-02-25 4:51 ` Sergey Senozhatsky
2025-02-27 12:05 ` Sebastian Andrzej Siewior
2025-02-27 12:42 ` Sergey Senozhatsky
2025-02-27 13:04 ` Sergey Senozhatsky
2025-02-27 13:12 ` Sebastian Andrzej Siewior
2025-02-27 13:20 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 02/17] zram: permit preemption with active compression stream Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 03/17] zram: remove unused crypto include Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 04/17] zram: remove max_comp_streams device attr Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 05/17] zram: remove second stage of handle allocation Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 06/17] zram: remove writestall zram_stats member Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 07/17] zram: limit max recompress prio to num_active_comps Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 08/17] zram: filter out recomp targets based on priority Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 09/17] zram: rework recompression loop Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 10/17] zsmalloc: rename pool lock Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 11/17] zsmalloc: make zspage lock preemptible Sergey Senozhatsky
2025-02-24 8:59 ` Sebastian Andrzej Siewior
2025-02-25 4:28 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 12/17] zsmalloc: introduce new object mapping API Sergey Senozhatsky
2025-02-24 9:01 ` Sebastian Andrzej Siewior
2025-02-25 4:29 ` Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 13/17] zram: switch to new zsmalloc " Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 14/17] zram: permit reclaim in zstd custom allocator Sergey Senozhatsky
2025-02-24 9:10 ` Sebastian Andrzej Siewior
2025-02-25 4:42 ` Sergey Senozhatsky
2025-02-26 3:01 ` Sergey Senozhatsky
2025-02-27 13:19 ` Sebastian Andrzej Siewior
2025-02-21 22:25 ` [PATCH v8 15/17] zram: do not leak page on recompress_store error path Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 16/17] zram: do not leak page on writeback_store " Sergey Senozhatsky
2025-02-21 22:25 ` [PATCH v8 17/17] zram: add might_sleep to zcomp API Sergey Senozhatsky
This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox