linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH V12 0/3] Charge loop device i/o to issuing cgroup
@ 2021-04-02 19:16 Dan Schatzberg
  2021-04-02 19:16 ` [PATCH 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
                   ` (4 more replies)
  0 siblings, 5 replies; 23+ messages in thread
From: Dan Schatzberg @ 2021-04-02 19:16 UTC (permalink / raw)
  Cc: Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner, Andrew Morton,
	Michal Hocko, Vladimir Davydov, Hugh Dickins, Shakeel Butt,
	Roman Gushchin, Muchun Song, Yang Shi, Alex Shi, Alexander Duyck,
	Wei Yang, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT

No major changes, rebased on top of latest mm tree

Changes since V12:

* Small change to get_mem_cgroup_from_mm to avoid needing
  get_active_memcg

Changes since V11:

* Removed WQ_MEM_RECLAIM flag from loop workqueue. Technically, this
  can be driven by writeback, but this was causing a warning in xfs
  and likely other filesystems aren't equipped to be driven by reclaim
  at the VFS layer.
* Included a small fix from Colin Ian King.
* reworked get_mem_cgroup_from_mm to institute the necessary charge
  priority.

Changes since V10:

* Added page-cache charging to mm: Charge active memcg when no mm is set

Changes since V9:

* Rebased against linus's branch which now includes Roman Gushchin's
  patch this series is based off of

Changes since V8:

* Rebased on top of Roman Gushchin's patch
  (https://lkml.org/lkml/2020/8/21/1464) which provides the nesting
  support for setting active memcg. Dropped the patch from this series
  that did the same thing.

Changes since V7:

* Rebased against linus's branch

Changes since V6:

* Added separate spinlock for worker synchronization
* Minor style changes

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

 drivers/block/loop.c       | 244 ++++++++++++++++++++++++++++++-------
 drivers/block/loop.h       |  15 ++-
 include/linux/memcontrol.h |   6 +
 kernel/cgroup/cgroup.c     |   1 +
 mm/filemap.c               |   2 +-
 mm/memcontrol.c            |  49 +++++---
 mm/shmem.c                 |   4 +-
 7 files changed, 253 insertions(+), 68 deletions(-)

-- 
2.30.2


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

* [PATCH 1/3] loop: Use worker per cgroup instead of kworker
  2021-04-02 19:16 [PATCH V12 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
@ 2021-04-02 19:16 ` Dan Schatzberg
  2021-04-06  1:44   ` Ming Lei
  2021-04-02 19:16 ` [PATCH 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
                   ` (3 subsequent siblings)
  4 siblings, 1 reply; 23+ messages in thread
From: Dan Schatzberg @ 2021-04-02 19:16 UTC (permalink / raw)
  Cc: Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner, Andrew Morton,
	Michal Hocko, Vladimir Davydov, Hugh Dickins, Shakeel Butt,
	Roman Gushchin, Muchun Song, Yang Shi, Alex Shi, Alexander Duyck,
	Wei Yang, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT

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

Fixes-from: Colin Ian King <colin.king@canonical.com>
Signed-off-by: Dan Schatzberg <schatzberg.dan@gmail.com>
---
 drivers/block/loop.c | 203 ++++++++++++++++++++++++++++++++++++-------
 drivers/block/loop.h |  12 ++-
 2 files changed, 178 insertions(+), 37 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index d58d68f3c7cd..4750b373d4bb 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -71,7 +71,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>
@@ -84,6 +83,8 @@
 
 #include <linux/uaccess.h>
 
+#define LOOP_IDLE_WORKER_TIMEOUT (60 * HZ)
+
 static DEFINE_IDR(loop_index_idr);
 static DEFINE_MUTEX(loop_ctl_mutex);
 
@@ -921,27 +922,83 @@ static void loop_config_discard(struct loop_device *lo)
 	q->limits.discard_alignment = 0;
 }
 
-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_LOCAL_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_work_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_work_lock);
 }
 
 static void loop_update_rotational(struct loop_device *lo)
@@ -1127,12 +1184,23 @@ static int loop_configure(struct loop_device *lo, fmode_t mode,
 	    !file->f_op->write_iter)
 		lo->lo_flags |= LO_FLAGS_READ_ONLY;
 
-	error = loop_prepare_queue(lo);
-	if (error)
+	lo->workqueue = alloc_workqueue("loop%d",
+					WQ_UNBOUND | WQ_FREEZABLE,
+					0,
+					lo->lo_number);
+	if (!lo->workqueue) {
+		error = -ENOMEM;
 		goto out_unlock;
+	}
 
 	set_disk_ro(lo->lo_disk, (lo->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 = lo->lo_flags & LO_FLAGS_DIRECT_IO;
 	lo->lo_device = bdev;
 	lo->lo_backing_file = file;
@@ -1200,6 +1268,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(&lo->lo_mutex);
 	if (WARN_ON_ONCE(lo->lo_state != Lo_rundown)) {
@@ -1219,6 +1288,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_work_lock);
+	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_work_lock);
+	del_timer_sync(&lo->timer);
+
 	spin_lock_irq(&lo->lo_lock);
 	lo->lo_backing_file = NULL;
 	spin_unlock_irq(&lo->lo_lock);
@@ -1255,7 +1336,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(&lo->lo_mutex);
 	if (partscan) {
@@ -2026,7 +2106,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;
 }
@@ -2056,26 +2136,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)
+{
+	int orig_flags = current->flags;
+	struct loop_cmd *cmd;
+
+	current->flags |= PF_LOCAL_THROTTLE | PF_MEMALLOC_NOIO;
+	spin_lock_irq(&lo->lo_work_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_work_lock);
+
+		loop_handle_cmd(cmd);
+		cond_resched();
+
+		spin_lock_irq(&lo->lo_work_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_work_lock);
+	current->flags = orig_flags;
+}
+
+static void loop_workfn(struct work_struct *work)
 {
-	struct loop_cmd *cmd =
-		container_of(work, struct loop_cmd, work);
+	struct loop_worker *worker =
+		container_of(work, struct loop_worker, work);
+	loop_process_work(worker, &worker->cmd_list, worker->lo);
+}
 
-	loop_handle_cmd(cmd);
+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 int loop_init_request(struct blk_mq_tag_set *set, struct request *rq,
-		unsigned int hctx_idx, unsigned int numa_node)
+static void loop_free_idle_workers(struct timer_list *timer)
 {
-	struct loop_cmd *cmd = blk_mq_rq_to_pdu(rq);
+	struct loop_device *lo = container_of(timer, struct loop_device, timer);
+	struct loop_worker *pos, *worker;
 
-	kthread_init_work(&cmd->work, loop_queue_work);
-	return 0;
+	spin_lock_irq(&lo->lo_work_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_work_lock);
 }
 
 static const struct blk_mq_ops loop_mq_ops = {
 	.queue_rq       = loop_queue_rq,
-	.init_request	= loop_init_request,
 	.complete	= lo_complete_rq,
 };
 
@@ -2164,6 +2300,7 @@ static int loop_add(struct loop_device **l, int i)
 	mutex_init(&lo->lo_mutex);
 	lo->lo_number		= i;
 	spin_lock_init(&lo->lo_lock);
+	spin_lock_init(&lo->lo_work_lock);
 	disk->major		= LOOP_MAJOR;
 	disk->first_minor	= i << part_shift;
 	disk->fops		= &lo_fops;
