linux-mm.kvack.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v10 0/3] Charge loop device i/o to issuing cgroup
@ 2021-03-16 15:36 Dan Schatzberg
  2021-03-16 15:36 ` [PATCH 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
                   ` (3 more replies)
  0 siblings, 4 replies; 15+ 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

No major changes, just rebasing and resubmitting

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       | 248 ++++++++++++++++++++++++++++++-------
 drivers/block/loop.h       |  15 ++-
 include/linux/memcontrol.h |  11 ++
 kernel/cgroup/cgroup.c     |   1 +
 mm/filemap.c               |   2 +-
 mm/memcontrol.c            |  15 ++-
 mm/shmem.c                 |   4 +-
 7 files changed, 242 insertions(+), 54 deletions(-)

-- 
2.30.2



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

* [PATCH 1/3] loop: Use worker per cgroup instead of kworker
  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 15:36 ` [PATCH 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
                   ` (2 subsequent siblings)
  3 siblings, 0 replies; 15+ 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

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 | 207 ++++++++++++++++++++++++++++++++++++-------
 drivers/block/loop.h |  12 ++-
 2 files changed, 182 insertions(+), 37 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index a370cde3ddd4..5c080af73caa 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);
 
@@ -920,27 +921,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)
@@ -1126,12 +1183,27 @@ 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)
+	error = -EFBIG;
+	size = get_loop_size(lo, file);
+	if ((loff_t)(sector_t)size != size)
 		goto out_unlock;
+	lo->workqueue = alloc_workqueue("loop%d",
+					WQ_UNBOUND | WQ_FREEZABLE |
+					WQ_MEM_RECLAIM,
+					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;
@@ -1199,6 +1271,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)) {
@@ -1218,6 +1291,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);
@@ -1254,7 +1339,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) {
@@ -2025,7 +2109,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;
 }
@@ -2055,26 +2139,82 @@ static void loop_handle_cmd(struct loop_cmd *cmd)
 	}
 }
 
