linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
@ 2020-04-28 16:13 Dan Schatzberg
  2020-04-28 16:13 ` [PATCH v5 1/4] loop: Use worker per cgroup instead of kworker Dan Schatzberg
                   ` (5 more replies)
  0 siblings, 6 replies; 20+ messages in thread
From: Dan Schatzberg @ 2020-04-28 16:13 UTC (permalink / raw)
  Cc: Dan Schatzberg, Jens Axboe, Alexander Viro, Jan Kara,
	Amir Goldstein, Tejun Heo, Li Zefan, Johannes Weiner,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Roman Gushchin, Shakeel Butt, Chris Down, Yang Shi, Ingo Molnar,
	Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

Changes since V5:

* Fixed a missing css_put when failing to allocate a worker
* Minor style changes

Changes since V4:

Only patches 1 and 2 have changed.

* Fixed irq lock ordering bug
* Simplified loop detach
* Added support for nesting memalloc_use_memcg

Changes since V3:

* Fix race on loop device destruction and deferred worker cleanup
* Ensure charge on shmem_swapin_page works just like getpage
* Minor style changes

Changes since V2:

* Deferred destruction of workqueue items so in the common case there
  is no allocation needed

Changes since V1:

* Split out and reordered patches so cgroup charging changes are
  separate from kworker -> workqueue change

* Add mem_css to struct loop_cmd to simplify logic

The loop device runs all i/o to the backing file on a separate kworker
thread which results in all i/o being charged to the root cgroup. This
allows a loop device to be used to trivially bypass resource limits
and other policy. This patch series fixes this gap in accounting.

A simple script to demonstrate this behavior on cgroupv2 machine:

'''
#!/bin/bash
set -e

CGROUP=/sys/fs/cgroup/test.slice
LOOP_DEV=/dev/loop0

if [[ ! -d $CGROUP ]]
then
    sudo mkdir $CGROUP
fi

grep oom_kill $CGROUP/memory.events

# Set a memory limit, write more than that limit to tmpfs -> OOM kill
sudo unshare -m bash -c "
echo \$\$ > $CGROUP/cgroup.procs;
echo 0 > $CGROUP/memory.swap.max;
echo 64M > $CGROUP/memory.max;
mount -t tmpfs -o size=512m tmpfs /tmp;
dd if=/dev/zero of=/tmp/file bs=1M count=256" || true

grep oom_kill $CGROUP/memory.events

# Set a memory limit, write more than that limit through loopback
# device -> no OOM kill
sudo unshare -m bash -c "
echo \$\$ > $CGROUP/cgroup.procs;
echo 0 > $CGROUP/memory.swap.max;
echo 64M > $CGROUP/memory.max;
mount -t tmpfs -o size=512m tmpfs /tmp;
truncate -s 512m /tmp/backing_file
losetup $LOOP_DEV /tmp/backing_file
dd if=/dev/zero of=$LOOP_DEV bs=1M count=256;
losetup -D $LOOP_DEV" || true

grep oom_kill $CGROUP/memory.events
'''

Naively charging cgroups could result in priority inversions through
the single kworker thread in the case where multiple cgroups are
reading/writing to the same loop device. This patch series does some
minor modification to the loop driver so that each cgroup can make
forward progress independently to avoid this inversion.

With this patch series applied, the above script triggers OOM kills
when writing through the loop device as expected.

Dan Schatzberg (3):
  loop: Use worker per cgroup instead of kworker
  mm: Charge active memcg when no mm is set
  loop: Charge i/o to mem and blk cg

Johannes Weiner (1):
  mm: support nesting memalloc_use_memcg()

 drivers/block/loop.c                 | 248 ++++++++++++++++++++++-----
 drivers/block/loop.h                 |  14 +-
 fs/buffer.c                          |   6 +-
 fs/notify/fanotify/fanotify.c        |   5 +-
 fs/notify/inotify/inotify_fsnotify.c |   5 +-
 include/linux/memcontrol.h           |   6 +
 include/linux/sched/mm.h             |  28 +--
 kernel/cgroup/cgroup.c               |   1 +
 mm/memcontrol.c                      |  11 +-
 mm/shmem.c                           |   4 +-
 10 files changed, 248 insertions(+), 80 deletions(-)

-- 
2.24.1


^ permalink raw reply	[flat|nested] 20+ messages in thread

