2019-02-08 20:11:39

by J. Bruce Fields

[permalink] [raw]
Subject: [PATCH 0/7] Eliminate delegation self-conflicts

From: "J. Bruce Fields" <[email protected]>

These patches allow NFSv4 clients holding delegations to keep them when
the operation that would break a delegation comes from the same client.

To do that, we somehow need to pass the identity of the
delegation-breaker down through the VFS.

This series uses the tgid, a solution suggested by Trond. To do that we
need nfsd tasks to share the same tgid. I do that by extending the
kthread code slightly to allow knfsd to run the kthreadd main loop in a
task of its own, and spawn its server threads off of that task.

Part of Trond's thinking was that this would work for userspace too.
Delegations are currently only available to knfsd, but Ganesha and Samba
may eventually be interested in a userspace interface (probably a minor
variation on the fcntl F_{GET,SET}LEASE interface). A threaded
userspace server would first resolve conflicts between its own clients,
and then call into the kernel to break any leases acquired by other
processes. That may require some careful locking of the server's own
data structures, but it should work.

Previously I considered instead adding a new field somewhere in the
struct task. That might require a new system call to expose to user
space. Or we might be able to put this in a keyring, if David Howells
thought that would work.

Before that I tried passing the identity of the breaker explicitly, but
that looks like it would require passing the new argument around to huge
swaths of the VFS.

I'm testing this with some a locally modified pynfs; I'll fix that up
and push it out at some point, but pynfs has a number of bugs in this
area.

I wasn't sure who to ask about the kthread.c changes, so I'm cc'ing a
random assortment of developers in recent changelogs, hope that's OK.

--b.

J. Bruce Fields (7):
kthreads: minor kthreadd refactoring
kthreads: Simplify tsk_fork_get_node
kthreads: allow multiple kthreadd's
kthreads: allow cloning threads with different flags
rpc: separate out body of svc_start_kthreads
rpc: move rpc server threads into their own thread group
nfsd: ignore delegation self-conflicts

fs/locks.c | 39 +++++++++++
fs/nfsd/nfs4state.c | 61 ++++++++++++++++
fs/nfsd/state.h | 2 +
fs/nfsd/vfs.c | 32 +++++++--
include/linux/fs.h | 2 +
include/linux/kthread.h | 21 +++++-
include/linux/sunrpc/svc.h | 1 +
init/init_task.c | 3 +
init/main.c | 4 +-
kernel/fork.c | 4 ++
kernel/kthread.c | 140 +++++++++++++++++++++++++++----------
net/sunrpc/svc.c | 83 ++++++++++++++--------
12 files changed, 317 insertions(+), 75 deletions(-)

--
2.20.1



2019-02-08 20:11:28

by J. Bruce Fields

[permalink] [raw]
Subject: [PATCH 4/7] kthreads: allow cloning threads with different flags

From: "J. Bruce Fields" <[email protected]>

This is so knfsd can add CLONE_THREAD.

Signed-off-by: J. Bruce Fields <[email protected]>
---
include/linux/kthread.h | 3 ++-
kernel/kthread.c | 11 +++++++----
2 files changed, 9 insertions(+), 5 deletions(-)

diff --git a/include/linux/kthread.h b/include/linux/kthread.h
index 10b5836dfb2a..988aa77f9811 100644
--- a/include/linux/kthread.h
+++ b/include/linux/kthread.h
@@ -11,11 +11,12 @@ struct kthread_group {
spinlock_t create_lock;
struct list_head create_list;
struct task_struct *task;
+ unsigned long flags;
};

extern struct kthread_group kthreadd_default;

-struct kthread_group *kthread_start_group(char *);
+struct kthread_group *kthread_start_group(unsigned long, char *);
void kthread_stop_group(struct kthread_group *);

struct task_struct *kthread_group_create_on_node(struct kthread_group *,
diff --git a/kernel/kthread.c b/kernel/kthread.c
index b515557b98c9..e24ef5afa651 100644
--- a/kernel/kthread.c
+++ b/kernel/kthread.c
@@ -26,6 +26,7 @@ struct kthread_group kthreadd_default = {
.name = "kthreadd",
.create_lock = __SPIN_LOCK_UNLOCKED(kthreadd_default.create_lock),
.create_list = LIST_HEAD_INIT(kthreadd_default.create_list),
+ .flags = CLONE_FS | CLONE_FILES | SIGCHLD,
};

void wake_kthreadd(struct kthread_group *kg)
@@ -33,7 +34,7 @@ void wake_kthreadd(struct kthread_group *kg)
wake_up_process(kg->task);
}

