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

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

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

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

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

-- 
2.24.1


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

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

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

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

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

The locking for the tree and queues is fairly heavy handed - we acquire
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.

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 da693e6a834e..378a68e5ccf3 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -70,7 +70,6 @@
 #include <linux/writeback.h>
 #include <linux/completion.h>
 #include <linux/highmem.h>
-#include <linux/kthread.h>
 #include <linux/splice.h>
 #include <linux/sysfs.h>
 #include <linux/miscdevice.h>
@@ -83,6 +82,8 @@
 
 #include <linux/uaccess.h>
 
+#define LOOP_IDLE_WORKER_TIMEOUT (60 * HZ)
+
 static DEFINE_IDR(loop_index_idr);
 static DEFINE_MUTEX(loop_ctl_mutex);
 
@@ -911,27 +912,83 @@ static void loop_config_discard(struct loop_device *lo)
 		blk_queue_flag_clear(QUEUE_FLAG_DISCARD, q);
 }
 
-static void loop_unprepare_queue(struct loop_device *lo)
-{
-	kthread_flush_worker(&lo->worker);
-	kthread_stop(lo->worker_task);
-}
+struct loop_worker {
+	struct rb_node rb_node;
+	struct work_struct work;
+	struct list_head cmd_list;
+	struct list_head idle_list;
+	struct loop_device *lo;
+	struct cgroup_subsys_state *css;
+	unsigned long last_ran_at;
+};
 
-static int loop_kthread_worker_fn(void *worker_ptr)
-{
-	current->flags |= PF_LESS_THROTTLE | PF_MEMALLOC_NOIO;
-	return kthread_worker_fn(worker_ptr);
-}
+static void loop_workfn(struct work_struct *work);
+static void loop_rootcg_workfn(struct work_struct *work);
+static void loop_free_idle_workers(struct timer_list *timer);
 