-static void loop_queue_work(struct kthread_work *work)
+static void loop_set_timer(struct loop_device *lo)
+{
+	timer_reduce(&lo->timer, jiffies + LOOP_IDLE_WORKER_TIMEOUT);
+}
+
+static void loop_process_work(struct loop_worker *worker,
+			struct list_head *cmd_list, struct loop_device *lo)
 {
-	struct loop_cmd *cmd =
-		container_of(work, struct loop_cmd, work);
+	int orig_flags = current->flags;
+	struct loop_cmd *cmd;
 
-	loop_handle_cmd(cmd);
+	current->flags |= PF_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 int loop_init_request(struct blk_mq_tag_set *set, struct request *rq,
-		unsigned int hctx_idx, unsigned int numa_node)
+static void loop_workfn(struct work_struct *work)
 {
-	struct loop_cmd *cmd = blk_mq_rq_to_pdu(rq);
+	struct loop_worker *worker =
+		container_of(work, struct loop_worker, work);
+	loop_process_work(worker, &worker->cmd_list, worker->lo);
+}
 
-	kthread_init_work(&cmd->work, loop_queue_work);
-	return 0;
+static void loop_rootcg_workfn(struct work_struct *work)
+{
+	struct loop_device *lo =
+		container_of(work, struct loop_device, rootcg_work);
+	loop_process_work(NULL, &lo->rootcg_cmd_list, lo);
+}
+
+static void loop_free_idle_workers(struct timer_list *timer)
+{
+	struct loop_device *lo = container_of(timer, struct loop_device, timer);
+	struct loop_worker *pos, *worker;
+
+	spin_lock_irq(&lo->lo_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,
 };
 
@@ -2163,6 +2303,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] 15+ messages in thread

* [PATCH 2/3] mm: Charge active memcg when no mm is set
  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 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
@ 2021-03-16 15:36 ` Dan Schatzberg
  2021-03-16 15:50   ` Shakeel Butt
  2021-03-16 15:36 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
  2021-03-17 22:30 ` [PATCH v10 0/3] Charge loop device i/o to issuing cgroup Jens Axboe
  3 siblings, 1 reply; 15+ 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

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/filemap.c    |  2 +-
 mm/memcontrol.c | 14 +++++++++++---
 mm/shmem.c      |  4 ++--
 3 files changed, 14 insertions(+), 6 deletions(-)

diff --git a/mm/filemap.c b/mm/filemap.c
index 43700480d897..5135f330f05c 100644
--- a/mm/filemap.c
+++ b/mm/filemap.c
@@ -843,7 +843,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 e064ac0d850a..9a1b23ed3412 100644
--- a/mm/memcontrol.c
+++ b/mm/memcontrol.c
@@ -6690,7 +6690,8 @@ void mem_cgroup_calculate_protection(struct mem_cgroup *root,
  * @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.
  *
  * Returns 0 on success. Otherwise, an error code is returned.
  */
@@ -6726,8 +6727,15 @@ int mem_cgroup_charge(struct page *page, struct mm_struct *mm, gfp_t gfp_mask)
 		rcu_read_unlock();
 	}
 
-	if (!memcg)
-		memcg = get_mem_cgroup_from_mm(mm);
+	if (!memcg) {
+		if (!mm) {
+			memcg = get_mem_cgroup_from_current();
+			if (!memcg)
+				memcg = get_mem_cgroup_from_mm(current->mm);
+		} else {
+			memcg = get_mem_cgroup_from_mm(mm);
+		}
+	}
 
 	ret = try_charge(memcg, gfp_mask, nr_pages);
 	if (ret)
diff --git a/mm/shmem.c b/mm/shmem.c
index b2db4ed0fbc7..353b362c370e 100644
--- a/mm/shmem.c
+++ b/mm/shmem.c
@@ -1695,7 +1695,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;
@@ -1816,7 +1816,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] 15+ 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 ` [PATCH 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
  2021-03-16 15:36 ` [PATCH 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
@ 2021-03-16 15:36 ` Dan Schatzberg
  2021-03-16 16:25   ` Shakeel Butt
  2021-03-17 22:30 ` [PATCH v10 0/3] Charge loop device i/o to issuing cgroup Jens Axboe
  3 siblings, 1 reply; 15+ 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] 15+ messages in thread

* Re: [PATCH 2/3] mm: Charge active memcg when no mm is set
  2021-03-16 15:36 ` [PATCH 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
@ 2021-03-16 15:50   ` Shakeel Butt
  2021-03-16 16:02     ` Dan Schatzberg
  0 siblings, 1 reply; 15+ messages in thread
From: Shakeel Butt @ 2021-03-16 15:50 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:
>
> memalloc_use_memcg() worked for kernel allocations but was silently
> ignored for user pages.

set_active_memcg()

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

It's a bit more sophisticated than current->active_memcg. It has been
extended to work in interrupt context as well.

>
> Previously, if a NULL mm was passed to mem_cgroup_try_charge (case 3) it

mem_cgroup_charge()

> 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/filemap.c    |  2 +-
>  mm/memcontrol.c | 14 +++++++++++---
>  mm/shmem.c      |  4 ++--
>  3 files changed, 14 insertions(+), 6 deletions(-)
>
> diff --git a/mm/filemap.c b/mm/filemap.c
> index 43700480d897..5135f330f05c 100644
> --- a/mm/filemap.c
> +++ b/mm/filemap.c
> @@ -843,7 +843,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 e064ac0d850a..9a1b23ed3412 100644
> --- a/mm/memcontrol.c
> +++ b/mm/memcontrol.c
> @@ -6690,7 +6690,8 @@ void mem_cgroup_calculate_protection(struct mem_cgroup *root,
>   * @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.
>   *
>   * Returns 0 on success. Otherwise, an error code is returned.
>   */
> @@ -6726,8 +6727,15 @@ int mem_cgroup_charge(struct page *page, struct mm_struct *mm, gfp_t gfp_mask)
>                 rcu_read_unlock();
>         }
>
> -       if (!memcg)
> -               memcg = get_mem_cgroup_from_mm(mm);
> +       if (!memcg) {
> +               if (!mm) {
> +                       memcg = get_mem_cgroup_from_current();
> +                       if (!memcg)
> +                               memcg = get_mem_cgroup_from_mm(current->mm);
> +               } else {
> +                       memcg = get_mem_cgroup_from_mm(mm);
> +               }
> +       }

You will need to rebase to the latest mm tree. This code has changed.


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

* Re: [PATCH 2/3] mm: Charge active memcg when no mm is set
  2021-03-16 15:50   ` Shakeel Butt
@ 2021-03-16 16:02     ` Dan Schatzberg
  0 siblings, 0 replies; 15+ messages in thread
From: Dan Schatzberg @ 2021-03-16 16:02 UTC (permalink / raw)
  To: Shakeel Butt
  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 08:50:16AM -0700, Shakeel Butt wrote:
> You will need to rebase to the latest mm tree. This code has changed.

Thanks for the feedback, I will address these comments in another
rebase. I'll wait and see if there's any comments concerning the
loop-related patches but it sounds like this will need to go through
the mm-tree


^ permalink raw reply	[flat|nested] 15+ 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; 15+ 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] 15+ messages in thread

* Re: [PATCH v10 0/3] Charge loop device i/o to issuing cgroup
  2021-03-16 15:36 [PATCH v10 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
                   ` (2 preceding siblings ...)
  2021-03-16 15:36 ` [PATCH 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
@ 2021-03-17 22:30 ` Jens Axboe
  2021-03-18 15:53   ` Shakeel Butt
  3 siblings, 1 reply; 15+ messages in thread
From: Jens Axboe @ 2021-03-17 22:30 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: 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

On 3/16/21 9:36 AM, Dan Schatzberg wrote:
> No major changes, just rebasing and resubmitting

Applied for 5.13, thanks.

-- 
Jens Axboe



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

* Re: [PATCH v10 0/3] Charge loop device i/o to issuing cgroup
  2021-03-17 22:30 ` [PATCH v10 0/3] Charge loop device i/o to issuing cgroup Jens Axboe
@ 2021-03-18 15:53   ` Shakeel Butt
  2021-03-18 16:00     ` Jens Axboe
  0 siblings, 1 reply; 15+ messages in thread
From: Shakeel Butt @ 2021-03-18 15:53 UTC (permalink / raw)
  To: Jens Axboe
  Cc: Dan Schatzberg, 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 Wed, Mar 17, 2021 at 3:30 PM Jens Axboe <axboe@kernel.dk> wrote:
>
> On 3/16/21 9:36 AM, Dan Schatzberg wrote:
> > No major changes, just rebasing and resubmitting
>
> Applied for 5.13, thanks.
>

I have requested a couple of changes in the patch series. Can this
applied series still be changed or new patches are required?


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

* Re: [PATCH v10 0/3] Charge loop device i/o to issuing cgroup
  2021-03-18 15:53   ` Shakeel Butt
@ 2021-03-18 16:00     ` Jens Axboe
  2021-03-18 23:46       ` Andrew Morton
  0 siblings, 1 reply; 15+ messages in thread
From: Jens Axboe @ 2021-03-18 16:00 UTC (permalink / raw)
  To: Shakeel Butt
  Cc: Dan Schatzberg, 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 3/18/21 9:53 AM, Shakeel Butt wrote:
> On Wed, Mar 17, 2021 at 3:30 PM Jens Axboe <axboe@kernel.dk> wrote:
>>
>> On 3/16/21 9:36 AM, Dan Schatzberg wrote:
>>> No major changes, just rebasing and resubmitting
>>
>> Applied for 5.13, thanks.
>>
> 
> I have requested a couple of changes in the patch series. Can this
> applied series still be changed or new patches are required?

I have nothing sitting on top of it for now, so as far as I'm concerned
we can apply a new series instead. Then we can also fold in that fix
from Colin that he posted this morning...

-- 
Jens Axboe



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

* Re: [PATCH v10 0/3] Charge loop device i/o to issuing cgroup
  2021-03-18 16:00     ` Jens Axboe
@ 2021-03-18 23:46       ` Andrew Morton
  2021-03-19  0:56         ` Shakeel Butt
  0 siblings, 1 reply; 15+ messages in thread
From: Andrew Morton @ 2021-03-18 23:46 UTC (permalink / raw)
  To: Jens Axboe
  Cc: Shakeel Butt, Dan Schatzberg, Tejun Heo, Zefan Li,
	Johannes Weiner, 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 Thu, 18 Mar 2021 10:00:17 -0600 Jens Axboe <axboe@kernel.dk> wrote:

> On 3/18/21 9:53 AM, Shakeel Butt wrote:
> > On Wed, Mar 17, 2021 at 3:30 PM Jens Axboe <axboe@kernel.dk> wrote:
> >>
> >> On 3/16/21 9:36 AM, Dan Schatzberg wrote:
> >>> No major changes, just rebasing and resubmitting
> >>
> >> Applied for 5.13, thanks.
> >>
> > 
> > I have requested a couple of changes in the patch series. Can this
> > applied series still be changed or new patches are required?
> 
> I have nothing sitting on top of it for now, so as far as I'm concerned
> we can apply a new series instead. Then we can also fold in that fix
> from Colin that he posted this morning...

The collision in memcontrol.c is a pain, but I guess as this is mainly
a loop patch, the block tree is an appropriate route.

Here's the collision between "mm: Charge active memcg when no mm is
set" and Shakeels's
https://lkml.kernel.org/r/20210305212639.775498-1-shakeelb@google.com


--- mm/memcontrol.c
+++ mm/memcontrol.c
@@ -6728,8 +6730,15 @@ int mem_cgroup_charge(struct page *page, struct mm_struct *mm, gfp_t gfp_mask)
 		rcu_read_unlock();
 	}
 
