linux-mm.kvack.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v3 0/3] Charge loop device i/o to issuing cgroup
@ 2020-02-20 16:51 Dan Schatzberg
  2020-02-20 16:51 ` [PATCH v3 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
                   ` (2 more replies)
  0 siblings, 3 replies; 17+ messages in thread
From: Dan Schatzberg @ 2020-02-20 16:51 UTC (permalink / raw)
  Cc: Dan Schatzberg, Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Roman Gushchin, Shakeel Butt, Chris Down, Yang Shi,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

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       | 229 +++++++++++++++++++++++++++++++------
 drivers/block/loop.h       |  14 ++-
 include/linux/memcontrol.h |   6 +
 kernel/cgroup/cgroup.c     |   1 +
 mm/memcontrol.c            |  11 +-
 mm/shmem.c                 |   2 +-
 6 files changed, 217 insertions(+), 46 deletions(-)

-- 
2.17.1



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

* [PATCH v3 1/3] loop: Use worker per cgroup instead of kworker
  2020-02-20 16:51 [PATCH v3 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
@ 2020-02-20 16:51 ` Dan Schatzberg
  2020-02-20 17:50   ` Johannes Weiner
  2020-02-20 22:00   ` Johannes Weiner
  2020-02-20 16:51 ` [PATCH v3 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
  2020-02-20 16:51 ` [PATCH v3 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
  2 siblings, 2 replies; 17+ messages in thread
From: Dan Schatzberg @ 2020-02-20 16:51 UTC (permalink / raw)
  Cc: Dan Schatzberg, Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Roman Gushchin, Shakeel Butt, Chris Down, Yang Shi,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

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

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

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

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

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

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 739b372a5112..78e5005c6742 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);
 
@@ -891,27 +892,101 @@ static void loop_config_discard(struct loop_device *lo)
 
 static void loop_unprepare_queue(struct loop_device *lo)
 {
-	kthread_flush_worker(&lo->worker);
-	kthread_stop(lo->worker_task);
-}
-
-static int loop_kthread_worker_fn(void *worker_ptr)
-{
-	current->flags |= PF_LESS_THROTTLE | PF_MEMALLOC_NOIO;
-	return kthread_worker_fn(worker_ptr);
+	destroy_workqueue(lo->workqueue);
 }
 
 static int loop_prepare_queue(struct loop_device *lo)
 {
-	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))
+	lo->workqueue = alloc_workqueue("loop%d",
+					WQ_UNBOUND | WQ_FREEZABLE |
+					WQ_MEM_RECLAIM,
+					lo->lo_number);
+	if (IS_ERR(lo->workqueue))
 		return -ENOMEM;
-	set_user_nice(lo->worker_task, MIN_NICE);
+
 	return 0;
 }
 
+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 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 void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
+{
+	struct rb_node **node = &(lo->worker_tree.rb_node), *parent = NULL;
+	struct loop_worker *cur_worker, *worker = NULL;
+	struct work_struct *work;
+	struct list_head *cmd_list;
+
+	spin_lock_irq(&lo->lo_lock);
+
+	if (!cmd->css)
+		goto queue_work;
+
+	node = &(lo->worker_tree.rb_node);
+
+	while (*node) {
+		parent = *node;
+		cur_worker = container_of(*node, struct loop_worker, rb_node);
+		if ((long)cur_worker->css == (long)cmd->css) {
+			worker = cur_worker;
+			break;
+		} else if ((long)cur_worker->css < (long)cmd->css) {
+			node = &((*node)->rb_left);
+		} else {
+			node = &((*node)->rb_right);
+		}
+	}
+	if (worker)
+		goto queue_work;
+
+	worker = kzalloc(sizeof(struct loop_worker),
+			GFP_NOWAIT | __GFP_NOWARN);
+	/*
+	 * In the event we cannot allocate a worker, just queue on the
+	 * rootcg worker
+	 */
+	if (!worker)
+		goto queue_work;
+
+	worker->css = cmd->css;
+	css_get(worker->css);
+	INIT_WORK(&worker->work, loop_workfn);
+	INIT_LIST_HEAD(&worker->cmd_list);
+	INIT_LIST_HEAD(&worker->idle_list);
+	worker->lo = lo;
+	rb_link_node(&worker->rb_node, parent, node);
+	rb_insert_color(&worker->rb_node, &lo->worker_tree);
+queue_work:
+	if (worker) {
+		/*
+		 * We need to remove from the idle list here while
+		 * holding the lock so that the idle timer doesn't
+		 * free the worker
+		 */
+		if (!list_empty(&worker->idle_list))
+			list_del_init(&worker->idle_list);
+		work = &worker->work;
+		cmd_list = &worker->cmd_list;
+	} else {
+		work = &lo->rootcg_work;
+		cmd_list = &lo->rootcg_cmd_list;
+	}
+	list_add_tail(&cmd->list_entry, cmd_list);
+	queue_work(lo->workqueue, work);
+	spin_unlock_irq(&lo->lo_lock);
+}
+
 static void loop_update_rotational(struct loop_device *lo)
 {
 	struct file *file = lo->lo_backing_file;
@@ -993,6 +1068,12 @@ static int loop_set_fd(struct loop_device *lo, fmode_t mode,
 
 	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;
@@ -1155,6 +1236,7 @@ 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);
+	del_timer_sync(&lo->timer);
 out_unlock:
 	mutex_unlock(&loop_ctl_mutex);
 	if (partscan) {
@@ -1932,7 +2014,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;
 }
@@ -1958,26 +2040,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)
 {
-	struct loop_cmd *cmd =
-		container_of(work, struct loop_cmd, work);
+	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;
+	while (1) {
+		spin_lock_irq(&lo->lo_lock);
+		if (list_empty(cmd_list))
+			break;
+
+		cmd = container_of(
+			cmd_list->next, struct loop_cmd, list_entry);
+		list_del(cmd_list->next);
+		spin_unlock_irq(&lo->lo_lock);
+		loop_handle_cmd(cmd);
+		cond_resched();
+	}
 
-	loop_handle_cmd(cmd);
+	/*
+	 * We only add to the idle list if there are no pending cmds
+	 * *and* the worker will not run again which ensures that it
+	 * is safe to free any worker on the idle list
+	 */
+	if (worker && !work_pending(&worker->work)) {
+		worker->last_ran_at = jiffies;
+		list_add_tail(&worker->idle_list, &lo->idle_worker_list);
+		loop_set_timer(lo);
+	}
+	spin_unlock_irq(&lo->lo_lock);
+	current->flags = orig_flags;
 }
 
-static int loop_init_request(struct blk_mq_tag_set *set, struct request *rq,
-		unsigned int hctx_idx, unsigned int numa_node)
+static void loop_workfn(struct work_struct *work)
 {
-	struct loop_cmd *cmd = blk_mq_rq_to_pdu(rq);
+	struct loop_worker *worker =
+		container_of(work, struct loop_worker, work);
+	loop_process_work(worker, &worker->cmd_list, worker->lo);
+}
 
-	kthread_init_work(&cmd->work, loop_queue_work);
-	return 0;
+static void loop_rootcg_workfn(struct work_struct *work)
+{
+	struct loop_device *lo =
+		container_of(work, struct loop_device, rootcg_work);
+	loop_process_work(NULL, &lo->rootcg_cmd_list, lo);
+}
+
+static void loop_free_idle_workers(struct timer_list *timer)
+{
+	struct loop_device *lo = container_of(timer, struct loop_device, timer);
+	struct loop_worker *pos, *worker;
+
+	spin_lock_irq(&lo->lo_lock);
+	list_for_each_entry_safe(worker, pos, &lo->idle_worker_list,
+				idle_list) {
+		if (time_is_after_jiffies(worker->last_ran_at +
+						LOOP_IDLE_WORKER_TIMEOUT))
+			break;
+		list_del(&worker->idle_list);
+		rb_erase(&worker->rb_node, &lo->worker_tree);
+		css_put(worker->css);
+		kfree(worker);
+	}
+	if (!list_empty(&lo->idle_worker_list))
+		loop_set_timer(lo);
+	spin_unlock_irq(&lo->lo_lock);
 }
 
 static const struct blk_mq_ops loop_mq_ops = {
 	.queue_rq       = loop_queue_rq,
-	.init_request	= loop_init_request,
 	.complete	= lo_complete_rq,
 };
 
diff --git a/drivers/block/loop.h b/drivers/block/loop.h
index af75a5ee4094..87fd0e372227 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -14,7 +14,6 @@
 #include <linux/blk-mq.h>
 #include <linux/spinlock.h>
 #include <linux/mutex.h>
-#include <linux/kthread.h>
 #include <uapi/linux/loop.h>
 
 /* Possible states of device */
@@ -54,8 +53,12 @@ struct loop_device {
 
 	spinlock_t		lo_lock;
 	int			lo_state;
-	struct kthread_worker	worker;
-	struct task_struct	*worker_task;
+	struct workqueue_struct *workqueue;
+	struct work_struct      rootcg_work;
+	struct list_head        rootcg_cmd_list;
+	struct list_head        idle_worker_list;
+	struct rb_root          worker_tree;
+	struct timer_list       timer;
 	bool			use_dio;
 	bool			sysfs_inited;
 
@@ -65,7 +68,7 @@ struct loop_device {
 };
 
 struct loop_cmd {
-	struct kthread_work work;
+	struct list_head list_entry;
 	bool use_aio; /* use AIO interface to handle I/O */
 	atomic_t ref; /* only for aio */
 	long ret;
-- 
2.17.1



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

* [PATCH v3 2/3] mm: Charge active memcg when no mm is set
  2020-02-20 16:51 [PATCH v3 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
  2020-02-20 16:51 ` [PATCH v3 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
@ 2020-02-20 16:51 ` Dan Schatzberg
  2020-02-20 18:14   ` Shakeel Butt
                     ` (3 more replies)
  2020-02-20 16:51 ` [PATCH v3 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
  2 siblings, 4 replies; 17+ messages in thread
From: Dan Schatzberg @ 2020-02-20 16:51 UTC (permalink / raw)
  Cc: Dan Schatzberg, Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Roman Gushchin, Shakeel Butt, Chris Down, Yang Shi,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	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>
---
 mm/memcontrol.c | 11 ++++++++---
 mm/shmem.c      |  2 +-
 2 files changed, 9 insertions(+), 4 deletions(-)

diff --git a/mm/memcontrol.c b/mm/memcontrol.c
index 6f6dc8712e39..b174aff4f069 100644
--- a/mm/memcontrol.c
+++ b/mm/memcontrol.c
@@ -6317,7 +6317,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.
@@ -6361,8 +6362,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 c8f7540ef048..7c7f5acf89d6 100644
--- a/mm/shmem.c
+++ b/mm/shmem.c
@@ -1766,7 +1766,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.17.1



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

* [PATCH v3 3/3] loop: Charge i/o to mem and blk cg
  2020-02-20 16:51 [PATCH v3 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
  2020-02-20 16:51 ` [PATCH v3 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
  2020-02-20 16:51 ` [PATCH v3 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
@ 2020-02-20 16:51 ` Dan Schatzberg
  2020-02-20 22:02   ` Johannes Weiner
  2 siblings, 1 reply; 17+ messages in thread
From: Dan Schatzberg @ 2020-02-20 16:51 UTC (permalink / raw)
  Cc: Dan Schatzberg, Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Roman Gushchin, Shakeel Butt, Chris Down, Yang Shi,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

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

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

Signed-off-by: Dan Schatzberg <schatzberg.dan@gmail.com>
---
 drivers/block/loop.c       | 59 ++++++++++++++++++++++++--------------
 drivers/block/loop.h       |  3 +-
 include/linux/memcontrol.h |  6 ++++
 kernel/cgroup/cgroup.c     |  1 +
 4 files changed, 47 insertions(+), 22 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 78e5005c6742..cc091a66b0d0 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"
 
@@ -134,7 +135,7 @@ static struct loop_func_table xor_funcs = {
 	.number = LO_CRYPT_XOR,
 	.transfer = transfer_xor,
 	.init = xor_init
-}; 
+};
 
 /* xfer_funcs[0] is special - its release function is never called */
 static struct loop_func_table *xfer_funcs[MAX_LO_CRYPT] = {
@@ -504,8 +505,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);
 }
@@ -566,8 +565,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);
@@ -575,7 +572,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);
@@ -913,7 +909,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;
 };
 
@@ -930,7 +926,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);
@@ -938,10 +934,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 ((long)cur_worker->css == (long)cmd->css) {
+		if ((long)cur_worker->blkcg_css == (long)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);
@@ -954,13 +950,16 @@ static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
 			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);
