All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH-v3 0/4] target/vhost-scsi: Add per-cpu ida tag pre-allocation for v3.12
@ 2013-08-16 23:09 ` Nicholas A. Bellinger
  0 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-16 23:09 UTC (permalink / raw)
  To: target-devel
  Cc: lf-virt, lkml, kvm-devel, Michael S. Tsirkin, Asias He,
	Kent Overstreet, Andrew Morton, Jens Axboe, Tejun Heo,
	Ingo Molnar, Andi Kleen, Christoph Lameter, Nicholas Bellinger

From: Nicholas Bellinger <nab@linux-iscsi.org>

Hi folks,

This is an updated series for adding tag pre-allocation support of
target fabric descriptor memory, utilizing Kent's latest per-cpu ida
bits here, along with Christoph Lameter's latest comments:

   [PATCH 04/10] idr: Percpu ida
   http://marc.info/?l=linux-kernel&m=137160026006974&w=2

The first patch is a standalone version of per-cpu-ida, seperate from
the full idr rewrite from Kent that is still being discussed.  Given
that I have a number of other target items aside from this series that
depend on this code, I'd like to go ahead and include this standalone
version into target-pending/for-next as a v3.12 item.

Jens has also expressed interest in a blk-mq conversion to use these
per-cpu-ida primatives, so getting this piece merged for v3.12 would
make life easier for both of us.  ;)

The second patch includes target-core setup of se_sess->sess_cmd_map +
se_sess->sess_tag_pool resources at session creation time, using
fabric independent code in transport_init_session_tags().

The third patch is the initial conversion of vhost-scsi fabric code
to use per-cpu ida logic for obtaining a new tcm_vhost_cmd descriptor
via vhost_scsi_get_tag() during vhost_work_fn_t->handle_kick() ->
vhost_scsi_handle_vq() callback execution.

And the forth patch is a vhost-scsi change that adds pre-allocation of
per tcm_vhost_cmd descriptor scatterlist + user-space page pointer
memory, that allows the last two fast-path allocations to be dropped
from tcm_vhost_submission_work() -> vhost_scsi_map_to_sgl() fast-path
execution.

Please review as v3.12 material.

Thanks!

--nab

Kent Overstreet (1):
  idr: Percpu ida

Nicholas Bellinger (3):
  target: Add transport_init_session_tags using per-cpu ida
  vhost/scsi: Convert to per-cpu ida_alloc + ida_free command map
  vhost/scsi: Add pre-allocation for tv_cmd SGL + upages memory

 drivers/target/target_core_transport.c |   33 ++++
 drivers/vhost/scsi.c                   |  132 +++++++++----
 include/linux/idr.h                    |   53 ++++++
 include/target/target_core_base.h      |    5 +
 include/target/target_core_fabric.h    |    1 +
 lib/idr.c                              |  316 +++++++++++++++++++++++++++++++-
 6 files changed, 501 insertions(+), 39 deletions(-)

-- 
1.7.10.4


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

* [PATCH-v3 0/4] target/vhost-scsi: Add per-cpu ida tag pre-allocation for v3.12
@ 2013-08-16 23:09 ` Nicholas A. Bellinger
  0 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-16 23:09 UTC (permalink / raw)
  To: target-devel
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, Kent Overstreet, Ingo Molnar, Christoph Lameter,
	Tejun Heo, Andrew Morton

From: Nicholas Bellinger <nab@linux-iscsi.org>

Hi folks,

This is an updated series for adding tag pre-allocation support of
target fabric descriptor memory, utilizing Kent's latest per-cpu ida
bits here, along with Christoph Lameter's latest comments:

   [PATCH 04/10] idr: Percpu ida
   http://marc.info/?l=linux-kernel&m=137160026006974&w=2

The first patch is a standalone version of per-cpu-ida, seperate from
the full idr rewrite from Kent that is still being discussed.  Given
that I have a number of other target items aside from this series that
depend on this code, I'd like to go ahead and include this standalone
version into target-pending/for-next as a v3.12 item.

Jens has also expressed interest in a blk-mq conversion to use these
per-cpu-ida primatives, so getting this piece merged for v3.12 would
make life easier for both of us.  ;)

The second patch includes target-core setup of se_sess->sess_cmd_map +
se_sess->sess_tag_pool resources at session creation time, using
fabric independent code in transport_init_session_tags().

The third patch is the initial conversion of vhost-scsi fabric code
to use per-cpu ida logic for obtaining a new tcm_vhost_cmd descriptor
via vhost_scsi_get_tag() during vhost_work_fn_t->handle_kick() ->
vhost_scsi_handle_vq() callback execution.

And the forth patch is a vhost-scsi change that adds pre-allocation of
per tcm_vhost_cmd descriptor scatterlist + user-space page pointer
memory, that allows the last two fast-path allocations to be dropped
from tcm_vhost_submission_work() -> vhost_scsi_map_to_sgl() fast-path
execution.

Please review as v3.12 material.

Thanks!

--nab

Kent Overstreet (1):
  idr: Percpu ida

Nicholas Bellinger (3):
  target: Add transport_init_session_tags using per-cpu ida
  vhost/scsi: Convert to per-cpu ida_alloc + ida_free command map
  vhost/scsi: Add pre-allocation for tv_cmd SGL + upages memory

 drivers/target/target_core_transport.c |   33 ++++
 drivers/vhost/scsi.c                   |  132 +++++++++----
 include/linux/idr.h                    |   53 ++++++
 include/target/target_core_base.h      |    5 +
 include/target/target_core_fabric.h    |    1 +
 lib/idr.c                              |  316 +++++++++++++++++++++++++++++++-
 6 files changed, 501 insertions(+), 39 deletions(-)

-- 
1.7.10.4

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

* [PATCH-v3 1/4] idr: Percpu ida
  2013-08-16 23:09 ` Nicholas A. Bellinger
@ 2013-08-16 23:09   ` Nicholas A. Bellinger
  -1 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-16 23:09 UTC (permalink / raw)
  To: target-devel
  Cc: lf-virt, lkml, kvm-devel, Michael S. Tsirkin, Asias He,
	Kent Overstreet, Andrew Morton, Jens Axboe, Tejun Heo,
	Ingo Molnar, Andi Kleen, Christoph Lameter, Oleg Nesterov,
	Christoph Lameter, Nicholas A. Bellinger

From: Kent Overstreet <kmo@daterainc.com>

Percpu frontend for allocating ids. With percpu allocation (that works),
it's impossible to guarantee it will always be possible to allocate all
nr_tags - typically, some will be stuck on a remote percpu freelist
where the current job can't get to them.

We do guarantee that it will always be possible to allocate at least
(nr_tags / 2) tags - this is done by keeping track of which and how many
cpus have tags on their percpu freelists. On allocation failure if
enough cpus have tags that there could potentially be (nr_tags / 2) tags
stuck on remote percpu freelists, we then pick a remote cpu at random to
steal from.

Note that there's no cpu hotplug notifier - we don't care, because
steal_tags() will eventually get the down cpu's tags. We _could_ satisfy
more allocations if we had a notifier - but we'll still meet our
guarantees and it's absolutely not a correctness issue, so I don't think
it's worth the extra code.

Signed-off-by: Kent Overstreet <koverstreet@google.com>
Cc: Tejun Heo <tj@kernel.org>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Christoph Lameter <cl@linux-foundation.org>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Andi Kleen <andi@firstfloor.org>
Cc: Jens Axboe <axboe@kernel.dk>
Cc: "Nicholas A. Bellinger" <nab@linux-iscsi.org>
Signed-off-by: Nicholas Bellinger <nab@linux-iscsi.org>
---
 include/linux/idr.h |   53 +++++++++
 lib/idr.c           |  316 +++++++++++++++++++++++++++++++++++++++++++++++++--
 2 files changed, 361 insertions(+), 8 deletions(-)

diff --git a/include/linux/idr.h b/include/linux/idr.h
index 871a213..f0db12b 100644
--- a/include/linux/idr.h
+++ b/include/linux/idr.h
@@ -16,6 +16,8 @@
 #include <linux/bitops.h>
 #include <linux/init.h>
 #include <linux/rcupdate.h>
+#include <linux/spinlock_types.h>
+#include <linux/wait.h>
 
 /*
  * We want shallower trees and thus more bits covered at each layer.  8
@@ -243,4 +245,55 @@ static inline int ida_get_new(struct ida *ida, int *p_id)
 
 void __init idr_init_cache(void);
 
+/* Percpu IDA/tag allocator */
+
+struct percpu_ida_cpu;
+
+struct percpu_ida {
+	/*
+	 * number of tags available to be allocated, as passed to
+	 * percpu_ida_init()
+	 */
+	unsigned			nr_tags;
+
+	struct percpu_ida_cpu __percpu	*tag_cpu;
+
+	/*
+	 * Bitmap of cpus that (may) have tags on their percpu freelists:
+	 * steal_tags() uses this to decide when to steal tags, and which cpus
+	 * to try stealing from.
+	 *
+	 * It's ok for a freelist to be empty when its bit is set - steal_tags()
+	 * will just keep looking - but the bitmap _must_ be set whenever a
+	 * percpu freelist does have tags.
+	 */
+	unsigned long			*cpus_have_tags;
+
+	struct {
+		spinlock_t		lock;
+		/*
+		 * When we go to steal tags from another cpu (see steal_tags()),
+		 * we want to pick a cpu at random. Cycling through them every
+		 * time we steal is a bit easier and more or less equivalent:
+		 */
+		unsigned		cpu_last_stolen;
+
+		/* For sleeping on allocation failure */
+		wait_queue_head_t	wait;
+
+		/*
+		 * Global freelist - it's a stack where nr_free points to the
+		 * top
+		 */
+		unsigned		nr_free;
+		unsigned		*freelist;
+	} ____cacheline_aligned_in_smp;
+};
+
+int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp);
+void percpu_ida_free(struct percpu_ida *pool, unsigned tag);
+
+void percpu_ida_destroy(struct percpu_ida *pool);
+int percpu_ida_init(struct percpu_ida *pool, unsigned long nr_tags);
+
 #endif /* __IDR_H__ */
diff --git a/lib/idr.c b/lib/idr.c
index bfe4db4..57bfabe 100644
--- a/lib/idr.c
+++ b/lib/idr.c
@@ -26,17 +26,20 @@
  * with the slab allocator.
  */
 
-#ifndef TEST                        // to test in user space...
-#include <linux/slab.h>
-#include <linux/init.h>
-#include <linux/export.h>
-#endif
+#include <linux/bitmap.h>
+#include <linux/bitops.h>
+#include <linux/bug.h>
 #include <linux/err.h>
-#include <linux/string.h>
+#include <linux/export.h>
+#include <linux/hardirq.h>
 #include <linux/idr.h>
-#include <linux/spinlock.h>
+#include <linux/init.h>
+#include <linux/kernel.h>
 #include <linux/percpu.h>
-#include <linux/hardirq.h>
+#include <linux/sched.h>
+#include <linux/slab.h>
+#include <linux/string.h>
+#include <linux/spinlock.h>
 
 #define MAX_IDR_SHIFT		(sizeof(int) * 8 - 1)
 #define MAX_IDR_BIT		(1U << MAX_IDR_SHIFT)
@@ -1159,3 +1162,300 @@ void ida_init(struct ida *ida)
 
 }
 EXPORT_SYMBOL(ida_init);
+
+/* Percpu IDA */
+
+/*
+ * Number of tags we move between the percpu freelist and the global freelist at
+ * a time
+ */
+#define IDA_PCPU_BATCH_MOVE	32U
+
+/* Max size of percpu freelist, */
+#define IDA_PCPU_SIZE		((IDA_PCPU_BATCH_MOVE * 3) / 2)
+
+struct percpu_ida_cpu {
+	spinlock_t			lock;
+	unsigned			nr_free;
+	unsigned			freelist[];
+};
+
+static inline void move_tags(unsigned *dst, unsigned *dst_nr,
+			     unsigned *src, unsigned *src_nr,
+			     unsigned nr)
+{
+	*src_nr -= nr;
+	memcpy(dst + *dst_nr, src + *src_nr, sizeof(unsigned) * nr);
+	*dst_nr += nr;
+}
+
+/*
+ * Try to steal tags from a remote cpu's percpu freelist.
+ *
+ * We first check how many percpu freelists have tags - we don't steal tags
+ * unless enough percpu freelists have tags on them that it's possible more than
+ * half the total tags could be stuck on remote percpu freelists.
+ *
+ * Then we iterate through the cpus until we find some tags - we don't attempt
+ * to find the "best" cpu to steal from, to keep cacheline bouncing to a
+ * minimum.
+ */
+static inline void steal_tags(struct percpu_ida *pool,
+			      struct percpu_ida_cpu *tags)
+{
+	unsigned cpus_have_tags, cpu = pool->cpu_last_stolen;
+	struct percpu_ida_cpu *remote;
+
+	for (cpus_have_tags = bitmap_weight(pool->cpus_have_tags, nr_cpu_ids);
+	     cpus_have_tags * IDA_PCPU_SIZE > pool->nr_tags / 2;
+	     cpus_have_tags--) {
+		cpu = find_next_bit(pool->cpus_have_tags, nr_cpu_ids, cpu);
+
+		if (cpu == nr_cpu_ids)
+			cpu = find_first_bit(pool->cpus_have_tags, nr_cpu_ids);
+
+		if (cpu == nr_cpu_ids)
+			BUG();
+
+		pool->cpu_last_stolen = cpu;
+		remote = per_cpu_ptr(pool->tag_cpu, cpu);
+
+		clear_bit(cpu, pool->cpus_have_tags);
+
+		if (remote == tags)
+			continue;
+
+		spin_lock(&remote->lock);
+
+		if (remote->nr_free) {
+			memcpy(tags->freelist,
+			       remote->freelist,
+			       sizeof(unsigned) * remote->nr_free);
+
+			tags->nr_free = remote->nr_free;
+			remote->nr_free = 0;
+		}
+
+		spin_unlock(&remote->lock);
+
+		if (tags->nr_free)
+			break;
+	}
+}
+
+static inline void alloc_global_tags(struct percpu_ida *pool,
+				     struct percpu_ida_cpu *tags)
+{
+	move_tags(tags->freelist, &tags->nr_free,
+		  pool->freelist, &pool->nr_free,
+		  min(pool->nr_free, IDA_PCPU_BATCH_MOVE));
+}
+
+static inline unsigned alloc_local_tag(struct percpu_ida *pool,
+				       struct percpu_ida_cpu *tags)
+{
+	int tag = -ENOSPC;
+
+	spin_lock(&tags->lock);
+	if (tags->nr_free)
+		tag = tags->freelist[--tags->nr_free];
+	spin_unlock(&tags->lock);
+
+	return tag;
+}
+
+/**
+ * percpu_ida_alloc - allocate a tag
+ * @pool: pool to allocate from
+ * @gfp: gfp flags
+ *
+ * Returns a tag - an integer in the range [0..nr_tags) (passed to
+ * tag_pool_init()), or otherwise -ENOSPC on allocation failure.
+ *
+ * Safe to be called from interrupt context (assuming it isn't passed
+ * __GFP_WAIT, of course).
+ *
+ * Will not fail if passed __GFP_WAIT.
+ */
+int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp)
+{
+	DEFINE_WAIT(wait);
+	struct percpu_ida_cpu *tags;
+	unsigned long flags;
+	int tag;
+
+	local_irq_save(flags);
+	tags = this_cpu_ptr(pool->tag_cpu);
+
+	/* Fastpath */
+	tag = alloc_local_tag(pool, tags);
+	if (likely(tag >= 0)) {
+		local_irq_restore(flags);
+		return tag;
+	}
+
+	while (1) {
+		spin_lock(&pool->lock);
+
+		/*
+		 * prepare_to_wait() must come before steal_tags(), in case
+		 * percpu_ida_free() on another cpu flips a bit in
+		 * cpus_have_tags
+		 *
+		 * global lock held and irqs disabled, don't need percpu lock
+		 */
+		prepare_to_wait(&pool->wait, &wait, TASK_UNINTERRUPTIBLE);
+
+		if (!tags->nr_free)
+			alloc_global_tags(pool, tags);
+		if (!tags->nr_free)
+			steal_tags(pool, tags);
+
+		if (tags->nr_free) {
+			tag = tags->freelist[--tags->nr_free];
+			if (tags->nr_free)
+				set_bit(smp_processor_id(),
+					pool->cpus_have_tags);
+		}
+
+		spin_unlock(&pool->lock);
+		local_irq_restore(flags);
+
+		if (tag >= 0 || !(gfp & __GFP_WAIT))
+			break;
+
+		schedule();
+
+		local_irq_save(flags);
+		tags = this_cpu_ptr(pool->tag_cpu);
+	}
+
+	finish_wait(&pool->wait, &wait);
+	return tag;
+}
+EXPORT_SYMBOL_GPL(percpu_ida_alloc);
+
+/**
+ * percpu_ida_free - free a tag
+ * @pool: pool @tag was allocated from
+ * @tag: a tag previously allocated with percpu_ida_alloc()
+ *
+ * Safe to be called from interrupt context.
+ */
+void percpu_ida_free(struct percpu_ida *pool, unsigned tag)
+{
+	struct percpu_ida_cpu *tags;
+	unsigned long flags;
+	unsigned nr_free;
+
+	BUG_ON(tag >= pool->nr_tags);
+
+	local_irq_save(flags);
+	tags = this_cpu_ptr(pool->tag_cpu);
+
+	spin_lock(&tags->lock);
+	tags->freelist[tags->nr_free++] = tag;
+
+	nr_free = tags->nr_free;
+	spin_unlock(&tags->lock);
+
+	if (nr_free == 1) {
+		set_bit(smp_processor_id(),
+			pool->cpus_have_tags);
+		wake_up(&pool->wait);
+	}
+
+	if (nr_free == IDA_PCPU_SIZE) {
+		spin_lock(&pool->lock);
+
+		/*
+		 * Global lock held and irqs disabled, don't need percpu
+		 * lock
+		 */
+		if (tags->nr_free == IDA_PCPU_SIZE) {
+			move_tags(pool->freelist, &pool->nr_free,
+				  tags->freelist, &tags->nr_free,
+				  IDA_PCPU_BATCH_MOVE);
+
+			wake_up(&pool->wait);
+		}
+		spin_unlock(&pool->lock);
+	}
+
+	local_irq_restore(flags);
+}
+EXPORT_SYMBOL_GPL(percpu_ida_free);
+
+/**
+ * percpu_ida_destroy - release a tag pool's resources
+ * @pool: pool to free
+ *
+ * Frees the resources allocated by percpu_ida_init().
+ */
+void percpu_ida_destroy(struct percpu_ida *pool)
+{
+	free_percpu(pool->tag_cpu);
+	kfree(pool->cpus_have_tags);
+	free_pages((unsigned long) pool->freelist,
+		   get_order(pool->nr_tags * sizeof(unsigned)));
+}
+EXPORT_SYMBOL_GPL(percpu_ida_destroy);
+
+/**
+ * percpu_ida_init - initialize a percpu tag pool
+ * @pool: pool to initialize
+ * @nr_tags: number of tags that will be available for allocation
+ *
+ * Initializes @pool so that it can be used to allocate tags - integers in the
+ * range [0, nr_tags). Typically, they'll be used by driver code to refer to a
+ * preallocated array of tag structures.
+ *
+ * Allocation is percpu, but sharding is limited by nr_tags - for best
+ * performance, the workload should not span more cpus than nr_tags / 128.
+ */
+int percpu_ida_init(struct percpu_ida *pool, unsigned long nr_tags)
+{
+	unsigned i, cpu, order;
+
+	memset(pool, 0, sizeof(*pool));
+
+	init_waitqueue_head(&pool->wait);
+	spin_lock_init(&pool->lock);
+	pool->nr_tags = nr_tags;
+
+	/* Guard against overflow */
+	if (nr_tags > (unsigned) INT_MAX + 1) {
+		pr_err("tags.c: nr_tags too large\n");
+		return -EINVAL;
+	}
+
+	order = get_order(nr_tags * sizeof(unsigned));
+	pool->freelist = (void *) __get_free_pages(GFP_KERNEL, order);
+	if (!pool->freelist)
+		return -ENOMEM;
+
+	for (i = 0; i < nr_tags; i++)
+		pool->freelist[i] = i;
+
+	pool->nr_free = nr_tags;
+
+	pool->cpus_have_tags = kzalloc(BITS_TO_LONGS(nr_cpu_ids) *
+				       sizeof(unsigned long), GFP_KERNEL);
+	if (!pool->cpus_have_tags)
+		goto err;
+
+	pool->tag_cpu = __alloc_percpu(sizeof(struct percpu_ida_cpu) +
+				       IDA_PCPU_SIZE * sizeof(unsigned),
+				       sizeof(unsigned));
+	if (!pool->tag_cpu)
+		goto err;
+
+	for_each_possible_cpu(cpu)
+		spin_lock_init(&per_cpu_ptr(pool->tag_cpu, cpu)->lock);
+
+	return 0;
+err:
+	percpu_ida_destroy(pool);
+	return -ENOMEM;
+}
+EXPORT_SYMBOL_GPL(percpu_ida_init);
-- 
1.7.10.4


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

