Joe,

here it goes (and it is at homer:/tmp/dm-multipath.c as well).

Heinz

On Tue, Jul 22, 2003 at 09:59:06PM +0100, Joe Thornber wrote:
> Heinz,
> 
> Could you mail me the latest multipath please ?
> 
> - Joe

/*
 * Copyright (C) 2003 Sistina Software Limited.
 *
 * This file is released under the GPL.
 */

/* Debugging macros
#define	DEBUG_SELECT
*/
#define	DEBUG_REQUEUE
#define	DEBUG_FAILURE_IO

#include "dm.h"

#include <linux/ctype.h>
#include <linux/init.h>
#include <linux/mempool.h>
#include <linux/module.h>
#include <linux/pagemap.h>
#include <linux/slab.h>
#include <linux/time.h>
#include <asm/atomic.h>

#define ARG_FORMAT	"%d"

/* range checks for target definition in _get_path() */
#define PRIORITY_MIN	0	/* minimum (lowest priority) */
#define PRIORITY_MAX	8192	/* maximum value (highest priority) */
#define FAILURE_MIN	0	/* min/max io failures... */
#define FAILURE_MAX	1024	/* ...before path is failed */
#define	IO_COUNT_MIN	0	/* minimum ios before load-switch */
#define	IO_COUNT_MAX	8192	/* maximum         " (unless throughput set) */
#define TIMEOUT_MIN	0	/* min timeout before path is retried[s] */
#define TIMEOUT_MAX	(365*24*60*60)	/* reasonable maximum[s] ? ;)) */

#define	ANY_PATH	-1	/* selects any path */
#define	ANY_PRIO	-1	/* selects any high priority path */

/* Path flags */
enum {
	SELECTABLE = 1,
	FAILED,
	LOAD_BALANCED,
	THROUGHPUT,
	TEST_IO,
};

/* Path properties */
struct path {
	struct dm_dev	*dev;

	unsigned long		flags;
	unsigned long long	latency;

	typeof(jiffies)		fail_jiffies;
	atomic_t		io_count;
	atomic_t		fail;
	int			fail_total;

	/* set in target ctr */
	int	priority;
	int	ios_min;
	int	fail_max;
	int	fail_timeout;

	/* failure io retry */
	struct dm_target 	*ti;
        struct buffer_head	*bh;
};

/*
 * Multipath context
 */
struct multipath_c {
	struct list_head list;

	spinlock_t	lock;
	int		default_path;
	int		paths_failed;
	int		paths;
	struct path	path[0]; /* dynamic allocation */
};

/* Multipath io job */
struct multipath_io {
	struct list_head	list;

	struct dm_target	*ti;
	struct buffer_head	*bh;
	int			rw;
	int			path;
	typeof(jiffies)		start_jiffies;
};

/* Slab for the io jobs */
static kmem_cache_t *_multipath_cache;
mempool_t *_multipath_pool;

static int ios = 0;
#define	DEFAULT_IOS	256
#define	MIN_IOS		16
#define	MAX_IOS		32768	/* maximum on 32 bit hw with mempool_create */


static int paths_failed = 0;
static spinlock_t _job_lock = SPIN_LOCK_UNLOCKED;
static LIST_HEAD(_timeout_jobs);
static LIST_HEAD(_io_jobs);

/*
 * Various inlines to set a single/all path(s) (in)operational,
 * check if path(s) is/are operational and (un)fail a path, allocate
 * and deallocate io job memory...
 */
static inline int _is_valid(struct multipath_c *mc, int path)
{
	return path > ANY_PATH && path < mc->paths;
}

/* Path selectable */
static inline int _is_selectable(struct multipath_c *mc, int path)
{
	return _is_valid(mc, path) ?
	       test_bit(SELECTABLE, &mc->path[path].flags) : 0;
}

static inline void _set_selectable(struct multipath_c *mc, int path)
{
	set_bit(SELECTABLE, &mc->path[path].flags);
}

static inline void _reset_selectable(struct multipath_c *mc, int path)
{
	clear_bit(SELECTABLE, &mc->path[path].flags);
}

/* Path failure */
static inline void _set_fail_time(struct multipath_c *mc, int path)
{
	mc->path[path].fail_jiffies = jiffies;
}