diff --git a/drivers/block/loop.h b/drivers/block/loop.h
index a3c04f310672..9289c1cd6374 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,13 @@ struct loop_device {
 
 	spinlock_t		lo_lock;
 	int			lo_state;
-	struct kthread_worker	worker;
-	struct task_struct	*worker_task;
+	spinlock_t              lo_work_lock;
+	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;
 
@@ -66,7 +70,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.30.2


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

* [PATCH 2/3] mm: Charge active memcg when no mm is set
  2021-04-02 19:16 [PATCH V12 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
  2021-04-02 19:16 ` [PATCH 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
@ 2021-04-02 19:16 ` Dan Schatzberg
  2021-04-03  5:47   ` [External] " Muchun Song
  2021-04-02 19:16 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
                   ` (2 subsequent siblings)
  4 siblings, 1 reply; 23+ messages in thread
From: Dan Schatzberg @ 2021-04-02 19:16 UTC (permalink / raw)
  Cc: Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner, Andrew Morton,
	Michal Hocko, Vladimir Davydov, Hugh Dickins, Shakeel Butt,
	Roman Gushchin, Muchun Song, Yang Shi, Alex Shi, Alexander Duyck,
	Wei Yang, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT, Chris Down

set_active_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 there is an
   active_memcg, use that. Otherwise, current->mm->memcg.

Previously, if a NULL mm was passed to mem_cgroup_charge (case 3) it
would always charge the root cgroup. Now it looks up the 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/filemap.c    |  2 +-
 mm/memcontrol.c | 48 +++++++++++++++++++++++++++++++-----------------
 mm/shmem.c      |  4 ++--
 3 files changed, 34 insertions(+), 20 deletions(-)

diff --git a/mm/filemap.c b/mm/filemap.c
index c03463cb72d6..38648f7d2106 100644
--- a/mm/filemap.c
+++ b/mm/filemap.c
@@ -872,7 +872,7 @@ noinline int __add_to_page_cache_locked(struct page *page,
 	page->index = offset;
 
 	if (!huge) {
-		error = mem_cgroup_charge(page, current->mm, gfp);
+		error = mem_cgroup_charge(page, NULL, gfp);
 		if (error)
 			goto error;
 		charged = true;
diff --git a/mm/memcontrol.c b/mm/memcontrol.c
index c0b83a396299..d2939d6602b3 100644
--- a/mm/memcontrol.c
+++ b/mm/memcontrol.c
@@ -886,13 +886,24 @@ struct mem_cgroup *mem_cgroup_from_task(struct task_struct *p)
 }
 EXPORT_SYMBOL(mem_cgroup_from_task);
 
+static __always_inline struct mem_cgroup *active_memcg(void)
+{
+	if (in_interrupt())
+		return this_cpu_read(int_active_memcg);
+	else
+		return current->active_memcg;
+}
+
 /**
  * get_mem_cgroup_from_mm: Obtain a reference on given mm_struct's memcg.
  * @mm: mm from which memcg should be extracted. It can be NULL.
  *
- * Obtain a reference on mm->memcg and returns it if successful. Otherwise
- * root_mem_cgroup is returned. However if mem_cgroup is disabled, NULL is
- * returned.
+ * Obtain a reference on mm->memcg and returns it if successful. If mm
+ * is NULL, then the memcg is chosen as follows:
+ * 1) The active memcg, if set.
+ * 2) current->mm->memcg, if available
+ * 3) root memcg
+ * If mem_cgroup is disabled, NULL is returned.
  */
 struct mem_cgroup *get_mem_cgroup_from_mm(struct mm_struct *mm)
 {
@@ -901,13 +912,23 @@ struct mem_cgroup *get_mem_cgroup_from_mm(struct mm_struct *mm)
 	if (mem_cgroup_disabled())
 		return NULL;
 
+	/*
+	 * Page cache insertions can happen without an
+	 * actual mm context, e.g. during disk probing
+	 * on boot, loopback IO, acct() writes etc.
+	 */
+	if (unlikely(!mm)) {
+		memcg = active_memcg();
+		if (unlikely(memcg)) {
+			/* remote memcg must hold a ref */
+			css_get(&memcg->css);
+			return memcg;
+		}
+		mm = current->mm;
+	}
+
 	rcu_read_lock();
 	do {
-		/*
-		 * Page cache insertions can happen without an
-		 * actual mm context, e.g. during disk probing
-		 * on boot, loopback IO, acct() writes etc.
-		 */
 		if (unlikely(!mm))
 			memcg = root_mem_cgroup;
 		else {
@@ -921,14 +942,6 @@ struct mem_cgroup *get_mem_cgroup_from_mm(struct mm_struct *mm)
 }
 EXPORT_SYMBOL(get_mem_cgroup_from_mm);
 
-static __always_inline struct mem_cgroup *active_memcg(void)
-{
-	if (in_interrupt())
-		return this_cpu_read(int_active_memcg);
-	else
-		return current->active_memcg;
-}
-
 static __always_inline bool memcg_kmem_bypass(void)
 {
 	/* Allow remote memcg charging from any context. */
@@ -6537,7 +6550,8 @@ static int __mem_cgroup_charge(struct page *page, struct mem_cgroup *memcg,
  * @gfp_mask: reclaim mode
  *
  * 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.
  *
  * Do not use this for pages allocated for swapin.
  *
diff --git a/mm/shmem.c b/mm/shmem.c
index 5cfd2fb6e52b..524fa5aa0459 100644
--- a/mm/shmem.c
+++ b/mm/shmem.c
@@ -1694,7 +1694,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 page *page;
 	swp_entry_t swap;
 	int error;
@@ -1815,7 +1815,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 = pagecache_get_page(mapping, index,
 					FGP_ENTRY | FGP_HEAD | FGP_LOCK, 0);
-- 
2.30.2


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

* [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-04-02 19:16 [PATCH V12 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
  2021-04-02 19:16 ` [PATCH 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
  2021-04-02 19:16 ` [PATCH 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
@ 2021-04-02 19:16 ` Dan Schatzberg
  2021-04-06  3:23   ` Ming Lei
       [not found] ` <20210403020902.1384-1-hdanton@sina.com>
  2021-04-12 15:45 ` [PATCH V12 0/3] Charge loop device i/o to issuing cgroup Johannes Weiner
  4 siblings, 1 reply; 23+ messages in thread
From: Dan Schatzberg @ 2021-04-02 19:16 UTC (permalink / raw)
  Cc: Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner, Andrew Morton,
	Michal Hocko, Vladimir Davydov, Hugh Dickins, Shakeel Butt,
	Roman Gushchin, Muchun Song, Yang Shi, Alex Shi, Alexander Duyck,
	Wei Yang, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT

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 and int_active_memcg 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 +
 mm/memcontrol.c            |  1 +
 5 files changed, 51 insertions(+), 21 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 4750b373d4bb..d2759f8a7c2a 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -78,6 +78,7 @@
 #include <linux/uio.h>
 #include <linux/ioprio.h>
 #include <linux/blk-cgroup.h>
+#include <linux/sched/mm.h>
 
 #include "loop.h"
 
@@ -516,8 +517,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);
 }
@@ -578,8 +577,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);
@@ -587,7 +584,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);
@@ -928,7 +924,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;
 };
 
@@ -945,7 +941,7 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 
 	spin_lock_irq(&lo->lo_work_lock);
 
-	if (!cmd->css)
+	if (!cmd->blkcg_css)
 		goto queue_work;
 
 	node = &lo->worker_tree.rb_node;
@@ -953,10 +949,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;
@@ -968,13 +964,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);
@@ -1294,7 +1295,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_work_lock);
@@ -2099,13 +2100,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;
@@ -2117,13 +2123,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 = set_active_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) {
+		set_active_memcg(old_memcg);
+		css_put(cmd->memcg_css);
+	}
  failed:
 	/* complete non-aio request */
 	if (!cmd->use_aio || ret) {
@@ -2202,7 +2223,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 9289c1cd6374..cd24a81e00e6 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -76,7 +76,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 b8b0a802852c..a92500734f90 100644
--- a/include/linux/memcontrol.h
+++ b/include/linux/memcontrol.h
@@ -1249,6 +1249,12 @@ static inline struct mem_cgroup *get_mem_cgroup_from_mm(struct mm_struct *mm)
 	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 e049edd66776..8c84a5374238 100644
--- a/kernel/cgroup/cgroup.c
+++ b/kernel/cgroup/cgroup.c
@@ -577,6 +577,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)
 {
diff --git a/mm/memcontrol.c b/mm/memcontrol.c
index d2939d6602b3..f12886a85e8b 100644
--- a/mm/memcontrol.c
+++ b/mm/memcontrol.c
@@ -78,6 +78,7 @@ struct mem_cgroup *root_mem_cgroup __read_mostly;
 
 /* Active memory cgroup to use from an interrupt context */
 DEFINE_PER_CPU(struct mem_cgroup *, int_active_memcg);
+EXPORT_PER_CPU_SYMBOL_GPL(int_active_memcg);
 
 /* Socket memory accounting disabled? */
 static bool cgroup_memory_nosocket;
-- 
2.30.2


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

* Re: [External] [PATCH 2/3] mm: Charge active memcg when no mm is set
  2021-04-02 19:16 ` [PATCH 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
@ 2021-04-03  5:47   ` Muchun Song
  0 siblings, 0 replies; 23+ messages in thread
From: Muchun Song @ 2021-04-03  5:47 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner, Andrew Morton,
	Michal Hocko, Vladimir Davydov, Hugh Dickins, Shakeel Butt,
	Roman Gushchin, Yang Shi, Alex Shi, Alexander Duyck, Wei Yang,
	open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT, Chris Down

On Sat, Apr 3, 2021 at 3:17 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
>
> set_active_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 there is an
>    active_memcg, use that. Otherwise, current->mm->memcg.
>
> Previously, if a NULL mm was passed to mem_cgroup_charge (case 3) it
> would always charge the root cgroup. Now it looks up the 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>

Reviewed-by: Muchun Song <songmuchun@bytedance.com>

Thanks.

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

* Re: [PATCH 1/3] loop: Use worker per cgroup instead of kworker
  2021-04-02 19:16 ` [PATCH 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
@ 2021-04-06  1:44   ` Ming Lei
  0 siblings, 0 replies; 23+ messages in thread
From: Ming Lei @ 2021-04-06  1:44 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner, Andrew Morton,
	Michal Hocko, Vladimir Davydov, Hugh Dickins, Shakeel Butt,
	Roman Gushchin, Muchun Song, Yang Shi, Alex Shi, Alexander Duyck,
	Wei Yang, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT

On Sat, Apr 3, 2021 at 3:17 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
>
> 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
> a 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.
>
> Fixes-from: Colin Ian King <colin.king@canonical.com>
> Signed-off-by: Dan Schatzberg <schatzberg.dan@gmail.com>
> ---
>  drivers/block/loop.c | 203 ++++++++++++++++++++++++++++++++++++-------
>  drivers/block/loop.h |  12 ++-
>  2 files changed, 178 insertions(+), 37 deletions(-)
>
> diff --git a/drivers/block/loop.c b/drivers/block/loop.c
> index d58d68f3c7cd..4750b373d4bb 100644
> --- a/drivers/block/loop.c
> +++ b/drivers/block/loop.c
> @@ -71,7 +71,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>
> @@ -84,6 +83,8 @@
>
>  #include <linux/uaccess.h>
>
> +#define LOOP_IDLE_WORKER_TIMEOUT (60 * HZ)
> +
>  static DEFINE_IDR(loop_index_idr);
>  static DEFINE_MUTEX(loop_ctl_mutex);
>
> @@ -921,27 +922,83 @@ static void loop_config_discard(struct loop_device *lo)
>         q->limits.discard_alignment = 0;
>  }
>
> -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_LOCAL_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_work_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_work_lock);
>  }
>
>  static void loop_update_rotational(struct loop_device *lo)
> @@ -1127,12 +1184,23 @@ static int loop_configure(struct loop_device *lo, fmode_t mode,
>             !file->f_op->write_iter)
>                 lo->lo_flags |= LO_FLAGS_READ_ONLY;
>
> -       error = loop_prepare_queue(lo);
> -       if (error)
> +       lo->workqueue = alloc_workqueue("loop%d",
> +                                       WQ_UNBOUND | WQ_FREEZABLE,
> +                                       0,
> +                                       lo->lo_number);
> +       if (!lo->workqueue) {
> +               error = -ENOMEM;
>                 goto out_unlock;
> +       }
>
>         set_disk_ro(lo->lo_disk, (lo->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 = lo->lo_flags & LO_FLAGS_DIRECT_IO;
>         lo->lo_device = bdev;
>         lo->lo_backing_file = file;
> @@ -1200,6 +1268,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(&lo->lo_mutex);
>         if (WARN_ON_ONCE(lo->lo_state != Lo_rundown)) {
> @@ -1219,6 +1288,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_work_lock);
> +       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_work_lock);
> +       del_timer_sync(&lo->timer);
> +
>         spin_lock_irq(&lo->lo_lock);
>         lo->lo_backing_file = NULL;
>         spin_unlock_irq(&lo->lo_lock);
> @@ -1255,7 +1336,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(&lo->lo_mutex);
>         if (partscan) {
> @@ -2026,7 +2106,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;
>  }
> @@ -2056,26 +2136,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)
> +{
> +       int orig_flags = current->flags;
> +       struct loop_cmd *cmd;
> +
> +       current->flags |= PF_LOCAL_THROTTLE | PF_MEMALLOC_NOIO;
> +       spin_lock_irq(&lo->lo_work_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_work_lock);
> +
> +               loop_handle_cmd(cmd);
> +               cond_resched();
> +
> +               spin_lock_irq(&lo->lo_work_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_work_lock);
> +       current->flags = orig_flags;
> +}
> +
> +static void loop_workfn(struct work_struct *work)
>  {
> -       struct loop_cmd *cmd =
> -               container_of(work, struct loop_cmd, work);
> +       struct loop_worker *worker =
> +               container_of(work, struct loop_worker, work);
> +       loop_process_work(worker, &worker->cmd_list, worker->lo);
> +}
>
> -       loop_handle_cmd(cmd);
> +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 int loop_init_request(struct blk_mq_tag_set *set, struct request *rq,
> -               unsigned int hctx_idx, unsigned int numa_node)
> +static void loop_free_idle_workers(struct timer_list *timer)
>  {
> -       struct loop_cmd *cmd = blk_mq_rq_to_pdu(rq);
> +       struct loop_device *lo = container_of(timer, struct loop_device, timer);
> +       struct loop_worker *pos, *worker;
>
> -       kthread_init_work(&cmd->work, loop_queue_work);
> -       return 0;
> +       spin_lock_irq(&lo->lo_work_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_work_lock);
>  }
>
>  static const struct blk_mq_ops loop_mq_ops = {
>         .queue_rq       = loop_queue_rq,
> -       .init_request   = loop_init_request,
>         .complete       = lo_complete_rq,
>  };
>
> @@ -2164,6 +2300,7 @@ static int loop_add(struct loop_device **l, int i)
>         mutex_init(&lo->lo_mutex);
>         lo->lo_number           = i;
>         spin_lock_init(&lo->lo_lock);
> +       spin_lock_init(&lo->lo_work_lock);
>         disk->major             = LOOP_MAJOR;
>         disk->first_minor       = i << part_shift;
>         disk->fops              = &lo_fops;
> diff --git a/drivers/block/loop.h b/drivers/block/loop.h
> index a3c04f310672..9289c1cd6374 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,13 @@ struct loop_device {
>
>         spinlock_t              lo_lock;
>         int                     lo_state;
> -       struct kthread_worker   worker;
> -       struct task_struct      *worker_task;
> +       spinlock_t              lo_work_lock;
> +       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;
>
> @@ -66,7 +70,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.30.2
>

Reviewed-by: Ming Lei <ming.lei@redhat.com>

-- 
Ming Lei

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

* Re: [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-04-02 19:16 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
@ 2021-04-06  3:23   ` Ming Lei
  0 siblings, 0 replies; 23+ messages in thread
From: Ming Lei @ 2021-04-06  3:23 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner, Andrew Morton,
	Michal Hocko, Vladimir Davydov, Hugh Dickins, Shakeel Butt,
	Roman Gushchin, Muchun Song, Yang Shi, Alex Shi, Alexander Duyck,
	Wei Yang, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT

On Sat, Apr 3, 2021 at 3:18 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
>
> 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 and int_active_memcg 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>

Reviewed-by: Ming Lei <ming.lei@redhat.com>

-- 
Ming Lei

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

* Re: [PATCH 1/3] loop: Use worker per cgroup instead of kworker
       [not found] ` <20210403020902.1384-1-hdanton@sina.com>
@ 2021-04-06 18:59   ` Dan Schatzberg
       [not found]   ` <20210407065300.1478-1-hdanton@sina.com>
  1 sibling, 0 replies; 23+ messages in thread
From: Dan Schatzberg @ 2021-04-06 18:59 UTC (permalink / raw)
  To: Hillf Danton; +Cc: Jens Axboe, linux-block, linux-kernel, linux-mm

Hi Hillf, thanks for the review

On Sat, Apr 03, 2021 at 10:09:02AM +0800, Hillf Danton wrote:
> On Fri,  2 Apr 2021 12:16:32 Dan Schatzberg wrote:
> > +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);
> 
> Nit, only queue work if the worker is inactive - otherwise it is taking
> care of the cmd_list.

By worker is inactive, you mean worker is on the idle_list? Yes, I
think you're right that queue_work() is unnecessary in that case since
each worker checks empty cmd_list then adds itself to idle_list under
the lock.

> 
> > +		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_work_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)) {
> 
> The empty cmd_list is a good enough reason for worker to become idle.

This is only true with the above change to avoid a gratuitous
queue_work(), right? Otherwise we run the risk of freeing a worker
concurrently with loop_process_work() being invoked.

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

* Re: [PATCH 1/3] loop: Use worker per cgroup instead of kworker
       [not found]   ` <20210407065300.1478-1-hdanton@sina.com>
@ 2021-04-07 14:43     ` Dan Schatzberg
  0 siblings, 0 replies; 23+ messages in thread
From: Dan Schatzberg @ 2021-04-07 14:43 UTC (permalink / raw)
  To: Hillf Danton; +Cc: Jens Axboe, linux-block, linux-kernel, linux-mm

On Wed, Apr 07, 2021 at 02:53:00PM +0800, Hillf Danton wrote:
> On Tue, 6 Apr 2021 Dan Schatzberg wrote:
> >On Sat, Apr 03, 2021 at 10:09:02AM +0800, Hillf Danton wrote:
> >> On Fri,  2 Apr 2021 12:16:32 Dan Schatzberg wrote:
> >> > +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);
> >> 
> >> Nit, only queue work if the worker is inactive - otherwise it is taking
> >> care of the cmd_list.
> >
> >By worker is inactive, you mean worker is on the idle_list? Yes, I
> >think you're right that queue_work() is unnecessary in that case since
> >each worker checks empty cmd_list then adds itself to idle_list under
> >the lock.