-	if (!memcg)
-		memcg = get_mem_cgroup_from_mm(mm);
+	if (!memcg) {
+		if (!mm) {
+			memcg = get_mem_cgroup_from_current();
+			if (!memcg)
+				memcg = get_mem_cgroup_from_mm(current->mm);
+		} else {
+			memcg = get_mem_cgroup_from_mm(mm);
+		}
+	}
 
 	ret = try_charge(memcg, gfp_mask, nr_pages);
 	if (ret)


Which I resolved thusly:

int mem_cgroup_charge(struct page *page, struct mm_struct *mm, gfp_t gfp_mask)
{
	struct mem_cgroup *memcg;
	int ret;

	if (mem_cgroup_disabled())
		return 0;

	if (!mm) {
		memcg = get_mem_cgroup_from_current();
		(!memcg)
			memcg = get_mem_cgroup_from_mm(current->mm);
	} else {
		memcg = get_mem_cgroup_from_mm(mm);
	}
				
	ret = __mem_cgroup_charge(page, memcg, gfp_mask);
	css_put(&memcg->css);

	return ret;
}




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

* Re: [PATCH v10 0/3] Charge loop device i/o to issuing cgroup
  2021-03-18 23:46       ` Andrew Morton
@ 2021-03-19  0:56         ` Shakeel Butt
  2021-03-19 15:51           ` Dan Schatzberg
  0 siblings, 1 reply; 15+ messages in thread
From: Shakeel Butt @ 2021-03-19  0:56 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jens Axboe, Dan Schatzberg, Tejun Heo, Zefan Li, Johannes Weiner,
	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 Thu, Mar 18, 2021 at 4:46 PM Andrew Morton <akpm@linux-foundation.org> wrote:
>
> On Thu, 18 Mar 2021 10:00:17 -0600 Jens Axboe <axboe@kernel.dk> wrote:
>
> > On 3/18/21 9:53 AM, Shakeel Butt wrote:
> > > On Wed, Mar 17, 2021 at 3:30 PM Jens Axboe <axboe@kernel.dk> wrote:
> > >>
> > >> On 3/16/21 9:36 AM, Dan Schatzberg wrote:
> > >>> No major changes, just rebasing and resubmitting
> > >>
> > >> Applied for 5.13, thanks.
> > >>
> > >
> > > I have requested a couple of changes in the patch series. Can this
> > > applied series still be changed or new patches are required?
> >
> > I have nothing sitting on top of it for now, so as far as I'm concerned
> > we can apply a new series instead. Then we can also fold in that fix
> > from Colin that he posted this morning...
>
> The collision in memcontrol.c is a pain, but I guess as this is mainly
> a loop patch, the block tree is an appropriate route.
>
> Here's the collision between "mm: Charge active memcg when no mm is
> set" and Shakeels's
> https://lkml.kernel.org/r/20210305212639.775498-1-shakeelb@google.com
>
>
> --- mm/memcontrol.c
> +++ mm/memcontrol.c
> @@ -6728,8 +6730,15 @@ int mem_cgroup_charge(struct page *page, struct mm_struct *mm, gfp_t gfp_mask)
>                 rcu_read_unlock();
>         }
>
> -       if (!memcg)
> -               memcg = get_mem_cgroup_from_mm(mm);
> +       if (!memcg) {
> +               if (!mm) {
> +                       memcg = get_mem_cgroup_from_current();
> +                       if (!memcg)
> +                               memcg = get_mem_cgroup_from_mm(current->mm);
> +               } else {
> +                       memcg = get_mem_cgroup_from_mm(mm);
> +               }
> +       }
>
>         ret = try_charge(memcg, gfp_mask, nr_pages);
>         if (ret)
>
>
> Which I resolved thusly:
>
> int mem_cgroup_charge(struct page *page, struct mm_struct *mm, gfp_t gfp_mask)
> {
>         struct mem_cgroup *memcg;
>         int ret;
>
>         if (mem_cgroup_disabled())
>                 return 0;
>
>         if (!mm) {
>                 memcg = get_mem_cgroup_from_current();
>                 (!memcg)
>                         memcg = get_mem_cgroup_from_mm(current->mm);
>         } else {
>                 memcg = get_mem_cgroup_from_mm(mm);
>         }
>
>         ret = __mem_cgroup_charge(page, memcg, gfp_mask);
>         css_put(&memcg->css);
>
>         return ret;
> }
>

We need something similar for mem_cgroup_swapin_charge_page() as well.

It is better to take this series in mm tree and Jens is ok with that [1].

[1] https://lore.kernel.org/linux-next/4fea89a5-0e18-0791-18a8-4c5907b0d2c4@kernel.dk/


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

* Re: [PATCH v10 0/3] Charge loop device i/o to issuing cgroup
  2021-03-19  0:56         ` Shakeel Butt