static inline int _is_failed(struct multipath_c *mc, int path)
{
	return test_bit(FAILED, &mc->path[path].flags);
}

static inline void _set_failed(struct multipath_io *io)
{
	int path = io->path;
	struct multipath_c *mc = (struct multipath_c *) io->ti->private;

	set_bit(FAILED, &mc->path[path].flags);
	_reset_selectable(mc, path);
	paths_failed++;
	mc->paths_failed++;
	mc->path[path].fail_total++;
	_set_fail_time(mc, path);
	dm_table_event(io->ti->table);
}

static inline void _reset_failures(struct multipath_c *mc, int path)
{
	atomic_set(&mc->path[path].fail, mc->path[path].fail_max);
}

static inline void _reset_failed(struct multipath_io *io)
{
	int path = io->path;
	struct multipath_c *mc = (struct multipath_c *) io->ti->private;

	if (!_is_failed(mc, path))
		return;

	clear_bit(FAILED, &mc->path[path].flags);
	_set_selectable(mc, path);
	mc->paths_failed--;
	paths_failed--;
	mc->path[path].fail_jiffies = 0;
	_reset_failures(mc, path);
	path = mc->paths;
	while (path--)
		mc->path[path].latency = 0;
	
	dm_table_event(io->ti->table);
}

static inline int _is_load_balanced(struct multipath_c *mc, int path)
{
	return test_bit(LOAD_BALANCED, &mc->path[path].flags);
}

static inline int _is_maximized(struct multipath_c *mc, int path)
{
	return test_bit(THROUGHPUT, &mc->path[path].flags);
}

static inline void _reset_test_io(struct multipath_c *mc, int path)
{
	clear_bit(TEST_IO, &mc->path[path].flags);
}

/* Automatic failback timeout calculation */
static inline int _get_fail_timeout(struct multipath_c *mc, int path)
{
	return mc->path[path].fail_timeout * HZ;
}

static inline void _reset_min_ios(struct multipath_c *mc, int path)
{
	atomic_set(&mc->path[path].io_count, mc->path[path].ios_min);
}

static inline int _timeout(struct multipath_c *mc, int path)
{
	return jiffies < mc->path[path].fail_jiffies ?
	       _get_fail_timeout(mc, path) :
	       (int) (jiffies - mc->path[path].fail_jiffies);
}

static inline int _timeout_expired(struct multipath_c *mc, int path)
{
	return _get_fail_timeout(mc, path) ?
	       (_timeout(mc, path) >= _get_fail_timeout(mc, path)) : 0;
}

static inline long _min_timeout(struct multipath_c *mc, int path, long timeout)
{
	long t = _get_fail_timeout(mc, path) - _timeout(mc, path);

	return t > 0 && t < timeout ? t : timeout;
}

/* Io job allocation/deallocation */
static inline struct multipath_io *alloc_io(void)
{
	return mempool_alloc(_multipath_pool, GFP_NOIO);
}

static inline void free_io(struct multipath_io *io)
{
	mempool_free(io, _multipath_pool);
}

/* Multipath context allocation (dynamic path array size) */
static inline struct multipath_c *_alloc_context(int paths)
{
	struct multipath_c *mc = NULL;

        if (!array_too_big(sizeof(struct multipath_c),
			   sizeof(struct path),
                           paths)) {
        	size_t size = sizeof(struct multipath_c) +
			      sizeof(struct path) * paths;

        	mc = kmalloc(size, GFP_KERNEL);
		if (mc) {
			memset(mc, 0, size);
			mc->lock = SPIN_LOCK_UNLOCKED;
		}
	}

	return mc;
}


/*
 * Daemon
 */

/* Push a job onto the tail of a daemon job queue */
static inline void push(struct list_head *joblist, struct list_head *job)
{
	int flags;

	spin_lock_irqsave(&_job_lock, flags);
	list_add_tail(job, joblist);
	spin_unlock_irqrestore(&_job_lock, flags);
}