A couple other corner cases - When worker is just allocated, it needs
a queue_work() and rootcg always needs a queue_work() since it never
sits on the idle_list. It does add to the logic a bit rather than just
unconditionally invoking queue_work()

> >
> >> 
> >> > +		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_work_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)) {
> >> 
> >> The empty cmd_list is a good enough reason for worker to become idle.
> >
> >This is only true with the above change to avoid a gratuitous
> >queue_work(), right?
> 
> It is always true because of the empty cmd_list - the idle_list is the only
> place for the worker to go at this point.
> 
> >Otherwise we run the risk of freeing a worker
> >concurrently with loop_process_work() being invoked.
> 
> My suggestion is a minor optimization at most without any change to removing
> worker off the idle_list on queuing work - that cuts the risk for you.

If I just change this line from

if (worker && !work_pending(&worker->work)) {

to

if (worker) {

then the following sequence of events is possible:

1) loop_queue_work runs, adds a command to the worker list
2) loop_process_work runs, processes a single command and then drops
the lock and reschedules
3) loop_queue_work runs again, acquires the lock, adds to the list and
invokes queue_work() again
4) loop_process_work resumes, acquires lock, processes work, notices
list is empty and adds itself to the idle_list
5) idle timer fires and frees the worker
6) loop_process_work runs again (because of the queue_work in 3) and
accesses freed memory