* [PATCH-v3 1/4] idr: Percpu ida
@ 2013-08-16 23:09   ` Nicholas A. Bellinger
  0 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-16 23:09 UTC (permalink / raw)
  To: target-devel
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, Kent Overstreet, Ingo Molnar, Christoph Lameter,
	Oleg Nesterov, Tejun Heo, Christoph Lameter, Andrew Morton

From: Kent Overstreet <kmo@daterainc.com>

Percpu frontend for allocating ids. With percpu allocation (that works),
it's impossible to guarantee it will always be possible to allocate all
nr_tags - typically, some will be stuck on a remote percpu freelist
where the current job can't get to them.

We do guarantee that it will always be possible to allocate at least
(nr_tags / 2) tags - this is done by keeping track of which and how many
cpus have tags on their percpu freelists. On allocation failure if
enough cpus have tags that there could potentially be (nr_tags / 2) tags
stuck on remote percpu freelists, we then pick a remote cpu at random to
steal from.

Note that there's no cpu hotplug notifier - we don't care, because
steal_tags() will eventually get the down cpu's tags. We _could_ satisfy
more allocations if we had a notifier - but we'll still meet our
guarantees and it's absolutely not a correctness issue, so I don't think
it's worth the extra code.

Signed-off-by: Kent Overstreet <koverstreet@google.com>
Cc: Tejun Heo <tj@kernel.org>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Christoph Lameter <cl@linux-foundation.org>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Andi Kleen <andi@firstfloor.org>
Cc: Jens Axboe <axboe@kernel.dk>
Cc: "Nicholas A. Bellinger" <nab@linux-iscsi.org>
Signed-off-by: Nicholas Bellinger <nab@linux-iscsi.org>
---
 include/linux/idr.h |   53 +++++++++
 lib/idr.c           |  316 +++++++++++++++++++++++++++++++++++++++++++++++++--
 2 files changed, 361 insertions(+), 8 deletions(-)

diff --git a/include/linux/idr.h b/include/linux/idr.h
index 871a213..f0db12b 100644
--- a/include/linux/idr.h
+++ b/include/linux/idr.h
@@ -16,6 +16,8 @@
 #include <linux/bitops.h>
 #include <linux/init.h>
 #include <linux/rcupdate.h>
+#include <linux/spinlock_types.h>
+#include <linux/wait.h>
 
 /*
  * We want shallower trees and thus more bits covered at each layer.  8
@@ -243,4 +245,55 @@ static inline int ida_get_new(struct ida *ida, int *p_id)
 
 void __init idr_init_cache(void);
 
+/* Percpu IDA/tag allocator */
+
+struct percpu_ida_cpu;
+
+struct percpu_ida {
+	/*
+	 * number of tags available to be allocated, as passed to
+	 * percpu_ida_init()
+	 */
+	unsigned			nr_tags;
+
+	struct percpu_ida_cpu __percpu	*tag_cpu;
+
+	/*
+	 * Bitmap of cpus that (may) have tags on their percpu freelists:
+	 * steal_tags() uses this to decide when to steal tags, and which cpus
+	 * to try stealing from.
+	 *
+	 * It's ok for a freelist to be empty when its bit is set - steal_tags()
+	 * will just keep looking - but the bitmap _must_ be set whenever a
+	 * percpu freelist does have tags.
+	 */
+	unsigned long			*cpus_have_tags;
+
+	struct {
+		spinlock_t		lock;
+		/*
+		 * When we go to steal tags from another cpu (see steal_tags()),
+		 * we want to pick a cpu at random. Cycling through them every
+		 * time we steal is a bit easier and more or less equivalent:
+		 */
+		unsigned		cpu_last_stolen;
+
+		/* For sleeping on allocation failure */
+		wait_queue_head_t	wait;
+
+		/*
+		 * Global freelist - it's a stack where nr_free points to the
+		 * top
+		 */
+		unsigned		nr_free;
+		unsigned		*freelist;
+	} ____cacheline_aligned_in_smp;
+};
+
+int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp);
+void percpu_ida_free(struct percpu_ida *pool, unsigned tag);
+
+void percpu_ida_destroy(struct percpu_ida *pool);
+int percpu_ida_init(struct percpu_ida *pool, unsigned long nr_tags);
+
 #endif /* __IDR_H__ */
diff --git a/lib/idr.c b/lib/idr.c
index bfe4db4..57bfabe 100644
--- a/lib/idr.c
+++ b/lib/idr.c
@@ -26,17 +26,20 @@
  * with the slab allocator.
  */
 
-#ifndef TEST                        // to test in user space...
-#include <linux/slab.h>
-#include <linux/init.h>
-#include <linux/export.h>
-#endif
+#include <linux/bitmap.h>
+#include <linux/bitops.h>
+#include <linux/bug.h>
 #include <linux/err.h>
-#include <linux/string.h>
+#include <linux/export.h>
+#include <linux/hardirq.h>
 #include <linux/idr.h>
-#include <linux/spinlock.h>
+#include <linux/init.h>
+#include <linux/kernel.h>
 #include <linux/percpu.h>
-#include <linux/hardirq.h>
+#include <linux/sched.h>
+#include <linux/slab.h>
+#include <linux/string.h>
+#include <linux/spinlock.h>
 
 #define MAX_IDR_SHIFT		(sizeof(int) * 8 - 1)
 #define MAX_IDR_BIT		(1U << MAX_IDR_SHIFT)
@@ -1159,3 +1162,300 @@ void ida_init(struct ida *ida)
 
 }
 EXPORT_SYMBOL(ida_init);
+
+/* Percpu IDA */
+
+/*
+ * Number of tags we move between the percpu freelist and the global freelist at
+ * a time
+ */
+#define IDA_PCPU_BATCH_MOVE	32U
+
+/* Max size of percpu freelist, */
+#define IDA_PCPU_SIZE		((IDA_PCPU_BATCH_MOVE * 3) / 2)
+
+struct percpu_ida_cpu {
+	spinlock_t			lock;
+	unsigned			nr_free;
+	unsigned			freelist[];
+};
+
+static inline void move_tags(unsigned *dst, unsigned *dst_nr,
+			     unsigned *src, unsigned *src_nr,
+			     unsigned nr)
+{
+	*src_nr -= nr;
+	memcpy(dst + *dst_nr, src + *src_nr, sizeof(unsigned) * nr);
+	*dst_nr += nr;
+}
+
+/*
+ * Try to steal tags from a remote cpu's percpu freelist.
+ *
+ * We first check how many percpu freelists have tags - we don't steal tags
+ * unless enough percpu freelists have tags on them that it's possible more than
+ * half the total tags could be stuck on remote percpu freelists.
+ *
+ * Then we iterate through the cpus until we find some tags - we don't attempt
+ * to find the "best" cpu to steal from, to keep cacheline bouncing to a
+ * minimum.
+ */
+static inline void steal_tags(struct percpu_ida *pool,
+			      struct percpu_ida_cpu *tags)
+{
+	unsigned cpus_have_tags, cpu = pool->cpu_last_stolen;
+	struct percpu_ida_cpu *remote;
+
+	for (cpus_have_tags = bitmap_weight(pool->cpus_have_tags, nr_cpu_ids);
+	     cpus_have_tags * IDA_PCPU_SIZE > pool->nr_tags / 2;
+	     cpus_have_tags--) {
+		cpu = find_next_bit(pool->cpus_have_tags, nr_cpu_ids, cpu);
+
+		if (cpu == nr_cpu_ids)
+			cpu = find_first_bit(pool->cpus_have_tags, nr_cpu_ids);
+
+		if (cpu == nr_cpu_ids)
+			BUG();
+
+		pool->cpu_last_stolen = cpu;
+		remote = per_cpu_ptr(pool->tag_cpu, cpu);
+
+		clear_bit(cpu, pool->cpus_have_tags);
+
+		if (remote == tags)
+			continue;
+
+		spin_lock(&remote->lock);
+
+		if (remote->nr_free) {
+			memcpy(tags->freelist,
+			       remote->freelist,
+			       sizeof(unsigned) * remote->nr_free);
+
+			tags->nr_free = remote->nr_free;
+			remote->nr_free = 0;
+		}
+
+		spin_unlock(&remote->lock);
+
+		if (tags->nr_free)
+			break;
+	}
+}
+
+static inline void alloc_global_tags(struct percpu_ida *pool,
+				     struct percpu_ida_cpu *tags)
+{
+	move_tags(tags->freelist, &tags->nr_free,
+		  pool->freelist, &pool->nr_free,
+		  min(pool->nr_free, IDA_PCPU_BATCH_MOVE));
+}
+
+static inline unsigned alloc_local_tag(struct percpu_ida *pool,
+				       struct percpu_ida_cpu *tags)
+{
+	int tag = -ENOSPC;
+
+	spin_lock(&tags->lock);
+	if (tags->nr_free)
+		tag = tags->freelist[--tags->nr_free];
+	spin_unlock(&tags->lock);
+
+	return tag;
+}
+
+/**
+ * percpu_ida_alloc - allocate a tag
+ * @pool: pool to allocate from
+ * @gfp: gfp flags
+ *
+ * Returns a tag - an integer in the range [0..nr_tags) (passed to
+ * tag_pool_init()), or otherwise -ENOSPC on allocation failure.
+ *
+ * Safe to be called from interrupt context (assuming it isn't passed
+ * __GFP_WAIT, of course).
+ *
+ * Will not fail if passed __GFP_WAIT.
+ */
+int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp)
+{
+	DEFINE_WAIT(wait);
+	struct percpu_ida_cpu *tags;
+	unsigned long flags;
+	int tag;
+
+	local_irq_save(flags);
+	tags = this_cpu_ptr(pool->tag_cpu);
+
+	/* Fastpath */
+	tag = alloc_local_tag(pool, tags);
+	if (likely(tag >= 0)) {
+		local_irq_restore(flags);
+		return tag;
+	}
+
+	while (1) {
+		spin_lock(&pool->lock);
+
+		/*
+		 * prepare_to_wait() must come before steal_tags(), in case
+		 * percpu_ida_free() on another cpu flips a bit in
+		 * cpus_have_tags
+		 *
+		 * global lock held and irqs disabled, don't need percpu lock
+		 */
+		prepare_to_wait(&pool->wait, &wait, TASK_UNINTERRUPTIBLE);
+
+		if (!tags->nr_free)
+			alloc_global_tags(pool, tags);
+		if (!tags->nr_free)
+			steal_tags(pool, tags);
+
+		if (tags->nr_free) {
+			tag = tags->freelist[--tags->nr_free];
+			if (tags->nr_free)
+				set_bit(smp_processor_id(),
+					pool->cpus_have_tags);
+		}
+
+		spin_unlock(&pool->lock);
+		local_irq_restore(flags);
+
+		if (tag >= 0 || !(gfp & __GFP_WAIT))
+			break;
+
+		schedule();
+
+		local_irq_save(flags);
+		tags = this_cpu_ptr(pool->tag_cpu);
+	}
+
+	finish_wait(&pool->wait, &wait);
+	return tag;
+}
+EXPORT_SYMBOL_GPL(percpu_ida_alloc);
+
+/**
+ * percpu_ida_free - free a tag
+ * @pool: pool @tag was allocated from
+ * @tag: a tag previously allocated with percpu_ida_alloc()
+ *
+ * Safe to be called from interrupt context.
+ */
+void percpu_ida_free(struct percpu_ida *pool, unsigned tag)
+{
+	struct percpu_ida_cpu *tags;
+	unsigned long flags;
+	unsigned nr_free;
+
+	BUG_ON(tag >= pool->nr_tags);
+
+	local_irq_save(flags);
+	tags = this_cpu_ptr(pool->tag_cpu);
+
+	spin_lock(&tags->lock);
+	tags->freelist[tags->nr_free++] = tag;
+
+	nr_free = tags->nr_free;
+	spin_unlock(&tags->lock);
+
+	if (nr_free == 1) {
+		set_bit(smp_processor_id(),
+			pool->cpus_have_tags);
+		wake_up(&pool->wait);
+	}
+
+	if (nr_free == IDA_PCPU_SIZE) {
+		spin_lock(&pool->lock);
+
+		/*
+		 * Global lock held and irqs disabled, don't need percpu
+		 * lock
+		 */
+		if (tags->nr_free == IDA_PCPU_SIZE) {
+			move_tags(pool->freelist, &pool->nr_free,
+				  tags->freelist, &tags->nr_free,
+				  IDA_PCPU_BATCH_MOVE);
+
+			wake_up(&pool->wait);
+		}
+		spin_unlock(&pool->lock);
+	}
+
+	local_irq_restore(flags);
+}
+EXPORT_SYMBOL_GPL(percpu_ida_free);
+
+/**
+ * percpu_ida_destroy - release a tag pool's resources
+ * @pool: pool to free
+ *
+ * Frees the resources allocated by percpu_ida_init().
+ */
+void percpu_ida_destroy(struct percpu_ida *pool)
+{
+	free_percpu(pool->tag_cpu);
+	kfree(pool->cpus_have_tags);
+	free_pages((unsigned long) pool->freelist,
+		   get_order(pool->nr_tags * sizeof(unsigned)));
+}
+EXPORT_SYMBOL_GPL(percpu_ida_destroy);
+
+/**
+ * percpu_ida_init - initialize a percpu tag pool
+ * @pool: pool to initialize
+ * @nr_tags: number of tags that will be available for allocation
+ *
+ * Initializes @pool so that it can be used to allocate tags - integers in the
+ * range [0, nr_tags). Typically, they'll be used by driver code to refer to a
+ * preallocated array of tag structures.
+ *
+ * Allocation is percpu, but sharding is limited by nr_tags - for best
+ * performance, the workload should not span more cpus than nr_tags / 128.
+ */
+int percpu_ida_init(struct percpu_ida *pool, unsigned long nr_tags)
+{
+	unsigned i, cpu, order;
+
+	memset(pool, 0, sizeof(*pool));
+
+	init_waitqueue_head(&pool->wait);
+	spin_lock_init(&pool->lock);
+	pool->nr_tags = nr_tags;
+
+	/* Guard against overflow */
+	if (nr_tags > (unsigned) INT_MAX + 1) {
+		pr_err("tags.c: nr_tags too large\n");
+		return -EINVAL;
+	}
+
+	order = get_order(nr_tags * sizeof(unsigned));
+	pool->freelist = (void *) __get_free_pages(GFP_KERNEL, order);
+	if (!pool->freelist)
+		return -ENOMEM;
+
+	for (i = 0; i < nr_tags; i++)
+		pool->freelist[i] = i;
+
+	pool->nr_free = nr_tags;
+
+	pool->cpus_have_tags = kzalloc(BITS_TO_LONGS(nr_cpu_ids) *
+				       sizeof(unsigned long), GFP_KERNEL);
+	if (!pool->cpus_have_tags)
+		goto err;
+
+	pool->tag_cpu = __alloc_percpu(sizeof(struct percpu_ida_cpu) +
+				       IDA_PCPU_SIZE * sizeof(unsigned),
+				       sizeof(unsigned));
+	if (!pool->tag_cpu)
+		goto err;
+
+	for_each_possible_cpu(cpu)
+		spin_lock_init(&per_cpu_ptr(pool->tag_cpu, cpu)->lock);
+
+	return 0;
+err:
+	percpu_ida_destroy(pool);
+	return -ENOMEM;
+}
+EXPORT_SYMBOL_GPL(percpu_ida_init);
-- 
1.7.10.4

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

* [PATCH-v3 2/4] target: Add transport_init_session_tags using per-cpu ida
  2013-08-16 23:09 ` Nicholas A. Bellinger
@ 2013-08-16 23:09   ` Nicholas A. Bellinger
  -1 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-16 23:09 UTC (permalink / raw)
  To: target-devel
  Cc: lf-virt, lkml, kvm-devel, Michael S. Tsirkin, Asias He,
	Kent Overstreet, Andrew Morton, Jens Axboe, Tejun Heo,
	Ingo Molnar, Andi Kleen, Christoph Lameter, Nicholas Bellinger

From: Nicholas Bellinger <nab@linux-iscsi.org>

This patch adds lib/idr.c based transport_init_session_tags() logic
that allows fabric drivers to setup a per-cpu se_sess->sess_tag_pool
and associated se_sess->sess_cmd_map for basic tagged pre-allocation
of fabric descriptor sized memory.

v3: Update to percpu-ida usage

Cc: Kent Overstreet <kmo@daterainc.com>
Cc: Asias He <asias@redhat.com>
Cc: Michael S. Tsirkin <mst@redhat.com>
Signed-off-by: Nicholas Bellinger <nab@linux-iscsi.org>
---
 drivers/target/target_core_transport.c |   33 ++++++++++++++++++++++++++++++++
 include/target/target_core_base.h      |    5 +++++
 include/target/target_core_fabric.h    |    1 +
 3 files changed, 39 insertions(+)

diff --git a/drivers/target/target_core_transport.c b/drivers/target/target_core_transport.c
index 7172d00..64ceb51 100644
--- a/drivers/target/target_core_transport.c
+++ b/drivers/target/target_core_transport.c
@@ -232,6 +232,35 @@ struct se_session *transport_init_session(void)
 }
 EXPORT_SYMBOL(transport_init_session);
 
+struct se_session *transport_init_session_tags(unsigned int tag_num,
+					       unsigned int tag_size)
+{
+	struct se_session *se_sess;
+	int rc;
+
+	se_sess = transport_init_session();
+	if (IS_ERR(se_sess))
+		return se_sess;
+
+	se_sess->sess_cmd_map = kzalloc(tag_num * tag_size, GFP_KERNEL);
+	if (!se_sess->sess_cmd_map) {
+		pr_err("Unable to allocate se_sess->sess_cmd_map\n");
+		transport_free_session(se_sess);
+		return ERR_PTR(-ENOMEM);
+	}
+
+	rc = percpu_ida_init(&se_sess->sess_tag_pool, tag_num);
+	if (rc < 0) {
+		pr_err("Unable to init se_sess->sess_tag_pool,"
+			" tag_num: %u\n", tag_num);
+		transport_free_session(se_sess);
+		return ERR_PTR(-ENOMEM);
+	}
+
+	return se_sess;
+}
+EXPORT_SYMBOL(transport_init_session_tags);
+
 /*
  * Called with spin_lock_irqsave(&struct se_portal_group->session_lock called.
  */
@@ -367,6 +396,10 @@ EXPORT_SYMBOL(transport_deregister_session_configfs);
 
 void transport_free_session(struct se_session *se_sess)
 {
+	if (se_sess->sess_cmd_map) {
+		percpu_ida_destroy(&se_sess->sess_tag_pool);
+		kfree(se_sess->sess_cmd_map);
+	}
 	kmem_cache_free(se_sess_cache, se_sess);
 }
 EXPORT_SYMBOL(transport_free_session);
diff --git a/include/target/target_core_base.h b/include/target/target_core_base.h
index e34fc90..360e4a3 100644
--- a/include/target/target_core_base.h
+++ b/include/target/target_core_base.h
@@ -5,6 +5,7 @@
 #include <linux/configfs.h>
 #include <linux/dma-mapping.h>
 #include <linux/blkdev.h>
+#include <linux/idr.h>
 #include <scsi/scsi_cmnd.h>
 #include <net/sock.h>
 #include <net/tcp.h>
@@ -415,6 +416,8 @@ struct se_cmd {
 	enum dma_data_direction	data_direction;
 	/* For SAM Task Attribute */
 	int			sam_task_attr;
+	/* Used for se_sess->sess_tag_pool */
+	unsigned int		map_tag;
 	/* Transport protocol dependent state, see transport_state_table */
 	enum transport_state_table t_state;
 	unsigned		cmd_wait_set:1;
@@ -536,6 +539,8 @@ struct se_session {
 	struct list_head	sess_wait_list;
 	spinlock_t		sess_cmd_lock;
 	struct kref		sess_kref;
+	void			*sess_cmd_map;
+	struct percpu_ida	sess_tag_pool;
 };
 
 struct se_device;
diff --git a/include/target/target_core_fabric.h b/include/target/target_core_fabric.h
index 7a16178..b13afb6 100644
--- a/include/target/target_core_fabric.h
+++ b/include/target/target_core_fabric.h
@@ -84,6 +84,7 @@ struct target_core_fabric_ops {
 };
 
 struct se_session *transport_init_session(void);
+struct se_session *transport_init_session_tags(unsigned int, unsigned int);
 void	__transport_register_session(struct se_portal_group *,
 		struct se_node_acl *, struct se_session *, void *);
 void	transport_register_session(struct se_portal_group *,
-- 
1.7.10.4


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

* [PATCH-v3 2/4] target: Add transport_init_session_tags using per-cpu ida
@ 2013-08-16 23:09   ` Nicholas A. Bellinger
  0 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-16 23:09 UTC (permalink / raw)
  To: target-devel
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, Kent Overstreet, Ingo Molnar, Christoph Lameter,
	Tejun Heo, Andrew Morton

From: Nicholas Bellinger <nab@linux-iscsi.org>

This patch adds lib/idr.c based transport_init_session_tags() logic
that allows fabric drivers to setup a per-cpu se_sess->sess_tag_pool
and associated se_sess->sess_cmd_map for basic tagged pre-allocation
of fabric descriptor sized memory.

v3: Update to percpu-ida usage

Cc: Kent Overstreet <kmo@daterainc.com>
Cc: Asias He <asias@redhat.com>
Cc: Michael S. Tsirkin <mst@redhat.com>
Signed-off-by: Nicholas Bellinger <nab@linux-iscsi.org>
---
 drivers/target/target_core_transport.c |   33 ++++++++++++++++++++++++++++++++
 include/target/target_core_base.h      |    5 +++++
 include/target/target_core_fabric.h    |    1 +
 3 files changed, 39 insertions(+)

diff --git a/drivers/target/target_core_transport.c b/drivers/target/target_core_transport.c
index 7172d00..64ceb51 100644
--- a/drivers/target/target_core_transport.c
+++ b/drivers/target/target_core_transport.c
@@ -232,6 +232,35 @@ struct se_session *transport_init_session(void)
 }
 EXPORT_SYMBOL(transport_init_session);
 
+struct se_session *transport_init_session_tags(unsigned int tag_num,
+					       unsigned int tag_size)
+{
+	struct se_session *se_sess;
+	int rc;
+
+	se_sess = transport_init_session();
+	if (IS_ERR(se_sess))
+		return se_sess;
+
+	se_sess->sess_cmd_map = kzalloc(tag_num * tag_size, GFP_KERNEL);
+	if (!se_sess->sess_cmd_map) {
+		pr_err("Unable to allocate se_sess->sess_cmd_map\n");
+		transport_free_session(se_sess);
+		return ERR_PTR(-ENOMEM);
+	}
+
+	rc = percpu_ida_init(&se_sess->sess_tag_pool, tag_num);
+	if (rc < 0) {
+		pr_err("Unable to init se_sess->sess_tag_pool,"
+			" tag_num: %u\n", tag_num);
+		transport_free_session(se_sess);
+		return ERR_PTR(-ENOMEM);
+	}
+
+	return se_sess;
+}
+EXPORT_SYMBOL(transport_init_session_tags);
+
 /*
  * Called with spin_lock_irqsave(&struct se_portal_group->session_lock called.
  */
@@ -367,6 +396,10 @@ EXPORT_SYMBOL(transport_deregister_session_configfs);
 
 void transport_free_session(struct se_session *se_sess)
 {
+	if (se_sess->sess_cmd_map) {
+		percpu_ida_destroy(&se_sess->sess_tag_pool);
+		kfree(se_sess->sess_cmd_map);
+	}
 	kmem_cache_free(se_sess_cache, se_sess);
 }
 EXPORT_SYMBOL(transport_free_session);
diff --git a/include/target/target_core_base.h b/include/target/target_core_base.h
index e34fc90..360e4a3 100644
--- a/include/target/target_core_base.h
+++ b/include/target/target_core_base.h
@@ -5,6 +5,7 @@
 #include <linux/configfs.h>
 #include <linux/dma-mapping.h>
 #include <linux/blkdev.h>
+#include <linux/idr.h>
 #include <scsi/scsi_cmnd.h>
 #include <net/sock.h>
 #include <net/tcp.h>