/* Pop a job from the io queue */
static inline struct multipath_io *pop(void)
{
	int flags;
	struct multipath_io *io = NULL;
	struct list_head *jobs = &_io_jobs;

	spin_lock_irqsave(&_job_lock, flags);
	if (!list_empty(jobs)) {
		io = list_entry(jobs->next, struct multipath_io, list);
		list_del(jobs->next);
	}
	spin_unlock_irqrestore(&_job_lock, flags);

	return io;
}

/* Requeue error ios */
static inline void _do_ios(void)
{
	struct multipath_io *io;

	while((io = pop()))
		generic_make_request(io->rw, io->bh);

	run_task_queue(&tq_disk);
}

/* Test io end_io function */
static void multipath_test_end_io(struct buffer_head *bh, int uptodate)
{
	struct multipath_io *io = (struct multipath_io *) bh->b_private;
	struct multipath_c *mc = (struct multipath_c *) io->ti->private;

#ifdef	DEBUG_FAILURE_IO
	DMINFO("dm-multipath: end of test io on device %s/sector %lu %s at %lu",
	       kdevname(to_kdev_t(mc->path[io->path].dev->dev)),
	       bh->b_rsector,
	       uptodate ? "_GOOD_" : "BAD", jiffies);
#endif
	if (uptodate) {
		int flags;

		spin_lock_irqsave(&mc->lock, flags);
		_reset_failed(io);
		mc->path[io->path].ti = NULL;
		spin_unlock_irqrestore(&mc->lock, flags);
	}

	UnlockPage(io->bh->b_page);
	__free_page(io->bh->b_page);
	kfree(io->bh);
	free_io(io);
	_reset_test_io(mc ,io->path);
}

/*
 * Queue a test read io to a failed path
 *
 * Returns
 *
 *	0: test io already in progress
 *	1: test io queued
 *	
 */
static inline int _queue_test_io(struct multipath_c *mc, int path)
{
	struct multipath_io *io;
	struct buffer_head *bh;
	struct path *p = &mc->path[path];

	if (test_and_set_bit(TEST_IO, &p->flags))
		goto out;

	bh = kmalloc(sizeof(*bh), GFP_NOIO);
	if (!bh)
		goto bad1;	/* we'll just retry a bit later */

	/* setup bh */
        memset(bh, 0, sizeof(*bh));

	/* Well, might be (a little) too large but it's easy */
 	bh->b_page = alloc_page(GFP_NOIO);
	if (!bh->b_page)
		goto bad2;

	LockPage(bh->b_page);

	/* Setup io */
	io = alloc_io();
	io->path = path;
	io->ti = p->ti;
	io->rw = READ;
	io->bh = bh;

	/* no need to set b_dev, b_blocknr, b_count
	   or initialize the wait queue here */
	bh->b_rdev = p->dev->dev;
        bh->b_rsector = p->bh->b_rsector;
        bh->b_size = p->bh->b_size;
	bh->b_data = page_address(bh->b_page);
	set_bit(BH_Lock, &bh->b_state);
	bh->b_end_io = multipath_test_end_io;
	bh->b_private = io;

#ifdef	DEBUG_FAILURE_IO
	DMINFO("dm-multipath: queueing failure test io to %s/sector %lu at %lu",
		kdevname(to_kdev_t(bh->b_rdev)), bh->b_rsector, jiffies);
#endif

	generic_make_request(io->rw, bh);
	run_task_queue(&tq_disk);

	return 1;

   bad2:
	kfree(bh);

   bad1:
	_reset_test_io(mc ,path);
	_set_fail_time(mc, path);

   out:
	return 0;
}

/* Check if paths need to be set to operational on failure timeout */
static inline long _do_timeouts(void)
{
	int flags;
	long timeout = MAX_SCHEDULE_TIMEOUT;
	struct multipath_c *mc;

	if (!paths_failed)
		goto out;

	spin_lock_irqsave(&_job_lock, flags);
	list_for_each_entry(mc, &_timeout_jobs, list) {
		int p;

		if (!mc->paths_failed)
			continue;

		p = mc->paths;
		while (p--) {
			if (!_is_failed(mc, p))
				continue;

			if (_timeout_expired(mc, p)) {
				_queue_test_io(mc, p);
				_set_fail_time(mc, p);
			}

			timeout = _min_timeout(mc, p, timeout);
		}

	}
	spin_unlock_irqrestore(&_job_lock, flags);

   out:
	return timeout;
}