The !work_pending... check prevents 4) from adding itself to the
idle_list so this is not possible. I believe we can only make this
change if we also make the other change you suggested to avoid
gratuitous queue_work()

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

* Re: [PATCH V12 0/3] Charge loop device i/o to issuing cgroup
  2021-04-02 19:16 [PATCH V12 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
                   ` (3 preceding siblings ...)
       [not found] ` <20210403020902.1384-1-hdanton@sina.com>
@ 2021-04-12 15:45 ` Johannes Weiner
  2021-04-12 15:50   ` Jens Axboe
  4 siblings, 1 reply; 23+ messages in thread
From: Johannes Weiner @ 2021-04-12 15:45 UTC (permalink / raw)
  To: Andrew Morton, Jens Axboe
  Cc: Dan Schatzberg, Tejun Heo, Zefan Li, Michal Hocko,
	Vladimir Davydov, Hugh Dickins, Shakeel Butt, Roman Gushchin,
	Muchun Song, Yang Shi, Alex Shi, Alexander Duyck, Wei Yang,
	open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT

It looks like all feedback has been addressed and there hasn't been
any new activity on it in a while.

As per the suggestion last time [1], Andrew, Jens, could this go
through the -mm tree to deal with the memcg conflicts?

[1] https://lore.kernel.org/lkml/CALvZod6FMQQC17Zsu9xoKs=dFWaJdMC2Qk3YiDPUUQHx8teLYg@mail.gmail.com/

On Fri, Apr 02, 2021 at 12:16:31PM -0700, Dan Schatzberg wrote:
> No major changes, rebased on top of latest mm tree
> 
> Changes since V12:
> 
> * Small change to get_mem_cgroup_from_mm to avoid needing
>   get_active_memcg
> 
> Changes since V11:
> 
> * Removed WQ_MEM_RECLAIM flag from loop workqueue. Technically, this
>   can be driven by writeback, but this was causing a warning in xfs
>   and likely other filesystems aren't equipped to be driven by reclaim
>   at the VFS layer.
> * Included a small fix from Colin Ian King.
> * reworked get_mem_cgroup_from_mm to institute the necessary charge
>   priority.
> 
> Changes since V10:
> 
> * Added page-cache charging to mm: Charge active memcg when no mm is set
> 
> Changes since V9:
> 
> * Rebased against linus's branch which now includes Roman Gushchin's
>   patch this series is based off of
> 
> Changes since V8:
> 
> * Rebased on top of Roman Gushchin's patch
>   (https://lkml.org/lkml/2020/8/21/1464) which provides the nesting
>   support for setting active memcg. Dropped the patch from this series
>   that did the same thing.
> 
> Changes since V7:
> 
> * Rebased against linus's branch
> 
> Changes since V6:
> 
> * Added separate spinlock for worker synchronization
> * Minor style changes
> 
> 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
> 
>  drivers/block/loop.c       | 244 ++++++++++++++++++++++++++++++-------
>  drivers/block/loop.h       |  15 ++-
>  include/linux/memcontrol.h |   6 +
>  kernel/cgroup/cgroup.c     |   1 +
>  mm/filemap.c               |   2 +-
>  mm/memcontrol.c            |  49 +++++---
>  mm/shmem.c                 |   4 +-
>  7 files changed, 253 insertions(+), 68 deletions(-)
> 
> -- 
> 2.30.2
> 
> 

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

* Re: [PATCH V12 0/3] Charge loop device i/o to issuing cgroup
  2021-04-12 15:45 ` [PATCH V12 0/3] Charge loop device i/o to issuing cgroup Johannes Weiner
@ 2021-04-12 15:50   ` Jens Axboe
  0 siblings, 0 replies; 23+ messages in thread
From: Jens Axboe @ 2021-04-12 15:50 UTC (permalink / raw)
  To: Johannes Weiner, Andrew Morton
  Cc: Dan Schatzberg, Tejun Heo, Zefan Li, Michal Hocko,
	Vladimir Davydov, Hugh Dickins, Shakeel Butt, Roman Gushchin,
	Muchun Song, Yang Shi, Alex Shi, Alexander Duyck, Wei Yang,
	open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT

On 4/12/21 9:45 AM, Johannes Weiner wrote:
> It looks like all feedback has been addressed and there hasn't been
> any new activity on it in a while.
> 
> As per the suggestion last time [1], Andrew, Jens, could this go
> through the -mm tree to deal with the memcg conflicts?

Yep, I think that would make it the most painless for everyone.

Dan/Andrew, you can add my Acked-by to the series.

-- 
Jens Axboe


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

* Re: [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-06-30  9:42           ` Michal Koutný
@ 2021-06-30 14:49             ` Dan Schatzberg
  0 siblings, 0 replies; 23+ messages in thread
From: Dan Schatzberg @ 2021-06-30 14:49 UTC (permalink / raw)
  To: Michal Koutný
  Cc: Andrew Morton, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT, Johannes Weiner, Jens Axboe

> This is how I understand it:
> 
> --- a/drivers/block/loop.c
> +++ b/drivers/block/loop.c
> @@ -996,6 +996,7 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
>         rb_insert_color(&worker->rb_node, &lo->worker_tree);
>  queue_work:
>         if (worker) {
> +               WARN_ON_ONCE(worker->blkcg_css != cmd->blkcg_css);

Yes, this is correct. Though the check here seems a bit obvious to me
- it must be correct because we assign worker above:

if (cur_worker->blkcg_css == cmd->blkcg_css) {
        worker = cur_worker;
        break;

or when we construct the worker:

worker->blkcg_css = cmd->blkcg_css;

I think this WARN_ON_ONCE check might be more interesting in
loop_process_work which invokes loop_handle_cmd and actually uses
cmd->blkcg_css. In any event, your understanding is correct here.

>                 /*
>                  * We need to remove from the idle list here while
>                  * holding the lock so that the idle timer doesn't
> @@ -2106,6 +2107,8 @@ static blk_status_t loop_queue_rq(struct blk_mq_hw_ctx *hctx,
>         cmd->memcg_css = NULL;
>  #ifdef CONFIG_BLK_CGROUP
>         if (rq->bio && rq->bio->bi_blkg) {
> +               /* reference to blkcg_css will be held by loop_worker (outlives
> +                * cmd) or it is the eternal root css */

Yes, this is correct. Feel free to add my Acked-by to such a patch

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

* Re: [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-06-29 14:03         ` Dan Schatzberg
@ 2021-06-30  9:42           ` Michal Koutný
  2021-06-30 14:49             ` Dan Schatzberg
  0 siblings, 1 reply; 23+ messages in thread
From: Michal Koutný @ 2021-06-30  9:42 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Andrew Morton, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT, Johannes Weiner, Jens Axboe

[-- Attachment #1: Type: text/plain, Size: 2000 bytes --]

On Tue, Jun 29, 2021 at 10:03:33AM -0400, Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
> Hmm, perhaps I'm not understanding how the reference counting works,
> but my understanding is that we enter loop_queue_rq with presumably
> some code earlier holding a reference to the blkcg, we only need to
> acquire a reference sometime before returning from loop_queue_rq. The
> "window" between loop_queue_rq and loop_queue_work is all
> straight-line code so there's no possibility for the earlier code to
> get control back and drop the reference.

I don't say the current implementation is wrong, it just looked
suspicious to me when the css address is copied without taking the
reference.
The straight path is clear, I'm not sure about later invocations through
loop_workfn where the blkcg_css is accessed via the cmd->blkcg_css.

> Where would you suggest putting such a comment?

This is how I understand it:

--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -996,6 +996,7 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
        rb_insert_color(&worker->rb_node, &lo->worker_tree);
 queue_work:
        if (worker) {
+               WARN_ON_ONCE(worker->blkcg_css != cmd->blkcg_css);
                /*
                 * We need to remove from the idle list here while
                 * holding the lock so that the idle timer doesn't
@@ -2106,6 +2107,8 @@ static blk_status_t loop_queue_rq(struct blk_mq_hw_ctx *hctx,
        cmd->memcg_css = NULL;
 #ifdef CONFIG_BLK_CGROUP
        if (rq->bio && rq->bio->bi_blkg) {
+               /* reference to blkcg_css will be held by loop_worker (outlives
+                * cmd) or it is the eternal root css */
                cmd->blkcg_css = &bio_blkcg(rq->bio)->css;
 #ifdef CONFIG_MEMCG
                cmd->memcg_css =

(On further thoughts, maybe the blkcg_css reference isn't needed even in
the loop_worker if it can be reasoned that blkcg_css won't go away while
there's an outstanding rq.)

HTH,
Michal

[-- Attachment #2: Digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-06-29 10:26       ` Michal Koutný
@ 2021-06-29 14:03         ` Dan Schatzberg
  2021-06-30  9:42           ` Michal Koutný
  0 siblings, 1 reply; 23+ messages in thread
From: Dan Schatzberg @ 2021-06-29 14:03 UTC (permalink / raw)
  To: Michal Koutný
  Cc: Andrew Morton, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT, Johannes Weiner, Jens Axboe

> Non-inlining in the loop module doesn't seem like a big trouble. OTOH,
> other callers may be more sensitive and would need to rely on inlining.

Yes, this is my concern as well.

> I can't currently think of a nice way to have both the exported and the
> exlicitly inlined variant at once. It seems it's either API or perf
> craft in the end but both are uncertain, so I guess the current approach
> is fine in the end.
> 
> > Yes it is intentional. All requests (not just aio) go through the loop
> > worker which grabs the blkcg reference in loop_queue_work() on
> > construction. So I believe grabbing a reference per request is
> > unnecessary.
> 
> Isn't there a window without the reference between loop_queue_rq and
> loop_queue_work?

Hmm, perhaps I'm not understanding how the reference counting works,
but my understanding is that we enter loop_queue_rq with presumably
some code earlier holding a reference to the blkcg, we only need to
acquire a reference sometime before returning from loop_queue_rq. The
"window" between loop_queue_rq and loop_queue_work is all
straight-line code so there's no possibility for the earlier code to
get control back and drop the reference.

> I don't know, you seem to know better, so I'd suggest
> dropping a comment line into the code explaining this.

I wouldn't be so sure that I know any better here :D - I'm fairly
inexperienced in this domain.

Where would you suggest putting such a comment? The change in question
removed a particular case where we explicitly grab a reference to the
blkcg because now we do it uniformly in one place. Would you like a
comment explaining why we acquire a reference for all loop workers or
one explaining specifically why we don't need to acquire one for aio?

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

* Re: [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-06-28 14:17     ` Dan Schatzberg
@ 2021-06-29 10:26       ` Michal Koutný
  2021-06-29 14:03         ` Dan Schatzberg
  0 siblings, 1 reply; 23+ messages in thread
From: Michal Koutný @ 2021-06-29 10:26 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Andrew Morton, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT, Johannes Weiner, Jens Axboe

[-- Attachment #1: Type: text/plain, Size: 1031 bytes --]

On Mon, Jun 28, 2021 at 10:17:18AM -0400, Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
> Agreed that exporting int_active_memcg is an implementation detail,
> but would this prevent set_active_memcg from being inlined?

Non-inlining in the loop module doesn't seem like a big trouble. OTOH,
other callers may be more sensitive and would need to rely on inlining.
I can't currently think of a nice way to have both the exported and the
exlicitly inlined variant at once. It seems it's either API or perf
craft in the end but both are uncertain, so I guess the current approach
is fine in the end.

> Yes it is intentional. All requests (not just aio) go through the loop
> worker which grabs the blkcg reference in loop_queue_work() on
> construction. So I believe grabbing a reference per request is
> unnecessary.

Isn't there a window without the reference between loop_queue_rq and
loop_queue_work? I don't know, you seem to know better, so I'd suggest
dropping a comment line into the code explaining this.

Thanks,
Michal

[-- Attachment #2: Digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-06-25 15:01   ` Michal Koutný
@ 2021-06-28 14:17     ` Dan Schatzberg
  2021-06-29 10:26       ` Michal Koutný
  0 siblings, 1 reply; 23+ messages in thread
From: Dan Schatzberg @ 2021-06-28 14:17 UTC (permalink / raw)
  To: Michal Koutný
  Cc: Andrew Morton, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT, Johannes Weiner, Jens Axboe

Hi Michal,

On Fri, Jun 25, 2021 at 05:01:03PM +0200, Michal Koutný wrote:
> Hi.
> 
> On Thu, Jun 10, 2021 at 10:39:44AM -0700, Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
> > 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 and int_active_memcg so it
> > can be used by the loop module.
> 
> Wouldn't it be clearer to export (not explicitly inlined anymore)
> set_active_memcg() instead of the int_active_memcg that's rather an
> implementation detail?

Agreed that exporting int_active_memcg is an implementation detail,
but would this prevent set_active_memcg from being inlined? Is that
desireable?

> 
> > @@ -2111,13 +2112,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);
> 
> I see you dropped the cmd->blkcg_css reference (while rq is handled). Is
> it intentional?

Yes it is intentional. All requests (not just aio) go through the loop
worker which grabs the blkcg reference in loop_queue_work() on
construction. So I believe grabbing a reference per request is
unnecessary.

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

* Re: [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-06-10 17:39 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
@ 2021-06-25 15:01   ` Michal Koutný
  2021-06-28 14:17     ` Dan Schatzberg
  0 siblings, 1 reply; 23+ messages in thread
From: Michal Koutný @ 2021-06-25 15:01 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Andrew Morton, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT, Johannes Weiner, Jens Axboe

[-- Attachment #1: Type: text/plain, Size: 1416 bytes --]

Hi.

On Thu, Jun 10, 2021 at 10:39:44AM -0700, Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
> 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 and int_active_memcg so it
> can be used by the loop module.

Wouldn't it be clearer to export (not explicitly inlined anymore)
set_active_memcg() instead of the int_active_memcg that's rather an
implementation detail?

> @@ -2111,13 +2112,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);

I see you dropped the cmd->blkcg_css reference (while rq is handled). Is
it intentional? 

Thanks,
Michal

[-- Attachment #2: Digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-06-10 17:39 [PATCH V14 " Dan Schatzberg
@ 2021-06-10 17:39 ` Dan Schatzberg
  2021-06-25 15:01   ` Michal Koutný
  0 siblings, 1 reply; 23+ messages in thread
From: Dan Schatzberg @ 2021-06-10 17:39 UTC (permalink / raw)
  To: Andrew Morton
  Cc: open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT, Johannes Weiner, Jens Axboe

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 and int_active_memcg 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>
Acked-by: Jens Axboe <axboe@kernel.dk>
---
 drivers/block/loop.c       | 61 +++++++++++++++++++++++++-------------
 drivers/block/loop.h       |  3 +-
 include/linux/memcontrol.h |  6 ++++
 kernel/cgroup/cgroup.c     |  1 +
 mm/memcontrol.c            |  1 +
 5 files changed, 51 insertions(+), 21 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index fc4a0186d381..5198d8ad181c 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -78,6 +78,7 @@
 #include <linux/uio.h>
 #include <linux/ioprio.h>
 #include <linux/blk-cgroup.h>
+#include <linux/sched/mm.h>
 
 #include "loop.h"
 
@@ -516,8 +517,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);
 }
@@ -578,8 +577,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);
@@ -587,7 +584,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);
@@ -928,7 +924,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;
 };
 
@@ -957,7 +953,7 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 
 	spin_lock_irq(&lo->lo_work_lock);
 
-	if (queue_on_root_worker(cmd->css))
+	if (queue_on_root_worker(cmd->blkcg_css))
 		goto queue_work;
 
 	node = &lo->worker_tree.rb_node;
@@ -965,10 +961,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;
@@ -980,13 +976,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);
@@ -1306,7 +1307,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_work_lock);
@@ -2111,13 +2112,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;
@@ -2129,13 +2135,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 = set_active_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) {
+		set_active_memcg(old_memcg);
+		css_put(cmd->memcg_css);
+	}
  failed:
 	/* complete non-aio request */
 	if (!cmd->use_aio || ret) {
@@ -2214,7 +2235,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 9289c1cd6374..cd24a81e00e6 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -76,7 +76,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 bd0644d3a6df..360e61de53d7 100644
--- a/include/linux/memcontrol.h
+++ b/include/linux/memcontrol.h
@@ -1230,6 +1230,12 @@ static inline struct mem_cgroup *get_mem_cgroup_from_mm(struct mm_struct *mm)
 	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 74e3cc801615..90329cfff48d 100644
--- a/kernel/cgroup/cgroup.c
+++ b/kernel/cgroup/cgroup.c
@@ -577,6 +577,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)
 {
diff --git a/mm/memcontrol.c b/mm/memcontrol.c
index 919736ee656b..ae1f5d0cb581 100644
--- a/mm/memcontrol.c
+++ b/mm/memcontrol.c
@@ -78,6 +78,7 @@ struct mem_cgroup *root_mem_cgroup __read_mostly;
 
 /* Active memory cgroup to use from an interrupt context */
 DEFINE_PER_CPU(struct mem_cgroup *, int_active_memcg);
+EXPORT_PER_CPU_SYMBOL_GPL(int_active_memcg);
 
 /* Socket memory accounting disabled? */
 static bool cgroup_memory_nosocket __ro_after_init;
-- 
2.30.2


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

* [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-06-03 14:57 [PATCH V13 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
@ 2021-06-03 14:57 ` Dan Schatzberg
  0 siblings, 0 replies; 23+ messages in thread
From: Dan Schatzberg @ 2021-06-03 14:57 UTC (permalink / raw)
  To: Jens Axboe
  Cc: open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT, Johannes Weiner

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 and int_active_memcg 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>
Acked-by: Jens Axboe <axboe@kernel.dk>
---
 drivers/block/loop.c       | 61 +++++++++++++++++++++++++-------------
 drivers/block/loop.h       |  3 +-
 include/linux/memcontrol.h |  6 ++++
 kernel/cgroup/cgroup.c     |  1 +
 mm/memcontrol.c            |  1 +
 5 files changed, 51 insertions(+), 21 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 935edcf7c7b1..b38115c91288 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -78,6 +78,7 @@
 #include <linux/uio.h>
 #include <linux/ioprio.h>
 #include <linux/blk-cgroup.h>
+#include <linux/sched/mm.h>
 
 #include "loop.h"
 
@@ -516,8 +517,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);
 }
@@ -578,8 +577,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);
@@ -587,7 +584,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);
@@ -928,7 +924,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;
 };
 
@@ -943,7 +939,7 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 
 	spin_lock_irq(&lo->lo_work_lock);
 
-	if (!cmd->css)
+	if (!cmd->blkcg_css)
 		goto queue_work;
 
 	node = &lo->worker_tree.rb_node;
@@ -951,10 +947,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;
@@ -966,13 +962,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);
@@ -1291,7 +1292,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_work_lock);
@@ -2096,13 +2097,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;
@@ -2114,13 +2120,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 = set_active_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) {
+		set_active_memcg(old_memcg);
+		css_put(cmd->memcg_css);
+	}
  failed:
 	/* complete non-aio request */
 	if (!cmd->use_aio || ret) {
@@ -2199,7 +2220,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 9289c1cd6374..cd24a81e00e6 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -76,7 +76,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 c193be760709..542d9cae336b 100644
--- a/include/linux/memcontrol.h
+++ b/include/linux/memcontrol.h
@@ -1255,6 +1255,12 @@ static inline struct mem_cgroup *get_mem_cgroup_from_mm(struct mm_struct *mm)
 	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 21ecc6ee6a6d..9cc8c3a686b1 100644
--- a/kernel/cgroup/cgroup.c
+++ b/kernel/cgroup/cgroup.c
@@ -577,6 +577,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)
 {
diff --git a/mm/memcontrol.c b/mm/memcontrol.c
index 26dc2dc0056a..8a8222df44b5 100644
--- a/mm/memcontrol.c
+++ b/mm/memcontrol.c
@@ -78,6 +78,7 @@ struct mem_cgroup *root_mem_cgroup __read_mostly;
 
 /* Active memory cgroup to use from an interrupt context */
 DEFINE_PER_CPU(struct mem_cgroup *, int_active_memcg);
+EXPORT_PER_CPU_SYMBOL_GPL(int_active_memcg);
 
 /* Socket memory accounting disabled? */
 static bool cgroup_memory_nosocket;
-- 
2.30.2


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

* [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-03-29 14:48 [PATCH V11 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
@ 2021-03-29 14:48 ` Dan Schatzberg
  0 siblings, 0 replies; 23+ messages in thread
From: Dan Schatzberg @ 2021-03-29 14:48 UTC (permalink / raw)
  Cc: Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner, Andrew Morton,
	Michal Hocko, Vladimir Davydov, Hugh Dickins, Shakeel Butt,
	Roman Gushchin, Yang Shi, Muchun Song, Alex Shi, Alexander Duyck,
	Yafang Shao, Wei Yang, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT, Chris Down

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 and int_active_memcg 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 +
 mm/memcontrol.c            |  1 +
 5 files changed, 51 insertions(+), 21 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 5c18e6b856c2..96ade57c9f7c 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -78,6 +78,7 @@
 #include <linux/uio.h>
 #include <linux/ioprio.h>
 #include <linux/blk-cgroup.h>
+#include <linux/sched/mm.h>
 
 #include "loop.h"
 
@@ -516,8 +517,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);
 }
@@ -578,8 +577,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);
@@ -587,7 +584,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);
@@ -928,7 +924,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;
 };
 