-static int loop_prepare_queue(struct loop_device *lo)
+static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 {
-	kthread_init_worker(&lo->worker);
-	lo->worker_task = kthread_run(loop_kthread_worker_fn,
-			&lo->worker, "loop%d", lo->lo_number);
-	if (IS_ERR(lo->worker_task))
-		return -ENOMEM;
-	set_user_nice(lo->worker_task, MIN_NICE);
-	return 0;
+	struct rb_node **node = &(lo->worker_tree.rb_node), *parent = NULL;
+	struct loop_worker *cur_worker, *worker = NULL;
+	struct work_struct *work;
+	struct list_head *cmd_list;
+
+	spin_lock_irq(&lo->lo_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)
@@ -1007,14 +1064,25 @@ static int loop_set_fd(struct loop_device *lo, fmode_t mode,
 	size = get_loop_size(lo, file);
 	if ((loff_t)(sector_t)size != size)
 		goto out_unlock;
-	error = loop_prepare_queue(lo);
-	if (error)
+	lo->workqueue = alloc_workqueue("loop%d",
+					WQ_UNBOUND | WQ_FREEZABLE |
+					WQ_MEM_RECLAIM,
+					lo->lo_number);
+	if (!lo->workqueue) {
+		error = -ENOMEM;
 		goto out_unlock;
+	}
 
 	error = 0;
 
 	set_device_ro(bdev, (lo_flags & LO_FLAGS_READ_ONLY) != 0);
 
+	INIT_WORK(&lo->rootcg_work, loop_rootcg_workfn);
+	INIT_LIST_HEAD(&lo->rootcg_cmd_list);
+	INIT_LIST_HEAD(&lo->idle_worker_list);
+	lo->worker_tree = RB_ROOT;
+	timer_setup(&lo->timer, loop_free_idle_workers,
+		TIMER_DEFERRABLE);
 	lo->use_dio = false;
 	lo->lo_device = bdev;
 	lo->lo_flags = lo_flags;
@@ -1123,6 +1191,7 @@ static int __loop_clr_fd(struct loop_device *lo, bool release)
 	int err = 0;
 	bool partscan = false;
 	int lo_number;
+	struct loop_worker *pos, *worker;
 
 	mutex_lock(&loop_ctl_mutex);
 	if (WARN_ON_ONCE(lo->lo_state != Lo_rundown)) {
@@ -1139,6 +1208,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);
@@ -1176,7 +1257,6 @@ static int __loop_clr_fd(struct loop_device *lo, bool release)
 
 	partscan = lo->lo_flags & LO_FLAGS_PARTSCAN && bdev;
 	lo_number = lo->lo_number;
-	loop_unprepare_queue(lo);
 out_unlock:
 	mutex_unlock(&loop_ctl_mutex);
 	if (partscan) {
@@ -1954,7 +2034,7 @@ static blk_status_t loop_queue_rq(struct blk_mq_hw_ctx *hctx,
 	} else
 #endif
 		cmd->css = NULL;
-	kthread_queue_work(&lo->worker, &cmd->work);
+	loop_queue_work(lo, cmd);
 
 	return BLK_STS_OK;
 }
@@ -1983,26 +2063,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_LESS_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,
 };
 
@@ -2090,6 +2226,7 @@ static int loop_add(struct loop_device **l, int i)
 	atomic_set(&lo->lo_refcnt, 0);
 	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 af75a5ee4094..0162b55a68e1 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;
 
@@ -65,7 +69,7 @@ struct loop_device {
 };
 
 struct loop_cmd {
-	struct kthread_work work;
+	struct list_head list_entry;
 	bool use_aio; /* use AIO interface to handle I/O */
 	atomic_t ref; /* only for aio */
 	long ret;
-- 
2.24.1


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

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

From: Johannes Weiner <hannes@cmpxchg.org>

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

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

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

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

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

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


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

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

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

This patch establishes a precedence order for who gets charged:

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

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

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

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

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

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


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

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

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

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

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

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 378a68e5ccf3..c238e274788e 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -77,6 +77,7 @@
 #include <linux/uio.h>
 #include <linux/ioprio.h>
 #include <linux/blk-cgroup.h>
+#include <linux/sched/mm.h>
 
 #include "loop.h"
 
@@ -507,8 +508,6 @@ static void lo_rw_aio_complete(struct kiocb *iocb, long ret, long ret2)
 {
 	struct loop_cmd *cmd = container_of(iocb, struct loop_cmd, iocb);
 
-	if (cmd->css)
-		css_put(cmd->css);
 	cmd->ret = ret;
 	lo_rw_aio_do_completion(cmd);
 }
@@ -569,8 +568,6 @@ static int lo_rw_aio(struct loop_device *lo, struct loop_cmd *cmd,
 	cmd->iocb.ki_complete = lo_rw_aio_complete;
 	cmd->iocb.ki_flags = IOCB_DIRECT;
 	cmd->iocb.ki_ioprio = IOPRIO_PRIO_VALUE(IOPRIO_CLASS_NONE, 0);
-	if (cmd->css)
-		kthread_associate_blkcg(cmd->css);
 
 	if (rw == WRITE)
 		ret = call_write_iter(file, &cmd->iocb, &iter);
@@ -578,7 +575,6 @@ static int lo_rw_aio(struct loop_device *lo, struct loop_cmd *cmd,
 		ret = call_read_iter(file, &cmd->iocb, &iter);
 
 	lo_rw_aio_do_completion(cmd);
-	kthread_associate_blkcg(NULL);
 
 	if (ret != -EIOCBQUEUED)
 		cmd->iocb.ki_complete(&cmd->iocb, ret, 0);
@@ -918,7 +914,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;
 };
 
@@ -935,7 +931,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;
@@ -943,10 +939,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;
@@ -958,13 +954,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);
@@ -1214,7 +1215,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);
@@ -2027,13 +2028,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;
@@ -2045,13 +2051,28 @@ static void loop_handle_cmd(struct loop_cmd *cmd)
 	const bool write = op_is_write(req_op(rq));
 	struct loop_device *lo = rq->q->queuedata;
 	int ret = 0;
+	struct mem_cgroup *old_memcg = NULL;
 
 	if (write && (lo->lo_flags & LO_FLAGS_READ_ONLY)) {
 		ret = -EIO;
 		goto failed;
 	}
 
+	if (cmd->blkcg_css)
+		kthread_associate_blkcg(cmd->blkcg_css);
+	if (cmd->memcg_css)
+		old_memcg = memalloc_use_memcg(
+			mem_cgroup_from_css(cmd->memcg_css));
+
 	ret = do_req_filebacked(lo, rq);
+
+	if (cmd->blkcg_css)
+		kthread_associate_blkcg(NULL);
+
+	if (cmd->memcg_css) {
+		memalloc_use_memcg(old_memcg);
+		css_put(cmd->memcg_css);
+	}
  failed:
 	/* complete non-aio request */
 	if (!cmd->use_aio || ret) {
@@ -2129,7 +2150,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 977edd3b7bd8..67a98db5be9e 100644
--- a/include/linux/memcontrol.h
+++ b/include/linux/memcontrol.h
@@ -925,6 +925,12 @@ static inline struct mem_cgroup *get_mem_cgroup_from_page(struct page *page)
 	return NULL;
 }
 
+static inline
+struct mem_cgroup *mem_cgroup_from_css(struct cgroup_subsys_state *css)
+{
+	return NULL;
+}
+
 static inline void mem_cgroup_put(struct mem_cgroup *memcg)
 {
 }
diff --git a/kernel/cgroup/cgroup.c b/kernel/cgroup/cgroup.c
index 06b5ea9d899d..a3c64d961d68 100644
--- a/kernel/cgroup/cgroup.c
+++ b/kernel/cgroup/cgroup.c
@@ -587,6 +587,7 @@ struct cgroup_subsys_state *cgroup_get_e_css(struct cgroup *cgrp,
 	rcu_read_unlock();
 	return css;
 }
+EXPORT_SYMBOL_GPL(cgroup_get_e_css);
 
 static void cgroup_get_live(struct cgroup *cgrp)
 {
-- 
2.24.1


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

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

On Thu, May 28, 2020 at 6:55 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
>
> Much of the discussion about this has died down. There's been a
> concern raised that we could generalize infrastructure across loop,
> md, etc. This may be possible, in the future, but it isn't clear to me
> how this would look like. I'm inclined to fix the existing issue with
> loop devices now (this is a problem we hit at FB) and address
> consolidation with other cases if and when those need to be addressed.
>

What's the status of this series?

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

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

On Thu, Aug 20, 2020 at 10:06:44AM -0700, Shakeel Butt wrote:
> On Thu, May 28, 2020 at 6:55 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
> >
> > Much of the discussion about this has died down. There's been a
> > concern raised that we could generalize infrastructure across loop,
> > md, etc. This may be possible, in the future, but it isn't clear to me
> > how this would look like. I'm inclined to fix the existing issue with
> > loop devices now (this is a problem we hit at FB) and address
> > consolidation with other cases if and when those need to be addressed.
> >
> 
> What's the status of this series?

Thanks for reminding me about this. I haven't got any further
feedback. I'll bug Jens to take a look and see if he has any concerns
and if not send a rebased version.

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

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

On 8/21/20 9:04 AM, Dan Schatzberg wrote:
> On Thu, Aug 20, 2020 at 10:06:44AM -0700, Shakeel Butt wrote:
>> On Thu, May 28, 2020 at 6:55 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
>>>
>>> Much of the discussion about this has died down. There's been a
>>> concern raised that we could generalize infrastructure across loop,
>>> md, etc. This may be possible, in the future, but it isn't clear to me
>>> how this would look like. I'm inclined to fix the existing issue with
>>> loop devices now (this is a problem we hit at FB) and address
>>> consolidation with other cases if and when those need to be addressed.
>>>
>>
>> What's the status of this series?
> 
> Thanks for reminding me about this. I haven't got any further
> feedback. I'll bug Jens to take a look and see if he has any concerns
> and if not send a rebased version.

No immediate concerns, I think rebasing and sending one against the
current tree is probably a good idea. Then we can hopefully get it
queued up for 5.10.

-- 
Jens Axboe


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

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

On Fri, Aug 21, 2020 at 11:04:05AM -0400, Dan Schatzberg wrote:
> On Thu, Aug 20, 2020 at 10:06:44AM -0700, Shakeel Butt wrote:
> > On Thu, May 28, 2020 at 6:55 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
> > >
> > > Much of the discussion about this has died down. There's been a
> > > concern raised that we could generalize infrastructure across loop,
> > > md, etc. This may be possible, in the future, but it isn't clear to me
> > > how this would look like. I'm inclined to fix the existing issue with
> > > loop devices now (this is a problem we hit at FB) and address
> > > consolidation with other cases if and when those need to be addressed.
> > >
> > 
> > What's the status of this series?
> 
> Thanks for reminding me about this. I haven't got any further
> feedback. I'll bug Jens to take a look and see if he has any concerns
> and if not send a rebased version.

Just as a note, I stole a patch from this series called
"mm: support nesting memalloc_use_memcg()" to use for the bpf memory accounting.
I rewrote the commit log and rebased to the tot with some trivial changes.

I just sent it upstream:
https://lore.kernel.org/bpf/20200821150134.2581465-1-guro@fb.com/T/#md7edb6b5b940cee1c4d15e3cef17aa8b07328c2e

It looks like we need it for two independent sub-systems, so I wonder
if we want to route it first through the mm tree as a standalone patch?

Thanks!

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

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

On Fri, Aug 21, 2020 at 9:02 AM Roman Gushchin <guro@fb.com> wrote:
>
> On Fri, Aug 21, 2020 at 11:04:05AM -0400, Dan Schatzberg wrote:
> > On Thu, Aug 20, 2020 at 10:06:44AM -0700, Shakeel Butt wrote:
> > > On Thu, May 28, 2020 at 6:55 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
> > > >
> > > > Much of the discussion about this has died down. There's been a
> > > > concern raised that we could generalize infrastructure across loop,
> > > > md, etc. This may be possible, in the future, but it isn't clear to me
> > > > how this would look like. I'm inclined to fix the existing issue with
> > > > loop devices now (this is a problem we hit at FB) and address
> > > > consolidation with other cases if and when those need to be addressed.
> > > >
> > >
> > > What's the status of this series?
> >
> > Thanks for reminding me about this. I haven't got any further
> > feedback. I'll bug Jens to take a look and see if he has any concerns
> > and if not send a rebased version.
>
> Just as a note, I stole a patch from this series called
> "mm: support nesting memalloc_use_memcg()" to use for the bpf memory accounting.
> I rewrote the commit log and rebased to the tot with some trivial changes.
>
> I just sent it upstream:
> https://lore.kernel.org/bpf/20200821150134.2581465-1-guro@fb.com/T/#md7edb6b5b940cee1c4d15e3cef17aa8b07328c2e
>
> It looks like we need it for two independent sub-systems, so I wonder
> if we want to route it first through the mm tree as a standalone patch?
>

Another way is to push that patch to 5.9-rc2 linus tree, so both block
and mm branches for 5.10 will have it. (Not sure if that's ok.)

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

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

On Fri, Aug 21, 2020 at 09:27:56AM -0700, Shakeel Butt wrote:
> On Fri, Aug 21, 2020 at 9:02 AM Roman Gushchin <guro@fb.com> wrote:
> >
> > On Fri, Aug 21, 2020 at 11:04:05AM -0400, Dan Schatzberg wrote:
> > > On Thu, Aug 20, 2020 at 10:06:44AM -0700, Shakeel Butt wrote:
> > > > On Thu, May 28, 2020 at 6:55 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
> > > > >
> > > > > Much of the discussion about this has died down. There's been a
> > > > > concern raised that we could generalize infrastructure across loop,
> > > > > md, etc. This may be possible, in the future, but it isn't clear to me
> > > > > how this would look like. I'm inclined to fix the existing issue with
> > > > > loop devices now (this is a problem we hit at FB) and address
> > > > > consolidation with other cases if and when those need to be addressed.
> > > > >
> > > >
> > > > What's the status of this series?
> > >
> > > Thanks for reminding me about this. I haven't got any further
> > > feedback. I'll bug Jens to take a look and see if he has any concerns
> > > and if not send a rebased version.
> >
> > Just as a note, I stole a patch from this series called
> > "mm: support nesting memalloc_use_memcg()" to use for the bpf memory accounting.
> > I rewrote the commit log and rebased to the tot with some trivial changes.
> >
> > I just sent it upstream:
> > https://lore.kernel.org/bpf/20200821150134.2581465-1-guro@fb.com/T/#md7edb6b5b940cee1c4d15e3cef17aa8b07328c2e
> >
> > It looks like we need it for two independent sub-systems, so I wonder
> > if we want to route it first through the mm tree as a standalone patch?
> >
> 
> Another way is to push that patch to 5.9-rc2 linus tree, so both block
> and mm branches for 5.10 will have it. (Not sure if that's ok.)

Ok, it looks like the patch provides a generally useful API enhancement.
And we do have at least two potential use cases for it.
Let me send it as a standalone patch to linux-mm@.

Btw, Shakeel, what do you think of s/memalloc_use_memcg()/set_active_memcg() ?

And thank you for reviews!

Roman

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

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

On Fri, Aug 21, 2020 at 1:05 PM Roman Gushchin <guro@fb.com> wrote:
>
> On Fri, Aug 21, 2020 at 09:27:56AM -0700, Shakeel Butt wrote:
> > On Fri, Aug 21, 2020 at 9:02 AM Roman Gushchin <guro@fb.com> wrote:
> > >
> > > On Fri, Aug 21, 2020 at 11:04:05AM -0400, Dan Schatzberg wrote:
> > > > On Thu, Aug 20, 2020 at 10:06:44AM -0700, Shakeel Butt wrote:
> > > > > On Thu, May 28, 2020 at 6:55 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
> > > > > >
> > > > > > Much of the discussion about this has died down. There's been a
> > > > > > concern raised that we could generalize infrastructure across loop,
> > > > > > md, etc. This may be possible, in the future, but it isn't clear to me
> > > > > > how this would look like. I'm inclined to fix the existing issue with
> > > > > > loop devices now (this is a problem we hit at FB) and address
> > > > > > consolidation with other cases if and when those need to be addressed.
> > > > > >
> > > > >
> > > > > What's the status of this series?
> > > >
> > > > Thanks for reminding me about this. I haven't got any further
> > > > feedback. I'll bug Jens to take a look and see if he has any concerns
> > > > and if not send a rebased version.
> > >
> > > Just as a note, I stole a patch from this series called
> > > "mm: support nesting memalloc_use_memcg()" to use for the bpf memory accounting.
> > > I rewrote the commit log and rebased to the tot with some trivial changes.
> > >
> > > I just sent it upstream:
> > > https://lore.kernel.org/bpf/20200821150134.2581465-1-guro@fb.com/T/#md7edb6b5b940cee1c4d15e3cef17aa8b07328c2e
> > >
> > > It looks like we need it for two independent sub-systems, so I wonder
> > > if we want to route it first through the mm tree as a standalone patch?
> > >
> >
> > Another way is to push that patch to 5.9-rc2 linus tree, so both block
> > and mm branches for 5.10 will have it. (Not sure if that's ok.)
>
> Ok, it looks like the patch provides a generally useful API enhancement.
> And we do have at least two potential use cases for it.
> Let me send it as a standalone patch to linux-mm@.
>
> Btw, Shakeel, what do you think of s/memalloc_use_memcg()/set_active_memcg() ?
>

I am fine with it.

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

* [PATCH 4/4] loop: Charge i/o to mem and blk cg
  2020-08-24 15:35 [PATCH v7 " Dan Schatzberg
@ 2020-08-24 15:36 ` Dan Schatzberg
  0 siblings, 0 replies; 15+ messages in thread
From: Dan Schatzberg @ 2020-08-24 15:36 UTC (permalink / raw)
  Cc: Dan Schatzberg, Johannes Weiner, Jens Axboe, Alexander Viro,
	Jan Kara, Amir Goldstein, Tejun Heo, Li Zefan, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Shakeel Butt,
	Roman Gushchin, Chris Down, Yafang Shao, Jakub Kicinski,
	Yang Shi, Peter Zijlstra, Ingo Molnar, Mathieu Desnoyers,
	Vlastimil Babka, Daniel Jordan, Thomas Gleixner,
	Michel Lespinasse, open list:BLOCK LAYER, open list,
	open list:FILESYSTEMS (VFS and infrastructure),
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

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

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

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

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index c8dae53f3914..102d236d1540 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);
@@ -929,7 +925,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;
 };
 
@@ -946,7 +942,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;
@@ -954,10 +950,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;
@@ -969,13 +965,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);
@@ -1301,7 +1302,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 = memalloc_use_memcg(
+			mem_cgroup_from_css(cmd->memcg_css));
+
 	ret = do_req_filebacked(lo, rq);
+
+	if (cmd->blkcg_css)
+		kthread_associate_blkcg(NULL);
+
+	if (cmd->memcg_css) {
+		memalloc_use_memcg(old_memcg);
+		css_put(cmd->memcg_css);
+	}
  failed:
 	/* complete non-aio request */
 	if (!cmd->use_aio || ret) {
@@ -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 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)
 {
diff --git a/mm/shmem.c b/mm/shmem.c
index 77c908730be4..1139f52ac4ee 100644
--- a/mm/shmem.c
+++ b/mm/shmem.c
@@ -1696,7 +1696,6 @@ 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 : NULL;
-	struct mem_cgroup *memcg;
 	struct page *page;
 	swp_entry_t swap;
 	int error;
-- 
2.24.1


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

* Re: [PATCH 4/4] loop: Charge i/o to mem and blk cg
       [not found] ` <20200421033337.13208-1-hdanton@sina.com>
@ 2020-04-21 13:57   ` Dan Schatzberg
  0 siblings, 0 replies; 15+ messages in thread
From: Dan Schatzberg @ 2020-04-21 13:57 UTC (permalink / raw)
  To: Hillf Danton; +Cc: Jens Axboe, linux-block, linux-kernel, linux-mm

On Tue, Apr 21, 2020 at 11:33:37AM +0800, Hillf Danton wrote:
> 
> On Mon, 20 Apr 2020 18:39:32 -0400 Dan Schatzberg wrote:
> > 
> > @@ -964,13 +960,16 @@ 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;
> > +		cmd->memcg_css = NULL;
> 
> Dunno if 	css_put(cmd->memcg_css);

Good catch. Need to drop the reference here.

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

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

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       | 59 +++++++++++++++++++++++++-------------
 drivers/block/loop.h       |  3 +-
 include/linux/memcontrol.h |  6 ++++
 kernel/cgroup/cgroup.c     |  1 +
 4 files changed, 48 insertions(+), 21 deletions(-)

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


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

end of thread, other threads:[~2020-08-24 15:41 UTC | newest]

Thread overview: 15+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-05-28 13:54 [PATCH v6 0/4] Charge loop device i/o to issuing cgroup Dan Schatzberg
2020-05-28 13:54 ` [PATCH 1/4] loop: Use worker per cgroup instead of kworker Dan Schatzberg
2020-05-28 13:54 ` [PATCH 2/4] mm: support nesting memalloc_use_memcg() Dan Schatzberg
2020-05-28 13:54 ` [PATCH 3/4] mm: Charge active memcg when no mm is set Dan Schatzberg
2020-05-28 13:54 ` [PATCH 4/4] loop: Charge i/o to mem and blk cg Dan Schatzberg
2020-08-20 17:06 ` [PATCH v6 0/4] Charge loop device i/o to issuing cgroup Shakeel Butt
2020-08-21 15:04   ` Dan Schatzberg
2020-08-21 15:08     ` Jens Axboe
2020-08-21 16:01     ` Roman Gushchin
2020-08-21 16:27       ` Shakeel Butt
2020-08-21 20:05         ` Roman Gushchin
2020-08-21 20:23           ` Shakeel Butt
  -- strict thread matches above, loose matches on Subject: below --
2020-08-24 15:35 [PATCH v7 " Dan Schatzberg
2020-08-24 15:36 ` [PATCH 4/4] loop: Charge i/o to mem and blk cg Dan Schatzberg
2020-04-20 22:39 [PATCH 0/4] Charge loop device i/o to issuing cgroup Dan Schatzberg
2020-04-20 22:39 ` [PATCH 4/4] loop: Charge i/o to mem and blk cg Dan Schatzberg
     [not found] ` <20200421033337.13208-1-hdanton@sina.com>
2020-04-21 13:57   ` 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).