/* Multipathd does this every time it runs */
static inline long _do_work(void)
{
	_do_ios();
	return _do_timeouts();
}

static DECLARE_MUTEX(_start_lock);
static DECLARE_MUTEX(_run_lock);
static DECLARE_WAIT_QUEUE_HEAD(_multipathd_queue);
static unsigned long _multipathd_flags = 0;
#define	RUN	1

/* The multipath daemon core */
static int multipathd(void *arg)
{
	DECLARE_WAITQUEUE(wq, current);

	set_current_state(TASK_RUNNING);
	daemonize();
	strcpy(current->comm, "multipathd");

	add_wait_queue(&_multipathd_queue, &wq);

	down(&_run_lock);
	up(&_start_lock);

	while (test_bit(RUN, &_multipathd_flags)) {
		long timeout = _do_work();

		if (timeout) {
			set_current_state(TASK_INTERRUPTIBLE);
			schedule_timeout(timeout);
			set_current_state(TASK_RUNNING);
		}
	}
	remove_wait_queue(&_multipathd_queue, &wq);

	up(&_run_lock);

	return 0;
}

static inline void wake_multipathd(void)
{
	wake_up_interruptible(&_multipathd_queue);
}

static inline int start_daemon(void)
{
	static pid_t pid = 0;

	down(&_start_lock);

	set_bit(RUN, &_multipathd_flags);
	pid = kernel_thread(multipathd, NULL, 0);
	if (pid <= 0) {
		DMERR("Failed to start multipathd thread");
		return -EAGAIN;
	}

	/* Wait for the daemon to up this mutex */
	down(&_start_lock);
	up(&_start_lock);

	return 0;
}

static inline void stop_daemon(void)
{
	clear_bit(RUN, &_multipathd_flags);
	wake_multipathd();

	/* Wait for thread exit (Don't need to up mutex. We exit anyway) */
	down(&_run_lock);
}
/* End daemon code */


/*
 * Select the next path of equal or highest priority
 *
 * @path is the actual path which can be returned
 * in case no other operational path exists
 *
 */
static int _select_path(struct multipath_c *mc, int path)
{
	int i, end = mc->paths;
	int p = ANY_PATH;
	int priority = ANY_PRIO;
	int highest_priority = ANY_PRIO;

	if (_is_valid(mc, path))
		priority = mc->path[path].priority;

	for (i = path + 1; i <= end; i++) { /* Try circular */
		if (i == mc->paths) {
			i = ANY_PATH;
			end = path;
			continue;
		}

		if (!_is_selectable(mc, i))
			continue;

		if (mc->path[i].priority == priority) {
			p = i;
			goto out;
		}

		if (mc->path[i].priority > highest_priority) {
			p = i;
			highest_priority = mc->path[i].priority;
		}
	}

   out:
#ifdef	DEBUG_SELECT
	DMINFO("dm-multipath: path %d/device %s selected",
	       p, kdevname(to_kdev_t(mc->path[p].dev->dev)));
#endif
	return p;
}

/*
 * Parse a single
 *
 * 	<path>
 *	<priority> <load-balancing> <min-ios>
 *	<failure-maximum> <reactivation-timeout>
 *
 * parameter set
 *
 */
/* Load-balancing flag characters */
#define	NO_CHAR		'N'
#define	LOAD_CHAR	'L'
#define	MAXIMIZE_CHAR	'M'

static inline int _get_lb_flags(char *arg, struct multipath_c *mc, int path)
{
	if (arg[1])
		return 0;

	switch (toupper(*arg)) {
	case NO_CHAR: 
		break;

	/* Maximize throughput needs to set load-balancing flag too */
	case MAXIMIZE_CHAR:
		set_bit(THROUGHPUT, &mc->path[path].flags);

	case LOAD_CHAR:
		set_bit(LOAD_BALANCED, &mc->path[path].flags);
		break;

	default:
		return 0;
	}

	return 1;
}