@@ -945,7 +941,7 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 
 	spin_lock_irq(&lo->lo_work_lock);
 
-	if (!cmd->css)
+	if (!cmd->blkcg_css)
 		goto queue_work;
 
 	node = &lo->worker_tree.rb_node;
@@ -953,10 +949,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;
@@ -968,13 +964,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);
@@ -1298,7 +1299,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_work_lock);
@@ -2103,13 +2104,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;
@@ -2121,13 +2127,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 = set_active_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) {
+		set_active_memcg(old_memcg);
+		css_put(cmd->memcg_css);
+	}
  failed:
 	/* complete non-aio request */
 	if (!cmd->use_aio || ret) {
@@ -2206,7 +2227,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 9289c1cd6374..cd24a81e00e6 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -76,7 +76,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 4064c9dda534..df42be35b5fb 100644
--- a/include/linux/memcontrol.h
+++ b/include/linux/memcontrol.h
@@ -1178,6 +1178,12 @@ static inline struct mem_cgroup *get_mem_cgroup_from_mm(struct mm_struct *mm)
 	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 e049edd66776..8c84a5374238 100644
--- a/kernel/cgroup/cgroup.c
+++ b/kernel/cgroup/cgroup.c
@@ -577,6 +577,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)
 {
diff --git a/mm/memcontrol.c b/mm/memcontrol.c
index adc618814fd2..4aacdf06c6c8 100644
--- a/mm/memcontrol.c
+++ b/mm/memcontrol.c
@@ -78,6 +78,7 @@ struct mem_cgroup *root_mem_cgroup __read_mostly;
 
 /* Active memory cgroup to use from an interrupt context */
 DEFINE_PER_CPU(struct mem_cgroup *, int_active_memcg);
+EXPORT_PER_CPU_SYMBOL_GPL(int_active_memcg);
 
 /* Socket memory accounting disabled? */
 static bool cgroup_memory_nosocket;
-- 
2.30.2


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

* Re: [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-03-16 15:36 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
@ 2021-03-16 16:25   ` Shakeel Butt
  0 siblings, 0 replies; 23+ messages in thread
From: Shakeel Butt @ 2021-03-16 16:25 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner, Andrew Morton,
	Michal Hocko, Vladimir Davydov, Hugh Dickins, Roman Gushchin,
	Muchun Song, Alex Shi, Alexander Duyck, Chris Down, Yafang Shao,
	Wei Yang, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT

On Tue, Mar 16, 2021 at 8:37 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
>
[...]
>
>  /* Support for loadable transfer modules */
> diff --git a/include/linux/memcontrol.h b/include/linux/memcontrol.h
> index 0c04d39a7967..fd5dd961d01f 100644
> --- a/include/linux/memcontrol.h
> +++ b/include/linux/memcontrol.h
> @@ -1187,6 +1187,17 @@ static inline struct mem_cgroup *get_mem_cgroup_from_mm(struct mm_struct *mm)
>         return NULL;
>  }
>
> +static inline struct mem_cgroup *get_mem_cgroup_from_page(struct page *page)
> +{
> +       return NULL;
> +}

The above function has been removed.

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

* [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2021-03-16 15:36 [PATCH v10 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
@ 2021-03-16 15:36 ` Dan Schatzberg
  2021-03-16 16:25   ` Shakeel Butt
  0 siblings, 1 reply; 23+ messages in thread
From: Dan Schatzberg @ 2021-03-16 15:36 UTC (permalink / raw)
  Cc: Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner, Andrew Morton,
	Michal Hocko, Vladimir Davydov, Hugh Dickins, Shakeel Butt,
	Roman Gushchin, Muchun Song, Alex Shi, Alexander Duyck,
	Chris Down, Yafang Shao, Wei Yang, open list:BLOCK LAYER,
	open list, open list:CONTROL GROUP (CGROUP),
	open list:MEMORY MANAGEMENT

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 and int_active_memcg 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 | 11 +++++++
 kernel/cgroup/cgroup.c     |  1 +
 mm/memcontrol.c            |  1 +
 5 files changed, 56 insertions(+), 21 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 5c080af73caa..6cf3086a2e75 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"
 
@@ -515,8 +516,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);
 }
@@ -577,8 +576,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);
@@ -586,7 +583,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);
@@ -927,7 +923,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;
 };
 
@@ -944,7 +940,7 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 
 	spin_lock_irq(&lo->lo_work_lock);
 
-	if (!cmd->css)
+	if (!cmd->blkcg_css)
 		goto queue_work;
 
 	node = &lo->worker_tree.rb_node;
@@ -952,10 +948,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;
@@ -967,13 +963,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);
@@ -1297,7 +1298,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_work_lock);
@@ -2102,13 +2103,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;
@@ -2120,13 +2126,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 = set_active_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) {
+		set_active_memcg(old_memcg);
+		css_put(cmd->memcg_css);
+	}
  failed:
 	/* complete non-aio request */
 	if (!cmd->use_aio || ret) {
@@ -2205,7 +2226,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 9289c1cd6374..cd24a81e00e6 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -76,7 +76,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 0c04d39a7967..fd5dd961d01f 100644
--- a/include/linux/memcontrol.h
+++ b/include/linux/memcontrol.h
@@ -1187,6 +1187,17 @@ static inline struct mem_cgroup *get_mem_cgroup_from_mm(struct mm_struct *mm)
 	return NULL;
 }
 
+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 9153b20e5cc6..94c88f7221c5 100644
--- a/kernel/cgroup/cgroup.c
+++ b/kernel/cgroup/cgroup.c
@@ -577,6 +577,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)
 {
diff --git a/mm/memcontrol.c b/mm/memcontrol.c
index 9a1b23ed3412..f05501669e29 100644
--- a/mm/memcontrol.c
+++ b/mm/memcontrol.c
@@ -78,6 +78,7 @@ struct mem_cgroup *root_mem_cgroup __read_mostly;
 
 /* Active memory cgroup to use from an interrupt context */
 DEFINE_PER_CPU(struct mem_cgroup *, int_active_memcg);
+EXPORT_PER_CPU_SYMBOL_GPL(int_active_memcg);
 
 /* Socket memory accounting disabled? */
 static bool cgroup_memory_nosocket;
-- 
2.30.2


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

* [PATCH 3/3] loop: Charge i/o to mem and blk cg
  2020-08-31 15:36 [PATCH v8 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
@ 2020-08-31 15:37 ` Dan Schatzberg
  0 siblings, 0 replies; 23+ messages in thread
From: Dan Schatzberg @ 2020-08-31 15:37 UTC (permalink / raw)
  Cc: Dan Schatzberg, Johannes Weiner, Jens Axboe, Tejun Heo, Li Zefan,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Shakeel Butt, Roman Gushchin, Joonsoo Kim, Chris Down,
	Yafang Shao, Yang Shi, open list:BLOCK LAYER, open list,
	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 771685a6c259..3da34d454287 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"
 
@@ -518,8 +519,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);
 }