@ 2021-03-19 15:51           ` Dan Schatzberg
  2021-03-19 16:20             ` Shakeel Butt
  0 siblings, 1 reply; 15+ messages in thread
From: Dan Schatzberg @ 2021-03-19 15:51 UTC (permalink / raw)
  To: Shakeel Butt
  Cc: Andrew Morton, Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner,
	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 Thu, Mar 18, 2021 at 05:56:28PM -0700, Shakeel Butt wrote:
> 
> We need something similar for mem_cgroup_swapin_charge_page() as well.
> 
> It is better to take this series in mm tree and Jens is ok with that [1].
> 
> [1] https://lore.kernel.org/linux-next/4fea89a5-0e18-0791-18a8-4c5907b0d2c4@kernel.dk/

It sounds like there are no concerns about the loop-related work in
the patch series. I'll rebase on the mm tree and resubmit.


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

* Re: [PATCH v10 0/3] Charge loop device i/o to issuing cgroup
  2021-03-19 15:51           ` Dan Schatzberg
@ 2021-03-19 16:20             ` Shakeel Butt
  2021-03-19 16:27               ` Dan Schatzberg
  0 siblings, 1 reply; 15+ messages in thread
From: Shakeel Butt @ 2021-03-19 16:20 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Andrew Morton, Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner,
	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 Fri, Mar 19, 2021 at 8:51 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
>
> On Thu, Mar 18, 2021 at 05:56:28PM -0700, Shakeel Butt wrote:
> >
> > We need something similar for mem_cgroup_swapin_charge_page() as well.
> >
> > It is better to take this series in mm tree and Jens is ok with that [1].
> >
> > [1] https://lore.kernel.org/linux-next/4fea89a5-0e18-0791-18a8-4c5907b0d2c4@kernel.dk/
>
> It sounds like there are no concerns about the loop-related work in
> the patch series. I'll rebase on the mm tree and resubmit.

One suggestion would be to make get_mem_cgroup_from_mm() more generic
(i.e. handle !mm && active_memcg() case) and avoid
get_mem_cgroup_from_current() as it might go away.


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

* Re: [PATCH v10 0/3] Charge loop device i/o to issuing cgroup
  2021-03-19 16:20             ` Shakeel Butt