#define	xx(a, s, c, v) \
        if (sscanf(argv[a], ARG_FORMAT, &tmp) != 1 || \
	    tmp < c ## _MIN || \
	    tmp > c ## _MAX) { \
		ti->error = "dm-multipath: Invalid " s; \
                return -EINVAL; \
	} \
        mc->path[path]. v = tmp;
	
#define	PATH_ARGS	6 /* _get_path() must check for this amount */
static inline int _get_path(struct multipath_c *mc, struct dm_target *ti,
			    int path, char **argv)
{
	int tmp;

	xx(1, "priority", PRIORITY, priority)
	if (!_get_lb_flags(argv[2], mc, path)) {
		ti->error = "load-balance flags";
		return -EINVAL;
	}

	xx(3, "ios min", IO_COUNT, ios_min)
	xx(4, "failure-maximum", FAILURE, fail_max)
	xx(5, "reactivation-timeout", TIMEOUT, fail_timeout)

	ti->error = "dm-multipath: Device lookup failure";
        if (dm_get_device(ti, argv[0], ti->begin, ti->len,
                          dm_table_get_mode(ti->table),
                          &mc->path[path].dev))
                return -ENXIO;

	ti->error = NULL;

	if (!mc->path[path].ios_min)
		mc->path[path].ios_min = 1;

	if (!mc->path[path].fail_max)
		mc->path[path].fail_max = 1;

	_reset_min_ios(mc, path);
	_reset_failures(mc, path);
        _set_selectable(mc, path);

        return 0;
}
#undef xx

/* Construct a multipath mapping: */
static int multipath_ctr(struct dm_target *ti, unsigned int argc, char **argv)
{
	int i, paths = argc / PATH_ARGS;
	struct multipath_c *mc;

	if (argc % PATH_ARGS) {
		ti->error = "dm-multipath: Wrong argument count";
		return -EINVAL;
	}

	mc = _alloc_context(paths);
	if (!mc) {
		ti->error = "dm-multipath: Cannot allocate multipath context";
		return -ENOMEM;
	}

        /* Get the paths parameter sets */
        for (i = 0; i < paths; i++) {
                int r = _get_path(mc, ti, i, argv);
		int p = i;

		if (!r) { /* Any double devices ? */
			while (p--) {
				if (mc->path[p].dev->dev ==
				    mc->path[i].dev->dev) {
					ti->error = "dm-multipath: Same device";
					i++;
					r = -EINVAL;
					break;
				}
			}
		}

                if (r < 0) {
                        while (i--)
                                dm_put_device(ti, mc->path[i].dev);

                        kfree(mc);
                        return r;
                }
		argv += PATH_ARGS;
	}

	ti->private = mc;
	mc->paths = paths;
	mc->default_path = _select_path(mc, ANY_PATH);
	push(&_timeout_jobs, &mc->list);

	return 0;
}
#undef	PATH_ARGS

static inline void _wait_for_test_ios(struct multipath_c *mc)
{
	int p = mc->paths;

	while (p--) {
		while(test_bit(TEST_IO, &mc->path[p].flags))
			schedule_timeout(HZ / 4);
	}
}

static inline void _remove_timeout_job(struct multipath_c *mc)
{
	int flags;
	struct multipath_c *mc_tmp;

	spin_lock_irqsave(&_job_lock, flags);
	list_for_each_entry(mc_tmp, &_timeout_jobs, list) {
		if (mc == mc_tmp) {
			list_del(&mc->list);
			break;
		}
	}
	spin_unlock_irqrestore(&_job_lock, flags);
}

/* Destruct a multipath mapping */
static void multipath_dtr(struct dm_target *ti)
{
	struct multipath_c *mc = (struct multipath_c *) ti->private;

	_wait_for_test_ios(mc);
	_remove_timeout_job(mc);
	while (mc->paths--)
		dm_put_device(ti, mc->path[mc->paths].dev);

	kfree(mc);
}

static inline void _calc_latency(struct multipath_c *mc,
				 struct multipath_io *io)
{
	if (_is_maximized(mc, io->path)) {
		int flags;
		int t = jiffies - io->start_jiffies;
	
		t *= t; /* Put heavy weight on long latencies */
		t *= t;

		spin_lock_irqsave(&mc->lock, flags);
		mc->path[io->path].latency += t;
		spin_unlock_irqrestore(&mc->lock, flags);
	}
}