-struct kthread_group *kthread_start_group(char *name)
+struct kthread_group *kthread_start_group(unsigned long flags, char *name)
{
struct kthread_group *new;
struct task_struct *task;
@@ -44,6 +45,7 @@ struct kthread_group *kthread_start_group(char *name)
spin_lock_init(&new->create_lock);
INIT_LIST_HEAD(&new->create_list);
new->name = name;
+ new->flags = flags;
task = kthread_run(kthreadd, new, name);
if (IS_ERR(task)) {
kfree(new);
@@ -292,7 +294,8 @@ int tsk_fork_get_node(struct task_struct *tsk)
return NUMA_NO_NODE;
}

-static void create_kthread(struct kthread_create_info *create)
+static void create_kthread(struct kthread_create_info *create,
+ unsigned long flags)
{
int pid;

@@ -300,7 +303,7 @@ static void create_kthread(struct kthread_create_info *create)
current->pref_node_fork = create->node;
#endif
/* We want our own signal handler (we take no signals by default). */
- pid = kernel_thread(kthread, create, CLONE_FS | CLONE_FILES | SIGCHLD);
+ pid = kernel_thread(kthread, create, flags);
if (pid < 0) {
/* If user was SIGKILLed, I release the structure. */
struct completion *done = xchg(&create->done, NULL);
@@ -623,7 +626,7 @@ void kthread_do_work(struct kthread_group *kg)
list_del_init(&create->list);
spin_unlock(&kg->create_lock);

- create_kthread(create);
+ create_kthread(create, kg->flags);

spin_lock(&kg->create_lock);
}
--
2.20.1


2019-02-08 20:12:01

by J. Bruce Fields

[permalink] [raw]
Subject: [PATCH 3/7] kthreads: allow multiple kthreadd's

From: "J. Bruce Fields" <[email protected]>

Allow subsystems to run their own kthreadd's.

I'm experimenting with this to allow nfsd to put its threads into its
own thread group to make it easy for the vfs to tell when nfsd is
breaking one of its own leases.

Signed-off-by: J. Bruce Fields <[email protected]>
---
include/linux/kthread.h | 20 ++++++-
init/main.c | 4 +-
kernel/kthread.c | 113 ++++++++++++++++++++++++++++++----------
3 files changed, 107 insertions(+), 30 deletions(-)

diff --git a/include/linux/kthread.h b/include/linux/kthread.h
index c1961761311d..10b5836dfb2a 100644
--- a/include/linux/kthread.h
+++ b/include/linux/kthread.h
@@ -6,6 +6,24 @@
#include <linux/sched.h>
#include <linux/cgroup.h>

+struct kthread_group {
+ char *name;
+ spinlock_t create_lock;
+ struct list_head create_list;
+ struct task_struct *task;
+};
+
+extern struct kthread_group kthreadd_default;
+
+struct kthread_group *kthread_start_group(char *);
+void kthread_stop_group(struct kthread_group *);
+
+struct task_struct *kthread_group_create_on_node(struct kthread_group *,
+ int (*threadfn)(void *data),
+ void *data,
+ int node,
+ const char namefmt[], ...);
+
__printf(4, 5)
struct task_struct *kthread_create_on_node(int (*threadfn)(void *data),
void *data,
@@ -63,7 +81,7 @@ int kthread_park(struct task_struct *k);
void kthread_unpark(struct task_struct *k);
void kthread_parkme(void);

-int kthreadd(void *unused);
+int kthreadd(void *);
extern struct task_struct *kthreadd_task;
extern int tsk_fork_get_node(struct task_struct *tsk);

diff --git a/init/main.c b/init/main.c
index e2e80ca3165a..c4ed4d75aca7 100644
--- a/init/main.c
+++ b/init/main.c
@@ -417,9 +417,9 @@ noinline void __ref rest_init(void)
rcu_read_unlock();

numa_default_policy();
- pid = kernel_thread(kthreadd, NULL, CLONE_FS | CLONE_FILES);
+ pid = kernel_thread(kthreadd, &kthreadd_default, CLONE_FS | CLONE_FILES);
rcu_read_lock();
- kthreadd_task = find_task_by_pid_ns(pid, &init_pid_ns);
+ kthreadd_default.task = find_task_by_pid_ns(pid, &init_pid_ns);
rcu_read_unlock();

/*
diff --git a/kernel/kthread.c b/kernel/kthread.c
index 4428fd586cd8..b515557b98c9 100644
--- a/kernel/kthread.c
+++ b/kernel/kthread.c
@@ -22,9 +22,44 @@
#include <linux/uaccess.h>
#include <trace/events/sched.h>

-static DEFINE_SPINLOCK(kthread_create_lock);
-static LIST_HEAD(kthread_create_list);
-struct task_struct *kthreadd_task;
+struct kthread_group kthreadd_default = {
+ .name = "kthreadd",
+ .create_lock = __SPIN_LOCK_UNLOCKED(kthreadd_default.create_lock),
+ .create_list = LIST_HEAD_INIT(kthreadd_default.create_list),
+};
+
+void wake_kthreadd(struct kthread_group *kg)
+{
+ wake_up_process(kg->task);
+}
+
+struct kthread_group *kthread_start_group(char *name)
+{
+ struct kthread_group *new;
+ struct task_struct *task;
+
+ new = kmalloc(sizeof(struct kthread_group), GFP_KERNEL);
+ if (!new)
+ return ERR_PTR(-ENOMEM);
+ spin_lock_init(&new->create_lock);
+ INIT_LIST_HEAD(&new->create_list);
+ new->name = name;
+ task = kthread_run(kthreadd, new, name);
+ if (IS_ERR(task)) {
+ kfree(new);
+ return ERR_CAST(task);
+ }
+ new->task = task;
+ return new;
+}
+EXPORT_SYMBOL_GPL(kthread_start_group);
+
+void kthread_stop_group(struct kthread_group *kg)
+{
+ kthread_stop(kg->task);
+ kfree(kg);
+}
+EXPORT_SYMBOL_GPL(kthread_stop_group);

struct kthread_create_info
{
@@ -279,11 +314,13 @@ static void create_kthread(struct kthread_create_info *create)
}
}

-static __printf(4, 0)
-struct task_struct *__kthread_create_on_node(int (*threadfn)(void *data),
- void *data, int node,
- const char namefmt[],
- va_list args)
+
+static __printf(5, 0)
+struct task_struct *__kthread_group_create_on_node(struct kthread_group *kg,
+ int (*threadfn)(void *data),
+ void *data, int node,
+ const char namefmt[],
+ va_list args)
{
DECLARE_COMPLETION_ONSTACK(done);
struct task_struct *task;
@@ -297,11 +334,11 @@ struct task_struct *__kthread_create_on_node(int (*threadfn)(void *data),
create->node = node;
create->done = &done;

- spin_lock(&kthread_create_lock);
- list_add_tail(&create->list, &kthread_create_list);
- spin_unlock(&kthread_create_lock);
+ spin_lock(&kg->create_lock);
+ list_add_tail(&create->list, &kg->create_list);
+ spin_unlock(&kg->create_lock);

- wake_up_process(kthreadd_task);
+ wake_kthreadd(kg);
/*
* Wait for completion in killable state, for I might be chosen by
* the OOM killer while kthreadd is trying to allocate memory for
@@ -343,6 +380,25 @@ struct task_struct *__kthread_create_on_node(int (*threadfn)(void *data),
return task;
}

+__printf(5, 0)
+struct task_struct *kthread_group_create_on_node(struct kthread_group *kg,
+ int (*threadfn)(void *data),
+ void *data, int node,
+ const char namefmt[],
+ ...)
+{
+ struct task_struct *task;
+ va_list args;
+
+ va_start(args, namefmt);
+ task = __kthread_group_create_on_node(kg, threadfn,
+ data, node, namefmt, args);
+ va_end(args);
+
+ return task;
+}
+EXPORT_SYMBOL_GPL(kthread_group_create_on_node);
+
/**
* kthread_create_on_node - create a kthread.
* @threadfn: the function to run until signal_pending(current).
@@ -375,7 +431,8 @@ struct task_struct *kthread_create_on_node(int (*threadfn)(void *data),
va_list args;

va_start(args, namefmt);
- task = __kthread_create_on_node(threadfn, data, node, namefmt, args);
+ task = __kthread_group_create_on_node(&kthreadd_default, threadfn,
+ data, node, namefmt, args);
va_end(args);

return task;
@@ -555,30 +612,31 @@ int kthread_stop(struct task_struct *k)
}
EXPORT_SYMBOL(kthread_stop);

-void kthread_do_work(void)
+void kthread_do_work(struct kthread_group *kg)
{
- spin_lock(&kthread_create_lock);
- while (!list_empty(&kthread_create_list)) {
+ spin_lock(&kg->create_lock);
+ while (!list_empty(&kg->create_list)) {
struct kthread_create_info *create;

- create = list_entry(kthread_create_list.next,
+ create = list_entry(kg->create_list.next,
struct kthread_create_info, list);
list_del_init(&create->list);
- spin_unlock(&kthread_create_lock);
+ spin_unlock(&kg->create_lock);

create_kthread(create);

- spin_lock(&kthread_create_lock);
+ spin_lock(&kg->create_lock);
}
- spin_unlock(&kthread_create_lock);
+ spin_unlock(&kg->create_lock);
}

-int kthreadd(void *unused)
+int kthreadd(void *data)
{
+ struct kthread_group *kg = data;
struct task_struct *tsk = current;

/* Setup a clean context for our children to inherit. */
- set_task_comm(tsk, "kthreadd");
+ set_task_comm(tsk, kg->name);
ignore_signals(tsk);
set_cpus_allowed_ptr(tsk, cpu_all_mask);
set_mems_allowed(node_states[N_MEMORY]);
@@ -586,13 +644,13 @@ int kthreadd(void *unused)
current->flags |= PF_NOFREEZE;
cgroup_init_kthreadd();

- for (;;) {
+ while (current == kthreadd_default.task || !kthread_should_stop()) {
set_current_state(TASK_INTERRUPTIBLE);
- if (list_empty(&kthread_create_list))
+ if (list_empty(&kg->create_list))
schedule();
__set_current_state(TASK_RUNNING);

- kthread_do_work();
+ kthread_do_work(kg);
}

return 0;
@@ -690,8 +748,9 @@ __kthread_create_worker(int cpu, unsigned int flags,
if (cpu >= 0)
node = cpu_to_node(cpu);

- task = __kthread_create_on_node(kthread_worker_fn, worker,
- node, namefmt, args);
+ task = __kthread_group_create_on_node(&kthreadd_default,
+ kthread_worker_fn,
+ worker, node, namefmt, args);
if (IS_ERR(task))
goto fail_task;

--
2.20.1


2019-02-08 20:12:18

by J. Bruce Fields

[permalink] [raw]
Subject: [PATCH 7/7] nfsd: ignore delegation self-conflicts

From: "J. Bruce Fields" <[email protected]>

A client's actions shouldn't revoke its own delegations, even if those
same actions (rename, link, etc.) would conflict if they came from a
different client.

Since nfsd has the necessary information to determine both who is
performing the action and who holds the relevant delegation, let nfsd
handle the revocation of delegations caused by nfs clients.

At the same time, modify the lease code to ignore conflicts between
delegations held by threads in the same tgid as the caller, assuming the
caller has taken care of any such conflicts itself.

Signed-off-by: J. Bruce Fields <[email protected]>
---
fs/locks.c | 39 +++++++++++++++++++++++++++++
fs/nfsd/nfs4state.c | 61 +++++++++++++++++++++++++++++++++++++++++++++
fs/nfsd/state.h | 2 ++
fs/nfsd/vfs.c | 32 +++++++++++++++++++-----
include/linux/fs.h | 2 ++
5 files changed, 130 insertions(+), 6 deletions(-)

diff --git a/fs/locks.c b/fs/locks.c
index ff6af2c32601..a1275e7fdfb4 100644
--- a/fs/locks.c
+++ b/fs/locks.c
@@ -1528,6 +1528,16 @@ static void time_out_leases(struct inode *inode, struct list_head *dispose)

static bool leases_conflict(struct file_lock *lease, struct file_lock *breaker)
{
+ /*
+ * We assume that threads in the same thread group are
+ * responsible for resolving conflicts among themselves.
+ * To avoid exposing this change in behavior to existing users,
+ * we only do this for delegations, which have not yet been
+ * exposed to userspace:
+ */
+ if ((lease->fl_flags & FL_DELEG) &&
+ (lease->fl_pid == current->tgid))
+ return false;
if ((breaker->fl_flags & FL_LAYOUT) != (lease->fl_flags & FL_LAYOUT))
return false;
if ((breaker->fl_flags & FL_DELEG) && (lease->fl_flags & FL_LEASE))
@@ -1666,6 +1676,35 @@ int __break_lease(struct inode *inode, unsigned int mode, unsigned int type)
}
EXPORT_SYMBOL(__break_lease);

+/*
+ * This is just a convenient way for knfsd to iterate over its
+ * delegations. We could possibly modify break_lease to work for
+ * this as well, or let nfsd find its delegations itself, with some
+ * changes to its data structures.
+ */
+bool foreach_delegation(struct inode *inode, bool cb(struct file_lock *, void *), void *arg)
+{
+ struct file_lock_context *ctx;
+ struct file_lock *fl;
+ bool delegs_broken = false;
+
+ if (!inode)
+ return false;
+ ctx = smp_load_acquire(&inode->i_flctx);
+ if (!ctx)
+ return false;
+ percpu_down_read_preempt_disable(&file_rwsem);
+ spin_lock(&ctx->flc_lock);
+ list_for_each_entry(fl, &ctx->flc_lease, fl_list) {
+ if (fl->fl_flags & FL_DELEG)
+ delegs_broken |= cb(fl, arg);
+ };
+ spin_unlock(&ctx->flc_lock);
+ percpu_up_read_preempt_enable(&file_rwsem);
+ return delegs_broken;
+}
+EXPORT_SYMBOL_GPL(foreach_delegation);
+
/**
* lease_get_mtime - update modified time of an inode with exclusive lease
* @inode: the inode
diff --git a/fs/nfsd/nfs4state.c b/fs/nfsd/nfs4state.c
index fb3c9844c82a..800c1625840e 100644
--- a/fs/nfsd/nfs4state.c
+++ b/fs/nfsd/nfs4state.c
@@ -4012,6 +4012,67 @@ nfsd_break_deleg_cb(struct file_lock *fl)
return ret;
}

+bool nfs4_client_owns_lease(struct file_lock *lease, void *arg)
+{
+ struct nfs4_client *clp = arg;
+ struct nfs4_delegation *dl = lease->fl_owner;
+
+ return dl->dl_stid.sc_client == clp;
+}
+
+bool break_nfsd_deleg(struct file_lock *fl, void *arg)
+{
+ struct nfs4_client *clp = arg;
+
+ /*
+ * XXX: may eventually also have to check whether this
+ * delegation is knfsd's; but, for now, we know all delegations
+ * are knfsd's.
+ */
+ if (!nfs4_client_owns_lease(fl, clp)) {
+ nfsd_break_deleg_cb(fl);
+ return true;
+ }
+ return false;
+}
+
+/*
+ * Break all delegations on ths file that are held by one of our clients
+ * and that conflict with write access by the given client.
+ */
+static int nfsd_break_delegations(struct inode *inode,
+ struct nfs4_client *clp)
+{
+ bool delegs_broken;
+
+ delegs_broken = foreach_delegation(inode, break_nfsd_deleg, clp);
+ return delegs_broken ? -EWOULDBLOCK : 0;
+}
+
+static struct nfs4_client *nfsd4_client_from_rqst(struct svc_rqst *rqst)
+{
+ struct nfsd4_compoundres *resp;
+
+ /*
+ * In case it's possible we could be called from NLM or ACL
+ * code?:
+ */
+ if (rqst->rq_prog != NFS_PROGRAM)
+ return NULL;
+ if (rqst->rq_vers != 4)
+ return NULL;
+ resp = rqst->rq_resp;
+ return resp->cstate.clp;
+}
+
+int nfsd_break_delegations_by_rqst(struct inode *inode,
+ struct svc_rqst *rqstp)
+{
+ struct nfs4_client *clp = nfsd4_client_from_rqst(rqstp);
+
+ return nfsd_break_delegations(inode, clp);
+}
+
static int
nfsd_change_deleg_cb(struct file_lock *onlist, int arg,
struct list_head *dispose)
diff --git a/fs/nfsd/state.h b/fs/nfsd/state.h
index 396c76755b03..3a7f6fdc92b4 100644
--- a/fs/nfsd/state.h
+++ b/fs/nfsd/state.h
@@ -649,6 +649,8 @@ static inline void get_nfs4_file(struct nfs4_file *fi)
}
struct file *find_any_file(struct nfs4_file *f);

+int nfsd_break_delegations_by_rqst(struct inode *, struct svc_rqst *);
+
/* grace period management */
void nfsd4_end_grace(struct nfsd_net *nn);

diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
index 9824e32b2f23..fb87d9f23ba8 100644
--- a/fs/nfsd/vfs.c
+++ b/fs/nfsd/vfs.c
@@ -456,6 +456,9 @@ nfsd_setattr(struct svc_rqst *rqstp, struct svc_fh *fhp, struct iattr *iap,
}

fh_lock(fhp);
+ host_err = nfsd_break_delegations_by_rqst(d_inode(dentry), rqstp);
+ if (host_err)
+ goto out_unlock;
if (size_change) {
/*
* RFC5661, Section 18.30.4:
@@ -697,14 +700,15 @@ nfsd_access(struct svc_rqst *rqstp, struct svc_fh *fhp, u32 *access, u32 *suppor
}
#endif /* CONFIG_NFSD_V3 */

-static int nfsd_open_break_lease(struct inode *inode, int access)
+static int nfsd_open_break_lease(struct inode *inode, int access,
+ struct svc_rqst *rqst)
{
- unsigned int mode;
-
if (access & NFSD_MAY_NOT_BREAK_LEASE)
return 0;
- mode = (access & NFSD_MAY_WRITE) ? O_WRONLY : O_RDONLY;
- return break_lease(inode, mode | O_NONBLOCK);
+ /* We don't implement write delegations yet: */
+ if (!(access & NFSD_MAY_WRITE))
+ return 0;
+ return nfsd_break_delegations_by_rqst(inode, rqst);
}

/*
@@ -764,7 +768,7 @@ nfsd_open(struct svc_rqst *rqstp, struct svc_fh *fhp, umode_t type,
if (!inode->i_fop)
goto out;

- host_err = nfsd_open_break_lease(inode, may_flags);
+ host_err = nfsd_open_break_lease(inode, may_flags, rqstp);
if (host_err) /* NOMEM or WOULDBLOCK */
goto out_nfserr;

@@ -1633,6 +1637,11 @@ nfsd_link(struct svc_rqst *rqstp, struct svc_fh *ffhp,
err = nfserr_noent;
if (d_really_is_negative(dold))
goto out_dput;
+
+ host_err = nfsd_break_delegations_by_rqst(d_inode(dold), rqstp);
+ if (host_err)
+ goto out_dput;
+
host_err = vfs_link(dold, dirp, dnew, NULL);
if (!host_err) {
err = nfserrno(commit_metadata(ffhp));
@@ -1726,6 +1735,13 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
if (ffhp->fh_export->ex_path.dentry != tfhp->fh_export->ex_path.dentry)
goto out_dput_new;

+ host_err = nfsd_break_delegations_by_rqst(d_inode(odentry), rqstp);
+ if (host_err)
+ goto out_dput_new;
+ host_err = nfsd_break_delegations_by_rqst(d_inode(ndentry), rqstp);
+ if (host_err)
+ goto out_dput_new;
+
host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
if (!host_err) {
host_err = commit_metadata(tfhp);
@@ -1795,6 +1811,10 @@ nfsd_unlink(struct svc_rqst *rqstp, struct svc_fh *fhp, int type,
if (!type)
type = d_inode(rdentry)->i_mode & S_IFMT;

+ host_err = nfsd_break_delegations_by_rqst(d_inode(rdentry), rqstp);
+ if (host_err)
+ goto out_nfserr;
+
if (type != S_IFDIR)
host_err = vfs_unlink(dirp, rdentry, NULL);
else
diff --git a/include/linux/fs.h b/include/linux/fs.h
index 29d8e2cfed0e..467981b5fd9d 100644
--- a/include/linux/fs.h
+++ b/include/linux/fs.h
@@ -2439,6 +2439,8 @@ static inline int break_deleg_wait(struct inode **delegated_inode)
return ret;
}

+bool foreach_delegation(struct inode *inode, bool cb(struct file_lock *, void *), void *arg);
+
static inline int break_layout(struct inode *inode, bool wait)
{
smp_mb();
--
2.20.1


2019-02-08 20:12:45

by J. Bruce Fields

[permalink] [raw]
Subject: [PATCH 2/7] kthreads: Simplify tsk_fork_get_node

From: "J. Bruce Fields" <[email protected]>

This will also simplify a following patch that allows multiple
kthreadd's.

Signed-off-by: J. Bruce Fields <[email protected]>
---
init/init_task.c | 3 +++
kernel/fork.c | 4 ++++
kernel/kthread.c | 3 +--
3 files changed, 8 insertions(+), 2 deletions(-)

diff --git a/init/init_task.c b/init/init_task.c
index 5aebe3be4d7c..47e4829ec53f 100644
--- a/init/init_task.c
+++ b/init/init_task.c
@@ -153,6 +153,9 @@ struct task_struct init_task
.vtime.starttime = 0,
.vtime.state = VTIME_SYS,
#endif
+#ifdef CONFIG_NUMA
+ .pref_node_fork = NUMA_NO_NODE,
+#endif
#ifdef CONFIG_NUMA_BALANCING
.numa_preferred_nid = -1,
.numa_group = NULL,
diff --git a/kernel/fork.c b/kernel/fork.c
index b69248e6f0e0..c5b5629de2a9 100644
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -912,6 +912,10 @@ static struct task_struct *dup_task_struct(struct task_struct *orig, int node)
tsk->fail_nth = 0;
#endif

+#ifdef CONFIG_NUMA
+ tsk->pref_node_fork = NUMA_NO_NODE;
+#endif
+
#ifdef CONFIG_BLK_CGROUP
tsk->throttle_queue = NULL;
tsk->use_memdelay = 0;
diff --git a/kernel/kthread.c b/kernel/kthread.c
index 672f0bbf4d89..4428fd586cd8 100644
--- a/kernel/kthread.c
+++ b/kernel/kthread.c
@@ -252,8 +252,7 @@ static int kthread(void *_create)
int tsk_fork_get_node(struct task_struct *tsk)
{
#ifdef CONFIG_NUMA
- if (tsk == kthreadd_task)
- return tsk->pref_node_fork;
+ return tsk->pref_node_fork;
#endif
return NUMA_NO_NODE;
}
--
2.20.1


2019-02-08 20:12:47

by J. Bruce Fields

[permalink] [raw]
Subject: [PATCH 6/7] rpc: move rpc server threads into their own thread group

From: "J. Bruce Fields" <[email protected]>

Signed-off-by: J. Bruce Fields <[email protected]>
---
include/linux/sunrpc/svc.h | 1 +
net/sunrpc/svc.c | 23 ++++++++++++++++++-----
2 files changed, 19 insertions(+), 5 deletions(-)

diff --git a/include/linux/sunrpc/svc.h b/include/linux/sunrpc/svc.h
index e52385340b3b..07d428dd067f 100644
--- a/include/linux/sunrpc/svc.h
+++ b/include/linux/sunrpc/svc.h
@@ -101,6 +101,7 @@ struct svc_serv {

unsigned int sv_nrpools; /* number of thread pools */
struct svc_pool * sv_pools; /* array of thread pools */
+ struct kthread_group * sv_kthread_grp;
const struct svc_serv_ops *sv_ops; /* server operations */
#if defined(CONFIG_SUNRPC_BACKCHANNEL)
struct list_head sv_cb_list; /* queue for callback requests
diff --git a/net/sunrpc/svc.c b/net/sunrpc/svc.c
index 502e09a28f84..d8586a57da40 100644
--- a/net/sunrpc/svc.c
+++ b/net/sunrpc/svc.c
@@ -464,10 +464,16 @@ __svc_create(struct svc_program *prog, unsigned int bufsize, int npools,
serv->sv_pools =
kcalloc(serv->sv_nrpools, sizeof(struct svc_pool),
GFP_KERNEL);
- if (!serv->sv_pools) {
- kfree(serv);
- return NULL;
- }
+ if (!serv->sv_pools)
+ goto out_free_serv;
+
+ serv->sv_kthread_grp = kthread_start_group(CLONE_THREAD |
+ CLONE_SIGHAND | CLONE_VM | CLONE_FS |
+ CLONE_FILES | SIGCHLD,
+ serv->sv_name);
+
+ if (IS_ERR(serv->sv_kthread_grp))
+ goto out_free_pools;

for (i = 0; i < serv->sv_nrpools; i++) {
struct svc_pool *pool = &serv->sv_pools[i];
@@ -482,6 +488,11 @@ __svc_create(struct svc_program *prog, unsigned int bufsize, int npools,
}

return serv;
+out_free_pools:
+ kfree(serv->sv_pools);
+out_free_serv:
+ kfree(serv);
+ return NULL;
}

struct svc_serv *
@@ -551,6 +562,7 @@ svc_destroy(struct svc_serv *serv)
if (svc_serv_is_pooled(serv))
svc_pool_map_put();

+ kthread_stop_group(serv->sv_kthread_grp);
kfree(serv->sv_pools);
kfree(serv);
}
@@ -719,7 +731,8 @@ svc_start_kthread(struct svc_serv *serv, struct svc_pool *pool,
return PTR_ERR(rqstp);

__module_get(serv->sv_ops->svo_module);
- task = kthread_create_on_node(serv->sv_ops->svo_function, rqstp,
+ task = kthread_group_create_on_node(serv->sv_kthread_grp,
+ serv->sv_ops->svo_function, rqstp,
node, "%s", serv->sv_name);
if (IS_ERR(task)) {
module_put(serv->sv_ops->svo_module);
--
2.20.1


2019-02-08 20:13:22

by J. Bruce Fields

[permalink] [raw]
Subject: [PATCH 1/7] kthreads: minor kthreadd refactoring

From: "J. Bruce Fields" <[email protected]>

Trivial refactoring, no change in behavior.

Not really necessary, a separate function for the inner loop just seems
a little nicer to me.

Signed-off-by: J. Bruce Fields <[email protected]>
---
kernel/kthread.c | 33 +++++++++++++++++++--------------
1 file changed, 19 insertions(+), 14 deletions(-)

diff --git a/kernel/kthread.c b/kernel/kthread.c
index 087d18d771b5..672f0bbf4d89 100644
--- a/kernel/kthread.c
+++ b/kernel/kthread.c
@@ -556,6 +556,24 @@ int kthread_stop(struct task_struct *k)
}
EXPORT_SYMBOL(kthread_stop);

+void kthread_do_work(void)
+{
+ spin_lock(&kthread_create_lock);
+ while (!list_empty(&kthread_create_list)) {
+ struct kthread_create_info *create;
+
+ create = list_entry(kthread_create_list.next,
+ struct kthread_create_info, list);
+ list_del_init(&create->list);
+ spin_unlock(&kthread_create_lock);
+
+ create_kthread(create);
+
+ spin_lock(&kthread_create_lock);
+ }
+ spin_unlock(&kthread_create_lock);
+}
+
int kthreadd(void *unused)
{
struct task_struct *tsk = current;
@@ -575,20 +593,7 @@ int kthreadd(void *unused)
schedule();
__set_current_state(TASK_RUNNING);

- spin_lock(&kthread_create_lock);
- while (!list_empty(&kthread_create_list)) {
- struct kthread_create_info *create;
-
- create = list_entry(kthread_create_list.next,
- struct kthread_create_info, list);
- list_del_init(&create->list);
- spin_unlock(&kthread_create_lock);
-
- create_kthread(create);
-
- spin_lock(&kthread_create_lock);
- }
- spin_unlock(&kthread_create_lock);
+ kthread_do_work();
}

return 0;
--
2.20.1


2019-02-08 20:13:42

by J. Bruce Fields

[permalink] [raw]
Subject: [PATCH 5/7] rpc: separate out body of svc_start_kthreads

From: "J. Bruce Fields" <[email protected]>

No change in behavior, just trivial refactoring.

Signed-off-by: J. Bruce Fields <[email protected]>
---
net/sunrpc/svc.c | 62 +++++++++++++++++++++++++++++-------------------
1 file changed, 37 insertions(+), 25 deletions(-)

diff --git a/net/sunrpc/svc.c b/net/sunrpc/svc.c
index e87ddb9f7feb..502e09a28f84 100644
--- a/net/sunrpc/svc.c
+++ b/net/sunrpc/svc.c
@@ -702,40 +702,52 @@ choose_victim(struct svc_serv *serv, struct svc_pool *pool, unsigned int *state)
return task;
}

-/* create new threads */
static int
-svc_start_kthreads(struct svc_serv *serv, struct svc_pool *pool, int nrservs)
+svc_start_kthread(struct svc_serv *serv, struct svc_pool *pool,
+ unsigned int *state)
{
struct svc_rqst *rqstp;
struct task_struct *task;
struct svc_pool *chosen_pool;
- unsigned int state = serv->sv_nrthreads-1;
int node;

- do {
- nrservs--;
- chosen_pool = choose_pool(serv, pool, &state);
-
- node = svc_pool_map_get_node(chosen_pool->sp_id);
- rqstp = svc_prepare_thread(serv, chosen_pool, node);
- if (IS_ERR(rqstp))
- return PTR_ERR(rqstp);
-
- __module_get(serv->sv_ops->svo_module);
- task = kthread_create_on_node(serv->sv_ops->svo_function, rqstp,
- node, "%s", serv->sv_name);
- if (IS_ERR(task)) {
- module_put(serv->sv_ops->svo_module);
- svc_exit_thread(rqstp);
- return PTR_ERR(task);
- }
+ chosen_pool = choose_pool(serv, pool, state);

- rqstp->rq_task = task;
- if (serv->sv_nrpools > 1)
- svc_pool_map_set_cpumask(task, chosen_pool->sp_id);
+ node = svc_pool_map_get_node(chosen_pool->sp_id);
+ rqstp = svc_prepare_thread(serv, chosen_pool, node);
+ if (IS_ERR(rqstp))
+ return PTR_ERR(rqstp);

- svc_sock_update_bufs(serv);
- wake_up_process(task);
+ __module_get(serv->sv_ops->svo_module);
+ task = kthread_create_on_node(serv->sv_ops->svo_function, rqstp,
+ node, "%s", serv->sv_name);
+ if (IS_ERR(task)) {
+ module_put(serv->sv_ops->svo_module);
+ svc_exit_thread(rqstp);
+ return PTR_ERR(task);
+ }
+
+ rqstp->rq_task = task;
+ if (serv->sv_nrpools > 1)
+ svc_pool_map_set_cpumask(task, chosen_pool->sp_id);
+
+ svc_sock_update_bufs(serv);
+ wake_up_process(task);
+ return 0;
+}
+
+/* create new threads */
+static int
+svc_start_kthreads(struct svc_serv *serv, struct svc_pool *pool, int nrservs)
+{
+ unsigned int state = serv->sv_nrthreads-1;
+ int ret;
+
+ do {
+ nrservs--;
+ ret = svc_start_kthread(serv, pool, &state);
+ if (ret)
+ return ret;
} while (nrservs > 0);

return 0;
--
2.20.1


2019-02-09 12:46:06

by Jeff Layton

[permalink] [raw]
Subject: Re: [PATCH 0/7] Eliminate delegation self-conflicts

On Fri, 2019-02-08 at 15:10 -0500, J. Bruce Fields wrote:
> From: "J. Bruce Fields" <[email protected]>
>
> These patches allow NFSv4 clients holding delegations to keep them when
> the operation that would break a delegation comes from the same client.
>
> To do that, we somehow need to pass the identity of the
> delegation-breaker down through the VFS.
>
> This series uses the tgid, a solution suggested by Trond. To do that we
> need nfsd tasks to share the same tgid. I do that by extending the
> kthread code slightly to allow knfsd to run the kthreadd main loop in a
> task of its own, and spawn its server threads off of that task.
>
> Part of Trond's thinking was that this would work for userspace too.
> Delegations are currently only available to knfsd, but Ganesha and Samba
> may eventually be interested in a userspace interface (probably a minor
> variation on the fcntl F_{GET,SET}LEASE interface). A threaded
> userspace server would first resolve conflicts between its own clients,
> and then call into the kernel to break any leases acquired by other
> processes. That may require some careful locking of the server's own
> data structures, but it should work.
>
> Previously I considered instead adding a new field somewhere in the
> struct task. That might require a new system call to expose to user
> space. Or we might be able to put this in a keyring, if David Howells
> thought that would work.
>
> Before that I tried passing the identity of the breaker explicitly, but
> that looks like it would require passing the new argument around to huge
> swaths of the VFS.
>
> I'm testing this with some a locally modified pynfs; I'll fix that up
> and push it out at some point, but pynfs has a number of bugs in this
> area.
>
> I wasn't sure who to ask about the kthread.c changes, so I'm cc'ing a
> random assortment of developers in recent changelogs, hope that's OK.
>
> --b.
>
> J. Bruce Fields (7):
> kthreads: minor kthreadd refactoring
> kthreads: Simplify tsk_fork_get_node
> kthreads: allow multiple kthreadd's
> kthreads: allow cloning threads with different flags
> rpc: separate out body of svc_start_kthreads
> rpc: move rpc server threads into their own thread group
> nfsd: ignore delegation self-conflicts
>
> fs/locks.c | 39 +++++++++++
> fs/nfsd/nfs4state.c | 61 ++++++++++++++++
> fs/nfsd/state.h | 2 +
> fs/nfsd/vfs.c | 32 +++++++--
> include/linux/fs.h | 2 +
> include/linux/kthread.h | 21 +++++-
> include/linux/sunrpc/svc.h | 1 +
> init/init_task.c | 3 +
> init/main.c | 4 +-
> kernel/fork.c | 4 ++
> kernel/kthread.c | 140 +++++++++++++++++++++++++++----------
> net/sunrpc/svc.c | 83 ++++++++++++++--------
> 12 files changed, 317 insertions(+), 75 deletions(-)
>

Nice work! I like the basic idea, the changes seem to be well-organized,
and the tgid semantics are clear and make sense.

Would this preclude us from moving to a workqueue-based model for knfsd
later? It's likely to still be worth it, but it'd be good to understand
the potential drawbacks.

Thanks,
--
Jeff Layton <[email protected]>


2019-02-11 16:00:23

by J. Bruce Fields

[permalink] [raw]
Subject: Re: [PATCH 0/7] Eliminate delegation self-conflicts

On Sat, Feb 09, 2019 at 07:43:54AM -0500, Jeff Layton wrote:
> On Fri, 2019-02-08 at 15:10 -0500, J. Bruce Fields wrote:
> > From: "J. Bruce Fields" <[email protected]>
> >
> > These patches allow NFSv4 clients holding delegations to keep them when
> > the operation that would break a delegation comes from the same client.
> >
> > To do that, we somehow need to pass the identity of the
> > delegation-breaker down through the VFS.
> >
> > This series uses the tgid, a solution suggested by Trond. To do that we
> > need nfsd tasks to share the same tgid. I do that by extending the
> > kthread code slightly to allow knfsd to run the kthreadd main loop in a
> > task of its own, and spawn its server threads off of that task.
...
> Nice work! I like the basic idea, the changes seem to be well-organized,
> and the tgid semantics are clear and make sense.
>
> Would this preclude us from moving to a workqueue-based model for knfsd
> later? It's likely to still be worth it, but it'd be good to understand
> the potential drawbacks.

I was wondering about that too, but I haven't looked into it yet.
Workqueues look a lot more complicated than kthreads.

--b.

2019-02-15 17:03:56

by J. Bruce Fields

[permalink] [raw]
Subject: Re: [PATCH 0/7] Eliminate delegation self-conflicts

On Mon, Feb 11, 2019 at 10:58:04AM -0500, J. Bruce Fields wrote:
> On Sat, Feb 09, 2019 at 07:43:54AM -0500, Jeff Layton wrote:
> > On Fri, 2019-02-08 at 15:10 -0500, J. Bruce Fields wrote:
> > > From: "J. Bruce Fields" <[email protected]>
> > >
> > > These patches allow NFSv4 clients holding delegations to keep them when
> > > the operation that would break a delegation comes from the same client.
> > >
> > > To do that, we somehow need to pass the identity of the
> > > delegation-breaker down through the VFS.
> > >
> > > This series uses the tgid, a solution suggested by Trond. To do that we
> > > need nfsd tasks to share the same tgid. I do that by extending the
> > > kthread code slightly to allow knfsd to run the kthreadd main loop in a
> > > task of its own, and spawn its server threads off of that task.
> ...
> > Nice work! I like the basic idea, the changes seem to be well-organized,
> > and the tgid semantics are clear and make sense.
> >
> > Would this preclude us from moving to a workqueue-based model for knfsd
> > later? It's likely to still be worth it, but it'd be good to understand
> > the potential drawbacks.
>
> I was wondering about that too, but I haven't looked into it yet.
> Workqueues look a lot more complicated than kthreads.

I spent some time staring, and... I still don't really understand the
workqueue code. But if this kthread_group[*] code is acceptable than I
can't see why it shouldn't be possible to create a workqueue whose work
items are all handled by threads spawned form the same kthread_group.

--b.

[*] Open to suggestions of better names.

2019-03-12 20:02:09

by J. Bruce Fields

[permalink] [raw]
Subject: Re: [PATCH 3/7] kthreads: allow multiple kthreadd's

I'd like to try to get this in 5.2. Can anyone give me an ACK for the
kthread parts? Or tell me who to ask? Let me know if I should resend.
Or, of course, tell me if I've missed something and this is all a
terrible idea.

--b.

On Fri, Feb 08, 2019 at 03:10:43PM -0500, J. Bruce Fields wrote:
> From: "J. Bruce Fields" <[email protected]>
>
> Allow subsystems to run their own kthreadd's.
>
> I'm experimenting with this to allow nfsd to put its threads into its
> own thread group to make it easy for the vfs to tell when nfsd is
> breaking one of its own leases.
>
> Signed-off-by: J. Bruce Fields <[email protected]>
> ---
> include/linux/kthread.h | 20 ++++++-
> init/main.c | 4 +-
> kernel/kthread.c | 113 ++++++++++++++++++++++++++++++----------
> 3 files changed, 107 insertions(+), 30 deletions(-)
>
> diff --git a/include/linux/kthread.h b/include/linux/kthread.h
> index c1961761311d..10b5836dfb2a 100644
> --- a/include/linux/kthread.h
> +++ b/include/linux/kthread.h
> @@ -6,6 +6,24 @@
> #include <linux/sched.h>
> #include <linux/cgroup.h>
>
> +struct kthread_group {
> + char *name;
> + spinlock_t create_lock;
> + struct list_head create_list;
> + struct task_struct *task;
> +};
> +
> +extern struct kthread_group kthreadd_default;
> +
> +struct kthread_group *kthread_start_group(char *);
> +void kthread_stop_group(struct kthread_group *);
> +
> +struct task_struct *kthread_group_create_on_node(struct kthread_group *,
> + int (*threadfn)(void *data),
> + void *data,
> + int node,
> + const char namefmt[], ...);
> +
> __printf(4, 5)
> struct task_struct *kthread_create_on_node(int (*threadfn)(void *data),
> void *data,
> @@ -63,7 +81,7 @@ int kthread_park(struct task_struct *k);
> void kthread_unpark(struct task_struct *k);
> void kthread_parkme(void);
>
> -int kthreadd(void *unused);
> +int kthreadd(void *);
> extern struct task_struct *kthreadd_task;
> extern int tsk_fork_get_node(struct task_struct *tsk);
>
> diff --git a/init/main.c b/init/main.c
> index e2e80ca3165a..c4ed4d75aca7 100644
> --- a/init/main.c
> +++ b/init/main.c
> @@ -417,9 +417,9 @@ noinline void __ref rest_init(void)
> rcu_read_unlock();
>
> numa_default_policy();
> - pid = kernel_thread(kthreadd, NULL, CLONE_FS | CLONE_FILES);
> + pid = kernel_thread(kthreadd, &kthreadd_default, CLONE_FS | CLONE_FILES);
> rcu_read_lock();
> - kthreadd_task = find_task_by_pid_ns(pid, &init_pid_ns);
> + kthreadd_default.task = find_task_by_pid_ns(pid, &init_pid_ns);
> rcu_read_unlock();
>
> /*
> diff --git a/kernel/kthread.c b/kernel/kthread.c
> index 4428fd586cd8..b515557b98c9 100644
> --- a/kernel/kthread.c
> +++ b/kernel/kthread.c
> @@ -22,9 +22,44 @@
> #include <linux/uaccess.h>
> #include <trace/events/sched.h>
>
> -static DEFINE_SPINLOCK(kthread_create_lock);
> -static LIST_HEAD(kthread_create_list);
> -struct task_struct *kthreadd_task;
> +struct kthread_group kthreadd_default = {
> + .name = "kthreadd",
> + .create_lock = __SPIN_LOCK_UNLOCKED(kthreadd_default.create_lock),
> + .create_list = LIST_HEAD_INIT(kthreadd_default.create_list),
> +};
> +
> +void wake_kthreadd(struct kthread_group *kg)
> +{
> + wake_up_process(kg->task);
> +}
> +
> +struct kthread_group *kthread_start_group(char *name)
> +{
> + struct kthread_group *new;
> + struct task_struct *task;
> +
> + new = kmalloc(sizeof(struct kthread_group), GFP_KERNEL);
> + if (!new)
> + return ERR_PTR(-ENOMEM);
> + spin_lock_init(&new->create_lock);
> + INIT_LIST_HEAD(&new->create_list);
> + new->name = name;
> + task = kthread_run(kthreadd, new, name);
> + if (IS_ERR(task)) {
> + kfree(new);
> + return ERR_CAST(task);
> + }
> + new->task = task;
> + return new;
> +}
> +EXPORT_SYMBOL_GPL(kthread_start_group);
> +
> +void kthread_stop_group(struct kthread_group *kg)
> +{
> + kthread_stop(kg->task);
> + kfree(kg);
> +}
> +EXPORT_SYMBOL_GPL(kthread_stop_group);
>
> struct kthread_create_info
> {
> @@ -279,11 +314,13 @@ static void create_kthread(struct kthread_create_info *create)
> }
> }
>
> -static __printf(4, 0)
> -struct task_struct *__kthread_create_on_node(int (*threadfn)(void *data),
> - void *data, int node,
> - const char namefmt[],
> - va_list args)
> +
> +static __printf(5, 0)
> +struct task_struct *__kthread_group_create_on_node(struct kthread_group *kg,
> + int (*threadfn)(void *data),
> + void *data, int node,
> + const char namefmt[],
> + va_list args)
> {
> DECLARE_COMPLETION_ONSTACK(done);
> struct task_struct *task;
> @@ -297,11 +334,11 @@ struct task_struct *__kthread_create_on_node(int (*threadfn)(void *data),
> create->node = node;
> create->done = &done;
>
> - spin_lock(&kthread_create_lock);
> - list_add_tail(&create->list, &kthread_create_list);
> - spin_unlock(&kthread_create_lock);
> + spin_lock(&kg->create_lock);
> + list_add_tail(&create->list, &kg->create_list);
> + spin_unlock(&kg->create_lock);
>
> - wake_up_process(kthreadd_task);
> + wake_kthreadd(kg);
> /*
> * Wait for completion in killable state, for I might be chosen by
> * the OOM killer while kthreadd is trying to allocate memory for
> @@ -343,6 +380,25 @@ struct task_struct *__kthread_create_on_node(int (*threadfn)(void *data),
> return task;
> }
>
> +__printf(5, 0)
> +struct task_struct *kthread_group_create_on_node(struct kthread_group *kg,
> + int (*threadfn)(void *data),
> + void *data, int node,
> + const char namefmt[],
> + ...)
> +{
> + struct task_struct *task;
> + va_list args;
> +
> + va_start(args, namefmt);
> + task = __kthread_group_create_on_node(kg, threadfn,
> + data, node, namefmt, args);
> + va_end(args);
> +
> + return task;
> +}
> +EXPORT_SYMBOL_GPL(kthread_group_create_on_node);
> +
> /**
> * kthread_create_on_node - create a kthread.
> * @threadfn: the function to run until signal_pending(current).
> @@ -375,7 +431,8 @@ struct task_struct *kthread_create_on_node(int (*threadfn)(void *data),
> va_list args;
>
> va_start(args, namefmt);
> - task = __kthread_create_on_node(threadfn, data, node, namefmt, args);
> + task = __kthread_group_create_on_node(&kthreadd_default, threadfn,
> + data, node, namefmt, args);
> va_end(args);
>
> return task;
> @@ -555,30 +612,31 @@ int kthread_stop(struct task_struct *k)
> }
> EXPORT_SYMBOL(kthread_stop);
>
> -void kthread_do_work(void)
> +void kthread_do_work(struct kthread_group *kg)
> {
> - spin_lock(&kthread_create_lock);
> - while (!list_empty(&kthread_create_list)) {
> + spin_lock(&kg->create_lock);
> + while (!list_empty(&kg->create_list)) {
> struct kthread_create_info *create;
>
> - create = list_entry(kthread_create_list.next,
> + create = list_entry(kg->create_list.next,
> struct kthread_create_info, list);
> list_del_init(&create->list);
> - spin_unlock(&kthread_create_lock);
> + spin_unlock(&kg->create_lock);
>
> create_kthread(create);
>
> - spin_lock(&kthread_create_lock);
> + spin_lock(&kg->create_lock);
> }
> - spin_unlock(&kthread_create_lock);
> + spin_unlock(&kg->create_lock);
> }
>
> -int kthreadd(void *unused)
> +int kthreadd(void *data)
> {
> + struct kthread_group *kg = data;
> struct task_struct *tsk = current;
>
> /* Setup a clean context for our children to inherit. */
> - set_task_comm(tsk, "kthreadd");
> + set_task_comm(tsk, kg->name);
> ignore_signals(tsk);
> set_cpus_allowed_ptr(tsk, cpu_all_mask);
> set_mems_allowed(node_states[N_MEMORY]);
> @@ -586,13 +644,13 @@ int kthreadd(void *unused)
> current->flags |= PF_NOFREEZE;
> cgroup_init_kthreadd();
>
> - for (;;) {
> + while (current == kthreadd_default.task || !kthread_should_stop()) {
> set_current_state(TASK_INTERRUPTIBLE);
> - if (list_empty(&kthread_create_list))
> + if (list_empty(&kg->create_list))
> schedule();
> __set_current_state(TASK_RUNNING);
>
> - kthread_do_work();
> + kthread_do_work(kg);
> }
>
> return 0;
> @@ -690,8 +748,9 @@ __kthread_create_worker(int cpu, unsigned int flags,
> if (cpu >= 0)
> node = cpu_to_node(cpu);
>
> - task = __kthread_create_on_node(kthread_worker_fn, worker,
> - node, namefmt, args);
> + task = __kthread_group_create_on_node(&kthreadd_default,
> + kthread_worker_fn,
> + worker, node, namefmt, args);
> if (IS_ERR(task))
> goto fail_task;
>
> --
> 2.20.1