@ 2021-03-19 16:27               ` Dan Schatzberg
  0 siblings, 0 replies; 15+ messages in thread
From: Dan Schatzberg @ 2021-03-19 16:27 UTC (permalink / raw)
  To: Shakeel Butt
  Cc: Andrew Morton, Jens Axboe, Tejun Heo, Zefan Li, Johannes Weiner,
	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 Fri, Mar 19, 2021 at 09:20:16AM -0700, Shakeel Butt wrote:
> One suggestion would be to make get_mem_cgroup_from_mm() more generic
> (i.e. handle !mm && active_memcg() case) and avoid
> get_mem_cgroup_from_current() as it might go away.

Yeah, that occurred to me as well. I'll take a stab at doing that.


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

end of thread, other threads:[~2021-03-19 16:28 UTC | newest]

Thread overview: 15+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
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 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
2021-03-16 15:36 ` [PATCH 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
2021-03-16 15:50   ` Shakeel Butt
2021-03-16 16:02     ` 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
2021-03-17 22:30 ` [PATCH v10 0/3] Charge loop device i/o to issuing cgroup Jens Axboe
2021-03-18 15:53   ` Shakeel Butt
2021-03-18 16:00     ` Jens Axboe
2021-03-18 23:46       ` Andrew Morton
2021-03-19  0:56         ` Shakeel Butt
2021-03-19 15:51           ` Dan Schatzberg
2021-03-19 16:20             ` Shakeel Butt
2021-03-19 16:27               ` 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).