static inline void prepare_io(struct multipath_c *mc, struct multipath_io *io)
{
	io->bh->b_rdev = mc->path[io->path].dev->dev;
	if (_is_maximized(mc, io->path))
		io->start_jiffies = jiffies;
}

static inline int _fail_path(struct multipath_io *io)
{
	int path = io->path;
	struct multipath_c *mc = (struct multipath_c *) io->ti->private;

	if (_is_failed(mc, path))
		return 1;

	if (!atomic_dec_and_test(&mc->path[path].fail))
		return 0;

	_set_failed(io);
	wake_multipathd(); /* For new schedule timeout calculation */

	return 1;
}

static inline void _save_failed_io_context(struct multipath_c *mc,
					   struct multipath_io *io)
{
	if (mc->path[io->path].ti)
		return;

	mc->path[io->path].ti = io->ti;
	mc->path[io->path].bh = io->bh;
}

static inline int _check_fail_path(struct multipath_c *mc,
				   struct multipath_io *io)
{
#ifdef	DEBUG_REQUEUE
	int former_path = io->path;
#endif

	if (_fail_path(io)) {
		/* Only select, if path fails after retries */
		int path = _select_path(mc, io->path);

		if (!_is_selectable(mc, path))
			return -1;

		_save_failed_io_context(mc, io);
		io->path = path;
	}

	prepare_io(mc, io);
	push(&_io_jobs, &io->list);
	wake_multipathd();
	
#ifdef	DEBUG_REQUEUE
#define	LEN	16
	{
		char buffer[LEN];

		snprintf(buffer, LEN, "%s",
			 kdevname(to_kdev_t(mc->path[former_path].dev->dev)));
		DMINFO("dm-multipath: requeueing sector %lu/device %s "
		       "to device %s",
		       io->bh->b_rsector, buffer,
		       kdevname(to_kdev_t(mc->path[io->path].dev->dev)));
	}
#endif

	return 0;
}

static int multipath_end_io(struct dm_target *ti, struct buffer_head *bh,
			    int rw, int error, union map_info *map_context)
{
	int r = 0;
	struct multipath_io *io = (struct multipath_io *) map_context->ptr;
	struct multipath_c *mc = (struct multipath_c *) io->ti->private;

	_calc_latency(mc, io);

	if (error) {
		r = _check_fail_path(mc, io);
		if (!r)
			return 1; /* Handle later */
	}

	free_io(io);
	return r;
}

static inline int _path(struct multipath_c *mc)
{
	int p, path = mc->default_path;

	if (!_is_selectable(mc, path)) {
		path = _select_path(mc, path);
		goto out;
	}

 	if (!_is_load_balanced(mc, path) ||
	    !atomic_dec_and_test(&mc->path[path].io_count))
		return 1;

	spin_lock_irq(&mc->lock);
	p = path = mc->default_path;
	_reset_min_ios(mc, path);

	/* Check circular, if there's a path with less latency */
	while ((p = _select_path(mc, p)) != path) {
		if (mc->path[p].priority != mc->path[path].priority)
			continue;

		if (!_is_maximized(mc, path) ||
		    mc->path[p].latency <= mc->path[path].latency)
			break;
	}

	path = mc->default_path = p;
	spin_unlock_irq(&mc->lock);

   out:
	return _is_selectable(mc, path);
}

/* Multipath mapping */
static int multipath_map(struct dm_target *ti, struct buffer_head *bh,
			 int rw, union map_info *map_context)
{
	struct multipath_c *mc = (struct multipath_c *) ti->private;
	struct multipath_io *io = alloc_io();

	io->ti = ti;
	io->rw = rw;
	io->bh = bh;

	/* Prevent end_io+daemon from changing multipath or io context */
	if (!_path(mc))
		goto err;

	io->path = mc->default_path;
	map_context->ptr = (void *) io;
	prepare_io(mc, io);

	return 1; /* Normal map */

   err:
	free_io(io);

	return -1;
}

static inline char _get_lb_char(struct multipath_c *mc, int path)
{
	return _is_load_balanced(mc, path) ?
	       (_is_maximized(mc, path) ? MAXIMIZE_CHAR : LOAD_CHAR) : NO_CHAR;
}