@@ -415,6 +416,8 @@ struct se_cmd {
 	enum dma_data_direction	data_direction;
 	/* For SAM Task Attribute */
 	int			sam_task_attr;
+	/* Used for se_sess->sess_tag_pool */
+	unsigned int		map_tag;
 	/* Transport protocol dependent state, see transport_state_table */
 	enum transport_state_table t_state;
 	unsigned		cmd_wait_set:1;
@@ -536,6 +539,8 @@ struct se_session {
 	struct list_head	sess_wait_list;
 	spinlock_t		sess_cmd_lock;
 	struct kref		sess_kref;
+	void			*sess_cmd_map;
+	struct percpu_ida	sess_tag_pool;
 };
 
 struct se_device;
diff --git a/include/target/target_core_fabric.h b/include/target/target_core_fabric.h
index 7a16178..b13afb6 100644
--- a/include/target/target_core_fabric.h
+++ b/include/target/target_core_fabric.h
@@ -84,6 +84,7 @@ struct target_core_fabric_ops {
 };
 
 struct se_session *transport_init_session(void);
+struct se_session *transport_init_session_tags(unsigned int, unsigned int);
 void	__transport_register_session(struct se_portal_group *,
 		struct se_node_acl *, struct se_session *, void *);
 void	transport_register_session(struct se_portal_group *,
-- 
1.7.10.4

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

* [PATCH-v3 3/4] vhost/scsi: Convert to per-cpu ida_alloc + ida_free command map
  2013-08-16 23:09 ` Nicholas A. Bellinger
@ 2013-08-16 23:09   ` Nicholas A. Bellinger
  -1 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-16 23:09 UTC (permalink / raw)
  To: target-devel
  Cc: lf-virt, lkml, kvm-devel, Michael S. Tsirkin, Asias He,
	Kent Overstreet, Andrew Morton, Jens Axboe, Tejun Heo,
	Ingo Molnar, Andi Kleen, Christoph Lameter, Nicholas Bellinger

From: Nicholas Bellinger <nab@linux-iscsi.org>

This patch changes vhost/scsi to use transport_init_session_tags()
pre-allocation logic for per-cpu session tag pooling with internal
ida_alloc() + ida_free() calls based upon the saved se_cmd->map_tag id.

FIXME: Make transport_init_session_tags() number of tags setup
configurable per vring client setting via configfs

v3 changes:
 - Update to percpu-ida usage
 - Rebase to v3.11-rc5 code

Cc: Michael S. Tsirkin <mst@redhat.com>
Cc: Asias He <asias@redhat.com>
Cc: Kent Overstreet <kmo@daterainc.com>
Signed-off-by: Nicholas Bellinger <nab@linux-iscsi.org>
---
 drivers/vhost/scsi.c |   33 +++++++++++++++++++++------------
 1 file changed, 21 insertions(+), 12 deletions(-)

diff --git a/drivers/vhost/scsi.c b/drivers/vhost/scsi.c
index 0c27c7d..af178b7 100644
--- a/drivers/vhost/scsi.c
+++ b/drivers/vhost/scsi.c
@@ -48,12 +48,14 @@
 #include <linux/virtio_scsi.h>
 #include <linux/llist.h>
 #include <linux/bitmap.h>
+#include <linux/idr.h>
 
 #include "vhost.h"
 
 #define TCM_VHOST_VERSION  "v0.1"
 #define TCM_VHOST_NAMELEN 256
 #define TCM_VHOST_MAX_CDB_SIZE 32
+#define TCM_VHOST_DEFAULT_TAGS 256
 
 struct vhost_scsi_inflight {
 	/* Wait for the flush operation to finish */
@@ -450,6 +452,7 @@ static void tcm_vhost_release_cmd(struct se_cmd *se_cmd)
 {
 	struct tcm_vhost_cmd *tv_cmd = container_of(se_cmd,
 				struct tcm_vhost_cmd, tvc_se_cmd);
+	struct se_session *se_sess = se_cmd->se_sess;
 
 	if (tv_cmd->tvc_sgl_count) {
 		u32 i;
@@ -460,7 +463,7 @@ static void tcm_vhost_release_cmd(struct se_cmd *se_cmd)
         }
 
 	tcm_vhost_put_inflight(tv_cmd->inflight);
-	kfree(tv_cmd);
+	percpu_ida_free(&se_sess->sess_tag_pool, se_cmd->map_tag);
 }
 
 static int tcm_vhost_shutdown_session(struct se_session *se_sess)
@@ -704,7 +707,7 @@ static void vhost_scsi_complete_cmd_work(struct vhost_work *work)
 }
 
 static struct tcm_vhost_cmd *
-vhost_scsi_allocate_cmd(struct vhost_virtqueue *vq,
+vhost_scsi_get_tag(struct vhost_virtqueue *vq,
 			struct tcm_vhost_tpg *tpg,
 			struct virtio_scsi_cmd_req *v_req,
 			u32 exp_data_len,
@@ -712,18 +715,21 @@ vhost_scsi_allocate_cmd(struct vhost_virtqueue *vq,
 {
 	struct tcm_vhost_cmd *cmd;
 	struct tcm_vhost_nexus *tv_nexus;
+	struct se_session *se_sess;
+	int tag;
 
 	tv_nexus = tpg->tpg_nexus;
 	if (!tv_nexus) {
 		pr_err("Unable to locate active struct tcm_vhost_nexus\n");
 		return ERR_PTR(-EIO);
 	}
+	se_sess = tv_nexus->tvn_se_sess;
 
-	cmd = kzalloc(sizeof(struct tcm_vhost_cmd), GFP_ATOMIC);
-	if (!cmd) {
-		pr_err("Unable to allocate struct tcm_vhost_cmd\n");
-		return ERR_PTR(-ENOMEM);
-	}
+	tag = percpu_ida_alloc(&se_sess->sess_tag_pool, GFP_KERNEL);
+	cmd = &((struct tcm_vhost_cmd *)se_sess->sess_cmd_map)[tag];
+	memset(cmd, 0, sizeof(struct tcm_vhost_cmd));
+
+	cmd->tvc_se_cmd.map_tag = tag;
 	cmd->tvc_tag = v_req->tag;
 	cmd->tvc_task_attr = v_req->task_attr;
 	cmd->tvc_exp_data_len = exp_data_len;
@@ -989,10 +995,10 @@ vhost_scsi_handle_vq(struct vhost_scsi *vs, struct vhost_virtqueue *vq)
 		for (i = 0; i < data_num; i++)
 			exp_data_len += vq->iov[data_first + i].iov_len;
 
-		cmd = vhost_scsi_allocate_cmd(vq, tpg, &v_req,
-					exp_data_len, data_direction);
+		cmd = vhost_scsi_get_tag(vq, tpg, &v_req,
+					 exp_data_len, data_direction);
 		if (IS_ERR(cmd)) {
-			vq_err(vq, "vhost_scsi_allocate_cmd failed %ld\n",
+			vq_err(vq, "vhost_scsi_get_tag failed %ld\n",
 					PTR_ERR(cmd));
 			goto err_cmd;
 		}
@@ -1675,9 +1681,12 @@ static int tcm_vhost_make_nexus(struct tcm_vhost_tpg *tpg,
 		return -ENOMEM;
 	}
 	/*
-	 *  Initialize the struct se_session pointer
+	 *  Initialize the struct se_session pointer and setup tagpool
+	 *  for struct tcm_vhost_cmd descriptors
 	 */
-	tv_nexus->tvn_se_sess = transport_init_session();
+	tv_nexus->tvn_se_sess = transport_init_session_tags(
+					TCM_VHOST_DEFAULT_TAGS,
+					sizeof(struct tcm_vhost_cmd));
 	if (IS_ERR(tv_nexus->tvn_se_sess)) {
 		mutex_unlock(&tpg->tv_tpg_mutex);
 		kfree(tv_nexus);
-- 
1.7.10.4


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

* [PATCH-v3 3/4] vhost/scsi: Convert to per-cpu ida_alloc + ida_free command map
@ 2013-08-16 23:09   ` Nicholas A. Bellinger
  0 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-16 23:09 UTC (permalink / raw)
  To: target-devel
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, Kent Overstreet, Ingo Molnar, Christoph Lameter,
	Tejun Heo, Andrew Morton

From: Nicholas Bellinger <nab@linux-iscsi.org>

This patch changes vhost/scsi to use transport_init_session_tags()
pre-allocation logic for per-cpu session tag pooling with internal
ida_alloc() + ida_free() calls based upon the saved se_cmd->map_tag id.

FIXME: Make transport_init_session_tags() number of tags setup
configurable per vring client setting via configfs

v3 changes:
 - Update to percpu-ida usage
 - Rebase to v3.11-rc5 code

Cc: Michael S. Tsirkin <mst@redhat.com>
Cc: Asias He <asias@redhat.com>
Cc: Kent Overstreet <kmo@daterainc.com>
Signed-off-by: Nicholas Bellinger <nab@linux-iscsi.org>
---
 drivers/vhost/scsi.c |   33 +++++++++++++++++++++------------
 1 file changed, 21 insertions(+), 12 deletions(-)

diff --git a/drivers/vhost/scsi.c b/drivers/vhost/scsi.c
index 0c27c7d..af178b7 100644
--- a/drivers/vhost/scsi.c
+++ b/drivers/vhost/scsi.c
@@ -48,12 +48,14 @@
 #include <linux/virtio_scsi.h>
 #include <linux/llist.h>
 #include <linux/bitmap.h>
+#include <linux/idr.h>
 
 #include "vhost.h"
 
 #define TCM_VHOST_VERSION  "v0.1"
 #define TCM_VHOST_NAMELEN 256
 #define TCM_VHOST_MAX_CDB_SIZE 32
+#define TCM_VHOST_DEFAULT_TAGS 256
 
 struct vhost_scsi_inflight {
 	/* Wait for the flush operation to finish */
@@ -450,6 +452,7 @@ static void tcm_vhost_release_cmd(struct se_cmd *se_cmd)
 {
 	struct tcm_vhost_cmd *tv_cmd = container_of(se_cmd,
 				struct tcm_vhost_cmd, tvc_se_cmd);
+	struct se_session *se_sess = se_cmd->se_sess;
 
 	if (tv_cmd->tvc_sgl_count) {
 		u32 i;
@@ -460,7 +463,7 @@ static void tcm_vhost_release_cmd(struct se_cmd *se_cmd)
         }
 
 	tcm_vhost_put_inflight(tv_cmd->inflight);
-	kfree(tv_cmd);
+	percpu_ida_free(&se_sess->sess_tag_pool, se_cmd->map_tag);
 }
 
 static int tcm_vhost_shutdown_session(struct se_session *se_sess)
@@ -704,7 +707,7 @@ static void vhost_scsi_complete_cmd_work(struct vhost_work *work)
 }
 
 static struct tcm_vhost_cmd *
-vhost_scsi_allocate_cmd(struct vhost_virtqueue *vq,
+vhost_scsi_get_tag(struct vhost_virtqueue *vq,
 			struct tcm_vhost_tpg *tpg,
 			struct virtio_scsi_cmd_req *v_req,
 			u32 exp_data_len,
@@ -712,18 +715,21 @@ vhost_scsi_allocate_cmd(struct vhost_virtqueue *vq,
 {
 	struct tcm_vhost_cmd *cmd;
 	struct tcm_vhost_nexus *tv_nexus;
+	struct se_session *se_sess;
+	int tag;
 
 	tv_nexus = tpg->tpg_nexus;
 	if (!tv_nexus) {
 		pr_err("Unable to locate active struct tcm_vhost_nexus\n");
 		return ERR_PTR(-EIO);
 	}
+	se_sess = tv_nexus->tvn_se_sess;
 
-	cmd = kzalloc(sizeof(struct tcm_vhost_cmd), GFP_ATOMIC);
-	if (!cmd) {
-		pr_err("Unable to allocate struct tcm_vhost_cmd\n");
-		return ERR_PTR(-ENOMEM);
-	}
+	tag = percpu_ida_alloc(&se_sess->sess_tag_pool, GFP_KERNEL);
+	cmd = &((struct tcm_vhost_cmd *)se_sess->sess_cmd_map)[tag];
+	memset(cmd, 0, sizeof(struct tcm_vhost_cmd));
+
+	cmd->tvc_se_cmd.map_tag = tag;
 	cmd->tvc_tag = v_req->tag;
 	cmd->tvc_task_attr = v_req->task_attr;
 	cmd->tvc_exp_data_len = exp_data_len;
@@ -989,10 +995,10 @@ vhost_scsi_handle_vq(struct vhost_scsi *vs, struct vhost_virtqueue *vq)
 		for (i = 0; i < data_num; i++)
 			exp_data_len += vq->iov[data_first + i].iov_len;
 
-		cmd = vhost_scsi_allocate_cmd(vq, tpg, &v_req,
-					exp_data_len, data_direction);
+		cmd = vhost_scsi_get_tag(vq, tpg, &v_req,
+					 exp_data_len, data_direction);
 		if (IS_ERR(cmd)) {
-			vq_err(vq, "vhost_scsi_allocate_cmd failed %ld\n",
+			vq_err(vq, "vhost_scsi_get_tag failed %ld\n",
 					PTR_ERR(cmd));
 			goto err_cmd;
 		}
@@ -1675,9 +1681,12 @@ static int tcm_vhost_make_nexus(struct tcm_vhost_tpg *tpg,
 		return -ENOMEM;
 	}
 	/*
-	 *  Initialize the struct se_session pointer
+	 *  Initialize the struct se_session pointer and setup tagpool
+	 *  for struct tcm_vhost_cmd descriptors
 	 */
-	tv_nexus->tvn_se_sess = transport_init_session();
+	tv_nexus->tvn_se_sess = transport_init_session_tags(
+					TCM_VHOST_DEFAULT_TAGS,
+					sizeof(struct tcm_vhost_cmd));
 	if (IS_ERR(tv_nexus->tvn_se_sess)) {
 		mutex_unlock(&tpg->tv_tpg_mutex);
 		kfree(tv_nexus);
-- 
1.7.10.4

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

* [PATCH-v3 4/4] vhost/scsi: Add pre-allocation for tv_cmd SGL + upages memory
  2013-08-16 23:09 ` Nicholas A. Bellinger
@ 2013-08-16 23:09   ` Nicholas A. Bellinger
  -1 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-16 23:09 UTC (permalink / raw)
  To: target-devel
  Cc: lf-virt, lkml, kvm-devel, Michael S. Tsirkin, Asias He,
	Kent Overstreet, Andrew Morton, Jens Axboe, Tejun Heo,
	Ingo Molnar, Andi Kleen, Christoph Lameter, Nicholas Bellinger

From: Nicholas Bellinger <nab@linux-iscsi.org>

This patch adds support for pre-allocation of per tv_cmd descriptor
scatterlist + user-space page pointer memory using se_sess->sess_cmd_map
within tcm_vhost_make_nexus() code.

This includes sanity checks within vhost_scsi_map_to_sgl()
to reject I/O that exceeds these initial hardcoded values, and
the necessary cleanup in tcm_vhost_make_nexus() failure path +
tcm_vhost_drop_nexus().

v3 changes:
  - Rebase to v3.11-rc5 code

Cc: Michael S. Tsirkin <mst@redhat.com>
Cc: Asias He <asias@redhat.com>
Cc: Kent Overstreet <kmo@daterainc.com>
Signed-off-by: Nicholas Bellinger <nab@linux-iscsi.org>
---
 drivers/vhost/scsi.c |   99 ++++++++++++++++++++++++++++++++++++++++----------
 1 file changed, 80 insertions(+), 19 deletions(-)

diff --git a/drivers/vhost/scsi.c b/drivers/vhost/scsi.c
index af178b7..8e8788e 100644
--- a/drivers/vhost/scsi.c
+++ b/drivers/vhost/scsi.c
@@ -56,6 +56,8 @@
 #define TCM_VHOST_NAMELEN 256
 #define TCM_VHOST_MAX_CDB_SIZE 32
 #define TCM_VHOST_DEFAULT_TAGS 256
+#define TCM_VHOST_PREALLOC_SGLS 2048
+#define TCM_VHOST_PREALLOC_PAGES 2048
 
 struct vhost_scsi_inflight {
 	/* Wait for the flush operation to finish */
@@ -81,6 +83,7 @@ struct tcm_vhost_cmd {
 	u32 tvc_lun;
 	/* Pointer to the SGL formatted memory from virtio-scsi */
 	struct scatterlist *tvc_sgl;
+	struct page **tvc_upages;
 	/* Pointer to response */
 	struct virtio_scsi_cmd_resp __user *tvc_resp;
 	/* Pointer to vhost_scsi for our device */
@@ -458,8 +461,6 @@ static void tcm_vhost_release_cmd(struct se_cmd *se_cmd)
 		u32 i;
 		for (i = 0; i < tv_cmd->tvc_sgl_count; i++)
 			put_page(sg_page(&tv_cmd->tvc_sgl[i]));
-
-		kfree(tv_cmd->tvc_sgl);
         }
 
 	tcm_vhost_put_inflight(tv_cmd->inflight);
@@ -716,6 +717,8 @@ vhost_scsi_get_tag(struct vhost_virtqueue *vq,
 	struct tcm_vhost_cmd *cmd;
 	struct tcm_vhost_nexus *tv_nexus;
 	struct se_session *se_sess;
+	struct scatterlist *sg;
+	struct page **pages;
 	int tag;
 
 	tv_nexus = tpg->tpg_nexus;
@@ -727,8 +730,12 @@ vhost_scsi_get_tag(struct vhost_virtqueue *vq,
 
 	tag = percpu_ida_alloc(&se_sess->sess_tag_pool, GFP_KERNEL);
 	cmd = &((struct tcm_vhost_cmd *)se_sess->sess_cmd_map)[tag];
+	sg = cmd->tvc_sgl;
+	pages = cmd->tvc_upages;
 	memset(cmd, 0, sizeof(struct tcm_vhost_cmd));
 
+	cmd->tvc_sgl = sg;
+	cmd->tvc_upages = pages;
 	cmd->tvc_se_cmd.map_tag = tag;
 	cmd->tvc_tag = v_req->tag;
 	cmd->tvc_task_attr = v_req->task_attr;
@@ -746,7 +753,8 @@ vhost_scsi_get_tag(struct vhost_virtqueue *vq,
  * Returns the number of scatterlist entries used or -errno on error.
  */
 static int
-vhost_scsi_map_to_sgl(struct scatterlist *sgl,
+vhost_scsi_map_to_sgl(struct tcm_vhost_cmd *tv_cmd,
+		      struct scatterlist *sgl,
 		      unsigned int sgl_count,
 		      struct iovec *iov,
 		      int write)
@@ -758,13 +766,25 @@ vhost_scsi_map_to_sgl(struct scatterlist *sgl,
 	struct page **pages;
 	int ret, i;
 
+	if (sgl_count > TCM_VHOST_PREALLOC_SGLS) {
+		pr_err("vhost_scsi_map_to_sgl() psgl_count: %u greater than"
+		       " preallocated TCM_VHOST_PREALLOC_SGLS: %u\n",
+			sgl_count, TCM_VHOST_PREALLOC_SGLS);
+		return -ENOBUFS;
+	}
+
 	pages_nr = iov_num_pages(iov);
 	if (pages_nr > sgl_count)
 		return -ENOBUFS;
 
-	pages = kmalloc(pages_nr * sizeof(struct page *), GFP_KERNEL);
-	if (!pages)
-		return -ENOMEM;
+	if (pages_nr > TCM_VHOST_PREALLOC_PAGES) {
+		pr_err("vhost_scsi_map_to_sgl() pages_nr: %u greater than"
+		       " preallocated TCM_VHOST_PREALLOC_PAGES: %u\n",
+			pages_nr, TCM_VHOST_PREALLOC_PAGES);
+		return -ENOBUFS;
+	}
+
+	pages = tv_cmd->tvc_upages;
 
 	ret = get_user_pages_fast((unsigned long)ptr, pages_nr, write, pages);
 	/* No pages were pinned */
@@ -789,7 +809,6 @@ vhost_scsi_map_to_sgl(struct scatterlist *sgl,
 	}
 
 out:
-	kfree(pages);
 	return ret;
 }
 
@@ -813,24 +832,20 @@ vhost_scsi_map_iov_to_sgl(struct tcm_vhost_cmd *cmd,
 
 	/* TODO overflow checking */
 
-	sg = kmalloc(sizeof(cmd->tvc_sgl[0]) * sgl_count, GFP_ATOMIC);
-	if (!sg)
-		return -ENOMEM;
-	pr_debug("%s sg %p sgl_count %u is_err %d\n", __func__,
-	       sg, sgl_count, !sg);
+	sg = cmd->tvc_sgl;
+	pr_debug("%s sg %p sgl_count %u\n", __func__, sg, sgl_count);
 	sg_init_table(sg, sgl_count);
 
-	cmd->tvc_sgl = sg;
 	cmd->tvc_sgl_count = sgl_count;
 
 	pr_debug("Mapping %u iovecs for %u pages\n", niov, sgl_count);
 	for (i = 0; i < niov; i++) {
-		ret = vhost_scsi_map_to_sgl(sg, sgl_count, &iov[i], write);
+		ret = vhost_scsi_map_to_sgl(cmd, sg, sgl_count, &iov[i],
+					    write);
 		if (ret < 0) {
 			for (i = 0; i < cmd->tvc_sgl_count; i++)
 				put_page(sg_page(&cmd->tvc_sgl[i]));
-			kfree(cmd->tvc_sgl);
-			cmd->tvc_sgl = NULL;
+
 			cmd->tvc_sgl_count = 0;
 			return ret;
 		}
@@ -1660,11 +1675,31 @@ static void tcm_vhost_drop_nodeacl(struct se_node_acl *se_acl)
 	kfree(nacl);
 }
 
+static void tcm_vhost_free_cmd_map_res(struct tcm_vhost_nexus *nexus,
+				       struct se_session *se_sess)
+{
+	struct tcm_vhost_cmd *tv_cmd;
+	unsigned int i;
+
+	if (!se_sess->sess_cmd_map)
+		return;
+
+	for (i = 0; i < TCM_VHOST_DEFAULT_TAGS; i++) {
+		tv_cmd = &((struct tcm_vhost_cmd *)se_sess->sess_cmd_map)[i];
+
+		kfree(tv_cmd->tvc_sgl);
+		kfree(tv_cmd->tvc_upages);
+	}
+}
+
 static int tcm_vhost_make_nexus(struct tcm_vhost_tpg *tpg,
 				const char *name)
 {
 	struct se_portal_group *se_tpg;
+	struct se_session *se_sess;
 	struct tcm_vhost_nexus *tv_nexus;
+	struct tcm_vhost_cmd *tv_cmd;
+	unsigned int i;
 
 	mutex_lock(&tpg->tv_tpg_mutex);
 	if (tpg->tpg_nexus) {
@@ -1692,6 +1727,26 @@ static int tcm_vhost_make_nexus(struct tcm_vhost_tpg *tpg,
 		kfree(tv_nexus);
 		return -ENOMEM;
 	}
+	se_sess = tv_nexus->tvn_se_sess;
+	for (i = 0; i < TCM_VHOST_DEFAULT_TAGS; i++) {
+		tv_cmd = &((struct tcm_vhost_cmd *)se_sess->sess_cmd_map)[i];
+
+		tv_cmd->tvc_sgl = kzalloc(sizeof(struct scatterlist) *
+					TCM_VHOST_PREALLOC_SGLS, GFP_KERNEL);
+		if (!tv_cmd->tvc_sgl) {
+			mutex_unlock(&tpg->tv_tpg_mutex);
+			pr_err("Unable to allocate tv_cmd->tvc_sgl\n");
+			goto out;
+		}
+
+		tv_cmd->tvc_upages = kzalloc(sizeof(struct page *) *
+					TCM_VHOST_PREALLOC_PAGES, GFP_KERNEL);
+		if (!tv_cmd->tvc_upages) {
+			mutex_unlock(&tpg->tv_tpg_mutex);
+			pr_err("Unable to allocate tv_cmd->tvc_upages\n");
+			goto out;
+		}
+	}
 	/*
 	 * Since we are running in 'demo mode' this call with generate a
 	 * struct se_node_acl for the tcm_vhost struct se_portal_group with
@@ -1703,9 +1758,7 @@ static int tcm_vhost_make_nexus(struct tcm_vhost_tpg *tpg,
 		mutex_unlock(&tpg->tv_tpg_mutex);
 		pr_debug("core_tpg_check_initiator_node_acl() failed"
 				" for %s\n", name);
-		transport_free_session(tv_nexus->tvn_se_sess);
-		kfree(tv_nexus);
-		return -ENOMEM;
+		goto out;
 	}
 	/*
 	 * Now register the TCM vhost virtual I_T Nexus as active with the
@@ -1717,6 +1770,12 @@ static int tcm_vhost_make_nexus(struct tcm_vhost_tpg *tpg,
 
 	mutex_unlock(&tpg->tv_tpg_mutex);
 	return 0;
+
+out:
+	tcm_vhost_free_cmd_map_res(tv_nexus, se_sess);
+	transport_free_session(se_sess);
+	kfree(tv_nexus);
+	return -ENOMEM;
 }
 
 static int tcm_vhost_drop_nexus(struct tcm_vhost_tpg *tpg)
@@ -1756,6 +1815,8 @@ static int tcm_vhost_drop_nexus(struct tcm_vhost_tpg *tpg)
 	pr_debug("TCM_vhost_ConfigFS: Removing I_T Nexus to emulated"
 		" %s Initiator Port: %s\n", tcm_vhost_dump_proto_id(tpg->tport),
 		tv_nexus->tvn_se_sess->se_node_acl->initiatorname);
+
+	tcm_vhost_free_cmd_map_res(tv_nexus, se_sess);
 	/*
 	 * Release the SCSI I_T Nexus to the emulated vhost Target Port
 	 */
-- 
1.7.10.4


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

* [PATCH-v3 4/4] vhost/scsi: Add pre-allocation for tv_cmd SGL + upages memory
@ 2013-08-16 23:09   ` Nicholas A. Bellinger
  0 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-16 23:09 UTC (permalink / raw)
  To: target-devel
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, Kent Overstreet, Ingo Molnar, Christoph Lameter,
	Tejun Heo, Andrew Morton

From: Nicholas Bellinger <nab@linux-iscsi.org>

This patch adds support for pre-allocation of per tv_cmd descriptor
scatterlist + user-space page pointer memory using se_sess->sess_cmd_map
within tcm_vhost_make_nexus() code.

This includes sanity checks within vhost_scsi_map_to_sgl()
to reject I/O that exceeds these initial hardcoded values, and
the necessary cleanup in tcm_vhost_make_nexus() failure path +
tcm_vhost_drop_nexus().

v3 changes:
  - Rebase to v3.11-rc5 code

Cc: Michael S. Tsirkin <mst@redhat.com>
Cc: Asias He <asias@redhat.com>
Cc: Kent Overstreet <kmo@daterainc.com>
Signed-off-by: Nicholas Bellinger <nab@linux-iscsi.org>
---
 drivers/vhost/scsi.c |   99 ++++++++++++++++++++++++++++++++++++++++----------
 1 file changed, 80 insertions(+), 19 deletions(-)

diff --git a/drivers/vhost/scsi.c b/drivers/vhost/scsi.c
index af178b7..8e8788e 100644
--- a/drivers/vhost/scsi.c
+++ b/drivers/vhost/scsi.c
@@ -56,6 +56,8 @@
 #define TCM_VHOST_NAMELEN 256
 #define TCM_VHOST_MAX_CDB_SIZE 32
 #define TCM_VHOST_DEFAULT_TAGS 256
+#define TCM_VHOST_PREALLOC_SGLS 2048
+#define TCM_VHOST_PREALLOC_PAGES 2048
 
 struct vhost_scsi_inflight {
 	/* Wait for the flush operation to finish */
@@ -81,6 +83,7 @@ struct tcm_vhost_cmd {
 	u32 tvc_lun;
 	/* Pointer to the SGL formatted memory from virtio-scsi */
 	struct scatterlist *tvc_sgl;
+	struct page **tvc_upages;
 	/* Pointer to response */
 	struct virtio_scsi_cmd_resp __user *tvc_resp;
 	/* Pointer to vhost_scsi for our device */
@@ -458,8 +461,6 @@ static void tcm_vhost_release_cmd(struct se_cmd *se_cmd)
 		u32 i;
 		for (i = 0; i < tv_cmd->tvc_sgl_count; i++)
 			put_page(sg_page(&tv_cmd->tvc_sgl[i]));
-
-		kfree(tv_cmd->tvc_sgl);
         }
 
 	tcm_vhost_put_inflight(tv_cmd->inflight);
@@ -716,6 +717,8 @@ vhost_scsi_get_tag(struct vhost_virtqueue *vq,
 	struct tcm_vhost_cmd *cmd;
 	struct tcm_vhost_nexus *tv_nexus;
 	struct se_session *se_sess;
+	struct scatterlist *sg;
+	struct page **pages;
 	int tag;
 
 	tv_nexus = tpg->tpg_nexus;
@@ -727,8 +730,12 @@ vhost_scsi_get_tag(struct vhost_virtqueue *vq,
 
 	tag = percpu_ida_alloc(&se_sess->sess_tag_pool, GFP_KERNEL);
 	cmd = &((struct tcm_vhost_cmd *)se_sess->sess_cmd_map)[tag];
+	sg = cmd->tvc_sgl;
+	pages = cmd->tvc_upages;
 	memset(cmd, 0, sizeof(struct tcm_vhost_cmd));
 
+	cmd->tvc_sgl = sg;
+	cmd->tvc_upages = pages;
 	cmd->tvc_se_cmd.map_tag = tag;
 	cmd->tvc_tag = v_req->tag;
 	cmd->tvc_task_attr = v_req->task_attr;
@@ -746,7 +753,8 @@ vhost_scsi_get_tag(struct vhost_virtqueue *vq,
  * Returns the number of scatterlist entries used or -errno on error.
  */
 static int
-vhost_scsi_map_to_sgl(struct scatterlist *sgl,
+vhost_scsi_map_to_sgl(struct tcm_vhost_cmd *tv_cmd,
+		      struct scatterlist *sgl,
 		      unsigned int sgl_count,
 		      struct iovec *iov,
 		      int write)
@@ -758,13 +766,25 @@ vhost_scsi_map_to_sgl(struct scatterlist *sgl,
 	struct page **pages;
 	int ret, i;
 
+	if (sgl_count > TCM_VHOST_PREALLOC_SGLS) {
+		pr_err("vhost_scsi_map_to_sgl() psgl_count: %u greater than"
+		       " preallocated TCM_VHOST_PREALLOC_SGLS: %u\n",
+			sgl_count, TCM_VHOST_PREALLOC_SGLS);
+		return -ENOBUFS;
+	}
+
 	pages_nr = iov_num_pages(iov);
 	if (pages_nr > sgl_count)
 		return -ENOBUFS;
 
-	pages = kmalloc(pages_nr * sizeof(struct page *), GFP_KERNEL);
-	if (!pages)
-		return -ENOMEM;
+	if (pages_nr > TCM_VHOST_PREALLOC_PAGES) {
+		pr_err("vhost_scsi_map_to_sgl() pages_nr: %u greater than"
+		       " preallocated TCM_VHOST_PREALLOC_PAGES: %u\n",
+			pages_nr, TCM_VHOST_PREALLOC_PAGES);
+		return -ENOBUFS;
+	}
+
+	pages = tv_cmd->tvc_upages;
 
 	ret = get_user_pages_fast((unsigned long)ptr, pages_nr, write, pages);
 	/* No pages were pinned */
@@ -789,7 +809,6 @@ vhost_scsi_map_to_sgl(struct scatterlist *sgl,
 	}
 
 out:
-	kfree(pages);
 	return ret;
 }
 
@@ -813,24 +832,20 @@ vhost_scsi_map_iov_to_sgl(struct tcm_vhost_cmd *cmd,
 
 	/* TODO overflow checking */
 
-	sg = kmalloc(sizeof(cmd->tvc_sgl[0]) * sgl_count, GFP_ATOMIC);
-	if (!sg)
-		return -ENOMEM;
-	pr_debug("%s sg %p sgl_count %u is_err %d\n", __func__,
-	       sg, sgl_count, !sg);
+	sg = cmd->tvc_sgl;
+	pr_debug("%s sg %p sgl_count %u\n", __func__, sg, sgl_count);
 	sg_init_table(sg, sgl_count);
 
-	cmd->tvc_sgl = sg;
 	cmd->tvc_sgl_count = sgl_count;
 
 	pr_debug("Mapping %u iovecs for %u pages\n", niov, sgl_count);
 	for (i = 0; i < niov; i++) {
-		ret = vhost_scsi_map_to_sgl(sg, sgl_count, &iov[i], write);
+		ret = vhost_scsi_map_to_sgl(cmd, sg, sgl_count, &iov[i],
+					    write);
 		if (ret < 0) {
 			for (i = 0; i < cmd->tvc_sgl_count; i++)
 				put_page(sg_page(&cmd->tvc_sgl[i]));
-			kfree(cmd->tvc_sgl);
-			cmd->tvc_sgl = NULL;
+
 			cmd->tvc_sgl_count = 0;
 			return ret;
 		}
@@ -1660,11 +1675,31 @@ static void tcm_vhost_drop_nodeacl(struct se_node_acl *se_acl)
 	kfree(nacl);
 }
 
+static void tcm_vhost_free_cmd_map_res(struct tcm_vhost_nexus *nexus,
+				       struct se_session *se_sess)
+{
+	struct tcm_vhost_cmd *tv_cmd;
+	unsigned int i;
+
+	if (!se_sess->sess_cmd_map)
+		return;
+
+	for (i = 0; i < TCM_VHOST_DEFAULT_TAGS; i++) {
+		tv_cmd = &((struct tcm_vhost_cmd *)se_sess->sess_cmd_map)[i];
+
+		kfree(tv_cmd->tvc_sgl);
+		kfree(tv_cmd->tvc_upages);
+	}
+}
+
 static int tcm_vhost_make_nexus(struct tcm_vhost_tpg *tpg,
 				const char *name)
 {
 	struct se_portal_group *se_tpg;
+	struct se_session *se_sess;
 	struct tcm_vhost_nexus *tv_nexus;
+	struct tcm_vhost_cmd *tv_cmd;
+	unsigned int i;
 
 	mutex_lock(&tpg->tv_tpg_mutex);
 	if (tpg->tpg_nexus) {
@@ -1692,6 +1727,26 @@ static int tcm_vhost_make_nexus(struct tcm_vhost_tpg *tpg,
 		kfree(tv_nexus);
 		return -ENOMEM;
 	}
+	se_sess = tv_nexus->tvn_se_sess;
+	for (i = 0; i < TCM_VHOST_DEFAULT_TAGS; i++) {
+		tv_cmd = &((struct tcm_vhost_cmd *)se_sess->sess_cmd_map)[i];
+
+		tv_cmd->tvc_sgl = kzalloc(sizeof(struct scatterlist) *
+					TCM_VHOST_PREALLOC_SGLS, GFP_KERNEL);
+		if (!tv_cmd->tvc_sgl) {
+			mutex_unlock(&tpg->tv_tpg_mutex);
+			pr_err("Unable to allocate tv_cmd->tvc_sgl\n");
+			goto out;
+		}
+
+		tv_cmd->tvc_upages = kzalloc(sizeof(struct page *) *
+					TCM_VHOST_PREALLOC_PAGES, GFP_KERNEL);
+		if (!tv_cmd->tvc_upages) {
+			mutex_unlock(&tpg->tv_tpg_mutex);
+			pr_err("Unable to allocate tv_cmd->tvc_upages\n");
+			goto out;
+		}
+	}
 	/*
 	 * Since we are running in 'demo mode' this call with generate a
 	 * struct se_node_acl for the tcm_vhost struct se_portal_group with
@@ -1703,9 +1758,7 @@ static int tcm_vhost_make_nexus(struct tcm_vhost_tpg *tpg,
 		mutex_unlock(&tpg->tv_tpg_mutex);
 		pr_debug("core_tpg_check_initiator_node_acl() failed"
 				" for %s\n", name);
-		transport_free_session(tv_nexus->tvn_se_sess);
-		kfree(tv_nexus);
-		return -ENOMEM;
+		goto out;
 	}
 	/*
 	 * Now register the TCM vhost virtual I_T Nexus as active with the
@@ -1717,6 +1770,12 @@ static int tcm_vhost_make_nexus(struct tcm_vhost_tpg *tpg,
 
 	mutex_unlock(&tpg->tv_tpg_mutex);
 	return 0;
+
+out:
+	tcm_vhost_free_cmd_map_res(tv_nexus, se_sess);
+	transport_free_session(se_sess);
+	kfree(tv_nexus);
+	return -ENOMEM;
 }
 
 static int tcm_vhost_drop_nexus(struct tcm_vhost_tpg *tpg)
@@ -1756,6 +1815,8 @@ static int tcm_vhost_drop_nexus(struct tcm_vhost_tpg *tpg)
 	pr_debug("TCM_vhost_ConfigFS: Removing I_T Nexus to emulated"
 		" %s Initiator Port: %s\n", tcm_vhost_dump_proto_id(tpg->tport),
 		tv_nexus->tvn_se_sess->se_node_acl->initiatorname);
+
+	tcm_vhost_free_cmd_map_res(tv_nexus, se_sess);
 	/*
 	 * Release the SCSI I_T Nexus to the emulated vhost Target Port
 	 */
-- 
1.7.10.4

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-16 23:09   ` Nicholas A. Bellinger
  (?)
  (?)
@ 2013-08-20 21:31   ` Andrew Morton
  2013-08-26 20:14     ` Kent Overstreet
                       ` (3 more replies)
  -1 siblings, 4 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-20 21:31 UTC (permalink / raw)
  To: Nicholas A. Bellinger
  Cc: target-devel, lf-virt, lkml, kvm-devel, Michael S. Tsirkin,
	Asias He, Kent Overstreet, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Fri, 16 Aug 2013 23:09:06 +0000 "Nicholas A. Bellinger" <nab@linux-iscsi.org> wrote:

> From: Kent Overstreet <kmo@daterainc.com>
> 
> Percpu frontend for allocating ids. With percpu allocation (that works),
> it's impossible to guarantee it will always be possible to allocate all
> nr_tags - typically, some will be stuck on a remote percpu freelist
> where the current job can't get to them.
> 
> We do guarantee that it will always be possible to allocate at least
> (nr_tags / 2) tags - this is done by keeping track of which and how many
> cpus have tags on their percpu freelists. On allocation failure if
> enough cpus have tags that there could potentially be (nr_tags / 2) tags
> stuck on remote percpu freelists, we then pick a remote cpu at random to
> steal from.
> 
> Note that there's no cpu hotplug notifier - we don't care, because
> steal_tags() will eventually get the down cpu's tags. We _could_ satisfy
> more allocations if we had a notifier - but we'll still meet our
> guarantees and it's absolutely not a correctness issue, so I don't think
> it's worth the extra code.
>
> ...
>
>  include/linux/idr.h |   53 +++++++++
>  lib/idr.c           |  316 +++++++++++++++++++++++++++++++++++++++++++++++++--

I don't think this should be in idr.[ch] at all.  It has no
relationship with the existing code.  Apart from duplicating its
functionality :(

> 
> ...
>
> @@ -243,4 +245,55 @@ static inline int ida_get_new(struct ida *ida, int *p_id)
>  
>  void __init idr_init_cache(void);
>  
> +/* Percpu IDA/tag allocator */
> +
> +struct percpu_ida_cpu;
> +
> +struct percpu_ida {
> +	/*
> +	 * number of tags available to be allocated, as passed to
> +	 * percpu_ida_init()
> +	 */
> +	unsigned			nr_tags;
> +
> +	struct percpu_ida_cpu __percpu	*tag_cpu;
> +
> +	/*
> +	 * Bitmap of cpus that (may) have tags on their percpu freelists:
> +	 * steal_tags() uses this to decide when to steal tags, and which cpus
> +	 * to try stealing from.
> +	 *
> +	 * It's ok for a freelist to be empty when its bit is set - steal_tags()
> +	 * will just keep looking - but the bitmap _must_ be set whenever a
> +	 * percpu freelist does have tags.
> +	 */
> +	unsigned long			*cpus_have_tags;

Why not cpumask_t?

> +	struct {
> +		spinlock_t		lock;
> +		/*
> +		 * When we go to steal tags from another cpu (see steal_tags()),
> +		 * we want to pick a cpu at random. Cycling through them every
> +		 * time we steal is a bit easier and more or less equivalent:
> +		 */
> +		unsigned		cpu_last_stolen;
> +
> +		/* For sleeping on allocation failure */
> +		wait_queue_head_t	wait;
> +
> +		/*
> +		 * Global freelist - it's a stack where nr_free points to the
> +		 * top
> +		 */
> +		unsigned		nr_free;
> +		unsigned		*freelist;
> +	} ____cacheline_aligned_in_smp;

Why the ____cacheline_aligned_in_smp?

> +};
> 
> ...
>
> +
> +/* Percpu IDA */
> +
> +/*
> + * Number of tags we move between the percpu freelist and the global freelist at
> + * a time

"between a percpu freelist" would be more accurate?

> + */
> +#define IDA_PCPU_BATCH_MOVE	32U
> +
> +/* Max size of percpu freelist, */
> +#define IDA_PCPU_SIZE		((IDA_PCPU_BATCH_MOVE * 3) / 2)
> +
> +struct percpu_ida_cpu {
> +	spinlock_t			lock;
> +	unsigned			nr_free;
> +	unsigned			freelist[];
> +};

Data structure needs documentation.  There's one of these per cpu.  I
guess nr_free and freelist are clear enough.  The presence of a lock
in a percpu data structure is a surprise.  It's for cross-cpu stealing,
I assume?

> +static inline void move_tags(unsigned *dst, unsigned *dst_nr,
> +			     unsigned *src, unsigned *src_nr,
> +			     unsigned nr)
> +{
> +	*src_nr -= nr;
> +	memcpy(dst + *dst_nr, src + *src_nr, sizeof(unsigned) * nr);
> +	*dst_nr += nr;
> +}
> +
> 
> ...
>
> +static inline void alloc_global_tags(struct percpu_ida *pool,
> +				     struct percpu_ida_cpu *tags)
> +{
> +	move_tags(tags->freelist, &tags->nr_free,
> +		  pool->freelist, &pool->nr_free,
> +		  min(pool->nr_free, IDA_PCPU_BATCH_MOVE));
> +}

Document this function?

> +static inline unsigned alloc_local_tag(struct percpu_ida *pool,
> +				       struct percpu_ida_cpu *tags)
> +{
> +	int tag = -ENOSPC;
> +
> +	spin_lock(&tags->lock);
> +	if (tags->nr_free)
> +		tag = tags->freelist[--tags->nr_free];
> +	spin_unlock(&tags->lock);
> +
> +	return tag;
> +}

I guess this one's clear enough, if the data structure relationships are
understood.

> +/**
> + * percpu_ida_alloc - allocate a tag
> + * @pool: pool to allocate from
> + * @gfp: gfp flags
> + *
> + * Returns a tag - an integer in the range [0..nr_tags) (passed to
> + * tag_pool_init()), or otherwise -ENOSPC on allocation failure.
> + *
> + * Safe to be called from interrupt context (assuming it isn't passed
> + * __GFP_WAIT, of course).
> + *
> + * Will not fail if passed __GFP_WAIT.
> + */
> +int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp)
> +{
> +	DEFINE_WAIT(wait);
> +	struct percpu_ida_cpu *tags;
> +	unsigned long flags;
> +	int tag;
> +
> +	local_irq_save(flags);
> +	tags = this_cpu_ptr(pool->tag_cpu);
> +
> +	/* Fastpath */
> +	tag = alloc_local_tag(pool, tags);
> +	if (likely(tag >= 0)) {
> +		local_irq_restore(flags);
> +		return tag;
> +	}
> +
> +	while (1) {
> +		spin_lock(&pool->lock);
> +
> +		/*
> +		 * prepare_to_wait() must come before steal_tags(), in case
> +		 * percpu_ida_free() on another cpu flips a bit in
> +		 * cpus_have_tags
> +		 *
> +		 * global lock held and irqs disabled, don't need percpu lock
> +		 */
> +		prepare_to_wait(&pool->wait, &wait, TASK_UNINTERRUPTIBLE);
> +
> +		if (!tags->nr_free)
> +			alloc_global_tags(pool, tags);
> +		if (!tags->nr_free)
> +			steal_tags(pool, tags);
> +
> +		if (tags->nr_free) {
> +			tag = tags->freelist[--tags->nr_free];
> +			if (tags->nr_free)
> +				set_bit(smp_processor_id(),
> +					pool->cpus_have_tags);
> +		}
> +
> +		spin_unlock(&pool->lock);
> +		local_irq_restore(flags);
> +
> +		if (tag >= 0 || !(gfp & __GFP_WAIT))
> +			break;
> +
> +		schedule();
> +
> +		local_irq_save(flags);
> +		tags = this_cpu_ptr(pool->tag_cpu);
> +	}

What guarantees that this wait will terminate?

> +	finish_wait(&pool->wait, &wait);
> +	return tag;
> +}
> +EXPORT_SYMBOL_GPL(percpu_ida_alloc);
> +
> +/**
> + * percpu_ida_free - free a tag
> + * @pool: pool @tag was allocated from
> + * @tag: a tag previously allocated with percpu_ida_alloc()
> + *
> + * Safe to be called from interrupt context.
> + */
> +void percpu_ida_free(struct percpu_ida *pool, unsigned tag)
> +{
> +	struct percpu_ida_cpu *tags;
> +	unsigned long flags;
> +	unsigned nr_free;
> +
> +	BUG_ON(tag >= pool->nr_tags);
> +
> +	local_irq_save(flags);
> +	tags = this_cpu_ptr(pool->tag_cpu);
> +
> +	spin_lock(&tags->lock);

Why do we need this lock, btw?  It's a cpu-local structure and local
irqs are disabled...

> +	tags->freelist[tags->nr_free++] = tag;
> +
> +	nr_free = tags->nr_free;
> +	spin_unlock(&tags->lock);
> +
> +	if (nr_free == 1) {
> +		set_bit(smp_processor_id(),
> +			pool->cpus_have_tags);
> +		wake_up(&pool->wait);
> +	}
> +
> +	if (nr_free == IDA_PCPU_SIZE) {
> +		spin_lock(&pool->lock);
> +
> +		/*
> +		 * Global lock held and irqs disabled, don't need percpu
> +		 * lock
> +		 */
> +		if (tags->nr_free == IDA_PCPU_SIZE) {
> +			move_tags(pool->freelist, &pool->nr_free,
> +				  tags->freelist, &tags->nr_free,
> +				  IDA_PCPU_BATCH_MOVE);
> +
> +			wake_up(&pool->wait);
> +		}
> +		spin_unlock(&pool->lock);
> +	}
> +
> +	local_irq_restore(flags);
> +}
> +EXPORT_SYMBOL_GPL(percpu_ida_free);
> 
> ...
>
> +int percpu_ida_init(struct percpu_ida *pool, unsigned long nr_tags)
> +{
> +	unsigned i, cpu, order;
> +
> +	memset(pool, 0, sizeof(*pool));
> +
> +	init_waitqueue_head(&pool->wait);
> +	spin_lock_init(&pool->lock);
> +	pool->nr_tags = nr_tags;
> +
> +	/* Guard against overflow */
> +	if (nr_tags > (unsigned) INT_MAX + 1) {
> +		pr_err("tags.c: nr_tags too large\n");

"tags.c"?

> +		return -EINVAL;
> +	}
> +
> +	order = get_order(nr_tags * sizeof(unsigned));
> +	pool->freelist = (void *) __get_free_pages(GFP_KERNEL, order);
> +	if (!pool->freelist)
> +		return -ENOMEM;
> +
> +	for (i = 0; i < nr_tags; i++)
> +		pool->freelist[i] = i;
> +
> +	pool->nr_free = nr_tags;
> +
> +	pool->cpus_have_tags = kzalloc(BITS_TO_LONGS(nr_cpu_ids) *
> +				       sizeof(unsigned long), GFP_KERNEL);
> +	if (!pool->cpus_have_tags)
> +		goto err;
> +
> +	pool->tag_cpu = __alloc_percpu(sizeof(struct percpu_ida_cpu) +
> +				       IDA_PCPU_SIZE * sizeof(unsigned),
> +				       sizeof(unsigned));
> +	if (!pool->tag_cpu)
> +		goto err;
> +
> +	for_each_possible_cpu(cpu)
> +		spin_lock_init(&per_cpu_ptr(pool->tag_cpu, cpu)->lock);
> +
> +	return 0;
> +err:
> +	percpu_ida_destroy(pool);
> +	return -ENOMEM;
> +}
> +EXPORT_SYMBOL_GPL(percpu_ida_init);

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-16 23:09   ` Nicholas A. Bellinger
  (?)
@ 2013-08-20 21:31   ` Andrew Morton
  -1 siblings, 0 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-20 21:31 UTC (permalink / raw)
  To: Nicholas A. Bellinger
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, Kent Overstreet, target-devel, Christoph Lameter,
	Oleg Nesterov, Tejun Heo, Christoph Lameter, Ingo Molnar

On Fri, 16 Aug 2013 23:09:06 +0000 "Nicholas A. Bellinger" <nab@linux-iscsi.org> wrote:

> From: Kent Overstreet <kmo@daterainc.com>
> 
> Percpu frontend for allocating ids. With percpu allocation (that works),
> it's impossible to guarantee it will always be possible to allocate all
> nr_tags - typically, some will be stuck on a remote percpu freelist
> where the current job can't get to them.
> 
> We do guarantee that it will always be possible to allocate at least
> (nr_tags / 2) tags - this is done by keeping track of which and how many
> cpus have tags on their percpu freelists. On allocation failure if
> enough cpus have tags that there could potentially be (nr_tags / 2) tags
> stuck on remote percpu freelists, we then pick a remote cpu at random to
> steal from.
> 
> Note that there's no cpu hotplug notifier - we don't care, because
> steal_tags() will eventually get the down cpu's tags. We _could_ satisfy
> more allocations if we had a notifier - but we'll still meet our
> guarantees and it's absolutely not a correctness issue, so I don't think
> it's worth the extra code.
>
> ...
>
>  include/linux/idr.h |   53 +++++++++
>  lib/idr.c           |  316 +++++++++++++++++++++++++++++++++++++++++++++++++--

I don't think this should be in idr.[ch] at all.  It has no
relationship with the existing code.  Apart from duplicating its
functionality :(

> 
> ...
>
> @@ -243,4 +245,55 @@ static inline int ida_get_new(struct ida *ida, int *p_id)
>  
>  void __init idr_init_cache(void);
>  
> +/* Percpu IDA/tag allocator */
> +
> +struct percpu_ida_cpu;
> +
> +struct percpu_ida {
> +	/*
> +	 * number of tags available to be allocated, as passed to
> +	 * percpu_ida_init()
> +	 */
> +	unsigned			nr_tags;
> +
> +	struct percpu_ida_cpu __percpu	*tag_cpu;
> +
> +	/*
> +	 * Bitmap of cpus that (may) have tags on their percpu freelists:
> +	 * steal_tags() uses this to decide when to steal tags, and which cpus
> +	 * to try stealing from.
> +	 *
> +	 * It's ok for a freelist to be empty when its bit is set - steal_tags()
> +	 * will just keep looking - but the bitmap _must_ be set whenever a
> +	 * percpu freelist does have tags.
> +	 */
> +	unsigned long			*cpus_have_tags;

Why not cpumask_t?

> +	struct {
> +		spinlock_t		lock;
> +		/*
> +		 * When we go to steal tags from another cpu (see steal_tags()),
> +		 * we want to pick a cpu at random. Cycling through them every
> +		 * time we steal is a bit easier and more or less equivalent:
> +		 */
> +		unsigned		cpu_last_stolen;
> +
> +		/* For sleeping on allocation failure */
> +		wait_queue_head_t	wait;
> +
> +		/*
> +		 * Global freelist - it's a stack where nr_free points to the
> +		 * top
> +		 */
> +		unsigned		nr_free;
> +		unsigned		*freelist;
> +	} ____cacheline_aligned_in_smp;

Why the ____cacheline_aligned_in_smp?

> +};
> 
> ...
>
> +
> +/* Percpu IDA */
> +
> +/*
> + * Number of tags we move between the percpu freelist and the global freelist at
> + * a time

"between a percpu freelist" would be more accurate?

> + */
> +#define IDA_PCPU_BATCH_MOVE	32U
> +
> +/* Max size of percpu freelist, */
> +#define IDA_PCPU_SIZE		((IDA_PCPU_BATCH_MOVE * 3) / 2)
> +
> +struct percpu_ida_cpu {
> +	spinlock_t			lock;
> +	unsigned			nr_free;
> +	unsigned			freelist[];
> +};

Data structure needs documentation.  There's one of these per cpu.  I
guess nr_free and freelist are clear enough.  The presence of a lock
in a percpu data structure is a surprise.  It's for cross-cpu stealing,
I assume?

> +static inline void move_tags(unsigned *dst, unsigned *dst_nr,
> +			     unsigned *src, unsigned *src_nr,
> +			     unsigned nr)
> +{
> +	*src_nr -= nr;
> +	memcpy(dst + *dst_nr, src + *src_nr, sizeof(unsigned) * nr);
> +	*dst_nr += nr;
> +}
> +
> 
> ...
>
> +static inline void alloc_global_tags(struct percpu_ida *pool,
> +				     struct percpu_ida_cpu *tags)
> +{
> +	move_tags(tags->freelist, &tags->nr_free,
> +		  pool->freelist, &pool->nr_free,
> +		  min(pool->nr_free, IDA_PCPU_BATCH_MOVE));
> +}

Document this function?

> +static inline unsigned alloc_local_tag(struct percpu_ida *pool,
> +				       struct percpu_ida_cpu *tags)
> +{
> +	int tag = -ENOSPC;
> +
> +	spin_lock(&tags->lock);
> +	if (tags->nr_free)
> +		tag = tags->freelist[--tags->nr_free];
> +	spin_unlock(&tags->lock);
> +
> +	return tag;
> +}

I guess this one's clear enough, if the data structure relationships are
understood.

> +/**
> + * percpu_ida_alloc - allocate a tag
> + * @pool: pool to allocate from
> + * @gfp: gfp flags
> + *
> + * Returns a tag - an integer in the range [0..nr_tags) (passed to
> + * tag_pool_init()), or otherwise -ENOSPC on allocation failure.
> + *
> + * Safe to be called from interrupt context (assuming it isn't passed
> + * __GFP_WAIT, of course).
> + *
> + * Will not fail if passed __GFP_WAIT.
> + */
> +int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp)
> +{
> +	DEFINE_WAIT(wait);
> +	struct percpu_ida_cpu *tags;
> +	unsigned long flags;
> +	int tag;
> +
> +	local_irq_save(flags);
> +	tags = this_cpu_ptr(pool->tag_cpu);
> +
> +	/* Fastpath */
> +	tag = alloc_local_tag(pool, tags);
> +	if (likely(tag >= 0)) {
> +		local_irq_restore(flags);
> +		return tag;
> +	}
> +
> +	while (1) {
> +		spin_lock(&pool->lock);
> +
> +		/*
> +		 * prepare_to_wait() must come before steal_tags(), in case
> +		 * percpu_ida_free() on another cpu flips a bit in
> +		 * cpus_have_tags
> +		 *
> +		 * global lock held and irqs disabled, don't need percpu lock
> +		 */
> +		prepare_to_wait(&pool->wait, &wait, TASK_UNINTERRUPTIBLE);
> +
> +		if (!tags->nr_free)
> +			alloc_global_tags(pool, tags);
> +		if (!tags->nr_free)
> +			steal_tags(pool, tags);
> +
> +		if (tags->nr_free) {
> +			tag = tags->freelist[--tags->nr_free];
> +			if (tags->nr_free)
> +				set_bit(smp_processor_id(),
> +					pool->cpus_have_tags);
> +		}
> +
> +		spin_unlock(&pool->lock);
> +		local_irq_restore(flags);
> +
> +		if (tag >= 0 || !(gfp & __GFP_WAIT))
> +			break;
> +
> +		schedule();
> +
> +		local_irq_save(flags);
> +		tags = this_cpu_ptr(pool->tag_cpu);
> +	}

What guarantees that this wait will terminate?

> +	finish_wait(&pool->wait, &wait);
> +	return tag;
> +}
> +EXPORT_SYMBOL_GPL(percpu_ida_alloc);
> +
> +/**
> + * percpu_ida_free - free a tag
> + * @pool: pool @tag was allocated from
> + * @tag: a tag previously allocated with percpu_ida_alloc()
> + *
> + * Safe to be called from interrupt context.
> + */
> +void percpu_ida_free(struct percpu_ida *pool, unsigned tag)
> +{
> +	struct percpu_ida_cpu *tags;
> +	unsigned long flags;
> +	unsigned nr_free;
> +
> +	BUG_ON(tag >= pool->nr_tags);
> +
> +	local_irq_save(flags);
> +	tags = this_cpu_ptr(pool->tag_cpu);
> +
> +	spin_lock(&tags->lock);

Why do we need this lock, btw?  It's a cpu-local structure and local
irqs are disabled...

> +	tags->freelist[tags->nr_free++] = tag;
> +
> +	nr_free = tags->nr_free;
> +	spin_unlock(&tags->lock);
> +
> +	if (nr_free == 1) {
> +		set_bit(smp_processor_id(),
> +			pool->cpus_have_tags);
> +		wake_up(&pool->wait);
> +	}
> +
> +	if (nr_free == IDA_PCPU_SIZE) {
> +		spin_lock(&pool->lock);
> +
> +		/*
> +		 * Global lock held and irqs disabled, don't need percpu
> +		 * lock
> +		 */
> +		if (tags->nr_free == IDA_PCPU_SIZE) {
> +			move_tags(pool->freelist, &pool->nr_free,
> +				  tags->freelist, &tags->nr_free,
> +				  IDA_PCPU_BATCH_MOVE);
> +
> +			wake_up(&pool->wait);
> +		}
> +		spin_unlock(&pool->lock);
> +	}
> +
> +	local_irq_restore(flags);
> +}
> +EXPORT_SYMBOL_GPL(percpu_ida_free);
> 
> ...
>
> +int percpu_ida_init(struct percpu_ida *pool, unsigned long nr_tags)
> +{
> +	unsigned i, cpu, order;
> +
> +	memset(pool, 0, sizeof(*pool));
> +
> +	init_waitqueue_head(&pool->wait);
> +	spin_lock_init(&pool->lock);
> +	pool->nr_tags = nr_tags;
> +
> +	/* Guard against overflow */
> +	if (nr_tags > (unsigned) INT_MAX + 1) {
> +		pr_err("tags.c: nr_tags too large\n");

"tags.c"?

> +		return -EINVAL;
> +	}
> +
> +	order = get_order(nr_tags * sizeof(unsigned));
> +	pool->freelist = (void *) __get_free_pages(GFP_KERNEL, order);
> +	if (!pool->freelist)
> +		return -ENOMEM;
> +
> +	for (i = 0; i < nr_tags; i++)
> +		pool->freelist[i] = i;
> +
> +	pool->nr_free = nr_tags;
> +
> +	pool->cpus_have_tags = kzalloc(BITS_TO_LONGS(nr_cpu_ids) *
> +				       sizeof(unsigned long), GFP_KERNEL);
> +	if (!pool->cpus_have_tags)
> +		goto err;
> +
> +	pool->tag_cpu = __alloc_percpu(sizeof(struct percpu_ida_cpu) +
> +				       IDA_PCPU_SIZE * sizeof(unsigned),
> +				       sizeof(unsigned));
> +	if (!pool->tag_cpu)
> +		goto err;
> +
> +	for_each_possible_cpu(cpu)
> +		spin_lock_init(&per_cpu_ptr(pool->tag_cpu, cpu)->lock);
> +
> +	return 0;
> +err:
> +	percpu_ida_destroy(pool);
> +	return -ENOMEM;
> +}
> +EXPORT_SYMBOL_GPL(percpu_ida_init);

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-16 23:09   ` Nicholas A. Bellinger
@ 2013-08-21 18:25     ` Christoph Lameter
  -1 siblings, 0 replies; 45+ messages in thread
From: Christoph Lameter @ 2013-08-21 18:25 UTC (permalink / raw)
  To: Nicholas A. Bellinger
  Cc: target-devel, lf-virt, lkml, kvm-devel, Michael S. Tsirkin,
	Asias He, Kent Overstreet, Andrew Morton, Jens Axboe, Tejun Heo,
	Ingo Molnar, Andi Kleen, Oleg Nesterov

On Fri, 16 Aug 2013, Nicholas A. Bellinger wrote:

> +	spinlock_t			lock;

Remove the spinlock.

> +	unsigned			nr_free;
> +	unsigned			freelist[];
> +};
> +
> +static inline void move_tags(unsigned *dst, unsigned *dst_nr,
> +			     unsigned *src, unsigned *src_nr,
> +			     unsigned nr)
> +{
> +	*src_nr -= nr;
> +	memcpy(dst + *dst_nr, src + *src_nr, sizeof(unsigned) * nr);
> +	*dst_nr += nr;
> +}
> +

> +static inline unsigned alloc_local_tag(struct percpu_ida *pool,
> +				       struct percpu_ida_cpu *tags)

Pass the __percpu offset and not the tags pointer.

> +{
> +	int tag = -ENOSPC;
> +
> +	spin_lock(&tags->lock);

Interupts are already disabled. Drop the spinlock.

> +	if (tags->nr_free)
> +		tag = tags->freelist[--tags->nr_free];

You can keep this or avoid address calculation through segment prefixes.
F.e.

if (__this_cpu_read(tags->nrfree) {
	int n = __this_cpu_dec_return(tags->nr_free);
	tag =  __this_cpu_read(tags->freelist[n]);
}

> +	spin_unlock(&tags->lock);

Drop.

> + * Returns a tag - an integer in the range [0..nr_tags) (passed to
> + * tag_pool_init()), or otherwise -ENOSPC on allocation failure.
> + *
> + * Safe to be called from interrupt context (assuming it isn't passed
> + * __GFP_WAIT, of course).
> + *
> + * Will not fail if passed __GFP_WAIT.
> + */
> +int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp)
> +{
> +	DEFINE_WAIT(wait);
> +	struct percpu_ida_cpu *tags;
> +	unsigned long flags;
> +	int tag;
> +
> +	local_irq_save(flags);
> +	tags = this_cpu_ptr(pool->tag_cpu);

You could drop this_cpu_ptr if you pass pool->tag_cpu to alloc_local_tag.

> +/**
> + * percpu_ida_free - free a tag
> + * @pool: pool @tag was allocated from
> + * @tag: a tag previously allocated with percpu_ida_alloc()
> + *
> + * Safe to be called from interrupt context.
> + */
> +void percpu_ida_free(struct percpu_ida *pool, unsigned tag)
> +{
> +	struct percpu_ida_cpu *tags;
> +	unsigned long flags;
> +	unsigned nr_free;
> +
> +	BUG_ON(tag >= pool->nr_tags);
> +
> +	local_irq_save(flags);
> +	tags = this_cpu_ptr(pool->tag_cpu);
> +
> +	spin_lock(&tags->lock);

No need for spinlocking
> +	tags->freelist[tags->nr_free++] = tag;

nr_free = __this_cpu_inc_return(pool->tag_cpu.nr_free) ?

__this_cpu_write(pool->tag_cpu.freelist[nr_free], tag)



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

* Re: [PATCH-v3 1/4] idr: Percpu ida
@ 2013-08-21 18:25     ` Christoph Lameter
  0 siblings, 0 replies; 45+ messages in thread
From: Christoph Lameter @ 2013-08-21 18:25 UTC (permalink / raw)
  To: Nicholas A. Bellinger
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, Kent Overstreet, target-devel, Oleg Nesterov, Tejun Heo,
	Ingo Molnar, Andrew Morton

On Fri, 16 Aug 2013, Nicholas A. Bellinger wrote:

> +	spinlock_t			lock;

Remove the spinlock.

> +	unsigned			nr_free;
> +	unsigned			freelist[];
> +};
> +
> +static inline void move_tags(unsigned *dst, unsigned *dst_nr,
> +			     unsigned *src, unsigned *src_nr,
> +			     unsigned nr)
> +{
> +	*src_nr -= nr;
> +	memcpy(dst + *dst_nr, src + *src_nr, sizeof(unsigned) * nr);
> +	*dst_nr += nr;
> +}
> +

> +static inline unsigned alloc_local_tag(struct percpu_ida *pool,
> +				       struct percpu_ida_cpu *tags)

Pass the __percpu offset and not the tags pointer.

> +{
> +	int tag = -ENOSPC;
> +
> +	spin_lock(&tags->lock);

Interupts are already disabled. Drop the spinlock.

> +	if (tags->nr_free)
> +		tag = tags->freelist[--tags->nr_free];

You can keep this or avoid address calculation through segment prefixes.
F.e.

if (__this_cpu_read(tags->nrfree) {
	int n = __this_cpu_dec_return(tags->nr_free);
	tag =  __this_cpu_read(tags->freelist[n]);
}

> +	spin_unlock(&tags->lock);

Drop.

> + * Returns a tag - an integer in the range [0..nr_tags) (passed to
> + * tag_pool_init()), or otherwise -ENOSPC on allocation failure.
> + *
> + * Safe to be called from interrupt context (assuming it isn't passed
> + * __GFP_WAIT, of course).
> + *
> + * Will not fail if passed __GFP_WAIT.
> + */
> +int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp)
> +{
> +	DEFINE_WAIT(wait);
> +	struct percpu_ida_cpu *tags;
> +	unsigned long flags;
> +	int tag;
> +
> +	local_irq_save(flags);
> +	tags = this_cpu_ptr(pool->tag_cpu);

You could drop this_cpu_ptr if you pass pool->tag_cpu to alloc_local_tag.

> +/**
> + * percpu_ida_free - free a tag
> + * @pool: pool @tag was allocated from
> + * @tag: a tag previously allocated with percpu_ida_alloc()
> + *
> + * Safe to be called from interrupt context.
> + */
> +void percpu_ida_free(struct percpu_ida *pool, unsigned tag)
> +{
> +	struct percpu_ida_cpu *tags;
> +	unsigned long flags;
> +	unsigned nr_free;
> +
> +	BUG_ON(tag >= pool->nr_tags);
> +
> +	local_irq_save(flags);
> +	tags = this_cpu_ptr(pool->tag_cpu);
> +
> +	spin_lock(&tags->lock);

No need for spinlocking
> +	tags->freelist[tags->nr_free++] = tag;

nr_free = __this_cpu_inc_return(pool->tag_cpu.nr_free) ?

__this_cpu_write(pool->tag_cpu.freelist[nr_free], tag)

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-20 21:31   ` Andrew Morton
  2013-08-26 20:14     ` Kent Overstreet
@ 2013-08-26 20:14     ` Kent Overstreet
  2013-08-28 19:53       ` Kent Overstreet
  2013-08-28 19:55       ` Kent Overstreet
  3 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-26 20:14 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Tue, Aug 20, 2013 at 02:31:57PM -0700, Andrew Morton wrote:
> On Fri, 16 Aug 2013 23:09:06 +0000 "Nicholas A. Bellinger" <nab@linux-iscsi.org> wrote:
> 
> > From: Kent Overstreet <kmo@daterainc.com>
> > 
> > Percpu frontend for allocating ids. With percpu allocation (that works),
> > it's impossible to guarantee it will always be possible to allocate all
> > nr_tags - typically, some will be stuck on a remote percpu freelist
> > where the current job can't get to them.
> > 
> > We do guarantee that it will always be possible to allocate at least
> > (nr_tags / 2) tags - this is done by keeping track of which and how many
> > cpus have tags on their percpu freelists. On allocation failure if
> > enough cpus have tags that there could potentially be (nr_tags / 2) tags
> > stuck on remote percpu freelists, we then pick a remote cpu at random to
> > steal from.
> > 
> > Note that there's no cpu hotplug notifier - we don't care, because
> > steal_tags() will eventually get the down cpu's tags. We _could_ satisfy
> > more allocations if we had a notifier - but we'll still meet our
> > guarantees and it's absolutely not a correctness issue, so I don't think
> > it's worth the extra code.
> >
> > ...
> >
> >  include/linux/idr.h |   53 +++++++++
> >  lib/idr.c           |  316 +++++++++++++++++++++++++++++++++++++++++++++++++--
> 
> I don't think this should be in idr.[ch] at all.  It has no
> relationship with the existing code.  Apart from duplicating its
> functionality :(

Well, in the full patch series it does make use of the non-percpu ida.
I'm still hoping to get the ida/idr rewrites in.

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-20 21:31   ` Andrew Morton
@ 2013-08-26 20:14     ` Kent Overstreet
  2013-08-26 20:14     ` Kent Overstreet
                       ` (2 subsequent siblings)
  3 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-26 20:14 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Tue, Aug 20, 2013 at 02:31:57PM -0700, Andrew Morton wrote:
> On Fri, 16 Aug 2013 23:09:06 +0000 "Nicholas A. Bellinger" <nab@linux-iscsi.org> wrote:
> 
> > From: Kent Overstreet <kmo@daterainc.com>
> > 
> > Percpu frontend for allocating ids. With percpu allocation (that works),
> > it's impossible to guarantee it will always be possible to allocate all
> > nr_tags - typically, some will be stuck on a remote percpu freelist
> > where the current job can't get to them.
> > 
> > We do guarantee that it will always be possible to allocate at least
> > (nr_tags / 2) tags - this is done by keeping track of which and how many
> > cpus have tags on their percpu freelists. On allocation failure if
> > enough cpus have tags that there could potentially be (nr_tags / 2) tags
> > stuck on remote percpu freelists, we then pick a remote cpu at random to
> > steal from.
> > 
> > Note that there's no cpu hotplug notifier - we don't care, because
> > steal_tags() will eventually get the down cpu's tags. We _could_ satisfy
> > more allocations if we had a notifier - but we'll still meet our
> > guarantees and it's absolutely not a correctness issue, so I don't think
> > it's worth the extra code.
> >
> > ...
> >
> >  include/linux/idr.h |   53 +++++++++
> >  lib/idr.c           |  316 +++++++++++++++++++++++++++++++++++++++++++++++++--
> 
> I don't think this should be in idr.[ch] at all.  It has no
> relationship with the existing code.  Apart from duplicating its
> functionality :(

Well, in the full patch series it does make use of the non-percpu ida.
I'm still hoping to get the ida/idr rewrites in.

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-21 18:25     ` Christoph Lameter
  (?)
  (?)
@ 2013-08-26 20:23     ` Kent Overstreet
  -1 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-26 20:23 UTC (permalink / raw)
  To: Christoph Lameter
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Andrew Morton, Jens Axboe,
	Tejun Heo, Ingo Molnar, Andi Kleen, Oleg Nesterov

On Wed, Aug 21, 2013 at 06:25:58PM +0000, Christoph Lameter wrote:
> On Fri, 16 Aug 2013, Nicholas A. Bellinger wrote:
> 
> > +	spinlock_t			lock;
> 
> Remove the spinlock.

As Andrew noted, the spinlock is needed because of tag stealing. (You
don't think I'd stick a spinlock on a percpu data structure without a
real reason, would you?)

> > +	unsigned			nr_free;
> > +	unsigned			freelist[];
> > +};
> > +
> > +static inline void move_tags(unsigned *dst, unsigned *dst_nr,
> > +			     unsigned *src, unsigned *src_nr,
> > +			     unsigned nr)
> > +{
> > +	*src_nr -= nr;
> > +	memcpy(dst + *dst_nr, src + *src_nr, sizeof(unsigned) * nr);
> > +	*dst_nr += nr;
> > +}
> > +
> 
> > +static inline unsigned alloc_local_tag(struct percpu_ida *pool,
> > +				       struct percpu_ida_cpu *tags)
> 
> Pass the __percpu offset and not the tags pointer.

Why? It just changes where the this_cpu_ptr

> 
> > +{
> > +	int tag = -ENOSPC;
> > +
> > +	spin_lock(&tags->lock);
> 
> Interupts are already disabled. Drop the spinlock.
> 
> > +	if (tags->nr_free)
> > +		tag = tags->freelist[--tags->nr_free];
> 
> You can keep this or avoid address calculation through segment prefixes.
> F.e.
> 
> if (__this_cpu_read(tags->nrfree) {
> 	int n = __this_cpu_dec_return(tags->nr_free);
> 	tag =  __this_cpu_read(tags->freelist[n]);
> }

Can you explain what the point of that change would be? It sounds like
it's preferable to do it that way and avoid this_cpu_ptr() for some
reason, but you're not explaining why.

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-21 18:25     ` Christoph Lameter
  (?)
@ 2013-08-26 20:23     ` Kent Overstreet
  -1 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-26 20:23 UTC (permalink / raw)
  To: Christoph Lameter
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Oleg Nesterov, Tejun Heo, Ingo Molnar,
	Andrew Morton

On Wed, Aug 21, 2013 at 06:25:58PM +0000, Christoph Lameter wrote:
> On Fri, 16 Aug 2013, Nicholas A. Bellinger wrote:
> 
> > +	spinlock_t			lock;
> 
> Remove the spinlock.

As Andrew noted, the spinlock is needed because of tag stealing. (You
don't think I'd stick a spinlock on a percpu data structure without a
real reason, would you?)

> > +	unsigned			nr_free;
> > +	unsigned			freelist[];
> > +};
> > +
> > +static inline void move_tags(unsigned *dst, unsigned *dst_nr,
> > +			     unsigned *src, unsigned *src_nr,
> > +			     unsigned nr)
> > +{
> > +	*src_nr -= nr;
> > +	memcpy(dst + *dst_nr, src + *src_nr, sizeof(unsigned) * nr);
> > +	*dst_nr += nr;
> > +}
> > +
> 
> > +static inline unsigned alloc_local_tag(struct percpu_ida *pool,
> > +				       struct percpu_ida_cpu *tags)
> 
> Pass the __percpu offset and not the tags pointer.

Why? It just changes where the this_cpu_ptr

> 
> > +{
> > +	int tag = -ENOSPC;
> > +
> > +	spin_lock(&tags->lock);
> 
> Interupts are already disabled. Drop the spinlock.
> 
> > +	if (tags->nr_free)
> > +		tag = tags->freelist[--tags->nr_free];
> 
> You can keep this or avoid address calculation through segment prefixes.
> F.e.
> 
> if (__this_cpu_read(tags->nrfree) {
> 	int n = __this_cpu_dec_return(tags->nr_free);
> 	tag =  __this_cpu_read(tags->freelist[n]);
> }

Can you explain what the point of that change would be? It sounds like
it's preferable to do it that way and avoid this_cpu_ptr() for some
reason, but you're not explaining why.

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-20 21:31   ` Andrew Morton
@ 2013-08-28 19:53       ` Kent Overstreet
  2013-08-26 20:14     ` Kent Overstreet
                         ` (2 subsequent siblings)
  3 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 19:53 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Tue, Aug 20, 2013 at 02:31:57PM -0700, Andrew Morton wrote:
> On Fri, 16 Aug 2013 23:09:06 +0000 "Nicholas A. Bellinger" <nab@linux-iscsi.org> wrote:
> > +	/*
> > +	 * Bitmap of cpus that (may) have tags on their percpu freelists:
> > +	 * steal_tags() uses this to decide when to steal tags, and which cpus
> > +	 * to try stealing from.
> > +	 *
> > +	 * It's ok for a freelist to be empty when its bit is set - steal_tags()
> > +	 * will just keep looking - but the bitmap _must_ be set whenever a
> > +	 * percpu freelist does have tags.
> > +	 */
> > +	unsigned long			*cpus_have_tags;
> 
> Why not cpumask_t?

I hadn't encountered it before - looks like it's probably what I want.

I don't see any explanation for the parallel set of operations for
working on cpumasks - e.g. next_cpu()/cpumask_next(). For now I'm going
with the cpumask_* versions, is that what I want?o

If you can have a look at the fixup patch that'll be most appreciated.

> > +	struct {
> > +		spinlock_t		lock;
> > +		/*
> > +		 * When we go to steal tags from another cpu (see steal_tags()),
> > +		 * we want to pick a cpu at random. Cycling through them every
> > +		 * time we steal is a bit easier and more or less equivalent:
> > +		 */
> > +		unsigned		cpu_last_stolen;
> > +
> > +		/* For sleeping on allocation failure */
> > +		wait_queue_head_t	wait;
> > +
> > +		/*
> > +		 * Global freelist - it's a stack where nr_free points to the
> > +		 * top
> > +		 */
> > +		unsigned		nr_free;
> > +		unsigned		*freelist;
> > +	} ____cacheline_aligned_in_smp;
> 
> Why the ____cacheline_aligned_in_smp?

It's separating the RW stuff that isn't always touched from the RO stuff
that's used on every allocation.

> 
> > +};
> > 
> > ...
> >
> > +
> > +/* Percpu IDA */
> > +
> > +/*
> > + * Number of tags we move between the percpu freelist and the global freelist at
> > + * a time
> 
> "between a percpu freelist" would be more accurate?

No, because when we're stealing tags we always grab all of the remote
percpu freelist's tags - IDA_PCPU_BATCH_MOVE is only used when moving
to/from the global freelist.

> 
> > + */
> > +#define IDA_PCPU_BATCH_MOVE	32U
> > +
> > +/* Max size of percpu freelist, */
> > +#define IDA_PCPU_SIZE		((IDA_PCPU_BATCH_MOVE * 3) / 2)
> > +
> > +struct percpu_ida_cpu {
> > +	spinlock_t			lock;
> > +	unsigned			nr_free;
> > +	unsigned			freelist[];
> > +};
> 
> Data structure needs documentation.  There's one of these per cpu.  I
> guess nr_free and freelist are clear enough.  The presence of a lock
> in a percpu data structure is a surprise.  It's for cross-cpu stealing,
> I assume?

Yeah, I'll add some comments.

> > +static inline void alloc_global_tags(struct percpu_ida *pool,
> > +				     struct percpu_ida_cpu *tags)
> > +{
> > +	move_tags(tags->freelist, &tags->nr_free,
> > +		  pool->freelist, &pool->nr_free,
> > +		  min(pool->nr_free, IDA_PCPU_BATCH_MOVE));
> > +}
> 
> Document this function?

Will do

> > +	while (1) {
> > +		spin_lock(&pool->lock);
> > +
> > +		/*
> > +		 * prepare_to_wait() must come before steal_tags(), in case
> > +		 * percpu_ida_free() on another cpu flips a bit in
> > +		 * cpus_have_tags
> > +		 *
> > +		 * global lock held and irqs disabled, don't need percpu lock
> > +		 */
> > +		prepare_to_wait(&pool->wait, &wait, TASK_UNINTERRUPTIBLE);
> > +
> > +		if (!tags->nr_free)
> > +			alloc_global_tags(pool, tags);
> > +		if (!tags->nr_free)
> > +			steal_tags(pool, tags);
> > +
> > +		if (tags->nr_free) {
> > +			tag = tags->freelist[--tags->nr_free];
> > +			if (tags->nr_free)
> > +				set_bit(smp_processor_id(),
> > +					pool->cpus_have_tags);
> > +		}
> > +
> > +		spin_unlock(&pool->lock);
> > +		local_irq_restore(flags);
> > +
> > +		if (tag >= 0 || !(gfp & __GFP_WAIT))
> > +			break;
> > +
> > +		schedule();
> > +
> > +		local_irq_save(flags);
> > +		tags = this_cpu_ptr(pool->tag_cpu);
> > +	}
> 
> What guarantees that this wait will terminate?

It seems fairly clear to me from the break statement a couple lines up;
if we were passed __GFP_WAIT we terminate iff we succesfully allocated a
tag. If we weren't passed __GFP_WAIT we never actually sleep.

I can add a comment if you think it needs one.

> > +	finish_wait(&pool->wait, &wait);
> > +	return tag;
> > +}
> > +EXPORT_SYMBOL_GPL(percpu_ida_alloc);
> > +
> > +/**
> > + * percpu_ida_free - free a tag
> > + * @pool: pool @tag was allocated from
> > + * @tag: a tag previously allocated with percpu_ida_alloc()
> > + *
> > + * Safe to be called from interrupt context.
> > + */
> > +void percpu_ida_free(struct percpu_ida *pool, unsigned tag)
> > +{
> > +	struct percpu_ida_cpu *tags;
> > +	unsigned long flags;
> > +	unsigned nr_free;
> > +
> > +	BUG_ON(tag >= pool->nr_tags);
> > +
> > +	local_irq_save(flags);
> > +	tags = this_cpu_ptr(pool->tag_cpu);
> > +
> > +	spin_lock(&tags->lock);
> 
> Why do we need this lock, btw?  It's a cpu-local structure and local
> irqs are disabled...

Tag stealing. I added a comment for the data structure explaining the
lock, do you think that suffices?

> > +	/* Guard against overflow */
> > +	if (nr_tags > (unsigned) INT_MAX + 1) {
> > +		pr_err("tags.c: nr_tags too large\n");
> 
> "tags.c"?

Whoops, out of date.

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
@ 2013-08-28 19:53       ` Kent Overstreet
  0 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 19:53 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Tue, Aug 20, 2013 at 02:31:57PM -0700, Andrew Morton wrote:
> On Fri, 16 Aug 2013 23:09:06 +0000 "Nicholas A. Bellinger" <nab@linux-iscsi.org> wrote:
> > +	/*
> > +	 * Bitmap of cpus that (may) have tags on their percpu freelists:
> > +	 * steal_tags() uses this to decide when to steal tags, and which cpus
> > +	 * to try stealing from.
> > +	 *
> > +	 * It's ok for a freelist to be empty when its bit is set - steal_tags()
> > +	 * will just keep looking - but the bitmap _must_ be set whenever a
> > +	 * percpu freelist does have tags.
> > +	 */
> > +	unsigned long			*cpus_have_tags;
> 
> Why not cpumask_t?

I hadn't encountered it before - looks like it's probably what I want.

I don't see any explanation for the parallel set of operations for
working on cpumasks - e.g. next_cpu()/cpumask_next(). For now I'm going
with the cpumask_* versions, is that what I want?o

If you can have a look at the fixup patch that'll be most appreciated.

> > +	struct {
> > +		spinlock_t		lock;
> > +		/*
> > +		 * When we go to steal tags from another cpu (see steal_tags()),
> > +		 * we want to pick a cpu at random. Cycling through them every
> > +		 * time we steal is a bit easier and more or less equivalent:
> > +		 */
> > +		unsigned		cpu_last_stolen;
> > +
> > +		/* For sleeping on allocation failure */
> > +		wait_queue_head_t	wait;
> > +
> > +		/*
> > +		 * Global freelist - it's a stack where nr_free points to the
> > +		 * top
> > +		 */
> > +		unsigned		nr_free;
> > +		unsigned		*freelist;
> > +	} ____cacheline_aligned_in_smp;
> 
> Why the ____cacheline_aligned_in_smp?

It's separating the RW stuff that isn't always touched from the RO stuff
that's used on every allocation.

> 
> > +};
> > 
> > ...
> >
> > +
> > +/* Percpu IDA */
> > +
> > +/*
> > + * Number of tags we move between the percpu freelist and the global freelist at
> > + * a time
> 
> "between a percpu freelist" would be more accurate?

No, because when we're stealing tags we always grab all of the remote
percpu freelist's tags - IDA_PCPU_BATCH_MOVE is only used when moving
to/from the global freelist.

> 
> > + */
> > +#define IDA_PCPU_BATCH_MOVE	32U
> > +
> > +/* Max size of percpu freelist, */
> > +#define IDA_PCPU_SIZE		((IDA_PCPU_BATCH_MOVE * 3) / 2)
> > +
> > +struct percpu_ida_cpu {
> > +	spinlock_t			lock;
> > +	unsigned			nr_free;
> > +	unsigned			freelist[];
> > +};
> 
> Data structure needs documentation.  There's one of these per cpu.  I
> guess nr_free and freelist are clear enough.  The presence of a lock
> in a percpu data structure is a surprise.  It's for cross-cpu stealing,
> I assume?

Yeah, I'll add some comments.

> > +static inline void alloc_global_tags(struct percpu_ida *pool,
> > +				     struct percpu_ida_cpu *tags)
> > +{
> > +	move_tags(tags->freelist, &tags->nr_free,
> > +		  pool->freelist, &pool->nr_free,
> > +		  min(pool->nr_free, IDA_PCPU_BATCH_MOVE));
> > +}
> 
> Document this function?

Will do

> > +	while (1) {
> > +		spin_lock(&pool->lock);
> > +
> > +		/*
> > +		 * prepare_to_wait() must come before steal_tags(), in case
> > +		 * percpu_ida_free() on another cpu flips a bit in
> > +		 * cpus_have_tags
> > +		 *
> > +		 * global lock held and irqs disabled, don't need percpu lock
> > +		 */
> > +		prepare_to_wait(&pool->wait, &wait, TASK_UNINTERRUPTIBLE);
> > +
> > +		if (!tags->nr_free)
> > +			alloc_global_tags(pool, tags);
> > +		if (!tags->nr_free)
> > +			steal_tags(pool, tags);
> > +
> > +		if (tags->nr_free) {
> > +			tag = tags->freelist[--tags->nr_free];
> > +			if (tags->nr_free)
> > +				set_bit(smp_processor_id(),
> > +					pool->cpus_have_tags);
> > +		}
> > +
> > +		spin_unlock(&pool->lock);
> > +		local_irq_restore(flags);
> > +
> > +		if (tag >= 0 || !(gfp & __GFP_WAIT))
> > +			break;
> > +
> > +		schedule();
> > +
> > +		local_irq_save(flags);
> > +		tags = this_cpu_ptr(pool->tag_cpu);
> > +	}
> 
> What guarantees that this wait will terminate?

It seems fairly clear to me from the break statement a couple lines up;
if we were passed __GFP_WAIT we terminate iff we succesfully allocated a
tag. If we weren't passed __GFP_WAIT we never actually sleep.

I can add a comment if you think it needs one.

> > +	finish_wait(&pool->wait, &wait);
> > +	return tag;
> > +}
> > +EXPORT_SYMBOL_GPL(percpu_ida_alloc);
> > +
> > +/**
> > + * percpu_ida_free - free a tag
> > + * @pool: pool @tag was allocated from
> > + * @tag: a tag previously allocated with percpu_ida_alloc()
> > + *
> > + * Safe to be called from interrupt context.
> > + */
> > +void percpu_ida_free(struct percpu_ida *pool, unsigned tag)
> > +{
> > +	struct percpu_ida_cpu *tags;
> > +	unsigned long flags;
> > +	unsigned nr_free;
> > +
> > +	BUG_ON(tag >= pool->nr_tags);
> > +
> > +	local_irq_save(flags);
> > +	tags = this_cpu_ptr(pool->tag_cpu);
> > +
> > +	spin_lock(&tags->lock);
> 
> Why do we need this lock, btw?  It's a cpu-local structure and local
> irqs are disabled...

Tag stealing. I added a comment for the data structure explaining the
lock, do you think that suffices?

> > +	/* Guard against overflow */
> > +	if (nr_tags > (unsigned) INT_MAX + 1) {
> > +		pr_err("tags.c: nr_tags too large\n");
> 
> "tags.c"?

Whoops, out of date.

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

* [PATCH] percpu ida: Switch to cpumask_t, add some comments
  2013-08-20 21:31   ` Andrew Morton
@ 2013-08-28 19:55       ` Kent Overstreet
  2013-08-26 20:14     ` Kent Overstreet
                         ` (2 subsequent siblings)
  3 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 19:55 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

Fixup patch, addressing Andrew's review feedback:

Signed-off-by: Kent Overstreet <kmo@daterainc.com>
---
 include/linux/idr.h |  2 +-
 lib/idr.c           | 38 +++++++++++++++++++++-----------------
 2 files changed, 22 insertions(+), 18 deletions(-)

diff --git a/include/linux/idr.h b/include/linux/idr.h
index f0db12b..cdf39be 100644
--- a/include/linux/idr.h
+++ b/include/linux/idr.h
@@ -267,7 +267,7 @@ struct percpu_ida {
 	 * will just keep looking - but the bitmap _must_ be set whenever a
 	 * percpu freelist does have tags.
 	 */
-	unsigned long			*cpus_have_tags;
+	cpumask_t			cpus_have_tags;
 
 	struct {
 		spinlock_t		lock;
diff --git a/lib/idr.c b/lib/idr.c
index 26495e1..15c021c 100644
--- a/lib/idr.c
+++ b/lib/idr.c
@@ -1178,7 +1178,13 @@ EXPORT_SYMBOL(ida_init);
 #define IDA_PCPU_SIZE		((IDA_PCPU_BATCH_MOVE * 3) / 2)
 
 struct percpu_ida_cpu {
+	/*
+	 * Even though this is percpu, we need a lock for tag stealing by remote
+	 * CPUs:
+	 */
 	spinlock_t			lock;
+
+	/* nr_free/freelist form a stack of free IDs */
 	unsigned			nr_free;
 	unsigned			freelist[];
 };
@@ -1209,21 +1215,21 @@ static inline void steal_tags(struct percpu_ida *pool,
 	unsigned cpus_have_tags, cpu = pool->cpu_last_stolen;
 	struct percpu_ida_cpu *remote;
 
-	for (cpus_have_tags = bitmap_weight(pool->cpus_have_tags, nr_cpu_ids);
+	for (cpus_have_tags = cpumask_weight(&pool->cpus_have_tags);
 	     cpus_have_tags * IDA_PCPU_SIZE > pool->nr_tags / 2;
 	     cpus_have_tags--) {
-		cpu = find_next_bit(pool->cpus_have_tags, nr_cpu_ids, cpu);
+		cpu = cpumask_next(cpu, &pool->cpus_have_tags);
 
-		if (cpu == nr_cpu_ids)
-			cpu = find_first_bit(pool->cpus_have_tags, nr_cpu_ids);
+		if (cpu >= nr_cpu_ids)
+			cpu = cpumask_first(&pool->cpus_have_tags);
 
-		if (cpu == nr_cpu_ids)
+		if (cpu >= nr_cpu_ids)
 			BUG();
 
 		pool->cpu_last_stolen = cpu;
 		remote = per_cpu_ptr(pool->tag_cpu, cpu);
 
-		clear_bit(cpu, pool->cpus_have_tags);
+		cpumask_clear_cpu(cpu, &pool->cpus_have_tags);
 
 		if (remote == tags)
 			continue;
@@ -1246,6 +1252,10 @@ static inline void steal_tags(struct percpu_ida *pool,
 	}
 }
 
+/*
+ * Pop up to IDA_PCPU_BATCH_MOVE IDs off the global freelist, and push them onto
+ * our percpu freelist:
+ */
 static inline void alloc_global_tags(struct percpu_ida *pool,
 				     struct percpu_ida_cpu *tags)
 {
@@ -1317,8 +1327,8 @@ int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp)
 		if (tags->nr_free) {
 			tag = tags->freelist[--tags->nr_free];
 			if (tags->nr_free)
-				set_bit(smp_processor_id(),
-					pool->cpus_have_tags);
+				cpumask_set_cpu(smp_processor_id(),
+						&pool->cpus_have_tags);
 		}
 
 		spin_unlock(&pool->lock);
@@ -1363,8 +1373,8 @@ void percpu_ida_free(struct percpu_ida *pool, unsigned tag)
 	spin_unlock(&tags->lock);
 
 	if (nr_free == 1) {
-		set_bit(smp_processor_id(),
-			pool->cpus_have_tags);
+		cpumask_set_cpu(smp_processor_id(),
+				&pool->cpus_have_tags);
 		wake_up(&pool->wait);
 	}
 
@@ -1398,7 +1408,6 @@ EXPORT_SYMBOL_GPL(percpu_ida_free);
 void percpu_ida_destroy(struct percpu_ida *pool)
 {
 	free_percpu(pool->tag_cpu);
-	kfree(pool->cpus_have_tags);
 	free_pages((unsigned long) pool->freelist,
 		   get_order(pool->nr_tags * sizeof(unsigned)));
 }
@@ -1428,7 +1437,7 @@ int percpu_ida_init(struct percpu_ida *pool, unsigned long nr_tags)
 
 	/* Guard against overflow */
 	if (nr_tags > (unsigned) INT_MAX + 1) {
-		pr_err("tags.c: nr_tags too large\n");
+		pr_err("percpu_ida_init(): nr_tags too large\n");
 		return -EINVAL;
 	}
 
@@ -1442,11 +1451,6 @@ int percpu_ida_init(struct percpu_ida *pool, unsigned long nr_tags)
 
 	pool->nr_free = nr_tags;
 
-	pool->cpus_have_tags = kzalloc(BITS_TO_LONGS(nr_cpu_ids) *
-				       sizeof(unsigned long), GFP_KERNEL);
-	if (!pool->cpus_have_tags)
-		goto err;
-
 	pool->tag_cpu = __alloc_percpu(sizeof(struct percpu_ida_cpu) +
 				       IDA_PCPU_SIZE * sizeof(unsigned),
 				       sizeof(unsigned));
-- 
1.8.4.rc3


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

* [PATCH] percpu ida: Switch to cpumask_t, add some comments
@ 2013-08-28 19:55       ` Kent Overstreet
  0 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 19:55 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

Fixup patch, addressing Andrew's review feedback:

Signed-off-by: Kent Overstreet <kmo@daterainc.com>
---
 include/linux/idr.h |  2 +-
 lib/idr.c           | 38 +++++++++++++++++++++-----------------
 2 files changed, 22 insertions(+), 18 deletions(-)

diff --git a/include/linux/idr.h b/include/linux/idr.h
index f0db12b..cdf39be 100644
--- a/include/linux/idr.h
+++ b/include/linux/idr.h
@@ -267,7 +267,7 @@ struct percpu_ida {
 	 * will just keep looking - but the bitmap _must_ be set whenever a
 	 * percpu freelist does have tags.
 	 */
-	unsigned long			*cpus_have_tags;
+	cpumask_t			cpus_have_tags;
 
 	struct {
 		spinlock_t		lock;
diff --git a/lib/idr.c b/lib/idr.c
index 26495e1..15c021c 100644
--- a/lib/idr.c
+++ b/lib/idr.c
@@ -1178,7 +1178,13 @@ EXPORT_SYMBOL(ida_init);
 #define IDA_PCPU_SIZE		((IDA_PCPU_BATCH_MOVE * 3) / 2)
 
 struct percpu_ida_cpu {
+	/*
+	 * Even though this is percpu, we need a lock for tag stealing by remote
+	 * CPUs:
+	 */
 	spinlock_t			lock;
+
+	/* nr_free/freelist form a stack of free IDs */
 	unsigned			nr_free;
 	unsigned			freelist[];
 };
@@ -1209,21 +1215,21 @@ static inline void steal_tags(struct percpu_ida *pool,
 	unsigned cpus_have_tags, cpu = pool->cpu_last_stolen;
 	struct percpu_ida_cpu *remote;
 
-	for (cpus_have_tags = bitmap_weight(pool->cpus_have_tags, nr_cpu_ids);
+	for (cpus_have_tags = cpumask_weight(&pool->cpus_have_tags);
 	     cpus_have_tags * IDA_PCPU_SIZE > pool->nr_tags / 2;
 	     cpus_have_tags--) {
-		cpu = find_next_bit(pool->cpus_have_tags, nr_cpu_ids, cpu);
+		cpu = cpumask_next(cpu, &pool->cpus_have_tags);
 
-		if (cpu == nr_cpu_ids)
-			cpu = find_first_bit(pool->cpus_have_tags, nr_cpu_ids);
+		if (cpu >= nr_cpu_ids)
+			cpu = cpumask_first(&pool->cpus_have_tags);
 
-		if (cpu == nr_cpu_ids)
+		if (cpu >= nr_cpu_ids)
 			BUG();
 
 		pool->cpu_last_stolen = cpu;
 		remote = per_cpu_ptr(pool->tag_cpu, cpu);
 
-		clear_bit(cpu, pool->cpus_have_tags);
+		cpumask_clear_cpu(cpu, &pool->cpus_have_tags);
 
 		if (remote == tags)
 			continue;
@@ -1246,6 +1252,10 @@ static inline void steal_tags(struct percpu_ida *pool,
 	}
 }
 
+/*
+ * Pop up to IDA_PCPU_BATCH_MOVE IDs off the global freelist, and push them onto
+ * our percpu freelist:
+ */
 static inline void alloc_global_tags(struct percpu_ida *pool,
 				     struct percpu_ida_cpu *tags)
 {
@@ -1317,8 +1327,8 @@ int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp)
 		if (tags->nr_free) {
 			tag = tags->freelist[--tags->nr_free];
 			if (tags->nr_free)
-				set_bit(smp_processor_id(),
-					pool->cpus_have_tags);
+				cpumask_set_cpu(smp_processor_id(),
+						&pool->cpus_have_tags);
 		}
 
 		spin_unlock(&pool->lock);
@@ -1363,8 +1373,8 @@ void percpu_ida_free(struct percpu_ida *pool, unsigned tag)
 	spin_unlock(&tags->lock);
 
 	if (nr_free == 1) {
-		set_bit(smp_processor_id(),
-			pool->cpus_have_tags);
+		cpumask_set_cpu(smp_processor_id(),
+				&pool->cpus_have_tags);
 		wake_up(&pool->wait);
 	}
 
@@ -1398,7 +1408,6 @@ EXPORT_SYMBOL_GPL(percpu_ida_free);
 void percpu_ida_destroy(struct percpu_ida *pool)
 {
 	free_percpu(pool->tag_cpu);
-	kfree(pool->cpus_have_tags);
 	free_pages((unsigned long) pool->freelist,
 		   get_order(pool->nr_tags * sizeof(unsigned)));
 }
@@ -1428,7 +1437,7 @@ int percpu_ida_init(struct percpu_ida *pool, unsigned long nr_tags)
 
 	/* Guard against overflow */
 	if (nr_tags > (unsigned) INT_MAX + 1) {
-		pr_err("tags.c: nr_tags too large\n");
+		pr_err("percpu_ida_init(): nr_tags too large\n");
 		return -EINVAL;
 	}
 
@@ -1442,11 +1451,6 @@ int percpu_ida_init(struct percpu_ida *pool, unsigned long nr_tags)
 
 	pool->nr_free = nr_tags;
 
-	pool->cpus_have_tags = kzalloc(BITS_TO_LONGS(nr_cpu_ids) *
-				       sizeof(unsigned long), GFP_KERNEL);
-	if (!pool->cpus_have_tags)
-		goto err;
-
 	pool->tag_cpu = __alloc_percpu(sizeof(struct percpu_ida_cpu) +
 				       IDA_PCPU_SIZE * sizeof(unsigned),
 				       sizeof(unsigned));
-- 
1.8.4.rc3

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-28 19:53       ` Kent Overstreet
  (?)
  (?)
@ 2013-08-28 20:23       ` Andrew Morton
  2013-08-28 20:44         ` Kent Overstreet
  2013-08-28 20:44         ` Kent Overstreet
  -1 siblings, 2 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 20:23 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Wed, 28 Aug 2013 12:53:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> > > +	while (1) {
> > > +		spin_lock(&pool->lock);
> > > +
> > > +		/*
> > > +		 * prepare_to_wait() must come before steal_tags(), in case
> > > +		 * percpu_ida_free() on another cpu flips a bit in
> > > +		 * cpus_have_tags
> > > +		 *
> > > +		 * global lock held and irqs disabled, don't need percpu lock
> > > +		 */
> > > +		prepare_to_wait(&pool->wait, &wait, TASK_UNINTERRUPTIBLE);
> > > +
> > > +		if (!tags->nr_free)
> > > +			alloc_global_tags(pool, tags);
> > > +		if (!tags->nr_free)
> > > +			steal_tags(pool, tags);
> > > +
> > > +		if (tags->nr_free) {
> > > +			tag = tags->freelist[--tags->nr_free];
> > > +			if (tags->nr_free)
> > > +				set_bit(smp_processor_id(),
> > > +					pool->cpus_have_tags);
> > > +		}
> > > +
> > > +		spin_unlock(&pool->lock);
> > > +		local_irq_restore(flags);
> > > +
> > > +		if (tag >= 0 || !(gfp & __GFP_WAIT))
> > > +			break;
> > > +
> > > +		schedule();
> > > +
> > > +		local_irq_save(flags);
> > > +		tags = this_cpu_ptr(pool->tag_cpu);
> > > +	}
> > 
> > What guarantees that this wait will terminate?
> 
> It seems fairly clear to me from the break statement a couple lines up;
> if we were passed __GFP_WAIT we terminate iff we succesfully allocated a
> tag. If we weren't passed __GFP_WAIT we never actually sleep.

OK ;)  Let me rephrase.  What guarantees that a tag will become available?

If what we have here is an open-coded __GFP_NOFAIL then that is
potentially problematic.

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-28 19:53       ` Kent Overstreet
  (?)
@ 2013-08-28 20:23       ` Andrew Morton
  -1 siblings, 0 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 20:23 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, 28 Aug 2013 12:53:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> > > +	while (1) {
> > > +		spin_lock(&pool->lock);
> > > +
> > > +		/*
> > > +		 * prepare_to_wait() must come before steal_tags(), in case
> > > +		 * percpu_ida_free() on another cpu flips a bit in
> > > +		 * cpus_have_tags
> > > +		 *
> > > +		 * global lock held and irqs disabled, don't need percpu lock
> > > +		 */
> > > +		prepare_to_wait(&pool->wait, &wait, TASK_UNINTERRUPTIBLE);
> > > +
> > > +		if (!tags->nr_free)
> > > +			alloc_global_tags(pool, tags);
> > > +		if (!tags->nr_free)
> > > +			steal_tags(pool, tags);
> > > +
> > > +		if (tags->nr_free) {
> > > +			tag = tags->freelist[--tags->nr_free];
> > > +			if (tags->nr_free)
> > > +				set_bit(smp_processor_id(),
> > > +					pool->cpus_have_tags);
> > > +		}
> > > +
> > > +		spin_unlock(&pool->lock);
> > > +		local_irq_restore(flags);
> > > +
> > > +		if (tag >= 0 || !(gfp & __GFP_WAIT))
> > > +			break;
> > > +
> > > +		schedule();
> > > +
> > > +		local_irq_save(flags);
> > > +		tags = this_cpu_ptr(pool->tag_cpu);
> > > +	}
> > 
> > What guarantees that this wait will terminate?
> 
> It seems fairly clear to me from the break statement a couple lines up;
> if we were passed __GFP_WAIT we terminate iff we succesfully allocated a
> tag. If we weren't passed __GFP_WAIT we never actually sleep.

OK ;)  Let me rephrase.  What guarantees that a tag will become available?

If what we have here is an open-coded __GFP_NOFAIL then that is
potentially problematic.

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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
  2013-08-28 19:55       ` Kent Overstreet
@ 2013-08-28 20:25         ` Andrew Morton
  -1 siblings, 0 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 20:25 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Wed, 28 Aug 2013 12:55:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> Fixup patch, addressing Andrew's review feedback:

Looks reasonable.

>  lib/idr.c           | 38 +++++++++++++++++++++-----------------

I still don't think it should be in this file.

You say that some as-yet-unmerged patches will tie the new code into
the old ida code.  But will it do it in a manner which requires that
the two reside in the same file?


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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
@ 2013-08-28 20:25         ` Andrew Morton
  0 siblings, 0 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 20:25 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, 28 Aug 2013 12:55:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> Fixup patch, addressing Andrew's review feedback:

Looks reasonable.

>  lib/idr.c           | 38 +++++++++++++++++++++-----------------

I still don't think it should be in this file.

You say that some as-yet-unmerged patches will tie the new code into
the old ida code.  But will it do it in a manner which requires that
the two reside in the same file?

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-28 20:23       ` Andrew Morton
@ 2013-08-28 20:44         ` Kent Overstreet
  2013-08-28 20:50             ` Andrew Morton
  2013-08-28 20:44         ` Kent Overstreet
  1 sibling, 1 reply; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 20:44 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Wed, Aug 28, 2013 at 01:23:32PM -0700, Andrew Morton wrote:
> On Wed, 28 Aug 2013 12:53:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> 
> > > > +	while (1) {
> > > > +		spin_lock(&pool->lock);
> > > > +
> > > > +		/*
> > > > +		 * prepare_to_wait() must come before steal_tags(), in case
> > > > +		 * percpu_ida_free() on another cpu flips a bit in
> > > > +		 * cpus_have_tags
> > > > +		 *
> > > > +		 * global lock held and irqs disabled, don't need percpu lock
> > > > +		 */
> > > > +		prepare_to_wait(&pool->wait, &wait, TASK_UNINTERRUPTIBLE);
> > > > +
> > > > +		if (!tags->nr_free)
> > > > +			alloc_global_tags(pool, tags);
> > > > +		if (!tags->nr_free)
> > > > +			steal_tags(pool, tags);
> > > > +
> > > > +		if (tags->nr_free) {
> > > > +			tag = tags->freelist[--tags->nr_free];
> > > > +			if (tags->nr_free)
> > > > +				set_bit(smp_processor_id(),
> > > > +					pool->cpus_have_tags);
> > > > +		}
> > > > +
> > > > +		spin_unlock(&pool->lock);
> > > > +		local_irq_restore(flags);
> > > > +
> > > > +		if (tag >= 0 || !(gfp & __GFP_WAIT))
> > > > +			break;
> > > > +
> > > > +		schedule();
> > > > +
> > > > +		local_irq_save(flags);
> > > > +		tags = this_cpu_ptr(pool->tag_cpu);
> > > > +	}
> > > 
> > > What guarantees that this wait will terminate?
> > 
> > It seems fairly clear to me from the break statement a couple lines up;
> > if we were passed __GFP_WAIT we terminate iff we succesfully allocated a
> > tag. If we weren't passed __GFP_WAIT we never actually sleep.
> 
> OK ;)  Let me rephrase.  What guarantees that a tag will become available?
> 
> If what we have here is an open-coded __GFP_NOFAIL then that is
> potentially problematic.

It's the same semantics as a mempool, really - it'll succeed when a tag
gets freed.  If we are sleeping then there isn't really anything else we
can do, there isn't anything we're trying in the __GFP_WAIT case that
we're not trying in the GFP_NOWAIT case.

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-28 20:23       ` Andrew Morton
  2013-08-28 20:44         ` Kent Overstreet
@ 2013-08-28 20:44         ` Kent Overstreet
  1 sibling, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 20:44 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, Aug 28, 2013 at 01:23:32PM -0700, Andrew Morton wrote:
> On Wed, 28 Aug 2013 12:53:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> 
> > > > +	while (1) {
> > > > +		spin_lock(&pool->lock);
> > > > +
> > > > +		/*
> > > > +		 * prepare_to_wait() must come before steal_tags(), in case
> > > > +		 * percpu_ida_free() on another cpu flips a bit in
> > > > +		 * cpus_have_tags
> > > > +		 *
> > > > +		 * global lock held and irqs disabled, don't need percpu lock
> > > > +		 */
> > > > +		prepare_to_wait(&pool->wait, &wait, TASK_UNINTERRUPTIBLE);
> > > > +
> > > > +		if (!tags->nr_free)
> > > > +			alloc_global_tags(pool, tags);
> > > > +		if (!tags->nr_free)
> > > > +			steal_tags(pool, tags);
> > > > +
> > > > +		if (tags->nr_free) {
> > > > +			tag = tags->freelist[--tags->nr_free];
> > > > +			if (tags->nr_free)
> > > > +				set_bit(smp_processor_id(),
> > > > +					pool->cpus_have_tags);
> > > > +		}
> > > > +
> > > > +		spin_unlock(&pool->lock);
> > > > +		local_irq_restore(flags);
> > > > +
> > > > +		if (tag >= 0 || !(gfp & __GFP_WAIT))
> > > > +			break;
> > > > +
> > > > +		schedule();
> > > > +
> > > > +		local_irq_save(flags);
> > > > +		tags = this_cpu_ptr(pool->tag_cpu);
> > > > +	}
> > > 
> > > What guarantees that this wait will terminate?
> > 
> > It seems fairly clear to me from the break statement a couple lines up;
> > if we were passed __GFP_WAIT we terminate iff we succesfully allocated a
> > tag. If we weren't passed __GFP_WAIT we never actually sleep.
> 
> OK ;)  Let me rephrase.  What guarantees that a tag will become available?
> 
> If what we have here is an open-coded __GFP_NOFAIL then that is
> potentially problematic.

It's the same semantics as a mempool, really - it'll succeed when a tag
gets freed.  If we are sleeping then there isn't really anything else we
can do, there isn't anything we're trying in the __GFP_WAIT case that
we're not trying in the GFP_NOWAIT case.

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-28 20:44         ` Kent Overstreet
@ 2013-08-28 20:50             ` Andrew Morton
  0 siblings, 0 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 20:50 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Wed, 28 Aug 2013 13:44:54 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> > > > What guarantees that this wait will terminate?
> > > 
> > > It seems fairly clear to me from the break statement a couple lines up;
> > > if we were passed __GFP_WAIT we terminate iff we succesfully allocated a
> > > tag. If we weren't passed __GFP_WAIT we never actually sleep.
> > 
> > OK ;)  Let me rephrase.  What guarantees that a tag will become available?
> > 
> > If what we have here is an open-coded __GFP_NOFAIL then that is
> > potentially problematic.
> 
> It's the same semantics as a mempool, really - it'll succeed when a tag
> gets freed.

OK, that's reasonable if the code is being used to generate IO tags -
we expect the in-flight tags to eventually be returned.

But if a client of this code is using the allocator for something
totally different, there is no guarantee that the act of waiting will
result in any tags being returned.

(These are core design principles/constraints which should be
explicitly documented in a place where future readers will see them!)


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

* Re: [PATCH-v3 1/4] idr: Percpu ida
@ 2013-08-28 20:50             ` Andrew Morton
  0 siblings, 0 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 20:50 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, 28 Aug 2013 13:44:54 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> > > > What guarantees that this wait will terminate?
> > > 
> > > It seems fairly clear to me from the break statement a couple lines up;
> > > if we were passed __GFP_WAIT we terminate iff we succesfully allocated a
> > > tag. If we weren't passed __GFP_WAIT we never actually sleep.
> > 
> > OK ;)  Let me rephrase.  What guarantees that a tag will become available?
> > 
> > If what we have here is an open-coded __GFP_NOFAIL then that is
> > potentially problematic.
> 
> It's the same semantics as a mempool, really - it'll succeed when a tag
> gets freed.

OK, that's reasonable if the code is being used to generate IO tags -
we expect the in-flight tags to eventually be returned.

But if a client of this code is using the allocator for something
totally different, there is no guarantee that the act of waiting will
result in any tags being returned.

(These are core design principles/constraints which should be
explicitly documented in a place where future readers will see them!)

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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
  2013-08-28 20:25         ` Andrew Morton
@ 2013-08-28 21:00           ` Kent Overstreet
  -1 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 21:00 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Wed, Aug 28, 2013 at 01:25:50PM -0700, Andrew Morton wrote:
> On Wed, 28 Aug 2013 12:55:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> 
> > Fixup patch, addressing Andrew's review feedback:
> 
> Looks reasonable.
> 
> >  lib/idr.c           | 38 +++++++++++++++++++++-----------------
> 
> I still don't think it should be in this file.
> 
> You say that some as-yet-unmerged patches will tie the new code into
> the old ida code.  But will it do it in a manner which requires that
> the two reside in the same file?

Not require, no - but it's just intimate enough with my ida rewrite that
I think it makes sense; it makes some use of stuff that should be
internal to the ida code.

Mostly just sharing the lock though, since I got rid of the ida
interfaces that don't do locking, but percpu ida needs a lock that also
covers what ida needs.

It also makes use of a ganged allocation interface, but there's no real
reason ida can't expose that, it's just unlikely to be useful to
anything but percpu ida.

The other reason I think it makes sense to live in idr.c is more for
users of the code; as you pointed out as far as the user's perspective
percpu ida isn't doing anything fundamentally different from ida, so I
think it makes sense for the code to live in the same place as a
kindness to future kernel developers who are trying to find their way
around the various library code.

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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
@ 2013-08-28 21:00           ` Kent Overstreet
  0 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 21:00 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, Aug 28, 2013 at 01:25:50PM -0700, Andrew Morton wrote:
> On Wed, 28 Aug 2013 12:55:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> 
> > Fixup patch, addressing Andrew's review feedback:
> 
> Looks reasonable.
> 
> >  lib/idr.c           | 38 +++++++++++++++++++++-----------------
> 
> I still don't think it should be in this file.
> 
> You say that some as-yet-unmerged patches will tie the new code into
> the old ida code.  But will it do it in a manner which requires that
> the two reside in the same file?

Not require, no - but it's just intimate enough with my ida rewrite that
I think it makes sense; it makes some use of stuff that should be
internal to the ida code.

Mostly just sharing the lock though, since I got rid of the ida
interfaces that don't do locking, but percpu ida needs a lock that also
covers what ida needs.

It also makes use of a ganged allocation interface, but there's no real
reason ida can't expose that, it's just unlikely to be useful to
anything but percpu ida.

The other reason I think it makes sense to live in idr.c is more for
users of the code; as you pointed out as far as the user's perspective
percpu ida isn't doing anything fundamentally different from ida, so I
think it makes sense for the code to live in the same place as a
kindness to future kernel developers who are trying to find their way
around the various library code.

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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
  2013-08-28 21:00           ` Kent Overstreet
  (?)
@ 2013-08-28 21:10           ` Andrew Morton
  2013-08-28 21:23             ` Kent Overstreet
                               ` (2 more replies)
  -1 siblings, 3 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 21:10 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Wed, 28 Aug 2013 14:00:10 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> On Wed, Aug 28, 2013 at 01:25:50PM -0700, Andrew Morton wrote:
> > On Wed, 28 Aug 2013 12:55:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> > 
> > > Fixup patch, addressing Andrew's review feedback:
> > 
> > Looks reasonable.
> > 
> > >  lib/idr.c           | 38 +++++++++++++++++++++-----------------
> > 
> > I still don't think it should be in this file.
> > 
> > You say that some as-yet-unmerged patches will tie the new code into
> > the old ida code.  But will it do it in a manner which requires that
> > the two reside in the same file?
> 
> Not require, no - but it's just intimate enough with my ida rewrite that
> I think it makes sense; it makes some use of stuff that should be
> internal to the ida code.
> 
> Mostly just sharing the lock though, since I got rid of the ida
> interfaces that don't do locking, but percpu ida needs a lock that also
> covers what ida needs.
> 
> It also makes use of a ganged allocation interface, but there's no real
> reason ida can't expose that, it's just unlikely to be useful to
> anything but percpu ida.
> 
> The other reason I think it makes sense to live in idr.c is more for
> users of the code; as you pointed out as far as the user's perspective
> percpu ida isn't doing anything fundamentally different from ida, so I
> think it makes sense for the code to live in the same place as a
> kindness to future kernel developers who are trying to find their way
> around the various library code.

I found things to be quite the opposite - it took 5 minutes of staring,
head-scratching, double-checking and penny-dropping before I was
confident that the newly-added code actually has nothing at all to do
with the current code.  Putting it in the same file was misleading, and
I got misled.

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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
  2013-08-28 21:00           ` Kent Overstreet
  (?)
  (?)
@ 2013-08-28 21:10           ` Andrew Morton
  -1 siblings, 0 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 21:10 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, 28 Aug 2013 14:00:10 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> On Wed, Aug 28, 2013 at 01:25:50PM -0700, Andrew Morton wrote:
> > On Wed, 28 Aug 2013 12:55:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> > 
> > > Fixup patch, addressing Andrew's review feedback:
> > 
> > Looks reasonable.
> > 
> > >  lib/idr.c           | 38 +++++++++++++++++++++-----------------
> > 
> > I still don't think it should be in this file.
> > 
> > You say that some as-yet-unmerged patches will tie the new code into
> > the old ida code.  But will it do it in a manner which requires that
> > the two reside in the same file?
> 
> Not require, no - but it's just intimate enough with my ida rewrite that
> I think it makes sense; it makes some use of stuff that should be
> internal to the ida code.
> 
> Mostly just sharing the lock though, since I got rid of the ida
> interfaces that don't do locking, but percpu ida needs a lock that also
> covers what ida needs.
> 
> It also makes use of a ganged allocation interface, but there's no real
> reason ida can't expose that, it's just unlikely to be useful to
> anything but percpu ida.
> 
> The other reason I think it makes sense to live in idr.c is more for
> users of the code; as you pointed out as far as the user's perspective
> percpu ida isn't doing anything fundamentally different from ida, so I
> think it makes sense for the code to live in the same place as a
> kindness to future kernel developers who are trying to find their way
> around the various library code.

I found things to be quite the opposite - it took 5 minutes of staring,
head-scratching, double-checking and penny-dropping before I was
confident that the newly-added code actually has nothing at all to do
with the current code.  Putting it in the same file was misleading, and
I got misled.

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-28 20:50             ` Andrew Morton
@ 2013-08-28 21:12               ` Kent Overstreet
  -1 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 21:12 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Wed, Aug 28, 2013 at 01:50:42PM -0700, Andrew Morton wrote:
> On Wed, 28 Aug 2013 13:44:54 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> 
> > > > > What guarantees that this wait will terminate?
> > > > 
> > > > It seems fairly clear to me from the break statement a couple lines up;
> > > > if we were passed __GFP_WAIT we terminate iff we succesfully allocated a
> > > > tag. If we weren't passed __GFP_WAIT we never actually sleep.
> > > 
> > > OK ;)  Let me rephrase.  What guarantees that a tag will become available?
> > > 
> > > If what we have here is an open-coded __GFP_NOFAIL then that is
> > > potentially problematic.
> > 
> > It's the same semantics as a mempool, really - it'll succeed when a tag
> > gets freed.
> 
> OK, that's reasonable if the code is being used to generate IO tags -
> we expect the in-flight tags to eventually be returned.
> 
> But if a client of this code is using the allocator for something
> totally different, there is no guarantee that the act of waiting will
> result in any tags being returned.

Yeah, and I did wonder a bit whether the waiting mechanism belonged in
the percpu ida code; arguably (certainly just looking at this code, not
any of the users) if it belongs in this code it should be common to
regular ida, not specific to percpu ida.

For now I've just decided to punt on changing that for now, since all
the percpu ida users I've come across do want the waiting mechanism, but
none of the regular ida users that I've looked at want it. There's
probably a reason for that I haven't thought of yet.

> (These are core design principles/constraints which should be
> explicitly documented in a place where future readers will see them!)

*nod* I suppose it should be said explicitly that the gfp_t parameter
indicates whether or not to wait until a _tag_ is available, and not
some internal memory allocation or something.

How's this look?

diff --git a/lib/idr.c b/lib/idr.c
index 15c021c..a3f8e9a 100644
--- a/lib/idr.c
+++ b/lib/idr.c
@@ -1288,6 +1288,11 @@ static inline unsigned alloc_local_tag(struct percpu_ida *pool,
  * Safe to be called from interrupt context (assuming it isn't passed
  * __GFP_WAIT, of course).
  *
+ * @gfp indicates whether or not to wait until a free id is available (it's not
+ * used for internal memory allocations); thus if passed __GFP_WAIT we may sleep
+ * however long it takes until another thread frees an id (same semantics as a
+ * mempool).
+ *
  * Will not fail if passed __GFP_WAIT.
  */
 int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp)

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
@ 2013-08-28 21:12               ` Kent Overstreet
  0 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 21:12 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, Aug 28, 2013 at 01:50:42PM -0700, Andrew Morton wrote:
> On Wed, 28 Aug 2013 13:44:54 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> 
> > > > > What guarantees that this wait will terminate?
> > > > 
> > > > It seems fairly clear to me from the break statement a couple lines up;
> > > > if we were passed __GFP_WAIT we terminate iff we succesfully allocated a
> > > > tag. If we weren't passed __GFP_WAIT we never actually sleep.
> > > 
> > > OK ;)  Let me rephrase.  What guarantees that a tag will become available?
> > > 
> > > If what we have here is an open-coded __GFP_NOFAIL then that is
> > > potentially problematic.
> > 
> > It's the same semantics as a mempool, really - it'll succeed when a tag
> > gets freed.
> 
> OK, that's reasonable if the code is being used to generate IO tags -
> we expect the in-flight tags to eventually be returned.
> 
> But if a client of this code is using the allocator for something
> totally different, there is no guarantee that the act of waiting will
> result in any tags being returned.

Yeah, and I did wonder a bit whether the waiting mechanism belonged in
the percpu ida code; arguably (certainly just looking at this code, not
any of the users) if it belongs in this code it should be common to
regular ida, not specific to percpu ida.

For now I've just decided to punt on changing that for now, since all
the percpu ida users I've come across do want the waiting mechanism, but
none of the regular ida users that I've looked at want it. There's
probably a reason for that I haven't thought of yet.

> (These are core design principles/constraints which should be
> explicitly documented in a place where future readers will see them!)

*nod* I suppose it should be said explicitly that the gfp_t parameter
indicates whether or not to wait until a _tag_ is available, and not
some internal memory allocation or something.

How's this look?

diff --git a/lib/idr.c b/lib/idr.c
index 15c021c..a3f8e9a 100644
--- a/lib/idr.c
+++ b/lib/idr.c
@@ -1288,6 +1288,11 @@ static inline unsigned alloc_local_tag(struct percpu_ida *pool,
  * Safe to be called from interrupt context (assuming it isn't passed
  * __GFP_WAIT, of course).
  *
+ * @gfp indicates whether or not to wait until a free id is available (it's not
+ * used for internal memory allocations); thus if passed __GFP_WAIT we may sleep
+ * however long it takes until another thread frees an id (same semantics as a
+ * mempool).
+ *
  * Will not fail if passed __GFP_WAIT.
  */
 int percpu_ida_alloc(struct percpu_ida *pool, gfp_t gfp)

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
  2013-08-28 21:12               ` Kent Overstreet
@ 2013-08-28 21:15                 ` Andrew Morton
  -1 siblings, 0 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 21:15 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Wed, 28 Aug 2013 14:12:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> How's this look?
> 
> diff --git a/lib/idr.c b/lib/idr.c
> index 15c021c..a3f8e9a 100644
> --- a/lib/idr.c
> +++ b/lib/idr.c
> @@ -1288,6 +1288,11 @@ static inline unsigned alloc_local_tag(struct percpu_ida *pool,
>   * Safe to be called from interrupt context (assuming it isn't passed
>   * __GFP_WAIT, of course).
>   *
> + * @gfp indicates whether or not to wait until a free id is available (it's not
> + * used for internal memory allocations); thus if passed __GFP_WAIT we may sleep
> + * however long it takes until another thread frees an id (same semantics as a
> + * mempool).

Looks good.  Mentioning the mempool thing is effective - people
understand that.

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

* Re: [PATCH-v3 1/4] idr: Percpu ida
@ 2013-08-28 21:15                 ` Andrew Morton
  0 siblings, 0 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 21:15 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, 28 Aug 2013 14:12:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> How's this look?
> 
> diff --git a/lib/idr.c b/lib/idr.c
> index 15c021c..a3f8e9a 100644
> --- a/lib/idr.c
> +++ b/lib/idr.c
> @@ -1288,6 +1288,11 @@ static inline unsigned alloc_local_tag(struct percpu_ida *pool,
>   * Safe to be called from interrupt context (assuming it isn't passed
>   * __GFP_WAIT, of course).
>   *
> + * @gfp indicates whether or not to wait until a free id is available (it's not
> + * used for internal memory allocations); thus if passed __GFP_WAIT we may sleep
> + * however long it takes until another thread frees an id (same semantics as a
> + * mempool).

Looks good.  Mentioning the mempool thing is effective - people
understand that.

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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
  2013-08-28 21:10           ` Andrew Morton
@ 2013-08-28 21:23             ` Kent Overstreet
  2013-08-28 21:26             ` Kent Overstreet
  2013-08-28 21:26             ` Kent Overstreet
  2 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 21:23 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, Aug 28, 2013 at 02:10:19PM -0700, Andrew Morton wrote:
> On Wed, 28 Aug 2013 14:00:10 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> 
> > On Wed, Aug 28, 2013 at 01:25:50PM -0700, Andrew Morton wrote:
> > > On Wed, 28 Aug 2013 12:55:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> > > 
> > > > Fixup patch, addressing Andrew's review feedback:
> > > 
> > > Looks reasonable.
> > > 
> > > >  lib/idr.c           | 38 +++++++++++++++++++++-----------------
> > > 
> > > I still don't think it should be in this file.
> > > 
> > > You say that some as-yet-unmerged patches will tie the new code into
> > > the old ida code.  But will it do it in a manner which requires that
> > > the two reside in the same file?
> > 
> > Not require, no - but it's just intimate enough with my ida rewrite that
> > I think it makes sense; it makes some use of stuff that should be
> > internal to the ida code.
> > 
> > Mostly just sharing the lock though, since I got rid of the ida
> > interfaces that don't do locking, but percpu ida needs a lock that also
> > covers what ida needs.
> > 
> > It also makes use of a ganged allocation interface, but there's no real
> > reason ida can't expose that, it's just unlikely to be useful to
> > anything but percpu ida.
> > 
> > The other reason I think it makes sense to live in idr.c is more for
> > users of the code; as you pointed out as far as the user's perspective
> > percpu ida isn't doing anything fundamentally different from ida, so I
> > think it makes sense for the code to live in the same place as a
> > kindness to future kernel developers who are trying to find their way
> > around the various library code.
> 
> I found things to be quite the opposite - it took 5 minutes of staring,
> head-scratching, double-checking and penny-dropping before I was
> confident that the newly-added code actually has nothing at all to do
> with the current code.  Putting it in the same file was misleading, and
> I got misled.

Ok... and I could see how the fact that it currently _doesn't_ have
anything to do with the existing code would be confusing...

Do you think that if/when it's making use of the ida rewrite it'll be
ok? Or would you still prefer to have it in a new file (and if so, any
preference on the naming?)

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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
  2013-08-28 21:10           ` Andrew Morton
  2013-08-28 21:23             ` Kent Overstreet
  2013-08-28 21:26             ` Kent Overstreet
@ 2013-08-28 21:26             ` Kent Overstreet
  2013-08-28 21:36                 ` Andrew Morton
  2 siblings, 1 reply; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 21:26 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Wed, Aug 28, 2013 at 02:10:19PM -0700, Andrew Morton wrote:
> On Wed, 28 Aug 2013 14:00:10 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> 
> > On Wed, Aug 28, 2013 at 01:25:50PM -0700, Andrew Morton wrote:
> > > On Wed, 28 Aug 2013 12:55:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> > > 
> > > > Fixup patch, addressing Andrew's review feedback:
> > > 
> > > Looks reasonable.
> > > 
> > > >  lib/idr.c           | 38 +++++++++++++++++++++-----------------
> > > 
> > > I still don't think it should be in this file.
> > > 
> > > You say that some as-yet-unmerged patches will tie the new code into
> > > the old ida code.  But will it do it in a manner which requires that
> > > the two reside in the same file?
> > 
> > Not require, no - but it's just intimate enough with my ida rewrite that
> > I think it makes sense; it makes some use of stuff that should be
> > internal to the ida code.
> > 
> > Mostly just sharing the lock though, since I got rid of the ida
> > interfaces that don't do locking, but percpu ida needs a lock that also
> > covers what ida needs.
> > 
> > It also makes use of a ganged allocation interface, but there's no real
> > reason ida can't expose that, it's just unlikely to be useful to
> > anything but percpu ida.
> > 
> > The other reason I think it makes sense to live in idr.c is more for
> > users of the code; as you pointed out as far as the user's perspective
> > percpu ida isn't doing anything fundamentally different from ida, so I
> > think it makes sense for the code to live in the same place as a
> > kindness to future kernel developers who are trying to find their way
> > around the various library code.
> 
> I found things to be quite the opposite - it took 5 minutes of staring,
> head-scratching, double-checking and penny-dropping before I was
> confident that the newly-added code actually has nothing at all to do
> with the current code.  Putting it in the same file was misleading, and
> I got misled.

Ok... and I could see how the fact that it currently _doesn't_ have
anything to do with the existing code would be confusing...

Do you think that if/when it's making use of the ida rewrite it'll be
ok? Or would you still prefer to have it in a new file (and if so, any
preference on the naming?)

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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
  2013-08-28 21:10           ` Andrew Morton
  2013-08-28 21:23             ` Kent Overstreet
@ 2013-08-28 21:26             ` Kent Overstreet
  2013-08-28 21:26             ` Kent Overstreet
  2 siblings, 0 replies; 45+ messages in thread
From: Kent Overstreet @ 2013-08-28 21:26 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, Aug 28, 2013 at 02:10:19PM -0700, Andrew Morton wrote:
> On Wed, 28 Aug 2013 14:00:10 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> 
> > On Wed, Aug 28, 2013 at 01:25:50PM -0700, Andrew Morton wrote:
> > > On Wed, 28 Aug 2013 12:55:17 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> > > 
> > > > Fixup patch, addressing Andrew's review feedback:
> > > 
> > > Looks reasonable.
> > > 
> > > >  lib/idr.c           | 38 +++++++++++++++++++++-----------------
> > > 
> > > I still don't think it should be in this file.
> > > 
> > > You say that some as-yet-unmerged patches will tie the new code into
> > > the old ida code.  But will it do it in a manner which requires that
> > > the two reside in the same file?
> > 
> > Not require, no - but it's just intimate enough with my ida rewrite that
> > I think it makes sense; it makes some use of stuff that should be
> > internal to the ida code.
> > 
> > Mostly just sharing the lock though, since I got rid of the ida
> > interfaces that don't do locking, but percpu ida needs a lock that also
> > covers what ida needs.
> > 
> > It also makes use of a ganged allocation interface, but there's no real
> > reason ida can't expose that, it's just unlikely to be useful to
> > anything but percpu ida.
> > 
> > The other reason I think it makes sense to live in idr.c is more for
> > users of the code; as you pointed out as far as the user's perspective
> > percpu ida isn't doing anything fundamentally different from ida, so I
> > think it makes sense for the code to live in the same place as a
> > kindness to future kernel developers who are trying to find their way
> > around the various library code.
> 
> I found things to be quite the opposite - it took 5 minutes of staring,
> head-scratching, double-checking and penny-dropping before I was
> confident that the newly-added code actually has nothing at all to do
> with the current code.  Putting it in the same file was misleading, and
> I got misled.

Ok... and I could see how the fact that it currently _doesn't_ have
anything to do with the existing code would be confusing...

Do you think that if/when it's making use of the ida rewrite it'll be
ok? Or would you still prefer to have it in a new file (and if so, any
preference on the naming?)

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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
  2013-08-28 21:26             ` Kent Overstreet
@ 2013-08-28 21:36                 ` Andrew Morton
  0 siblings, 0 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 21:36 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Nicholas A. Bellinger, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Wed, 28 Aug 2013 14:23:58 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> > I found things to be quite the opposite - it took 5 minutes of staring,
> > head-scratching, double-checking and penny-dropping before I was
> > confident that the newly-added code actually has nothing at all to do
> > with the current code.  Putting it in the same file was misleading, and
> > I got misled.
> 
> Ok... and I could see how the fact that it currently _doesn't_ have
> anything to do with the existing code would be confusing...
> 
> Do you think that if/when it's making use of the ida rewrite it'll be
> ok? Or would you still prefer to have it in a new file

I'm constitutionally reluctant to ever assume that any out-of-tree code
will be merged.  Maybe you'll get hit by a bus, and maybe the code
sucks ;)

Are you sure that the two things are so tangled together that they must
live in the same file?  If there's some nice layering between ida and
percpu_ida then perhaps such a physical separation would remain
appropriate?

> (and if so, any preference on the naming?)

percpu_ida.c?

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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
@ 2013-08-28 21:36                 ` Andrew Morton
  0 siblings, 0 replies; 45+ messages in thread
From: Andrew Morton @ 2013-08-28 21:36 UTC (permalink / raw)
  To: Kent Overstreet
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, target-devel, Christoph Lameter, Oleg Nesterov,
	Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, 28 Aug 2013 14:23:58 -0700 Kent Overstreet <kmo@daterainc.com> wrote:

> > I found things to be quite the opposite - it took 5 minutes of staring,
> > head-scratching, double-checking and penny-dropping before I was
> > confident that the newly-added code actually has nothing at all to do
> > with the current code.  Putting it in the same file was misleading, and
> > I got misled.
> 
> Ok... and I could see how the fact that it currently _doesn't_ have
> anything to do with the existing code would be confusing...
> 
> Do you think that if/when it's making use of the ida rewrite it'll be
> ok? Or would you still prefer to have it in a new file

I'm constitutionally reluctant to ever assume that any out-of-tree code
will be merged.  Maybe you'll get hit by a bus, and maybe the code
sucks ;)

Are you sure that the two things are so tangled together that they must
live in the same file?  If there's some nice layering between ida and
percpu_ida then perhaps such a physical separation would remain
appropriate?

> (and if so, any preference on the naming?)

percpu_ida.c?

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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
  2013-08-28 21:36                 ` Andrew Morton
@ 2013-08-31  3:10                   ` Nicholas A. Bellinger
  -1 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-31  3:10 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Kent Overstreet, target-devel, lf-virt, lkml, kvm-devel,
	Michael S. Tsirkin, Asias He, Jens Axboe, Tejun Heo, Ingo Molnar,
	Andi Kleen, Christoph Lameter, Oleg Nesterov, Christoph Lameter

On Wed, 2013-08-28 at 14:36 -0700, Andrew Morton wrote:
> On Wed, 28 Aug 2013 14:23:58 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> 
> > > I found things to be quite the opposite - it took 5 minutes of staring,
> > > head-scratching, double-checking and penny-dropping before I was
> > > confident that the newly-added code actually has nothing at all to do
> > > with the current code.  Putting it in the same file was misleading, and
> > > I got misled.
> > 
> > Ok... and I could see how the fact that it currently _doesn't_ have
> > anything to do with the existing code would be confusing...
> > 
> > Do you think that if/when it's making use of the ida rewrite it'll be
> > ok? Or would you still prefer to have it in a new file
> 
> I'm constitutionally reluctant to ever assume that any out-of-tree code
> will be merged.  Maybe you'll get hit by a bus, and maybe the code
> sucks ;)
> 
> Are you sure that the two things are so tangled together that they must
> live in the same file?  If there's some nice layering between ida and
> percpu_ida then perhaps such a physical separation would remain
> appropriate?
> 
> > (and if so, any preference on the naming?)
> 
> percpu_ida.c?

Hi Andrew,

I've folded Kent's two patches from this thread into the -v4 commit, and
moved the logic from idr.[c,h] to percpu_ida.[c,h] as per your above
recommendation.

The cpumask_t changes are working as expected thus far, and will be
going out a -v5 series for you to review -> signoff shortly.

Thank you,

--nab


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

* Re: [PATCH] percpu ida: Switch to cpumask_t, add some comments
@ 2013-08-31  3:10                   ` Nicholas A. Bellinger
  0 siblings, 0 replies; 45+ messages in thread
From: Nicholas A. Bellinger @ 2013-08-31  3:10 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jens Axboe, Andi Kleen, kvm-devel, Michael S. Tsirkin, lkml,
	lf-virt, Kent Overstreet, target-devel, Christoph Lameter,
	Oleg Nesterov, Tejun Heo, Christoph Lameter, Ingo Molnar

On Wed, 2013-08-28 at 14:36 -0700, Andrew Morton wrote:
> On Wed, 28 Aug 2013 14:23:58 -0700 Kent Overstreet <kmo@daterainc.com> wrote:
> 
> > > I found things to be quite the opposite - it took 5 minutes of staring,
> > > head-scratching, double-checking and penny-dropping before I was
> > > confident that the newly-added code actually has nothing at all to do
> > > with the current code.  Putting it in the same file was misleading, and
> > > I got misled.
> > 
> > Ok... and I could see how the fact that it currently _doesn't_ have
> > anything to do with the existing code would be confusing...
> > 
> > Do you think that if/when it's making use of the ida rewrite it'll be
> > ok? Or would you still prefer to have it in a new file
> 
> I'm constitutionally reluctant to ever assume that any out-of-tree code
> will be merged.  Maybe you'll get hit by a bus, and maybe the code
> sucks ;)
> 
> Are you sure that the two things are so tangled together that they must
> live in the same file?  If there's some nice layering between ida and
> percpu_ida then perhaps such a physical separation would remain
> appropriate?
> 
> > (and if so, any preference on the naming?)
> 
> percpu_ida.c?

Hi Andrew,

I've folded Kent's two patches from this thread into the -v4 commit, and
moved the logic from idr.[c,h] to percpu_ida.[c,h] as per your above
recommendation.

The cpumask_t changes are working as expected thus far, and will be
going out a -v5 series for you to review -> signoff shortly.

Thank you,

--nab

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

end of thread, other threads:[~2013-08-31  3:10 UTC | newest]

Thread overview: 45+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2013-08-16 23:09 [PATCH-v3 0/4] target/vhost-scsi: Add per-cpu ida tag pre-allocation for v3.12 Nicholas A. Bellinger
2013-08-16 23:09 ` Nicholas A. Bellinger
2013-08-16 23:09 ` [PATCH-v3 1/4] idr: Percpu ida Nicholas A. Bellinger
2013-08-16 23:09   ` Nicholas A. Bellinger
2013-08-20 21:31   ` Andrew Morton
2013-08-20 21:31   ` Andrew Morton
2013-08-26 20:14     ` Kent Overstreet
2013-08-26 20:14     ` Kent Overstreet
2013-08-28 19:53     ` Kent Overstreet
2013-08-28 19:53       ` Kent Overstreet
2013-08-28 20:23       ` Andrew Morton
2013-08-28 20:23       ` Andrew Morton
2013-08-28 20:44         ` Kent Overstreet
2013-08-28 20:50           ` Andrew Morton
2013-08-28 20:50             ` Andrew Morton
2013-08-28 21:12             ` Kent Overstreet
2013-08-28 21:12               ` Kent Overstreet
2013-08-28 21:15               ` Andrew Morton
2013-08-28 21:15                 ` Andrew Morton
2013-08-28 20:44         ` Kent Overstreet
2013-08-28 19:55     ` [PATCH] percpu ida: Switch to cpumask_t, add some comments Kent Overstreet
2013-08-28 19:55       ` Kent Overstreet
2013-08-28 20:25       ` Andrew Morton
2013-08-28 20:25         ` Andrew Morton
2013-08-28 21:00         ` Kent Overstreet
2013-08-28 21:00           ` Kent Overstreet
2013-08-28 21:10           ` Andrew Morton
2013-08-28 21:23             ` Kent Overstreet
2013-08-28 21:26             ` Kent Overstreet
2013-08-28 21:26             ` Kent Overstreet
2013-08-28 21:36               ` Andrew Morton
2013-08-28 21:36                 ` Andrew Morton
2013-08-31  3:10                 ` Nicholas A. Bellinger
2013-08-31  3:10                   ` Nicholas A. Bellinger
2013-08-28 21:10           ` Andrew Morton
2013-08-21 18:25   ` [PATCH-v3 1/4] idr: Percpu ida Christoph Lameter
2013-08-21 18:25     ` Christoph Lameter
2013-08-26 20:23     ` Kent Overstreet
2013-08-26 20:23     ` Kent Overstreet
2013-08-16 23:09 ` [PATCH-v3 2/4] target: Add transport_init_session_tags using per-cpu ida Nicholas A. Bellinger
2013-08-16 23:09   ` Nicholas A. Bellinger
2013-08-16 23:09 ` [PATCH-v3 3/4] vhost/scsi: Convert to per-cpu ida_alloc + ida_free command map Nicholas A. Bellinger
2013-08-16 23:09   ` Nicholas A. Bellinger
2013-08-16 23:09 ` [PATCH-v3 4/4] vhost/scsi: Add pre-allocation for tv_cmd SGL + upages memory Nicholas A. Bellinger
2013-08-16 23:09   ` Nicholas A. Bellinger

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.