@@ -580,8 +579,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);
@@ -589,7 +586,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);
@@ -932,7 +928,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;
 };
 
@@ -949,7 +945,7 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 
 	spin_lock_irq(&lo->lo_work_lock);
 
-	if (!cmd->css)
+	if (!cmd->blkcg_css)
 		goto queue_work;
 
 	node = &lo->worker_tree.rb_node;
@@ -957,10 +953,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;
@@ -972,13 +968,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);
@@ -1304,7 +1305,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_work_lock);
@@ -2105,13 +2106,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;
@@ -2123,13 +2129,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 = set_active_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) {
+		set_active_memcg(old_memcg);
+		css_put(cmd->memcg_css);
+	}
  failed:
 	/* complete non-aio request */
 	if (!cmd->use_aio || ret) {
@@ -2208,7 +2229,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 0162b55a68e1..4d6886d9855a 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -75,7 +75,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 d0b036123c6a..fceac9f66d96 100644
--- a/include/linux/memcontrol.h
+++ b/include/linux/memcontrol.h
@@ -1031,6 +1031,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 dd247747ec14..16d059a89a68 100644
--- a/kernel/cgroup/cgroup.c
+++ b/kernel/cgroup/cgroup.c
@@ -580,6 +580,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] 23+ messages in thread