/* Multipath status */
static int multipath_status(struct dm_target *ti, status_type_t type,
			      char *result, unsigned int maxlen)
{
	int sz = 0;
	long i;
	struct multipath_c *mc = (struct multipath_c *) ti->private;

	switch (type) {
	case STATUSTYPE_INFO:
		for (i = 0; i < mc->paths; i++) {
			sz += snprintf(result + sz, maxlen - sz, "%s ",
				       kdevname(to_kdev_t(mc->path[i].dev->dev)));

			if (_is_failed(mc, i)) {
				struct timespec fail;

				jiffies_to_timespec(_timeout(mc, i), &fail);
				sz += snprintf(result + sz, maxlen -sz,
					       "I(%lu/" ARG_FORMAT ")",
					       (unsigned long) fail.tv_sec,
					       _get_fail_timeout(mc, i) / HZ);
			} else
				sz += snprintf(result + sz, maxlen -sz, "O");

			if (mc->path[i].fail_total)
				sz += snprintf(result + sz, maxlen -sz, "[%d]",
					       mc->path[i].fail_total);

			sz += snprintf(result + sz, maxlen -sz, " ");
		}
		break;

	case STATUSTYPE_TABLE:
		for (i = 0; i < mc->paths; i++) {
			sz += snprintf(
			   result + sz, maxlen - sz,
			   "%s " ARG_FORMAT " %c " ARG_FORMAT " "
			   ARG_FORMAT " " ARG_FORMAT " ",
			   kdevname(to_kdev_t(mc->path[i].dev->dev)),
			   mc->path[i].priority,
			   _get_lb_char(mc, i),
			   mc->path[i].ios_min,
			   mc->path[i].fail_max,
			   mc->path[i].fail_timeout
			);
		}

		break;
	}

	return 0;
}

static struct target_type multipath_target = {
	.name   = "multipath",
	.module = THIS_MODULE,
	.ctr    = multipath_ctr,
	.dtr    = multipath_dtr,
	.map    = multipath_map,
	.end_io	= multipath_end_io,
	.status = multipath_status,
};

int __init dm_multipath_init(void)
{
	int r = -EINVAL;

	if (!ios)
		ios = DEFAULT_IOS;
	else if (ios < MIN_IOS ||
		 ios > MAX_IOS)
		goto bad;

	r = -ENOMEM;

	/* Allocate a slab for the multipath ios */
	_multipath_cache = kmem_cache_create("dm multipath io",
					     sizeof(struct multipath_io),
					     0, 0, NULL, NULL);
	if (!_multipath_cache)
		goto bad;

	/* Create multipath io mempool */
	_multipath_pool = mempool_create(ios, mempool_alloc_slab,
					 mempool_free_slab, _multipath_cache);
	if (!_multipath_pool)
		goto bad1;

	r = dm_register_target(&multipath_target);
	if (r < 0)  {
		DMERR("%s: register failed %d", multipath_target.name, r); 
		goto bad2;
	}

	r = start_daemon();
	if (!r) {
		DMINFO("dm_multipath v0.1.5 (%d io contexts preallocated)",
		       ios);
		return 0;
	}

	dm_unregister_target(&multipath_target);

   bad2:
	mempool_destroy(_multipath_pool);

   bad1:
	kmem_cache_destroy(_multipath_cache);

   bad:
	return r;
}

void __exit dm_multipath_exit(void)
{
	int r;

	stop_daemon();

	r = dm_unregister_target(&multipath_target);
	if (r < 0)
		DMERR("%s: unregister failed %d", multipath_target.name, r);

	mempool_destroy(_multipath_pool);
	kmem_cache_destroy(_multipath_cache);
}

/* Module hooks */
module_init(dm_multipath_init);
module_exit(dm_multipath_exit);

MODULE_DESCRIPTION(DM_NAME " multipath target");
MODULE_AUTHOR("Heinz Mauelshagen <mge@sistina.com>");
MODULE_LICENSE("GPL");
MODULE_PARM(ios, "i");
MODULE_PARM_DESC(ios, "number of preallocated io contexts");