* [PATCH v5 1/4] loop: Use worker per cgroup instead of kworker
  2020-04-28 16:13 [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dan Schatzberg
@ 2020-04-28 16:13 ` Dan Schatzberg
  2020-04-28 16:13 ` [PATCH v5 2/4] mm: support nesting memalloc_use_memcg() Dan Schatzberg
                   ` (4 subsequent siblings)
  5 siblings, 0 replies; 20+ messages in thread
From: Dan Schatzberg @ 2020-04-28 16:13 UTC (permalink / raw)
  Cc: Dan Schatzberg, Jens Axboe, Alexander Viro, Jan Kara,
	Amir Goldstein, Tejun Heo, Li Zefan, Johannes Weiner,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Roman Gushchin, Shakeel Butt, Chris Down, Yang Shi,
	Peter Zijlstra (Intel),
	Ingo Molnar, Mathieu Desnoyers, Thomas Gleixner,
	Andrea Arcangeli, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

Existing uses of loop device may have multiple cgroups reading/writing
to the same device. Simply charging resources for I/O to the backing
file could result in priority inversion where one cgroup gets
synchronously blocked, holding up all other I/O to the loop device.

In order to avoid this priority inversion, we use a single workqueue
where each work item is a "struct loop_worker" which contains a queue of
struct loop_cmds to issue. The loop device maintains a tree mapping blk
css_id -> loop_worker. This allows each cgroup to independently make
forward progress issuing I/O to the backing file.

There is also a single queue for I/O associated with the rootcg which
can be used in cases of extreme memory shortage where we cannot allocate
a loop_worker.

The locking for the tree and queues is fairly heavy handed - we acquire
the per-loop-device spinlock any time either is accessed. The existing
implementation serializes all I/O through a single thread anyways, so I
don't believe this is any worse.

Signed-off-by: Dan Schatzberg <schatzberg.dan@gmail.com>
---
 drivers/block/loop.c | 207 ++++++++++++++++++++++++++++++++++++-------
 drivers/block/loop.h |  11 ++-
 2 files changed, 180 insertions(+), 38 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index da693e6a834e..49d7d1f62d88 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -70,7 +70,6 @@
 #include <linux/writeback.h>
 #include <linux/completion.h>
 #include <linux/highmem.h>
-#include <linux/kthread.h>
 #include <linux/splice.h>
 #include <linux/sysfs.h>
 #include <linux/miscdevice.h>
@@ -83,6 +82,8 @@
 
 #include <linux/uaccess.h>
 
+#define LOOP_IDLE_WORKER_TIMEOUT (60 * HZ)
+
 static DEFINE_IDR(loop_index_idr);
 static DEFINE_MUTEX(loop_ctl_mutex);
 
@@ -778,12 +779,18 @@ static ssize_t loop_attr_backing_file_show(struct loop_device *lo, char *buf)
 {
 	ssize_t ret;
 	char *p = NULL;
+	struct file *filp = NULL;
 
 	spin_lock_irq(&lo->lo_lock);
 	if (lo->lo_backing_file)
-		p = file_path(lo->lo_backing_file, buf, PAGE_SIZE - 1);
+		filp = get_file(lo->lo_backing_file);
 	spin_unlock_irq(&lo->lo_lock);
 
+	if (filp) {
+		p = file_path(filp, buf, PAGE_SIZE - 1);
+		fput(filp);
+	}
+
 	if (IS_ERR_OR_NULL(p))
 		ret = PTR_ERR(p);
 	else {
@@ -911,27 +918,83 @@ static void loop_config_discard(struct loop_device *lo)
 		blk_queue_flag_clear(QUEUE_FLAG_DISCARD, q);
 }
 
-static void loop_unprepare_queue(struct loop_device *lo)
-{
-	kthread_flush_worker(&lo->worker);
-	kthread_stop(lo->worker_task);
-}
+struct loop_worker {
+	struct rb_node rb_node;
+	struct work_struct work;
+	struct list_head cmd_list;
+	struct list_head idle_list;
+	struct loop_device *lo;
+	struct cgroup_subsys_state *css;
+	unsigned long last_ran_at;
+};
 
-static int loop_kthread_worker_fn(void *worker_ptr)
-{
-	current->flags |= PF_LESS_THROTTLE | PF_MEMALLOC_NOIO;
-	return kthread_worker_fn(worker_ptr);
-}
+static void loop_workfn(struct work_struct *work);
+static void loop_rootcg_workfn(struct work_struct *work);
+static void loop_free_idle_workers(struct timer_list *timer);
 
-static int loop_prepare_queue(struct loop_device *lo)
+static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 {
-	kthread_init_worker(&lo->worker);
-	lo->worker_task = kthread_run(loop_kthread_worker_fn,
-			&lo->worker, "loop%d", lo->lo_number);
-	if (IS_ERR(lo->worker_task))
-		return -ENOMEM;
-	set_user_nice(lo->worker_task, MIN_NICE);
-	return 0;
+	struct rb_node **node = &(lo->worker_tree.rb_node), *parent = NULL;
+	struct loop_worker *cur_worker, *worker = NULL;
+	struct work_struct *work;
+	struct list_head *cmd_list;
+
+	spin_lock_irq(&lo->lo_lock);
+
+	if (!cmd->css)
+		goto queue_work;
+
+	node = &lo->worker_tree.rb_node;
+
+	while (*node) {
+		parent = *node;
+		cur_worker = container_of(*node, struct loop_worker, rb_node);
+		if (cur_worker->css == cmd->css) {
+			worker = cur_worker;
+			break;
+		} else if ((long)cur_worker->css < (long)cmd->css) {
+			node = &(*node)->rb_left;
+		} else {
+			node = &(*node)->rb_right;
+		}
+	}
+	if (worker)
+		goto queue_work;
+
+	worker = kzalloc(sizeof(struct loop_worker), GFP_NOWAIT | __GFP_NOWARN);
+	/*
+	 * In the event we cannot allocate a worker, just queue on the
+	 * rootcg worker
+	 */
+	if (!worker)
+		goto queue_work;
+
+	worker->css = cmd->css;
+	css_get(worker->css);
+	INIT_WORK(&worker->work, loop_workfn);
+	INIT_LIST_HEAD(&worker->cmd_list);
+	INIT_LIST_HEAD(&worker->idle_list);
+	worker->lo = lo;
+	rb_link_node(&worker->rb_node, parent, node);
+	rb_insert_color(&worker->rb_node, &lo->worker_tree);
+queue_work:
+	if (worker) {
+		/*
+		 * We need to remove from the idle list here while
+		 * holding the lock so that the idle timer doesn't
+		 * free the worker
+		 */
+		if (!list_empty(&worker->idle_list))
+			list_del_init(&worker->idle_list);
+		work = &worker->work;
+		cmd_list = &worker->cmd_list;
+	} else {
+		work = &lo->rootcg_work;
+		cmd_list = &lo->rootcg_cmd_list;
+	}
+	list_add_tail(&cmd->list_entry, cmd_list);
+	queue_work(lo->workqueue, work);
+	spin_unlock_irq(&lo->lo_lock);
 }
 
 static void loop_update_rotational(struct loop_device *lo)
@@ -1007,14 +1070,25 @@ static int loop_set_fd(struct loop_device *lo, fmode_t mode,
 	size = get_loop_size(lo, file);
 	if ((loff_t)(sector_t)size != size)
 		goto out_unlock;
-	error = loop_prepare_queue(lo);
-	if (error)
+	lo->workqueue = alloc_workqueue("loop%d",
+					WQ_UNBOUND | WQ_FREEZABLE |
+					WQ_MEM_RECLAIM,
+					lo->lo_number);
+	if (!lo->workqueue) {
+		error = -ENOMEM;
 		goto out_unlock;
+	}
 
 	error = 0;
 
 	set_device_ro(bdev, (lo_flags & LO_FLAGS_READ_ONLY) != 0);
 
+	INIT_WORK(&lo->rootcg_work, loop_rootcg_workfn);
+	INIT_LIST_HEAD(&lo->rootcg_cmd_list);
+	INIT_LIST_HEAD(&lo->idle_worker_list);
+	lo->worker_tree = RB_ROOT;
+	timer_setup(&lo->timer, loop_free_idle_workers,
+		TIMER_DEFERRABLE);
 	lo->use_dio = false;
 	lo->lo_device = bdev;
 	lo->lo_flags = lo_flags;
@@ -1123,6 +1197,7 @@ static int __loop_clr_fd(struct loop_device *lo, bool release)
 	int err = 0;
 	bool partscan = false;
 	int lo_number;
+	struct loop_worker *pos, *worker;
 
 	mutex_lock(&loop_ctl_mutex);
 	if (WARN_ON_ONCE(lo->lo_state != Lo_rundown)) {
@@ -1139,9 +1214,18 @@ static int __loop_clr_fd(struct loop_device *lo, bool release)
 	/* freeze request queue during the transition */
 	blk_mq_freeze_queue(lo->lo_queue);
 
+	destroy_workqueue(lo->workqueue);
 	spin_lock_irq(&lo->lo_lock);
 	lo->lo_backing_file = NULL;
+	list_for_each_entry_safe(worker, pos, &lo->idle_worker_list,
+				idle_list) {
+		list_del(&worker->idle_list);
+		rb_erase(&worker->rb_node, &lo->worker_tree);
+		css_put(worker->css);
+		kfree(worker);
+	}
 	spin_unlock_irq(&lo->lo_lock);
+	del_timer_sync(&lo->timer);
 
 	loop_release_xfer(lo);
 	lo->transfer = NULL;
@@ -1176,7 +1260,6 @@ static int __loop_clr_fd(struct loop_device *lo, bool release)
 
 	partscan = lo->lo_flags & LO_FLAGS_PARTSCAN && bdev;
 	lo_number = lo->lo_number;
-	loop_unprepare_queue(lo);
 out_unlock:
 	mutex_unlock(&loop_ctl_mutex);
 	if (partscan) {
@@ -1954,7 +2037,7 @@ static blk_status_t loop_queue_rq(struct blk_mq_hw_ctx *hctx,
 	} else
 #endif
 		cmd->css = NULL;
-	kthread_queue_work(&lo->worker, &cmd->work);
+	loop_queue_work(lo, cmd);
 
 	return BLK_STS_OK;
 }
@@ -1983,26 +2066,82 @@ static void loop_handle_cmd(struct loop_cmd *cmd)
 	}
 }
 
-static void loop_queue_work(struct kthread_work *work)
+static void loop_set_timer(struct loop_device *lo)
+{
+	timer_reduce(&lo->timer, jiffies + LOOP_IDLE_WORKER_TIMEOUT);
+}
+
+static void loop_process_work(struct loop_worker *worker,
+			struct list_head *cmd_list, struct loop_device *lo)
 {
-	struct loop_cmd *cmd =
-		container_of(work, struct loop_cmd, work);
+	int orig_flags = current->flags;
+	struct loop_cmd *cmd;
 
-	loop_handle_cmd(cmd);
+	current->flags |= PF_LESS_THROTTLE | PF_MEMALLOC_NOIO;
+	spin_lock_irq(&lo->lo_lock);
+	while (!list_empty(cmd_list)) {
+		cmd = container_of(
+			cmd_list->next, struct loop_cmd, list_entry);
+		list_del(cmd_list->next);
+		spin_unlock_irq(&lo->lo_lock);
+
+		loop_handle_cmd(cmd);
+		cond_resched();
+
+		spin_lock_irq(&lo->lo_lock);
+	}
+
+	/*
+	 * We only add to the idle list if there are no pending cmds
+	 * *and* the worker will not run again which ensures that it
+	 * is safe to free any worker on the idle list
+	 */
+	if (worker && !work_pending(&worker->work)) {
+		worker->last_ran_at = jiffies;
+		list_add_tail(&worker->idle_list, &lo->idle_worker_list);
+		loop_set_timer(lo);
+	}
+	spin_unlock_irq(&lo->lo_lock);
+	current->flags = orig_flags;
 }
 
-static int loop_init_request(struct blk_mq_tag_set *set, struct request *rq,
-		unsigned int hctx_idx, unsigned int numa_node)
+static void loop_workfn(struct work_struct *work)
 {
-	struct loop_cmd *cmd = blk_mq_rq_to_pdu(rq);
+	struct loop_worker *worker =
+		container_of(work, struct loop_worker, work);
+	loop_process_work(worker, &worker->cmd_list, worker->lo);
+}
 
-	kthread_init_work(&cmd->work, loop_queue_work);
-	return 0;
+static void loop_rootcg_workfn(struct work_struct *work)
+{
+	struct loop_device *lo =
+		container_of(work, struct loop_device, rootcg_work);
+	loop_process_work(NULL, &lo->rootcg_cmd_list, lo);
+}
+
+static void loop_free_idle_workers(struct timer_list *timer)
+{
+	struct loop_device *lo = container_of(timer, struct loop_device, timer);
+	struct loop_worker *pos, *worker;
+
+	spin_lock_irq(&lo->lo_lock);
+	list_for_each_entry_safe(worker, pos, &lo->idle_worker_list,
+				idle_list) {
+		if (time_is_after_jiffies(worker->last_ran_at +
+						LOOP_IDLE_WORKER_TIMEOUT))
+			break;
+		list_del(&worker->idle_list);
+		rb_erase(&worker->rb_node, &lo->worker_tree);
+		css_put(worker->css);
+		kfree(worker);
+	}
+	if (!list_empty(&lo->idle_worker_list))
+		loop_set_timer(lo);
+	spin_unlock_irq(&lo->lo_lock);
 }
 
 static const struct blk_mq_ops loop_mq_ops = {
 	.queue_rq       = loop_queue_rq,
-	.init_request	= loop_init_request,
 	.complete	= lo_complete_rq,
 };
 
diff --git a/drivers/block/loop.h b/drivers/block/loop.h
index af75a5ee4094..87fd0e372227 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -14,7 +14,6 @@
 #include <linux/blk-mq.h>
 #include <linux/spinlock.h>
 #include <linux/mutex.h>
-#include <linux/kthread.h>
 #include <uapi/linux/loop.h>
 
 /* Possible states of device */
@@ -54,8 +53,12 @@ struct loop_device {
 
 	spinlock_t		lo_lock;
 	int			lo_state;
-	struct kthread_worker	worker;
-	struct task_struct	*worker_task;
+	struct workqueue_struct *workqueue;
+	struct work_struct      rootcg_work;
+	struct list_head        rootcg_cmd_list;
+	struct list_head        idle_worker_list;
+	struct rb_root          worker_tree;
+	struct timer_list       timer;
 	bool			use_dio;
 	bool			sysfs_inited;
 
@@ -65,7 +68,7 @@ struct loop_device {
 };
 
 struct loop_cmd {
-	struct kthread_work work;
+	struct list_head list_entry;
 	bool use_aio; /* use AIO interface to handle I/O */
 	atomic_t ref; /* only for aio */
 	long ret;
-- 
2.24.1


^ permalink raw reply related	[flat|nested] 20+ messages in thread

* [PATCH v5 2/4] mm: support nesting memalloc_use_memcg()
  2020-04-28 16:13 [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dan Schatzberg
  2020-04-28 16:13 ` [PATCH v5 1/4] loop: Use worker per cgroup instead of kworker Dan Schatzberg
@ 2020-04-28 16:13 ` Dan Schatzberg
  2020-04-28 16:13 ` [PATCH v5 3/4] mm: Charge active memcg when no mm is set Dan Schatzberg
                   ` (3 subsequent siblings)
  5 siblings, 0 replies; 20+ messages in thread
From: Dan Schatzberg @ 2020-04-28 16:13 UTC (permalink / raw)
  Cc: Johannes Weiner, Shakeel Butt, Roman Gushchin, Jens Axboe,
	Alexander Viro, Jan Kara, Amir Goldstein, Tejun Heo, Li Zefan,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Chris Down, Yang Shi, Ingo Molnar, Peter Zijlstra (Intel),
	Mathieu Desnoyers, Andrea Arcangeli, Dan Schatzberg,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

From: Johannes Weiner <hannes@cmpxchg.org>

The memalloc_use_memcg() function to override the default memcg
accounting context currently doesn't nest. But the patches to make the
loop driver cgroup-aware will end up nesting:

[   98.137605]  alloc_page_buffers+0x210/0x288
[   98.141799]  __getblk_gfp+0x1d4/0x400
[   98.145475]  ext4_read_block_bitmap_nowait+0x148/0xbc8
[   98.150628]  ext4_mb_init_cache+0x25c/0x9b0
[   98.154821]  ext4_mb_init_group+0x270/0x390
[   98.159014]  ext4_mb_good_group+0x264/0x270
[   98.163208]  ext4_mb_regular_allocator+0x480/0x798
[   98.168011]  ext4_mb_new_blocks+0x958/0x10f8
[   98.172294]  ext4_ext_map_blocks+0xec8/0x1618
[   98.176660]  ext4_map_blocks+0x1b8/0x8a0
[   98.180592]  ext4_writepages+0x830/0xf10
[   98.184523]  do_writepages+0xb4/0x198
[   98.188195]  __filemap_fdatawrite_range+0x170/0x1c8
[   98.193086]  filemap_write_and_wait_range+0x40/0xb0
[   98.197974]  ext4_punch_hole+0x4a4/0x660
[   98.201907]  ext4_fallocate+0x294/0x1190
[   98.205839]  loop_process_work+0x690/0x1100
[   98.210032]  loop_workfn+0x2c/0x110
[   98.213529]  process_one_work+0x3e0/0x648
[   98.217546]  worker_thread+0x70/0x670
[   98.221217]  kthread+0x1b8/0x1c0
[   98.224452]  ret_from_fork+0x10/0x18

where loop_process_work() sets the memcg override to the memcg that
submitted the IO request, and alloc_page_buffers() sets the override
to the memcg that instantiated the cache page, which may differ.

Make memalloc_use_memcg() return the old memcg and convert existing
users to a stacking model. Delete the unused memalloc_unuse_memcg().

Signed-off-by: Johannes Weiner <hannes@cmpxchg.org>
Reviewed-by: Shakeel Butt <shakeelb@google.com>
Acked-by: Roman Gushchin <guro@fb.com>
---
 fs/buffer.c                          |  6 +++---
 fs/notify/fanotify/fanotify.c        |  5 +++--
 fs/notify/inotify/inotify_fsnotify.c |  5 +++--
 include/linux/sched/mm.h             | 28 +++++++++-------------------
 4 files changed, 18 insertions(+), 26 deletions(-)

diff --git a/fs/buffer.c b/fs/buffer.c
index 599a0bf7257b..b4e99c6b52ec 100644
--- a/fs/buffer.c
+++ b/fs/buffer.c
@@ -851,13 +851,13 @@ struct buffer_head *alloc_page_buffers(struct page *page, unsigned long size,
 	struct buffer_head *bh, *head;
 	gfp_t gfp = GFP_NOFS | __GFP_ACCOUNT;
 	long offset;
-	struct mem_cgroup *memcg;
+	struct mem_cgroup *memcg, *old_memcg;
 
 	if (retry)
 		gfp |= __GFP_NOFAIL;
 
 	memcg = get_mem_cgroup_from_page(page);
-	memalloc_use_memcg(memcg);
+	old_memcg = memalloc_use_memcg(memcg);
 
 	head = NULL;
 	offset = PAGE_SIZE;
@@ -876,7 +876,7 @@ struct buffer_head *alloc_page_buffers(struct page *page, unsigned long size,
 		set_bh_page(bh, page, offset);
 	}
 out:
-	memalloc_unuse_memcg();
+	memalloc_use_memcg(old_memcg);
 	mem_cgroup_put(memcg);
 	return head;
 /*
diff --git a/fs/notify/fanotify/fanotify.c b/fs/notify/fanotify/fanotify.c
index 5435a40f82be..54c787cd6efb 100644
--- a/fs/notify/fanotify/fanotify.c
+++ b/fs/notify/fanotify/fanotify.c
@@ -353,6 +353,7 @@ struct fanotify_event *fanotify_alloc_event(struct fsnotify_group *group,
 	gfp_t gfp = GFP_KERNEL_ACCOUNT;
 	struct inode *id = fanotify_fid_inode(inode, mask, data, data_type);
 	const struct path *path = fsnotify_data_path(data, data_type);
+	struct mem_cgroup *oldmemcg;
 
 	/*
 	 * For queues with unlimited length lost events are not expected and
@@ -366,7 +367,7 @@ struct fanotify_event *fanotify_alloc_event(struct fsnotify_group *group,
 		gfp |= __GFP_RETRY_MAYFAIL;
 
 	/* Whoever is interested in the event, pays for the allocation. */
-	memalloc_use_memcg(group->memcg);
+	oldmemcg = memalloc_use_memcg(group->memcg);
 
 	if (fanotify_is_perm_event(mask)) {
 		struct fanotify_perm_event *pevent;
@@ -451,7 +452,7 @@ struct fanotify_event *fanotify_alloc_event(struct fsnotify_group *group,
 		}
 	}
 out:
-	memalloc_unuse_memcg();
+	memalloc_use_memcg(oldmemcg);
 	return event;
 }
 
diff --git a/fs/notify/inotify/inotify_fsnotify.c b/fs/notify/inotify/inotify_fsnotify.c
index 2ebc89047153..d27c6e83cea6 100644
--- a/fs/notify/inotify/inotify_fsnotify.c
+++ b/fs/notify/inotify/inotify_fsnotify.c
@@ -69,6 +69,7 @@ int inotify_handle_event(struct fsnotify_group *group,
 	int ret;
 	int len = 0;
 	int alloc_len = sizeof(struct inotify_event_info);
+	struct mem_cgroup *oldmemcg;
 
 	if (WARN_ON(fsnotify_iter_vfsmount_mark(iter_info)))
 		return 0;
@@ -93,9 +94,9 @@ int inotify_handle_event(struct fsnotify_group *group,
 	 * trigger OOM killer in the target monitoring memcg as it may have
 	 * security repercussion.
 	 */
-	memalloc_use_memcg(group->memcg);
+	oldmemcg = memalloc_use_memcg(group->memcg);
 	event = kmalloc(alloc_len, GFP_KERNEL_ACCOUNT | __GFP_RETRY_MAYFAIL);
-	memalloc_unuse_memcg();
+	memalloc_use_memcg(oldmemcg);
 
 	if (unlikely(!event)) {
 		/*
diff --git a/include/linux/sched/mm.h b/include/linux/sched/mm.h
index c49257a3b510..95e8bfb0cab1 100644
--- a/include/linux/sched/mm.h
+++ b/include/linux/sched/mm.h
@@ -316,31 +316,21 @@ static inline void memalloc_nocma_restore(unsigned int flags)
  * __GFP_ACCOUNT allocations till the end of the scope will be charged to the
  * given memcg.
  *
- * NOTE: This function is not nesting safe.
+ * NOTE: This function can nest. Users must save the return value and
+ * reset the previous value after their own charging scope is over
  */
-static inline void memalloc_use_memcg(struct mem_cgroup *memcg)
+static inline struct mem_cgroup *
+memalloc_use_memcg(struct mem_cgroup *memcg)
 {
-	WARN_ON_ONCE(current->active_memcg);
+	struct mem_cgroup *old = current->active_memcg;
 	current->active_memcg = memcg;
-}
-
-/**
- * memalloc_unuse_memcg - Ends the remote memcg charging scope.
- *
- * This function marks the end of the remote memcg charging scope started by
- * memalloc_use_memcg().
- */
-static inline void memalloc_unuse_memcg(void)
-{
-	current->active_memcg = NULL;
+	return old;
 }
 #else
-static inline void memalloc_use_memcg(struct mem_cgroup *memcg)
-{
-}
-
-static inline void memalloc_unuse_memcg(void)
+static inline struct mem_cgroup *
+memalloc_use_memcg(struct mem_cgroup *memcg)
 {
+	return NULL;
 }
 #endif
 
-- 
2.24.1


^ permalink raw reply related	[flat|nested] 20+ messages in thread

* [PATCH v5 3/4] mm: Charge active memcg when no mm is set
  2020-04-28 16:13 [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dan Schatzberg
  2020-04-28 16:13 ` [PATCH v5 1/4] loop: Use worker per cgroup instead of kworker Dan Schatzberg
  2020-04-28 16:13 ` [PATCH v5 2/4] mm: support nesting memalloc_use_memcg() Dan Schatzberg
@ 2020-04-28 16:13 ` Dan Schatzberg
  2020-04-28 16:13 ` [PATCH v5 4/4] loop: Charge i/o to mem and blk cg Dan Schatzberg
                   ` (2 subsequent siblings)
  5 siblings, 0 replies; 20+ messages in thread
From: Dan Schatzberg @ 2020-04-28 16:13 UTC (permalink / raw)
  Cc: Dan Schatzberg, Johannes Weiner, Tejun Heo, Chris Down,
	Shakeel Butt, Jens Axboe, Alexander Viro, Jan Kara,
	Amir Goldstein, Li Zefan, Michal Hocko, Vladimir Davydov,
	Andrew Morton, Hugh Dickins, Roman Gushchin, Yang Shi,
	Peter Zijlstra (Intel),
	Ingo Molnar, Mathieu Desnoyers, Kirill A. Shutemov,
	Andrea Arcangeli, Thomas Gleixner, open list:BLOCK LAYER,
	open list, open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

memalloc_use_memcg() worked for kernel allocations but was silently
ignored for user pages.

This patch establishes a precedence order for who gets charged:

1. If there is a memcg associated with the page already, that memcg is
   charged. This happens during swapin.

2. If an explicit mm is passed, mm->memcg is charged. This happens
   during page faults, which can be triggered in remote VMs (eg gup).

3. Otherwise consult the current process context. If it has configured
   a current->active_memcg, use that. Otherwise, current->mm->memcg.

Previously, if a NULL mm was passed to mem_cgroup_try_charge (case 3) it
would always charge the root cgroup. Now it looks up the current
active_memcg first (falling back to charging the root cgroup if not
set).

Signed-off-by: Dan Schatzberg <schatzberg.dan@gmail.com>
Acked-by: Johannes Weiner <hannes@cmpxchg.org>
Acked-by: Tejun Heo <tj@kernel.org>
Acked-by: Chris Down <chris@chrisdown.name>
Reviewed-by: Shakeel Butt <shakeelb@google.com>
---
 mm/memcontrol.c | 11 ++++++++---
 mm/shmem.c      |  4 ++--
 2 files changed, 10 insertions(+), 5 deletions(-)

diff --git a/mm/memcontrol.c b/mm/memcontrol.c
index 5beea03dd58a..af68d1d7b456 100644
--- a/mm/memcontrol.c
+++ b/mm/memcontrol.c
@@ -6435,7 +6435,8 @@ enum mem_cgroup_protection mem_cgroup_protected(struct mem_cgroup *root,
  * @compound: charge the page as compound or small page
  *
  * Try to charge @page to the memcg that @mm belongs to, reclaiming
- * pages according to @gfp_mask if necessary.
+ * pages according to @gfp_mask if necessary. if @mm is NULL, try to
+ * charge to the active memcg.
  *
  * Returns 0 on success, with *@memcgp pointing to the charged memcg.
  * Otherwise, an error code is returned.
@@ -6479,8 +6480,12 @@ int mem_cgroup_try_charge(struct page *page, struct mm_struct *mm,
 		}
 	}
 
-	if (!memcg)
-		memcg = get_mem_cgroup_from_mm(mm);
+	if (!memcg) {
+		if (!mm)
+			memcg = get_mem_cgroup_from_current();
+		else
+			memcg = get_mem_cgroup_from_mm(mm);
+	}
 
 	ret = try_charge(memcg, gfp_mask, nr_pages);
 
diff --git a/mm/shmem.c b/mm/shmem.c
index d722eb830317..8c8ffc35a957 100644
--- a/mm/shmem.c
+++ b/mm/shmem.c
@@ -1618,7 +1618,7 @@ static int shmem_swapin_page(struct inode *inode, pgoff_t index,
 {
 	struct address_space *mapping = inode->i_mapping;
 	struct shmem_inode_info *info = SHMEM_I(inode);
-	struct mm_struct *charge_mm = vma ? vma->vm_mm : current->mm;
+	struct mm_struct *charge_mm = vma ? vma->vm_mm : NULL;
 	struct mem_cgroup *memcg;
 	struct page *page;
 	swp_entry_t swap;
@@ -1753,7 +1753,7 @@ static int shmem_getpage_gfp(struct inode *inode, pgoff_t index,
 	}
 
 	sbinfo = SHMEM_SB(inode->i_sb);
-	charge_mm = vma ? vma->vm_mm : current->mm;
+	charge_mm = vma ? vma->vm_mm : NULL;
 
 	page = find_lock_entry(mapping, index);
 	if (xa_is_value(page)) {
-- 
2.24.1


^ permalink raw reply related	[flat|nested] 20+ messages in thread

* [PATCH v5 4/4] loop: Charge i/o to mem and blk cg
  2020-04-28 16:13 [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dan Schatzberg
                   ` (2 preceding siblings ...)
  2020-04-28 16:13 ` [PATCH v5 3/4] mm: Charge active memcg when no mm is set Dan Schatzberg
@ 2020-04-28 16:13 ` Dan Schatzberg
  2020-04-28 21:47 ` [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dave Chinner
  2020-05-12 13:25 ` Dan Schatzberg
  5 siblings, 0 replies; 20+ messages in thread
From: Dan Schatzberg @ 2020-04-28 16:13 UTC (permalink / raw)
  Cc: Dan Schatzberg, Johannes Weiner, Jens Axboe, Alexander Viro,
	Jan Kara, Amir Goldstein, Tejun Heo, Li Zefan, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Shakeel Butt, Chris Down, Yang Shi, Ingo Molnar,
	Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Thomas Gleixner,
	Andrea Arcangeli, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

The current code only associates with the existing blkcg when aio is
used to access the backing file. This patch covers all types of i/o to
the backing file and also associates the memcg so if the backing file is
on tmpfs, memory is charged appropriately.

This patch also exports cgroup_get_e_css so it can be used by the loop
module.

Signed-off-by: Dan Schatzberg <schatzberg.dan@gmail.com>
Acked-by: Johannes Weiner <hannes@cmpxchg.org>
---
 drivers/block/loop.c       | 61 +++++++++++++++++++++++++-------------
 drivers/block/loop.h       |  3 +-
 include/linux/memcontrol.h |  6 ++++
 kernel/cgroup/cgroup.c     |  1 +
 4 files changed, 50 insertions(+), 21 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 49d7d1f62d88..4da0836f58be 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -77,6 +77,7 @@
 #include <linux/uio.h>
 #include <linux/ioprio.h>
 #include <linux/blk-cgroup.h>
+#include <linux/sched/mm.h>
 
 #include "loop.h"
 
@@ -507,8 +508,6 @@ static void lo_rw_aio_complete(struct kiocb *iocb, long ret, long ret2)
 {
 	struct loop_cmd *cmd = container_of(iocb, struct loop_cmd, iocb);
 
-	if (cmd->css)
-		css_put(cmd->css);
 	cmd->ret = ret;
 	lo_rw_aio_do_completion(cmd);
 }
@@ -569,8 +568,6 @@ static int lo_rw_aio(struct loop_device *lo, struct loop_cmd *cmd,
 	cmd->iocb.ki_complete = lo_rw_aio_complete;
 	cmd->iocb.ki_flags = IOCB_DIRECT;
 	cmd->iocb.ki_ioprio = IOPRIO_PRIO_VALUE(IOPRIO_CLASS_NONE, 0);
-	if (cmd->css)
-		kthread_associate_blkcg(cmd->css);
 
 	if (rw == WRITE)
 		ret = call_write_iter(file, &cmd->iocb, &iter);
@@ -578,7 +575,6 @@ static int lo_rw_aio(struct loop_device *lo, struct loop_cmd *cmd,
 		ret = call_read_iter(file, &cmd->iocb, &iter);
 
 	lo_rw_aio_do_completion(cmd);
-	kthread_associate_blkcg(NULL);
 
 	if (ret != -EIOCBQUEUED)
 		cmd->iocb.ki_complete(&cmd->iocb, ret, 0);
@@ -924,7 +920,7 @@ struct loop_worker {
 	struct list_head cmd_list;
 	struct list_head idle_list;
 	struct loop_device *lo;
-	struct cgroup_subsys_state *css;
+	struct cgroup_subsys_state *blkcg_css;
 	unsigned long last_ran_at;
 };
 
@@ -941,7 +937,7 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 
 	spin_lock_irq(&lo->lo_lock);
 
-	if (!cmd->css)
+	if (!cmd->blkcg_css)
 		goto queue_work;
 
 	node = &lo->worker_tree.rb_node;
@@ -949,10 +945,10 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 	while (*node) {
 		parent = *node;
 		cur_worker = container_of(*node, struct loop_worker, rb_node);
-		if (cur_worker->css == cmd->css) {
+		if (cur_worker->blkcg_css == cmd->blkcg_css) {
 			worker = cur_worker;
 			break;
-		} else if ((long)cur_worker->css < (long)cmd->css) {
+		} else if ((long)cur_worker->blkcg_css < (long)cmd->blkcg_css) {
 			node = &(*node)->rb_left;
 		} else {
 			node = &(*node)->rb_right;
@@ -964,13 +960,18 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 	worker = kzalloc(sizeof(struct loop_worker), GFP_NOWAIT | __GFP_NOWARN);
 	/*
 	 * In the event we cannot allocate a worker, just queue on the
-	 * rootcg worker
+	 * rootcg worker and issue the I/O as the rootcg
 	 */
-	if (!worker)
+	if (!worker) {
+		cmd->blkcg_css = NULL;
+		if (cmd->memcg_css)
+			css_put(cmd->memcg_css);
+		cmd->memcg_css = NULL;
 		goto queue_work;
+	}
 
-	worker->css = cmd->css;
-	css_get(worker->css);
+	worker->blkcg_css = cmd->blkcg_css;
+	css_get(worker->blkcg_css);
 	INIT_WORK(&worker->work, loop_workfn);
 	INIT_LIST_HEAD(&worker->cmd_list);
 	INIT_LIST_HEAD(&worker->idle_list);
@@ -1221,7 +1222,7 @@ static int __loop_clr_fd(struct loop_device *lo, bool release)
 				idle_list) {
 		list_del(&worker->idle_list);
 		rb_erase(&worker->rb_node, &lo->worker_tree);
-		css_put(worker->css);
+		css_put(worker->blkcg_css);
 		kfree(worker);
 	}
 	spin_unlock_irq(&lo->lo_lock);
@@ -2030,13 +2031,18 @@ static blk_status_t loop_queue_rq(struct blk_mq_hw_ctx *hctx,
 	}
 
 	/* always use the first bio's css */
+	cmd->blkcg_css = NULL;
+	cmd->memcg_css = NULL;
 #ifdef CONFIG_BLK_CGROUP
-	if (cmd->use_aio && rq->bio && rq->bio->bi_blkg) {
-		cmd->css = &bio_blkcg(rq->bio)->css;
-		css_get(cmd->css);
-	} else
+	if (rq->bio && rq->bio->bi_blkg) {
+		cmd->blkcg_css = &bio_blkcg(rq->bio)->css;
+#ifdef CONFIG_MEMCG
+		cmd->memcg_css =
+			cgroup_get_e_css(cmd->blkcg_css->cgroup,
+					&memory_cgrp_subsys);
+#endif
+	}
 #endif
-		cmd->css = NULL;
 	loop_queue_work(lo, cmd);
 
 	return BLK_STS_OK;
@@ -2048,13 +2054,28 @@ static void loop_handle_cmd(struct loop_cmd *cmd)
 	const bool write = op_is_write(req_op(rq));
 	struct loop_device *lo = rq->q->queuedata;
 	int ret = 0;
+	struct mem_cgroup *old_memcg = NULL;
 
 	if (write && (lo->lo_flags & LO_FLAGS_READ_ONLY)) {
 		ret = -EIO;
 		goto failed;
 	}
 
+	if (cmd->blkcg_css)
+		kthread_associate_blkcg(cmd->blkcg_css);
+	if (cmd->memcg_css)
+		old_memcg = memalloc_use_memcg(
+			mem_cgroup_from_css(cmd->memcg_css));
+
 	ret = do_req_filebacked(lo, rq);
+
+	if (cmd->blkcg_css)
+		kthread_associate_blkcg(NULL);
+
+	if (cmd->memcg_css) {
+		memalloc_use_memcg(old_memcg);
+		css_put(cmd->memcg_css);
+	}
  failed:
 	/* complete non-aio request */
 	if (!cmd->use_aio || ret) {
@@ -2132,7 +2153,7 @@ static void loop_free_idle_workers(struct timer_list *timer)
 			break;
 		list_del(&worker->idle_list);
 		rb_erase(&worker->rb_node, &lo->worker_tree);
-		css_put(worker->css);
+		css_put(worker->blkcg_css);
 		kfree(worker);
 	}
 	if (!list_empty(&lo->idle_worker_list))
diff --git a/drivers/block/loop.h b/drivers/block/loop.h
index 87fd0e372227..3e65acf7a0e9 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -74,7 +74,8 @@ struct loop_cmd {
 	long ret;
 	struct kiocb iocb;
 	struct bio_vec *bvec;
-	struct cgroup_subsys_state *css;
+	struct cgroup_subsys_state *blkcg_css;
+	struct cgroup_subsys_state *memcg_css;
 };
 
 /* Support for loadable transfer modules */
diff --git a/include/linux/memcontrol.h b/include/linux/memcontrol.h
index d275c72c4f8e..a85cbafae6af 100644
--- a/include/linux/memcontrol.h
+++ b/include/linux/memcontrol.h
@@ -923,6 +923,12 @@ static inline struct mem_cgroup *get_mem_cgroup_from_page(struct page *page)
 	return NULL;
 }
 
+static inline
+struct mem_cgroup *mem_cgroup_from_css(struct cgroup_subsys_state *css)
+{
+	return NULL;
+}
+
 static inline void mem_cgroup_put(struct mem_cgroup *memcg)
 {
 }
diff --git a/kernel/cgroup/cgroup.c b/kernel/cgroup/cgroup.c
index 06b5ea9d899d..a3c64d961d68 100644
--- a/kernel/cgroup/cgroup.c
+++ b/kernel/cgroup/cgroup.c
@@ -587,6 +587,7 @@ struct cgroup_subsys_state *cgroup_get_e_css(struct cgroup *cgrp,
 	rcu_read_unlock();
 	return css;
 }
+EXPORT_SYMBOL_GPL(cgroup_get_e_css);
 
 static void cgroup_get_live(struct cgroup *cgrp)
 {
-- 
2.24.1


^ permalink raw reply related	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-04-28 16:13 [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dan Schatzberg
                   ` (3 preceding siblings ...)
  2020-04-28 16:13 ` [PATCH v5 4/4] loop: Charge i/o to mem and blk cg Dan Schatzberg
@ 2020-04-28 21:47 ` Dave Chinner
  2020-04-29  2:27   ` Johannes Weiner
                     ` (2 more replies)
  2020-05-12 13:25 ` Dan Schatzberg
  5 siblings, 3 replies; 20+ messages in thread
From: Dave Chinner @ 2020-04-28 21:47 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Alexander Viro, Jan Kara, Amir Goldstein, Tejun Heo,
	Li Zefan, Johannes Weiner, Michal Hocko, Vladimir Davydov,
	Andrew Morton, Hugh Dickins, Roman Gushchin, Shakeel Butt,
	Chris Down, Yang Shi, Ingo Molnar, Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Tue, Apr 28, 2020 at 12:13:46PM -0400, Dan Schatzberg wrote:
> The loop device runs all i/o to the backing file on a separate kworker
> thread which results in all i/o being charged to the root cgroup. This
> allows a loop device to be used to trivially bypass resource limits
> and other policy. This patch series fixes this gap in accounting.

How is this specific to the loop device? Isn't every block device
that offloads work to a kthread or single worker thread susceptible
to the same "exploit"?

Or is the problem simply that the loop worker thread is simply not
taking the IO's associated cgroup and submitting the IO with that
cgroup associated with it? That seems kinda simple to fix....

> Naively charging cgroups could result in priority inversions through
> the single kworker thread in the case where multiple cgroups are
> reading/writing to the same loop device.

And that's where all the complexity and serialisation comes from,
right?

So, again: how is this unique to the loop device? Other block
devices also offload IO to kthreads to do blocking work and IO
submission to lower layers. Hence this seems to me like a generic
"block device does IO submission from different task" issue that
should be handled by generic infrastructure and not need to be
reimplemented multiple times in every block device driver that
offloads work to other threads...

> This patch series does some
> minor modification to the loop driver so that each cgroup can make
> forward progress independently to avoid this inversion.
> 
> With this patch series applied, the above script triggers OOM kills
> when writing through the loop device as expected.

NACK!

The IO that is disallowed should fail with ENOMEM or some similar
error, not trigger an OOM kill that shoots some innocent bystander
in the head. That's worse than using BUG() to report errors...

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-04-28 21:47 ` [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dave Chinner
@ 2020-04-29  2:27   ` Johannes Weiner
  2020-05-05  6:29     ` Dave Chinner
  2020-04-29 10:25   ` Jan Kara
  2020-04-29 14:03   ` Dan Schatzberg
  2 siblings, 1 reply; 20+ messages in thread
From: Johannes Weiner @ 2020-04-29  2:27 UTC (permalink / raw)
  To: Dave Chinner
  Cc: Dan Schatzberg, Jens Axboe, Alexander Viro, Jan Kara,
	Amir Goldstein, Tejun Heo, Li Zefan, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Shakeel Butt, Chris Down, Yang Shi, Ingo Molnar,
	Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Wed, Apr 29, 2020 at 07:47:34AM +1000, Dave Chinner wrote:
> On Tue, Apr 28, 2020 at 12:13:46PM -0400, Dan Schatzberg wrote:
> > This patch series does some
> > minor modification to the loop driver so that each cgroup can make
> > forward progress independently to avoid this inversion.
> > 
> > With this patch series applied, the above script triggers OOM kills
> > when writing through the loop device as expected.
> 
> NACK!
> 
> The IO that is disallowed should fail with ENOMEM or some similar
> error, not trigger an OOM kill that shoots some innocent bystander
> in the head. That's worse than using BUG() to report errors...

Did you actually read the script?

It's OOMing because it's creating 256M worth of tmpfs pages inside a
64M cgroup. It's not killing an innocent bystander, it's killing in
the cgroup that is allocating all that memory - after Dan makes sure
that memory is accounted to its rightful owner.

As opposed to before this series, where all this memory isn't
accounted properly and goes to the root cgroup - where, ironically, it
could cause OOM and kill an actually innocent bystander.

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-04-28 21:47 ` [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dave Chinner
  2020-04-29  2:27   ` Johannes Weiner
@ 2020-04-29 10:25   ` Jan Kara
  2020-04-29 14:22     ` Tejun Heo
  2020-05-05  6:41     ` Dave Chinner
  2020-04-29 14:03   ` Dan Schatzberg
  2 siblings, 2 replies; 20+ messages in thread
From: Jan Kara @ 2020-04-29 10:25 UTC (permalink / raw)
  To: Dave Chinner
  Cc: Dan Schatzberg, Jens Axboe, Alexander Viro, Jan Kara,
	Amir Goldstein, Tejun Heo, Li Zefan, Johannes Weiner,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Roman Gushchin, Shakeel Butt, Chris Down, Yang Shi, Ingo Molnar,
	Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Wed 29-04-20 07:47:34, Dave Chinner wrote:
> On Tue, Apr 28, 2020 at 12:13:46PM -0400, Dan Schatzberg wrote:
> > The loop device runs all i/o to the backing file on a separate kworker
> > thread which results in all i/o being charged to the root cgroup. This
> > allows a loop device to be used to trivially bypass resource limits
> > and other policy. This patch series fixes this gap in accounting.
> 
> How is this specific to the loop device? Isn't every block device
> that offloads work to a kthread or single worker thread susceptible
> to the same "exploit"?
> 
> Or is the problem simply that the loop worker thread is simply not
> taking the IO's associated cgroup and submitting the IO with that
> cgroup associated with it? That seems kinda simple to fix....
> 
> > Naively charging cgroups could result in priority inversions through
> > the single kworker thread in the case where multiple cgroups are
> > reading/writing to the same loop device.
> 
> And that's where all the complexity and serialisation comes from,
> right?
> 
> So, again: how is this unique to the loop device? Other block
> devices also offload IO to kthreads to do blocking work and IO
> submission to lower layers. Hence this seems to me like a generic
> "block device does IO submission from different task" issue that
> should be handled by generic infrastructure and not need to be
> reimplemented multiple times in every block device driver that
> offloads work to other threads...

Yeah, I was thinking about the same when reading the patch series
description. We already have some cgroup workarounds for btrfs kthreads if
I remember correctly, we have cgroup handling for flush workers, now we are
adding cgroup handling for loopback device workers, and soon I'd expect
someone comes with a need for DM/MD worker processes and IMHO it's getting
out of hands because the complexity spreads through the kernel with every
subsystem comming with slightly different solution to the problem and also
the number of kthreads gets multiplied by the number of cgroups. So I
agree some generic solution how to approach IO throttling of kthreads /
workers would be desirable.

OTOH I don't have a great idea how the generic infrastructure should look
like...

								Honza
-- 
Jan Kara <jack@suse.com>
SUSE Labs, CR

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-04-28 21:47 ` [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dave Chinner
  2020-04-29  2:27   ` Johannes Weiner
  2020-04-29 10:25   ` Jan Kara
@ 2020-04-29 14:03   ` Dan Schatzberg
  2 siblings, 0 replies; 20+ messages in thread
From: Dan Schatzberg @ 2020-04-29 14:03 UTC (permalink / raw)
  To: Dave Chinner
  Cc: Jens Axboe, Alexander Viro, Jan Kara, Amir Goldstein, Tejun Heo,
	Li Zefan, Johannes Weiner, Michal Hocko, Vladimir Davydov,
	Andrew Morton, Hugh Dickins, Roman Gushchin, Shakeel Butt,
	Chris Down, Yang Shi, Ingo Molnar, Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Wed, Apr 29, 2020 at 07:47:34AM +1000, Dave Chinner wrote:
> On Tue, Apr 28, 2020 at 12:13:46PM -0400, Dan Schatzberg wrote:
> > The loop device runs all i/o to the backing file on a separate kworker
> > thread which results in all i/o being charged to the root cgroup. This
> > allows a loop device to be used to trivially bypass resource limits
> > and other policy. This patch series fixes this gap in accounting.
> 
> How is this specific to the loop device? Isn't every block device
> that offloads work to a kthread or single worker thread susceptible
> to the same "exploit"?

I believe this is fairly loop device specific. The issue is that the
loop driver issues I/O by re-entering the VFS layer (resulting in
tmpfs like in my example or entering the block layer). Normally, I/O
through the VFS layer is accounted for and controlled (e.g. you can
OOM if writing to tmpfs, or get throttled by the I/O controller) but
the loop device completely side-steps the accounting.

> 
> Or is the problem simply that the loop worker thread is simply not
> taking the IO's associated cgroup and submitting the IO with that
> cgroup associated with it? That seems kinda simple to fix....
> 
> > Naively charging cgroups could result in priority inversions through
> > the single kworker thread in the case where multiple cgroups are
> > reading/writing to the same loop device.
> 
> And that's where all the complexity and serialisation comes from,
> right?
> 
> So, again: how is this unique to the loop device? Other block
> devices also offload IO to kthreads to do blocking work and IO
> submission to lower layers. Hence this seems to me like a generic
> "block device does IO submission from different task" issue that
> should be handled by generic infrastructure and not need to be
> reimplemented multiple times in every block device driver that
> offloads work to other threads...

I'm not familiar with other block device drivers that behave like
this. Could you point me at a few?

> 
> > This patch series does some
> > minor modification to the loop driver so that each cgroup can make
> > forward progress independently to avoid this inversion.
> > 
> > With this patch series applied, the above script triggers OOM kills
> > when writing through the loop device as expected.
> 
> NACK!
> 
> The IO that is disallowed should fail with ENOMEM or some similar
> error, not trigger an OOM kill that shoots some innocent bystander
> in the head. That's worse than using BUG() to report errors...

The OOM behavior is due to cgroup limit. It mirrors the behavior one
sees when writing to a too-large tmpfs.

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-04-29 10:25   ` Jan Kara
@ 2020-04-29 14:22     ` Tejun Heo
  2020-04-29 16:21       ` Jan Kara
  2020-05-05  6:41     ` Dave Chinner
  1 sibling, 1 reply; 20+ messages in thread
From: Tejun Heo @ 2020-04-29 14:22 UTC (permalink / raw)
  To: Jan Kara
  Cc: Dave Chinner, Dan Schatzberg, Jens Axboe, Alexander Viro,
	Amir Goldstein, Li Zefan, Johannes Weiner, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Shakeel Butt, Chris Down, Yang Shi, Ingo Molnar,
	Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

Hello,

On Wed, Apr 29, 2020 at 12:25:40PM +0200, Jan Kara wrote:
> Yeah, I was thinking about the same when reading the patch series
> description. We already have some cgroup workarounds for btrfs kthreads if
> I remember correctly, we have cgroup handling for flush workers, now we are
> adding cgroup handling for loopback device workers, and soon I'd expect
> someone comes with a need for DM/MD worker processes and IMHO it's getting
> out of hands because the complexity spreads through the kernel with every
> subsystem comming with slightly different solution to the problem and also
> the number of kthreads gets multiplied by the number of cgroups. So I
> agree some generic solution how to approach IO throttling of kthreads /
> workers would be desirable.
> 
> OTOH I don't have a great idea how the generic infrastructure should look
> like...

I don't really see a way around that. The only generic solution would be
letting all IOs through as root and handle everything through backcharging,
which we already can do as backcharging is already in use to handle metadata
updates which can't be controlled directly. However, doing that for all IOs
would make the control quality a lot worse as all control would be based on
first incurring deficit and then try to punish the issuer after the fact.

The infrastructure work done to make IO control work for btrfs is generic
and the changes needed on btrfs side was pretty small. Most of the work was
identifying non-regular IO pathways (bouncing through different kthreads and
whatnot) and making sure they're annotating IO ownership and the needed
mechanism correctly. The biggest challenge probably is ensuring that the
filesystem doesn't add ordering dependency between separate data IOs, which
is a nice property to have with or without cgroup support.

That leaves the nesting drivers, loop and md/dm. Given that they sit in the
middle of IO stack and proxy a lot of its roles, they'll have to be updated
to be transparent in terms of cgroup ownership if IO control is gonna work
through them. Maybe we can have a common infra shared between loop, dm and
md but they aren't many and may also be sufficiently different. idk

Thanks.

-- 
tejun

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-04-29 14:22     ` Tejun Heo
@ 2020-04-29 16:21       ` Jan Kara
  0 siblings, 0 replies; 20+ messages in thread
From: Jan Kara @ 2020-04-29 16:21 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Jan Kara, Dave Chinner, Dan Schatzberg, Jens Axboe,
	Alexander Viro, Amir Goldstein, Li Zefan, Johannes Weiner,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Roman Gushchin, Shakeel Butt, Chris Down, Yang Shi, Ingo Molnar,
	Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Wed 29-04-20 10:22:30, Tejun Heo wrote:
> Hello,
> 
> On Wed, Apr 29, 2020 at 12:25:40PM +0200, Jan Kara wrote:
> > Yeah, I was thinking about the same when reading the patch series
> > description. We already have some cgroup workarounds for btrfs kthreads if
> > I remember correctly, we have cgroup handling for flush workers, now we are
> > adding cgroup handling for loopback device workers, and soon I'd expect
> > someone comes with a need for DM/MD worker processes and IMHO it's getting
> > out of hands because the complexity spreads through the kernel with every
> > subsystem comming with slightly different solution to the problem and also
> > the number of kthreads gets multiplied by the number of cgroups. So I
> > agree some generic solution how to approach IO throttling of kthreads /
> > workers would be desirable.
> > 
> > OTOH I don't have a great idea how the generic infrastructure should look
> > like...
> 
> I don't really see a way around that. The only generic solution would be
> letting all IOs through as root and handle everything through backcharging,
> which we already can do as backcharging is already in use to handle metadata
> updates which can't be controlled directly. However, doing that for all IOs
> would make the control quality a lot worse as all control would be based on
> first incurring deficit and then try to punish the issuer after the fact.

Yeah, it will be probably somewhat worse but OTOH given we'd track the IO
balance per cgroup there will deficit only when a cgroup is starting so it
could be bearable. I'm more concerned about issues like that for some IO
controllers (e.g. for blk-iolatency or for the work preserving
controllers), it is not obvious how to sensibly estimate some cost to
charge to a cgroup since these controllers are more about giving priority
to IO of some cgroup in presence of IO from another cgroup rather than some
hard throughput limit or something like that.

> The infrastructure work done to make IO control work for btrfs is generic
> and the changes needed on btrfs side was pretty small. Most of the work was
> identifying non-regular IO pathways (bouncing through different kthreads and
> whatnot) and making sure they're annotating IO ownership and the needed
> mechanism correctly. The biggest challenge probably is ensuring that the
> filesystem doesn't add ordering dependency between separate data IOs, which
> is a nice property to have with or without cgroup support.
> 
> That leaves the nesting drivers, loop and md/dm. Given that they sit in the
> middle of IO stack and proxy a lot of its roles, they'll have to be updated
> to be transparent in terms of cgroup ownership if IO control is gonna work
> through them. Maybe we can have a common infra shared between loop, dm and
> md but they aren't many and may also be sufficiently different. idk

Yeah, as I said, I don't really have a better alternative :-|

								Honza
-- 
Jan Kara <jack@suse.com>
SUSE Labs, CR

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-04-29  2:27   ` Johannes Weiner
@ 2020-05-05  6:29     ` Dave Chinner
  2020-05-05 13:55       ` Shakeel Butt
  2020-05-05 15:02       ` Johannes Weiner
  0 siblings, 2 replies; 20+ messages in thread
From: Dave Chinner @ 2020-05-05  6:29 UTC (permalink / raw)
  To: Johannes Weiner
  Cc: Dan Schatzberg, Jens Axboe, Alexander Viro, Jan Kara,
	Amir Goldstein, Tejun Heo, Li Zefan, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Shakeel Butt, Chris Down, Yang Shi, Ingo Molnar,
	Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Tue, Apr 28, 2020 at 10:27:32PM -0400, Johannes Weiner wrote:
> On Wed, Apr 29, 2020 at 07:47:34AM +1000, Dave Chinner wrote:
> > On Tue, Apr 28, 2020 at 12:13:46PM -0400, Dan Schatzberg wrote:
> > > This patch series does some
> > > minor modification to the loop driver so that each cgroup can make
> > > forward progress independently to avoid this inversion.
> > > 
> > > With this patch series applied, the above script triggers OOM kills
> > > when writing through the loop device as expected.
> > 
> > NACK!
> > 
> > The IO that is disallowed should fail with ENOMEM or some similar
> > error, not trigger an OOM kill that shoots some innocent bystander
> > in the head. That's worse than using BUG() to report errors...
> 
> Did you actually read the script?

Of course I did. You specifically mean this bit:

echo 64M > $CGROUP/memory.max;
mount -t tmpfs -o size=512m tmpfs /tmp;
truncate -s 512m /tmp/backing_file
losetup $LOOP_DEV /tmp/backing_file
dd if=/dev/zero of=$LOOP_DEV bs=1M count=256;

And with this patch set the dd gets OOM killed because the
/tmp/backing_file usage accounted to the cgroup goes over 64MB and
so tmpfs OOM kills the IO...

As I said: that's very broken behaviour from a storage stack
perspective.

> It's OOMing because it's creating 256M worth of tmpfs pages inside a
> 64M cgroup. It's not killing an innocent bystander, it's killing in
> the cgroup that is allocating all that memory - after Dan makes sure
> that memory is accounted to its rightful owner.

What this example does is turn /tmp into thinly provisioned storage
for $CGROUP via a restricted quota. It has a device size of 512MB,
but only has physical storage capacity of 64MB, as constrained by
the cgroup memory limit.  You're dealing with management of -storage
devices- and -IO error reporting- here, not memory management.

When thin provisioned storage runs out of space - for whatever
reason - and it cannot resolve the issue by blocking, then it *must*
return ENOSPC to the IO submitter to tell it the IO has failed. This
is no different to if we set a quota on /tmp/backing_file and it is
exhausted at 64MB of data written - we fail the IO with ENOSPC or
EDQUOT, depending on which quota we used.

IOWs, we have solid expectations on how block devices report
unsolvable resource shortages during IO because those errors have to
be handled correctly by whatever is submitting the IO. We do not use
the OOM-killer to report or resolve ENOSPC errors.

Indeed, once you've killed the dd, that CGROUP still consumes 64MB
of tmpfs space in /tmp/backing_file, in which case any further IO to
$LOOP_DEV is also going to OOM kill. These are horrible semantics
for reporting errors to userspace.

And if the OOM-killer actually frees the 64MB of data written to
/tmp/backing_file through the $LOOP_DEV, then you're actually
corrupting the storage and ensuring that nobody can read the data
that was written to $LOOP_DEV.

So now lets put a filesystem on $LOOP_DEV in the above example, and
write out data to the filesystem which does IO to $LOOP_DEV. Just by
chance, the filesystem does some metadata writes iin the context of
the user process doing the writes (because journalling, etc) and
that metadata IO is what pushes the loop device over the cgroup's
memory limit.

You kill the user application even though it wasn't directly
responsible for going over the 64MB limit of space in $LOOP_DEV.
What happens now to the filesystem's metadata IO?  Did $LOOP_DEV
return an error, or after the OOM kill did the IO succeed?  What
happens if all the IO being generated from the user application is
metadata and that starts failing - killing the user application
doesn't help anything - the filesystem IO is failing and that's
where the errors need to be reported.

And if the answer is "metadata IO isn't accounted to the $CGROUP"
then what happens when the tmpfs actually runs out of it's 512MB of
space because of all the metadata the filesystem wrote (e.g. create
lots of zero length files)? That's an ENOSPC error, and we'll get
that from $LOOP_DEV just fine.

So why should the same error - running out of tmpfs space vs running
out of CGROUP quota space on tmpfs be handled differently? Either
they are both ENOSPC IO errors, or they are both OOM kill vectors
because tmpfs space has run out...

See the problem here? We report storage resource shortages
(whatever the cause) by IO errors, not by killing userspace
processes. Userspace may be able to handle the IO error sanely; it
has no warning or choice when you use OOM kill to report ENOSPC
errors...

> As opposed to before this series, where all this memory isn't
> accounted properly and goes to the root cgroup - where, ironically, it
> could cause OOM and kill an actually innocent bystander.

Johannes, I didn't question the need for the functionality. I
questioned the implementation and pointed out fundamental problems
it has as well as the architectural questions raised by needing
special kthread-based handling for correct accounting of
cgroup-aware IO.

It's a really bad look to go shoot the messenger when it's clear you
haven't understood the message that was delivered.

-Dave.
-- 
Dave Chinner
david@fromorbit.com

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-04-29 10:25   ` Jan Kara
  2020-04-29 14:22     ` Tejun Heo
@ 2020-05-05  6:41     ` Dave Chinner
  2020-05-05 15:38       ` Tejun Heo
  1 sibling, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2020-05-05  6:41 UTC (permalink / raw)
  To: Jan Kara
  Cc: Dan Schatzberg, Jens Axboe, Alexander Viro, Amir Goldstein,
	Tejun Heo, Li Zefan, Johannes Weiner, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Shakeel Butt, Chris Down, Yang Shi, Ingo Molnar,
	Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Wed, Apr 29, 2020 at 12:25:40PM +0200, Jan Kara wrote:
> On Wed 29-04-20 07:47:34, Dave Chinner wrote:
> > On Tue, Apr 28, 2020 at 12:13:46PM -0400, Dan Schatzberg wrote:
> > > The loop device runs all i/o to the backing file on a separate kworker
> > > thread which results in all i/o being charged to the root cgroup. This
> > > allows a loop device to be used to trivially bypass resource limits
> > > and other policy. This patch series fixes this gap in accounting.
> > 
> > How is this specific to the loop device? Isn't every block device
> > that offloads work to a kthread or single worker thread susceptible
> > to the same "exploit"?
> > 
> > Or is the problem simply that the loop worker thread is simply not
> > taking the IO's associated cgroup and submitting the IO with that
> > cgroup associated with it? That seems kinda simple to fix....
> > 
> > > Naively charging cgroups could result in priority inversions through
> > > the single kworker thread in the case where multiple cgroups are
> > > reading/writing to the same loop device.
> > 
> > And that's where all the complexity and serialisation comes from,
> > right?
> > 
> > So, again: how is this unique to the loop device? Other block
> > devices also offload IO to kthreads to do blocking work and IO
> > submission to lower layers. Hence this seems to me like a generic
> > "block device does IO submission from different task" issue that
> > should be handled by generic infrastructure and not need to be
> > reimplemented multiple times in every block device driver that
> > offloads work to other threads...
> 
> Yeah, I was thinking about the same when reading the patch series
> description. We already have some cgroup workarounds for btrfs kthreads if
> I remember correctly, we have cgroup handling for flush workers, now we are
> adding cgroup handling for loopback device workers, and soon I'd expect
> someone comes with a need for DM/MD worker processes and IMHO it's getting
> out of hands because the complexity spreads through the kernel with every
> subsystem comming with slightly different solution to the problem and also
> the number of kthreads gets multiplied by the number of cgroups. So I
> agree some generic solution how to approach IO throttling of kthreads /
> workers would be desirable.

Yup, that's pretty much what I was thinking: it's yet another
special snowflake for cgroup-aware IO....

> OTOH I don't have a great idea how the generic infrastructure should look
> like...

I haven't given it any thought - it's not something I have any
bandwidth to spend time on.  I'll happily review a unified
generic cgroup-aware kthread-based IO dispatch mechanism, but I
don't have the time to design and implement that myself....

OTOH, I will make time to stop people screwing up filesystems and
block devices with questionable complexity and unique, storage
device dependent userspace visible error behaviour. This sort of
change is objectively worse for users than not supporting the
functionality in the first place.

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-05-05  6:29     ` Dave Chinner
@ 2020-05-05 13:55       ` Shakeel Butt
  2020-05-05 15:02       ` Johannes Weiner
  1 sibling, 0 replies; 20+ messages in thread
From: Shakeel Butt @ 2020-05-05 13:55 UTC (permalink / raw)
  To: Dave Chinner
  Cc: Johannes Weiner, Dan Schatzberg, Jens Axboe, Alexander Viro,
	Jan Kara, Amir Goldstein, Tejun Heo, Li Zefan, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Chris Down, Yang Shi, Ingo Molnar, Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Mon, May 4, 2020 at 11:30 PM Dave Chinner <david@fromorbit.com> wrote:
>
> On Tue, Apr 28, 2020 at 10:27:32PM -0400, Johannes Weiner wrote:
> > On Wed, Apr 29, 2020 at 07:47:34AM +1000, Dave Chinner wrote:
> > > On Tue, Apr 28, 2020 at 12:13:46PM -0400, Dan Schatzberg wrote:
> > > > This patch series does some
> > > > minor modification to the loop driver so that each cgroup can make
> > > > forward progress independently to avoid this inversion.
> > > >
> > > > With this patch series applied, the above script triggers OOM kills
> > > > when writing through the loop device as expected.
> > >
> > > NACK!
> > >
> > > The IO that is disallowed should fail with ENOMEM or some similar
> > > error, not trigger an OOM kill that shoots some innocent bystander
> > > in the head. That's worse than using BUG() to report errors...
> >
> > Did you actually read the script?
>

Before responding, I want to make it clear that the OOM behavior which
you are objecting to is already present in the kernel and is
independent of this patch series. This patch series is only connecting
the charging links which were missing for the loop device.

> Of course I did. You specifically mean this bit:
>
> echo 64M > $CGROUP/memory.max;
> mount -t tmpfs -o size=512m tmpfs /tmp;
> truncate -s 512m /tmp/backing_file
> losetup $LOOP_DEV /tmp/backing_file
> dd if=/dev/zero of=$LOOP_DEV bs=1M count=256;
>
> And with this patch set the dd gets OOM killed because the
> /tmp/backing_file usage accounted to the cgroup goes over 64MB and
> so tmpfs OOM kills the IO...
>

A few queries to better understand your objection:

Let's forget about the cgroup for a second. Let's suppose I am running
this script on a system/VM having 64 MiB. In your opinion what should
happen?

Next let's add the swap to the 64 MiB system/VM/cgroup and re-run the
script, what should be the correct behavior?

Next replace the tmpfs with any other disk backed file system and
re-run the script in a 64 MiB system/VM/cgroup, what should be the
correct behavior?

> As I said: that's very broken behaviour from a storage stack
> perspective.
>
> > It's OOMing because it's creating 256M worth of tmpfs pages inside a
> > 64M cgroup. It's not killing an innocent bystander, it's killing in
> > the cgroup that is allocating all that memory - after Dan makes sure
> > that memory is accounted to its rightful owner.
>
> What this example does is turn /tmp into thinly provisioned storage
> for $CGROUP via a restricted quota. It has a device size of 512MB,
> but only has physical storage capacity of 64MB, as constrained by
> the cgroup memory limit.  You're dealing with management of -storage
> devices- and -IO error reporting- here, not memory management.
>
> When thin provisioned storage runs out of space - for whatever
> reason - and it cannot resolve the issue by blocking, then it *must*
> return ENOSPC to the IO submitter to tell it the IO has failed. This
> is no different to if we set a quota on /tmp/backing_file and it is
> exhausted at 64MB of data written - we fail the IO with ENOSPC or
> EDQUOT, depending on which quota we used.
>
> IOWs, we have solid expectations on how block devices report
> unsolvable resource shortages during IO because those errors have to
> be handled correctly by whatever is submitting the IO. We do not use
> the OOM-killer to report or resolve ENOSPC errors.
>
> Indeed, once you've killed the dd, that CGROUP still consumes 64MB
> of tmpfs space in /tmp/backing_file, in which case any further IO to
> $LOOP_DEV is also going to OOM kill. These are horrible semantics
> for reporting errors to userspace.
>
> And if the OOM-killer actually frees the 64MB of data written to
> /tmp/backing_file through the $LOOP_DEV, then you're actually
> corrupting the storage and ensuring that nobody can read the data
> that was written to $LOOP_DEV.
>
> So now lets put a filesystem on $LOOP_DEV in the above example, and
> write out data to the filesystem which does IO to $LOOP_DEV. Just by
> chance, the filesystem does some metadata writes iin the context of
> the user process doing the writes (because journalling, etc) and
> that metadata IO is what pushes the loop device over the cgroup's
> memory limit.
>
> You kill the user application even though it wasn't directly
> responsible for going over the 64MB limit of space in $LOOP_DEV.
> What happens now to the filesystem's metadata IO?  Did $LOOP_DEV
> return an error, or after the OOM kill did the IO succeed?  What
> happens if all the IO being generated from the user application is
> metadata and that starts failing - killing the user application
> doesn't help anything - the filesystem IO is failing and that's
> where the errors need to be reported.
>
> And if the answer is "metadata IO isn't accounted to the $CGROUP"
> then what happens when the tmpfs actually runs out of it's 512MB of
> space because of all the metadata the filesystem wrote (e.g. create
> lots of zero length files)? That's an ENOSPC error, and we'll get
> that from $LOOP_DEV just fine.
>
> So why should the same error - running out of tmpfs space vs running
> out of CGROUP quota space on tmpfs be handled differently? Either
> they are both ENOSPC IO errors, or they are both OOM kill vectors
> because tmpfs space has run out...
>
> See the problem here? We report storage resource shortages
> (whatever the cause) by IO errors, not by killing userspace
> processes. Userspace may be able to handle the IO error sanely; it
> has no warning or choice when you use OOM kill to report ENOSPC
> errors...
>
> > As opposed to before this series, where all this memory isn't
> > accounted properly and goes to the root cgroup - where, ironically, it
> > could cause OOM and kill an actually innocent bystander.
>
> Johannes, I didn't question the need for the functionality. I
> questioned the implementation and pointed out fundamental problems
> it has as well as the architectural questions raised by needing
> special kthread-based handling for correct accounting of
> cgroup-aware IO.
>
> It's a really bad look to go shoot the messenger when it's clear you
> haven't understood the message that was delivered.
>
> -Dave.
> --
> Dave Chinner
> david@fromorbit.com

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-05-05  6:29     ` Dave Chinner
  2020-05-05 13:55       ` Shakeel Butt
@ 2020-05-05 15:02       ` Johannes Weiner
  1 sibling, 0 replies; 20+ messages in thread
From: Johannes Weiner @ 2020-05-05 15:02 UTC (permalink / raw)
  To: Dave Chinner
  Cc: Dan Schatzberg, Jens Axboe, Alexander Viro, Jan Kara,
	Amir Goldstein, Tejun Heo, Li Zefan, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Shakeel Butt, Chris Down, Yang Shi, Ingo Molnar,
	Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Tue, May 05, 2020 at 04:29:46PM +1000, Dave Chinner wrote:
> On Tue, Apr 28, 2020 at 10:27:32PM -0400, Johannes Weiner wrote:
> > On Wed, Apr 29, 2020 at 07:47:34AM +1000, Dave Chinner wrote:
> > > On Tue, Apr 28, 2020 at 12:13:46PM -0400, Dan Schatzberg wrote:
> > > > This patch series does some
> > > > minor modification to the loop driver so that each cgroup can make
> > > > forward progress independently to avoid this inversion.
> > > > 
> > > > With this patch series applied, the above script triggers OOM kills
> > > > when writing through the loop device as expected.
> > > 
> > > NACK!
> > > 
> > > The IO that is disallowed should fail with ENOMEM or some similar
> > > error, not trigger an OOM kill that shoots some innocent bystander
> > > in the head. That's worse than using BUG() to report errors...
> > 
> > Did you actually read the script?
> 
> Of course I did. You specifically mean this bit:
> 
> echo 64M > $CGROUP/memory.max;
> mount -t tmpfs -o size=512m tmpfs /tmp;
> truncate -s 512m /tmp/backing_file
> losetup $LOOP_DEV /tmp/backing_file
> dd if=/dev/zero of=$LOOP_DEV bs=1M count=256;
> 
> And with this patch set the dd gets OOM killed because the
> /tmp/backing_file usage accounted to the cgroup goes over 64MB and
> so tmpfs OOM kills the IO...
> 
> As I said: that's very broken behaviour from a storage stack
> perspective.
> 
> > It's OOMing because it's creating 256M worth of tmpfs pages inside a
> > 64M cgroup. It's not killing an innocent bystander, it's killing in
> > the cgroup that is allocating all that memory - after Dan makes sure
> > that memory is accounted to its rightful owner.
> 
> What this example does is turn /tmp into thinly provisioned storage
> for $CGROUP via a restricted quota. It has a device size of 512MB,
> but only has physical storage capacity of 64MB, as constrained by
> the cgroup memory limit.  You're dealing with management of -storage
> devices- and -IO error reporting- here, not memory management.
> 
> When thin provisioned storage runs out of space - for whatever
> reason - and it cannot resolve the issue by blocking, then it *must*
> return ENOSPC to the IO submitter to tell it the IO has failed. This
> is no different to if we set a quota on /tmp/backing_file and it is
> exhausted at 64MB of data written - we fail the IO with ENOSPC or
> EDQUOT, depending on which quota we used.
> 
> IOWs, we have solid expectations on how block devices report
> unsolvable resource shortages during IO because those errors have to
> be handled correctly by whatever is submitting the IO. We do not use
> the OOM-killer to report or resolve ENOSPC errors.
>
> Indeed, once you've killed the dd, that CGROUP still consumes 64MB
> of tmpfs space in /tmp/backing_file, in which case any further IO to
> $LOOP_DEV is also going to OOM kill. These are horrible semantics
> for reporting errors to userspace.
>
> And if the OOM-killer actually frees the 64MB of data written to
> /tmp/backing_file through the $LOOP_DEV, then you're actually
> corrupting the storage and ensuring that nobody can read the data
> that was written to $LOOP_DEV.

Right, but that's just tmpfs. It doesn't have much to do with the loop
device or its semantics as a block device. (Although I don't think
most users really see loop as a true block device, but rather as a
namespacing tool that for better or worse happens to be implemented at
the block layer). But remove the loop device indirection and the tmpfs
semantics would be exactly the same.

tmpfs returns -ENOSPC when you run out of filesystem quota, but when
it tries to allocate memory and can't, it'll invoke the OOM killer as
a means to reclaim memory. And when that fails, it'll return -ENOMEM.

Dan's patches don't change the block device semantics of loop. They
just ensure that the chain of causality between writer and memory
allocation isn't broken.

In fact, it barely has anything to do with loop itself. If loop were
to do its redirections synchronously and in the context of the process
that is making requests, we wouldn't have this problem.

The generic problem is that of one process performing work on behalf
of another process with side-effects relevant to the originator. The
generic solution is to have the worker impersonate the process that
created the work in all the various aspects that matter.

Like io_uring and various other kthreads and workers doing use_mm()
when the address space of the process creating the work matters.

> So now lets put a filesystem on $LOOP_DEV in the above example, and
> write out data to the filesystem which does IO to $LOOP_DEV. Just by
> chance, the filesystem does some metadata writes iin the context of
> the user process doing the writes (because journalling, etc) and
> that metadata IO is what pushes the loop device over the cgroup's
> memory limit.
> 
> You kill the user application even though it wasn't directly
> responsible for going over the 64MB limit of space in $LOOP_DEV.
> What happens now to the filesystem's metadata IO?  Did $LOOP_DEV
> return an error, or after the OOM kill did the IO succeed?  What
> happens if all the IO being generated from the user application is
> metadata and that starts failing - killing the user application
> doesn't help anything - the filesystem IO is failing and that's
> where the errors need to be reported.
> 
> And if the answer is "metadata IO isn't accounted to the $CGROUP"
> then what happens when the tmpfs actually runs out of it's 512MB of
> space because of all the metadata the filesystem wrote (e.g. create
> lots of zero length files)? That's an ENOSPC error, and we'll get
> that from $LOOP_DEV just fine.

Well, what happens today if you write to a loop mount backed by tmpfs,
but the machine is *physically* out of memory?

None of these questions are new in the context of this patch set. The
cgroup annotations don't inject anything that isn't already happening.

When you use the loop device on a tmpfs backing today, logically
speaking you're charging the root cgroup. That may not have a user-set
limit, but it's limited by physical RAM.

With or without cgroup annotation, tmpfs needs to allocate memory, and
that can fail. The function that charges to a specific cgroup is just
a few lines below the function that physically allocates the
page. Both can invoke the OOM killer for slightly different reasons
that aren't really relevant to the loop device on top of it.

> So why should the same error - running out of tmpfs space vs running
> out of CGROUP quota space on tmpfs be handled differently? Either
> they are both ENOSPC IO errors, or they are both OOM kill vectors
> because tmpfs space has run out...

Because charging memory has allocation semantics, and tmpfs already
defines what those are.

> > As opposed to before this series, where all this memory isn't
> > accounted properly and goes to the root cgroup - where, ironically, it
> > could cause OOM and kill an actually innocent bystander.
> 
> Johannes, I didn't question the need for the functionality. I
> questioned the implementation and pointed out fundamental problems
> it has as well as the architectural questions raised by needing
> special kthread-based handling for correct accounting of
> cgroup-aware IO.
>
> It's a really bad look to go shoot the messenger when it's clear you
> haven't understood the message that was delivered.

Do I need to point out the irony here? ;)

Maybe let's focus on the technical questions and misunderstandings
first before throwing NAKs around.

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-05-05  6:41     ` Dave Chinner
@ 2020-05-05 15:38       ` Tejun Heo
  0 siblings, 0 replies; 20+ messages in thread
From: Tejun Heo @ 2020-05-05 15:38 UTC (permalink / raw)
  To: Dave Chinner
  Cc: Jan Kara, Dan Schatzberg, Jens Axboe, Alexander Viro,
	Amir Goldstein, Li Zefan, Johannes Weiner, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Shakeel Butt, Chris Down, Yang Shi, Ingo Molnar,
	Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

Hello, Dave.

On Tue, May 05, 2020 at 04:41:14PM +1000, Dave Chinner wrote:
> > OTOH I don't have a great idea how the generic infrastructure should look
> > like...
> 
> I haven't given it any thought - it's not something I have any
> bandwidth to spend time on.  I'll happily review a unified
> generic cgroup-aware kthread-based IO dispatch mechanism, but I
> don't have the time to design and implement that myself....
> 
> OTOH, I will make time to stop people screwing up filesystems and
> block devices with questionable complexity and unique, storage
> device dependent userspace visible error behaviour. This sort of
> change is objectively worse for users than not supporting the
> functionality in the first place.

That probably is too strong a position to hold without spending at least
some thoughts on a subject, whatever the subject may be, and it doesn't seem
like your understanding of userspace implications is accurate.

I don't necessarily disagree that it'd be nice to have a common
infrastructure and there may be some part which can actually be factored
out. However, there isn't gonna be a magic bullet which magically makes
every IO thing in the kernel cgroup aware automatically. Please consider the
followings.

* Avoding IO priority inversions requires splitting IO channels according to
  cgroups and working around (e.g. with backcharging) when they can't be.
  It's a substantial feature which may require substantial changes. Each IO
  subsystem has different constraints and existing structures and many of
  them would require their own solutions. It's not different from different
  filesystems needing their own solutions for similar problems.

* Because different filesystems and IO stacking layers already have their
  own internal infrastructure, the right way to add cgroup support is
  adapting to and modifying the existing infrastructure rather than trying
  to restructure them to use the same cgroup mechanism, which I don't think
  would be possible in many cases.

* Among the three IO stacking / redirecting mechanisms - md/dm, loop and
  fuse - the requirements and what's possible vary quite a bit. md/dm
  definitely need to support full-on IO channel splitting cgroup support.
  loop can go either way, but given existing uses, full splitting makes a
  sense. fuse, as it currently stands, can't support that because the
  priority inversions extend all the way to userspace and the kernel API
  isn't built for that. If it wants to support cgroup containment, each
  instance would have to be assigned to a cgroup.

Between dm/md and loop, it's maybe possible that some of the sub-threading
code can be reused, but I don't see a point in blocking loop updates given
that it clearly fixes userspace visible malfunctions, is not that much code
and how the shared code should look is unclear yet. We'll be able to answer
the sharing question when we actually get to dm/md conversion.

Thanks.

-- 
tejun

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-04-28 16:13 [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dan Schatzberg
                   ` (4 preceding siblings ...)
  2020-04-28 21:47 ` [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dave Chinner
@ 2020-05-12 13:25 ` Dan Schatzberg
  2020-05-12 13:35   ` Christoph Hellwig
  5 siblings, 1 reply; 20+ messages in thread
From: Dan Schatzberg @ 2020-05-12 13:25 UTC (permalink / raw)
  To: Jens Axboe, Alexander Viro, Jan Kara, Amir Goldstein, Tejun Heo,
	Li Zefan, Johannes Weiner, Michal Hocko, Vladimir Davydov,
	Andrew Morton, Hugh Dickins, Roman Gushchin, Shakeel Butt,
	Chris Down, Yang Shi, Ingo Molnar, Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

Seems like discussion on this patch series has died down. There's been
a concern raised that we could generalize infrastructure across loop,
md, etc. This may be possible, in the future, but it isn't clear to me
how this would look like. I'm inclined to fix the existing issue with
loop devices now (this is a problem we hit at FB) and address
consolidation with other cases if and when those are addressed.

Jens, you've expressed interest in seeing this series go through the
block tree so I'm interested in your perspective here. Barring any
concrete implementation bugs, would you be okay merging this version?

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-05-12 13:25 ` Dan Schatzberg
@ 2020-05-12 13:35   ` Christoph Hellwig
  2020-05-26 14:28     ` Dan Schatzberg
  0 siblings, 1 reply; 20+ messages in thread
From: Christoph Hellwig @ 2020-05-12 13:35 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Alexander Viro, Jan Kara, Amir Goldstein, Tejun Heo,
	Li Zefan, Johannes Weiner, Michal Hocko, Vladimir Davydov,
	Andrew Morton, Hugh Dickins, Roman Gushchin, Shakeel Butt,
	Chris Down, Yang Shi, Ingo Molnar, Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Tue, May 12, 2020 at 09:25:21AM -0400, Dan Schatzberg wrote:
> Seems like discussion on this patch series has died down. There's been
> a concern raised that we could generalize infrastructure across loop,
> md, etc. This may be possible, in the future, but it isn't clear to me
> how this would look like. I'm inclined to fix the existing issue with
> loop devices now (this is a problem we hit at FB) and address
> consolidation with other cases if and when those are addressed.
> 
> Jens, you've expressed interest in seeing this series go through the
> block tree so I'm interested in your perspective here. Barring any
> concrete implementation bugs, would you be okay merging this version?

Independ of any higher level issues you need to sort out the spinlock
mess I pointed out.

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-05-12 13:35   ` Christoph Hellwig
@ 2020-05-26 14:28     ` Dan Schatzberg
  2020-05-27  5:09       ` Christoph Hellwig
  0 siblings, 1 reply; 20+ messages in thread
From: Dan Schatzberg @ 2020-05-26 14:28 UTC (permalink / raw)
  To: Christoph Hellwig
  Cc: Jens Axboe, Alexander Viro, Jan Kara, Amir Goldstein, Tejun Heo,
	Li Zefan, Johannes Weiner, Michal Hocko, Vladimir Davydov,
	Andrew Morton, Hugh Dickins, Roman Gushchin, Shakeel Butt,
	Chris Down, Yang Shi, Ingo Molnar, Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Tue, May 12, 2020 at 06:35:45AM -0700, Christoph Hellwig wrote:
> On Tue, May 12, 2020 at 09:25:21AM -0400, Dan Schatzberg wrote:
> > Seems like discussion on this patch series has died down. There's been
> > a concern raised that we could generalize infrastructure across loop,
> > md, etc. This may be possible, in the future, but it isn't clear to me
> > how this would look like. I'm inclined to fix the existing issue with
> > loop devices now (this is a problem we hit at FB) and address
> > consolidation with other cases if and when those are addressed.
> > 
> > Jens, you've expressed interest in seeing this series go through the
> > block tree so I'm interested in your perspective here. Barring any
> > concrete implementation bugs, would you be okay merging this version?
> 
> Independ of any higher level issues you need to sort out the spinlock
> mess I pointed out.

Will do - I'll split out the lock-use refactor into a separate
patch. Do you have particular concerns about re-using the existing
spinlock? Its existing use is not contended so I didn't see any harm
in extending its use. I'll add this justification to the commit
message as well, but I'm tempted to leave the re-use as is instead of
creating a new lock.

^ permalink raw reply	[flat|nested] 20+ messages in thread

* Re: [PATCH v5 0/4] Charge loop device i/o to issuing cgroup
  2020-05-26 14:28     ` Dan Schatzberg
@ 2020-05-27  5:09       ` Christoph Hellwig
  0 siblings, 0 replies; 20+ messages in thread
From: Christoph Hellwig @ 2020-05-27  5:09 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Christoph Hellwig, Jens Axboe, Alexander Viro, Jan Kara,
	Amir Goldstein, Tejun Heo, Li Zefan, Johannes Weiner,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Roman Gushchin, Shakeel Butt, Chris Down, Yang Shi, Ingo Molnar,
	Peter Zijlstra (Intel),
	Mathieu Desnoyers, Kirill A. Shutemov, Andrea Arcangeli,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Tue, May 26, 2020 at 10:28:03AM -0400, Dan Schatzberg wrote:
> Will do - I'll split out the lock-use refactor into a separate
> patch. Do you have particular concerns about re-using the existing
> spinlock? Its existing use is not contended so I didn't see any harm
> in extending its use. I'll add this justification to the commit
> message as well, but I'm tempted to leave the re-use as is instead of
> creating a new lock.

Please don't share a lock for entirely separate critical sections that
are used from different contexts.

^ permalink raw reply	[flat|nested] 20+ messages in thread

end of thread, other threads:[~2020-05-27  5:09 UTC | newest]

Thread overview: 20+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-04-28 16:13 [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dan Schatzberg
2020-04-28 16:13 ` [PATCH v5 1/4] loop: Use worker per cgroup instead of kworker Dan Schatzberg
2020-04-28 16:13 ` [PATCH v5 2/4] mm: support nesting memalloc_use_memcg() Dan Schatzberg
2020-04-28 16:13 ` [PATCH v5 3/4] mm: Charge active memcg when no mm is set Dan Schatzberg
2020-04-28 16:13 ` [PATCH v5 4/4] loop: Charge i/o to mem and blk cg Dan Schatzberg
2020-04-28 21:47 ` [PATCH v5 0/4] Charge loop device i/o to issuing cgroup Dave Chinner
2020-04-29  2:27   ` Johannes Weiner
2020-05-05  6:29     ` Dave Chinner
2020-05-05 13:55       ` Shakeel Butt
2020-05-05 15:02       ` Johannes Weiner
2020-04-29 10:25   ` Jan Kara
2020-04-29 14:22     ` Tejun Heo
2020-04-29 16:21       ` Jan Kara
2020-05-05  6:41     ` Dave Chinner
2020-05-05 15:38       ` Tejun Heo
2020-04-29 14:03   ` Dan Schatzberg
2020-05-12 13:25 ` Dan Schatzberg
2020-05-12 13:35   ` Christoph Hellwig
2020-05-26 14:28     ` Dan Schatzberg
2020-05-27  5:09       ` Christoph Hellwig

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).