@@ -2007,13 +2006,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;
@@ -2031,8 +2035,21 @@ static void loop_handle_cmd(struct loop_cmd *cmd)
 		goto failed;
 	}
 
+	if (cmd->blkcg_css)
+		kthread_associate_blkcg(cmd->blkcg_css);
+	if (cmd->memcg_css)
+		memalloc_use_memcg(mem_cgroup_from_css(cmd->memcg_css));
+
 	ret = do_req_filebacked(lo, rq);
- failed:
+
+	if (cmd->blkcg_css)
+		kthread_associate_blkcg(NULL);
+
+	if (cmd->memcg_css) {
+		memalloc_unuse_memcg();
+		css_put(cmd->memcg_css);
+	}
+failed:
 	/* complete non-aio request */
 	if (!cmd->use_aio || ret) {
 		cmd->ret = ret ? -EIO : 0;
@@ -2106,7 +2123,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 a7a0a1a5c8d5..aeb51f2ded46 100644
--- a/include/linux/memcontrol.h
+++ b/include/linux/memcontrol.h
@@ -922,6 +922,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 75f687301bbf..c3896c2e0942 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.17.1



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

* Re: [PATCH v3 1/3] loop: Use worker per cgroup instead of kworker
  2020-02-20 16:51 ` [PATCH v3 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
@ 2020-02-20 17:50   ` Johannes Weiner
  2020-02-20 22:00   ` Johannes Weiner
  1 sibling, 0 replies; 17+ messages in thread
From: Johannes Weiner @ 2020-02-20 17:50 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Li Zefan, Michal Hocko, Vladimir Davydov,
	Andrew Morton, Hugh Dickins, Roman Gushchin, Shakeel Butt,
	Chris Down, Yang Shi, Thomas Gleixner, open list:BLOCK LAYER,
	open list, open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

Hello Dan,

On Thu, Feb 20, 2020 at 11:51:51AM -0500, Dan Schatzberg wrote:
> +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;
> +	while (1) {
> +		spin_lock_irq(&lo->lo_lock);
> +		if (list_empty(cmd_list))
> +			break;
> +
> +		cmd = container_of(
> +			cmd_list->next, struct loop_cmd, list_entry);
> +		list_del(cmd_list->next);
> +		spin_unlock_irq(&lo->lo_lock);
> +		loop_handle_cmd(cmd);
> +		cond_resched();
> +	}

The loop structure tripped me up, because it's not immediately obvious
that the lock will be held coming out. How about the following to make
the lock section stand out visually?

	spin_lock_irq(&lo->lo_lock);
	while (!list_empty(cmd_list)) {
		cmd = container_of(cmd_list->next, struct loop_cmd, list_entry);
		list_del(&cmd->list_entry);
		spin_unlock_irq(&lo->lo_lock);		

		loop_handle_cmd(cmd);
		cond_resched();

		spin_lock_irq(&lo->lo_lock);
	}

> -	loop_handle_cmd(cmd);
> +	/*
> +	 * We only add to the idle list if there are no pending cmds
> +	 * *and* the worker will not run again which ensures that it
> +	 * is safe to free any worker on the idle list
> +	 */
> +	if (worker && !work_pending(&worker->work)) {
> +		worker->last_ran_at = jiffies;
> +		list_add_tail(&worker->idle_list, &lo->idle_worker_list);
> +		loop_set_timer(lo);
> +	}
> +	spin_unlock_irq(&lo->lo_lock);
> +	current->flags = orig_flags;


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

* Re: [PATCH v3 2/3] mm: Charge active memcg when no mm is set
  2020-02-20 16:51 ` [PATCH v3 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
@ 2020-02-20 18:14   ` Shakeel Butt
  2020-02-20 21:03     ` Johannes Weiner
  2020-02-20 18:35   ` Chris Down
                     ` (2 subsequent siblings)
  3 siblings, 1 reply; 17+ messages in thread
From: Shakeel Butt @ 2020-02-20 18:14 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Chris Down, Yang Shi, Thomas Gleixner, open list:BLOCK LAYER,
	open list, open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Thu, Feb 20, 2020 at 8:52 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
>
> 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.

What if css_tryget_online(current->active_memcg) in
get_mem_cgroup_from_current() fails? Do we want to change this to
css_tryget() and even if that fails should we fallback to
root_mem_cgroup or current->mm->memcg?

> 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>
> ---
>  mm/memcontrol.c | 11 ++++++++---
>  mm/shmem.c      |  2 +-
>  2 files changed, 9 insertions(+), 4 deletions(-)
>
> diff --git a/mm/memcontrol.c b/mm/memcontrol.c
> index 6f6dc8712e39..b174aff4f069 100644
> --- a/mm/memcontrol.c
> +++ b/mm/memcontrol.c
> @@ -6317,7 +6317,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.
> @@ -6361,8 +6362,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 c8f7540ef048..7c7f5acf89d6 100644
> --- a/mm/shmem.c
> +++ b/mm/shmem.c
> @@ -1766,7 +1766,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.17.1
>


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

* Re: [PATCH v3 2/3] mm: Charge active memcg when no mm is set
  2020-02-20 16:51 ` [PATCH v3 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
  2020-02-20 18:14   ` Shakeel Butt
@ 2020-02-20 18:35   ` Chris Down
  2020-02-20 21:15   ` Shakeel Butt
  2020-02-23 19:08   ` Hugh Dickins
  3 siblings, 0 replies; 17+ messages in thread
From: Chris Down @ 2020-02-20 18:35 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Shakeel Butt, Yang Shi, Thomas Gleixner, open list:BLOCK LAYER,
	open list, open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

Dan Schatzberg writes:
>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>

Thanks! The clarification the v2 thread for this made things clear to me.


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

* Re: [PATCH v3 2/3] mm: Charge active memcg when no mm is set
  2020-02-20 18:14   ` Shakeel Butt
@ 2020-02-20 21:03     ` Johannes Weiner
  2020-02-20 21:14       ` Shakeel Butt
  0 siblings, 1 reply; 17+ messages in thread
From: Johannes Weiner @ 2020-02-20 21:03 UTC (permalink / raw)
  To: Shakeel Butt
  Cc: Dan Schatzberg, Jens Axboe, Tejun Heo, Li Zefan, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Chris Down, Yang Shi, Thomas Gleixner, open list:BLOCK LAYER,
	open list, open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

Hey Shakeel!

On Thu, Feb 20, 2020 at 10:14:45AM -0800, Shakeel Butt wrote:
> On Thu, Feb 20, 2020 at 8:52 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
> >
> > 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.
> 
> What if css_tryget_online(current->active_memcg) in
> get_mem_cgroup_from_current() fails? Do we want to change this to
> css_tryget() and even if that fails should we fallback to
> root_mem_cgroup or current->mm->memcg?

Good questions.

I think we can switch to css_tryget(). If a cgroup goes offline
between issuing the IO and the loop layer executing that IO, the
resources used could end up in the root instead of the closest
ancestor of the offlined group. However, the risk of that actually
happening and causing problems is probably pretty small, and the
behavior isn't really worse than before Dan's patches.

Would you mind sending a separate patch for this? AFAICS similar
concerns apply to all users of foreign charging.

As for tryget failing: can that actually happen? AFAICS, all current
users acquire a reference first (get_memcg_from_somewhere()) that they
assign to current->active_memcg. We should probably codify this rule
and do WARN_ON(!css_tryget()) /* current->active_memcg must hold a ref */


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

* Re: [PATCH v3 2/3] mm: Charge active memcg when no mm is set
  2020-02-20 21:03     ` Johannes Weiner
@ 2020-02-20 21:14       ` Shakeel Butt
  0 siblings, 0 replies; 17+ messages in thread
From: Shakeel Butt @ 2020-02-20 21:14 UTC (permalink / raw)
  To: Johannes Weiner
  Cc: Dan Schatzberg, Jens Axboe, Tejun Heo, Li Zefan, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Chris Down, Yang Shi, Thomas Gleixner, open list:BLOCK LAYER,
	open list, open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

Hi Johannes,

On Thu, Feb 20, 2020 at 1:03 PM Johannes Weiner <hannes@cmpxchg.org> wrote:
>
> Hey Shakeel!
>
> On Thu, Feb 20, 2020 at 10:14:45AM -0800, Shakeel Butt wrote:
> > On Thu, Feb 20, 2020 at 8:52 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
> > >
> > > 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.
> >
> > What if css_tryget_online(current->active_memcg) in
> > get_mem_cgroup_from_current() fails? Do we want to change this to
> > css_tryget() and even if that fails should we fallback to
> > root_mem_cgroup or current->mm->memcg?
>
> Good questions.
>
> I think we can switch to css_tryget(). If a cgroup goes offline
> between issuing the IO and the loop layer executing that IO, the
> resources used could end up in the root instead of the closest
> ancestor of the offlined group. However, the risk of that actually
> happening and causing problems is probably pretty small, and the
> behavior isn't really worse than before Dan's patches.

Agreed.

>
> Would you mind sending a separate patch for this? AFAICS similar
> concerns apply to all users of foreign charging.

Sure and yes similar concerns apply to other users as well.

>
> As for tryget failing: can that actually happen? AFAICS, all current
> users acquire a reference first (get_memcg_from_somewhere()) that they
> assign to current->active_memcg. We should probably codify this rule
> and do WARN_ON(!css_tryget()) /* current->active_memcg must hold a ref */

Yes, we should WARN_ON().

Shakeel


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

* Re: [PATCH v3 2/3] mm: Charge active memcg when no mm is set
  2020-02-20 16:51 ` [PATCH v3 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
  2020-02-20 18:14   ` Shakeel Butt
  2020-02-20 18:35   ` Chris Down
@ 2020-02-20 21:15   ` Shakeel Butt
  2020-02-23 19:08   ` Hugh Dickins
  3 siblings, 0 replies; 17+ messages in thread
From: Shakeel Butt @ 2020-02-20 21:15 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Chris Down, Yang Shi, Thomas Gleixner, open list:BLOCK LAYER,
	open list, open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Thu, Feb 20, 2020 at 8:52 AM Dan Schatzberg <schatzberg.dan@gmail.com> wrote:
>
> 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>

Reviewed-by: Shakeel Butt <shakeelb@google.com>


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

* Re: [PATCH v3 1/3] loop: Use worker per cgroup instead of kworker
  2020-02-20 16:51 ` [PATCH v3 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
  2020-02-20 17:50   ` Johannes Weiner
@ 2020-02-20 22:00   ` Johannes Weiner
  1 sibling, 0 replies; 17+ messages in thread
From: Johannes Weiner @ 2020-02-20 22:00 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Li Zefan, Michal Hocko, Vladimir Davydov,
	Andrew Morton, Hugh Dickins, Roman Gushchin, Shakeel Butt,
	Chris Down, Yang Shi, Thomas Gleixner, open list:BLOCK LAYER,
	open list, open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Thu, Feb 20, 2020 at 11:51:51AM -0500, Dan Schatzberg wrote:
> Existing uses of loop device may have multiple cgroups reading/writing
> to the same device. Simply charging resources for I/O to the backing
> file could result in priority inversion where one cgroup gets
> synchronously blocked, holding up all other I/O to the loop device.
> 
> In order to avoid this priority inversion, we use a single workqueue
> where each work item is a "struct loop_worker" which contains a queue of
> struct loop_cmds to issue. The loop device maintains a tree mapping blk
> css_id -> loop_worker. This allows each cgroup to independently make
> forward progress issuing I/O to the backing file.
> 
> There is also a single queue for I/O associated with the rootcg which
> can be used in cases of extreme memory shortage where we cannot allocate
> a loop_worker.
> 
> The locking for the tree and queues is fairly heavy handed - we acquire
> the per-loop-device spinlock any time either is accessed. The existing
> implementation serializes all I/O through a single thread anyways, so I
> don't believe this is any worse.
> 
> Signed-off-by: Dan Schatzberg <schatzberg.dan@gmail.com>

FWIW, this looks good to me, please feel free to include:

Acked-by: Johannes Weiner <hannes@cmpxchg.org>

I have only some minor style nitpicks (along with the other email I
sent earlier on this patch), that would be nice to get fixed:

> +static void loop_queue_work(struct loop_device *lo, struct loop_cmd *cmd)
> +{
> +	struct rb_node **node = &(lo->worker_tree.rb_node), *parent = NULL;
> +	struct loop_worker *cur_worker, *worker = NULL;
> +	struct work_struct *work;
> +	struct list_head *cmd_list;
> +
> +	spin_lock_irq(&lo->lo_lock);
> +
> +	if (!cmd->css)
> +		goto queue_work;
> +
> +	node = &(lo->worker_tree.rb_node);

-> and . are > &, the parentheses aren't necessary.

> +	while (*node) {
> +		parent = *node;
> +		cur_worker = container_of(*node, struct loop_worker, rb_node);
> +		if ((long)cur_worker->css == (long)cmd->css) {

The casts aren't necessary, but they made me doubt myself and look up
the types. I wouldn't add them just to be symmetrical with the other
arm of the branch.

> +			worker = cur_worker;
> +			break;
> +		} else if ((long)cur_worker->css < (long)cmd->css) {
> +			node = &((*node)->rb_left);
> +		} else {
> +			node = &((*node)->rb_right);

The outer parentheses aren't necessary.

> +		}
> +	}
> +	if (worker)
> +		goto queue_work;
> +
> +	worker = kzalloc(sizeof(struct loop_worker), 
> +			GFP_NOWAIT | __GFP_NOWARN);

This fits on an 80 character line.


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

* Re: [PATCH v3 3/3] loop: Charge i/o to mem and blk cg
  2020-02-20 16:51 ` [PATCH v3 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
@ 2020-02-20 22:02   ` Johannes Weiner
  0 siblings, 0 replies; 17+ messages in thread
From: Johannes Weiner @ 2020-02-20 22:02 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Li Zefan, Michal Hocko, Vladimir Davydov,
	Andrew Morton, Hugh Dickins, Roman Gushchin, Shakeel Butt,
	Chris Down, Yang Shi, Thomas Gleixner, open list:BLOCK LAYER,
	open list, open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Thu, Feb 20, 2020 at 11:51:53AM -0500, Dan Schatzberg wrote:
> The current code only associates with the existing blkcg when aio is
> used to access the backing file. This patch covers all types of i/o to
> the backing file and also associates the memcg so if the backing file is
> on tmpfs, memory is charged appropriately.
> 
> This patch also exports cgroup_get_e_css so it can be used by the loop
> module.
> 
> Signed-off-by: Dan Schatzberg <schatzberg.dan@gmail.com>

The cgroup bits look good to me.

Acked-by: Johannes Weiner <hannes@cmpxchg.org>


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

* Re: [PATCH v3 2/3] mm: Charge active memcg when no mm is set
  2020-02-20 16:51 ` [PATCH v3 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
                     ` (2 preceding siblings ...)
  2020-02-20 21:15   ` Shakeel Butt
@ 2020-02-23 19:08   ` Hugh Dickins
  2020-02-24  1:11     ` Hugh Dickins
  3 siblings, 1 reply; 17+ messages in thread
From: Hugh Dickins @ 2020-02-23 19:08 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Shakeel Butt, Chris Down, Yang Shi, Thomas Gleixner,
	open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Thu, 20 Feb 2020, Dan Schatzberg wrote:

> 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: Hugh Dickins <hughd@google.com>

Yes, internally we have some further not-yet-upstreamed complications
here (mainly, the "memcg=" mount option for all charges on a tmpfs to
be charged to that memcg); but what you're doing here does not obstruct
adding that later, they fit in well with the hierarchy that you (and
Johannes) mapped out above, and it's really an improvement for shmem
not to be referring to current there - thanks.

> ---
>  mm/memcontrol.c | 11 ++++++++---
>  mm/shmem.c      |  2 +-
>  2 files changed, 9 insertions(+), 4 deletions(-)
> 
> diff --git a/mm/memcontrol.c b/mm/memcontrol.c
> index 6f6dc8712e39..b174aff4f069 100644
> --- a/mm/memcontrol.c
> +++ b/mm/memcontrol.c
> @@ -6317,7 +6317,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.
> @@ -6361,8 +6362,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 c8f7540ef048..7c7f5acf89d6 100644
> --- a/mm/shmem.c
> +++ b/mm/shmem.c
> @@ -1766,7 +1766,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.17.1


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

* Re: [PATCH v3 2/3] mm: Charge active memcg when no mm is set
  2020-02-23 19:08   ` Hugh Dickins
@ 2020-02-24  1:11     ` Hugh Dickins
  2020-02-24 21:37       ` Dan Schatzberg
  0 siblings, 1 reply; 17+ messages in thread
From: Hugh Dickins @ 2020-02-24  1:11 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Hugh Dickins, Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Roman Gushchin,
	Shakeel Butt, Chris Down, Yang Shi, Thomas Gleixner,
	open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Sun, 23 Feb 2020, Hugh Dickins wrote:
> On Thu, 20 Feb 2020, Dan Schatzberg wrote:
> 
> > 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: Hugh Dickins <hughd@google.com>
> 
> Yes, internally we have some further not-yet-upstreamed complications
> here (mainly, the "memcg=" mount option for all charges on a tmpfs to
> be charged to that memcg); but what you're doing here does not obstruct
> adding that later, they fit in well with the hierarchy that you (and
> Johannes) mapped out above, and it's really an improvement for shmem
> not to be referring to current there - thanks.

I acked slightly too soon. There are two other uses of "try_charge" in
mm/shmem.c: we can be confident that the userfaultfd one knows what mm
it's dealing with, but the shmem_swapin_page() instance has a similar
use of current->mm, that you also want to adjust to NULL, don't you?

Hugh


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

* Re: [PATCH v3 2/3] mm: Charge active memcg when no mm is set
  2020-02-24  1:11     ` Hugh Dickins
@ 2020-02-24 21:37       ` Dan Schatzberg
  0 siblings, 0 replies; 17+ messages in thread
From: Dan Schatzberg @ 2020-02-24 21:37 UTC (permalink / raw)
  To: Hugh Dickins
  Cc: Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Roman Gushchin, Shakeel Butt,
	Chris Down, Yang Shi, Thomas Gleixner, open list:BLOCK LAYER,
	open list, open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Sun, Feb 23, 2020 at 05:11:12PM -0800, Hugh Dickins wrote:
> On Sun, 23 Feb 2020, Hugh Dickins wrote:
> > On Thu, 20 Feb 2020, Dan Schatzberg wrote:
> > 
> > > 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: Hugh Dickins <hughd@google.com>
> > 
> > Yes, internally we have some further not-yet-upstreamed complications
> > here (mainly, the "memcg=" mount option for all charges on a tmpfs to
> > be charged to that memcg); but what you're doing here does not obstruct
> > adding that later, they fit in well with the hierarchy that you (and
> > Johannes) mapped out above, and it's really an improvement for shmem
> > not to be referring to current there - thanks.
> 
> I acked slightly too soon. There are two other uses of "try_charge" in
> mm/shmem.c: we can be confident that the userfaultfd one knows what mm
> it's dealing with, but the shmem_swapin_page() instance has a similar
> use of current->mm, that you also want to adjust to NULL, don't you?
> 
> Hugh

Yes, you're right. I'll change shmem_swapin_page as well


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

* Re: [PATCH v3 2/3] mm: Charge active memcg when no mm is set
  2020-02-24 22:17 ` [PATCH v3 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
@ 2020-02-24 22:26   ` Hugh Dickins
  0 siblings, 0 replies; 17+ messages in thread
From: Hugh Dickins @ 2020-02-24 22:26 UTC (permalink / raw)
  To: Dan Schatzberg
  Cc: Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner, Michal Hocko,
	Vladimir Davydov, Andrew Morton, Hugh Dickins, Roman Gushchin,
	Shakeel Butt, Chris Down, Yang Shi, Thomas Gleixner,
	open list:BLOCK LAYER, open list,
	open list:CONTROL GROUP (CGROUP),
	open list:CONTROL GROUP - MEMORY RESOURCE CONTROLLER (MEMCG)

On Mon, 24 Feb 2020, Dan Schatzberg wrote:

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

Acked-by: Hugh Dickins <hughd@google.com>

> Reviewed-by: Shakeel Butt <shakeelb@google.com>
> Signed-off-by: Dan Schatzberg <schatzberg.dan@gmail.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 d09776cd6e10..222e4aac0c85 100644
> --- a/mm/memcontrol.c
> +++ b/mm/memcontrol.c
> @@ -6319,7 +6319,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.
> @@ -6363,8 +6364,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 c8f7540ef048..8664c97851f2 100644
> --- a/mm/shmem.c
> +++ b/mm/shmem.c
> @@ -1631,7 +1631,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;
> @@ -1766,7 +1766,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.17.1


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

* [PATCH v3 2/3] mm: Charge active memcg when no mm is set
  2020-02-24 22:17 [PATCH v3 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
@ 2020-02-24 22:17 ` Dan Schatzberg
  2020-02-24 22:26   ` Hugh Dickins
  0 siblings, 1 reply; 17+ messages in thread
From: Dan Schatzberg @ 2020-02-24 22:17 UTC (permalink / raw)
  Cc: Dan Schatzberg, Jens Axboe, Tejun Heo, Li Zefan, Johannes Weiner,
	Michal Hocko, Vladimir Davydov, Andrew Morton, Hugh Dickins,
	Roman Gushchin, Shakeel Butt, Chris Down, Yang Shi,
	Thomas Gleixner, open list:BLOCK LAYER, open list,
	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>
Signed-off-by: Dan Schatzberg <schatzberg.dan@gmail.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 d09776cd6e10..222e4aac0c85 100644
--- a/mm/memcontrol.c
+++ b/mm/memcontrol.c
@@ -6319,7 +6319,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.
@@ -6363,8 +6364,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 c8f7540ef048..8664c97851f2 100644
--- a/mm/shmem.c
+++ b/mm/shmem.c
@@ -1631,7 +1631,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;
@@ -1766,7 +1766,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.17.1



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

end of thread, other threads:[~2020-02-24 22:27 UTC | newest]

Thread overview: 17+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-02-20 16:51 [PATCH v3 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
2020-02-20 16:51 ` [PATCH v3 1/3] loop: Use worker per cgroup instead of kworker Dan Schatzberg
2020-02-20 17:50   ` Johannes Weiner
2020-02-20 22:00   ` Johannes Weiner
2020-02-20 16:51 ` [PATCH v3 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
2020-02-20 18:14   ` Shakeel Butt
2020-02-20 21:03     ` Johannes Weiner
2020-02-20 21:14       ` Shakeel Butt
2020-02-20 18:35   ` Chris Down
2020-02-20 21:15   ` Shakeel Butt
2020-02-23 19:08   ` Hugh Dickins
2020-02-24  1:11     ` Hugh Dickins
2020-02-24 21:37       ` Dan Schatzberg
2020-02-20 16:51 ` [PATCH v3 3/3] loop: Charge i/o to mem and blk cg Dan Schatzberg
2020-02-20 22:02   ` Johannes Weiner
2020-02-24 22:17 [PATCH v3 0/3] Charge loop device i/o to issuing cgroup Dan Schatzberg
2020-02-24 22:17 ` [PATCH v3 2/3] mm: Charge active memcg when no mm is set Dan Schatzberg
2020-02-24 22:26   ` Hugh Dickins

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