end of thread, other threads:[~2021-06-30 14:50 UTC | newest]

Thread overview: 23+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-04-02 19:16 [PATCH V12 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
2021-04-02 19:16 ` [PATCH 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
2021-04-06  1:44   ` Ming Lei
2021-04-02 19:16 ` [PATCH 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
2021-04-03  5:47   ` [External] " Muchun Song
2021-04-02 19:16 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
2021-04-06  3:23   ` Ming Lei
     [not found] ` <20210403020902.1384-1-hdanton@sina.com>
2021-04-06 18:59   ` [PATCH 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
     [not found]   ` <20210407065300.1478-1-hdanton@sina.com>
2021-04-07 14:43     ` Dan Schatzberg
2021-04-12 15:45 ` [PATCH V12 0/3] Charge loop device i/o to issuing cgroup Johannes Weiner
2021-04-12 15:50   ` Jens Axboe
  -- strict thread matches above, loose matches on Subject: below --
2021-06-10 17:39 [PATCH V14 " Dan Schatzberg
2021-06-10 17:39 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
2021-06-25 15:01   ` Michal Koutný
2021-06-28 14:17     ` Dan Schatzberg
2021-06-29 10:26       ` Michal Koutný
2021-06-29 14:03         ` Dan Schatzberg
2021-06-30  9:42           ` Michal Koutný
2021-06-30 14:49             ` Dan Schatzberg
2021-06-03 14:57 [PATCH V13 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
2021-06-03 14:57 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
2021-03-29 14:48 [PATCH V11 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
2021-03-29 14:48 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
2021-03-16 15:36 [PATCH v10 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
2021-03-16 15:36 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
2021-03-16 16:25   ` Shakeel Butt
2020-08-31 15:36 [PATCH v8 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
2020-08-31 15:37 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg

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